SOLR-10292: Adds CartesianProductStream to turn multivalued fields into multiple tuples

This commit is contained in:
Dennis Gove 2017-03-20 16:36:05 -04:00 committed by Shalin Shekhar Mangar
parent 3d67576307
commit c389148328
10 changed files with 611 additions and 1 deletions

View File

@ -58,6 +58,9 @@ New Features
* SOLR-9835: Create another replication mode for SolrCloud * SOLR-9835: Create another replication mode for SolrCloud
* SOLR-10292: Adds CartesianProductStream which turns a single tuple with a multi-valued field into N
tuples, one for each value in the multi-valued field. (Dennis Gove)
Bug Fixes Bug Fixes
---------------------- ----------------------
* SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509. * SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509.

View File

@ -154,6 +154,9 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
.withFunctionName("gatherNodes", GatherNodesStream.class) .withFunctionName("gatherNodes", GatherNodesStream.class)
.withFunctionName("nodes", GatherNodesStream.class) .withFunctionName("nodes", GatherNodesStream.class)
.withFunctionName("select", SelectStream.class) .withFunctionName("select", SelectStream.class)
.withFunctionName("shortestPath", ShortestPathStream.class)
.withFunctionName("gatherNodes", GatherNodesStream.class)
.withFunctionName("nodes", GatherNodesStream.class)
.withFunctionName("scoreNodes", ScoreNodesStream.class) .withFunctionName("scoreNodes", ScoreNodesStream.class)
.withFunctionName("model", ModelStream.class) .withFunctionName("model", ModelStream.class)
.withFunctionName("classify", ClassifyStream.class) .withFunctionName("classify", ClassifyStream.class)
@ -162,6 +165,8 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
.withFunctionName("null", NullStream.class) .withFunctionName("null", NullStream.class)
.withFunctionName("priority", PriorityStream.class) .withFunctionName("priority", PriorityStream.class)
.withFunctionName("significantTerms", SignificantTermsStream.class) .withFunctionName("significantTerms", SignificantTermsStream.class)
.withFunctionName("cartesianProduct", CartesianProductStream.class)
// metrics // metrics
.withFunctionName("min", MinMetric.class) .withFunctionName("min", MinMetric.class)
.withFunctionName("max", MaxMetric.class) .withFunctionName("max", MaxMetric.class)

View File

@ -166,4 +166,9 @@ public class FieldComparator implements StreamComparator {
order order
); );
} }
@Override
public StreamComparator append(StreamComparator other){
return new MultipleFieldComparator(this).append(other);
}
} }

View File

@ -17,6 +17,8 @@
package org.apache.solr.client.solrj.io.comp; package org.apache.solr.client.solrj.io.comp;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.UUID; import java.util.UUID;
@ -113,4 +115,24 @@ public class MultipleFieldComparator implements StreamComparator {
return new MultipleFieldComparator(aliasedComps); return new MultipleFieldComparator(aliasedComps);
} }
@Override
public StreamComparator append(StreamComparator other){
List<StreamComparator> newComps = new ArrayList<>();
for(StreamComparator comp : comps){
newComps.add(comp);
}
if(other instanceof FieldComparator){
newComps.add(other);
}
else if(other instanceof MultipleFieldComparator){
for(StreamComparator comp : ((MultipleFieldComparator)other).comps){
newComps.add(comp);
}
}
return new MultipleFieldComparator(newComps.toArray(new StreamComparator[newComps.size()]));
}
} }

View File

@ -27,4 +27,5 @@ import org.apache.solr.client.solrj.io.stream.expr.Expressible;
public interface StreamComparator extends Comparator<Tuple>, Expressible, Serializable { public interface StreamComparator extends Comparator<Tuple>, Expressible, Serializable {
public boolean isDerivedFrom(StreamComparator base); public boolean isDerivedFrom(StreamComparator base);
public StreamComparator copyAliased(Map<String,String> aliases); public StreamComparator copyAliased(Map<String,String> aliases);
public StreamComparator append(StreamComparator other);
} }

View File

@ -20,6 +20,8 @@
package org.apache.solr.client.solrj.io.eval; package org.apache.solr.client.solrj.io.eval;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.stream.expr.Explanation; import org.apache.solr.client.solrj.io.stream.expr.Explanation;
@ -43,7 +45,31 @@ public class FieldEvaluator extends SimpleEvaluator {
@Override @Override
public Object evaluate(Tuple tuple) { public Object evaluate(Tuple tuple) {
return tuple.get(fieldName); // returns null if field doesn't exist in tuple Object value = tuple.get(fieldName);
// if we have an array then convert to an ArrayList
// if we have an iterable that is not a list then convert to ArrayList
// lists are good to go
if(null != value){
if(value instanceof Object[]){
Object[] array = (Object[])value;
List<Object> list = new ArrayList<Object>(array.length);
for(Object obj : array){
list.add(obj);
}
return list;
}
else if(value instanceof Iterable && !(value instanceof List<?>)){
Iterable<?> iter = (Iterable<?>)value;
List<Object> list = new ArrayList<Object>();
for(Object obj : iter){
list.add(obj);
}
return list;
}
}
return value;
} }
@Override @Override

View File

@ -0,0 +1,301 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.client.solrj.io.stream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.comp.FieldComparator;
import org.apache.solr.client.solrj.io.comp.StreamComparator;
import org.apache.solr.client.solrj.io.eval.FieldEvaluator;
import org.apache.solr.client.solrj.io.eval.StreamEvaluator;
import org.apache.solr.client.solrj.io.stream.expr.Explanation;
import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
import org.apache.solr.client.solrj.io.stream.expr.Expressible;
import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParameter;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParser;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
public class CartesianProductStream extends TupleStream implements Expressible {
private static final long serialVersionUID = 1;
private TupleStream stream;
private List<NamedEvaluator> evaluators;
private StreamComparator orderBy;
// Used to contain the sorted queue of generated tuples
private LinkedList<Tuple> generatedTuples;
public CartesianProductStream(StreamExpression expression,StreamFactory factory) throws IOException {
String functionName = factory.getFunctionName(getClass());
// grab all parameters out
List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
List<StreamExpressionParameter> evaluateAsExpressions = factory.getOperandsOfType(expression, StreamExpressionValue.class);
StreamExpressionNamedParameter orderByExpression = factory.getNamedOperand(expression, "productSort");
// validate expression contains only what we want.
if(expression.getParameters().size() != streamExpressions.size() + evaluateAsExpressions.size() + (null == orderByExpression ? 0 : 1)){
throw new IOException(String.format(Locale.ROOT,"Invalid %s expression %s - unknown operands found", functionName, expression));
}
if(1 != streamExpressions.size()){
throw new IOException(String.format(Locale.ROOT,"Invalid %s expression %s - expecting single stream but found %d (must be TupleStream types)", functionName, expression, streamExpressions.size()));
}
stream = factory.constructStream(streamExpressions.get(0));
orderBy = null == orderByExpression ? null : factory.constructComparator(((StreamExpressionValue)orderByExpression.getParameter()).getValue(), FieldComparator.class);
evaluators = new ArrayList<>();
for(StreamExpressionParameter evaluateAsExpression : evaluateAsExpressions){
String fullString = ((StreamExpressionValue)evaluateAsExpression).getValue().trim();
String originalFullString = fullString; // used for error messages
// remove possible wrapping quotes
if(fullString.length() > 2 && fullString.startsWith("\"") && fullString.endsWith("\"")){
fullString = fullString.substring(1, fullString.length() - 1).trim();
}
String evaluatorPart = null;
String asNamePart = null;
if(fullString.toLowerCase(Locale.ROOT).contains(" as ")){
String[] parts = fullString.split("(?i) as "); // ensure we are splitting in a case-insensitive way
if(2 != parts.length){
throw new IOException(String.format(Locale.ROOT,"Invalid %s expression %s - expecting evaluator of form 'fieldA' or 'fieldA as alias' but found %s", functionName, expression, originalFullString));
}
evaluatorPart = parts[0].trim();
asNamePart = parts[1].trim();
}
else{
evaluatorPart = fullString;
// no rename
}
boolean wasHandledAsEvaluatorFunction = false;
StreamEvaluator evaluator = null;
if(evaluatorPart.contains("(")){
// is a possible evaluator
try{
StreamExpression asValueExpression = StreamExpressionParser.parse(evaluatorPart);
if(factory.doesRepresentTypes(asValueExpression, StreamEvaluator.class)){
evaluator = factory.constructEvaluator(asValueExpression);
wasHandledAsEvaluatorFunction = true;
}
}
catch(Throwable e){
// it was not handled, so treat as a non-evaluator
}
}
if(!wasHandledAsEvaluatorFunction){
// treat as a straight field evaluator
evaluator = new FieldEvaluator(evaluatorPart);
if(null == asNamePart){
asNamePart = evaluatorPart; // just use the field name
}
}
if(null == evaluator || null == asNamePart){
throw new IOException(String.format(Locale.ROOT,"Invalid %s expression %s - failed to parse evaluator '%s'", functionName, expression, originalFullString));
}
evaluators.add(new NamedEvaluator(asNamePart, evaluator));
}
}
@Override
public StreamExpression toExpression(StreamFactory factory) throws IOException{
return toExpression(factory, true);
}
private StreamExpression toExpression(StreamFactory factory, boolean includeStreams) throws IOException {
// function name
StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
if(includeStreams){
// we know stream is expressible
expression.addParameter(((Expressible)stream).toExpression(factory));
}
else{
expression.addParameter("<stream>");
}
// selected evaluators
for(NamedEvaluator evaluator : evaluators) {
expression.addParameter(String.format(Locale.ROOT, "%s as %s", evaluator.getEvaluator().toExpression(factory), evaluator.getName()));
}
expression.addParameter(new StreamExpressionNamedParameter("productSort", orderBy.toExpression(factory)));
return expression;
}
@Override
public Explanation toExplanation(StreamFactory factory) throws IOException {
Explanation explanation = new StreamExplanation(getStreamNodeId().toString())
.withChildren(new Explanation[]{
stream.toExplanation(factory)
})
.withFunctionName(factory.getFunctionName(this.getClass()))
.withImplementingClass(this.getClass().getName())
.withExpressionType(ExpressionType.STREAM_DECORATOR)
.withExpression(toExpression(factory, false).toString());
for(NamedEvaluator evaluator : evaluators){
explanation.addHelper(evaluator.getEvaluator().toExplanation(factory));
}
explanation.addHelper(orderBy.toExplanation(factory));
return explanation;
}
public Tuple read() throws IOException {
if(generatedTuples.isEmpty()){
Tuple tuple = stream.read();
if(tuple.EOF){
return tuple;
}
// returns tuples in desired sorted order
generatedTuples = generateTupleList(tuple);
}
return generatedTuples.pop();
}
private LinkedList<Tuple> generateTupleList(Tuple original) throws IOException{
Map<String, Object> evaluatedValues = new HashMap<>();
for(NamedEvaluator evaluator : evaluators){
evaluatedValues.put(evaluator.getName(), evaluator.getEvaluator().evaluate(original));
}
// use an array list internally because it has better sort performance
// in Java 8. We do pay a conversion to a linked list but ..... oh well
ArrayList<Tuple> generatedTupleList = new ArrayList<>();
int[] workingIndexes = new int[evaluators.size()]; // java language spec ensures all values are 0
do{
Tuple generated = original.clone();
for(int offset = 0; offset < workingIndexes.length; ++offset){
String fieldName = evaluators.get(offset).getName();
Object evaluatedValue = evaluatedValues.get(fieldName);
if(evaluatedValue instanceof Collection){
// because of the way a FieldEvaluator works we know that
// any collection is a list.
generated.put(fieldName, ((List<Object>)evaluatedValue).get(workingIndexes[offset]));
}
}
generatedTupleList.add(generated);
}while(iterate(evaluators, workingIndexes, evaluatedValues));
// order if we need to
if(null != orderBy){
generatedTupleList.sort(orderBy);
}
return new LinkedList<>(generatedTupleList);
}
private boolean iterate(List<NamedEvaluator> evaluators, int[] indexes, Map<String, Object> evaluatedValues){
// this assumes evaluators and indexes are the same length, which is ok cause we created it so we know it is
// go right to left and increment, returning true if we're not at the end
for(int offset = indexes.length - 1; offset >= 0; --offset){
Object evaluatedValue = evaluatedValues.get(evaluators.get(offset).getName());
if(evaluatedValue instanceof Collection){
int currentIndexValue = indexes[offset];
if(currentIndexValue < ((Collection)evaluatedValue).size() - 1){
indexes[offset] = currentIndexValue + 1;
return true;
}
else if(0 != offset){
indexes[offset] = 0;
// move to the left
}
}
}
// no more
return false;
}
/** Return the incoming sort + the sort applied to the generated tuples */
public StreamComparator getStreamSort(){
if(null != orderBy){
return stream.getStreamSort().append(orderBy);
}
return stream.getStreamSort();
}
public void setStreamContext(StreamContext context) {
this.stream.setStreamContext(context);
}
public List<TupleStream> children() {
List<TupleStream> l = new ArrayList<>();
l.add(stream);
return l;
}
public void open() throws IOException {
stream.open();
generatedTuples = new LinkedList<>();
}
public void close() throws IOException {
stream.close();
generatedTuples.clear();
}
public int getCost() {
return 0;
}
class NamedEvaluator{
private String name;
private StreamEvaluator evaluator;
public NamedEvaluator(String name, StreamEvaluator evaluator){
this.name = name;
this.evaluator = evaluator;
}
public String getName(){
return name;
}
public StreamEvaluator getEvaluator(){
return evaluator;
}
}
}

View File

@ -526,6 +526,7 @@
<dynamicField name="*_ss" type="string" indexed="true" stored="true" multiValued="true"/> <dynamicField name="*_ss" type="string" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_s1" type="string" indexed="true" stored="true" multiValued="false"/> <dynamicField name="*_s1" type="string" indexed="true" stored="true" multiValued="false"/>
<dynamicField name="*_l" type="long" indexed="true" stored="true"/> <dynamicField name="*_l" type="long" indexed="true" stored="true"/>
<dynamicField name="*_ls" type="long" indexed="true" stored="true" multiValued="true"/>
<dynamicField name="*_l1" type="long" indexed="true" stored="true" multiValued="false"/> <dynamicField name="*_l1" type="long" indexed="true" stored="true" multiValued="false"/>
<dynamicField name="*_t" type="text" indexed="true" stored="true"/> <dynamicField name="*_t" type="text" indexed="true" stored="true"/>
<dynamicField name="*_b" type="boolean" indexed="true" stored="true"/> <dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>

View File

@ -4959,6 +4959,138 @@ public class StreamExpressionTest extends SolrCloudTestCase {
} }
@Test
public void testCartesianProductStream() throws Exception {
new UpdateRequest()
.add(id, "0", "a_ss", "a", "a_ss", "b", "a_ss", "c", "a_ss", "d", "a_ss", "e", "b_ls", "1", "b_ls", "2", "b_ls", "3")
.add(id, "1", "a_ss", "a", "a_ss", "b", "a_ss", "c", "a_ss", "d", "a_ss", "e")
.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
StreamExpression expression;
TupleStream stream;
List<Tuple> tuples;
StreamFactory factory = new StreamFactory()
.withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress())
.withFunctionName("search", CloudSolrStream.class)
.withFunctionName("cartesian", CartesianProductStream.class);
// single selection, no sort
stream = factory.constructStream("cartesian("
+ "search(collection1, q=*:*, fl=\"id,a_ss\", sort=\"id asc\"),"
+ "a_ss"
+ ")");
tuples = getTuples(stream);
assertEquals(10, tuples.size());
assertOrder(tuples, 0,0,0,0,0,1,1,1,1,1);
assertEquals("a", tuples.get(0).get("a_ss"));
assertEquals("c", tuples.get(2).get("a_ss"));
assertEquals("a", tuples.get(5).get("a_ss"));
assertEquals("c", tuples.get(7).get("a_ss"));
// single selection, sort
stream = factory.constructStream("cartesian("
+ "search(collection1, q=*:*, fl=\"id,a_ss\", sort=\"id asc\"),"
+ "a_ss,"
+ "productSort=\"a_ss DESC\""
+ ")");
tuples = getTuples(stream);
assertEquals(10, tuples.size());
assertOrder(tuples, 0,0,0,0,0,1,1,1,1,1);
assertEquals("e", tuples.get(0).get("a_ss"));
assertEquals("c", tuples.get(2).get("a_ss"));
assertEquals("e", tuples.get(5).get("a_ss"));
assertEquals("c", tuples.get(7).get("a_ss"));
// multi selection, sort
stream = factory.constructStream("cartesian("
+ "search(collection1, q=*:*, fl=\"id,a_ss,b_ls\", sort=\"id asc\"),"
+ "a_ss,"
+ "b_ls,"
+ "productSort=\"a_ss ASC\""
+ ")");
tuples = getTuples(stream);
assertEquals(20, tuples.size()); // (5 * 3) + 5
assertOrder(tuples, 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,1,1,1,1);
assertEquals("a", tuples.get(0).get("a_ss"));
assertEquals(1L, tuples.get(0).get("b_ls"));
assertEquals("a", tuples.get(1).get("a_ss"));
assertEquals(2L, tuples.get(1).get("b_ls"));
assertEquals("a", tuples.get(2).get("a_ss"));
assertEquals(3L, tuples.get(2).get("b_ls"));
assertEquals("b", tuples.get(3).get("a_ss"));
assertEquals(1L, tuples.get(3).get("b_ls"));
assertEquals("b", tuples.get(4).get("a_ss"));
assertEquals(2L, tuples.get(4).get("b_ls"));
assertEquals("b", tuples.get(5).get("a_ss"));
assertEquals(3L, tuples.get(5).get("b_ls"));
// multi selection, sort
stream = factory.constructStream("cartesian("
+ "search(collection1, q=*:*, fl=\"id,a_ss,b_ls\", sort=\"id asc\"),"
+ "a_ss,"
+ "b_ls,"
+ "productSort=\"a_ss ASC, b_ls DESC\""
+ ")");
tuples = getTuples(stream);
assertEquals(20, tuples.size()); // (5 * 3) + 5
assertOrder(tuples, 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,1,1,1,1);
assertEquals("a", tuples.get(0).get("a_ss"));
assertEquals(3L, tuples.get(0).get("b_ls"));
assertEquals("a", tuples.get(1).get("a_ss"));
assertEquals(2L, tuples.get(1).get("b_ls"));
assertEquals("a", tuples.get(2).get("a_ss"));
assertEquals(1L, tuples.get(2).get("b_ls"));
assertEquals("b", tuples.get(3).get("a_ss"));
assertEquals(3L, tuples.get(3).get("b_ls"));
assertEquals("b", tuples.get(4).get("a_ss"));
assertEquals(2L, tuples.get(4).get("b_ls"));
assertEquals("b", tuples.get(5).get("a_ss"));
assertEquals(1L, tuples.get(5).get("b_ls"));
// multi selection, sort
stream = factory.constructStream("cartesian("
+ "search(collection1, q=*:*, fl=\"id,a_ss,b_ls\", sort=\"id asc\"),"
+ "a_ss,"
+ "b_ls,"
+ "productSort=\"b_ls DESC\""
+ ")");
tuples = getTuples(stream);
assertEquals(20, tuples.size()); // (5 * 3) + 5
assertOrder(tuples, 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,1,1,1,1);
assertEquals("a", tuples.get(0).get("a_ss"));
assertEquals(3L, tuples.get(0).get("b_ls"));
assertEquals("b", tuples.get(1).get("a_ss"));
assertEquals(3L, tuples.get(1).get("b_ls"));
assertEquals("c", tuples.get(2).get("a_ss"));
assertEquals(3L, tuples.get(2).get("b_ls"));
assertEquals("d", tuples.get(3).get("a_ss"));
assertEquals(3L, tuples.get(3).get("b_ls"));
assertEquals("e", tuples.get(4).get("a_ss"));
assertEquals(3L, tuples.get(4).get("b_ls"));
assertEquals("a", tuples.get(5).get("a_ss"));
assertEquals(2L, tuples.get(5).get("b_ls"));
assertEquals("b", tuples.get(6).get("a_ss"));
assertEquals(2L, tuples.get(6).get("b_ls"));
assertEquals("c", tuples.get(7).get("a_ss"));
assertEquals(2L, tuples.get(7).get("b_ls"));
assertEquals("d", tuples.get(8).get("a_ss"));
assertEquals(2L, tuples.get(8).get("b_ls"));
assertEquals("e", tuples.get(9).get("a_ss"));
assertEquals(2L, tuples.get(9).get("b_ls"));
}
@Test @Test
public void testParallelComplementStream() throws Exception { public void testParallelComplementStream() throws Exception {

View File

@ -0,0 +1,114 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.client.solrj.io.stream.eval;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import java.util.PriorityQueue;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.eval.FieldEvaluator;
import org.junit.Test;
import junit.framework.Assert;
public class FieldEvaluatorTest extends LuceneTestCase {
Map<String, Object> values;
public FieldEvaluatorTest() {
super();
values = new HashMap<String,Object>();
}
@SuppressWarnings("serial")
@Test
public void listTypes() throws Exception{
values.clear();
values.put("a", new ArrayList<Boolean>(){{ add(true); add(false); }});
values.put("b", new ArrayList<Double>(){{ add(0.0); add(1.1); }});
values.put("c", new ArrayList<Integer>(){{ add(0); add(1); }});
values.put("d", new ArrayList<Long>(){{ add(0L); add(1L); }});
values.put("e", new ArrayList<String>(){{ add("first"); add("second"); }});
Tuple tuple = new Tuple(values);
for(String fieldName : new String[]{ "a", "b", "c", "d", "e" }){
Assert.assertTrue(new FieldEvaluator(fieldName).evaluate(tuple) instanceof Collection);
Assert.assertEquals(2, ((Collection<?>)new FieldEvaluator(fieldName).evaluate(tuple)).size());
}
Assert.assertEquals(false, ((Collection<?>)new FieldEvaluator("a").evaluate(tuple)).toArray()[1]);
Assert.assertEquals(1.1, ((Collection<?>)new FieldEvaluator("b").evaluate(tuple)).toArray()[1]);
Assert.assertEquals(1, ((Collection<?>)new FieldEvaluator("c").evaluate(tuple)).toArray()[1]);
Assert.assertEquals(1L, ((Collection<?>)new FieldEvaluator("d").evaluate(tuple)).toArray()[1]);
Assert.assertEquals("second", ((Collection<?>)new FieldEvaluator("e").evaluate(tuple)).toArray()[1]);
}
@Test
public void arrayTypes() throws Exception{
values.clear();
values.put("a", new Boolean[]{ true, false });
values.put("b", new Double[]{ 0.0, 1.1 });
values.put("c", new Integer[]{ 0, 1 });
values.put("d", new Long[]{ 0L, 1L });
values.put("e", new String[]{ "first", "second" });
Tuple tuple = new Tuple(values);
for(String fieldName : new String[]{ "a", "b", "c", "d", "e" }){
Assert.assertTrue(new FieldEvaluator(fieldName).evaluate(tuple) instanceof Collection);
Assert.assertEquals(2, ((Collection<?>)new FieldEvaluator(fieldName).evaluate(tuple)).size());
}
Assert.assertEquals(false, ((Collection<?>)new FieldEvaluator("a").evaluate(tuple)).toArray()[1]);
Assert.assertEquals(1.1, ((Collection<?>)new FieldEvaluator("b").evaluate(tuple)).toArray()[1]);
Assert.assertEquals(1, ((Collection<?>)new FieldEvaluator("c").evaluate(tuple)).toArray()[1]);
Assert.assertEquals(1L, ((Collection<?>)new FieldEvaluator("d").evaluate(tuple)).toArray()[1]);
Assert.assertEquals("second", ((Collection<?>)new FieldEvaluator("e").evaluate(tuple)).toArray()[1]);
}
@SuppressWarnings("serial")
@Test
public void iterableTypes() throws Exception{
values.clear();
values.put("a", new PriorityQueue<Boolean>(){{ add(true); add(false); }});
values.put("b", new PriorityQueue<Double>(){{ add(0.0); add(1.1); }});
values.put("c", new PriorityQueue<Integer>(){{ add(0); add(1); }});
values.put("d", new PriorityQueue<Long>(){{ add(0L); add(1L); }});
values.put("e", new PriorityQueue<String>(){{ add("first"); add("second"); }});
Tuple tuple = new Tuple(values);
for(String fieldName : new String[]{ "a", "b", "c", "d", "e" }){
Assert.assertTrue(new FieldEvaluator(fieldName).evaluate(tuple) instanceof Collection);
Assert.assertEquals(2, ((Collection<?>)new FieldEvaluator(fieldName).evaluate(tuple)).size());
}
// the priority queue is doing natural ordering, so false is first
Assert.assertEquals(true, ((Collection<?>)new FieldEvaluator("a").evaluate(tuple)).toArray()[1]);
Assert.assertEquals(1.1, ((Collection<?>)new FieldEvaluator("b").evaluate(tuple)).toArray()[1]);
Assert.assertEquals(1, ((Collection<?>)new FieldEvaluator("c").evaluate(tuple)).toArray()[1]);
Assert.assertEquals(1L, ((Collection<?>)new FieldEvaluator("d").evaluate(tuple)).toArray()[1]);
Assert.assertEquals("second", ((Collection<?>)new FieldEvaluator("e").evaluate(tuple)).toArray()[1]);
}
}