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 2020/04/19 00:31:35 UTC

[lucene-solr] branch branch_8x updated: SOLR-14391: getDocSet(Query[]) can use search(query, collector) Refactoring to simplify SolrIndexSearcher. ScoreFilter interface is obsolete now. Fixed #1409

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 1c11962  SOLR-14391: getDocSet(Query[]) can use search(query,collector) Refactoring to simplify SolrIndexSearcher. ScoreFilter interface is obsolete now. Fixed #1409
1c11962 is described below

commit 1c11962076e0dbdb90511219544aefdcdc7536c1
Author: David Smiley <ds...@apache.org>
AuthorDate: Mon Apr 6 12:13:08 2020 -0400

    SOLR-14391: getDocSet(Query[]) can use search(query,collector)
    Refactoring to simplify SolrIndexSearcher.
    ScoreFilter interface is obsolete now.
    Fixed #1409
    
    (cherry picked from commit f5d91395db217b03fcec5e1799c0c77ebbdc8ca6)
---
 solr/CHANGES.txt                                   |  3 +
 .../solr/search/CollapsingQParserPlugin.java       |  2 +-
 .../java/org/apache/solr/search/ScoreFilter.java   | 21 -------
 .../org/apache/solr/search/SolrIndexSearcher.java  | 67 +---------------------
 4 files changed, 6 insertions(+), 87 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1caad62..63bd7dc 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -105,6 +105,9 @@ Other Changes
   removed from solr-core in favor of SolrNamedThreadFactory in solrj package and all solr-core classes now use
   SolrNamedThreadFactory. (Andras Salamon, shalin)
 
+* SOLR-14391: Removed internal-ish ScoreFilter marker interface; only used by {!collapse}.  Removed needless code in
+  SolrIndexSearcher.getDocSet(List<Query>) (processes some filter queries).  (David Smiley)
+
 ==================  8.5.1 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
diff --git a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
index 690f806..7cb5be0 100644
--- a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java
@@ -210,7 +210,7 @@ public class CollapsingQParserPlugin extends QParserPlugin {
     }
   }
 
-  public static class CollapsingPostFilter extends ExtendedQueryBase implements PostFilter, ScoreFilter {
+  public static class CollapsingPostFilter extends ExtendedQueryBase implements PostFilter {
 
     private String collapseField;
     private final GroupHeadSelector groupHeadSelector;
diff --git a/solr/core/src/java/org/apache/solr/search/ScoreFilter.java b/solr/core/src/java/org/apache/solr/search/ScoreFilter.java
deleted file mode 100644
index f566ab1..0000000
--- a/solr/core/src/java/org/apache/solr/search/ScoreFilter.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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.search;
-
-public interface ScoreFilter {
-
-}
\ No newline at end of file
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 97bac32..66fdb96 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -882,25 +882,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
 
   private static Comparator<Query> sortByCost = (q1, q2) -> ((ExtendedQuery) q1).getCost() - ((ExtendedQuery) q2).getCost();
 
-  private DocSet getDocSetScore(List<Query> queries) throws IOException {
-    Query main = queries.remove(0);
-    ProcessedFilter pf = getProcessedFilter(null, queries);
-    DocSetCollector setCollector = new DocSetCollector(maxDoc());
-    Collector collector = setCollector;
-    if (pf.postFilter != null) {
-      pf.postFilter.setLastDelegate(collector);
-      collector = pf.postFilter;
-    }
-
-    search(QueryUtils.combineQueryAndFilter(main, pf.filter), collector);
-
-    if (collector instanceof DelegatingCollector) {
-      ((DelegatingCollector) collector).finish();
-    }
-
-    return DocSetUtil.getDocSet(setCollector, this);
-  }
-
   /**
    * Returns the set of document ids matching all queries. This method is cache-aware and attempts to retrieve the
    * answer from the cache if possible. If the answer was not cached, it may have been inserted into the cache as a
@@ -911,14 +892,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
    */
   public DocSet getDocSet(List<Query> queries) throws IOException {
 
-    if (queries != null) {
-      for (Query q : queries) {
-        if (q instanceof ScoreFilter) {
-          return getDocSetScore(queries);
-        }
-      }
-    }
-
     ProcessedFilter pf = getProcessedFilter(null, queries);
 
     if (pf.postFilter == null) {
@@ -936,43 +909,9 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
       collector = pf.postFilter;
     }
 
-    for (final LeafReaderContext leaf : leafContexts) {
-      final LeafReader reader = leaf.reader();
-      Bits liveDocs = reader.getLiveDocs();
-      DocIdSet idSet = null;
-      if (pf.filter != null) {
-        idSet = pf.filter.getDocIdSet(leaf, liveDocs);
-        if (idSet == null) continue;
-      }
-      DocIdSetIterator idIter = null;
-      if (idSet != null) {
-        idIter = idSet.iterator();
-        if (idIter == null) continue;
-        if (!pf.hasDeletedDocs) liveDocs = null; // no need to check liveDocs
-      }
-
-      final LeafCollector leafCollector = collector.getLeafCollector(leaf);
-      int max = reader.maxDoc();
-
-      if (idIter == null) {
-        for (int docid = 0; docid < max; docid++) {
-          if (liveDocs != null && !liveDocs.get(docid)) continue;
-          leafCollector.collect(docid);
-        }
-      } else {
-        if (liveDocs != null) {
-          for (int docid = -1; (docid = idIter.advance(docid + 1)) < max; ) {
-            if (liveDocs.get(docid))
-              leafCollector.collect(docid);
-          }
-        } else {
-          for (int docid = -1; (docid = idIter.advance(docid + 1)) < max;) {
-            leafCollector.collect(docid);
-          }
-        }
-      }
+    Query query = pf.filter != null ? pf.filter : matchAllDocsQuery;
 
-    }
+    search(query, collector);
 
     if (collector instanceof DelegatingCollector) {
       ((DelegatingCollector) collector).finish();
@@ -991,7 +930,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     public DocSet answer; // maybe null. Sometimes we have a docSet answer that represents the complete answer / result.
     public Filter filter; // maybe null
     public DelegatingCollector postFilter; // maybe null
-    public boolean hasDeletedDocs;  // true if it's possible that filter may match deleted docs
   }
 
   /**
@@ -1117,7 +1055,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
         weights.add(createWeight(rewrite(qq), ScoreMode.COMPLETE_NO_SCORES, 1));
       }
       pf.filter = new FilterImpl(answer, weights);
-      pf.hasDeletedDocs = (answer == null);  // if all clauses were uncached, the resulting filter may match deleted docs
     }
 
     // Set pf.postFilter