You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@solr.apache.org by GitBox <gi...@apache.org> on 2022/01/26 16:33:22 UTC

[GitHub] [solr] Caa52 opened a new pull request #529: SOLR-12336 Remove Filter from Solr

Caa52 opened a new pull request #529:
URL: https://github.com/apache/solr/pull/529


   https://issues.apache.org/jira/browse/SOLR-12336
   
   
   # Description
   
   Currently, we use the `getTopFilter` method in Filter to access a Query based on a DocSet. This is done via calling Filter's main method `getDocIdSet` and then calling an `iterator()`. However, with Filter going away, we still need the ability to have a Query based on a DocSet. Removing Filter requires reducing and possibly eliminating dependencies on `getDocIdSet` and implies that changes must be made to all callers that call Filter specific methods as well. 
   
   # Solution
   
   To access a Query based on a DocSet without going through Filter: 
   
   - Create a new Query, `DocSetQuery`, that emulates the behavior of Filter 
   - Delete all calls to `getDocIdSet` which is Filter's main method
   - Implement change to DocSet's API by replacing `getTopFilter` with `makeQuery` to help go from DocSet to a Query of the DocSet
   - Delete Filter class 
   
   # Tests
   
   Refactored all existing tests that were affected as a result of deleting Filter. Expanded test coverage by updating test suite to account for changes made. All tests are passing. 
   
   # Checklist
   
   Please review the following and check all that apply:
   
   - [X] I have reviewed the guidelines for [How to Contribute](https://wiki.apache.org/solr/HowToContribute) and my code conforms to the standards described there to the best of my ability.
   - [X] I have created a Jira issue and added the issue ID to my pull request title.
   - [X] I have given Solr maintainers [access](https://help.github.com/en/articles/allowing-changes-to-a-pull-request-branch-created-from-a-fork) to contribute to my PR branch. (optional but recommended)
   - [X] I have developed this patch against the `main` branch.
   - [X] I have run `./gradlew check`.
   - [X] I have added tests for my changes.
   - [X] I have added documentation for the [Reference Guide](https://github.com/apache/solr/tree/main/solr/solr-ref-guide)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789803776



##########
File path: solr/contrib/analytics/src/java/org/apache/solr/analytics/AnalyticsDriver.java
##########
@@ -37,28 +37,28 @@
    *
    * @param manager of the request to drive
    * @param searcher the results of the query
-   * @param filter that represents the overall query
+   * @param query that represents the overall query
    * @param queryRequest used for the search request
    * @throws IOException if an error occurs while reading from Solr
    */
-  public static void drive(AnalyticsRequestManager manager, SolrIndexSearcher searcher, Filter filter, SolrQueryRequest queryRequest) throws IOException {
+  public static void drive(AnalyticsRequestManager manager, SolrIndexSearcher searcher, Query query, SolrQueryRequest queryRequest) throws IOException {

Review comment:
       Thanks for pointing this out. I made the changes accordingly. 
   I made the change to `DocSet` in `AnalyticsDriver` but not in other areas because the change from `Query` to `DocSet` affects a lot of other APIs further downstream. I left a TODO comment in `AnalyticsDriver` indicating this. Maybe we can punt the changes to a future issue?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787319124



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery {
+    protected final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSet getDocSet() {
+        return docSet;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj).docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+        return new SolrConstantScoreQuery.ConstantWeight(searcher, scoreMode, boost) {

Review comment:
       Can we please not use SolrConstantScoreQuery?  Try ConstantScoreWeight in Lucene

##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -39,10 +40,20 @@ public FunctionRangeQuery(ValueSourceRangeFilter filter) {
     this.cost = 100; // default behavior should be PostFiltering
   }
 
+  @Override

Review comment:
       I'm not sure why changes are here but I suspect it's related to the increased scope.  Maybe this can be punted for another issue.

##########
File path: solr/contrib/analytics/src/java/org/apache/solr/analytics/facet/AbstractSolrQueryFacet.java
##########
@@ -50,11 +49,11 @@ protected AbstractSolrQueryFacet(String name) {
    *
    * Each of these executors will be executed after the streaming phase in the {@link AnalyticsDriver}.
    *
-   * @param filter the overall filter representing the documents being used for the analytics request

Review comment:
       I think the name "filter" is reasonable as it conveys it's purpose is only for filtering and not also for scoring.  If possible, we could use DocSet?

##########
File path: solr/core/src/java/org/apache/solr/search/BitDocSet.java
##########
@@ -311,72 +307,8 @@ public long cost() {
     };
   }
 
-  @Override
-  public Filter getTopFilter() {
-    // TODO: if cardinality isn't cached, do a quick measure of sparseness
-    // and return null from bits() if too sparse.
-
-    return new Filter() {
-      final FixedBitSet bs = bits;
-
-      @Override
-      public DocIdSet getDocIdSet(final LeafReaderContext context, final Bits acceptDocs) {
-        // all Solr DocSets that are used as filters only include live docs
-        final Bits acceptDocs2 = acceptDocs == null ? null : (context.reader().getLiveDocs() == acceptDocs ? null : acceptDocs);
-
-        return BitsFilteredDocIdSet.wrap(new DocIdSet() {
-          @Override
-          public DocIdSetIterator iterator() {
-            return BitDocSet.this.iterator(context);
-          }
-
-          @Override
-          public long ramBytesUsed() {
-            return BitDocSet.this.ramBytesUsed();
-          }
-
-          @Override
-          public Bits bits() {
-            if (context.isTopLevel) {
-              return bits;
-            }
-
-            final int base = context.docBase;
-            final int length = context.reader().maxDoc();
-            final FixedBitSet bs = bits;
-
-            return new Bits() {
-              @Override
-              public boolean get(int index) {
-                return bs.get(index + base);
-              }
-
-              @Override
-              public int length() {
-                return length;
-              }
-            };
-          }
-
-        }, acceptDocs2);
-      }
-
-      @Override
-      public String toString(String field) {
-        return "BitSetDocTopFilter";
-      }
-
-      @Override
-      public boolean equals(Object other) {
-        return sameClassAs(other) &&
-               Objects.equals(bs, getClass().cast(other).bs);
-      }
-      
-      @Override
-      public int hashCode() {
-        return classHash() * 31 + bs.hashCode();
-      }
-    };
+  public DocSetQuery makeQuery() {
+    return new DocSetQuery(this);

Review comment:
       All implementations of bits() return null these days.  It's not obvious; you need to find-usages a couple layers and look carefully.

##########
File path: solr/contrib/analytics/src/java/org/apache/solr/analytics/facet/QueryFacet.java
##########
@@ -40,7 +39,7 @@ public QueryFacet(String name, Map<String, String> queries) {
   }
 
   @Override
-  public void createFacetValueExecuters(final Filter filter, SolrQueryRequest queryRequest, Consumer<FacetValueQueryExecuter> consumer) {
+  public void createFacetValueExecuters(final Query qry, SolrQueryRequest queryRequest, Consumer<FacetValueQueryExecuter> consumer) {

Review comment:
       again, "filter" is a reasonable name

##########
File path: solr/contrib/analytics/src/java/org/apache/solr/analytics/AnalyticsDriver.java
##########
@@ -37,28 +37,28 @@
    *
    * @param manager of the request to drive
    * @param searcher the results of the query
-   * @param filter that represents the overall query
+   * @param query that represents the overall query
    * @param queryRequest used for the search request
    * @throws IOException if an error occurs while reading from Solr
    */
-  public static void drive(AnalyticsRequestManager manager, SolrIndexSearcher searcher, Filter filter, SolrQueryRequest queryRequest) throws IOException {
+  public static void drive(AnalyticsRequestManager manager, SolrIndexSearcher searcher, Query query, SolrQueryRequest queryRequest) throws IOException {

Review comment:
       It appears the caller should pass a DocSet (and leave named "filter")?  When calling getFacetExecutors, you can then call filter.makeQuery().  A DocIdSetIterator can be retrieved from the DocSet. 

##########
File path: solr/core/src/test/org/apache/solr/search/TestFilteredDocIdSet.java
##########
@@ -127,22 +130,49 @@ public void testNullDocIdSet() throws Exception {
     IndexSearcher searcher = newSearcher(reader);
     Assert.assertEquals(1, searcher.search(new MatchAllDocsQuery(), 10).totalHits.value);
     
-    // Now search w/ a Filter which returns a null DocIdSet
-    Filter f = new Filter() {
+    // Now search w/ a Query which returns a null DocIdSet
+    Query f = new Query() {

Review comment:
       This test should be testing DocSetQuery.  If it's a null set of IDs, then the underlying DocIdSetIterator will be null.

##########
File path: solr/contrib/analytics/src/java/org/apache/solr/analytics/facet/RangeFacet.java
##########
@@ -54,7 +53,7 @@ public RangeFacet(String name, SchemaField field, String start, String end, List
   }
 
   @Override
-  public void createFacetValueExecuters(final Filter filter, SolrQueryRequest queryRequest, Consumer<FacetValueQueryExecuter> consumer) {
+  public void createFacetValueExecuters(final Query query, SolrQueryRequest queryRequest, Consumer<FacetValueQueryExecuter> consumer) {

Review comment:
       "filter" is a reasonable name

##########
File path: solr/core/src/test/org/apache/solr/search/TestSort.java
##########
@@ -236,16 +240,43 @@ public void testSort() throws Exception {
       assertTrue(reader.leaves().size() > 1);
 
       for (int i=0; i<qiter; i++) {
-        Filter filt = new Filter() {

Review comment:
       appears out of scope of this JIRA

##########
File path: solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
##########
@@ -41,17 +41,20 @@
  * Experimental and subject to change.
  */
 public class SolrConstantScoreQuery extends Query implements ExtendedQuery {
-  private final Filter filter;

Review comment:
       Out of scope of this issue, but SolrConstantScoreQuery can go away.  It's fundamental job is to make a Filter look like a Query way back when Filter wasn't a Query.  For many years, Filter has been one.  It also does what SolrExtendedQuery (or whatever it's called) does.

##########
File path: solr/core/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java
##########
@@ -35,7 +37,7 @@
 /**
  * RangeFilter over a ValueSource.
  */
-public class ValueSourceRangeFilter extends SolrFilter {

Review comment:
       This in not in scope of this JIRA issue

##########
File path: solr/core/src/java/org/apache/solr/search/BitsFilteredDocIdSet.java
##########
@@ -23,15 +23,11 @@
 
 /**
  * This implementation supplies a filtered DocIdSet, that excludes all
- * docids which are not in a Bits instance. This is especially useful in
- * {@link org.apache.solr.search.Filter} to apply the {@code acceptDocs}
- * passed to {@code getDocIdSet()} before returning the final DocIdSet.
+ * docids which are not in a Bits instance.
  *
  * @see DocIdSet
- * @see org.apache.solr.search.Filter
  */
 public final class BitsFilteredDocIdSet extends FilteredDocIdSet {

Review comment:
       Nice catch Mike!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 edited a comment on pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 edited a comment on pull request #529:
URL: https://github.com/apache/solr/pull/529#issuecomment-1022402464


   This PR was closed in favor of the new one https://github.com/apache/solr/pull/566 which is a continuation of this one (SOLR-15257). The PR was changed as a result of renaming the branch to correctly reference the parent issue SOLR-12336. Please refer to new PR [ SOLR-12336](https://github.com/apache/solr/pull/566


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799607211



##########
File path: solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
##########
@@ -287,24 +288,23 @@ public static void createGraphField(FacetRequest.Domain domain, Map<String,Objec
        * Creates a Query that can be used to recompute the new "base" for this domain, relative to the
        * current base of the FacetContext.
        */
-      public Query createDomainQuery(FacetContext fcontext) throws IOException {
-        final SolrConstantScoreQuery fromQuery = new SolrConstantScoreQuery(fcontext.base.getTopFilter());
+      public Query createDomainQuery(FacetContext fcontext) {
+        final ConstantScoreQuery fromQuery = new ConstantScoreQuery(fcontext.base.makeQuery());

Review comment:
       Refactored it so `fromQuery` is not wrapped in a constant score query given we are using makeQuery which is a constant scoring query šŸ‘




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r803130966



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && equalsTo(getClass().cast(obj));
+    }
+
+    private boolean equalsTo(DocSetQuery other) {
+        return Objects.equals(docSet, other.docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + (docSet != null ? docSet.hashCode() : 0);
+    }
+
+    /**
+     * @param searcher is not used because we already have a DocSet created in DocSetQuery
+     * @return the DocSet created in DocSetQuery
+     */
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
+        return docSet;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+        //This should probably use the provided boost as scorer. However, that causes
+        // TestSolrQueryParser.testFilter to fail.

Review comment:
       SolrConstantScoreQuery allowed boostable (non-zero) scores; so removing it in this PR but keeping the 0 here on DocSetQuery (formerly Filter) is making the use of 0 score wider/worse.  Code paths that used SolrConstantScoreQuery were getting this and now if they use DocSetQuery, they have 0 score.  These cases are not tested as it's an obscure matter.  Any way; I'd be happy to clean this up in SOLR-14800 this weekend if you want to just let this be in this PR.

##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -73,4 +86,25 @@ protected void doSetNextReader(LeafReaderContext context) throws IOException {
       scorer = dv.getRangeScorer(weight, context, rangeFilt.getLowerVal(), rangeFilt.getUpperVal(), rangeFilt.isIncludeLower(), rangeFilt.isIncludeUpper());
     }
   }
+
+  @Override
+  public String toString(String field) {
+    return "FunctionRangeQuery(" + field + ")";
+  }
+
+  @Override
+  public void visit(QueryVisitor visitor) {
+    visitor.visitLeaf(this);
+  }
+
+  @Override
+  public boolean equals(Object obj) {

Review comment:
       Nice to see equals & hashCode added!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob merged pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
madrob merged pull request #529:
URL: https://github.com/apache/solr/pull/529


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799627099



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && equalsTo(getClass().cast(obj));
+    }
+
+    private boolean equalsTo(DocSetQuery other) {
+        return Objects.equals(docSet, other.docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();

Review comment:
       Thanks for highlighting that šŸ‘. I have implemented a check for null docSet 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r794642566



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -25,18 +25,16 @@
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.ValueSourceScorer;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Weight;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.search.function.ValueSourceRangeFilter;
 
 // This class works as either a normal constant score query, or as a PostFilter using a collector
-public class FunctionRangeQuery extends Query implements PostFilter {
+public class FunctionRangeQuery extends ExtendedQueryBase implements PostFilter {
 
   final ValueSourceRangeFilter rangeFilt;
-  boolean cache = true; // cache by default
   int cost;

Review comment:
       Do we still need this?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r792770814



##########
File path: solr/CHANGES.txt
##########
@@ -256,11 +256,15 @@ when told to. The admin UI now tells it to. (Nazerke Seidan, David Smiley)
 * SOLR-15209: The old LegacyAssignStrategy has been refactored into the SimplePlacementPlugin. This is still the default
   placement policy for Solr. (Houston Putman, Ilan Ginzburg)
 
+* SOLR-15257: Add DocSetQuery in lieu of DocSet.getTopFilter. This issue led to removing
+  Filter.java and eliminating all dependencies on its main method getDocIdSet. With Filter deleted ,
+  and replaced with Lucene's Query.java, the changes might spark some performance optimizations (Collins Abanda, Mike Drob, Tim Potter)

Review comment:
       This is too implementation specific. Something more high level for a user or developer to understand.
   
   ```suggestion
   * SOLR-15257: Remove Filter and SolrFilter, replaced by DocSetQuery which is a Query and DocSetProducer. (Collins Abanda, Mike Drob, Tim Potter)
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
madrob commented on pull request #529:
URL: https://github.com/apache/solr/pull/529#issuecomment-1022351724


   @dsmiley is this a 9.0 change because of the API surface we touch?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787223054



##########
File path: solr/core/src/java/org/apache/solr/search/DocSet.java
##########
@@ -121,7 +121,7 @@ public int andNotSize(DocSet other) {
    * was generated from the top-level MultiReader that the Lucene search
    * methods will be invoked with.
    */
-  public abstract Filter getTopFilter();
+  public abstract DocSetQuery makeQuery();

Review comment:
       No, not at all. Thanks for bringing it up. I concur that it's indeed best for it to simply return a `Query` rather than a `DocSetQuery`. I have made the correction. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787248034



##########
File path: solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
##########
@@ -75,35 +78,42 @@ public int getCost() {
   }
 
   protected class ConstantWeight extends ConstantScoreWeight {
-    private Map<Object,Object> context;
+    private Map<Object, Object> context;

Review comment:
       @sonatype-lift help




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787177641



##########
File path: solr/core/src/java/org/apache/solr/search/PostFilter.java
##########
@@ -18,6 +18,7 @@
 
 import org.apache.lucene.search.IndexSearcher;
 
+

Review comment:
       Thanks for the catch! Fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789999177



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{

Review comment:
       Given that we are transitioning away from `SolrConstantScoreQuery`, I will correct that to extend `Query.java` instead




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r790010642



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{
+    DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj).docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
+        return null;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+        return new ConstantScoreWeight(this, boost) {
+            @Override
+            public Scorer scorer(LeafReaderContext context) throws IOException {
+            DocIdSetIterator disi = docSet.iterator(context);
+            if (disi == null) {
+                return null;
+            }
+            return new ConstantScoreScorer(this, score(), scoreMode, disi);
+            }
+
+            @Override
+            public boolean isCacheable(LeafReaderContext ctx) {
+                return false;

Review comment:
       Fixed it. Thanks for pointing this out!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r790195317



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{
+    DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj).docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
+        return null;

Review comment:
       Thanks for pointing this out. I'll add in a //nocommit




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787248045



##########
File path: solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
##########
@@ -75,35 +78,42 @@ public int getCost() {
   }
 
   protected class ConstantWeight extends ConstantScoreWeight {
-    private Map<Object,Object> context;
+    private Map<Object, Object> context;

Review comment:
       Iā€™m LiftBot and I automatically analyze new code in code review, and comment when I find potential bugs. I also accept comments as commands. Just `@sonatype-lift` followed by the command: **ignore** to mark as false positive, **unignore** to undo, and **help** to see this message. [Click here](https://github.com/apps/sonatype-lift/installations/new) to add LiftBot to another repo.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787215152



##########
File path: solr/core/src/test/org/apache/solr/search/TestFilteredDocIdSet.java
##########
@@ -127,22 +130,49 @@ public void testNullDocIdSet() throws Exception {
     IndexSearcher searcher = newSearcher(reader);
     Assert.assertEquals(1, searcher.search(new MatchAllDocsQuery(), 10).totalHits.value);
     
-    // Now search w/ a Filter which returns a null DocIdSet
-    Filter f = new Filter() {
+    // Now search w/ a Query which returns a null DocIdSet
+    Query f = new Query() {
       @Override
-      public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) {
-        return null;
+      public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) {
+        return new Weight(this) {
+
+          @Override
+          public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+            final Scorer scorer = scorer(context);
+            final boolean match = (scorer != null && scorer.iterator().advance(doc) == doc);
+            if (match) {
+              assert scorer.score() == 0f;
+              return Explanation.match(0f, "Match on id " + doc);

Review comment:
       Yes, I agree. I will have that fixed. Thanks for pointing it out! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787177884



##########
File path: solr/core/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java
##########
@@ -76,33 +78,50 @@ public boolean isIncludeUpper() {
     return includeUpper;
   }
 
+//  @Override
+//  public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
+//    return null;
+//  }

Review comment:
       Yes, I'll clean that up šŸ‘




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r790868847



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -26,19 +26,29 @@
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.solr.util.TestInjection;
 
 import java.io.IOException;
 import java.util.Objects;
 
+/**
+ * A class that accesses Queries based on a DocSet
+ * This class was created as a replacement to DocSet.getTopFilter
+ * which has since been removed with Filter

Review comment:
       I don't think we need this? What value will it provide to somebody new looking at the class?

##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -63,7 +73,20 @@ public int hashCode() {
 
     @Override
     public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
-        return null;
+        return newDocSet(searcher, Math.min(64,(searcher.maxDoc()>>>10)+4));

Review comment:
       This doesn't make sense...
   
   Shouldn't it be `return docSet;`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r791040364



##########
File path: solr/core/src/java/org/apache/solr/search/BitDocSet.java
##########
@@ -311,72 +307,8 @@ public long cost() {
     };
   }
 
-  @Override
-  public Filter getTopFilter() {
-    // TODO: if cardinality isn't cached, do a quick measure of sparseness
-    // and return null from bits() if too sparse.
-
-    return new Filter() {
-      final FixedBitSet bs = bits;
-
-      @Override
-      public DocIdSet getDocIdSet(final LeafReaderContext context, final Bits acceptDocs) {
-        // all Solr DocSets that are used as filters only include live docs
-        final Bits acceptDocs2 = acceptDocs == null ? null : (context.reader().getLiveDocs() == acceptDocs ? null : acceptDocs);
-
-        return BitsFilteredDocIdSet.wrap(new DocIdSet() {
-          @Override
-          public DocIdSetIterator iterator() {
-            return BitDocSet.this.iterator(context);
-          }
-
-          @Override
-          public long ramBytesUsed() {
-            return BitDocSet.this.ramBytesUsed();
-          }
-
-          @Override
-          public Bits bits() {
-            if (context.isTopLevel) {
-              return bits;
-            }
-
-            final int base = context.docBase;
-            final int length = context.reader().maxDoc();
-            final FixedBitSet bs = bits;
-
-            return new Bits() {
-              @Override
-              public boolean get(int index) {
-                return bs.get(index + base);
-              }
-
-              @Override
-              public int length() {
-                return length;
-              }
-            };
-          }
-
-        }, acceptDocs2);
-      }
-
-      @Override
-      public String toString(String field) {
-        return "BitSetDocTopFilter";
-      }
-
-      @Override
-      public boolean equals(Object other) {
-        return sameClassAs(other) &&
-               Objects.equals(bs, getClass().cast(other).bs);
-      }
-      
-      @Override
-      public int hashCode() {
-        return classHash() * 31 + bs.hashCode();
-      }
-    };
+  public DocSetQuery makeQuery() {
+    return new DocSetQuery(this);

Review comment:
       If all implementations return null, then can we get rid of BitDocSet? Follow-on issue?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793194710



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+        docSet = null;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj));

Review comment:
       Thanks for catching this David & Sonatype-lift šŸ™‚! I have made some changes there to address this
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r794647968



##########
File path: solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
##########
@@ -234,9 +234,8 @@ public Query createDomainQuery(FacetContext fcontext) throws IOException {
         // the number of classes that have to know about the number of possible settings on the join
         // (ie: if we add a score mode, or some other modifier to how the joins are done)
 
-        final SolrConstantScoreQuery fromQuery = new SolrConstantScoreQuery(fcontext.base.makeQuery());
+        final ConstantScoreQuery fromQuery = new ConstantScoreQuery(fcontext.base.makeQuery());
         // this shouldn't matter once we're wrapped in a join query, but just in case it ever does...
-        fromQuery.setCache(false);

Review comment:
       The intent here looks to be that we don't want to cache the individual sub-queries that would get wrapped in the JoinQuery later (presumably because we are caching the whole join?)
   
   Since lucene query doesn't have this idea of caching, we need to wrap fromQuery in an ExtendedQuery to disable the caching.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #529: SOLR-15257 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r792913954



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+        docSet = null;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj));

Review comment:
       *EqualsIncompatibleType:*  Calling equals on incompatible types DocSet and DocSetQuery [(details)](https://errorprone.info/bugpattern/EqualsIncompatibleType)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793926645



##########
File path: solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
##########
@@ -234,9 +234,8 @@ public Query createDomainQuery(FacetContext fcontext) throws IOException {
         // the number of classes that have to know about the number of possible settings on the join
         // (ie: if we add a score mode, or some other modifier to how the joins are done)
 
-        final SolrConstantScoreQuery fromQuery = new SolrConstantScoreQuery(fcontext.base.makeQuery());
+        final ConstantScoreQuery fromQuery = new ConstantScoreQuery(fcontext.base.makeQuery());
         // this shouldn't matter once we're wrapped in a join query, but just in case it ever does...
-        fromQuery.setCache(false);

Review comment:
       I couldn't see how it was necessary. Why would we still need it? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799890106



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -36,12 +36,11 @@
 
   final ValueSourceRangeFilter rangeFilt;
   boolean cache = true; // cache by default

Review comment:
       Yes, I cleaned that up, thanks! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799502566



##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -89,13 +87,13 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
 
     if (!(searcher instanceof SolrIndexSearcher)) {
       // delete-by-query won't have SolrIndexSearcher
-      return new BoostQuery(new ConstantScoreQuery(q), 0).createWeight(searcher, scoreMode, 1f);
+      return new ConstantScoreQuery(q).createWeight(searcher, scoreMode, 1f);

Review comment:
       @dsmiley Does this take care of SOLR-14800 as well?

##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && equalsTo(getClass().cast(obj));
+    }
+
+    private boolean equalsTo(DocSetQuery other) {
+        return Objects.equals(docSet, other.docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    /**
+     * @param searcher is not used because we already have a DocSet created in DocSetQuery
+     * @return the DocSet created in DocSetQuery
+     */
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
+        return docSet;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+        return new ConstantScoreWeight(this, 0) {

Review comment:
       Should this be zero or the provided boost?

##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && equalsTo(getClass().cast(obj));
+    }
+
+    private boolean equalsTo(DocSetQuery other) {
+        return Objects.equals(docSet, other.docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    /**
+     * @param searcher is not used because we already have a DocSet created in DocSetQuery
+     * @return the DocSet created in DocSetQuery
+     */
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
+        return docSet;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+        return new ConstantScoreWeight(this, 0) {
+            @Override
+            public Scorer scorer(LeafReaderContext context) throws IOException {
+                DocIdSetIterator disi = docSet.iterator(context);

Review comment:
       If you check the docset for null here, then you can simplify some of the implementations in TestFilteredDocIdSet.java, but let's leave that for another time.

##########
File path: solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
##########
@@ -229,19 +230,19 @@ public static void createJoinField(FacetRequest.Domain domain, Map<String,Object
        * Creates a Query that can be used to recompute the new "base" for this domain, relative to the
        * current base of the FacetContext.
        */
-      public Query createDomainQuery(FacetContext fcontext) throws IOException {
+      public Query createDomainQuery(FacetContext fcontext) {
         // NOTE: this code lives here, instead of in FacetProcessor.handleJoin, in order to minimize
         // the number of classes that have to know about the number of possible settings on the join
         // (ie: if we add a score mode, or some other modifier to how the joins are done)
 
-        final SolrConstantScoreQuery fromQuery = new SolrConstantScoreQuery(fcontext.base.getTopFilter());
+        final ConstantScoreQuery fromQuery = new ConstantScoreQuery(fcontext.base.makeQuery());

Review comment:
       Do we need to wrap this in a constant score query?

##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -89,13 +87,13 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
 
     if (!(searcher instanceof SolrIndexSearcher)) {
       // delete-by-query won't have SolrIndexSearcher
-      return new BoostQuery(new ConstantScoreQuery(q), 0).createWeight(searcher, scoreMode, 1f);
+      return new ConstantScoreQuery(q).createWeight(searcher, scoreMode, 1f);
     }
 
     SolrIndexSearcher solrSearcher = (SolrIndexSearcher)searcher;
     DocSet docs = solrSearcher.getDocSet(q);
     // reqInfo.addCloseHook(docs);  // needed for off-heap refcounting
 
-    return new BoostQuery(new SolrConstantScoreQuery(docs.getTopFilter()), 0).createWeight(searcher, scoreMode, 1f);
+    return docs.makeQuery().createWeight(searcher, scoreMode, 1f);

Review comment:
       I'm confused what is going on here. We take the query and get the docs and then use the docs to make a query to get a constant score weight. This seems wasteful but maybe there is some nuance in the processing that I'm missing.

##########
File path: solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
##########
@@ -287,24 +288,23 @@ public static void createGraphField(FacetRequest.Domain domain, Map<String,Objec
        * Creates a Query that can be used to recompute the new "base" for this domain, relative to the
        * current base of the FacetContext.
        */
-      public Query createDomainQuery(FacetContext fcontext) throws IOException {
-        final SolrConstantScoreQuery fromQuery = new SolrConstantScoreQuery(fcontext.base.getTopFilter());
+      public Query createDomainQuery(FacetContext fcontext) {
+        final ConstantScoreQuery fromQuery = new ConstantScoreQuery(fcontext.base.makeQuery());

Review comment:
       Do we need to wrap this in a constant score query?

##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && equalsTo(getClass().cast(obj));
+    }
+
+    private boolean equalsTo(DocSetQuery other) {
+        return Objects.equals(docSet, other.docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();

Review comment:
       docSet can be null




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799021367



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -36,12 +36,13 @@
 
   final ValueSourceRangeFilter rangeFilt;
   boolean cache = true; // cache by default
-  int cost;
+  //int cost;

Review comment:
       Forgot this commented out code in. Will clean that out




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799609711



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && equalsTo(getClass().cast(obj));
+    }
+
+    private boolean equalsTo(DocSetQuery other) {
+        return Objects.equals(docSet, other.docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    /**
+     * @param searcher is not used because we already have a DocSet created in DocSetQuery
+     * @return the DocSet created in DocSetQuery
+     */
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
+        return docSet;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+        return new ConstantScoreWeight(this, 0) {

Review comment:
       It should be the provided boost of the wrapped query. Thanks for pointing this out! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787233734



##########
File path: solr/core/src/java/org/apache/solr/search/BitDocSet.java
##########
@@ -311,72 +307,8 @@ public long cost() {
     };
   }
 
-  @Override
-  public Filter getTopFilter() {
-    // TODO: if cardinality isn't cached, do a quick measure of sparseness
-    // and return null from bits() if too sparse.
-
-    return new Filter() {
-      final FixedBitSet bs = bits;
-
-      @Override
-      public DocIdSet getDocIdSet(final LeafReaderContext context, final Bits acceptDocs) {
-        // all Solr DocSets that are used as filters only include live docs
-        final Bits acceptDocs2 = acceptDocs == null ? null : (context.reader().getLiveDocs() == acceptDocs ? null : acceptDocs);
-
-        return BitsFilteredDocIdSet.wrap(new DocIdSet() {
-          @Override
-          public DocIdSetIterator iterator() {
-            return BitDocSet.this.iterator(context);
-          }
-
-          @Override
-          public long ramBytesUsed() {
-            return BitDocSet.this.ramBytesUsed();
-          }
-
-          @Override
-          public Bits bits() {
-            if (context.isTopLevel) {
-              return bits;
-            }
-
-            final int base = context.docBase;
-            final int length = context.reader().maxDoc();
-            final FixedBitSet bs = bits;
-
-            return new Bits() {
-              @Override
-              public boolean get(int index) {
-                return bs.get(index + base);
-              }
-
-              @Override
-              public int length() {
-                return length;
-              }
-            };
-          }
-
-        }, acceptDocs2);
-      }
-
-      @Override
-      public String toString(String field) {
-        return "BitSetDocTopFilter";
-      }
-
-      @Override
-      public boolean equals(Object other) {
-        return sameClassAs(other) &&
-               Objects.equals(bs, getClass().cast(other).bs);
-      }
-      
-      @Override
-      public int hashCode() {
-        return classHash() * 31 + bs.hashCode();
-      }
-    };
+  public DocSetQuery makeQuery() {
+    return new DocSetQuery(this);

Review comment:
       Wasn't too sure about how to handle this. Might need to do some more digging into it if it will indeed be a major concern (either in this, or as a separate issue?). 
   
   Hi @dsmiley, what are your thoughts here? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787219384



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery {
+    protected final DocSet docSet;

Review comment:
       There was no particular reason for having it as `protected` when I created the class. I have changed it to be package-private. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
dsmiley commented on pull request #529:
URL: https://github.com/apache/solr/pull/529#issuecomment-1021848332


   I was chatting with Mike about splitting this vs not.  It's fine to not split this, and instead edit the PR title to refer to the parent issue about Filter disappearing.  I'll handle https://issues.apache.org/jira/browse/SOLR-14800 separately.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r791040608



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+        docSet = null;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj));
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {

Review comment:
       Javadoc indicating that the searcher will not be used.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r791125848



##########
File path: solr/core/src/java/org/apache/solr/search/BitDocSet.java
##########
@@ -311,72 +307,8 @@ public long cost() {
     };
   }
 
-  @Override
-  public Filter getTopFilter() {
-    // TODO: if cardinality isn't cached, do a quick measure of sparseness
-    // and return null from bits() if too sparse.
-
-    return new Filter() {
-      final FixedBitSet bs = bits;
-
-      @Override
-      public DocIdSet getDocIdSet(final LeafReaderContext context, final Bits acceptDocs) {
-        // all Solr DocSets that are used as filters only include live docs
-        final Bits acceptDocs2 = acceptDocs == null ? null : (context.reader().getLiveDocs() == acceptDocs ? null : acceptDocs);
-
-        return BitsFilteredDocIdSet.wrap(new DocIdSet() {
-          @Override
-          public DocIdSetIterator iterator() {
-            return BitDocSet.this.iterator(context);
-          }
-
-          @Override
-          public long ramBytesUsed() {
-            return BitDocSet.this.ramBytesUsed();
-          }
-
-          @Override
-          public Bits bits() {
-            if (context.isTopLevel) {
-              return bits;
-            }
-
-            final int base = context.docBase;
-            final int length = context.reader().maxDoc();
-            final FixedBitSet bs = bits;
-
-            return new Bits() {
-              @Override
-              public boolean get(int index) {
-                return bs.get(index + base);
-              }
-
-              @Override
-              public int length() {
-                return length;
-              }
-            };
-          }
-
-        }, acceptDocs2);
-      }
-
-      @Override
-      public String toString(String field) {
-        return "BitSetDocTopFilter";
-      }
-
-      @Override
-      public boolean equals(Object other) {
-        return sameClassAs(other) &&
-               Objects.equals(bs, getClass().cast(other).bs);
-      }
-      
-      @Override
-      public int hashCode() {
-        return classHash() * 31 + bs.hashCode();
-      }
-    };
+  public DocSetQuery makeQuery() {
+    return new DocSetQuery(this);

Review comment:
       I don't follow; why would we remove BitDocSet?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on pull request #529:
URL: https://github.com/apache/solr/pull/529#issuecomment-1020417262


   I rebased and force pushed the latest commit to due to recent changes in `CHANGES.txt` renaming `contrib` to `module`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r792764998



##########
File path: solr/core/src/java/org/apache/solr/search/BitDocSet.java
##########
@@ -311,72 +307,8 @@ public long cost() {
     };
   }
 
-  @Override
-  public Filter getTopFilter() {
-    // TODO: if cardinality isn't cached, do a quick measure of sparseness
-    // and return null from bits() if too sparse.
-
-    return new Filter() {
-      final FixedBitSet bs = bits;
-
-      @Override
-      public DocIdSet getDocIdSet(final LeafReaderContext context, final Bits acceptDocs) {
-        // all Solr DocSets that are used as filters only include live docs
-        final Bits acceptDocs2 = acceptDocs == null ? null : (context.reader().getLiveDocs() == acceptDocs ? null : acceptDocs);
-
-        return BitsFilteredDocIdSet.wrap(new DocIdSet() {
-          @Override
-          public DocIdSetIterator iterator() {
-            return BitDocSet.this.iterator(context);
-          }
-
-          @Override
-          public long ramBytesUsed() {
-            return BitDocSet.this.ramBytesUsed();
-          }
-
-          @Override
-          public Bits bits() {
-            if (context.isTopLevel) {
-              return bits;
-            }
-
-            final int base = context.docBase;
-            final int length = context.reader().maxDoc();
-            final FixedBitSet bs = bits;
-
-            return new Bits() {
-              @Override
-              public boolean get(int index) {
-                return bs.get(index + base);
-              }
-
-              @Override
-              public int length() {
-                return length;
-              }
-            };
-          }
-
-        }, acceptDocs2);
-      }
-
-      @Override
-      public String toString(String field) {
-        return "BitSetDocTopFilter";
-      }
-
-      @Override
-      public boolean equals(Object other) {
-        return sameClassAs(other) &&
-               Objects.equals(bs, getClass().cast(other).bs);
-      }
-      
-      @Override
-      public int hashCode() {
-        return classHash() * 31 + bs.hashCode();
-      }
-    };
+  public DocSetQuery makeQuery() {
+    return new DocSetQuery(this);

Review comment:
       Was just thinking out loud. If it's not used then we remove it. If it is used, then we leave it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on pull request #529:
URL: https://github.com/apache/solr/pull/529#issuecomment-1022402464


   This PR was closed in favor of the new one[ SOLR-12336](https://github.com/apache/solr/pull/566) which is a continuation of this one (SOLR-15257). The PR was changed as a result of renaming the branch to correctly reference the parent issue SOLR-12336. Please refer to new PR [ SOLR-12336](https://github.com/apache/solr/pull/566


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793013536



##########
File path: solr/CHANGES.txt
##########
@@ -256,9 +256,8 @@ when told to. The admin UI now tells it to. (Nazerke Seidan, David Smiley)
 * SOLR-15209: The old LegacyAssignStrategy has been refactored into the SimplePlacementPlugin. This is still the default
   placement policy for Solr. (Houston Putman, Ilan Ginzburg)
 
-* SOLR-15257: Add DocSetQuery in lieu of DocSet.getTopFilter. This issue led to removing
-  Filter.java and eliminating all dependencies on its main method getDocIdSet. With Filter deleted ,
-  and replaced with Lucene's Query.java, the changes might spark some performance optimizations (Collins Abanda, Mike Drob, Tim Potter)
+* SOLR-12336: Remove Filter and SolrFilter, replaced by DocSetQuery which is a Query

Review comment:
       Instead, I'd prefer two JIRAs to reference in the CHANGES.txt.  One is DocSet.getTopFilter being replaced by DocSet.makeQuery.  This is one discrete change, even though you're choosing to do so in one commit/PR including other stuff.  The other is that Filter no longer has a need to exist due to multiple JIRA issues, implemented by a number of issues.  Perhaps reference nobody in particular there, or reference all of us at your discretion.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793194710



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+        docSet = null;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj));

Review comment:
       Thanks for catching this David! I have made some changes there to fix it




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r796137646



##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -96,6 +95,6 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
     DocSet docs = solrSearcher.getDocSet(q);
     // reqInfo.addCloseHook(docs);  // needed for off-heap refcounting
 
-    return new BoostQuery(new SolrConstantScoreQuery(docs.makeQuery()), 0).createWeight(searcher, scoreMode, 1f);
+    return new BoostQuery(q, 0).createWeight(searcher, scoreMode, 1f);

Review comment:
       This is still a work in progress. I am making some changes here




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799607786



##########
File path: solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
##########
@@ -229,19 +230,19 @@ public static void createJoinField(FacetRequest.Domain domain, Map<String,Object
        * Creates a Query that can be used to recompute the new "base" for this domain, relative to the
        * current base of the FacetContext.
        */
-      public Query createDomainQuery(FacetContext fcontext) throws IOException {
+      public Query createDomainQuery(FacetContext fcontext) {
         // NOTE: this code lives here, instead of in FacetProcessor.handleJoin, in order to minimize
         // the number of classes that have to know about the number of possible settings on the join
         // (ie: if we add a score mode, or some other modifier to how the joins are done)
 
-        final SolrConstantScoreQuery fromQuery = new SolrConstantScoreQuery(fcontext.base.getTopFilter());
+        final ConstantScoreQuery fromQuery = new ConstantScoreQuery(fcontext.base.makeQuery());

Review comment:
       Refactored it so `fromQuery` is not wrapped in a constant score query given we are using makeQuery which is a constant scoring query šŸ‘




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799808853



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -36,12 +36,11 @@
 
   final ValueSourceRangeFilter rangeFilt;
   boolean cache = true; // cache by default

Review comment:
       `cache` can now be removed I think?

##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -89,13 +87,13 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
 
     if (!(searcher instanceof SolrIndexSearcher)) {
       // delete-by-query won't have SolrIndexSearcher
-      return new BoostQuery(new ConstantScoreQuery(q), 0).createWeight(searcher, scoreMode, 1f);
+      return new ConstantScoreQuery(q).createWeight(searcher, scoreMode, 1f);

Review comment:
       It's fine to roll in SOLR-14800 scope as well; yeah let's do that.
   
   This line can be `return new ConstantScoreWeight(q, 1f);`

##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -89,13 +87,13 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
 
     if (!(searcher instanceof SolrIndexSearcher)) {
       // delete-by-query won't have SolrIndexSearcher
-      return new BoostQuery(new ConstantScoreQuery(q), 0).createWeight(searcher, scoreMode, 1f);
+      return new ConstantScoreQuery(q).createWeight(searcher, scoreMode, 1f);
     }
 
     SolrIndexSearcher solrSearcher = (SolrIndexSearcher)searcher;
     DocSet docs = solrSearcher.getDocSet(q);
     // reqInfo.addCloseHook(docs);  // needed for off-heap refcounting
 
-    return new BoostQuery(new SolrConstantScoreQuery(docs.getTopFilter()), 0).createWeight(searcher, scoreMode, 1f);
+    return docs.makeQuery().createWeight(searcher, scoreMode, 1f);

Review comment:
       The call to `solrSearcher.getDocSet(q)` is the point of all this -- which is cached.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793927255



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -18,40 +18,70 @@
 
 import java.io.IOException;
 import java.util.Map;
+import java.util.Objects;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.ValueSourceScorer;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Weight;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.search.function.ValueSourceRangeFilter;
 
 // This class works as either a normal constant score query, or as a PostFilter using a collector
-public class FunctionRangeQuery extends SolrConstantScoreQuery implements PostFilter {
+public class FunctionRangeQuery extends Query implements PostFilter {

Review comment:
       Oh ok, I will extend ExtendedQueryBase there, thanks for pointing that out! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r796128037



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+        docSet = null;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj));
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {

Review comment:
       Thanks, I made the changes in the javadoc noting searcher isn't being used 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
dsmiley commented on pull request #529:
URL: https://github.com/apache/solr/pull/529#issuecomment-1029347759


   You force pushed again.  Can you please avoid doing that?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799021241



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -36,12 +36,13 @@
 
   final ValueSourceRangeFilter rangeFilt;
   boolean cache = true; // cache by default
-  int cost;
+  //int cost;
 
   public FunctionRangeQuery(ValueSourceRangeFilter filter) {
     super();
     this.rangeFilt = filter;
-    this.cost = 100; // default behavior should be PostFiltering
+    super.setCost(100);
+    //this.cost = 100; // default behavior should be PostFiltering
   }

Review comment:
       Carelessly left this commented out code in. Will clean that out 

##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -36,12 +36,13 @@
 
   final ValueSourceRangeFilter rangeFilt;
   boolean cache = true; // cache by default
-  int cost;
+  //int cost;

Review comment:
       Carelessly left this commented out code in. Will clean that out




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799021241



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -36,12 +36,13 @@
 
   final ValueSourceRangeFilter rangeFilt;
   boolean cache = true; // cache by default
-  int cost;
+  //int cost;
 
   public FunctionRangeQuery(ValueSourceRangeFilter filter) {
     super();
     this.rangeFilt = filter;
-    this.cost = 100; // default behavior should be PostFiltering
+    super.setCost(100);
+    //this.cost = 100; // default behavior should be PostFiltering
   }

Review comment:
       Forgot this commented out code in. Will clean that out 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787177884



##########
File path: solr/core/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java
##########
@@ -76,33 +78,50 @@ public boolean isIncludeUpper() {
     return includeUpper;
   }
 
+//  @Override
+//  public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
+//    return null;
+//  }

Review comment:
       Yes, I'll clean that up 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789207336



##########
File path: solr/core/src/test/org/apache/solr/search/TestFilteredDocIdSet.java
##########
@@ -127,22 +130,49 @@ public void testNullDocIdSet() throws Exception {
     IndexSearcher searcher = newSearcher(reader);
     Assert.assertEquals(1, searcher.search(new MatchAllDocsQuery(), 10).totalHits.value);
     
-    // Now search w/ a Filter which returns a null DocIdSet
-    Filter f = new Filter() {
+    // Now search w/ a Query which returns a null DocIdSet
+    Query f = new Query() {

Review comment:
       Thanks for the catch. I just had that corrected! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787223054



##########
File path: solr/core/src/java/org/apache/solr/search/DocSet.java
##########
@@ -121,7 +121,7 @@ public int andNotSize(DocSet other) {
    * was generated from the top-level MultiReader that the Lucene search
    * methods will be invoked with.
    */
-  public abstract Filter getTopFilter();
+  public abstract DocSetQuery makeQuery();

Review comment:
       No, not at all. Thanks for bringing it up. I agree that it's indeed best for it to simply return a `Query` rather than a `DocSetQuery`. I have made the correction. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r790010190



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{
+    DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj).docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
+        return null;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+        return new ConstantScoreWeight(this, boost) {

Review comment:
       Ok, I'll make the change to be a score of 0




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789999177



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{

Review comment:
       Given that we are transitioning away from `SolrConstantScoreQuery`, I will correct that to extend `Query.java` instead. Thanks for alerting me to that!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793259272



##########
File path: solr/CHANGES.txt
##########
@@ -256,9 +256,8 @@ when told to. The admin UI now tells it to. (Nazerke Seidan, David Smiley)
 * SOLR-15209: The old LegacyAssignStrategy has been refactored into the SimplePlacementPlugin. This is still the default
   placement policy for Solr. (Houston Putman, Ilan Ginzburg)
 
-* SOLR-15257: Add DocSetQuery in lieu of DocSet.getTopFilter. This issue led to removing
-  Filter.java and eliminating all dependencies on its main method getDocIdSet. With Filter deleted ,
-  and replaced with Lucene's Query.java, the changes might spark some performance optimizations (Collins Abanda, Mike Drob, Tim Potter)
+* SOLR-12336: Remove Filter and SolrFilter, replaced by DocSetQuery which is a Query

Review comment:
       Ok, that makes sense. I made the edit referencing two separate JIRAs: SOLR-15257 and SOLR-12336 šŸ‘




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793946283



##########
File path: solr/CHANGES.txt
##########
@@ -256,9 +256,8 @@ when told to. The admin UI now tells it to. (Nazerke Seidan, David Smiley)
 * SOLR-15209: The old LegacyAssignStrategy has been refactored into the SimplePlacementPlugin. This is still the default
   placement policy for Solr. (Houston Putman, Ilan Ginzburg)
 
-* SOLR-15257: Add DocSetQuery in lieu of DocSet.getTopFilter. This issue led to removing
-  Filter.java and eliminating all dependencies on its main method getDocIdSet. With Filter deleted ,
-  and replaced with Lucene's Query.java, the changes might spark some performance optimizations (Collins Abanda, Mike Drob, Tim Potter)
+* SOLR-12336: Remove Filter and SolrFilter, replaced by DocSetQuery which is a Query

Review comment:
       I don't think the summary of SOLR-12336 (removal of Filter) shouldn't reference DocSetQuery; that had its own issue (SOLR-15257).  Also, the cast of characters, if you look at the JIRAs referenced from this umbrella issue, included Michael Gibney (SOLR-14185).  I suppose in this one you might mention SolrConstantScoreQuery's removal too.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793016743



##########
File path: solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
##########
@@ -41,17 +41,20 @@
  * Experimental and subject to change.
  */
 public class SolrConstantScoreQuery extends Query implements ExtendedQuery {
-  private final Filter filter;

Review comment:
       Justification:  SolrConstantScoreQuery's purpose *was* to bridge the Filter API to a Query API.  It became instant tech-debt the moment Filter subclassed Query (years ago).  Now with Filter going away, SCSQ has even less reason to exist -- it's basically a Lucene ConstantScoreQuery + Solr ExtendedQuery which we don't need.  Anyone who wants such a thing can simply do `new WrappedQuery(someOtherQuery)`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793259272



##########
File path: solr/CHANGES.txt
##########
@@ -256,9 +256,8 @@ when told to. The admin UI now tells it to. (Nazerke Seidan, David Smiley)
 * SOLR-15209: The old LegacyAssignStrategy has been refactored into the SimplePlacementPlugin. This is still the default
   placement policy for Solr. (Houston Putman, Ilan Ginzburg)
 
-* SOLR-15257: Add DocSetQuery in lieu of DocSet.getTopFilter. This issue led to removing
-  Filter.java and eliminating all dependencies on its main method getDocIdSet. With Filter deleted ,
-  and replaced with Lucene's Query.java, the changes might spark some performance optimizations (Collins Abanda, Mike Drob, Tim Potter)
+* SOLR-12336: Remove Filter and SolrFilter, replaced by DocSetQuery which is a Query

Review comment:
       Ok, made the edit referencing two separate JIRAs: SOLR-15257 and SOLR-12336 šŸ‘




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789999177



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{

Review comment:
       Given that we are transitioning away from `SolrConstantScoreQuery`, I will correct that to extend `Query.java` instead. Thanks! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789991711



##########
File path: solr/contrib/analytics/src/java/org/apache/solr/analytics/facet/RangeFacet.java
##########
@@ -54,7 +53,7 @@ public RangeFacet(String name, SchemaField field, String start, String end, List
   }
 
   @Override
-  public void createFacetValueExecuters(final Filter filter, SolrQueryRequest queryRequest, Consumer<FacetValueQueryExecuter> consumer) {
+  public void createFacetValueExecuters(final Query query, SolrQueryRequest queryRequest, Consumer<FacetValueQueryExecuter> consumer) {

Review comment:
       Thanks, missed it in the initial revision. Fixed it now! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789811761



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -42,7 +43,12 @@ public FunctionRangeQuery(ValueSourceRangeFilter filter) {
   @Override
   public DelegatingCollector getFilterCollector(IndexSearcher searcher) {
     Map<Object,Object> fcontext = ValueSource.newContext(searcher);
-    Weight weight = rangeFilt.createWeight(searcher, ScoreMode.COMPLETE, 1);
+    Weight weight = null;
+    try {
+      weight = rangeFilt.createWeight(searcher, ScoreMode.COMPLETE, 1);
+    } catch (IOException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+    }

Review comment:
       Thanks for catching this, fixed!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r790001455



##########
File path: solr/core/src/test/org/apache/solr/search/TestDocSet.java
##########
@@ -556,41 +558,55 @@ public void doFilterTest(IndexReader reader) throws IOException {
     doTestIteratorEqual(da, db);
     ***/
 
-    DocIdSet da;
-    DocIdSet db;
     List<LeafReaderContext> leaves = topLevelContext.leaves();
-
     // first test in-sequence sub readers
     for (LeafReaderContext readerContext : leaves) {
-      da = fa.getDocIdSet(readerContext, null);
-      db = fb.getDocIdSet(readerContext, null);
-
       // there are various ways that disis can be retrieved for each leafReader; they should all be equivalent.
-      doTestIteratorEqual(da.bits(), disiSupplier(da), disiSupplier(db), () -> a.iterator(readerContext), () -> b.iterator(readerContext));
-
+         doTestIteratorEqual(getExpectedBits(a, readerContext), () -> a.iterator(readerContext), () -> b.iterator(readerContext));
       // set b is SortedIntDocSet, so derivatives should not support random-access via Bits

Review comment:
       Good catch, it's now deleted! Thanks šŸ‘




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r790007449



##########
File path: solr/core/src/test/org/apache/solr/search/TestFilteredDocIdSet.java
##########
@@ -127,22 +130,42 @@ public void testNullDocIdSet() throws Exception {
     IndexSearcher searcher = newSearcher(reader);
     Assert.assertEquals(1, searcher.search(new MatchAllDocsQuery(), 10).totalHits.value);
     
-    // Now search w/ a Filter which returns a null DocIdSet
-    Filter f = new Filter() {
+    // Now search w/ a Query which returns a null DocIdSet

Review comment:
       Thanks, will change that 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 closed pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 closed pull request #529:
URL: https://github.com/apache/solr/pull/529


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 edited a comment on pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 edited a comment on pull request #529:
URL: https://github.com/apache/solr/pull/529#issuecomment-1022402464


   This PR was closed in favor of the new one https://github.com/apache/solr/pull/566 which is a continuation of this one. The PR was closed as a result of renaming the branch to correctly reference the parent issue SOLR-12336. Please refer to new PR https://github.com/apache/solr/pull/566


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r794843972



##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -96,6 +95,6 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
     DocSet docs = solrSearcher.getDocSet(q);
     // reqInfo.addCloseHook(docs);  // needed for off-heap refcounting
 
-    return new BoostQuery(new SolrConstantScoreQuery(docs.makeQuery()), 0).createWeight(searcher, scoreMode, 1f);
+    return new BoostQuery(new ConstantScoreQuery(docs.makeQuery()), 0).createWeight(searcher, scoreMode, 1f);

Review comment:
       Thanks for pointing that out. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r794843972



##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -96,6 +95,6 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
     DocSet docs = solrSearcher.getDocSet(q);
     // reqInfo.addCloseHook(docs);  // needed for off-heap refcounting
 
-    return new BoostQuery(new SolrConstantScoreQuery(docs.makeQuery()), 0).createWeight(searcher, scoreMode, 1f);
+    return new BoostQuery(new ConstantScoreQuery(docs.makeQuery()), 0).createWeight(searcher, scoreMode, 1f);

Review comment:
       Thanks for the correction here! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789901078



##########
File path: solr/core/src/java/org/apache/solr/search/DocSet.java
##########
@@ -117,11 +118,11 @@ public int andNotSize(DocSet other) {
   }
 
   /**
-   * Returns a Filter for use in Lucene search methods, assuming this DocSet
+   * Returns a Query for use in Lucene search methods, assuming this DocSet

Review comment:
       Say it runs a constant scoring Query in particular.  And I'd further remind the reader that DocSets and thus this query do not match deleted docs.

##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{
+    DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj).docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
+        return null;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+        return new ConstantScoreWeight(this, boost) {
+            @Override
+            public Scorer scorer(LeafReaderContext context) throws IOException {
+            DocIdSetIterator disi = docSet.iterator(context);
+            if (disi == null) {
+                return null;
+            }
+            return new ConstantScoreScorer(this, score(), scoreMode, disi);
+            }
+
+            @Override
+            public boolean isCacheable(LeafReaderContext ctx) {
+                return false;

Review comment:
       This is Cacheable in fact.  This is supposed to be weights on DocValues fields that might have had realtime updates.

##########
File path: solr/contrib/analytics/src/java/org/apache/solr/analytics/AnalyticsGroupingManager.java
##########
@@ -98,16 +98,15 @@ public boolean getStreamingFacets(Consumer<StreamingFacet> cons) {
    * One {@link FacetValueQueryExecuter} is created for each facet value to be returned for a facet.
    * Since every {@link AbstractSolrQueryFacet} has discrete and user-defined facet values,
    * unlike {@link StreamingFacet}s, a discrete number of {@link FacetValueQueryExecuter}s are created and returned.
-   *
-   * @param filter representing the overall Solr Query of the request,
+   * @param query representing the overall Solr Query of the request,
    * will be combined with the facet value queries
    * @param queryRequest from the overall search request
    * @param cons where the executers are passed to
    */
-  public void getFacetExecuters(Filter filter, SolrQueryRequest queryRequest, Consumer<FacetValueQueryExecuter> cons) {
+  public void getFacetExecuters(Query query, SolrQueryRequest queryRequest, Consumer<FacetValueQueryExecuter> cons) {

Review comment:
       again; keep named "filter"

##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{
+    DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj).docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
+        return null;

Review comment:
       uh?  I suspect this is your IDE default, which IMO is a bad default.  FWIW I changed my IDE default here to be a //nocommit.  

##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{
+    DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj).docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
+        return null;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+        return new ConstantScoreWeight(this, boost) {

Review comment:
       The scores of a Filter used to be 0, so in this PR lets just use 0.  In a separate PR for JIRA SOLR-14800 I'll change it

##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{

Review comment:
       Why does this extend SolrConstantScoreQuery?

##########
File path: solr/core/src/test/org/apache/solr/search/TestFilteredDocIdSet.java
##########
@@ -127,22 +130,42 @@ public void testNullDocIdSet() throws Exception {
     IndexSearcher searcher = newSearcher(reader);
     Assert.assertEquals(1, searcher.search(new MatchAllDocsQuery(), 10).totalHits.value);
     
-    // Now search w/ a Filter which returns a null DocIdSet
-    Filter f = new Filter() {
+    // Now search w/ a Query which returns a null DocIdSet

Review comment:
       Query doesn't return DocIdSet.  You changed it to return a null Scorer which is close enough to the original intent.

##########
File path: solr/core/src/test/org/apache/solr/search/TestFilteredDocIdSet.java
##########
@@ -113,7 +116,7 @@ protected boolean match(int docid) {
   }
   
   public void testNullDocIdSet() throws Exception {
-    // Tests that if a Filter produces a null DocIdSet, which is given to
+    // Tests that if a Query produces a null DocIdSet, which is given to

Review comment:
       Filters produces a DocIdSet; Query does not.  You could try prefixing the comment with "(historical note)" or something

##########
File path: solr/core/src/test/org/apache/solr/search/TestFilteredDocIdSet.java
##########
@@ -171,25 +194,25 @@ public void testNullIteratorFilteredDocIdSet() throws Exception {
     IndexSearcher searcher = newSearcher(reader);
     Assert.assertEquals(1, searcher.search(new MatchAllDocsQuery(), 10).totalHits.value);
     
-      // Now search w/ a Filter which returns a null DocIdSet
-    Filter f = new Filter() {
+    // Now search w/ a Query which returns a null DocIdSet

Review comment:
       no; a null *Scorer*

##########
File path: solr/core/src/test/org/apache/solr/search/TestDocSet.java
##########
@@ -556,41 +558,55 @@ public void doFilterTest(IndexReader reader) throws IOException {
     doTestIteratorEqual(da, db);
     ***/
 
-    DocIdSet da;
-    DocIdSet db;
     List<LeafReaderContext> leaves = topLevelContext.leaves();
-
     // first test in-sequence sub readers
     for (LeafReaderContext readerContext : leaves) {
-      da = fa.getDocIdSet(readerContext, null);
-      db = fb.getDocIdSet(readerContext, null);
-
       // there are various ways that disis can be retrieved for each leafReader; they should all be equivalent.
-      doTestIteratorEqual(da.bits(), disiSupplier(da), disiSupplier(db), () -> a.iterator(readerContext), () -> b.iterator(readerContext));
-
+         doTestIteratorEqual(getExpectedBits(a, readerContext), () -> a.iterator(readerContext), () -> b.iterator(readerContext));
       // set b is SortedIntDocSet, so derivatives should not support random-access via Bits

Review comment:
       this comment is now obsolete once you removed the line that followed

##########
File path: solr/core/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java
##########
@@ -35,7 +37,7 @@
 /**
  * RangeFilter over a ValueSource.
  */
-public class ValueSourceRangeFilter extends SolrFilter {

Review comment:
       This will certainly change *then*.  This issue is about DocSet.getTopFilter.  VSRF doesn't use that nor the reverse.

##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{
+    DocSet docSet;

Review comment:
       private final

##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{

Review comment:
       Add javadocs!  Refer to who created this.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789897662



##########
File path: solr/contrib/analytics/src/java/org/apache/solr/analytics/facet/AbstractSolrQueryFacet.java
##########
@@ -50,11 +49,11 @@ protected AbstractSolrQueryFacet(String name) {
    *
    * Each of these executors will be executed after the streaming phase in the {@link AnalyticsDriver}.
    *
-   * @param filter the overall filter representing the documents being used for the analytics request

Review comment:
       Looks good to me as you have it; thanks.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789897875



##########
File path: solr/contrib/analytics/src/java/org/apache/solr/analytics/facet/RangeFacet.java
##########
@@ -54,7 +53,7 @@ public RangeFacet(String name, SchemaField field, String start, String end, List
   }
 
   @Override
-  public void createFacetValueExecuters(final Filter filter, SolrQueryRequest queryRequest, Consumer<FacetValueQueryExecuter> consumer) {
+  public void createFacetValueExecuters(final Query query, SolrQueryRequest queryRequest, Consumer<FacetValueQueryExecuter> consumer) {

Review comment:
       TODO still




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r790195268



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{
+    DocSet docSet;

Review comment:
       Made the change




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789214298



##########
File path: solr/contrib/analytics/src/java/org/apache/solr/analytics/facet/AbstractSolrQueryFacet.java
##########
@@ -50,11 +49,11 @@ protected AbstractSolrQueryFacet(String name) {
    *
    * Each of these executors will be executed after the streaming phase in the {@link AnalyticsDriver}.
    *
-   * @param filter the overall filter representing the documents being used for the analytics request

Review comment:
       Agreed! 
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789791743



##########
File path: solr/CHANGES.txt
##########
@@ -72,6 +72,10 @@ New Features
 
 Improvements
 ----------------------
+* SOLR-15257: Add DocSetQuery in lieu of DocSet.getTopFilter. This issue led to removing

Review comment:
       We typically add changes entires to the end of the section rather than the beginning.

##########
File path: solr/core/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java
##########
@@ -35,7 +37,7 @@
 /**
  * RangeFilter over a ValueSource.
  */
-public class ValueSourceRangeFilter extends SolrFilter {

Review comment:
       @dsmiley - It kind of is though, if Filter and SolrFilter go away then this has to change?

##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -42,7 +43,12 @@ public FunctionRangeQuery(ValueSourceRangeFilter filter) {
   @Override
   public DelegatingCollector getFilterCollector(IndexSearcher searcher) {
     Map<Object,Object> fcontext = ValueSource.newContext(searcher);
-    Weight weight = rangeFilt.createWeight(searcher, ScoreMode.COMPLETE, 1);
+    Weight weight = null;
+    try {
+      weight = rangeFilt.createWeight(searcher, ScoreMode.COMPLETE, 1);
+    } catch (IOException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+    }

Review comment:
       nit: indentation here seems inconsistent.

##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{
+    DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj).docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
+        return null;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+        return new ConstantScoreWeight(this, boost) {
+            @Override
+            public Scorer scorer(LeafReaderContext context) throws IOException {
+            DocIdSetIterator disi = docSet.iterator(context);
+            if (disi == null) {
+                return null;
+            }
+            return new ConstantScoreScorer(this, score(), scoreMode, disi);

Review comment:
       nit: indentation here is off.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on pull request #529:
URL: https://github.com/apache/solr/pull/529#issuecomment-1022539143


   > This PR was closed in favor of the new one #566 which is a continuation of this one. The PR was closed as a result of renaming the branch to correctly reference the parent issue [SOLR-12336](https://issues.apache.org/jira/browse/SOLR-12336). Please refer to the [new PR here](https://github.com/apache/solr/pull/566)
   
   This comment is no longer valid because I restored the branch and reopened the PR to preserve the conversation. I simply changed the PR title instead. The new PR has now been deleted 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r791040364



##########
File path: solr/core/src/java/org/apache/solr/search/BitDocSet.java
##########
@@ -311,72 +307,8 @@ public long cost() {
     };
   }
 
-  @Override
-  public Filter getTopFilter() {
-    // TODO: if cardinality isn't cached, do a quick measure of sparseness
-    // and return null from bits() if too sparse.
-
-    return new Filter() {
-      final FixedBitSet bs = bits;
-
-      @Override
-      public DocIdSet getDocIdSet(final LeafReaderContext context, final Bits acceptDocs) {
-        // all Solr DocSets that are used as filters only include live docs
-        final Bits acceptDocs2 = acceptDocs == null ? null : (context.reader().getLiveDocs() == acceptDocs ? null : acceptDocs);
-
-        return BitsFilteredDocIdSet.wrap(new DocIdSet() {
-          @Override
-          public DocIdSetIterator iterator() {
-            return BitDocSet.this.iterator(context);
-          }
-
-          @Override
-          public long ramBytesUsed() {
-            return BitDocSet.this.ramBytesUsed();
-          }
-
-          @Override
-          public Bits bits() {
-            if (context.isTopLevel) {
-              return bits;
-            }
-
-            final int base = context.docBase;
-            final int length = context.reader().maxDoc();
-            final FixedBitSet bs = bits;
-
-            return new Bits() {
-              @Override
-              public boolean get(int index) {
-                return bs.get(index + base);
-              }
-
-              @Override
-              public int length() {
-                return length;
-              }
-            };
-          }
-
-        }, acceptDocs2);
-      }
-
-      @Override
-      public String toString(String field) {
-        return "BitSetDocTopFilter";
-      }
-
-      @Override
-      public boolean equals(Object other) {
-        return sameClassAs(other) &&
-               Objects.equals(bs, getClass().cast(other).bs);
-      }
-      
-      @Override
-      public int hashCode() {
-        return classHash() * 31 + bs.hashCode();
-      }
-    };
+  public DocSetQuery makeQuery() {
+    return new DocSetQuery(this);

Review comment:
       If all implementations return null, then can we get rid of BitDocSet? Follow-on issue?

##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+        docSet = null;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj));
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {

Review comment:
       Javadoc indicating that the searcher will not be used.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on pull request #529:
URL: https://github.com/apache/solr/pull/529#issuecomment-1020417262


   I rebased and force pushed the latest commit to due to recent changes in `CHANGES.txt` renaming `contrib` to `module`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r790970401



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -26,19 +26,29 @@
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.solr.util.TestInjection;
 
 import java.io.IOException;
 import java.util.Objects;
 
+/**
+ * A class that accesses Queries based on a DocSet
+ * This class was created as a replacement to DocSet.getTopFilter
+ * which has since been removed with Filter

Review comment:
       That's true, thanks for the correction. I deleted it




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r796127161



##########
File path: solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
##########
@@ -234,9 +234,8 @@ public Query createDomainQuery(FacetContext fcontext) throws IOException {
         // the number of classes that have to know about the number of possible settings on the join
         // (ie: if we add a score mode, or some other modifier to how the joins are done)
 
-        final SolrConstantScoreQuery fromQuery = new SolrConstantScoreQuery(fcontext.base.makeQuery());
+        final ConstantScoreQuery fromQuery = new ConstantScoreQuery(fcontext.base.makeQuery());
         // this shouldn't matter once we're wrapped in a join query, but just in case it ever does...
-        fromQuery.setCache(false);

Review comment:
       Thanks for the explanation! I found it really helpful




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r798927117



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -25,18 +25,16 @@
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.ValueSourceScorer;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Weight;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.search.function.ValueSourceRangeFilter;
 
 // This class works as either a normal constant score query, or as a PostFilter using a collector
-public class FunctionRangeQuery extends Query implements PostFilter {
+public class FunctionRangeQuery extends ExtendedQueryBase implements PostFilter {
 
   final ValueSourceRangeFilter rangeFilt;
-  boolean cache = true; // cache by default
   int cost;

Review comment:
       That's weird.  Do you understand why the distinction of extending ExtendedQueryBase vis implementing ExtendedQuery caused that test to fail?  I could help look if you can't figure it out.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r798930218



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -25,18 +25,16 @@
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.ValueSourceScorer;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Weight;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.search.function.ValueSourceRangeFilter;
 
 // This class works as either a normal constant score query, or as a PostFilter using a collector
-public class FunctionRangeQuery extends Query implements PostFilter {
+public class FunctionRangeQuery extends ExtendedQueryBase implements PostFilter {
 
   final ValueSourceRangeFilter rangeFilt;
-  boolean cache = true; // cache by default
   int cost;

Review comment:
       Right now I'm really not sure why that is 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793909859



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -18,40 +18,70 @@
 
 import java.io.IOException;
 import java.util.Map;
+import java.util.Objects;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.ValueSourceScorer;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Weight;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.search.function.ValueSourceRangeFilter;
 
 // This class works as either a normal constant score query, or as a PostFilter using a collector
-public class FunctionRangeQuery extends SolrConstantScoreQuery implements PostFilter {
+public class FunctionRangeQuery extends Query implements PostFilter {

Review comment:
       If we extend ExtendedQueryBase then I think some of these implementations come for free.

##########
File path: solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
##########
@@ -234,9 +234,8 @@ public Query createDomainQuery(FacetContext fcontext) throws IOException {
         // the number of classes that have to know about the number of possible settings on the join
         // (ie: if we add a score mode, or some other modifier to how the joins are done)
 
-        final SolrConstantScoreQuery fromQuery = new SolrConstantScoreQuery(fcontext.base.makeQuery());
+        final ConstantScoreQuery fromQuery = new ConstantScoreQuery(fcontext.base.makeQuery());
         // this shouldn't matter once we're wrapped in a join query, but just in case it ever does...
-        fromQuery.setCache(false);

Review comment:
       Why did this go away?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r794843334



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -25,18 +25,16 @@
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.ValueSourceScorer;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Weight;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.search.function.ValueSourceRangeFilter;
 
 // This class works as either a normal constant score query, or as a PostFilter using a collector
-public class FunctionRangeQuery extends Query implements PostFilter {
+public class FunctionRangeQuery extends ExtendedQueryBase implements PostFilter {
 
   final ValueSourceRangeFilter rangeFilt;
-  boolean cache = true; // cache by default
   int cost;

Review comment:
       No we don't need that anymore since we are extending ExtendedQueryBase. Thanks for highlighting this. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787145033



##########
File path: solr/core/src/java/org/apache/solr/search/DocSet.java
##########
@@ -121,7 +121,7 @@ public int andNotSize(DocSet other) {
    * was generated from the top-level MultiReader that the Lucene search
    * methods will be invoked with.
    */
-  public abstract Filter getTopFilter();
+  public abstract DocSetQuery makeQuery();

Review comment:
       Does this need to return the specialized `DocSetQuery` or could it simply be `Query`?

##########
File path: solr/core/src/java/org/apache/solr/search/PostFilter.java
##########
@@ -18,6 +18,7 @@
 
 import org.apache.lucene.search.IndexSearcher;
 
+

Review comment:
       dangling whitespace change

##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery {
+    protected final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSet getDocSet() {
+        return docSet;
+    }

Review comment:
       Unused?

##########
File path: solr/core/src/java/org/apache/solr/search/BitsFilteredDocIdSet.java
##########
@@ -23,15 +23,11 @@
 
 /**
  * This implementation supplies a filtered DocIdSet, that excludes all
- * docids which are not in a Bits instance. This is especially useful in
- * {@link org.apache.solr.search.Filter} to apply the {@code acceptDocs}
- * passed to {@code getDocIdSet()} before returning the final DocIdSet.
+ * docids which are not in a Bits instance.
  *
  * @see DocIdSet
- * @see org.apache.solr.search.Filter
  */
 public final class BitsFilteredDocIdSet extends FilteredDocIdSet {

Review comment:
       Since we got rid of the Filter in BitDocSet, I don't think this class is used anywhere anymore and can be deleted.

##########
File path: solr/core/src/test/org/apache/solr/search/TestFilteredDocIdSet.java
##########
@@ -127,22 +130,49 @@ public void testNullDocIdSet() throws Exception {
     IndexSearcher searcher = newSearcher(reader);
     Assert.assertEquals(1, searcher.search(new MatchAllDocsQuery(), 10).totalHits.value);
     
-    // Now search w/ a Filter which returns a null DocIdSet
-    Filter f = new Filter() {
+    // Now search w/ a Query which returns a null DocIdSet
+    Query f = new Query() {
       @Override
-      public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) {
-        return null;
+      public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) {
+        return new Weight(this) {
+
+          @Override
+          public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+            final Scorer scorer = scorer(context);
+            final boolean match = (scorer != null && scorer.iterator().advance(doc) == doc);
+            if (match) {
+              assert scorer.score() == 0f;
+              return Explanation.match(0f, "Match on id " + doc);

Review comment:
       This code will never be called, should simplify our implementation of `explain` based on knowing that `scorer` always returns `null`. This same comment applies later in the same file and also in TestSort.

##########
File path: solr/core/src/test/org/apache/solr/search/TestDocSet.java
##########
@@ -533,21 +535,21 @@ private void doTestIteratorEqual(Bits bits, Supplier<DocIdSetIterator>... disiSu
     }
   }
 
+
   /**
    * Tests equivalence among {@link DocIdSetIterator} instances retrieved from {@link BitDocSet} and {@link SortedIntDocSet}
-   * implementations, via {@link DocSet#getTopFilter()}/{@link Filter#getDocIdSet(LeafReaderContext, Bits)} and directly
-   * via {@link DocSet#iterator(LeafReaderContext)}.
-   * Also tests corresponding random-access {@link Bits} instances retrieved via {@link DocSet#getTopFilter()}/
-   * {@link Filter#getDocIdSet(LeafReaderContext, Bits)}/{@link DocIdSet#bits()}.
+   * implementations, via {@link DocSet#makeQuery()} and directly via {@link DocSet#iterator(LeafReaderContext)}.
+   * Also tests corresponding random-access {@link Bits} instances retrieved via {@link DocSet#makeQuery()}/
+   * {@link DocIdSet#bits()}.
    */
   public void doFilterTest(IndexReader reader) throws IOException {
     IndexReaderContext topLevelContext = reader.getContext();
     FixedBitSet bs = getRandomSet(reader.maxDoc(), rand.nextInt(reader.maxDoc()+1));
     DocSet a = new BitDocSet(bs);
     DocSet b = getIntDocSet(bs);
 
-    Filter fa = a.getTopFilter();
-    Filter fb = b.getTopFilter();
+//    Query fa = a.makeQuery();
+//    Query fb = b.makeQuery();

Review comment:
       This feels like we're now testing less than we were before. Can you explain why your patch removes testing without adding in equivalent functionality?

##########
File path: solr/core/src/java/org/apache/solr/search/BitDocSet.java
##########
@@ -311,72 +307,8 @@ public long cost() {
     };
   }
 
-  @Override
-  public Filter getTopFilter() {
-    // TODO: if cardinality isn't cached, do a quick measure of sparseness
-    // and return null from bits() if too sparse.
-
-    return new Filter() {
-      final FixedBitSet bs = bits;
-
-      @Override
-      public DocIdSet getDocIdSet(final LeafReaderContext context, final Bits acceptDocs) {
-        // all Solr DocSets that are used as filters only include live docs
-        final Bits acceptDocs2 = acceptDocs == null ? null : (context.reader().getLiveDocs() == acceptDocs ? null : acceptDocs);
-
-        return BitsFilteredDocIdSet.wrap(new DocIdSet() {
-          @Override
-          public DocIdSetIterator iterator() {
-            return BitDocSet.this.iterator(context);
-          }
-
-          @Override
-          public long ramBytesUsed() {
-            return BitDocSet.this.ramBytesUsed();
-          }
-
-          @Override
-          public Bits bits() {
-            if (context.isTopLevel) {
-              return bits;
-            }
-
-            final int base = context.docBase;
-            final int length = context.reader().maxDoc();
-            final FixedBitSet bs = bits;
-
-            return new Bits() {
-              @Override
-              public boolean get(int index) {
-                return bs.get(index + base);
-              }
-
-              @Override
-              public int length() {
-                return length;
-              }
-            };
-          }
-
-        }, acceptDocs2);
-      }
-
-      @Override
-      public String toString(String field) {
-        return "BitSetDocTopFilter";
-      }
-
-      @Override
-      public boolean equals(Object other) {
-        return sameClassAs(other) &&
-               Objects.equals(bs, getClass().cast(other).bs);
-      }
-      
-      @Override
-      public int hashCode() {
-        return classHash() * 31 + bs.hashCode();
-      }
-    };
+  public DocSetQuery makeQuery() {
+    return new DocSetQuery(this);

Review comment:
       Should we be concerned that DocSetQuery doesn't implement an equivalent to `bits()`?

##########
File path: solr/core/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java
##########
@@ -76,33 +78,50 @@ public boolean isIncludeUpper() {
     return includeUpper;
   }
 
+//  @Override
+//  public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
+//    return null;
+//  }

Review comment:
       This should be deleted.

##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery {
+    protected final DocSet docSet;

Review comment:
       Why not private?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799609711



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && equalsTo(getClass().cast(obj));
+    }
+
+    private boolean equalsTo(DocSetQuery other) {
+        return Objects.equals(docSet, other.docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    /**
+     * @param searcher is not used because we already have a DocSet created in DocSetQuery
+     * @return the DocSet created in DocSetQuery
+     */
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
+        return docSet;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+        return new ConstantScoreWeight(this, 0) {

Review comment:
       It should be the provided boost. Thanks for pointing this out! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799890106



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -36,12 +36,11 @@
 
   final ValueSourceRangeFilter rangeFilt;
   boolean cache = true; // cache by default

Review comment:
       Yes, I cleaned it up, thanks! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789811443



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{
+    DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj).docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
+        return null;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+        return new ConstantScoreWeight(this, boost) {
+            @Override
+            public Scorer scorer(LeafReaderContext context) throws IOException {
+            DocIdSetIterator disi = docSet.iterator(context);
+            if (disi == null) {
+                return null;
+            }
+            return new ConstantScoreScorer(this, score(), scoreMode, disi);

Review comment:
       Thanks for catching this, fixed! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r791125848



##########
File path: solr/core/src/java/org/apache/solr/search/BitDocSet.java
##########
@@ -311,72 +307,8 @@ public long cost() {
     };
   }
 
-  @Override
-  public Filter getTopFilter() {
-    // TODO: if cardinality isn't cached, do a quick measure of sparseness
-    // and return null from bits() if too sparse.
-
-    return new Filter() {
-      final FixedBitSet bs = bits;
-
-      @Override
-      public DocIdSet getDocIdSet(final LeafReaderContext context, final Bits acceptDocs) {
-        // all Solr DocSets that are used as filters only include live docs
-        final Bits acceptDocs2 = acceptDocs == null ? null : (context.reader().getLiveDocs() == acceptDocs ? null : acceptDocs);
-
-        return BitsFilteredDocIdSet.wrap(new DocIdSet() {
-          @Override
-          public DocIdSetIterator iterator() {
-            return BitDocSet.this.iterator(context);
-          }
-
-          @Override
-          public long ramBytesUsed() {
-            return BitDocSet.this.ramBytesUsed();
-          }
-
-          @Override
-          public Bits bits() {
-            if (context.isTopLevel) {
-              return bits;
-            }
-
-            final int base = context.docBase;
-            final int length = context.reader().maxDoc();
-            final FixedBitSet bs = bits;
-
-            return new Bits() {
-              @Override
-              public boolean get(int index) {
-                return bs.get(index + base);
-              }
-
-              @Override
-              public int length() {
-                return length;
-              }
-            };
-          }
-
-        }, acceptDocs2);
-      }
-
-      @Override
-      public String toString(String field) {
-        return "BitSetDocTopFilter";
-      }
-
-      @Override
-      public boolean equals(Object other) {
-        return sameClassAs(other) &&
-               Objects.equals(bs, getClass().cast(other).bs);
-      }
-      
-      @Override
-      public int hashCode() {
-        return classHash() * 31 + bs.hashCode();
-      }
-    };
+  public DocSetQuery makeQuery() {
+    return new DocSetQuery(this);

Review comment:
       I don't follow; why would we remove BitDocSet?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r790195317



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{
+    DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj).docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
+        return null;

Review comment:
       Thanks for pointing this out. I'll add in a //nocommit




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r790195260



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{

Review comment:
       Done! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
dsmiley commented on pull request #529:
URL: https://github.com/apache/solr/pull/529#issuecomment-1016096271


   Also, I think DocSetQuery ought to implement DocSetProducer so that DocSetUtil.createDocSet will see this.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789999177



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery implements DocSetProducer{

Review comment:
       Given that we are transitioning away from `SolrConstantScoreQuery`, I will correct that to extend `Query.java` instead. Thanks for alerting me to this!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r790001455



##########
File path: solr/core/src/test/org/apache/solr/search/TestDocSet.java
##########
@@ -556,41 +558,55 @@ public void doFilterTest(IndexReader reader) throws IOException {
     doTestIteratorEqual(da, db);
     ***/
 
-    DocIdSet da;
-    DocIdSet db;
     List<LeafReaderContext> leaves = topLevelContext.leaves();
-
     // first test in-sequence sub readers
     for (LeafReaderContext readerContext : leaves) {
-      da = fa.getDocIdSet(readerContext, null);
-      db = fb.getDocIdSet(readerContext, null);
-
       // there are various ways that disis can be retrieved for each leafReader; they should all be equivalent.
-      doTestIteratorEqual(da.bits(), disiSupplier(da), disiSupplier(db), () -> a.iterator(readerContext), () -> b.iterator(readerContext));
-
+         doTestIteratorEqual(getExpectedBits(a, readerContext), () -> a.iterator(readerContext), () -> b.iterator(readerContext));
       // set b is SortedIntDocSet, so derivatives should not support random-access via Bits

Review comment:
       Good catch! Thanks šŸ‘




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793014491



##########
File path: solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
##########
@@ -41,17 +41,20 @@
  * Experimental and subject to change.
  */
 public class SolrConstantScoreQuery extends Query implements ExtendedQuery {
-  private final Filter filter;

Review comment:
       Since this issue is now bigger in scope, it should remove SolrConstantScoreQuery.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793022178



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+        docSet = null;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj));

Review comment:
       This appears to be a real bug; you are not comparing the docSet in `obj.docSet`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r796191450



##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -96,6 +95,6 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
     DocSet docs = solrSearcher.getDocSet(q);

Review comment:
       *UnusedVariable:*  The local variable 'docs' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r796128037



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+        docSet = null;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj));
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {

Review comment:
       Thanks, I made the note in javadoc that searcher isn't being used 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r798923010



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -25,18 +25,16 @@
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.ValueSourceScorer;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Weight;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.search.function.ValueSourceRangeFilter;
 
 // This class works as either a normal constant score query, or as a PostFilter using a collector
-public class FunctionRangeQuery extends Query implements PostFilter {
+public class FunctionRangeQuery extends ExtendedQueryBase implements PostFilter {
 
   final ValueSourceRangeFilter rangeFilt;
-  boolean cache = true; // cache by default
   int cost;

Review comment:
       It's still here along with the other stuff not needed due to ExtendedQueryBase

##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -89,8 +90,10 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
       return new ConstantScoreQuery(q).createWeight(searcher, scoreMode, 1f);
     }
 
+    SolrIndexSearcher solrSearcher = (SolrIndexSearcher)searcher;
+    DocSet docs = solrSearcher.getDocSet(q);
     // reqInfo.addCloseHook(docs);  // needed for off-heap refcounting
 
-    return new ConstantScoreQuery(q).createWeight(searcher, scoreMode, 1f);
+    return new ConstantScoreQuery(docs.makeQuery()).createWeight(searcher, scoreMode, 1f);

Review comment:
       DocSet.makeQuery *is* a constant scoring query, so wrapping in ConstantScoreQuery is pointless.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r798925751



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -25,18 +25,16 @@
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.ValueSourceScorer;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Weight;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.search.function.ValueSourceRangeFilter;
 
 // This class works as either a normal constant score query, or as a PostFilter using a collector
-public class FunctionRangeQuery extends Query implements PostFilter {
+public class FunctionRangeQuery extends ExtendedQueryBase implements PostFilter {
 
   final ValueSourceRangeFilter rangeFilt;
-  boolean cache = true; // cache by default
   int cost;

Review comment:
       Yes, I initially removed it and all the other stuff, but TestFiltering.testCache was failing. Upon investigating, I found out the test was failing because I removed them so I resulted to undo the changes 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r798926400



##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -89,8 +90,10 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
       return new ConstantScoreQuery(q).createWeight(searcher, scoreMode, 1f);
     }
 
+    SolrIndexSearcher solrSearcher = (SolrIndexSearcher)searcher;
+    DocSet docs = solrSearcher.getDocSet(q);
     // reqInfo.addCloseHook(docs);  // needed for off-heap refcounting
 
-    return new ConstantScoreQuery(q).createWeight(searcher, scoreMode, 1f);
+    return new ConstantScoreQuery(docs.makeQuery()).createWeight(searcher, scoreMode, 1f);

Review comment:
       I will fix that, thanks




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r798926400



##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -89,8 +90,10 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
       return new ConstantScoreQuery(q).createWeight(searcher, scoreMode, 1f);
     }
 
+    SolrIndexSearcher solrSearcher = (SolrIndexSearcher)searcher;
+    DocSet docs = solrSearcher.getDocSet(q);
     // reqInfo.addCloseHook(docs);  // needed for off-heap refcounting
 
-    return new ConstantScoreQuery(q).createWeight(searcher, scoreMode, 1f);
+    return new ConstantScoreQuery(docs.makeQuery()).createWeight(searcher, scoreMode, 1f);

Review comment:
       I will fix that. Got wrapped up in the failing test and forgot to take that out, thanks




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787233734



##########
File path: solr/core/src/java/org/apache/solr/search/BitDocSet.java
##########
@@ -311,72 +307,8 @@ public long cost() {
     };
   }
 
-  @Override
-  public Filter getTopFilter() {
-    // TODO: if cardinality isn't cached, do a quick measure of sparseness
-    // and return null from bits() if too sparse.
-
-    return new Filter() {
-      final FixedBitSet bs = bits;
-
-      @Override
-      public DocIdSet getDocIdSet(final LeafReaderContext context, final Bits acceptDocs) {
-        // all Solr DocSets that are used as filters only include live docs
-        final Bits acceptDocs2 = acceptDocs == null ? null : (context.reader().getLiveDocs() == acceptDocs ? null : acceptDocs);
-
-        return BitsFilteredDocIdSet.wrap(new DocIdSet() {
-          @Override
-          public DocIdSetIterator iterator() {
-            return BitDocSet.this.iterator(context);
-          }
-
-          @Override
-          public long ramBytesUsed() {
-            return BitDocSet.this.ramBytesUsed();
-          }
-
-          @Override
-          public Bits bits() {
-            if (context.isTopLevel) {
-              return bits;
-            }
-
-            final int base = context.docBase;
-            final int length = context.reader().maxDoc();
-            final FixedBitSet bs = bits;
-
-            return new Bits() {
-              @Override
-              public boolean get(int index) {
-                return bs.get(index + base);
-              }
-
-              @Override
-              public int length() {
-                return length;
-              }
-            };
-          }
-
-        }, acceptDocs2);
-      }
-
-      @Override
-      public String toString(String field) {
-        return "BitSetDocTopFilter";
-      }
-
-      @Override
-      public boolean equals(Object other) {
-        return sameClassAs(other) &&
-               Objects.equals(bs, getClass().cast(other).bs);
-      }
-      
-      @Override
-      public int hashCode() {
-        return classHash() * 31 + bs.hashCode();
-      }
-    };
+  public DocSetQuery makeQuery() {
+    return new DocSetQuery(this);

Review comment:
       Wasn't too sure about how to handle this. Might need to do some more digging into it if it will indeed be a major concern (either in this, or as a separate issue?). 
   
   Hi @dsmiley, would love to get your thoughts here as well? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787220018



##########
File path: solr/core/src/java/org/apache/solr/search/BitsFilteredDocIdSet.java
##########
@@ -23,15 +23,11 @@
 
 /**
  * This implementation supplies a filtered DocIdSet, that excludes all
- * docids which are not in a Bits instance. This is especially useful in
- * {@link org.apache.solr.search.Filter} to apply the {@code acceptDocs}
- * passed to {@code getDocIdSet()} before returning the final DocIdSet.
+ * docids which are not in a Bits instance.
  *
  * @see DocIdSet
- * @see org.apache.solr.search.Filter
  */
 public final class BitsFilteredDocIdSet extends FilteredDocIdSet {

Review comment:
       No it isn't being used anywhere. I agree it should be deleted. I'll do that 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787187632



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery {
+    protected final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSet getDocSet() {
+        return docSet;
+    }

Review comment:
       Yes, deleted. Thanks for pointing that out




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789814138



##########
File path: solr/CHANGES.txt
##########
@@ -72,6 +72,10 @@ New Features
 
 Improvements
 ----------------------
+* SOLR-15257: Add DocSetQuery in lieu of DocSet.getTopFilter. This issue led to removing

Review comment:
       Revised! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r790970904



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -63,7 +73,20 @@ public int hashCode() {
 
     @Override
     public DocSet createDocSet(SolrIndexSearcher searcher) throws IOException {
-        return null;
+        return newDocSet(searcher, Math.min(64,(searcher.maxDoc()>>>10)+4));

Review comment:
       Yes it should since DocSetQuery already has a DocSet. That was my mistake. I was over thinking the implementation there 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789996382



##########
File path: solr/core/src/java/org/apache/solr/search/DocSet.java
##########
@@ -117,11 +118,11 @@ public int andNotSize(DocSet other) {
   }
 
   /**
-   * Returns a Filter for use in Lucene search methods, assuming this DocSet
+   * Returns a Query for use in Lucene search methods, assuming this DocSet

Review comment:
       Thanks, I've updated the documentation




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789803776



##########
File path: solr/contrib/analytics/src/java/org/apache/solr/analytics/AnalyticsDriver.java
##########
@@ -37,28 +37,28 @@
    *
    * @param manager of the request to drive
    * @param searcher the results of the query
-   * @param filter that represents the overall query
+   * @param query that represents the overall query
    * @param queryRequest used for the search request
    * @throws IOException if an error occurs while reading from Solr
    */
-  public static void drive(AnalyticsRequestManager manager, SolrIndexSearcher searcher, Filter filter, SolrQueryRequest queryRequest) throws IOException {
+  public static void drive(AnalyticsRequestManager manager, SolrIndexSearcher searcher, Query query, SolrQueryRequest queryRequest) throws IOException {

Review comment:
       Thanks for pointing this out. I made the changes accordingly. 
   I made the change to `DocSet` in `AnalyticsDriver` but not in other areas because the change from `Query` to `DocSet` affects a lot of other APIs further downstream. I left a _TODO_ comment in `AnalyticsDriver` indicating this. Maybe we can punt the changes to a future issue?

##########
File path: solr/contrib/analytics/src/java/org/apache/solr/analytics/facet/AbstractSolrQueryFacet.java
##########
@@ -50,11 +49,11 @@ protected AbstractSolrQueryFacet(String name) {
    *
    * Each of these executors will be executed after the streaming phase in the {@link AnalyticsDriver}.
    *
-   * @param filter the overall filter representing the documents being used for the analytics request

Review comment:
       I made the change to `DocSet` in `AnalyticsDriver` but not in other areas because the change from `Query` to `DocSet` affects a lot of other APIs further downstream. I left a _TODO_ comment in `AnalyticsDriver` indicating this. Maybe we can punt the changes to a future issue?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789213044



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -39,10 +40,20 @@ public FunctionRangeQuery(ValueSourceRangeFilter filter) {
     this.cost = 100; // default behavior should be PostFiltering
   }
 
+  @Override

Review comment:
       Yea, I really can't remember why that's there, maybe some debug code. I have cleaned it up though. Thanks for catching that! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 closed pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 closed pull request #529:
URL: https://github.com/apache/solr/pull/529


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r796137646



##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -96,6 +95,6 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
     DocSet docs = solrSearcher.getDocSet(q);
     // reqInfo.addCloseHook(docs);  // needed for off-heap refcounting
 
-    return new BoostQuery(new SolrConstantScoreQuery(docs.makeQuery()), 0).createWeight(searcher, scoreMode, 1f);
+    return new BoostQuery(q, 0).createWeight(searcher, scoreMode, 1f);

Review comment:
       This is still a work in progress. I will correct it and push 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793927255



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -18,40 +18,70 @@
 
 import java.io.IOException;
 import java.util.Map;
+import java.util.Objects;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.ValueSourceScorer;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Weight;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.search.function.ValueSourceRangeFilter;
 
 // This class works as either a normal constant score query, or as a PostFilter using a collector
-public class FunctionRangeQuery extends SolrConstantScoreQuery implements PostFilter {
+public class FunctionRangeQuery extends Query implements PostFilter {

Review comment:
       Oh ok, I will extend ExtendedQueryBase there




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799607211



##########
File path: solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java
##########
@@ -287,24 +288,23 @@ public static void createGraphField(FacetRequest.Domain domain, Map<String,Objec
        * Creates a Query that can be used to recompute the new "base" for this domain, relative to the
        * current base of the FacetContext.
        */
-      public Query createDomainQuery(FacetContext fcontext) throws IOException {
-        final SolrConstantScoreQuery fromQuery = new SolrConstantScoreQuery(fcontext.base.getTopFilter());
+      public Query createDomainQuery(FacetContext fcontext) {
+        final ConstantScoreQuery fromQuery = new ConstantScoreQuery(fcontext.base.makeQuery());

Review comment:
       Refactored it so `fromQuery` is not wrapped in a constant score query given we are using makeQuery which is a constant scoring query




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799820941



##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -89,13 +87,13 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
 
     if (!(searcher instanceof SolrIndexSearcher)) {
       // delete-by-query won't have SolrIndexSearcher
-      return new BoostQuery(new ConstantScoreQuery(q), 0).createWeight(searcher, scoreMode, 1f);
+      return new ConstantScoreQuery(q).createWeight(searcher, scoreMode, 1f);

Review comment:
       Good point.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
madrob commented on pull request #529:
URL: https://github.com/apache/solr/pull/529#issuecomment-1031647210


   `gradlew :solr:core:test --tests "org.apache.solr.search.TestSolrQueryParser.testFilter" -Ptests.jvms=16 -Ptests.jvmargs=-XX:TieredStopAtLevel=1 -Ptests.seed=6B406B744E68E8EE -Ptests.file.encoding=US-ASCII` test failure.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799819737



##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -89,13 +87,13 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
 
     if (!(searcher instanceof SolrIndexSearcher)) {
       // delete-by-query won't have SolrIndexSearcher
-      return new BoostQuery(new ConstantScoreQuery(q), 0).createWeight(searcher, scoreMode, 1f);
+      return new ConstantScoreQuery(q).createWeight(searcher, scoreMode, 1f);

Review comment:
       ConstantScoreWeight is abstract, we'd have to implement several more methods if we did that. Wouldn't be as concise I think.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r799014765



##########
File path: solr/core/src/java/org/apache/solr/search/FunctionRangeQuery.java
##########
@@ -25,18 +25,16 @@
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.ValueSourceScorer;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Weight;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.search.function.ValueSourceRangeFilter;
 
 // This class works as either a normal constant score query, or as a PostFilter using a collector
-public class FunctionRangeQuery extends Query implements PostFilter {
+public class FunctionRangeQuery extends ExtendedQueryBase implements PostFilter {
 
   final ValueSourceRangeFilter rangeFilt;
-  boolean cache = true; // cache by default
   int cost;

Review comment:
       **Update:** Got some suggestions from @madrob that I think should fix it 
   **re:** calling setCost in `FunctionRangeQuery` to access super class 
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793194710



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+/**
+ * A class that accesses Queries based on a DocSet
+ *
+ * Refer SOLR-15257
+ */
+public class DocSetQuery extends Query implements DocSetProducer{
+    private final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSetQuery() {
+        docSet = null;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj));

Review comment:
       Thanks for catching this David! I have made some changes there to address this
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793933483



##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -96,6 +95,6 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
     DocSet docs = solrSearcher.getDocSet(q);
     // reqInfo.addCloseHook(docs);  // needed for off-heap refcounting
 
-    return new BoostQuery(new SolrConstantScoreQuery(docs.makeQuery()), 0).createWeight(searcher, scoreMode, 1f);
+    return new BoostQuery(new ConstantScoreQuery(docs.makeQuery()), 0).createWeight(searcher, scoreMode, 1f);

Review comment:
       Why boost a constant scoring query where we pick the score? :-)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r794843972



##########
File path: solr/core/src/java/org/apache/solr/query/FilterQuery.java
##########
@@ -96,6 +95,6 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
     DocSet docs = solrSearcher.getDocSet(q);
     // reqInfo.addCloseHook(docs);  // needed for off-heap refcounting
 
-    return new BoostQuery(new SolrConstantScoreQuery(docs.makeQuery()), 0).createWeight(searcher, scoreMode, 1f);
+    return new BoostQuery(new ConstantScoreQuery(docs.makeQuery()), 0).createWeight(searcher, scoreMode, 1f);

Review comment:
       Thanks for pointing that out. I've changed it to boost the query in FilterQuery




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 edited a comment on pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 edited a comment on pull request #529:
URL: https://github.com/apache/solr/pull/529#issuecomment-1022402464


   This PR was closed in favor of the new one https://github.com/apache/solr/pull/566 which is a continuation of this one. The PR was closed as a result of renaming the branch to correctly reference the parent issue SOLR-12336. Please refer to the [new PR here](https://github.com/apache/solr/pull/566)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-12336 Remove Filter from Solr

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r793317142



##########
File path: solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
##########
@@ -41,17 +41,20 @@
  * Experimental and subject to change.
  */
 public class SolrConstantScoreQuery extends Query implements ExtendedQuery {
-  private final Filter filter;

Review comment:
       Ok. I did some more work to remove SolrConstantScoreQuery. It's now up with my latest commit. Please give it a look, would love to get some feedback 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r787226673



##########
File path: solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
##########
@@ -75,35 +78,42 @@ public int getCost() {
   }
 
   protected class ConstantWeight extends ConstantScoreWeight {
-    private Map<Object,Object> context;
+    private Map<Object, Object> context;

Review comment:
       *UnusedVariable:*  The field 'context' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789213761



##########
File path: solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
##########
@@ -41,17 +41,20 @@
  * Experimental and subject to change.
  */
 public class SolrConstantScoreQuery extends Query implements ExtendedQuery {
-  private final Filter filter;

Review comment:
       Thanks for this info. I would be open to picking it up in another issue. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789205989



##########
File path: solr/core/src/java/org/apache/solr/search/DocSetQuery.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class DocSetQuery extends SolrConstantScoreQuery {
+    protected final DocSet docSet;
+
+    public DocSetQuery(DocSet docSet) {
+        super();
+        this.docSet = docSet;
+    }
+
+    public DocSet getDocSet() {
+        return docSet;
+    }
+
+    @Override
+    public String toString(String field) {
+        return "DocSetQuery(" + field + ")";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+        visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return sameClassAs(obj) && Objects.equals(docSet, getClass().cast(obj).docSet);
+    }
+
+    @Override
+    public int hashCode() {
+        return classHash() * 31 + docSet.hashCode();
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+        return new SolrConstantScoreQuery.ConstantWeight(searcher, scoreMode, boost) {

Review comment:
       Ok, thanks for alerting me to that. I will make the correction to using Lucene's ConstantScoreWeight




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789797372



##########
File path: solr/contrib/analytics/src/java/org/apache/solr/analytics/facet/AbstractSolrQueryFacet.java
##########
@@ -50,11 +49,11 @@ protected AbstractSolrQueryFacet(String name) {
    *
    * Each of these executors will be executed after the streaming phase in the {@link AnalyticsDriver}.
    *
-   * @param filter the overall filter representing the documents being used for the analytics request

Review comment:
       I made the change to `DocSet` in `AnalyticsDriver` but not in other areas because the change from `Query` to `DocSet` affects a lot of other APIs further downstream. I left a TODO comment in `AnalyticsDriver` indicating this. Maybe we can punt the changes to a future issue?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789804691



##########
File path: solr/CHANGES.txt
##########
@@ -72,6 +72,10 @@ New Features
 
 Improvements
 ----------------------
+* SOLR-15257: Add DocSetQuery in lieu of DocSet.getTopFilter. This issue led to removing

Review comment:
       Oh... Thanks for pointing this out. I will revise that 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r789996382



##########
File path: solr/core/src/java/org/apache/solr/search/DocSet.java
##########
@@ -117,11 +118,11 @@ public int andNotSize(DocSet other) {
   }
 
   /**
-   * Returns a Filter for use in Lucene search methods, assuming this DocSet
+   * Returns a Query for use in Lucene search methods, assuming this DocSet

Review comment:
       Thanks, I'll update the documentation




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] Caa52 commented on a change in pull request #529: SOLR-15257 Add DocSetQuery in lieu of DocSet.getTopFilter

Posted by GitBox <gi...@apache.org>.
Caa52 commented on a change in pull request #529:
URL: https://github.com/apache/solr/pull/529#discussion_r790007312



##########
File path: solr/core/src/test/org/apache/solr/search/TestFilteredDocIdSet.java
##########
@@ -113,7 +116,7 @@ protected boolean match(int docid) {
   }
   
   public void testNullDocIdSet() throws Exception {
-    // Tests that if a Filter produces a null DocIdSet, which is given to
+    // Tests that if a Query produces a null DocIdSet, which is given to

Review comment:
       Thanks. Prefixed it with :(historical note)" per your suggestion




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org