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 2021/08/18 22:37:06 UTC

[GitHub] [solr] makosten opened a new pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

makosten opened a new pull request #267:
URL: https://github.com/apache/solr/pull/267


   https://issues.apache.org/jira/browse/SOLR-15595
   
   <!--
   _(If you are a project committer then you may remove some/all of the following template.)_
   
   Before creating a pull request, please file an issue in the ASF Jira system for Solr:
   
   * https://issues.apache.org/jira/projects/SOLR
   
   You will need to create an account in Jira in order to create an issue.
   
   The title of the PR should reference the Jira issue number in the form:
   
   * SOLR-####: <short description of problem or changes>
   
   SOLR must be fully capitalized. A short description helps people scanning pull requests for items they can work on.
   
   Properly referencing the issue in the title ensures that Jira is correctly updated with code review comments and commits. -->
   
   
   # Description
   
   Partial shard results due to timeouts are not merged into the final query results. This is unnecessary for queries sorted only by score and/or docid.
   
   # Solution
   
   No longer exit the shard merge early when partial results are returned for queries sorted just by score or docid.
   
   # Tests
   
   Additional tests added for SortSpecParser for detecting that the sort only includes the score and/or docid.
   
   # 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.
   - [ ] 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] makosten commented on a change in pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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



##########
File path: solr/core/src/test/org/apache/solr/handler/component/QueryComponentPartialResultsTest.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.*;
+import org.apache.solr.search.SortSpec;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.*;
+
+public class QueryComponentPartialResultsTest extends SolrTestCaseJ4 {
+    private static int id = 0;
+    private static final String SORT_FIELD_NAME = "category";
+
+    @BeforeClass
+    public static void setup() {
+        assumeWorkingMockito();
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingImplicitScoreSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingExplicitScoreSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{SortField.FIELD_SCORE})
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingExplicitDocSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{SortField.FIELD_DOC})
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void excludesPartialShardResultWhenUsingNonScoreOrDocSortField() {
+        SortField sortField = new SortField(SORT_FIELD_NAME, SortField.Type.INT);
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{sortField})
+                .withIncludesNonScoreOrDocSortField(true)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, false);
+    }
+
+    private void testPartialResultsForSortSpec(SortSpec sortSpec, boolean shouldIncludePartialShardResult) {
+        final int shard1Size = 2;
+        final int shard2Size = 3;
+
+        MockResponseBuilder responseBuilder = MockResponseBuilder.create().withSortSpec(sortSpec);
+
+        // shard 1 is marked partial
+        NamedList<Object> responseHeader1 = new NamedList<>();
+        responseHeader1.add(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+
+        // shard 2 is not partial
+        NamedList<Object> responseHeader2 = new NamedList<>();
+
+        MockShardRequest shardRequest = MockShardRequest.create()
+                .withShardResponse(responseHeader1, createSolrDocumentList(shard1Size))
+                .withShardResponse(responseHeader2, createSolrDocumentList(shard2Size));

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] epugh edited a comment on pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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


   What do we think @madrob @makosten ?  Ready for merging?  It would be nice to get this into 8.10!   Do we need to update the 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] makosten commented on a change in pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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



##########
File path: solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
##########
@@ -955,9 +955,12 @@ protected void mergeIds(ResponseBuilder rb, ShardRequest sreq) {
 
         @SuppressWarnings("unchecked")
         NamedList<List<Object>> sortFieldValues = (NamedList<List<Object>>)(srsp.getSolrResponse().getResponse().get("sort_values"));
-        if ((null == sortFieldValues || sortFieldValues.size()==0) && // we bypass merging this response only if it's partial itself
-                            thisResponseIsPartial) { // but not the previous one!!
-          continue; //fsv timeout yields empty sort_vlaues
+        if (null == sortFieldValues) {
+          sortFieldValues = new NamedList<>();
+        }
+        // skip merging results for this shard if the sortSpec includes a non-scoredoc field but the sortFieldValues is empty.
+        if (thisResponseIsPartial && sortFieldValues.size() == 0 && ss.includesNonScoreOrDocField()) {

Review comment:
       I'm adding skipping calling unmarshalSortValues if it is a no-op.




-- 
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] epugh commented on pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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


   I  just triggered the precommit check ;-)


-- 
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] epugh commented on pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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


   The test that @makosten added looks good to me, thoughts @madrob ?


-- 
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] epugh commented on pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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


   What do we think @madrob @makosten ?  Ready for merging?    Do we need to update the 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] makosten commented on a change in pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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



##########
File path: solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
##########
@@ -955,11 +955,15 @@ protected void mergeIds(ResponseBuilder rb, ShardRequest sreq) {
 
         @SuppressWarnings("unchecked")
         NamedList<List<Object>> sortFieldValues = (NamedList<List<Object>>)(srsp.getSolrResponse().getResponse().get("sort_values"));
-        if ((null == sortFieldValues || sortFieldValues.size()==0) && // we bypass merging this response only if it's partial itself
-                            thisResponseIsPartial) { // but not the previous one!!
-          continue; //fsv timeout yields empty sort_vlaues
+        if (null == sortFieldValues) {
+          sortFieldValues = new NamedList<>();
         }
-        NamedList<List<Object>> unmarshalledSortFieldValues = unmarshalSortValues(ss, sortFieldValues, schema);
+        boolean needsUnmarshalling = ss.includesNonScoreOrDocField();
+        // skip merging results for this shard if the sortSpec sort values need to be marshalled but the sortFieldValues is empty.
+        if (thisResponseIsPartial && sortFieldValues.size() == 0 && needsUnmarshalling) {
+          continue;
+        }
+        NamedList<List<Object>> unmarshalledSortFieldValues = needsUnmarshalling ? unmarshalSortValues(ss, sortFieldValues, schema) : new NamedList<>();

Review comment:
       I added some comments. Hope they 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] makosten commented on a change in pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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



##########
File path: solr/core/src/test/org/apache/solr/handler/component/QueryComponentPartialResultsTest.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.*;
+import org.apache.solr.search.SortSpec;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.*;
+
+public class QueryComponentPartialResultsTest extends SolrTestCaseJ4 {
+    private static int id = 0;
+    private static final String SORT_FIELD_NAME = "category";
+
+    @BeforeClass
+    public static void setup() {
+        assumeWorkingMockito();
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingImplicitScoreSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingExplicitScoreSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{SortField.FIELD_SCORE})
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingExplicitDocSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{SortField.FIELD_DOC})
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void excludesPartialShardResultWhenUsingNonScoreOrDocSortField() {
+        SortField sortField = new SortField(SORT_FIELD_NAME, SortField.Type.INT);
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{sortField})
+                .withIncludesNonScoreOrDocSortField(true)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, false);
+    }
+
+    private void testPartialResultsForSortSpec(SortSpec sortSpec, boolean shouldIncludePartialShardResult) {
+        final int shard1Size = 2;
+        final int shard2Size = 3;
+
+        MockResponseBuilder responseBuilder = MockResponseBuilder.create().withSortSpec(sortSpec);
+
+        // shard 1 is marked partial
+        NamedList<Object> responseHeader1 = new NamedList<>();
+        responseHeader1.add(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+
+        // shard 2 is not partial
+        NamedList<Object> responseHeader2 = new NamedList<>();
+
+        MockShardRequest shardRequest = MockShardRequest.create()
+                .withShardResponse(responseHeader1, createSolrDocumentList(shard1Size))
+                .withShardResponse(responseHeader2, createSolrDocumentList(shard2Size));
+
+        QueryComponent queryComponent = new QueryComponent();
+        queryComponent.mergeIds(responseBuilder, shardRequest);
+
+        // do we have the expected document count
+        assertEquals((shouldIncludePartialShardResult ? shard1Size : 0) + shard2Size, responseBuilder.getResponseDocs().size());
+    }
+
+    private static class MockResponseBuilder extends ResponseBuilder {

Review comment:
       Extracted the Mock classes.




-- 
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 #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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



##########
File path: solr/core/src/test/org/apache/solr/handler/component/QueryComponentPartialResultsTest.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.*;
+import org.apache.solr.search.SortSpec;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.*;

Review comment:
       nit: no wildcard imports please

##########
File path: solr/core/src/test/org/apache/solr/handler/component/QueryComponentPartialResultsTest.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.*;
+import org.apache.solr.search.SortSpec;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.*;
+
+public class QueryComponentPartialResultsTest extends SolrTestCaseJ4 {
+    private static int id = 0;
+    private static final String SORT_FIELD_NAME = "category";
+
+    @BeforeClass
+    public static void setup() {
+        assumeWorkingMockito();
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingImplicitScoreSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingExplicitScoreSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{SortField.FIELD_SCORE})
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingExplicitDocSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{SortField.FIELD_DOC})
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void excludesPartialShardResultWhenUsingNonScoreOrDocSortField() {
+        SortField sortField = new SortField(SORT_FIELD_NAME, SortField.Type.INT);
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{sortField})
+                .withIncludesNonScoreOrDocSortField(true)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, false);
+    }
+
+    private void testPartialResultsForSortSpec(SortSpec sortSpec, boolean shouldIncludePartialShardResult) {
+        final int shard1Size = 2;
+        final int shard2Size = 3;
+
+        MockResponseBuilder responseBuilder = MockResponseBuilder.create().withSortSpec(sortSpec);
+
+        // shard 1 is marked partial
+        NamedList<Object> responseHeader1 = new NamedList<>();
+        responseHeader1.add(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+
+        // shard 2 is not partial
+        NamedList<Object> responseHeader2 = new NamedList<>();
+
+        MockShardRequest shardRequest = MockShardRequest.create()
+                .withShardResponse(responseHeader1, createSolrDocumentList(shard1Size))
+                .withShardResponse(responseHeader2, createSolrDocumentList(shard2Size));
+
+        QueryComponent queryComponent = new QueryComponent();
+        queryComponent.mergeIds(responseBuilder, shardRequest);
+
+        // do we have the expected document count
+        assertEquals((shouldIncludePartialShardResult ? shard1Size : 0) + shard2Size, responseBuilder.getResponseDocs().size());
+    }
+
+    private static class MockResponseBuilder extends ResponseBuilder {
+
+        private MockResponseBuilder(SolrQueryRequest request, SolrQueryResponse response, List<SearchComponent> components) {
+            super(request, response, components);
+        }
+
+        public static MockResponseBuilder create() {
+
+            // the mocks
+            SolrQueryRequest request = Mockito.mock(SolrQueryRequest.class);
+            SolrQueryResponse response = Mockito.mock(SolrQueryResponse.class);
+            IndexSchema indexSchema = Mockito.mock(IndexSchema.class);
+            SolrParams params = Mockito.mock(SolrParams.class);
+
+            // SchemaField must be concrete due to field access
+            SchemaField uniqueIdField = new SchemaField("id", new StrField());
+
+            // we need this because QueryComponent adds a property to it.
+            NamedList<Object> responseHeader = new NamedList<>();
+
+            // the mock implementations
+            Mockito.when(request.getSchema()).thenReturn(indexSchema);
+            Mockito.when(indexSchema.getUniqueKeyField()).thenReturn(uniqueIdField);
+            Mockito.when(params.getBool(ShardParams.SHARDS_INFO)).thenReturn(false);
+            Mockito.when(request.getParams()).thenReturn(params);
+            Mockito.when(response.getResponseHeader()).thenReturn(responseHeader);
+
+            List<SearchComponent> components = new ArrayList<>();
+            return new MockResponseBuilder(request, response, components);
+
+        }
+
+        public MockResponseBuilder withSortSpec(SortSpec sortSpec) {
+            this.setSortSpec(sortSpec);
+            return this;
+        }
+
+    }
+
+    private static class MockShardRequest extends ShardRequest {
+
+        public static MockShardRequest create() {
+            MockShardRequest mockShardRequest = new MockShardRequest();
+            mockShardRequest.responses = new ArrayList<>();
+            return mockShardRequest;
+        }
+
+        public MockShardRequest withShardResponse(NamedList<Object> responseHeader, SolrDocumentList solrDocuments) {
+            ShardResponse shardResponse = buildShardResponse(responseHeader, solrDocuments);
+            responses.add(shardResponse);
+            return this;
+        }
+
+        private ShardResponse buildShardResponse(NamedList<Object> responseHeader, SolrDocumentList solrDocuments) {
+            SolrResponse solrResponse = Mockito.mock(SolrResponse.class);
+            ShardResponse shardResponse = new ShardResponse();
+            NamedList<Object> response = new NamedList<>();
+            response.add("response", solrDocuments);
+            shardResponse.setSolrResponse(solrResponse);
+            response.add("responseHeader", responseHeader);
+            Mockito.when(solrResponse.getResponse()).thenReturn(response);
+
+            return shardResponse;
+        }
+
+    }
+
+    private static class MockSortSpecBuilder {
+        private final SortSpec sortSpec;
+
+        public MockSortSpecBuilder() {
+            this.sortSpec = Mockito.mock(SortSpec.class);
+            Mockito.when(sortSpec.getCount()).thenReturn(10);
+        }
+
+        public static MockSortSpecBuilder create() {
+            return new MockSortSpecBuilder();
+        }
+
+        public MockSortSpecBuilder withSortFields(SortField[] sortFields) {
+            Sort sort = Mockito.mock(Sort.class);
+            Mockito.when(sort.getSort()).thenReturn(sortFields);
+            Mockito.when(sortSpec.getSort()).thenReturn(sort);
+            return this;
+        }
+
+        public MockSortSpecBuilder withIncludesNonScoreOrDocSortField(boolean include) {
+            Mockito.when(sortSpec.includesNonScoreOrDocField()).thenReturn(include);
+            return this;
+        }
+
+        public SortSpec build() {
+            return sortSpec;
+        }
+
+    }
+
+    private static SolrDocumentList createSolrDocumentList(int size) {
+        SolrDocumentList solrDocuments = new SolrDocumentList();
+        for(int i = 0; i < size; i++) {
+            SolrDocument solrDocument = new SolrDocument();
+            solrDocument.addField("id", id++);
+            solrDocument.addField("score", id * 1.1F);
+            solrDocument.addField(SORT_FIELD_NAME, id * 10);

Review comment:
       Do these values actually matter? What if we made them constant. Or equal to id?

##########
File path: solr/core/src/test/org/apache/solr/handler/component/QueryComponentPartialResultsTest.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.*;
+import org.apache.solr.search.SortSpec;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.*;
+
+public class QueryComponentPartialResultsTest extends SolrTestCaseJ4 {
+    private static int id = 0;
+    private static final String SORT_FIELD_NAME = "category";
+
+    @BeforeClass
+    public static void setup() {
+        assumeWorkingMockito();
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingImplicitScoreSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingExplicitScoreSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{SortField.FIELD_SCORE})
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingExplicitDocSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{SortField.FIELD_DOC})
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void excludesPartialShardResultWhenUsingNonScoreOrDocSortField() {
+        SortField sortField = new SortField(SORT_FIELD_NAME, SortField.Type.INT);
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{sortField})
+                .withIncludesNonScoreOrDocSortField(true)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, false);
+    }
+
+    private void testPartialResultsForSortSpec(SortSpec sortSpec, boolean shouldIncludePartialShardResult) {
+        final int shard1Size = 2;
+        final int shard2Size = 3;
+
+        MockResponseBuilder responseBuilder = MockResponseBuilder.create().withSortSpec(sortSpec);
+
+        // shard 1 is marked partial
+        NamedList<Object> responseHeader1 = new NamedList<>();
+        responseHeader1.add(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+
+        // shard 2 is not partial
+        NamedList<Object> responseHeader2 = new NamedList<>();
+
+        MockShardRequest shardRequest = MockShardRequest.create()
+                .withShardResponse(responseHeader1, createSolrDocumentList(shard1Size))
+                .withShardResponse(responseHeader2, createSolrDocumentList(shard2Size));
+
+        QueryComponent queryComponent = new QueryComponent();
+        queryComponent.mergeIds(responseBuilder, shardRequest);
+
+        // do we have the expected document count
+        assertEquals((shouldIncludePartialShardResult ? shard1Size : 0) + shard2Size, responseBuilder.getResponseDocs().size());
+    }
+
+    private static class MockResponseBuilder extends ResponseBuilder {

Review comment:
       I almost feel like all the Mock* classes shouldn't be in here, but could have their own classes. It looks like the convention we use is to do inner Mock classes when they are nearly trivial and give them their own files otherwise.

##########
File path: solr/core/src/test/org/apache/solr/handler/component/QueryComponentPartialResultsTest.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.*;
+import org.apache.solr.search.SortSpec;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.*;
+
+public class QueryComponentPartialResultsTest extends SolrTestCaseJ4 {
+    private static int id = 0;
+    private static final String SORT_FIELD_NAME = "category";
+
+    @BeforeClass
+    public static void setup() {
+        assumeWorkingMockito();
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingImplicitScoreSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingExplicitScoreSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{SortField.FIELD_SCORE})
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingExplicitDocSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{SortField.FIELD_DOC})
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void excludesPartialShardResultWhenUsingNonScoreOrDocSortField() {
+        SortField sortField = new SortField(SORT_FIELD_NAME, SortField.Type.INT);
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{sortField})
+                .withIncludesNonScoreOrDocSortField(true)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, false);
+    }
+
+    private void testPartialResultsForSortSpec(SortSpec sortSpec, boolean shouldIncludePartialShardResult) {
+        final int shard1Size = 2;
+        final int shard2Size = 3;
+
+        MockResponseBuilder responseBuilder = MockResponseBuilder.create().withSortSpec(sortSpec);
+
+        // shard 1 is marked partial
+        NamedList<Object> responseHeader1 = new NamedList<>();
+        responseHeader1.add(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+
+        // shard 2 is not partial
+        NamedList<Object> responseHeader2 = new NamedList<>();
+
+        MockShardRequest shardRequest = MockShardRequest.create()
+                .withShardResponse(responseHeader1, createSolrDocumentList(shard1Size))
+                .withShardResponse(responseHeader2, createSolrDocumentList(shard2Size));

Review comment:
       Can this be pulled out into a field that only gets created once?

##########
File path: solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
##########
@@ -955,11 +955,15 @@ protected void mergeIds(ResponseBuilder rb, ShardRequest sreq) {
 
         @SuppressWarnings("unchecked")
         NamedList<List<Object>> sortFieldValues = (NamedList<List<Object>>)(srsp.getSolrResponse().getResponse().get("sort_values"));
-        if ((null == sortFieldValues || sortFieldValues.size()==0) && // we bypass merging this response only if it's partial itself
-                            thisResponseIsPartial) { // but not the previous one!!
-          continue; //fsv timeout yields empty sort_vlaues
+        if (null == sortFieldValues) {
+          sortFieldValues = new NamedList<>();
         }
-        NamedList<List<Object>> unmarshalledSortFieldValues = unmarshalSortValues(ss, sortFieldValues, schema);
+        boolean needsUnmarshalling = ss.includesNonScoreOrDocField();
+        // skip merging results for this shard if the sortSpec sort values need to be marshalled but the sortFieldValues is empty.
+        if (thisResponseIsPartial && sortFieldValues.size() == 0 && needsUnmarshalling) {
+          continue;
+        }
+        NamedList<List<Object>> unmarshalledSortFieldValues = needsUnmarshalling ? unmarshalSortValues(ss, sortFieldValues, schema) : new NamedList<>();

Review comment:
       I feel like this needs more comments, or I'm being dense here. I have to mentally trace the code every time I come back to it and convince myself that it's right (which I'm sure it is because I trust you've done the testing, not because I actually understand 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] makosten commented on a change in pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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



##########
File path: solr/core/src/test/org/apache/solr/handler/component/QueryComponentPartialResultsTest.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.*;
+import org.apache.solr.search.SortSpec;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.*;

Review comment:
       Removed import wildcard.




-- 
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 #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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


   LGTM, yes!


-- 
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 #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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



##########
File path: solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
##########
@@ -955,9 +955,12 @@ protected void mergeIds(ResponseBuilder rb, ShardRequest sreq) {
 
         @SuppressWarnings("unchecked")
         NamedList<List<Object>> sortFieldValues = (NamedList<List<Object>>)(srsp.getSolrResponse().getResponse().get("sort_values"));
-        if ((null == sortFieldValues || sortFieldValues.size()==0) && // we bypass merging this response only if it's partial itself
-                            thisResponseIsPartial) { // but not the previous one!!
-          continue; //fsv timeout yields empty sort_vlaues
+        if (null == sortFieldValues) {
+          sortFieldValues = new NamedList<>();
+        }
+        // skip merging results for this shard if the sortSpec includes a non-scoredoc field but the sortFieldValues is empty.
+        if (thisResponseIsPartial && sortFieldValues.size() == 0 && ss.includesNonScoreOrDocField()) {

Review comment:
       it's interesting to me that we do a similar check later on line 1122, not sure if we can combine or remove some of this logic.

##########
File path: solr/core/src/java/org/apache/solr/search/SortSpec.java
##########
@@ -76,6 +76,18 @@ public boolean includesScore() {
     return includesScore(sort);
   }
 
+  public boolean includesNonScoreOrDocField() {
+    return includesNonScoreOrDocField(sort);
+  }
+
+  public static boolean includesNonScoreOrDocField(Sort sort) {

Review comment:
       nit: add javadoc please




-- 
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] makosten commented on a change in pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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



##########
File path: solr/core/src/test/org/apache/solr/handler/component/QueryComponentPartialResultsTest.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.*;
+import org.apache.solr.search.SortSpec;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.*;
+
+public class QueryComponentPartialResultsTest extends SolrTestCaseJ4 {
+    private static int id = 0;
+    private static final String SORT_FIELD_NAME = "category";
+
+    @BeforeClass
+    public static void setup() {
+        assumeWorkingMockito();
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingImplicitScoreSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingExplicitScoreSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{SortField.FIELD_SCORE})
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingExplicitDocSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{SortField.FIELD_DOC})
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void excludesPartialShardResultWhenUsingNonScoreOrDocSortField() {
+        SortField sortField = new SortField(SORT_FIELD_NAME, SortField.Type.INT);
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{sortField})
+                .withIncludesNonScoreOrDocSortField(true)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, false);
+    }
+
+    private void testPartialResultsForSortSpec(SortSpec sortSpec, boolean shouldIncludePartialShardResult) {
+        final int shard1Size = 2;
+        final int shard2Size = 3;
+
+        MockResponseBuilder responseBuilder = MockResponseBuilder.create().withSortSpec(sortSpec);
+
+        // shard 1 is marked partial
+        NamedList<Object> responseHeader1 = new NamedList<>();
+        responseHeader1.add(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+
+        // shard 2 is not partial
+        NamedList<Object> responseHeader2 = new NamedList<>();
+
+        MockShardRequest shardRequest = MockShardRequest.create()
+                .withShardResponse(responseHeader1, createSolrDocumentList(shard1Size))
+                .withShardResponse(responseHeader2, createSolrDocumentList(shard2Size));
+
+        QueryComponent queryComponent = new QueryComponent();
+        queryComponent.mergeIds(responseBuilder, shardRequest);
+
+        // do we have the expected document count
+        assertEquals((shouldIncludePartialShardResult ? shard1Size : 0) + shard2Size, responseBuilder.getResponseDocs().size());
+    }
+
+    private static class MockResponseBuilder extends ResponseBuilder {

Review comment:
       The mock implementations are very bespoke to the new tests, but this way others can extend them.




-- 
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] makosten commented on a change in pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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



##########
File path: solr/core/src/test/org/apache/solr/handler/component/QueryComponentPartialResultsTest.java
##########
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.component;
+
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.*;
+import org.apache.solr.search.SortSpec;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.*;
+
+public class QueryComponentPartialResultsTest extends SolrTestCaseJ4 {
+    private static int id = 0;
+    private static final String SORT_FIELD_NAME = "category";
+
+    @BeforeClass
+    public static void setup() {
+        assumeWorkingMockito();
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingImplicitScoreSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingExplicitScoreSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{SortField.FIELD_SCORE})
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void includesPartialShardResultWhenUsingExplicitDocSort() {
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{SortField.FIELD_DOC})
+                .withIncludesNonScoreOrDocSortField(false)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, true);
+    }
+
+    @Test
+    public void excludesPartialShardResultWhenUsingNonScoreOrDocSortField() {
+        SortField sortField = new SortField(SORT_FIELD_NAME, SortField.Type.INT);
+        SortSpec sortSpec = MockSortSpecBuilder.create()
+                .withSortFields(new SortField[]{sortField})
+                .withIncludesNonScoreOrDocSortField(true)
+                .build();
+        testPartialResultsForSortSpec(sortSpec, false);
+    }
+
+    private void testPartialResultsForSortSpec(SortSpec sortSpec, boolean shouldIncludePartialShardResult) {
+        final int shard1Size = 2;
+        final int shard2Size = 3;
+
+        MockResponseBuilder responseBuilder = MockResponseBuilder.create().withSortSpec(sortSpec);
+
+        // shard 1 is marked partial
+        NamedList<Object> responseHeader1 = new NamedList<>();
+        responseHeader1.add(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
+
+        // shard 2 is not partial
+        NamedList<Object> responseHeader2 = new NamedList<>();
+
+        MockShardRequest shardRequest = MockShardRequest.create()
+                .withShardResponse(responseHeader1, createSolrDocumentList(shard1Size))
+                .withShardResponse(responseHeader2, createSolrDocumentList(shard2Size));
+
+        QueryComponent queryComponent = new QueryComponent();
+        queryComponent.mergeIds(responseBuilder, shardRequest);
+
+        // do we have the expected document count
+        assertEquals((shouldIncludePartialShardResult ? shard1Size : 0) + shard2Size, responseBuilder.getResponseDocs().size());
+    }
+
+    private static class MockResponseBuilder extends ResponseBuilder {
+
+        private MockResponseBuilder(SolrQueryRequest request, SolrQueryResponse response, List<SearchComponent> components) {
+            super(request, response, components);
+        }
+
+        public static MockResponseBuilder create() {
+
+            // the mocks
+            SolrQueryRequest request = Mockito.mock(SolrQueryRequest.class);
+            SolrQueryResponse response = Mockito.mock(SolrQueryResponse.class);
+            IndexSchema indexSchema = Mockito.mock(IndexSchema.class);
+            SolrParams params = Mockito.mock(SolrParams.class);
+
+            // SchemaField must be concrete due to field access
+            SchemaField uniqueIdField = new SchemaField("id", new StrField());
+
+            // we need this because QueryComponent adds a property to it.
+            NamedList<Object> responseHeader = new NamedList<>();
+
+            // the mock implementations
+            Mockito.when(request.getSchema()).thenReturn(indexSchema);
+            Mockito.when(indexSchema.getUniqueKeyField()).thenReturn(uniqueIdField);
+            Mockito.when(params.getBool(ShardParams.SHARDS_INFO)).thenReturn(false);
+            Mockito.when(request.getParams()).thenReturn(params);
+            Mockito.when(response.getResponseHeader()).thenReturn(responseHeader);
+
+            List<SearchComponent> components = new ArrayList<>();
+            return new MockResponseBuilder(request, response, components);
+
+        }
+
+        public MockResponseBuilder withSortSpec(SortSpec sortSpec) {
+            this.setSortSpec(sortSpec);
+            return this;
+        }
+
+    }
+
+    private static class MockShardRequest extends ShardRequest {
+
+        public static MockShardRequest create() {
+            MockShardRequest mockShardRequest = new MockShardRequest();
+            mockShardRequest.responses = new ArrayList<>();
+            return mockShardRequest;
+        }
+
+        public MockShardRequest withShardResponse(NamedList<Object> responseHeader, SolrDocumentList solrDocuments) {
+            ShardResponse shardResponse = buildShardResponse(responseHeader, solrDocuments);
+            responses.add(shardResponse);
+            return this;
+        }
+
+        private ShardResponse buildShardResponse(NamedList<Object> responseHeader, SolrDocumentList solrDocuments) {
+            SolrResponse solrResponse = Mockito.mock(SolrResponse.class);
+            ShardResponse shardResponse = new ShardResponse();
+            NamedList<Object> response = new NamedList<>();
+            response.add("response", solrDocuments);
+            shardResponse.setSolrResponse(solrResponse);
+            response.add("responseHeader", responseHeader);
+            Mockito.when(solrResponse.getResponse()).thenReturn(response);
+
+            return shardResponse;
+        }
+
+    }
+
+    private static class MockSortSpecBuilder {
+        private final SortSpec sortSpec;
+
+        public MockSortSpecBuilder() {
+            this.sortSpec = Mockito.mock(SortSpec.class);
+            Mockito.when(sortSpec.getCount()).thenReturn(10);
+        }
+
+        public static MockSortSpecBuilder create() {
+            return new MockSortSpecBuilder();
+        }
+
+        public MockSortSpecBuilder withSortFields(SortField[] sortFields) {
+            Sort sort = Mockito.mock(Sort.class);
+            Mockito.when(sort.getSort()).thenReturn(sortFields);
+            Mockito.when(sortSpec.getSort()).thenReturn(sort);
+            return this;
+        }
+
+        public MockSortSpecBuilder withIncludesNonScoreOrDocSortField(boolean include) {
+            Mockito.when(sortSpec.includesNonScoreOrDocField()).thenReturn(include);
+            return this;
+        }
+
+        public SortSpec build() {
+            return sortSpec;
+        }
+
+    }
+
+    private static SolrDocumentList createSolrDocumentList(int size) {
+        SolrDocumentList solrDocuments = new SolrDocumentList();
+        for(int i = 0; i < size; i++) {
+            SolrDocument solrDocument = new SolrDocument();
+            solrDocument.addField("id", id++);
+            solrDocument.addField("score", id * 1.1F);
+            solrDocument.addField(SORT_FIELD_NAME, id * 10);

Review comment:
       Used just the id since the values don't matter,




-- 
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] makosten commented on a change in pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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



##########
File path: solr/core/src/java/org/apache/solr/search/SortSpec.java
##########
@@ -76,6 +76,18 @@ public boolean includesScore() {
     return includesScore(sort);
   }
 
+  public boolean includesNonScoreOrDocField() {
+    return includesNonScoreOrDocField(sort);
+  }
+
+  public static boolean includesNonScoreOrDocField(Sort sort) {

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] makosten commented on a change in pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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



##########
File path: solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
##########
@@ -955,9 +955,12 @@ protected void mergeIds(ResponseBuilder rb, ShardRequest sreq) {
 
         @SuppressWarnings("unchecked")
         NamedList<List<Object>> sortFieldValues = (NamedList<List<Object>>)(srsp.getSolrResponse().getResponse().get("sort_values"));
-        if ((null == sortFieldValues || sortFieldValues.size()==0) && // we bypass merging this response only if it's partial itself
-                            thisResponseIsPartial) { // but not the previous one!!
-          continue; //fsv timeout yields empty sort_vlaues
+        if (null == sortFieldValues) {
+          sortFieldValues = new NamedList<>();
+        }
+        // skip merging results for this shard if the sortSpec includes a non-scoredoc field but the sortFieldValues is empty.
+        if (thisResponseIsPartial && sortFieldValues.size() == 0 && ss.includesNonScoreOrDocField()) {

Review comment:
       The unmarshalSortValues method just returns an empty list if the sort if by score and/or docid and no other field. I could revise the mergeIds method to not call it for this case. The only thing that gives me pause is that unmarshalSortValues is protected, so presumably its being overridden someplace? It doesn't appear to be anywhere in the code base.




-- 
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] makosten commented on a change in pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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



##########
File path: solr/core/src/test/org/apache/solr/handler/component/MockSortSpecBuilder.java
##########
@@ -0,0 +1,36 @@
+package org.apache.solr.handler.component;

Review comment:
       Doh!




-- 
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] epugh merged pull request #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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


   


-- 
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 #267: SOLR-15595: Partial results from shard queries needlessly discarded for queries without sort fields

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



##########
File path: solr/core/src/test/org/apache/solr/handler/component/MockSortSpecBuilder.java
##########
@@ -0,0 +1,36 @@
+package org.apache.solr.handler.component;

Review comment:
       license header

##########
File path: solr/core/src/test/org/apache/solr/handler/component/MockResponseBuilder.java
##########
@@ -0,0 +1,54 @@
+package org.apache.solr.handler.component;

Review comment:
       license header

##########
File path: solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
##########
@@ -955,11 +955,15 @@ protected void mergeIds(ResponseBuilder rb, ShardRequest sreq) {
 
         @SuppressWarnings("unchecked")
         NamedList<List<Object>> sortFieldValues = (NamedList<List<Object>>)(srsp.getSolrResponse().getResponse().get("sort_values"));
-        if ((null == sortFieldValues || sortFieldValues.size()==0) && // we bypass merging this response only if it's partial itself
-                            thisResponseIsPartial) { // but not the previous one!!
-          continue; //fsv timeout yields empty sort_vlaues
+        if (null == sortFieldValues) {
+          sortFieldValues = new NamedList<>();
         }
-        NamedList<List<Object>> unmarshalledSortFieldValues = unmarshalSortValues(ss, sortFieldValues, schema);
+        boolean needsUnmarshalling = ss.includesNonScoreOrDocField();
+        // skip merging results for this shard if the sortSpec sort values need to be marshalled but the sortFieldValues is empty.
+        if (thisResponseIsPartial && sortFieldValues.size() == 0 && needsUnmarshalling) {
+          continue;
+        }
+        NamedList<List<Object>> unmarshalledSortFieldValues = needsUnmarshalling ? unmarshalSortValues(ss, sortFieldValues, schema) : new NamedList<>();

Review comment:
       They do! I am super happy with these, and they help me understand it much better than I thought I got it before.




-- 
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