SOLR-14535: Fix or suppress warnings in apache/solr/handler/component, sql and loader

This commit is contained in:
Erick Erickson 2020-06-04 17:49:46 -04:00
parent dec6922528
commit 0c4d8fb116
41 changed files with 180 additions and 63 deletions

View File

@ -282,6 +282,8 @@ Other Changes
* SOLR-14533: Fix or suppress warnings in solr/handler/admin (Andras Salamon, Erick Erickson)
* SOLR-14535: Fix or suppress warnings in apache/solr/handler/component (Erick Erickson)
================== 8.5.2 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

View File

@ -59,6 +59,7 @@ class CloudReplicaSource implements ReplicaSource {
}
}
@SuppressWarnings({"unchecked", "rawtypes"})
private void withClusterState(Builder builder, SolrParams params) {
ClusterState clusterState = builder.zkStateReader.getClusterState();
String shardKeys = params.get(ShardParams._ROUTE_);
@ -98,6 +99,7 @@ class CloudReplicaSource implements ReplicaSource {
}
}
@SuppressWarnings({"unchecked", "rawtypes"})
private void withShardsParam(Builder builder, String shardsParam) {
List<String> sliceOrUrls = StrUtils.splitSmart(shardsParam, ",", true);
this.slices = new String[sliceOrUrls.size()];

View File

@ -100,9 +100,11 @@ public class DebugComponent extends SearchComponent
results = rb.getResults().docList;
}
@SuppressWarnings({"rawtypes"})
NamedList stdinfo = SolrPluginUtils.doStandardDebug( rb.req,
rb.getQueryString(), rb.wrap(rb.getQuery()), results, rb.isDebugQuery(), rb.isDebugResults());
@SuppressWarnings({"rawtypes"})
NamedList info = rb.getDebugInfo();
if( info == null ) {
rb.setDebugInfo( stdinfo );
@ -225,11 +227,13 @@ public class DebugComponent extends SearchComponent
private final static Set<String> EXCLUDE_SET = Set.of("explain");
@Override
@SuppressWarnings({"unchecked"})
public void finishStage(ResponseBuilder rb) {
if (rb.isDebug() && rb.stage == ResponseBuilder.STAGE_GET_FIELDS) {
NamedList<Object> info = rb.getDebugInfo();
NamedList<Object> explain = new SimpleOrderedMap<>();
@SuppressWarnings({"rawtypes"})
Map.Entry<String, Object>[] arr = new NamedList.NamedListEntry[rb.resultIds.size()];
// Will be set to true if there is at least one response with PURPOSE_GET_DEBUG
boolean hasGetDebugResponses = false;
@ -241,11 +245,14 @@ public class DebugComponent extends SearchComponent
// this should only happen when using shards.tolerant=true
continue;
}
@SuppressWarnings({"rawtypes"})
NamedList sdebug = (NamedList)srsp.getSolrResponse().getResponse().get("debug");
info = (NamedList)merge(sdebug, info, EXCLUDE_SET);
if ((sreq.purpose & ShardRequest.PURPOSE_GET_DEBUG) != 0) {
hasGetDebugResponses = true;
if (rb.isDebugResults()) {
@SuppressWarnings({"rawtypes"})
NamedList sexplain = (NamedList)sdebug.get("explain");
SolrPluginUtils.copyNamedListIntoArrayByDocPosInResponse(sexplain, rb.resultIds, arr);
}
@ -306,6 +313,7 @@ public class DebugComponent extends SearchComponent
return namedList;
}
@SuppressWarnings({"unchecked", "rawtypes"})
protected Object merge(Object source, Object dest, Set<String> exclude) {
if (source == null) return dest;
if (dest == null) {

View File

@ -419,6 +419,8 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
ReturnFields returnFields = rb.rsp.getReturnFields();
LongObjectMap<Collector> groups = ((GroupCollector) groupExpandCollector).getGroups();
@SuppressWarnings({"rawtypes"})
NamedList outMap = new SimpleOrderedMap();
CharsRefBuilder charsRef = new CharsRefBuilder();
for (LongObjectCursor<Collector> cursor : groups) {
@ -454,7 +456,10 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
rb.rsp.add("expanded", outMap);
}
private void addGroupSliceToOutputMap(FieldType fieldType, IntObjectHashMap<BytesRef> ordBytes, NamedList outMap, CharsRefBuilder charsRef, long groupValue, DocSlice slice) {
@SuppressWarnings({"unchecked"})
private void addGroupSliceToOutputMap(FieldType fieldType, IntObjectHashMap<BytesRef> ordBytes,
@SuppressWarnings({"rawtypes"})NamedList outMap, CharsRefBuilder charsRef, long groupValue, DocSlice slice) {
if(fieldType instanceof StrField) {
final BytesRef bytesRef = ordBytes.get((int)groupValue);
fieldType.indexedToReadable(bytesRef, charsRef);
@ -484,8 +489,9 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
}
}
@SuppressWarnings("unchecked")
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public void handleResponses(ResponseBuilder rb, ShardRequest sreq) {
if (!rb.doExpand) {
@ -511,6 +517,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
}
}
@SuppressWarnings("rawtypes")
@Override
public void finishStage(ResponseBuilder rb) {

View File

@ -143,13 +143,13 @@ public class FieldFacetStats {
facetStatsTerms.add(new HashMap<String, Integer>());
}
for (Map.Entry<String, Integer> pairs : facetStatsTerms.get(statsTermNum).entrySet()) {
String key = (String) pairs.getKey();
String key = pairs.getKey();
StatsValues facetStats = facetStatsValues.get(key);
if (facetStats == null) {
facetStats = StatsValuesFactory.createStatsValues(statsField);
facetStatsValues.put(key, facetStats);
}
Integer count = (Integer) pairs.getValue();
Integer count = pairs.getValue();
if (count != null) {
facetStats.accumulate(value, count);
}

View File

@ -168,6 +168,7 @@ public class HighlightComponent extends SearchComponent implements PluginInfoIni
// No highlighting if there is no query -- consider q.alt=*:*
if( highlightQuery != null ) {
@SuppressWarnings({"rawtypes"})
NamedList sumData = highlighter.doHighlighting(
rb.getResults().docList,
highlightQuery,
@ -276,22 +277,26 @@ public class HighlightComponent extends SearchComponent implements PluginInfoIni
return "highlighting";
}
protected Object convertHighlights(NamedList hl) {
protected Object convertHighlights(@SuppressWarnings({"rawtypes"})NamedList hl) {
return hl;
}
@SuppressWarnings({"rawtypes"})
protected Object[] newHighlightsArray(int size) {
return new NamedList.NamedListEntry[size];
}
protected void addHighlights(Object[] objArr, Object obj, Map<Object, ShardDoc> resultIds) {
@SuppressWarnings({"unchecked"})
Map.Entry<String, Object>[] arr = (Map.Entry<String, Object>[])objArr;
@SuppressWarnings({"rawtypes"})
NamedList hl = (NamedList)obj;
SolrPluginUtils.copyNamedListIntoArrayByDocPosInResponse(hl, resultIds, arr);
}
protected Object getAllHighlights(Object[] objArr) {
final Map.Entry<String, Object>[] arr = (Map.Entry<String, Object>[])objArr;
@SuppressWarnings({"unchecked"})
final Map.Entry<String, Object>[] arr = (Map.Entry<String, Object>[])objArr;
// remove nulls in case not all docs were able to be retrieved
return SolrPluginUtils.removeNulls(arr, new SimpleOrderedMap<>());
}

View File

@ -73,7 +73,7 @@ public class HttpShardHandler extends ShardHandler {
}
}
protected NamedList<Object> request(String url, SolrRequest req) throws IOException, SolrServerException {
protected NamedList<Object> request(String url, @SuppressWarnings({"rawtypes"})SolrRequest req) throws IOException, SolrServerException {
req.setBasePath(url);
return httpClient.request(req);
}

View File

@ -166,7 +166,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
// a little hack for backward-compatibility when we are moving from apache http client to jetty client
return new HttpShardHandler(this, null) {
@Override
protected NamedList<Object> request(String url, SolrRequest req) throws IOException, SolrServerException {
protected NamedList<Object> request(String url, @SuppressWarnings({"rawtypes"})SolrRequest req) throws IOException, SolrServerException {
try (SolrClient client = new HttpSolrClient.Builder(url).withHttpClient(httpClient).build()) {
return client.request(req);
}
@ -212,7 +212,8 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
}
}
private void initReplicaListTransformers(NamedList routingConfig) {
@SuppressWarnings({"unchecked"})
private void initReplicaListTransformers(@SuppressWarnings({"rawtypes"})NamedList routingConfig) {
String defaultRouting = null;
ReplicaListTransformerFactory stableRltFactory = null;
ReplicaListTransformerFactory defaultRltFactory;
@ -253,6 +254,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
@Override
public void init(PluginInfo info) {
StringBuilder sb = new StringBuilder();
@SuppressWarnings({"rawtypes"})
NamedList args = info.initArgs;
this.scheme = getParameter(args, INIT_URL_SCHEME, null,sb);
if(StringUtils.endsWith(this.scheme, "://")) {
@ -330,9 +332,10 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
clientBuilderPlugin.setup(defaultClient);
}
protected <T> T getParameter(NamedList initArgs, String configKey, T defaultValue, StringBuilder sb) {
protected <T> T getParameter(@SuppressWarnings({"rawtypes"})NamedList initArgs, String configKey, T defaultValue, StringBuilder sb) {
T toReturn = defaultValue;
if (initArgs != null) {
@SuppressWarnings({"unchecked"})
T temp = (T) initArgs.get(configKey);
toReturn = (temp != null) ? temp : defaultValue;
}

View File

@ -112,6 +112,7 @@ public abstract class IterativeMergeStrategy implements MergeStrategy {
}
public List<Future<CallBack>> callBack(List<ShardResponse> responses, QueryRequest req) {
@SuppressWarnings({"unchecked", "rawtypes"})
List<Future<CallBack>> futures = new ArrayList();
for(ShardResponse response : responses) {
futures.add(this.executorService.submit(new CallBack(response, req)));

View File

@ -66,6 +66,7 @@ public interface MergeStrategy {
* */
public int getCost();
@SuppressWarnings({"rawtypes"})
final Comparator MERGE_COMP = (o1, o2) -> {
MergeStrategy m1 = (MergeStrategy) o1;
MergeStrategy m2 = (MergeStrategy) o2;

View File

@ -217,8 +217,10 @@ public class PhrasesIdentificationComponent extends SearchComponent {
if (null == rsp) continue;
final NamedList<Object> top = rsp.getResponse();
if (null == top) continue;
@SuppressWarnings({"unchecked"})
final NamedList<Object> phrasesWrapper = (NamedList<Object>) top.get("phrases");
if (null == phrasesWrapper) continue;
@SuppressWarnings({"unchecked"})
final List<NamedList<Object>> shardPhrases = (List<NamedList<Object>>) phrasesWrapper.get("_all");
if (null == shardPhrases) continue;
@ -660,6 +662,7 @@ public class PhrasesIdentificationComponent extends SearchComponent {
* Populates the phrases with (merged) stats from a remote shard
* @see #formatShardResponse
*/
@SuppressWarnings({"unchecked"})
public static void populateStats(final List<Phrase> phrases, final List<NamedList<Object>> shardData) {
final int numPhrases = phrases.size();
if (shardData.size() != numPhrases) {
@ -800,6 +803,7 @@ public class PhrasesIdentificationComponent extends SearchComponent {
+ "[" + position_start + ":" + position_end + "]";
}
@SuppressWarnings({"rawtypes"})
public NamedList getDetails() {
SimpleOrderedMap<Object> out = new SimpleOrderedMap<Object>();
out.add("text", subSequence);

View File

@ -342,6 +342,7 @@ public class PivotFacetFieldValueCollection implements Iterable<PivotFacetValue>
* A helper method for use in <code>Comparator</code> classes where object properties
* are <code>Comparable</code> but may be null.
*/
@SuppressWarnings({"unchecked"})
static int compareWithNullLast(final Comparable o1, final Comparable o2) {
if (null == o1) {
if (null == o2) {

View File

@ -92,6 +92,7 @@ public class PivotFacetHelper {
}
/** @see PivotListEntry#VALUE */
@SuppressWarnings({"rawtypes"})
public static Comparable getValue(NamedList<Object> pivotList) {
return (Comparable) PivotListEntry.VALUE.extract(pivotList);
}
@ -107,21 +108,25 @@ public class PivotFacetHelper {
}
/** @see PivotListEntry#PIVOT */
@SuppressWarnings({"unchecked"})
public static List<NamedList<Object>> getPivots(NamedList<Object> pivotList) {
return (List<NamedList<Object>>) PivotListEntry.PIVOT.extract(pivotList);
}
/** @see PivotListEntry#STATS */
@SuppressWarnings({"unchecked"})
public static NamedList<NamedList<NamedList<?>>> getStats(NamedList<Object> pivotList) {
return (NamedList<NamedList<NamedList<?>>>) PivotListEntry.STATS.extract(pivotList);
}
/** @see PivotListEntry#QUERIES */
@SuppressWarnings({"unchecked"})
public static NamedList<Number> getQueryCounts(NamedList<Object> pivotList) {
return (NamedList<Number>) PivotListEntry.QUERIES.extract(pivotList);
}
/** @see PivotListEntry#RANGES */
@SuppressWarnings({"unchecked"})
public static SimpleOrderedMap<SimpleOrderedMap<Object>> getRanges(NamedList<Object> pivotList) {
return (SimpleOrderedMap<SimpleOrderedMap<Object>>) PivotListEntry.RANGES.extract(pivotList);
}

View File

@ -650,7 +650,7 @@ public class QueryComponent extends SearchComponent
protected static final EndResultTransformer MAIN_END_RESULT_TRANSFORMER = new MainEndResultTransformer();
protected static final EndResultTransformer SIMPLE_END_RESULT_TRANSFORMER = new SimpleEndResultTransformer();
@SuppressWarnings("unchecked")
@SuppressWarnings({"unchecked", "rawtypes"})
protected void groupedFinishStage(final ResponseBuilder rb) {
// To have same response as non-distributed request.
GroupingSpecification groupSpec = rb.getGroupingSpec();
@ -806,6 +806,7 @@ public class QueryComponent extends SearchComponent
return true;
}
@SuppressWarnings({"unchecked"})
protected void mergeIds(ResponseBuilder rb, ShardRequest sreq) {
List<MergeStrategy> mergeStrategies = rb.getMergeStrategies();
if(mergeStrategies != null) {
@ -929,11 +930,13 @@ public class QueryComponent extends SearchComponent
hitCountIsExact = false;
}
@SuppressWarnings({"rawtypes"})
NamedList sortFieldValues = (NamedList)(srsp.getSolrResponse().getResponse().get("sort_values"));
if (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
}
@SuppressWarnings({"rawtypes"})
NamedList unmarshalledSortFieldValues = unmarshalSortValues(ss, sortFieldValues, schema);
// go through every doc in this response, construct a ShardDoc, and
@ -1069,6 +1072,7 @@ public class QueryComponent extends SearchComponent
nextCursorMarkValues.add(lastDoc.score);
} else {
assert null != sf.getField() : "SortField has null field";
@SuppressWarnings({"unchecked"})
List<Object> fieldVals = (List<Object>) lastDoc.sortFieldValues.get(sf.getField());
nextCursorMarkValues.add(fieldVals.get(lastDoc.orderInShard));
}
@ -1078,7 +1082,8 @@ public class QueryComponent extends SearchComponent
rb.setNextCursorMark(nextCursorMark);
}
protected NamedList unmarshalSortValues(SortSpec sortSpec,
@SuppressWarnings({"unchecked", "rawtypes"})
protected NamedList unmarshalSortValues(SortSpec sortSpec,
NamedList sortFieldValues,
IndexSchema schema) {
NamedList unmarshalledSortValsPerField = new NamedList();

View File

@ -150,7 +150,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
private final Map<IndexReader, ElevationProvider> elevationProviderCache = new WeakHashMap<>();
@Override
public void init(NamedList args) {
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
this.initArgs = args.toSolrParams();
}
@ -633,7 +633,9 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore
* @param context the {@link SolrQueryRequest#getContext()} or null if none. We'll cache our results here.
*/
//TODO consider simplifying to remove "boosted" arg which can be looked up in context via BOOSTED key?
public static IntIntHashMap getBoostDocs(SolrIndexSearcher indexSearcher, Set<BytesRef> boosted, Map context) throws IOException {
@SuppressWarnings({"unchecked"})
public static IntIntHashMap getBoostDocs(SolrIndexSearcher indexSearcher, Set<BytesRef> boosted,
@SuppressWarnings({"rawtypes"})Map context) throws IOException {
IntIntHashMap boostDocs = null;

View File

@ -98,6 +98,7 @@ public class RangeFacetProcessor extends SimpleFacets {
}
}
@SuppressWarnings({"rawtypes"})
private <T extends Comparable<T>> NamedList getFacetRangeCounts(final RangeFacetRequest rfr)
throws IOException, SyntaxError {

View File

@ -246,6 +246,7 @@ public class RealTimeGetComponent extends SearchComponent
Object o = ulog.lookup(idBytes.get());
if (o != null) {
// should currently be a List<Oper,Ver,Doc/Id>
@SuppressWarnings({"rawtypes"})
List entry = (List)o;
assert entry.size() >= 3;
int oper = (Integer)entry.get(UpdateLog.FLAGS_IDX) & UpdateLog.OPERATION_MASK;
@ -409,7 +410,9 @@ public class RealTimeGetComponent extends SearchComponent
* after the resolving began)
*/
private static SolrDocument resolveFullDocument(SolrCore core, BytesRef idBytes,
ReturnFields returnFields, SolrInputDocument partialDoc, List logEntry, Set<String> onlyTheseFields) throws IOException {
ReturnFields returnFields, SolrInputDocument partialDoc,
@SuppressWarnings({"rawtypes"}) List logEntry,
Set<String> onlyTheseFields) throws IOException {
if (idBytes == null || (logEntry.size() != 5 && logEntry.size() != 6)) {
throw new SolrException(ErrorCode.INVALID_STATE, "Either Id field not present in partial document or log entry doesn't have previous version.");
}
@ -547,6 +550,7 @@ public class RealTimeGetComponent extends SearchComponent
* was an in-place update. In that case, should this partial document be resolved to a full document (by following
* back prevPointer/prevVersion)?
*/
@SuppressWarnings({"fallthrough"})
public static SolrInputDocument getInputDocumentFromTlog(SolrCore core, BytesRef idBytes, AtomicLong versionReturned,
Set<String> onlyTheseNonStoredDVs, boolean resolveFullDocument) {
@ -556,6 +560,7 @@ public class RealTimeGetComponent extends SearchComponent
Object o = ulog.lookup(idBytes);
if (o != null) {
// should currently be a List<Oper,Ver,Doc/Id>
@SuppressWarnings({"rawtypes"})
List entry = (List)o;
assert entry.size() >= 3;
int oper = (Integer)entry.get(0) & UpdateLog.OPERATION_MASK;
@ -693,7 +698,8 @@ public class RealTimeGetComponent extends SearchComponent
return sid;
}
private static void decorateDocValueFields(SolrDocumentFetcher docFetcher, SolrDocumentBase doc, int docid, Set<String> onlyTheseNonStoredDVs, boolean resolveNestedFields) throws IOException {
private static void decorateDocValueFields(SolrDocumentFetcher docFetcher,
@SuppressWarnings({"rawtypes"})SolrDocumentBase doc, int docid, Set<String> onlyTheseNonStoredDVs, boolean resolveNestedFields) throws IOException {
if (onlyTheseNonStoredDVs != null) {
docFetcher.decorateDocValueFields(doc, docid, onlyTheseNonStoredDVs);
} else {
@ -1004,6 +1010,7 @@ public class RealTimeGetComponent extends SearchComponent
// can get more than one response
for (ShardResponse srsp : sreq.responses) {
SolrResponse sr = srsp.getSolrResponse();
@SuppressWarnings({"rawtypes"})
NamedList nl = sr.getResponse();
SolrDocumentList subList = (SolrDocumentList)nl.get("response");
docList.addAll(subList);

View File

@ -215,6 +215,7 @@ public class ResponseBuilder
NamedList<Object> target = debugInfo;
for (int i=0; i<path.length-1; i++) {
String elem = path[i];
@SuppressWarnings({"unchecked"})
NamedList<Object> newTarget = (NamedList<Object>)debugInfo.get(elem);
if (newTarget == null) {
newTarget = new SimpleOrderedMap<>();
@ -250,7 +251,7 @@ public class ResponseBuilder
public void addMergeStrategy(MergeStrategy mergeStrategy) {
if(mergeStrategies == null) {
mergeStrategies = new ArrayList();
mergeStrategies = new ArrayList<>();
}
mergeStrategies.add(mergeStrategy);

View File

@ -92,7 +92,7 @@ public abstract class SearchComponent implements SolrInfoBean, NamedListInitiali
//////////////////////// NamedListInitializedPlugin methods //////////////////////
@Override
public void init( NamedList args )
public void init( @SuppressWarnings({"rawtypes"})NamedList args )
{
// By default do nothing
}

View File

@ -170,6 +170,7 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware,
}
@SuppressWarnings({"unchecked"})
private void initComponents() {
Object declaredComponents = initArgs.get(INIT_COMPONENTS);
List<String> first = (List<String>) initArgs.get(INIT_FIRST_COMPONENTS);
@ -279,6 +280,7 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware,
}
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception
{
List<SearchComponent> components = getComponents();

View File

@ -32,6 +32,7 @@ public class ShardDoc extends FieldDoc {
// this is currently the uniqueKeyField but
// may be replaced with internal docid in a future release.
@SuppressWarnings({"rawtypes"})
public NamedList sortFieldValues;
// sort field values for *all* docs in a particular shard.
// this doc's values are in position orderInShard

View File

@ -41,10 +41,10 @@ public class ShardFieldSortedHitQueue extends PriorityQueue<ShardDoc> {
/** The order of these fieldNames should correspond to the order of sort field values retrieved from the shard */
protected List<String> fieldNames = new ArrayList<>();
@SuppressWarnings({"unchecked", "rawtypes"})
public ShardFieldSortedHitQueue(SortField[] fields, int size, IndexSearcher searcher) {
super(size);
final int n = fields.length;
//noinspection unchecked
comparators = new Comparator[n];
this.fields = new SortField[n];
for (int i = 0; i < n; ++i) {
@ -143,12 +143,14 @@ public class ShardFieldSortedHitQueue extends PriorityQueue<ShardDoc> {
Object sortVal(ShardDoc shardDoc) {
assert(shardDoc.sortFieldValues.getName(fieldNum).equals(fieldName));
@SuppressWarnings({"rawtypes"})
List lst = (List)shardDoc.sortFieldValues.getVal(fieldNum);
return lst.get(shardDoc.orderInShard);
}
}
Comparator<ShardDoc> comparatorFieldComparator(SortField sortField) {
@SuppressWarnings({"rawtypes"})
final FieldComparator fieldComparator = sortField.getComparator(0, 0);
return new ShardComparator(sortField) {
// Since the PriorityQueue keeps the biggest elements by default,
@ -156,6 +158,7 @@ public class ShardFieldSortedHitQueue extends PriorityQueue<ShardDoc> {
// smallest elements are kept instead of the largest... hence
// the negative sign.
@Override
@SuppressWarnings({"unchecked"})
public int compare(final ShardDoc o1, final ShardDoc o2) {
//noinspection unchecked
return -fieldComparator.compareValues(sortVal(o1), sortVal(o2));

View File

@ -40,7 +40,7 @@ public class SortedDateStatsValues implements StatsValues {
}
@Override
public void accumulate(NamedList stv) {
public void accumulate(@SuppressWarnings({"rawtypes"})NamedList stv) {
dsv.accumulate(stv);
}

View File

@ -42,7 +42,7 @@ public class SortedNumericStatsValues implements StatsValues {
}
@Override
public void accumulate(NamedList stv) {
public void accumulate(@SuppressWarnings({"rawtypes"})NamedList stv) {
nsv.accumulate(stv);
}

View File

@ -45,6 +45,7 @@ public class SpatialHeatmapFacets {
public static final String RESPONSE_KEY = "facet_heatmaps";
/** Called by {@link org.apache.solr.request.SimpleFacets} to compute heatmap facets. */
@SuppressWarnings({"unchecked", "rawtypes"})
public static NamedList<Object> getHeatmapForField(String fieldKey, String fieldName, ResponseBuilder rb, SolrParams params, DocSet docSet) throws IOException {
final FacetRequest facetRequest = createHeatmapRequest(fieldKey, fieldName, rb, params);
return (NamedList) facetRequest.process(rb.req, docSet);
@ -106,7 +107,7 @@ public class SpatialHeatmapFacets {
/** Called by FacetComponent.countFacets which is in turn called by FC's impl of
* {@link org.apache.solr.handler.component.SearchComponent#handleResponses(ResponseBuilder, ShardRequest)}. */
@SuppressWarnings("unchecked")
public static void distribHandleResponse(LinkedHashMap<String, HeatmapFacet> heatmapFacets, NamedList srsp_facet_counts) {
public static void distribHandleResponse(LinkedHashMap<String, HeatmapFacet> heatmapFacets, @SuppressWarnings({"rawtypes"})NamedList srsp_facet_counts) {
NamedList<NamedList<Object>> facet_heatmaps = (NamedList<NamedList<Object>>) srsp_facet_counts.get(RESPONSE_KEY);
if (facet_heatmaps == null) {
return;
@ -127,6 +128,7 @@ public class SpatialHeatmapFacets {
/** Called by FacetComponent's impl of
* {@link org.apache.solr.handler.component.SearchComponent#finishStage(ResponseBuilder)}. */
@SuppressWarnings({"unchecked", "rawtypes"})
public static NamedList distribFinish(LinkedHashMap<String, HeatmapFacet> heatmapInfos, ResponseBuilder rb) {
NamedList<NamedList<Object>> result = new SimpleOrderedMap<>();
for (Map.Entry<String, HeatmapFacet> entry : heatmapInfos.entrySet()) {

View File

@ -97,7 +97,7 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
*/
public static final String COMPONENT_NAME = "spellcheck";
@SuppressWarnings("unchecked")
@SuppressWarnings({"rawtypes"})
protected NamedList initParams;
@ -109,8 +109,7 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
protected QueryConverter queryConverter;
@Override
@SuppressWarnings("unchecked")
public void init(NamedList args) {
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
super.init(args);
this.initParams = args;
}
@ -198,7 +197,9 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
}
boolean isCorrectlySpelled = hits > (maxResultsForSuggest==null ? 0 : maxResultsForSuggest);
@SuppressWarnings({"rawtypes"})
NamedList response = new SimpleOrderedMap();
@SuppressWarnings({"rawtypes"})
NamedList suggestions = toNamedList(shardRequest, spellingResult, q, extendedResults);
response.add("suggestions", suggestions);
@ -276,7 +277,7 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
@SuppressWarnings("unchecked")
protected void addCollationsToResponse(SolrParams params, SpellingResult spellingResult, ResponseBuilder rb, String q,
NamedList response, boolean suggestionsMayOverlap) {
@SuppressWarnings({"rawtypes"})NamedList response, boolean suggestionsMayOverlap) {
int maxCollations = params.getInt(SPELLCHECK_MAX_COLLATIONS, 1);
int maxCollationTries = params.getInt(SPELLCHECK_MAX_COLLATION_TRIES, 0);
int maxCollationEvaluations = params.getInt(SPELLCHECK_MAX_COLLATION_EVALUATIONS, 10000);
@ -300,9 +301,11 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
//even in cases when the internal rank is the same.
Collections.sort(collations);
@SuppressWarnings({"rawtypes"})
NamedList collationList = new NamedList();
for (SpellCheckCollation collation : collations) {
if (collationExtendedResults) {
@SuppressWarnings({"rawtypes"})
NamedList extendedResult = new SimpleOrderedMap();
extendedResult.add("collationQuery", collation.getCollationQuery());
extendedResult.add("hits", collation.getHits());
@ -322,7 +325,8 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
response.add("collations", collationList);
}
private void addOriginalTermsToResponse(NamedList response, Collection<Token> originalTerms) {
@SuppressWarnings({"unchecked"})
private void addOriginalTermsToResponse(@SuppressWarnings({"rawtypes"})NamedList response, Collection<Token> originalTerms) {
List<String> originalTermStr = new ArrayList<String>();
for(Token t : originalTerms) {
originalTermStr.add(t.toString());
@ -397,6 +401,7 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
if (maxResultsForSuggest==null || !isCorrectlySpelled) {
for (ShardRequest sreq : rb.finished) {
for (ShardResponse srsp : sreq.responses) {
@SuppressWarnings({"rawtypes"})
NamedList nl = null;
try {
nl = (NamedList) srsp.getSolrResponse().getResponse().get("spellcheck");
@ -424,8 +429,10 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
SolrSpellChecker checker = getSpellChecker(rb.req.getParams());
SpellingResult result = checker.mergeSuggestions(mergeData, numSug, count, extendedResults);
@SuppressWarnings({"rawtypes"})
NamedList response = new SimpleOrderedMap();
@SuppressWarnings({"rawtypes"})
NamedList suggestions = toNamedList(false, result, origQuery, extendedResults);
response.add("suggestions", suggestions);
@ -438,12 +445,14 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
.toArray(new SpellCheckCollation[mergeData.collations.size()]);
Arrays.sort(sortedCollations);
@SuppressWarnings({"rawtypes"})
NamedList collations = new NamedList();
int i = 0;
while (i < maxCollations && i < sortedCollations.length) {
SpellCheckCollation collation = sortedCollations[i];
i++;
if (collationExtendedResults) {
@SuppressWarnings({"rawtypes"})
SimpleOrderedMap extendedResult = new SimpleOrderedMap();
extendedResult.add("collationQuery", collation.getCollationQuery());
extendedResult.add("hits", collation.getHits());
@ -462,7 +471,7 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
}
@SuppressWarnings("unchecked")
private void collectShardSuggestions(NamedList nl, SpellCheckMergeData mergeData) {
private void collectShardSuggestions(@SuppressWarnings({"rawtypes"})NamedList nl, SpellCheckMergeData mergeData) {
SpellCheckResponse spellCheckResp = new SpellCheckResponse(nl);
Iterable<Object> originalTermStrings = (Iterable<Object>) nl.get("originalTerms");
if(originalTermStrings!=null) {
@ -515,8 +524,9 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
}
@SuppressWarnings("unchecked")
private void collectShardCollations(SpellCheckMergeData mergeData, NamedList spellCheckResponse, int maxCollationTries) {
private void collectShardCollations(SpellCheckMergeData mergeData, @SuppressWarnings({"rawtypes"})NamedList spellCheckResponse, int maxCollationTries) {
Map<String, SpellCheckCollation> collations = mergeData.collations;
@SuppressWarnings({"rawtypes"})
NamedList collationHolder = (NamedList) spellCheckResponse.get("collations");
if(collationHolder != null) {
List<Object> collationList = collationHolder.getAll("collation");
@ -542,6 +552,7 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
collations.put(coll.getCollationQuery(), coll);
} else
{
@SuppressWarnings({"rawtypes"})
NamedList expandedCollation = (NamedList) o;
SpellCheckCollation coll = new SpellCheckCollation();
coll.setCollationQuery((String) expandedCollation.get("collationQuery"));
@ -636,6 +647,7 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
return spellCheckers.get(name);
}
@SuppressWarnings({"unchecked", "rawtypes"})
protected NamedList toNamedList(boolean shardRequest,
SpellingResult spellingResult, String origQuery, boolean extendedResults) {
NamedList result = new NamedList();
@ -704,6 +716,7 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
}
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public void inform(SolrCore core) {
if (initParams != null) {
log.info("Initializing spell checkers");
@ -748,7 +761,7 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
}
}
private boolean addSpellChecker(SolrCore core, boolean hasDefault, NamedList spellchecker) {
@SuppressWarnings({"rawtypes"})private boolean addSpellChecker(SolrCore core, boolean hasDefault, @SuppressWarnings({"rawtypes"})NamedList spellchecker) {
String className = (String) spellchecker.get("classname");
if (className == null) className = (String) spellchecker.get("class");
// TODO: this is a little bit sneaky: warn if class isnt supplied
@ -805,7 +818,7 @@ public class SpellCheckComponent extends SearchComponent implements SolrCoreAwar
}
@Override
public void init(NamedList args) {
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
}
@Override

View File

@ -28,6 +28,7 @@ import org.apache.solr.common.util.StrUtils;
class StandaloneReplicaSource implements ReplicaSource {
private List<String>[] replicas;
@SuppressWarnings({"unchecked", "rawtypes"})
public StandaloneReplicaSource(Builder builder) {
List<String> list = StrUtils.splitSmart(builder.shardsParam, ",", true);
replicas = new List[list.size()];

View File

@ -84,6 +84,7 @@ public class StatsComponent extends SearchComponent {
}
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public void handleResponses(ResponseBuilder rb, ShardRequest sreq) {
if (!rb.doStats || (sreq.purpose & ShardRequest.PURPOSE_GET_STATS) == 0) return;
@ -107,6 +108,7 @@ public class StatsComponent extends SearchComponent {
for (int i = 0; i < stats_fields.size(); i++) {
String key = stats_fields.getName(i);
StatsValues stv = allStatsValues.get(key);
@SuppressWarnings({"rawtypes"})
NamedList shardStv = (NamedList) stats_fields.get(key);
stv.accumulate(shardStv);
}
@ -149,6 +151,7 @@ public class StatsComponent extends SearchComponent {
for (Map.Entry<String, StatsValues> entry : statsValues.entrySet()) {
String key = entry.getKey();
@SuppressWarnings({"rawtypes"})
NamedList stv = entry.getValue().getStatsValues();
stats_fields.add(key, stv);
}

View File

@ -35,7 +35,7 @@ public interface StatsValues {
*
* @param stv NamedList whose values will be used to accumulate the current values
*/
void accumulate(NamedList stv);
void accumulate(@SuppressWarnings({"rawtypes"})NamedList stv);
/** Accumulate the value associated with <code>docID</code>.
* @see #setNextReader(org.apache.lucene.index.LeafReaderContext) */

View File

@ -128,6 +128,7 @@ public class StatsValuesFactory {
* Context to use when retrieving FunctionValues, will be null until/unless
* {@link #setNextReader} is called at least once
*/
@SuppressWarnings({"rawtypes"})
private Map vsContext;
/**
* Values to collect, will be null until/unless {@link #setNextReader} is
@ -199,7 +200,8 @@ public class StatsValuesFactory {
}
@Override
public void accumulate(NamedList stv) {
@SuppressWarnings({"unchecked"})
public void accumulate(@SuppressWarnings({"rawtypes"})NamedList stv) {
if (computeCount) {
count += (Long) stv.get("count");
}
@ -232,6 +234,7 @@ public class StatsValuesFactory {
updateTypeSpecificStats(stv);
@SuppressWarnings({"rawtypes"})
NamedList f = (NamedList) stv.get(FACETS);
if (f == null) {
return;
@ -239,6 +242,7 @@ public class StatsValuesFactory {
for (int i = 0; i < f.size(); i++) {
String field = f.getName(i);
@SuppressWarnings({"rawtypes"})
NamedList vals = (NamedList) f.getVal(i);
Map<String, StatsValues> addTo = facets.get(field);
if (addTo == null) {
@ -258,6 +262,7 @@ public class StatsValuesFactory {
}
@Override
@SuppressWarnings({"unchecked"})
public void accumulate(BytesRef value, int count) {
if (null == ft) {
throw new IllegalStateException(
@ -409,7 +414,7 @@ public class StatsValuesFactory {
* List containing values the current statistics should be updated
* against
*/
protected abstract void updateTypeSpecificStats(NamedList stv);
protected abstract void updateTypeSpecificStats(@SuppressWarnings({"rawtypes"})NamedList stv);
/**
* Add any type specific statistics to the given NamedList
@ -483,7 +488,7 @@ public class StatsValuesFactory {
}
@Override
public void updateTypeSpecificStats(NamedList stv) {
public void updateTypeSpecificStats(@SuppressWarnings({"rawtypes"})NamedList stv) {
if (computeSum) {
sum += ((Number) stv.get("sum")).doubleValue();
}
@ -645,7 +650,7 @@ public class StatsValuesFactory {
}
@Override
protected void updateTypeSpecificStats(NamedList stv) {
protected void updateTypeSpecificStats(@SuppressWarnings({"rawtypes"})NamedList stv) {
// No type specific stats
}
@ -696,7 +701,7 @@ public class StatsValuesFactory {
}
@Override
protected void updateTypeSpecificStats(NamedList stv) {
protected void updateTypeSpecificStats(@SuppressWarnings({"rawtypes"})NamedList stv) {
if (computeSum) {
sum += ((Number) stv.get("sum")).doubleValue();
}
@ -796,7 +801,7 @@ public class StatsValuesFactory {
}
@Override
protected void updateTypeSpecificStats(NamedList stv) {
protected void updateTypeSpecificStats(@SuppressWarnings({"rawtypes"})NamedList stv) {
// No type specific stats
}

View File

@ -84,8 +84,8 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
/** SolrConfig label to identify boolean value to build suggesters on startup */
private static final String BUILD_ON_STARTUP_LABEL = "buildOnStartup";
@SuppressWarnings("unchecked")
@SuppressWarnings({"rawtypes"})
protected NamedList initParams;
/**
@ -105,7 +105,7 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
@Override
@SuppressWarnings("unchecked")
public void init(NamedList args) {
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
super.init(args);
this.initParams = args;
}
@ -117,6 +117,7 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
boolean hasDefault = false;
for (int i = 0; i < initParams.size(); i++) {
if (initParams.getName(i).equals(CONFIG_PARAM_LABEL)) {
@SuppressWarnings({"rawtypes"})
NamedList suggesterParams = (NamedList) initParams.getVal(i);
SolrSuggester suggester = new SolrSuggester();
String dictionary = suggester.init(suggesterParams, core);
@ -501,7 +502,7 @@ public class SuggestComponent extends SearchComponent implements SolrCoreAware,
}
@Override
public void init(NamedList args) {}
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {}
@Override
public void newSearcher(SolrIndexSearcher newSearcher,

View File

@ -84,6 +84,7 @@ public class TermVectorComponent extends SearchComponent implements SolrCoreAwar
private static final String TV_KEY_WARNINGS = "warnings";
@SuppressWarnings({"rawtypes"})
protected NamedList initParams;
/**
@ -412,6 +413,8 @@ public class TermVectorComponent extends SearchComponent implements SolrCoreAwar
if (rb.stage == ResponseBuilder.STAGE_GET_FIELDS) {
NamedList<Object> termVectorsNL = new NamedList<>();
@SuppressWarnings({"unchecked", "rawtypes"})
Map.Entry<String, Object>[] arr = new NamedList.NamedListEntry[rb.resultIds.size()];
for (ShardRequest sreq : rb.finished) {
@ -419,6 +422,7 @@ public class TermVectorComponent extends SearchComponent implements SolrCoreAwar
continue;
}
for (ShardResponse srsp : sreq.responses) {
@SuppressWarnings({"unchecked"})
NamedList<Object> nl = (NamedList<Object>)srsp.getSolrResponse().getResponse().get(TERM_VECTORS);
// Add metadata (that which isn't a uniqueKey value):
@ -450,7 +454,7 @@ public class TermVectorComponent extends SearchComponent implements SolrCoreAwar
//////////////////////// NamedListInitializedPlugin methods //////////////////////
@Override
public void init(NamedList args) {
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
super.init(args);
this.initParams = args;
}

View File

@ -89,7 +89,7 @@ public class TermsComponent extends SearchComponent {
!HttpShardHandlerFactory.doGetDisableShardsWhitelist());
@Override
public void init( NamedList args )
public void init( @SuppressWarnings({"rawtypes"})NamedList args )
{
super.init(args);
whitelistHostChecker = new WhitelistHostChecker(
@ -415,6 +415,7 @@ public class TermsComponent extends SearchComponent {
th.parse(terms);
@SuppressWarnings({"unchecked"})
NamedList<Number> stats = (NamedList<Number>)srsp.getSolrResponse().getResponse().get("indexstats");
if(stats != null) {
th.numDocs += stats.get("numDocs").longValue();
@ -431,6 +432,7 @@ public class TermsComponent extends SearchComponent {
}
TermsHelper ti = rb._termsHelper;
@SuppressWarnings({"rawtypes"})
NamedList terms = ti.buildResponse();
rb.rsp.add("terms", terms);

View File

@ -219,7 +219,7 @@ public class JsonLoader extends ContentStreamLoader {
JsonRecordReader jsonRecordReader = JsonRecordReader.getInst(split, Arrays.asList(fields));
jsonRecordReader.streamRecords(parser, new JsonRecordReader.Handler() {
ArrayList docs = null;
ArrayList<Map<String, Object>> docs = null;
@Override
public void handle(Map<String, Object> record, String path) {
@ -227,7 +227,7 @@ public class JsonLoader extends ContentStreamLoader {
if (echo) {
if (docs == null) {
docs = new ArrayList();
docs = new ArrayList<>();
rsp.add("docs", docs);
}
changeChildDoc(copy);
@ -247,6 +247,7 @@ public class JsonLoader extends ContentStreamLoader {
});
}
@SuppressWarnings({"unchecked", "rawtypes"})
private SolrInputDocument buildDoc(Map<String, Object> m) {
SolrInputDocument result = new SolrInputDocument();
for (Map.Entry<String, Object> e : m.entrySet()) {
@ -272,6 +273,7 @@ public class JsonLoader extends ContentStreamLoader {
return result;
}
@SuppressWarnings({"unchecked", "rawtypes"})
private Map<String, Object> getDocMap(Map<String, Object> record, JSONParser parser, String srcField, boolean mapUniqueKeyOnly) {
Map result = record;
if (srcField != null && parser instanceof RecordingJSONParser) {
@ -418,6 +420,7 @@ public class JsonLoader extends ContentStreamLoader {
void parseCommitOptions(CommitUpdateCommand cmd) throws IOException {
assertNextEvent(JSONParser.OBJECT_START);
@SuppressWarnings({"unchecked"})
final Map<String, Object> map = (Map) ObjectBuilder.getVal(parser);
// SolrParams currently expects string values...
@ -583,9 +586,11 @@ public class JsonLoader extends ContentStreamLoader {
}
}
@SuppressWarnings({"unchecked"})
private List<Object> parseArrayFieldValue(int ev, String fieldName) throws IOException {
assert ev == JSONParser.ARRAY_START;
@SuppressWarnings({"rawtypes"})
ArrayList lst = new ArrayList(2);
for (; ; ) {
ev = parser.nextEvent();
@ -622,6 +627,7 @@ public class JsonLoader extends ContentStreamLoader {
private boolean mapEntryIsChildDoc(Object val) {
if(val instanceof List) {
@SuppressWarnings({"rawtypes"})
List listVal = (List) val;
if (listVal.size() == 0) return false;
return listVal.get(0) instanceof Map;
@ -630,6 +636,7 @@ public class JsonLoader extends ContentStreamLoader {
}
}
@SuppressWarnings({"unchecked", "rawtypes"})
private static Object changeChildDoc(Object o) {
if (o instanceof List) {
return ((List) o)

View File

@ -385,6 +385,7 @@ public class XMLLoader extends ContentStreamLoader {
*
* @since solr 1.3
*/
@SuppressWarnings({"unchecked"})
public SolrInputDocument readDoc(XMLStreamReader parser) throws XMLStreamException {
SolrInputDocument doc = new SolrInputDocument();
@ -445,6 +446,7 @@ public class XMLLoader extends ContentStreamLoader {
} else {
// multiple val are present
if (val instanceof List) {
@SuppressWarnings({"rawtypes"})
List list = (List) val;
list.add(v);
} else {

View File

@ -89,6 +89,7 @@ class SolrAggregate extends Aggregate implements SolrRel {
}
}
@SuppressWarnings({"fallthrough"})
private Pair<String, String> toSolrMetric(Implementor implementor, AggregateCall aggCall, List<String> inNames) {
SqlAggFunction aggregation = aggCall.getAggregation();
List<Integer> args = aggCall.getArgList();

View File

@ -33,6 +33,7 @@ class SolrEnumerator implements Enumerator<Object> {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
private final TupleStream tupleStream;
@SuppressWarnings({"rawtypes"})
private final List<Map.Entry<String, Class>> fields;
private Tuple current;
private char sep = 31;
@ -42,6 +43,7 @@ class SolrEnumerator implements Enumerator<Object> {
* @param tupleStream Solr TupleStream
* @param fields Fields to get from each Tuple
*/
@SuppressWarnings({"rawtypes"})
SolrEnumerator(TupleStream tupleStream, List<Map.Entry<String, Class>> fields) {
this.tupleStream = tupleStream;
@ -72,6 +74,7 @@ class SolrEnumerator implements Enumerator<Object> {
}
}
@SuppressWarnings({"rawtypes"})
private Object getter(Tuple tuple, Map.Entry<String, Class> field) {
Object val = tuple.get(field.getKey());

View File

@ -97,11 +97,11 @@ class SolrFilter extends Filter implements SolrRel {
}
private String translateAnd(RexNode node0) {
List<String> andStrings = new ArrayList();
List<String> notStrings = new ArrayList();
List<String> andStrings = new ArrayList<>();
List<String> notStrings = new ArrayList<>();
List<RexNode> ands = new ArrayList();
List<RexNode> nots = new ArrayList();
List<RexNode> ands = new ArrayList<>();
List<RexNode> nots = new ArrayList<>();
RelOptUtil.decomposeConjunction(node0, ands, nots);
@ -253,11 +253,11 @@ class SolrFilter extends Filter implements SolrRel {
}
private String translateAnd(RexNode node0) {
List<String> andStrings = new ArrayList();
List<String> notStrings = new ArrayList();
List<String> andStrings = new ArrayList<>();
List<String> notStrings = new ArrayList<>();
List<RexNode> ands = new ArrayList();
List<RexNode> nots = new ArrayList();
List<RexNode> ands = new ArrayList<>();
List<RexNode> nots = new ArrayList<>();
RelOptUtil.decomposeConjunction(node0, ands, nots);

View File

@ -38,6 +38,7 @@ enum SolrMethod {
public final Method method;
@SuppressWarnings({"rawtypes"})
SolrMethod(Class clazz, String methodName, Class... argumentTypes) {
this.method = Types.lookupMethod(clazz, methodName, argumentTypes);
}

View File

@ -97,7 +97,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
* @return Enumerator of results
*/
private Enumerable<Object> query(final Properties properties,
final List<Map.Entry<String, Class>> fields,
@SuppressWarnings("rawtypes") final List<Map.Entry<String, Class>> fields,
final String query,
final List<Pair<String, String>> orders,
final List<String> buckets,
@ -251,6 +251,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
}
}
@SuppressWarnings({"rawtypes"})
private TupleStream handleSelect(String zk,
String collection,
String query,
@ -318,6 +319,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
return buf.toString();
}
@SuppressWarnings({"rawtypes"})
private String getFields(List<Map.Entry<String, Class>> fields) {
StringBuilder buf = new StringBuilder();
for(Map.Entry<String, Class> field : fields) {
@ -348,8 +350,9 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
}
@SuppressWarnings({"unchecked", "rawtypes"})
private Set<String> getFieldSet(Metric[] metrics, List<Map.Entry<String, Class>> fields) {
HashSet set = new HashSet();
HashSet set = new HashSet<>();
for(Metric metric : metrics) {
for(String column : metric.getColumns()) {
set.add(column);
@ -423,6 +426,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
return true;
}
@SuppressWarnings({"rawtypes"})
private TupleStream handleGroupByMapReduce(String zk,
String collection,
Properties properties,
@ -434,7 +438,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
final String limit,
final String havingPredicate) throws IOException {
Map<String, Class> fmap = new HashMap();
Map<String, Class> fmap = new HashMap<>();
for(Map.Entry<String, Class> entry : fields) {
fmap.put(entry.getKey(), entry.getValue());
}
@ -549,6 +553,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
return tupleStream;
}
@SuppressWarnings({"rawtypes"})
private Bucket[] buildBuckets(List<String> buckets, List<Map.Entry<String, Class>> fields) {
Bucket[] bucketsArray = new Bucket[buckets.size()];
@ -563,6 +568,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
return bucketsArray;
}
@SuppressWarnings({"rawtypes"})
private TupleStream handleGroupByFacet(String zkHost,
String collection,
final List<Map.Entry<String, Class>> fields,
@ -574,7 +580,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
final String havingPredicate) throws IOException {
Map<String, Class> fmap = new HashMap();
Map<String, Class> fmap = new HashMap<>();
for(Map.Entry<String, Class> f : fields) {
fmap.put(f.getKey(), f.getValue());
}
@ -653,6 +659,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
return tupleStream;
}
@SuppressWarnings({"rawtypes"})
private TupleStream handleSelectDistinctMapReduce(final String zkHost,
final String collection,
final Properties properties,
@ -761,9 +768,9 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
private StreamComparator[] adjustSorts(List<Pair<String, String>> orders, Bucket[] buckets) throws IOException {
List<FieldComparator> adjustedSorts = new ArrayList();
Set<String> bucketFields = new HashSet();
Set<String> sortFields = new HashSet();
List<FieldComparator> adjustedSorts = new ArrayList<>();
Set<String> bucketFields = new HashSet<>();
Set<String> sortFields = new HashSet<>();
ComparatorOrder comparatorOrder = ComparatorOrder.ASCENDING;
for(Pair<String, String> order : orders) {
@ -795,6 +802,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
return adjustedSorts.toArray(new FieldComparator[adjustedSorts.size()]);
}
@SuppressWarnings({"rawtypes"})
private TupleStream handleStats(String zk,
String collection,
String query,
@ -802,7 +810,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
List<Map.Entry<String, Class>> fields) throws IOException {
Map<String, Class> fmap = new HashMap();
Map<String, Class> fmap = new HashMap<>();
for(Map.Entry<String, Class> entry : fields) {
fmap.put(entry.getKey(), entry.getValue());
}
@ -854,7 +862,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
*
* @see SolrMethod#SOLR_QUERYABLE_QUERY
*/
@SuppressWarnings("UnusedDeclaration")
@SuppressWarnings({"rawtypes","UnusedDeclaration"})
public Enumerable<Object> query(List<Map.Entry<String, Class>> fields, String query, List<Pair<String, String>> order,
List<String> buckets, List<Pair<String, String>> metricPairs, String limit, String negativeQuery, String havingPredicate) {
return getTable().query(getProperties(), fields, query, order, buckets, metricPairs, limit, negativeQuery, havingPredicate);

View File

@ -62,12 +62,14 @@ class SolrToEnumerableConverter extends ConverterImpl implements EnumerableRel {
final RelDataType rowType = getRowType();
final PhysType physType = PhysTypeImpl.of(implementor.getTypeFactory(), rowType, pref.prefer(JavaRowFormat.ARRAY));
final Expression table = list.append("table", solrImplementor.table.getExpression(SolrTable.SolrQueryable.class));
@SuppressWarnings({"rawtypes"})
final Expression fields =
list.append("fields",
constantArrayList(
Pair.zip(generateFields(SolrRules.solrFieldNames(rowType), solrImplementor.fieldMappings),
new AbstractList<Class>() {
@Override
@SuppressWarnings({"rawtypes"})
public Class get(int index) {
return physType.fieldClass(index);
}
@ -122,6 +124,7 @@ class SolrToEnumerableConverter extends ConverterImpl implements EnumerableRel {
* E.g. {@code constantArrayList("x", "y")} returns
* "Arrays.asList('x', 'y')".
*/
@SuppressWarnings({"rawtypes"})
private static <T> MethodCallExpression constantArrayList(List<T> values, Class clazz) {
return Expressions.call(BuiltInMethod.ARRAYS_AS_LIST.method,
Expressions.newArrayInit(clazz, constantList(values)));