SOLR-14547: Fix or suppress warnings in solr/client/solrj/io/stream

This commit is contained in:
Erick Erickson 2020-06-09 19:16:14 -04:00
parent 7b5d2442ac
commit c8f48d4701
45 changed files with 204 additions and 93 deletions

View File

@ -39,7 +39,7 @@ if (!hasDefaults) {
"# These settings have been generated automatically on the first run.",
"# See gradlew :helpLocalSettings for more information.",
"systemProp.file.encoding=UTF-8",
"org.gradle.jvmargs=-Xmx2g",
"org.gradle.jvmargs=-Xmx3g", // TODO figure out why "gradlew check" runs out of memory if 2g
"org.gradle.parallel=true",
"org.gradle.priority=normal",
"org.gradle.warning.mode=none", // Silence gradle warnings. We'll deal with them when we upgrade the wrapper.

View File

@ -319,6 +319,8 @@ Other Changes
* SOLR-14548: Address warning: static member should be qualified by type name (Mike Drob)
* SOLR-14547: Fix or suppress warnings in solr/client/solrj/io/stream
================== 8.5.2 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

View File

@ -199,6 +199,7 @@ public class CartesianProductStream extends TupleStream implements Expressible {
return generatedTuples.pop();
}
@SuppressWarnings({"unchecked"})
private LinkedList<Tuple> generateTupleList(Tuple original) throws IOException{
Map<String, Object> evaluatedValues = new HashMap<>();

View File

@ -121,7 +121,7 @@ public class CellStream extends TupleStream implements Expressible {
public void open() throws IOException {
try {
stream.open();
List<Tuple> list = new ArrayList();
List<Tuple> list = new ArrayList<>();
while(true) {
Tuple tuple = stream.read();
if(tuple.EOF) {

View File

@ -272,15 +272,15 @@ public class CloudSolrStream extends TupleStream implements Expressible {
*
***/
public void open() throws IOException {
this.tuples = new TreeSet();
this.solrStreams = new ArrayList();
this.eofTuples = Collections.synchronizedMap(new HashMap());
this.tuples = new TreeSet<>();
this.solrStreams = new ArrayList<>();
this.eofTuples = Collections.synchronizedMap(new HashMap<>());
constructStreams();
openStreams();
}
public Map getEofTuples() {
public Map<String, Tuple> getEofTuples() {
return this.eofTuples;
}
@ -288,9 +288,11 @@ public class CloudSolrStream extends TupleStream implements Expressible {
return solrStreams;
}
@SuppressWarnings({"unchecked"})
private StreamComparator parseComp(String sort, String fl) throws IOException {
String[] fls = fl.split(",");
@SuppressWarnings({"rawtypes"})
HashSet fieldSet = new HashSet();
for(String f : fls) {
fieldSet.add(f.trim()); //Handle spaces in the field list.
@ -339,7 +341,7 @@ public class CloudSolrStream extends TupleStream implements Expressible {
// check for alias or collection
List<String> allCollections = new ArrayList();
List<String> allCollections = new ArrayList<>();
String[] collectionNames = collectionName.split(",");
for(String col : collectionNames) {
List<String> collections = checkAlias
@ -397,7 +399,7 @@ public class CloudSolrStream extends TupleStream implements Expressible {
private void openStreams() throws IOException {
ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("CloudSolrStream"));
try {
List<Future<TupleWrapper>> futures = new ArrayList();
List<Future<TupleWrapper>> futures = new ArrayList<>();
for (TupleStream solrStream : solrStreams) {
StreamOpener so = new StreamOpener((SolrStream) solrStream, comp);
Future<TupleWrapper> future = service.submit(so);
@ -461,6 +463,7 @@ public class CloudSolrStream extends TupleStream implements Expressible {
}
}
@SuppressWarnings({"overrides"})
protected class TupleWrapper implements Comparable<TupleWrapper> {
private Tuple tuple;
private SolrStream stream;

View File

@ -168,6 +168,7 @@ public class DaemonStream extends TupleStream implements Expressible {
init(tupleStream, id, runInterval, queueSize, false);
}
@SuppressWarnings({"unchecked", "rawtypes"})
public void init(TupleStream tupleStream, String id, long runInterval, int queueSize, boolean terminate) {
this.tupleStream = tupleStream;
this.id = id;
@ -245,7 +246,7 @@ public class DaemonStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
List<TupleStream> children = new ArrayList();
List<TupleStream> children = new ArrayList<>();
children.add(tupleStream);
return children;
}

View File

@ -257,9 +257,9 @@ public class DeepRandomStream extends TupleStream implements Expressible {
}
public void open() throws IOException {
this.tuples = new LinkedList();
this.solrStreams = new ArrayList();
this.eofTuples = Collections.synchronizedMap(new HashMap());
this.tuples = new LinkedList<>();
this.solrStreams = new ArrayList<>();
this.eofTuples = Collections.synchronizedMap(new HashMap<>());
constructStreams();
openStreams();
}
@ -278,7 +278,7 @@ public class DeepRandomStream extends TupleStream implements Expressible {
// check for alias or collection
List<String> allCollections = new ArrayList();
List<String> allCollections = new ArrayList<>();
String[] collectionNames = collectionName.split(",");
for(String col : collectionNames) {
List<String> collections = checkAlias
@ -349,7 +349,7 @@ public class DeepRandomStream extends TupleStream implements Expressible {
private void openStreams() throws IOException {
ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("DeepRandomStream"));
try {
List<Future<TupleWrapper>> futures = new ArrayList();
List<Future<TupleWrapper>> futures = new ArrayList<>();
for (TupleStream solrStream : solrStreams) {
StreamOpener so = new StreamOpener((SolrStream) solrStream, comp);
Future<TupleWrapper> future = service.submit(so);
@ -411,6 +411,7 @@ public class DeepRandomStream extends TupleStream implements Expressible {
}
}
@SuppressWarnings({"overrides"})
protected class TupleWrapper implements Comparable<TupleWrapper> {
private Tuple tuple;
private SolrStream stream;

View File

@ -100,7 +100,7 @@ public class EvalStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
List<TupleStream> l = new ArrayList<>();
l.add(stream);
return l;
}

View File

@ -132,7 +132,7 @@ public class ExecutorStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
List<TupleStream> l = new ArrayList<>();
l.add(stream);
return l;
}
@ -153,6 +153,7 @@ public class ExecutorStream extends TupleStream implements Expressible {
}
public Tuple read() throws IOException {
@SuppressWarnings({"unchecked", "rawtypes"})
ArrayBlockingQueue<Tuple> queue = new ArrayBlockingQueue(10000);
while(true) {
Tuple tuple = stream.read();
@ -183,7 +184,8 @@ public class ExecutorStream extends TupleStream implements Expressible {
private StreamFactory streamFactory;
private StreamContext streamContext;
public StreamTask(ArrayBlockingQueue queue, StreamFactory streamFactory, StreamContext streamContext) {
@SuppressWarnings({"unchecked"})
public StreamTask(@SuppressWarnings({"rawtypes"})ArrayBlockingQueue queue, StreamFactory streamFactory, StreamContext streamContext) {
this.queue = queue;
this.streamFactory = streamFactory;
this.streamContext = new StreamContext();

View File

@ -271,7 +271,7 @@ public class Facet2DStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
return new ArrayList();
return new ArrayList<>();
}
public void open() throws IOException {
@ -293,6 +293,7 @@ public class Facet2DStream extends TupleStream implements Expressible {
QueryRequest request = new QueryRequest(paramsLoc, SolrRequest.METHOD.POST);
try {
@SuppressWarnings({"rawtypes"})
NamedList response = cloudSolrClient.request(request, collection);
getTuples(response, x, y, metric);
this.out = tuples.iterator();
@ -389,21 +390,26 @@ public class Facet2DStream extends TupleStream implements Expressible {
return null;
}
private void getTuples(NamedList response, Bucket x, Bucket y, Metric metric) {
private void getTuples(@SuppressWarnings({"rawtypes"})NamedList response, Bucket x, Bucket y, Metric metric) {
Tuple tuple = new Tuple();
@SuppressWarnings({"rawtypes"})
NamedList facets = (NamedList) response.get("facets");
fillTuples(0, tuples, tuple, facets, x, y, metric);
}
private void fillTuples(int level, List<Tuple> tuples, Tuple currentTuple, NamedList facets, Bucket x, Bucket y, Metric metric) {
private void fillTuples(int level, List<Tuple> tuples, Tuple currentTuple,
@SuppressWarnings({"rawtypes"})NamedList facets, Bucket x, Bucket y, Metric metric) {
String bucketXName = x.toString();
String bucketYName = y.toString();
@SuppressWarnings({"rawtypes"})
NamedList allXBuckets = (NamedList) facets.get("x");
for (int b = 0; b < allXBuckets.size(); b++) {
@SuppressWarnings({"rawtypes"})
List buckets = (List) allXBuckets.get("buckets");
for(int s=0; s<buckets.size(); s++) {
@SuppressWarnings({"rawtypes"})
NamedList bucket = (NamedList)buckets.get(s);
Object val = bucket.get("val");
if (val instanceof Integer) {
@ -412,10 +418,13 @@ public class Facet2DStream extends TupleStream implements Expressible {
Tuple tx = currentTuple.clone();
tx.put(bucketXName, val);
@SuppressWarnings({"rawtypes"})
NamedList allYBuckets = (NamedList) bucket.get("y");
@SuppressWarnings({"rawtypes"})
List ybuckets = (List)allYBuckets.get("buckets");
for (int d = 0; d < ybuckets.size(); d++) {
@SuppressWarnings({"rawtypes"})
NamedList bucketY = (NamedList) ybuckets.get(d);
Object valY = bucketY.get("val");
if (valY instanceof Integer) {

View File

@ -373,7 +373,7 @@ public class FacetStream extends TupleStream implements Expressible {
}
private String[] parseSorts(String sortString) {
List<String> sorts = new ArrayList();
List<String> sorts = new ArrayList<>();
boolean inParam = false;
StringBuilder buff = new StringBuilder();
for(int i=0; i<sortString.length(); i++) {
@ -533,7 +533,7 @@ public class FacetStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
return new ArrayList();
return new ArrayList<>();
}
public void open() throws IOException {
@ -556,6 +556,7 @@ public class FacetStream extends TupleStream implements Expressible {
QueryRequest request = new QueryRequest(paramsLoc, SolrRequest.METHOD.POST);
try {
@SuppressWarnings({"rawtypes"})
NamedList response = cloudSolrClient.request(request, collection);
getTuples(response, buckets, metrics);
@ -761,11 +762,12 @@ public class FacetStream extends TupleStream implements Expressible {
return "index";
}
private void getTuples(NamedList response,
private void getTuples(@SuppressWarnings({"rawtypes"})NamedList response,
Bucket[] buckets,
Metric[] metrics) {
Tuple tuple = new Tuple();
@SuppressWarnings({"rawtypes"})
NamedList facets = (NamedList)response.get("facets");
fillTuples(0,
tuples,
@ -779,17 +781,20 @@ public class FacetStream extends TupleStream implements Expressible {
private void fillTuples(int level,
List<Tuple> tuples,
Tuple currentTuple,
NamedList facets,
@SuppressWarnings({"rawtypes"}) NamedList facets,
Bucket[] _buckets,
Metric[] _metrics) {
String bucketName = _buckets[level].toString();
@SuppressWarnings({"rawtypes"})
NamedList nl = (NamedList)facets.get(bucketName);
if(nl == null) {
return;
}
@SuppressWarnings({"rawtypes"})
List allBuckets = (List)nl.get("buckets");
for(int b=0; b<allBuckets.size(); b++) {
@SuppressWarnings({"rawtypes"})
NamedList bucket = (NamedList)allBuckets.get(b);
Object val = bucket.get("val");
if (val instanceof Integer) {

View File

@ -91,7 +91,7 @@ public class FeaturesSelectionStream extends TupleStream implements Expressible{
public FeaturesSelectionStream(String zkHost,
String collectionName,
Map params,
@SuppressWarnings({"rawtypes"})Map params,
String field,
String outcome,
String featureSet,
@ -213,9 +213,10 @@ public class FeaturesSelectionStream extends TupleStream implements Expressible{
return expression;
}
@SuppressWarnings({"unchecked"})
private void init(String collectionName,
String zkHost,
Map params,
@SuppressWarnings({"rawtypes"})Map params,
String field,
String outcome,
String featureSet,
@ -288,6 +289,7 @@ public class FeaturesSelectionStream extends TupleStream implements Expressible{
}
}
@SuppressWarnings({"rawtypes"})
private List<Future<NamedList>> callShards(List<String> baseUrls) throws IOException {
List<Future<NamedList>> futures = new ArrayList<>();
@ -336,10 +338,13 @@ public class FeaturesSelectionStream extends TupleStream implements Expressible{
long numDocs = 0;
for (Future<NamedList> getTopTermsCall : callShards(getShardUrls())) {
for (@SuppressWarnings({"rawtypes"})Future<NamedList> getTopTermsCall : callShards(getShardUrls())) {
@SuppressWarnings({"rawtypes"})
NamedList resp = getTopTermsCall.get();
@SuppressWarnings({"unchecked"})
NamedList<Double> shardTopTerms = (NamedList<Double>)resp.get("featuredTerms");
@SuppressWarnings({"unchecked"})
NamedList<Integer> shardDocFreqs = (NamedList<Integer>)resp.get("docFreq");
numDocs += (Integer)resp.get("numDocs");
@ -397,6 +402,7 @@ public class FeaturesSelectionStream extends TupleStream implements Expressible{
return result;
}
@SuppressWarnings({"rawtypes"})
protected class FeaturesSelectionCall implements Callable<NamedList> {
private String baseUrl;
@ -415,6 +421,7 @@ public class FeaturesSelectionStream extends TupleStream implements Expressible{
this.paramsMap = paramsMap;
}
@SuppressWarnings({"unchecked"})
public NamedList<Double> call() throws Exception {
ModifiableSolrParams params = new ModifiableSolrParams();
HttpSolrClient solrClient = cache.getHttpSolrClient(baseUrl);

View File

@ -199,11 +199,12 @@ public class FetchStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
List<TupleStream> l = new ArrayList<>();
l.add(stream);
return l;
}
@SuppressWarnings({"unchecked", "rawtypes"})
public void open() throws IOException {
tuples = new ArrayList().iterator();
stream.open();

View File

@ -83,7 +83,7 @@ public class GetStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
List<TupleStream> l = new ArrayList<>();
return l;
}
@ -99,9 +99,11 @@ public class GetStream extends TupleStream implements Expressible {
public void close() throws IOException {
}
@SuppressWarnings({"unchecked"})
public void open() throws IOException {
Map<String, Object> lets = streamContext.getLets();
Object o = lets.get(name);
@SuppressWarnings({"rawtypes"})
List l = null;
if(o instanceof List) {
l = (List)o;

View File

@ -190,13 +190,15 @@ public class HashRollupStream extends TupleStream implements Expressible {
tupleIterator = null;
}
@SuppressWarnings({"unchecked"})
public Tuple read() throws IOException {
//On the first call to read build the tupleIterator.
if(tupleIterator == null) {
Map<HashKey, Metric[]> metricMap = new HashMap();
Map<HashKey, Metric[]> metricMap = new HashMap<>();
while (true) {
Tuple tuple = tupleStream.read();
if (tuple.EOF) {
@SuppressWarnings({"rawtypes"})
List tuples = new ArrayList();
for(Map.Entry<HashKey, Metric[]> entry : metricMap.entrySet()) {
Tuple t = new Tuple();

View File

@ -73,6 +73,7 @@ public class JSONTupleStream implements TupleStreamParser {
/** returns the next Tuple or null */
@Override
@SuppressWarnings({"unchecked"})
public Map<String,Object> next() throws IOException {
if (!atDocs) {
boolean found = advanceToDocs();

View File

@ -96,6 +96,7 @@ public class JavabinTupleStreamParser extends JavaBinCodec implements TupleStrea
return tagByte == SOLRDOCLST;
}
@SuppressWarnings({"unchecked", "rawtypes"})
private Map readAsMap(DataInputInputStream dis) throws IOException {
int sz = readSize(dis);
Map m = new LinkedHashMap<>();
@ -107,6 +108,7 @@ public class JavabinTupleStreamParser extends JavaBinCodec implements TupleStrea
return m;
}
@SuppressWarnings({"unchecked", "rawtypes"})
private Map readSolrDocumentAsMap(DataInputInputStream dis) throws IOException {
tagByte = dis.readByte();
int size = readSize(dis);
@ -174,6 +176,7 @@ public class JavabinTupleStreamParser extends JavaBinCodec implements TupleStrea
@Override
@SuppressWarnings({"unchecked"})
public Map<String, Object> next() throws IOException {
if (arraySize == 0) return null;
Object o = readVal(fis);

View File

@ -183,7 +183,7 @@ public class KnnStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
List<TupleStream> l = new ArrayList<>();
return l;
}

View File

@ -47,8 +47,10 @@ public class LetStream extends TupleStream implements Expressible {
private static final long serialVersionUID = 1;
private TupleStream stream;
private StreamContext streamContext;
@SuppressWarnings({"rawtypes"})
private Map letParams = new LinkedHashMap();
@SuppressWarnings({"unchecked", "rawtypes"})
public LetStream(StreamExpression expression, StreamFactory factory) throws IOException {
List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
@ -160,6 +162,7 @@ public class LetStream extends TupleStream implements Expressible {
stream.close();
}
@SuppressWarnings({"unchecked"})
public void open() throws IOException {
Map<String, Object> lets = streamContext.getLets();
Set<Map.Entry<String, Object>> entries = letParams.entrySet();
@ -169,7 +172,7 @@ public class LetStream extends TupleStream implements Expressible {
String name = entry.getKey();
Object o = entry.getValue();
if(o instanceof TupleStream) {
List<Tuple> tuples = new ArrayList();
List<Tuple> tuples = new ArrayList<>();
TupleStream tStream = (TupleStream)o;
tStream.setStreamContext(streamContext);
try {
@ -196,6 +199,7 @@ public class LetStream extends TupleStream implements Expressible {
evaluator.setStreamContext(streamContext);
Object eo = evaluator.evaluate(eTuple);
if(evaluator instanceof MemsetEvaluator) {
@SuppressWarnings({"rawtypes"})
Map mem = (Map)eo;
lets.putAll(mem);
} else {

View File

@ -162,7 +162,7 @@ public class ModelStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
List<TupleStream> l = new ArrayList<>();
return l;
}

View File

@ -136,7 +136,7 @@ public class ParallelListStream extends TupleStream implements Expressible {
private void openStreams() throws IOException {
ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("ParallelListStream"));
try {
List<Future<StreamIndex>> futures = new ArrayList();
List<Future<StreamIndex>> futures = new ArrayList<>();
int i=0;
for (TupleStream tupleStream : streams) {
StreamOpener so = new StreamOpener(new StreamIndex(tupleStream, i++));

View File

@ -206,7 +206,7 @@ public class ParallelStream extends CloudSolrStream implements Expressible {
}
public List<TupleStream> children() {
List l = new ArrayList();
List<TupleStream> l = new ArrayList<>();
l.add(tupleStream);
return l;
}

View File

@ -49,8 +49,8 @@ public class PlotStream extends TupleStream implements Expressible {
private Map<String,String> stringParams = new HashMap<>();
private Map<String,StreamEvaluator> evaluatorParams = new HashMap<>();
private Map<String,TupleStream> streamParams = new HashMap<>();
private List<String> fieldNames = new ArrayList();
private Map<String, String> fieldLabels = new HashMap();
private List<String> fieldNames = new ArrayList<>();
private Map<String, String> fieldLabels = new HashMap<>();
private boolean finished;
@ -148,6 +148,7 @@ public class PlotStream extends TupleStream implements Expressible {
return l;
}
@SuppressWarnings({"unchecked"})
public Tuple read() throws IOException {
if (finished) {
@ -177,15 +178,15 @@ public class PlotStream extends TupleStream implements Expressible {
if(x == null) {
//x is null so add a sequence
x = new ArrayList();
x = new ArrayList<>();
for(int i=0; i<y.size(); i++) {
x.add(i+1);
}
}
List<List<Number>> xy = new ArrayList();
List<List<Number>> xy = new ArrayList<>();
for(int i=0; i<x.size(); i++) {
List<Number> pair = new ArrayList();
List<Number> pair = new ArrayList<>();
pair.add(x.get(i));
pair.add(y.get(i));
xy.add(pair);

View File

@ -120,7 +120,7 @@ public class PriorityStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
List<TupleStream> l = new ArrayList<>();
l.add(highPriorityTasks);
l.add(tasks);
return l;

View File

@ -193,7 +193,7 @@ public class RandomStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
List<TupleStream> l = new ArrayList<>();
return l;
}

View File

@ -65,7 +65,7 @@ public class ScoreNodesStream extends TupleStream implements Expressible
protected String zkHost;
private TupleStream stream;
private transient SolrClientCache clientCache;
private Map<String, Tuple> nodes = new HashMap();
private Map<String, Tuple> nodes = new HashMap<>();
private Iterator<Tuple> tuples;
private String termFreq;
private boolean facet;
@ -165,7 +165,7 @@ public class ScoreNodesStream extends TupleStream implements Expressible
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
List<TupleStream> l = new ArrayList<>();
l.add(stream);
return l;
}
@ -222,9 +222,12 @@ public class ScoreNodesStream extends TupleStream implements Expressible
try {
//Get the response from the terms component
@SuppressWarnings({"rawtypes"})
NamedList response = client.request(request, collection);
@SuppressWarnings({"unchecked"})
NamedList<Number> stats = (NamedList<Number>)response.get("indexstats");
long numDocs = stats.get("numDocs").longValue();
@SuppressWarnings({"unchecked"})
NamedList<NamedList<Number>> fields = (NamedList<NamedList<Number>>)response.get("terms");
int size = fields.size();

View File

@ -174,7 +174,7 @@ public class SearchStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
List<TupleStream> l = new ArrayList<>();
return l;
}
@ -227,6 +227,7 @@ public class SearchStream extends TupleStream implements Expressible {
return comp;
}
@SuppressWarnings({"unchecked", "rawtypes"})
private StreamComparator parseComp(String sort, String fl) throws IOException {
HashSet fieldSet = null;

View File

@ -64,16 +64,17 @@ public class SelectStream extends TupleStream implements Expressible {
this.selectedFields.put(selectedField, selectedField);
}
operations = new ArrayList<>();
selectedEvaluators = new LinkedHashMap();
selectedEvaluators = new LinkedHashMap<>();
}
public SelectStream(TupleStream stream, Map<String,String> selectedFields) throws IOException {
this.stream = stream;
this.selectedFields = selectedFields;
operations = new ArrayList<>();
selectedEvaluators = new LinkedHashMap();
selectedEvaluators = new LinkedHashMap<>();
}
@SuppressWarnings({"unchecked"})
public SelectStream(StreamExpression expression,StreamFactory factory) throws IOException {
// grab all parameters out
List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
@ -101,7 +102,7 @@ public class SelectStream extends TupleStream implements Expressible {
stream = factory.constructStream(streamExpressions.get(0));
selectedFields = new HashMap<String,String>();
selectedEvaluators = new LinkedHashMap();
selectedEvaluators = new LinkedHashMap<>();
for(StreamExpressionParameter parameter : selectAsFieldsExpressions){
StreamExpressionValue selectField = (StreamExpressionValue)parameter;
String value = selectField.getValue().trim();
@ -236,7 +237,7 @@ public class SelectStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
List<TupleStream> l = new ArrayList<>();
l.add(stream);
return l;
}

View File

@ -77,7 +77,7 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
public SignificantTermsStream(String zkHost,
String collectionName,
Map params,
@SuppressWarnings({"rawtypes"})Map params,
String field,
float minDocFreq,
float maxDocFreq,
@ -202,9 +202,10 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
return expression;
}
@SuppressWarnings({"unchecked"})
private void init(String collectionName,
String zkHost,
Map params,
@SuppressWarnings({"rawtypes"})Map params,
String field,
float minDocFreq,
float maxDocFreq,
@ -240,6 +241,7 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
return null;
}
@SuppressWarnings({"rawtypes"})
private List<Future<NamedList>> callShards(List<String> baseUrls) throws IOException {
List<Future<NamedList>> futures = new ArrayList<>();
@ -252,6 +254,7 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
this.minTermLength,
this.numTerms);
@SuppressWarnings({"rawtypes"})
Future<NamedList> future = executorService.submit(lc);
futures.add(future);
}
@ -281,14 +284,17 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
.withExpression(toExpression(factory).toString());
}
@SuppressWarnings({"unchecked"})
public Tuple read() throws IOException {
try {
if (tupleIterator == null) {
Map<String, int[]> mergeFreqs = new HashMap<>();
long numDocs = 0;
long resultCount = 0;
for (Future<NamedList> getTopTermsCall : callShards(getShards(zkHost, collection, streamContext))) {
for (@SuppressWarnings({"rawtypes"})Future<NamedList> getTopTermsCall : callShards(getShards(zkHost, collection, streamContext))) {
@SuppressWarnings({"rawtypes"})
NamedList fullResp = getTopTermsCall.get();
@SuppressWarnings({"rawtypes"})
Map stResp = (Map)fullResp.get("significantTerms");
List<String> terms = (List<String>)stResp.get("sterms");
@ -313,10 +319,12 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
}
}
List<Map> maps = new ArrayList();
@SuppressWarnings({"rawtypes"})
List<Map> maps = new ArrayList<>();
for(Map.Entry<String, int[]> entry : mergeFreqs.entrySet()) {
int[] freqs = entry.getValue();
@SuppressWarnings({"rawtypes"})
Map map = new HashMap();
map.put("term", entry.getKey());
map.put("background", freqs[0]);
@ -329,8 +337,8 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
}
Collections.sort(maps, new ScoreComp());
List<Tuple> tuples = new ArrayList();
for (Map map : maps) {
List<Tuple> tuples = new ArrayList<>();
for (@SuppressWarnings({"rawtypes"})Map map : maps) {
if (tuples.size() == numTerms) break;
tuples.add(new Tuple(map));
}
@ -345,6 +353,7 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
}
}
@SuppressWarnings({"rawtypes"})
private static class ScoreComp implements Comparator<Map> {
public int compare(Map a, Map b) {
Float scorea = (Float)a.get("score");
@ -353,6 +362,7 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
}
}
@SuppressWarnings({"unchecked", "rawtypes"})
protected class SignificantTermsCall implements Callable<NamedList> {
private String baseUrl;
@ -380,6 +390,7 @@ public class SignificantTermsStream extends TupleStream implements Expressible{
this.minTermLength = minTermLength;
}
@SuppressWarnings({"unchecked", "rawtypes"})
public NamedList<Double> call() throws Exception {
ModifiableSolrParams params = new ModifiableSolrParams();
HttpSolrClient solrClient = cache.getHttpSolrClient(baseUrl);

View File

@ -85,7 +85,7 @@ public class SolrStream extends TupleStream {
}
public List<TupleStream> children() {
return new ArrayList();
return new ArrayList<>();
}
public String getBaseUrl() {
@ -196,8 +196,10 @@ public class SolrStream extends TupleStream {
* Reads a Tuple from the stream. The Stream is completed when Tuple.EOF == true.
**/
@SuppressWarnings({"unchecked"})
public Tuple read() throws IOException {
try {
@SuppressWarnings({"rawtypes"})
Map fields = tupleStreamParser.next();
if (fields == null) {
@ -250,6 +252,7 @@ public class SolrStream extends TupleStream {
return null;
}
@SuppressWarnings({"unchecked", "rawtypes"})
private Map mapFields(Map fields, Map<String,String> mappings) {
Iterator<Map.Entry<String,String>> it = mappings.entrySet().iterator();

View File

@ -208,7 +208,7 @@ public class StatsStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
return new ArrayList();
return new ArrayList<>();
}
public void open() throws IOException {
@ -219,11 +219,13 @@ public class StatsStream extends TupleStream implements Expressible {
paramsLoc.set("json.facet", json);
paramsLoc.set("rows", "0");
@SuppressWarnings({"unchecked"})
Map<String, List<String>> shardsMap = (Map<String, List<String>>)context.get("shards");
if(shardsMap == null) {
QueryRequest request = new QueryRequest(paramsLoc, SolrRequest.METHOD.POST);
cloudSolrClient = cache.getCloudSolrClient(zkHost);
try {
@SuppressWarnings({"rawtypes"})
NamedList response = cloudSolrClient.request(request, collection);
getTuples(response, metrics);
} catch (Exception e) {
@ -241,6 +243,7 @@ public class StatsStream extends TupleStream implements Expressible {
QueryRequest request = new QueryRequest(paramsLoc, SolrRequest.METHOD.POST);
try {
@SuppressWarnings({"rawtypes"})
NamedList response = client.request(request);
getTuples(response, metrics);
} catch (Exception e) {
@ -301,16 +304,17 @@ public class StatsStream extends TupleStream implements Expressible {
}
}
private void getTuples(NamedList response,
private void getTuples(@SuppressWarnings({"rawtypes"})NamedList response,
Metric[] metrics) {
this.tuple = new Tuple();
@SuppressWarnings({"rawtypes"})
NamedList facets = (NamedList)response.get("facets");
fillTuple(tuple, facets, metrics);
}
private void fillTuple(Tuple t,
NamedList nl,
@SuppressWarnings({"rawtypes"})NamedList nl,
Metric[] _metrics) {
if(nl == null) {

View File

@ -38,9 +38,12 @@ import org.apache.solr.common.params.SolrParams;
public class StreamContext implements Serializable {
@SuppressWarnings({"rawtypes"})
private Map entries = new HashMap();
@SuppressWarnings({"rawtypes"})
private Map tupleContext = new HashMap();
private Map<String, Object> lets = new HashMap();
private Map<String, Object> lets = new HashMap<>();
@SuppressWarnings({"rawtypes"})
private ConcurrentMap objectCache;
public int workerID;
public int numWorkers;
@ -51,11 +54,12 @@ public class StreamContext implements Serializable {
private SolrParams requestParams;
private RequestReplicaListTransformerGenerator requestReplicaListTransformerGenerator;
@SuppressWarnings({"rawtypes"})
public ConcurrentMap getObjectCache() {
return this.objectCache;
}
public void setObjectCache(ConcurrentMap objectCache) {
public void setObjectCache(@SuppressWarnings({"rawtypes"})ConcurrentMap objectCache) {
this.objectCache = objectCache;
}
@ -67,6 +71,7 @@ public class StreamContext implements Serializable {
return entries.get(key);
}
@SuppressWarnings({"unchecked"})
public void put(Object key, Object value) {
this.entries.put(key, value);
}
@ -75,6 +80,7 @@ public class StreamContext implements Serializable {
return entries.containsKey(key);
}
@SuppressWarnings({"rawtypes"})
public Map getEntries() {
return this.entries;
}
@ -99,6 +105,7 @@ public class StreamContext implements Serializable {
this.streamFactory = streamFactory;
}
@SuppressWarnings({"rawtypes"})
public Map getTupleContext() {
return tupleContext;
}

View File

@ -99,7 +99,7 @@ public class TextLogitStream extends TupleStream implements Expressible {
public TextLogitStream(String zkHost,
String collectionName,
Map params,
@SuppressWarnings({"rawtypes"})Map params,
String name,
String field,
TupleStream termsStream,
@ -283,9 +283,10 @@ public class TextLogitStream extends TupleStream implements Expressible {
return expression;
}
@SuppressWarnings({"unchecked"})
private void init(String collectionName,
String zkHost,
Map params,
@SuppressWarnings({"rawtypes"})Map params,
String name,
String feature,
TupleStream termsStream,
@ -332,7 +333,7 @@ public class TextLogitStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
List<TupleStream> l = new ArrayList<>();
l.add(termsStream);
return l;
}
@ -371,7 +372,7 @@ public class TextLogitStream extends TupleStream implements Expressible {
private List<Future<Tuple>> callShards(List<String> baseUrls) throws IOException {
List<Future<Tuple>> futures = new ArrayList();
List<Future<Tuple>> futures = new ArrayList<>();
for (String baseUrl : baseUrls) {
LogitCall lc = new LogitCall(baseUrl,
this.params,
@ -423,7 +424,7 @@ public class TextLogitStream extends TupleStream implements Expressible {
if (this.terms == null) {
termsStream.open();
this.terms = new ArrayList<>();
this.idfs = new ArrayList();
this.idfs = new ArrayList<>();
while (true) {
Tuple termTuple = termsStream.read();
@ -438,6 +439,7 @@ public class TextLogitStream extends TupleStream implements Expressible {
}
}
@SuppressWarnings({"unchecked"})
public Tuple read() throws IOException {
try {
@ -453,7 +455,7 @@ public class TextLogitStream extends TupleStream implements Expressible {
}
}
List<List<Double>> allWeights = new ArrayList();
List<List<Double>> allWeights = new ArrayList<>();
this.evaluation = new ClassificationEvaluation();
this.error = 0;
@ -463,11 +465,13 @@ public class TextLogitStream extends TupleStream implements Expressible {
List<Double> shardWeights = (List<Double>) tuple.get("weights");
allWeights.add(shardWeights);
this.error += tuple.getDouble("error");
@SuppressWarnings({"rawtypes"})
Map shardEvaluation = (Map) tuple.get("evaluation");
this.evaluation.addEvaluation(shardEvaluation);
}
this.weights = averageWeights(allWeights);
@SuppressWarnings({"rawtypes"})
Map map = new HashMap();
map.put(ID, name+"_"+iteration);
map.put("name_s", name);
@ -514,7 +518,7 @@ public class TextLogitStream extends TupleStream implements Expressible {
working[i] = working[i] / allWeights.size();
}
List<Double> ave = new ArrayList();
List<Double> ave = new ArrayList<>();
for(double d : working) {
ave.add(d);
}
@ -522,7 +526,7 @@ public class TextLogitStream extends TupleStream implements Expressible {
return ave;
}
static String toString(List items) {
static String toString(@SuppressWarnings({"rawtypes"})List items) {
StringBuilder buf = new StringBuilder();
for(Object item : items) {
if(buf.length() > 0) {
@ -640,10 +644,13 @@ public class TextLogitStream extends TupleStream implements Expressible {
QueryRequest request= new QueryRequest(params, SolrRequest.METHOD.POST);
QueryResponse response = request.process(solrClient);
@SuppressWarnings({"rawtypes"})
NamedList res = response.getResponse();
@SuppressWarnings({"rawtypes"})
NamedList logit = (NamedList)res.get("logit");
@SuppressWarnings({"unchecked"})
List<Double> shardWeights = (List<Double>)logit.get("weights");
double shardError = (double)logit.get("error");

View File

@ -65,7 +65,7 @@ public class TimeSeriesStream extends TupleStream implements Expressible {
private DateTimeFormatter formatter;
private Metric[] metrics;
private List<Tuple> tuples = new ArrayList();
private List<Tuple> tuples = new ArrayList<>();
private int index;
private String zkHost;
private SolrParams params;
@ -287,7 +287,7 @@ public class TimeSeriesStream extends TupleStream implements Expressible {
}
public List<TupleStream> children() {
return new ArrayList();
return new ArrayList<>();
}
public void open() throws IOException {
@ -307,6 +307,7 @@ public class TimeSeriesStream extends TupleStream implements Expressible {
QueryRequest request = new QueryRequest(paramsLoc, SolrRequest.METHOD.POST);
try {
@SuppressWarnings({"rawtypes"})
NamedList response = cloudSolrClient.request(request, collection);
getTuples(response, field, metrics);
} catch (Exception e) {
@ -374,28 +375,32 @@ public class TimeSeriesStream extends TupleStream implements Expressible {
buf.append("}}");
}
private void getTuples(NamedList response,
private void getTuples(@SuppressWarnings({"rawtypes"})NamedList response,
String field,
Metric[] metrics) {
Tuple tuple = new Tuple();
@SuppressWarnings({"rawtypes"})
NamedList facets = (NamedList)response.get("facets");
fillTuples(tuples, tuple, facets, field, metrics);
}
private void fillTuples(List<Tuple> tuples,
Tuple currentTuple,
NamedList facets,
@SuppressWarnings({"rawtypes"})NamedList facets,
String field,
Metric[] _metrics) {
@SuppressWarnings({"rawtypes"})
NamedList nl = (NamedList)facets.get("timeseries");
if(nl == null) {
return;
}
@SuppressWarnings({"rawtypes"})
List allBuckets = (List)nl.get("buckets");
for(int b=0; b<allBuckets.size(); b++) {
@SuppressWarnings({"rawtypes"})
NamedList bucket = (NamedList)allBuckets.get(b);
Object val = bucket.get("val");

View File

@ -273,14 +273,14 @@ public class TopicStream extends CloudSolrStream implements Expressible {
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList();
List<TupleStream> l = new ArrayList<>();
return l;
}
public void open() throws IOException {
this.tuples = new TreeSet();
this.solrStreams = new ArrayList();
this.eofTuples = Collections.synchronizedMap(new HashMap());
this.tuples = new TreeSet<>();
this.solrStreams = new ArrayList<>();
this.eofTuples = Collections.synchronizedMap(new HashMap<>());
if(checkpoints.size() == 0 && streamContext.numWorkers > 1) {
//Each worker must maintain its own checkpoints
@ -313,7 +313,7 @@ public class TopicStream extends CloudSolrStream implements Expressible {
ExecutorService service = ExecutorUtil.newMDCAwareCachedThreadPool(new SolrNamedThreadFactory("TopicStream"));
try {
List<Future<TupleWrapper>> futures = new ArrayList();
List<Future<TupleWrapper>> futures = new ArrayList<>();
for (TupleStream solrStream : solrStreams) {
StreamOpener so = new StreamOpener((SolrStream) solrStream, comp);
Future<TupleWrapper> future = service.submit(so);
@ -489,6 +489,7 @@ public class TopicStream extends CloudSolrStream implements Expressible {
try {
SolrDocument doc = httpClient.getById(id);
if(doc != null) {
@SuppressWarnings({"unchecked"})
List<String> checkpoints = (List<String>)doc.getFieldValue("checkpoint_ss");
for (String checkpoint : checkpoints) {
String[] pair = checkpoint.split("~");

View File

@ -50,8 +50,8 @@ public class TupStream extends TupleStream implements Expressible {
private Map<String,String> stringParams = new HashMap<>();
private Map<String,StreamEvaluator> evaluatorParams = new HashMap<>();
private Map<String,TupleStream> streamParams = new HashMap<>();
private List<String> fieldNames = new ArrayList();
private Map<String, String> fieldLabels = new HashMap();
private List<String> fieldNames = new ArrayList<>();
private Map<String, String> fieldLabels = new HashMap<>();
private Tuple tup = null;
private Tuple unnestedTuple = null;
private Iterator<Tuple> unnestedTuples = null;
@ -174,6 +174,7 @@ public class TupStream extends TupleStream implements Expressible {
// Nothing to do here
}
@SuppressWarnings({"unchecked"})
public void open() throws IOException {
Map<String, Object> values = new HashMap<>();
@ -197,7 +198,7 @@ public class TupStream extends TupleStream implements Expressible {
for(Entry<String,TupleStream> param : streamParams.entrySet()){
try{
List<Tuple> streamTuples = new ArrayList();
List<Tuple> streamTuples = new ArrayList<>();
// open the stream, closed in finally block
param.getValue().open();
@ -221,6 +222,7 @@ public class TupStream extends TupleStream implements Expressible {
if(o instanceof Tuple) {
unnestedTuple = (Tuple)o;
} else if(o instanceof List) {
@SuppressWarnings({"rawtypes"})
List l = (List)o;
if(l.size() > 0 && l.get(0) instanceof Tuple) {
List<Tuple> tl = (List<Tuple>)l;

View File

@ -123,13 +123,14 @@ public abstract class TupleStream implements Closeable, Serializable, MapWriter
return getShards(zkHost, collection, streamContext, new ModifiableSolrParams());
}
@SuppressWarnings({"unchecked"})
public static List<String> getShards(String zkHost,
String collection,
StreamContext streamContext,
SolrParams requestParams)
throws IOException {
Map<String, List<String>> shardsMap = null;
List<String> shards = new ArrayList();
List<String> shards = new ArrayList<>();
if(streamContext != null) {
shardsMap = (Map<String, List<String>>)streamContext.get("shards");

View File

@ -66,7 +66,7 @@ public class UpdateStream extends TupleStream implements Expressible {
private PushBackStream tupleSource;
private transient SolrClientCache cache;
private transient CloudSolrClient cloudSolrClient;
private List<SolrInputDocument> documentBatch = new ArrayList();
private List<SolrInputDocument> documentBatch = new ArrayList<>();
private String coreName;
public UpdateStream(StreamExpression expression, StreamFactory factory) throws IOException {
@ -294,6 +294,7 @@ public class UpdateStream extends TupleStream implements Expressible {
}
}
@SuppressWarnings({"unchecked"})
private SolrInputDocument convertTupleToSolrDocument(Tuple tuple) {
SolrInputDocument doc = new SolrInputDocument();
for (Object field : tuple.getFields().keySet()) {

View File

@ -53,9 +53,11 @@ public class ZplotStream extends TupleStream implements Expressible {
private static final long serialVersionUID = 1;
private StreamContext streamContext;
@SuppressWarnings({"rawtypes"})
private Map letParams = new LinkedHashMap();
private Iterator<Tuple> out;
@SuppressWarnings({"unchecked"})
public ZplotStream(StreamExpression expression, StreamFactory factory) throws IOException {
List<StreamExpressionNamedParameter> namedParams = factory.getNamedOperands(expression);
@ -118,10 +120,11 @@ public class ZplotStream extends TupleStream implements Expressible {
public void close() throws IOException {
}
@SuppressWarnings({"unchecked", "rawtypes"})
public void open() throws IOException {
Map<String, Object> lets = streamContext.getLets();
Set<Map.Entry<String, Object>> entries = letParams.entrySet();
Map<String, Object> evaluated = new HashMap();
Map<String, Object> evaluated = new HashMap<>();
//Load up the StreamContext with the data created by the letParams.
int numTuples = -1;
@ -191,7 +194,7 @@ public class ZplotStream extends TupleStream implements Expressible {
}
//Load the values into tuples
List<Tuple> outTuples = new ArrayList();
List<Tuple> outTuples = new ArrayList<>();
if(!table && !distribution && !clusters && !heat) {
//Handle the vectors
for (int i = 0; i < numTuples; i++) {
@ -287,7 +290,7 @@ public class ZplotStream extends TupleStream implements Expressible {
}
Iterator it = frequency.valuesIterator();
List<Long> values = new ArrayList();
List<Long> values = new ArrayList<>();
while(it.hasNext()) {
values.add((Long)it.next());
}

View File

@ -128,6 +128,7 @@ public class Explanation implements MapSerializable {
}
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public Map toMap(Map<String, Object> map) {
if(null != expressionNodeId){ map.put("expressionNodeId",expressionNodeId); }
if(null != expressionType){ map.put("expressionType",expressionType); }

View File

@ -55,6 +55,7 @@ public class StreamExplanation extends Explanation {
children.add(child);
}
@SuppressWarnings({"unchecked"})
public Map<String,Object> toMap(Map<String,Object> map){
map = super.toMap(map);

View File

@ -22,6 +22,7 @@ import java.util.List;
/**
* Expression containing a function and set of parameters
*/
@SuppressWarnings({"overrides"})
public class StreamExpression implements StreamExpressionParameter {
private String functionName;
private List<StreamExpressionParameter> parameters;

View File

@ -20,6 +20,7 @@ package org.apache.solr.client.solrj.io.stream.expr;
/**
* Provides a named parameter
*/
@SuppressWarnings({"overrides"})
public class StreamExpressionNamedParameter implements StreamExpressionParameter {
private String name;
private StreamExpressionParameter parameter;

View File

@ -19,6 +19,7 @@ package org.apache.solr.client.solrj.io.stream.expr;
/**
* Basic string stream expression
*/
@SuppressWarnings({"overrides"})
public class StreamExpressionValue implements StreamExpressionParameter {
private String value;

View File

@ -182,12 +182,14 @@ public class StreamFactory implements Serializable {
return namedParameters;
}
public List<StreamExpressionParameter> getOperandsOfType(StreamExpression expression, Class ... clazzes) {
@SuppressWarnings({"unchecked"})
public List<StreamExpressionParameter> getOperandsOfType(StreamExpression expression,
@SuppressWarnings({"rawtypes"})Class ... clazzes) {
List<StreamExpressionParameter> parameters = new ArrayList<>();
parameterLoop:
for (StreamExpressionParameter parameter : expression.getParameters()) {
for (Class clazz : clazzes) {
for (@SuppressWarnings({"rawtypes"})Class clazz : clazzes) {
if (!clazz.isAssignableFrom(parameter.getClass())) {
continue parameterLoop; // go to the next parameter since this parameter cannot be assigned to at least one of the classes
}
@ -197,7 +199,9 @@ public class StreamFactory implements Serializable {
return parameters;
}
public List<StreamExpression> getExpressionOperandsRepresentingTypes(StreamExpression expression, Class ... clazzes) {
@SuppressWarnings({"unchecked"})
public List<StreamExpression> getExpressionOperandsRepresentingTypes(StreamExpression expression,
@SuppressWarnings({"rawtypes"})Class ... clazzes) {
List<StreamExpression> matchingStreamExpressions = new ArrayList<>();
List<StreamExpression> allStreamExpressions = getExpressionOperands(expression);
@ -205,7 +209,7 @@ public class StreamFactory implements Serializable {
for (StreamExpression streamExpression : allStreamExpressions) {
Supplier<Class<? extends Expressible>> classSupplier = functionNames.get(streamExpression.getFunctionName());
if (classSupplier != null) {
for (Class clazz : clazzes) {
for (@SuppressWarnings({"rawtypes"})Class clazz : clazzes) {
if (!clazz.isAssignableFrom(classSupplier.get())) {
continue parameterLoop;
}
@ -216,10 +220,11 @@ public class StreamFactory implements Serializable {
return matchingStreamExpressions;
}
public boolean doesRepresentTypes(StreamExpression expression, Class ... clazzes) {
@SuppressWarnings({"unchecked"})
public boolean doesRepresentTypes(StreamExpression expression, @SuppressWarnings({"rawtypes"})Class ... clazzes) {
Supplier<Class<? extends Expressible>> classSupplier = functionNames.get(expression.getFunctionName());
if (classSupplier != null) {
for (Class clazz : clazzes) {
for (@SuppressWarnings({"rawtypes"})Class clazz : clazzes) {
if (!clazz.isAssignableFrom(classSupplier.get())) {
return false;
}
@ -265,6 +270,7 @@ public class StreamFactory implements Serializable {
public TupleStream constructStream(String expressionClause) throws IOException {
return constructStream(StreamExpressionParser.parse(expressionClause));
}
@SuppressWarnings({"rawtypes"})
public TupleStream constructStream(StreamExpression expression) throws IOException {
String function = expression.getFunctionName();
Supplier<Class<? extends Expressible>> classSupplier = functionNames.get(function);
@ -283,6 +289,7 @@ public class StreamFactory implements Serializable {
return constructMetric(StreamExpressionParser.parse(expressionClause));
}
@SuppressWarnings({"rawtypes"})
public Metric constructMetric(StreamExpression expression) throws IOException {
String function = expression.getFunctionName();
Supplier<Class<? extends Expressible>> classSupplier = functionNames.get(function);
@ -296,7 +303,8 @@ public class StreamFactory implements Serializable {
throw new IOException(String.format(Locale.ROOT, "Invalid metric expression %s - function '%s' is unknown (not mapped to a valid Metric)", expression, expression.getFunctionName()));
}
public StreamComparator constructComparator(String comparatorString, Class comparatorType) throws IOException {
@SuppressWarnings({"unchecked", "rawtypes"})
public StreamComparator constructComparator(String comparatorString, @SuppressWarnings({"rawtypes"})Class comparatorType) throws IOException {
if (comparatorString.contains(",")) {
String[] parts = comparatorString.split(",");
StreamComparator[] comps = new StreamComparator[parts.length];
@ -349,6 +357,7 @@ public class StreamFactory implements Serializable {
}
}
@SuppressWarnings({"unchecked", "rawtypes"})
public StreamEqualitor constructEqualitor(String equalitorString, Class equalitorType) throws IOException {
if (equalitorString.contains(",")) {
String[] parts = equalitorString.split(",");
@ -381,6 +390,7 @@ public class StreamFactory implements Serializable {
return constructMetric(StreamExpressionParser.parse(expressionClause));
}
@SuppressWarnings({"rawtypes"})
public StreamOperation constructOperation(StreamExpression expression) throws IOException {
String function = expression.getFunctionName();
Supplier<Class<? extends Expressible>> classSupplier = functionNames.get(function);
@ -398,6 +408,7 @@ public class StreamFactory implements Serializable {
return constructEvaluator(StreamExpressionParser.parse(expressionClause));
}
@SuppressWarnings({"rawtypes"})
public org.apache.solr.client.solrj.io.eval.StreamEvaluator constructEvaluator(StreamExpression expression) throws IOException {
String function = expression.getFunctionName();
Supplier<Class<? extends Expressible>> classSupplier = functionNames.get(function);