SOLR-15070: Remove HashMap usage in SuggestComponent rsp (#2183)

Prior to this commit, SuggestComponent used a HashMap as part of the
response it built on the server side.  This class is serialized/
deserialized differently depending on the SolrJ ResponseParser used:
a LinkedHashMap when javabin was used, and a SimpleOrderedMap when XML
was used.  This discrepancy led to ClassCastException's in downstream
SolrJ code.

This commit fixes the issue by changing SuggestComponent to avoid these
types that are serialized differently.  "suggest" response sections now
deserialize as a NamedList in SolrJ, and the SuggesterResponse POJO has
been updated accordingly.
This commit is contained in:
Jason Gerlowski 2021-01-11 07:31:26 -05:00 committed by GitHub
parent 7e94a56e81
commit 98c51ca34b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 125 additions and 97 deletions

View File

@ -325,6 +325,8 @@ Bug Fixes
* SOLR-12559: Fix placeholder valuesource 'FIELDNAME' not working with de-referenced values in JSON aggregator parsing.
(hossman, Munendra S N)
* SOLR-15070: Suggester requests made with SolrJ can now use XMLResponseParser (Jason Gerlowski)
Other Changes
---------------------

View File

@ -16,22 +16,6 @@
*/
package org.apache.solr.handler.component;
import java.io.File;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.search.suggest.Lookup;
import org.apache.lucene.search.suggest.Lookup.LookupResult;
import org.apache.lucene.util.Accountable;
@ -59,6 +43,23 @@ import org.apache.solr.util.plugin.SolrCoreAware;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
/**
* SuggestComponent: interacts with multiple {@link SolrSuggester} to serve up suggestions
* Responsible for routing commands and queries to the appropriate {@link SolrSuggester}
@ -226,14 +227,14 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
boolean buildAll = params.getBool(SUGGEST_BUILD_ALL, false);
boolean reloadAll = params.getBool(SUGGEST_RELOAD_ALL, false);
Set<SolrSuggester> querySuggesters;
List<SolrSuggester> querySuggesters;
try {
querySuggesters = getSuggesters(params);
} catch(SolrException ex) {
if (!buildAll && !reloadAll) {
throw ex;
} else {
querySuggesters = new HashSet<>();
querySuggesters = new ArrayList<>();
}
}
@ -258,8 +259,7 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
}
SuggesterOptions options = new SuggesterOptions(new CharsRef(query), count, contextFilter, allTermsRequired, highlight);
Map<String, SimpleOrderedMap<NamedList<Object>>> namedListResults =
new HashMap<>();
SimpleOrderedMap<SimpleOrderedMap<NamedList<Object>>> namedListResults = new SimpleOrderedMap<>();
for (SolrSuggester suggester : querySuggesters) {
SuggesterResult suggesterResult = suggester.getSuggestions(options);
toNamedList(suggesterResult, namedListResults);
@ -287,8 +287,8 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
NamedList<Object> resp;
if((resp = srsp.getSolrResponse().getResponse()) != null) {
@SuppressWarnings("unchecked")
Map<String, SimpleOrderedMap<NamedList<Object>>> namedList =
(Map<String, SimpleOrderedMap<NamedList<Object>>>) resp.get(SuggesterResultLabels.SUGGEST);
SimpleOrderedMap<SimpleOrderedMap<NamedList<Object>>> namedList =
(SimpleOrderedMap<SimpleOrderedMap<NamedList<Object>>>) resp.get(SuggesterResultLabels.SUGGEST);
if (log.isInfoEnabled()) {
log.info("{} : {}", srsp.getShard(), namedList);
}
@ -299,8 +299,7 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
// Merge Shard responses
SuggesterResult suggesterResult = merge(suggesterResults, count);
Map<String, SimpleOrderedMap<NamedList<Object>>> namedListResults =
new HashMap<>();
SimpleOrderedMap<SimpleOrderedMap<NamedList<Object>>> namedListResults = new SimpleOrderedMap<>();
toNamedList(suggesterResult, namedListResults);
rb.rsp.add(SuggesterResultLabels.SUGGEST, namedListResults);
@ -315,7 +314,7 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
private static SuggesterResult merge(List<SuggesterResult> suggesterResults, int count) {
SuggesterResult result = new SuggesterResult();
Set<String> allTokens = new HashSet<>();
Set<String> suggesterNames = new HashSet<>();
SortedSet<String> suggesterNames = new TreeSet<>();
// collect all tokens
for (SuggesterResult shardResult : suggesterResults) {
@ -380,8 +379,8 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
return Accountables.namedAccountables("field", suggesters);
}
private Set<SolrSuggester> getSuggesters(SolrParams params) {
Set<SolrSuggester> solrSuggesters = new HashSet<>();
private List<SolrSuggester> getSuggesters(SolrParams params) {
List<SolrSuggester> solrSuggesters = new ArrayList<>();
for(String suggesterName : getSuggesterNames(params)) {
SolrSuggester curSuggester = suggesters.get(suggesterName);
if (curSuggester != null) {
@ -398,8 +397,8 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
}
private Set<String> getSuggesterNames(SolrParams params) {
Set<String> suggesterNames = new HashSet<>();
private SortedSet<String> getSuggesterNames(SolrParams params) {
SortedSet<String> suggesterNames = new TreeSet<>();
String[] suggesterNamesFromParams = params.getParams(SUGGEST_DICT);
if (suggesterNamesFromParams == null) {
suggesterNames.add(DEFAULT_DICT_NAME);
@ -412,8 +411,9 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
}
/** Convert {@link SuggesterResult} to NamedList for constructing responses */
private void toNamedList(SuggesterResult suggesterResult, Map<String, SimpleOrderedMap<NamedList<Object>>> resultObj) {
for(String suggesterName : suggesterResult.getSuggesterNames()) {
private void toNamedList(SuggesterResult suggesterResult, SimpleOrderedMap<SimpleOrderedMap<NamedList<Object>>> resultObj) {
final SortedSet<String> sortedSuggesterNames = new TreeSet<>(suggesterResult.getSuggesterNames());
for(String suggesterName : sortedSuggesterNames) {
SimpleOrderedMap<NamedList<Object>> results = new SimpleOrderedMap<>();
for (String token : suggesterResult.getTokens(suggesterName)) {
SimpleOrderedMap<Object> suggestionBody = new SimpleOrderedMap<>();
@ -437,18 +437,18 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
suggestionBody.add(SuggesterResultLabels.SUGGESTIONS, suggestEntriesNamedList);
results.add(token, suggestionBody);
}
resultObj.put(suggesterName, results);
resultObj.add(suggesterName, results);
}
}
/** Convert NamedList (suggester response) to {@link SuggesterResult} */
private SuggesterResult toSuggesterResult(Map<String, SimpleOrderedMap<NamedList<Object>>> suggestionsMap) {
private SuggesterResult toSuggesterResult(SimpleOrderedMap<SimpleOrderedMap<NamedList<Object>>> suggestionsMap) {
SuggesterResult result = new SuggesterResult();
if (suggestionsMap == null) {
return result;
}
// for each token
for(Map.Entry<String, SimpleOrderedMap<NamedList<Object>>> entry : suggestionsMap.entrySet()) {
for(Map.Entry<String, SimpleOrderedMap<NamedList<Object>>> entry : suggestionsMap) {
String suggesterName = entry.getKey();
for (Iterator<Map.Entry<String, NamedList<Object>>> suggestionsIter = entry.getValue().iterator(); suggestionsIter.hasNext();) {
Map.Entry<String, NamedList<Object>> suggestions = suggestionsIter.next();

View File

@ -16,13 +16,7 @@
*/
package org.apache.solr.handler.component;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import junit.framework.Assert;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.BaseDistributedSearchTestCase;
import org.apache.solr.client.solrj.response.QueryResponse;
@ -32,6 +26,10 @@ import org.apache.solr.spelling.suggest.SuggesterParams;
import org.junit.BeforeClass;
import org.junit.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
/**
* Test for SuggestComponent's distributed querying
*
@ -59,7 +57,7 @@ public class DistributedSuggestComponentTest extends BaseDistributedSearchTestCa
{
NamedList<Object> nl = control.getResponse();
@SuppressWarnings("unchecked")
Map<String, SimpleOrderedMap<NamedList<Object>>> sc = (Map<String, SimpleOrderedMap<NamedList<Object>>>) nl.get("suggest");
NamedList<SimpleOrderedMap<NamedList<Object>>> sc = (NamedList<SimpleOrderedMap<NamedList<Object>>>) nl.get("suggest");
String command = (String) nl.get("command");
if(sc.size() == 0 && command == null) {
Assert.fail("Control data did not return any suggestions or execute any command");

View File

@ -51,7 +51,7 @@ public class QueryResponse extends SolrResponseBase
private NamedList<Object> _spellInfo = null;
private List<NamedList<Object>> _clusterInfo = null;
private NamedList<Object> _jsonFacetingInfo = null;
private Map<String,NamedList<Object>> _suggestInfo = null;
private NamedList<NamedList<Object>> _suggestInfo = null;
private NamedList<Object> _statsInfo = null;
private NamedList<NamedList<Object>> _termsInfo = null;
private NamedList<SolrDocumentList> _moreLikeThisInfo = null;
@ -170,7 +170,7 @@ public class QueryResponse extends SolrResponseBase
// Don't call extractJsonFacetingInfo(_jsonFacetingInfo) here in an effort to do it lazily
}
else if ( "suggest".equals( n ) ) {
_suggestInfo = (Map<String,NamedList<Object>>) res.getVal( i );
_suggestInfo = (NamedList<NamedList<Object>>) res.getVal( i );
extractSuggesterInfo(_suggestInfo);
}
else if ( "stats".equals( n ) ) {
@ -203,7 +203,7 @@ public class QueryResponse extends SolrResponseBase
_jsonFacetingResponse = new NestableJsonFacet(facetInfo);
}
private void extractSuggesterInfo(Map<String, NamedList<Object>> suggestInfo) {
private void extractSuggesterInfo(NamedList<NamedList<Object>> suggestInfo) {
_suggestResponse = new SuggesterResponse(suggestInfo);
}

View File

@ -36,9 +36,12 @@ public class SuggesterResponse {
private final Map<String, List<Suggestion>> suggestionsPerDictionary = new LinkedHashMap<>();
@SuppressWarnings({"unchecked", "rawtypes"})
public SuggesterResponse(Map<String, NamedList<Object>> suggestInfo) {
for (Map.Entry<String, NamedList<Object>> entry : suggestInfo.entrySet()) {
SimpleOrderedMap suggestionsNode = (SimpleOrderedMap) entry.getValue().getVal(0);
public SuggesterResponse(NamedList<NamedList<Object>> suggestInfo) {
for (int i = 0 ; i < suggestInfo.size(); i++) {
final String outerName = suggestInfo.getName(i);
final NamedList<Object> outerValue = suggestInfo.getVal(i);
SimpleOrderedMap suggestionsNode = (SimpleOrderedMap) outerValue.getVal(0);
List<SimpleOrderedMap> suggestionListToParse;
List<Suggestion> suggestionList = new LinkedList<>();
if (suggestionsNode != null) {
@ -52,7 +55,7 @@ public class SuggesterResponse {
Suggestion parsedSuggestion = new Suggestion(term, weight, payload);
suggestionList.add(parsedSuggestion);
}
suggestionsPerDictionary.put(entry.getKey(), suggestionList);
suggestionsPerDictionary.put(outerName, suggestionList);
}
}
}

View File

@ -20,12 +20,18 @@ import java.io.IOException;
import java.util.List;
import java.util.Map;
import org.apache.solr.EmbeddedSolrServerTestBase;
import org.apache.solr.SolrJettyTestBase;
import org.apache.solr.client.solrj.ResponseParser;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.BinaryResponseParser;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.impl.XMLResponseParser;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.CommonParams;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@ -33,79 +39,88 @@ import org.junit.Test;
* Test for SuggesterComponent's response in Solrj
*
*/
public class TestSuggesterResponse extends EmbeddedSolrServerTestBase {
public class TestSuggesterResponse extends SolrJettyTestBase {
@BeforeClass
public static void beforeClass() throws Exception {
initCore();
createAndStartJetty(legacyExampleCollection1SolrHome());
}
@Before
public void setUpClient() {
getSolrClient();
}
static String field = "cat";
@Test
public void testSuggesterResponseObject() throws Exception {
getSolrClient();
addSampleDocs();
SolrQuery query = new SolrQuery("*:*");
query.set(CommonParams.QT, "/suggest");
query.set("suggest.dictionary", "mySuggester");
query.set("suggest.q", "Com");
query.set("suggest.build", true);
QueryRequest request = new QueryRequest(query);
QueryResponse queryResponse = request.process(client);
SuggesterResponse response = queryResponse.getSuggesterResponse();
Map<String, List<Suggestion>> dictionary2suggestions = response.getSuggestions();
assertTrue(dictionary2suggestions.keySet().contains("mySuggester"));
try (SolrClient solrClient = createSuggestSolrClient()) {
SolrQuery query = new SolrQuery("*:*");
query.set(CommonParams.QT, "/suggest");
query.set("suggest.dictionary", "mySuggester");
query.set("suggest.q", "Com");
query.set("suggest.build", true);
QueryRequest request = new QueryRequest(query);
QueryResponse queryResponse = request.process(solrClient);
SuggesterResponse response = queryResponse.getSuggesterResponse();
Map<String, List<Suggestion>> dictionary2suggestions = response.getSuggestions();
assertTrue(dictionary2suggestions.keySet().contains("mySuggester"));
List<Suggestion> mySuggester = dictionary2suggestions.get("mySuggester");
assertEquals("Computational framework", mySuggester.get(0).getTerm());
assertEquals(0, mySuggester.get(0).getWeight());
assertEquals("", mySuggester.get(0).getPayload());
assertEquals("Computer", mySuggester.get(1).getTerm());
assertEquals(0, mySuggester.get(1).getWeight());
assertEquals("", mySuggester.get(1).getPayload());
}
List<Suggestion> mySuggester = dictionary2suggestions.get("mySuggester");
assertEquals("Computational framework", mySuggester.get(0).getTerm());
assertEquals(0, mySuggester.get(0).getWeight());
assertEquals("", mySuggester.get(0).getPayload());
assertEquals("Computer", mySuggester.get(1).getTerm());
assertEquals(0, mySuggester.get(1).getWeight());
assertEquals("", mySuggester.get(1).getPayload());
}
@Test
public void testSuggesterResponseTerms() throws Exception {
getSolrClient();
addSampleDocs();
SolrQuery query = new SolrQuery("*:*");
query.set(CommonParams.QT, "/suggest");
query.set("suggest.dictionary", "mySuggester");
query.set("suggest.q", "Com");
query.set("suggest.build", true);
QueryRequest request = new QueryRequest(query);
QueryResponse queryResponse = request.process(client);
SuggesterResponse response = queryResponse.getSuggesterResponse();
Map<String, List<String>> dictionary2suggestions = response.getSuggestedTerms();
assertTrue(dictionary2suggestions.keySet().contains("mySuggester"));
try (SolrClient solrClient = createSuggestSolrClient()) {
SolrQuery query = new SolrQuery("*:*");
query.set(CommonParams.QT, "/suggest");
query.set("suggest.dictionary", "mySuggester");
query.set("suggest.q", "Com");
query.set("suggest.build", true);
QueryRequest request = new QueryRequest(query);
QueryResponse queryResponse = request.process(solrClient);
SuggesterResponse response = queryResponse.getSuggesterResponse();
Map<String, List<String>> dictionary2suggestions = response.getSuggestedTerms();
assertTrue(dictionary2suggestions.keySet().contains("mySuggester"));
List<String> mySuggester = dictionary2suggestions.get("mySuggester");
assertEquals("Computational framework", mySuggester.get(0));
assertEquals("Computer", mySuggester.get(1));
List<String> mySuggester = dictionary2suggestions.get("mySuggester");
assertEquals("Computational framework", mySuggester.get(0));
assertEquals("Computer", mySuggester.get(1));
}
}
@Test
public void testEmptySuggesterResponse() throws Exception {
getSolrClient();
addSampleDocs();
SolrQuery query = new SolrQuery("*:*");
query.set(CommonParams.QT, "/suggest");
query.set("suggest.dictionary", "mySuggester");
query.set("suggest.q", "Empty");
query.set("suggest.build", true);
QueryRequest request = new QueryRequest(query);
QueryResponse queryResponse = request.process(client);
SuggesterResponse response = queryResponse.getSuggesterResponse();
Map<String, List<String>> dictionary2suggestions = response.getSuggestedTerms();
assertTrue(dictionary2suggestions.keySet().contains("mySuggester"));
try (SolrClient solrClient = createSuggestSolrClient()) {
SolrQuery query = new SolrQuery("*:*");
query.set(CommonParams.QT, "/suggest");
query.set("suggest.dictionary", "mySuggester");
query.set("suggest.q", "Empty");
query.set("suggest.build", true);
QueryRequest request = new QueryRequest(query);
QueryResponse queryResponse = request.process(solrClient);
SuggesterResponse response = queryResponse.getSuggesterResponse();
Map<String, List<String>> dictionary2suggestions = response.getSuggestedTerms();
assertTrue(dictionary2suggestions.keySet().contains("mySuggester"));
List<String> mySuggester = dictionary2suggestions.get("mySuggester");
assertEquals(0, mySuggester.size());
List<String> mySuggester = dictionary2suggestions.get("mySuggester");
assertEquals(0, mySuggester.size());
}
}
private void addSampleDocs() throws SolrServerException, IOException {
@ -126,4 +141,14 @@ public class TestSuggesterResponse extends EmbeddedSolrServerTestBase {
client.commit(true, true);
}
/*
* Randomizes the ResponseParser to test that both javabin and xml responses parse correctly. See SOLR-15070
*/
private SolrClient createSuggestSolrClient() {
final ResponseParser randomParser = random().nextBoolean() ? new BinaryResponseParser() : new XMLResponseParser();
return new HttpSolrClient.Builder()
.withBaseSolrUrl(jetty.getBaseUrl().toString() + "/collection1")
.withResponseParser(randomParser)
.build();
}
}