You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cp...@apache.org on 2017/04/19 12:38:34 UTC

[1/3] lucene-solr:master: SOLR-10394: a few more essentially non-public sortWithinGroup to withinGroupSort renames

Repository: lucene-solr
Updated Branches:
  refs/heads/master e62a3ff46 -> 6eac7fa02


SOLR-10394: a few more essentially non-public sortWithinGroup to withinGroupSort renames


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

Branch: refs/heads/master
Commit: 323c972922c5d2e272ef4558cf725e5d9ccef623
Parents: f9ca49a
Author: Christine Poerschke <cp...@apache.org>
Authored: Wed Apr 19 11:50:24 2017 +0100
Committer: Christine Poerschke <cp...@apache.org>
Committed: Wed Apr 19 12:28:14 2017 +0100

----------------------------------------------------------------------
 .../command/TopGroupsFieldCommand.java          | 24 ++++++++++----------
 .../SearchGroupShardResponseProcessor.java      |  6 ++---
 .../TopGroupsShardResponseProcessor.java        | 12 +++++-----
 .../SearchGroupsResultTransformer.java          |  2 +-
 .../ShardResultTransformer.java                 |  4 ++--
 .../TopGroupsResultTransformer.java             | 14 ++++++------
 6 files changed, 31 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/323c9729/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
index 80ea4cd..a496278 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/TopGroupsFieldCommand.java
@@ -47,7 +47,7 @@ public class TopGroupsFieldCommand implements Command<TopGroups<BytesRef>> {
 
     private SchemaField field;
     private Sort groupSort;
-    private Sort sortWithinGroup;
+    private Sort withinGroupSort;
     private Collection<SearchGroup<BytesRef>> firstPhaseGroups;
     private Integer maxDocPerGroup;
     private boolean needScores = false;
@@ -63,8 +63,8 @@ public class TopGroupsFieldCommand implements Command<TopGroups<BytesRef>> {
       return this;
     }
 
-    public Builder setSortWithinGroup(Sort sortWithinGroup) {
-      this.sortWithinGroup = sortWithinGroup;
+    public Builder setSortWithinGroup(Sort withinGroupSort) {
+      this.withinGroupSort = withinGroupSort;
       return this;
     }
 
@@ -89,19 +89,19 @@ public class TopGroupsFieldCommand implements Command<TopGroups<BytesRef>> {
     }
 
     public TopGroupsFieldCommand build() {
-      if (field == null || groupSort == null ||  sortWithinGroup == null || firstPhaseGroups == null ||
+      if (field == null || groupSort == null ||  withinGroupSort == null || firstPhaseGroups == null ||
           maxDocPerGroup == null) {
         throw new IllegalStateException("All required fields must be set");
       }
 
-      return new TopGroupsFieldCommand(field, groupSort, sortWithinGroup, firstPhaseGroups, maxDocPerGroup, needScores, needMaxScore);
+      return new TopGroupsFieldCommand(field, groupSort, withinGroupSort, firstPhaseGroups, maxDocPerGroup, needScores, needMaxScore);
     }
 
   }
 
   private final SchemaField field;
   private final Sort groupSort;
-  private final Sort sortWithinGroup;
+  private final Sort withinGroupSort;
   private final Collection<SearchGroup<BytesRef>> firstPhaseGroups;
   private final int maxDocPerGroup;
   private final boolean needScores;
@@ -110,14 +110,14 @@ public class TopGroupsFieldCommand implements Command<TopGroups<BytesRef>> {
 
   private TopGroupsFieldCommand(SchemaField field,
                                 Sort groupSort,
-                                Sort sortWithinGroup,
+                                Sort withinGroupSort,
                                 Collection<SearchGroup<BytesRef>> firstPhaseGroups,
                                 int maxDocPerGroup,
                                 boolean needScores,
                                 boolean needMaxScore) {
     this.field = field;
     this.groupSort = groupSort;
-    this.sortWithinGroup = sortWithinGroup;
+    this.withinGroupSort = withinGroupSort;
     this.firstPhaseGroups = firstPhaseGroups;
     this.maxDocPerGroup = maxDocPerGroup;
     this.needScores = needScores;
@@ -136,11 +136,11 @@ public class TopGroupsFieldCommand implements Command<TopGroups<BytesRef>> {
       ValueSource vs = fieldType.getValueSource(field, null);
       Collection<SearchGroup<MutableValue>> v = GroupConverter.toMutable(field, firstPhaseGroups);
       secondPassCollector = new TopGroupsCollector<>(new ValueSourceGroupSelector(vs, new HashMap<>()),
-          v, groupSort, sortWithinGroup, maxDocPerGroup, needScores, needMaxScore, true
+          v, groupSort, withinGroupSort, maxDocPerGroup, needScores, needMaxScore, true
       );
     } else {
       secondPassCollector = new TopGroupsCollector<>(new TermGroupSelector(field.getName()),
-          firstPhaseGroups, groupSort, sortWithinGroup, maxDocPerGroup, needScores, needMaxScore, true
+          firstPhaseGroups, groupSort, withinGroupSort, maxDocPerGroup, needScores, needMaxScore, true
       );
     }
     collectors.add(secondPassCollector);
@@ -151,7 +151,7 @@ public class TopGroupsFieldCommand implements Command<TopGroups<BytesRef>> {
   @SuppressWarnings("unchecked")
   public TopGroups<BytesRef> result() {
     if (firstPhaseGroups.isEmpty()) {
-      return new TopGroups<>(groupSort.getSort(), sortWithinGroup.getSort(), 0, 0, new GroupDocs[0], Float.NaN);
+      return new TopGroups<>(groupSort.getSort(), withinGroupSort.getSort(), 0, 0, new GroupDocs[0], Float.NaN);
     }
 
     FieldType fieldType = field.getType();
@@ -174,6 +174,6 @@ public class TopGroupsFieldCommand implements Command<TopGroups<BytesRef>> {
 
   @Override
   public Sort getWithinGroupSort() {
-    return sortWithinGroup;
+    return withinGroupSort;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/323c9729/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java
index a12cad1..ab13f72 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/SearchGroupShardResponseProcessor.java
@@ -55,8 +55,8 @@ public class SearchGroupShardResponseProcessor implements ShardResponseProcessor
     SortSpec ss = rb.getSortSpec();
     Sort groupSort = rb.getGroupingSpec().getGroupSort();
     final String[] fields = rb.getGroupingSpec().getFields();
-    Sort sortWithinGroup = rb.getGroupingSpec().getSortWithinGroup();
-    assert sortWithinGroup != null;
+    Sort withinGroupSort = rb.getGroupingSpec().getSortWithinGroup();
+    assert withinGroupSort != null;
 
     final Map<String, List<Collection<SearchGroup<BytesRef>>>> commandSearchGroups = new HashMap<>(fields.length, 1.0f);
     final Map<String, Map<SearchGroup<BytesRef>, Set<String>>> tempSearchGroupToShards = new HashMap<>(fields.length, 1.0f);
@@ -111,7 +111,7 @@ public class SearchGroupShardResponseProcessor implements ShardResponseProcessor
       maxElapsedTime = (int) Math.max(maxElapsedTime, srsp.getSolrResponse().getElapsedTime());
       @SuppressWarnings("unchecked")
       NamedList<NamedList> firstPhaseResult = (NamedList<NamedList>) srsp.getSolrResponse().getResponse().get("firstPhase");
-      final Map<String, SearchGroupsFieldCommandResult> result = serializer.transformToNative(firstPhaseResult, groupSort, sortWithinGroup, srsp.getShard());
+      final Map<String, SearchGroupsFieldCommandResult> result = serializer.transformToNative(firstPhaseResult, groupSort, withinGroupSort, srsp.getShard());
       for (String field : commandSearchGroups.keySet()) {
         final SearchGroupsFieldCommandResult firstPhaseCommandResult = result.get(field);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/323c9729/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/TopGroupsShardResponseProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/TopGroupsShardResponseProcessor.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/TopGroupsShardResponseProcessor.java
index 2ac83c6..231e9bd 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/TopGroupsShardResponseProcessor.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/responseprocessor/TopGroupsShardResponseProcessor.java
@@ -58,8 +58,8 @@ public class TopGroupsShardResponseProcessor implements ShardResponseProcessor {
     Sort groupSort = rb.getGroupingSpec().getGroupSort();
     String[] fields = rb.getGroupingSpec().getFields();
     String[] queries = rb.getGroupingSpec().getQueries();
-    Sort sortWithinGroup = rb.getGroupingSpec().getSortWithinGroup();
-    assert sortWithinGroup != null;
+    Sort withinGroupSort = rb.getGroupingSpec().getSortWithinGroup();
+    assert withinGroupSort != null;
 
     // If group.format=simple group.offset doesn't make sense
     int groupOffsetDefault;
@@ -122,7 +122,7 @@ public class TopGroupsShardResponseProcessor implements ShardResponseProcessor {
       NamedList<NamedList> secondPhaseResult = (NamedList<NamedList>) srsp.getSolrResponse().getResponse().get("secondPhase");
       if(secondPhaseResult == null)
         continue;
-      Map<String, ?> result = serializer.transformToNative(secondPhaseResult, groupSort, sortWithinGroup, srsp.getShard());
+      Map<String, ?> result = serializer.transformToNative(secondPhaseResult, groupSort, withinGroupSort, srsp.getShard());
       int numFound = 0;
       float maxScore = Float.NaN;
       for (String field : commandTopGroups.keySet()) {
@@ -164,7 +164,7 @@ public class TopGroupsShardResponseProcessor implements ShardResponseProcessor {
           docsPerGroup += subTopGroups.totalGroupedHitCount;
         }
       }
-      rb.mergedTopGroups.put(groupField, TopGroups.merge(topGroups.toArray(topGroupsArr), groupSort, sortWithinGroup, groupOffsetDefault, docsPerGroup, TopGroups.ScoreMergeMode.None));
+      rb.mergedTopGroups.put(groupField, TopGroups.merge(topGroups.toArray(topGroupsArr), groupSort, withinGroupSort, groupOffsetDefault, docsPerGroup, TopGroups.ScoreMergeMode.None));
     }
 
     for (String query : commandTopDocs.keySet()) {
@@ -178,10 +178,10 @@ public class TopGroupsShardResponseProcessor implements ShardResponseProcessor {
 
       int topN = rb.getGroupingSpec().getOffset() + rb.getGroupingSpec().getLimit();
       final TopDocs mergedTopDocs;
-      if (sortWithinGroup.equals(Sort.RELEVANCE)) {
+      if (withinGroupSort.equals(Sort.RELEVANCE)) {
         mergedTopDocs = TopDocs.merge(topN, topDocs.toArray(new TopDocs[topDocs.size()]));
       } else {
-        mergedTopDocs = TopDocs.merge(sortWithinGroup, topN, topDocs.toArray(new TopFieldDocs[topDocs.size()]));
+        mergedTopDocs = TopDocs.merge(withinGroupSort, topN, topDocs.toArray(new TopFieldDocs[topDocs.size()]));
       }
       rb.mergedQueryCommandResults.put(query, new QueryCommandResult(mergedTopDocs, mergedMatches));
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/323c9729/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java
index 2602221..77dfcef 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java
@@ -77,7 +77,7 @@ public class SearchGroupsResultTransformer implements ShardResultTransformer<Lis
    * {@inheritDoc}
    */
   @Override
-  public Map<String, SearchGroupsFieldCommandResult> transformToNative(NamedList<NamedList> shardResponse, Sort groupSort, Sort sortWithinGroup, String shard) {
+  public Map<String, SearchGroupsFieldCommandResult> transformToNative(NamedList<NamedList> shardResponse, Sort groupSort, Sort withinGroupSort, String shard) {
     final Map<String, SearchGroupsFieldCommandResult> result = new HashMap<>(shardResponse.size());
     for (Map.Entry<String, NamedList> command : shardResponse) {
       List<SearchGroup<BytesRef>> searchGroups = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/323c9729/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/ShardResultTransformer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/ShardResultTransformer.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/ShardResultTransformer.java
index 04a3dfc..47e20a0 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/ShardResultTransformer.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/ShardResultTransformer.java
@@ -44,10 +44,10 @@ public interface ShardResultTransformer<T, R> {
    *
    * @param shardResponse The shard response containing data in a {@link NamedList} structure
    * @param groupSort The group sort
-   * @param sortWithinGroup The sort inside a group
+   * @param withinGroupSort The sort inside a group
    * @param shard The shard address where the response originated from
    * @return native structure of the data
    */
-  R transformToNative(NamedList<NamedList> shardResponse, Sort groupSort, Sort sortWithinGroup, String shard);
+  R transformToNative(NamedList<NamedList> shardResponse, Sort groupSort, Sort withinGroupSort, String shard);
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/323c9729/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java
index 83c81e5..41145ba 100644
--- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java
+++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java
@@ -92,7 +92,7 @@ public class TopGroupsResultTransformer implements ShardResultTransformer<List<C
    * {@inheritDoc}
    */
   @Override
-  public Map<String, ?> transformToNative(NamedList<NamedList> shardResponse, Sort groupSort, Sort sortWithinGroup, String shard) {
+  public Map<String, ?> transformToNative(NamedList<NamedList> shardResponse, Sort groupSort, Sort withinGroupSort, String shard) {
     Map<String, Object> result = new HashMap<>();
 
     final IndexSchema schema = rb.req.getSearcher().getSchema();
@@ -113,10 +113,10 @@ public class TopGroupsResultTransformer implements ShardResultTransformer<List<C
         List<NamedList<Object>> documents = (List<NamedList<Object>>) commandResult.get("documents");
         ScoreDoc[] scoreDocs = transformToNativeShardDoc(documents, groupSort, shard, schema);
         final TopDocs topDocs;
-        if (sortWithinGroup.equals(Sort.RELEVANCE)) {
+        if (withinGroupSort.equals(Sort.RELEVANCE)) {
           topDocs = new TopDocs(totalHits, scoreDocs, maxScore);
         } else {
-          topDocs = new TopFieldDocs(totalHits, scoreDocs, sortWithinGroup.getSort(), maxScore);
+          topDocs = new TopFieldDocs(totalHits, scoreDocs, withinGroupSort.getSort(), maxScore);
         }
         result.put(key, new QueryCommandResult(topDocs, matches));
         continue;
@@ -137,7 +137,7 @@ public class TopGroupsResultTransformer implements ShardResultTransformer<List<C
 
         @SuppressWarnings("unchecked")
         List<NamedList<Object>> documents = (List<NamedList<Object>>) groupResult.get("documents");
-        ScoreDoc[] scoreDocs = transformToNativeShardDoc(documents, sortWithinGroup, shard, schema);
+        ScoreDoc[] scoreDocs = transformToNativeShardDoc(documents, withinGroupSort, shard, schema);
 
         BytesRef groupValueRef = groupValue != null ? new BytesRef(groupValue) : null;
         groupDocs.add(new GroupDocs<>(Float.NaN, maxScore, totalGroupHits, scoreDocs, groupValueRef, null));
@@ -146,7 +146,7 @@ public class TopGroupsResultTransformer implements ShardResultTransformer<List<C
       @SuppressWarnings("unchecked")
       GroupDocs<BytesRef>[] groupDocsArr = groupDocs.toArray(new GroupDocs[groupDocs.size()]);
       TopGroups<BytesRef> topGroups = new TopGroups<>(
-           groupSort.getSort(), sortWithinGroup.getSort(), totalHitCount, totalGroupedHitCount, groupDocsArr, Float.NaN
+           groupSort.getSort(), withinGroupSort.getSort(), totalHitCount, totalGroupedHitCount, groupDocsArr, Float.NaN
       );
 
       result.put(key, topGroups);
@@ -222,8 +222,8 @@ public class TopGroupsResultTransformer implements ShardResultTransformer<List<C
         Object[] convertedSortValues  = new Object[fieldDoc.fields.length];
         for (int j = 0; j < fieldDoc.fields.length; j++) {
           Object sortValue  = fieldDoc.fields[j];
-          Sort sortWithinGroup = rb.getGroupingSpec().getSortWithinGroup();
-          SchemaField field = sortWithinGroup.getSort()[j].getField() != null ? schema.getFieldOrNull(sortWithinGroup.getSort()[j].getField()) : null;
+          Sort withinGroupSort = rb.getGroupingSpec().getSortWithinGroup();
+          SchemaField field = withinGroupSort.getSort()[j].getField() != null ? schema.getFieldOrNull(withinGroupSort.getSort()[j].getField()) : null;
           if (field != null) {
             FieldType fieldType = field.getType();
             if (sortValue != null) {


[2/3] lucene-solr:master: SOLR-5127: Multiple highlight fields and wildcards are now supported e.g. hl.fl=title, text_* (Sven-S. Porst, Daniel Debray, Simon Endele, Christine Poerschke)

Posted by cp...@apache.org.
SOLR-5127: Multiple highlight fields and wildcards are now supported e.g. hl.fl=title,text_*
(Sven-S. Porst, Daniel Debray, Simon Endele, 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/f9ca49a8
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/f9ca49a8
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/f9ca49a8

Branch: refs/heads/master
Commit: f9ca49a8d59a89e30ce670e2eedcf6560e7ed91d
Parents: e62a3ff
Author: Christine Poerschke <cp...@apache.org>
Authored: Wed Apr 19 11:45:59 2017 +0100
Committer: Christine Poerschke <cp...@apache.org>
Committed: Wed Apr 19 12:28:14 2017 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 ++
 .../apache/solr/highlight/SolrHighlighter.java  | 51 ++++++++++++--------
 .../apache/solr/highlight/HighlighterTest.java  | 20 ++++++++
 .../highlight/TestPostingsSolrHighlighter.java  | 13 +++++
 .../highlight/TestUnifiedSolrHighlighter.java   | 13 +++++
 5 files changed, 80 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f9ca49a8/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index e865311..2b82024 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -215,6 +215,9 @@ Bug Fixes
 
 * SOLR-10472: Fixed uninversion (aka: FieldCache) bugs with the numeric PointField classes, and CurrencyField (hossman)
 
+* SOLR-5127: Multiple highlight fields and wildcards are now supported e.g. hl.fl=title,text_*
+  (Sven-S. Porst, Daniel Debray, Simon Endele, Christine Poerschke)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f9ca49a8/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java b/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java
index e526c57..ade4c6b 100644
--- a/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java
+++ b/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java
@@ -24,9 +24,9 @@ import org.apache.solr.search.DocList;
 import org.apache.solr.util.SolrPluginUtils;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.List;
+import java.util.LinkedHashSet;
+import java.util.Set;
 
 public abstract class SolrHighlighter
 {
@@ -60,27 +60,19 @@ public abstract class SolrHighlighter
       if (emptyArray(defaultFields)) {
         String defaultSearchField = request.getSchema().getDefaultSearchFieldName();
         fields = null == defaultSearchField ? new String[]{} : new String[]{defaultSearchField};
-      }
-      else {
+      } else {
         fields = defaultFields;
       }
-    }
-    else if (fields.length == 1) {
-      if (fields[0].contains("*")) {
-        // create a Java regular expression from the wildcard string
-        String fieldRegex = fields[0].replaceAll("\\*", ".*");
-        Collection<String> storedHighlightFieldNames = request.getSearcher().getDocFetcher().getStoredHighlightFieldNames();
-        List<String> storedFieldsToHighlight = new ArrayList<>();
-        for (String storedFieldName: storedHighlightFieldNames) {
-          if (storedFieldName.matches(fieldRegex)) {
-            storedFieldsToHighlight.add(storedFieldName);
-          }
-        }
-        fields = storedFieldsToHighlight.toArray(new String[storedFieldsToHighlight.size()]);
-      } else {
-        // if there's a single request/handler value, it may be a space/comma separated list
-        fields = SolrPluginUtils.split(fields[0]);
+    } else {
+      Set<String> expandedFields = new LinkedHashSet<String>();
+      Collection<String> storedHighlightFieldNames = request.getSearcher().getDocFetcher().getStoredHighlightFieldNames();
+      for (String field : fields) {
+        expandWildcardsInHighlightFields(
+            expandedFields,
+            storedHighlightFieldNames,
+            SolrPluginUtils.split(field));
       }
+      fields = expandedFields.toArray(new String[]{});
     }
 
     // Trim them now in case they haven't been yet.  Not needed for all code-paths above but do it here.
@@ -94,6 +86,25 @@ public abstract class SolrHighlighter
     return (arr == null || arr.length == 0 || arr[0] == null || arr[0].trim().length() == 0);
   }
 
+  static private void expandWildcardsInHighlightFields (
+      Set<String> expandedFields,
+      Collection<String> storedHighlightFieldNames,
+      String... fields) {
+    for (String field : fields) {
+      if (field.contains("*")) {
+        // create a Java regular expression from the wildcard string
+        String fieldRegex = field.replaceAll("\\*", ".*");
+        for (String storedFieldName : storedHighlightFieldNames) {
+          if (storedFieldName.matches(fieldRegex)) {
+            expandedFields.add(storedFieldName);
+          }
+        }
+      } else {
+        expandedFields.add(field);
+      }
+    }
+  }
+
   /**
    * Generates a list of Highlighted query fragments for each item in a list
    * of documents, or returns null if highlighting is disabled.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f9ca49a8/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java b/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java
index 6506f98..f0b58cd 100644
--- a/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java
+++ b/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java
@@ -20,7 +20,9 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
@@ -35,6 +37,7 @@ import org.apache.solr.common.params.HighlightParams;
 import org.apache.solr.handler.component.HighlightComponent;
 import org.apache.solr.handler.component.ResponseBuilder;
 import org.apache.solr.handler.component.SearchComponent;
+import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.DocSet;
@@ -868,6 +871,8 @@ public class HighlighterTest extends SolrTestCaseJ4 {
         "text", "test", // static not stored
         "foo_s", "test", // dynamic stored
         "foo_sI", "test", // dynamic not stored
+        "bar_s", "test", // dynamic stored
+        "bar_sI", "test", // dynamic not stored
         "weight", "1.0")); // stored but not text
     assertU(commit());
     assertU(optimize());
@@ -898,6 +903,21 @@ public class HighlighterTest extends SolrTestCaseJ4 {
     assertEquals("Expected to highlight on field \"foo_s\"", "foo_s",
         highlightFieldNames.get(0));
     request.close();
+
+    // SOLR-5127
+    args.put("hl.fl", (random().nextBoolean() ? "foo_*,bar_*" : "bar_*,foo_*"));
+    lrf = h.getRequestFactory("standard", 0, 10, args);
+    // hl.fl ordering need not be preserved in output
+    final Set<String> highlightedSetExpected = new HashSet<String>();
+    highlightedSetExpected.add("foo_s");
+    highlightedSetExpected.add("bar_s");
+    try (LocalSolrQueryRequest localRequest = lrf.makeRequest("test")) {
+      highlighter = HighlightComponent.getHighlighter(h.getCore());
+      final Set<String> highlightedSetActual = new HashSet<String>(
+          Arrays.asList(highlighter.getHighlightFields(null,
+              localRequest, new String[] {})));
+      assertEquals(highlightedSetExpected, highlightedSetActual);
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f9ca49a8/solr/core/src/test/org/apache/solr/highlight/TestPostingsSolrHighlighter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/highlight/TestPostingsSolrHighlighter.java b/solr/core/src/test/org/apache/solr/highlight/TestPostingsSolrHighlighter.java
index 3f25464..3862fa6 100644
--- a/solr/core/src/test/org/apache/solr/highlight/TestPostingsSolrHighlighter.java
+++ b/solr/core/src/test/org/apache/solr/highlight/TestPostingsSolrHighlighter.java
@@ -99,6 +99,19 @@ public class TestPostingsSolrHighlighter extends SolrTestCaseJ4 {
         "//lst[@name='highlighting']/lst[@name='102']/arr[@name='text3']/str='crappier <em>document</em>'");
   }
   
+  // SOLR-5127
+  public void testMultipleFieldsViaWildcard() {
+    assertQ("highlighting text and text3*",
+        req("q", (random().nextBoolean() ? "text:document text3:document" : "text3:document text:document"),
+            "sort", "id asc", "hl", "true",
+            "hl.fl", (random().nextBoolean() ? "text,text3*" : "text3*,text")),
+        "count(//lst[@name='highlighting']/*)=2",
+        "//lst[@name='highlighting']/lst[@name='101']/arr[@name='text']/str='<em>document</em> one'",
+        "//lst[@name='highlighting']/lst[@name='101']/arr[@name='text3']/str='crappy <em>document</em>'",
+        "//lst[@name='highlighting']/lst[@name='102']/arr[@name='text']/str='second <em>document</em>'",
+        "//lst[@name='highlighting']/lst[@name='102']/arr[@name='text3']/str='crappier <em>document</em>'");
+  }
+
   public void testMisconfiguredField() {
     ignoreException("was indexed without offsets");
     try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f9ca49a8/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
index 2f7a003..ad1ca03 100644
--- a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
+++ b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java
@@ -179,6 +179,19 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 {
         "//lst[@name='highlighting']/lst[@name='102']/arr[@name='text3']/str='crappier <em>document</em>'");
   }
 
+  // SOLR-5127
+  public void testMultipleFieldsViaWildcard() {
+    assertQ("highlighting text and text3*",
+        req("q", (random().nextBoolean() ? "text:document text3:document" : "text3:document text:document"),
+            "sort", "id asc", "hl", "true",
+            "hl.fl", (random().nextBoolean() ? "text,text3*" : "text3*,text")),
+        "count(//lst[@name='highlighting']/*)=2",
+        "//lst[@name='highlighting']/lst[@name='101']/arr[@name='text']/str='<em>document</em> one'",
+        "//lst[@name='highlighting']/lst[@name='101']/arr[@name='text3']/str='crappy <em>document</em>'",
+        "//lst[@name='highlighting']/lst[@name='102']/arr[@name='text']/str='second <em>document</em>'",
+        "//lst[@name='highlighting']/lst[@name='102']/arr[@name='text3']/str='crappier <em>document</em>'");
+  }
+
   public void testTags() {
     assertQ("different pre/post tags", 
         req("q", "text:document", "sort", "id asc", "hl", "true", "hl.tag.pre", "[", "hl.tag.post", "]"),


[3/3] lucene-solr:master: (part 1 of several) SOLR-10415: use parameterized debug logging in SearchHandler and RealTimeGetComponent (Michael Braun via Christine Poerschke)

Posted by cp...@apache.org.
(part 1 of several) SOLR-10415: use parameterized debug logging in SearchHandler and RealTimeGetComponent (Michael Braun via 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/6eac7fa0
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/6eac7fa0
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/6eac7fa0

Branch: refs/heads/master
Commit: 6eac7fa0211565606ac4acbdc848ad1ef57e6aa1
Parents: 323c972
Author: Christine Poerschke <cp...@apache.org>
Authored: Wed Apr 19 12:05:00 2017 +0100
Committer: Christine Poerschke <cp...@apache.org>
Committed: Wed Apr 19 12:28:15 2017 +0100

----------------------------------------------------------------------
 .../org/apache/solr/handler/component/RealTimeGetComponent.java  | 2 +-
 .../java/org/apache/solr/handler/component/SearchHandler.java    | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6eac7fa0/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
index 4f2c933..c0ceddb 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
@@ -760,7 +760,7 @@ public class RealTimeGetComponent extends SearchComponent
           out.add(f);
         }
       } else {
-        log.debug("Don't know how to handle field " + f);
+        log.debug("Don't know how to handle field {}", f);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6eac7fa0/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
index 926fc00..8b732da 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
@@ -185,12 +185,12 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware ,
         dbgCmp = (DebugComponent) comp;
       } else {
         components.add(comp);
-        log.debug("Adding  component:"+comp);
+        log.debug("Adding  component:{}", comp);
       }
     }
     if (makeDebugLast == true && dbgCmp != null){
       components.add(dbgCmp);
-      log.debug("Adding  debug component:" + dbgCmp);
+      log.debug("Adding  debug component:{}", dbgCmp);
     }
     this.components = components;
   }