SOLR-14476: Add percentiles and standard deviation aggregations to stats, facet and timeseries Streaming Expressions

This commit is contained in:
Joel Bernstein 2020-05-18 16:22:56 -04:00
parent 3b09d30e69
commit 16aad55369
10 changed files with 668 additions and 186 deletions

View File

@ -799,7 +799,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable {
String collection,
String query,
List<Pair<String, String>> metricPairs,
List<Map.Entry<String, Class>> fields) {
List<Map.Entry<String, Class>> fields) throws IOException {
Map<String, Class> fmap = new HashMap();

View File

@ -1439,8 +1439,7 @@ public class TestSQLHandler extends SolrCloudTestCase {
// Test zero hits
sParams = mapParams(CommonParams.QT, "/sql",
"stmt", "select count(*), sum(a_i), min(a_i), max(a_i), cast(avg(1.0 * a_i) as float), sum(a_f), " +
"min(a_f), max(a_f), avg(a_f) from collection1 where a_s = 'blah'");
"min(a_f), max(a_f), avg(a_f) from collection1 where a_s = 'blah'");
tuples = getTuples(sParams, baseUrl);

View File

@ -28,6 +28,8 @@ import org.apache.solr.client.solrj.io.stream.metrics.CountMetric;
import org.apache.solr.client.solrj.io.stream.metrics.MaxMetric;
import org.apache.solr.client.solrj.io.stream.metrics.MeanMetric;
import org.apache.solr.client.solrj.io.stream.metrics.MinMetric;
import org.apache.solr.client.solrj.io.stream.metrics.PercentileMetric;
import org.apache.solr.client.solrj.io.stream.metrics.StdMetric;
import org.apache.solr.client.solrj.io.stream.metrics.SumMetric;
public class Lang {
@ -103,6 +105,8 @@ public class Lang {
.withFunctionName("max", MaxMetric.class)
.withFunctionName("avg", MeanMetric.class)
.withFunctionName("sum", SumMetric.class)
.withFunctionName("per", PercentileMetric.class)
.withFunctionName("std", StdMetric.class)
.withFunctionName("count", CountMetric.class)
// tuple manipulation operations

View File

@ -349,18 +349,24 @@ public class FacetStream extends TupleStream implements Expressible {
private FieldComparator[] parseBucketSorts(String bucketSortString, Bucket[] buckets) throws IOException {
String[] sorts = bucketSortString.split(",");
String[] sorts = parseSorts(bucketSortString);
FieldComparator[] comps = new FieldComparator[sorts.length];
for(int i=0; i<sorts.length; i++) {
String s = sorts[i];
String[] spec = s.trim().split("\\s+"); //This should take into account spaces in the sort spec.
if(2 != spec.length){
throw new IOException(String.format(Locale.ROOT,"invalid expression - bad bucketSort '%s'. Expected form 'field order'",bucketSortString));
String fieldName = null;
String order = null;
if(s.endsWith("asc") || s.endsWith("ASC")) {
order = "asc";
fieldName = s.substring(0, s.length()-3).trim().replace(" ", "");
} else if(s.endsWith("desc") || s.endsWith("DESC")) {
order = "desc";
fieldName = s.substring(0, s.length()-4).trim().replace(" ", "");
} else {
throw new IOException(String.format(Locale.ROOT,"invalid expression - bad bucketSort '%s'.",bucketSortString));
}
String fieldName = spec[0].trim();
String order = spec[1].trim();
comps[i] = new FieldComparator(fieldName, order.equalsIgnoreCase("asc") ? ComparatorOrder.ASCENDING : ComparatorOrder.DESCENDING);
}
@ -368,6 +374,34 @@ public class FacetStream extends TupleStream implements Expressible {
return comps;
}
private String[] parseSorts(String sortString) {
List<String> sorts = new ArrayList();
boolean inParam = false;
StringBuilder buff = new StringBuilder();
for(int i=0; i<sortString.length(); i++) {
char c = sortString.charAt(i);
if(c == '(') {
inParam=true;
buff.append(c);
} else if (c == ')') {
inParam = false;
buff.append(c);
} else if (c == ',' && !inParam) {
sorts.add(buff.toString().trim());
buff = new StringBuilder();
} else {
buff.append(c);
}
}
if(buff.length() > 0) {
sorts.add(buff.toString());
}
return sorts.toArray(new String[sorts.size()]);
}
private void init(String collection, SolrParams params, Bucket[] buckets, FieldComparator[] bucketSorts, Metric[] metrics, int rows, int offset, int bucketSizeLimit, boolean refine, String method, boolean serializeBucketSizeLimit, int overfetch, String zkHost) throws IOException {
this.zkHost = zkHost;
this.params = new ModifiableSolrParams(params);
@ -568,7 +602,7 @@ public class FacetStream extends TupleStream implements Expressible {
for(Metric metric: metrics) {
String func = metric.getFunctionName();
if(!func.equals("count")) {
if(!func.equals("count") && !func.equals("per") && !func.equals("std")) {
if (!json.contains(metric.getIdentifier())) {
return false;
}
@ -680,18 +714,27 @@ public class FacetStream extends TupleStream implements Expressible {
++level;
boolean comma = false;
for(Metric metric : _metrics) {
//Only compute the metric if it's a leaf node or if the branch level sort equals is the metric
String facetKey = "facet_"+metricCount;
if(level == _buckets.length || fsort.equals(facetKey) ) {
String identifier = metric.getIdentifier();
if (!identifier.startsWith("count(")) {
if (metricCount > 0) {
buf.append(",");
}
buf.append('"').append(facetKey).append("\":\"").append(identifier).append('"');
++metricCount;
String identifier = metric.getIdentifier();
if (!identifier.startsWith("count(")) {
if (comma) {
buf.append(",");
}
if(level == _buckets.length || fsort.equals(facetKey) ) {
comma = true;
if (identifier.startsWith("per(")) {
buf.append("\"facet_").append(metricCount).append("\":\"").append(identifier.replaceFirst("per", "percentile")).append('"');
} else if (identifier.startsWith("std(")) {
buf.append("\"facet_").append(metricCount).append("\":\"").append(identifier.replaceFirst("std", "stddev")).append('"');
} else {
buf.append('"').append(facetKey).append("\":\"").append(identifier).append('"');
}
}
++metricCount;
}
}

View File

@ -17,7 +17,9 @@
package org.apache.solr.client.solrj.io.stream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
@ -40,67 +42,75 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParamete
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
import org.apache.solr.client.solrj.io.stream.metrics.CountMetric;
import org.apache.solr.client.solrj.io.stream.metrics.Metric;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList;
/**
* @since 6.0.0
*/
* @since 6.6.0
*/
public class StatsStream extends TupleStream implements Expressible {
private static final long serialVersionUID = 1;
private Metric[] metrics;
private String zkHost;
private Tuple tuple;
private int index;
private String zkHost;
private SolrParams params;
private String collection;
private boolean done;
private boolean doCount;
private Map<String, Metric> metricMap;
protected transient SolrClientCache cache;
protected transient CloudSolrClient cloudSolrClient;
protected StreamContext streamContext;
private StreamContext context;
public StatsStream(String zkHost,
String collection,
SolrParams params,
Metric[] metrics) {
init(zkHost, collection, params, metrics);
}
private void init(String zkHost, String collection, SolrParams params, Metric[] metrics) {
this.zkHost = zkHost;
this.params = params;
this.metrics = metrics;
this.collection = collection;
metricMap = new HashMap();
for(Metric metric : metrics) {
metricMap.put(metric.getIdentifier(), metric);
}
String collection,
SolrParams params,
Metric[] metrics
) throws IOException {
init(collection, params, metrics, zkHost);
}
public StatsStream(StreamExpression expression, StreamFactory factory) throws IOException{
// grab all parameters out
String collectionName = factory.getValueOperand(expression, 0);
List<StreamExpressionNamedParameter> namedParams = factory.getNamedOperands(expression);
List<StreamExpression> metricExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Metric.class);
StreamExpressionNamedParameter zkHostExpression = factory.getNamedOperand(expression, "zkHost");
if(collectionName.indexOf('"') > -1) {
collectionName = collectionName.replaceAll("\"", "").replaceAll(" ", "");
}
List<StreamExpressionNamedParameter> namedParams = factory.getNamedOperands(expression);
StreamExpressionNamedParameter zkHostExpression = factory.getNamedOperand(expression, "zkHost");
List<StreamExpression> metricExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, Metric.class);
// Collection Name
if(null == collectionName){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - collectionName expected as first operand",expression));
}
// Construct the metrics
Metric[] metrics = null;
if(metricExpressions.size() > 0) {
metrics = new Metric[metricExpressions.size()];
for(int idx = 0; idx < metricExpressions.size(); ++idx){
metrics[idx] = factory.constructMetric(metricExpressions.get(idx));
}
} else {
metrics = new Metric[1];
metrics[0] = new CountMetric();
}
// pull out known named params
ModifiableSolrParams params = new ModifiableSolrParams();
for(StreamExpressionNamedParameter namedParam : namedParams){
if(!namedParam.getName().equals("zkHost")){
params.set(namedParam.getName(), namedParam.getParameter().toString().trim());
params.add(namedParam.getName(), namedParam.getParameter().toString().trim());
}
}
@ -115,51 +125,55 @@ public class StatsStream extends TupleStream implements Expressible {
if(zkHost == null) {
zkHost = factory.getDefaultZkHost();
}
}
else if(zkHostExpression.getParameter() instanceof StreamExpressionValue){
} else if(zkHostExpression.getParameter() instanceof StreamExpressionValue){
zkHost = ((StreamExpressionValue)zkHostExpression.getParameter()).getValue();
}
/*
if(null == zkHost){
throw new IOException(String.format(Locale.ROOT,"invalid expression %s - zkHost not found for collection '%s'",expression,collectionName));
}
*/
// metrics, optional - if not provided then why are you using this?
Metric[] metrics = new Metric[metricExpressions.size()];
for(int idx = 0; idx < metricExpressions.size(); ++idx){
metrics[idx] = factory.constructMetric(metricExpressions.get(idx));
}
// We've got all the required items
init(zkHost, collectionName, params, metrics);
init(collectionName, params, metrics, zkHost);
}
public String getCollection() {
return this.collection;
}
private void init(String collection,
SolrParams params,
Metric[] metrics,
String zkHost) throws IOException {
this.zkHost = zkHost;
this.collection = collection;
this.metrics = metrics;
this.params = params;
}
@Override
public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
// functionName(collectionName, param1, param2, ..., paramN, sort="comp", sum(fieldA), avg(fieldB))
// function name
StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
// collection
expression.addParameter(collection);
// parameters
ModifiableSolrParams mParams = new ModifiableSolrParams(params);
for (Entry<String, String[]> param : mParams.getMap().entrySet()) {
expression.addParameter(new StreamExpressionNamedParameter(param.getKey(), String.join(",", param.getValue())));
if(collection.indexOf(',') > -1) {
expression.addParameter("\""+collection+"\"");
} else {
expression.addParameter(collection);
}
// zkHost
expression.addParameter(new StreamExpressionNamedParameter("zkHost", zkHost));
// parameters
ModifiableSolrParams tmpParams = new ModifiableSolrParams(params);
for (Entry<String, String[]> param : tmpParams.getMap().entrySet()) {
expression.addParameter(new StreamExpressionNamedParameter(param.getKey(),
String.join(",", param.getValue())));
}
// metrics
for(Metric metric : metrics){
expression.addParameter(metric.toExpression(factory));
}
// zkHost
expression.addParameter(new StreamExpressionNamedParameter("zkHost", zkHost));
return expression;
}
@ -173,45 +187,46 @@ public class StatsStream extends TupleStream implements Expressible {
explanation.setExpressionType(ExpressionType.STREAM_SOURCE);
explanation.setExpression(toExpression(factory).toString());
// child is a datastore so add it at this point
StreamExplanation child = new StreamExplanation(getStreamNodeId() + "-datastore");
child.setFunctionName(String.format(Locale.ROOT, "solr (worker ? of ?)"));
// TODO: fix this so we know the # of workers - check with Joel about a Stat's ability to be in a
// parallel stream.
child.setFunctionName(String.format(Locale.ROOT, "solr (%s)", collection));
// TODO: fix this so we know the # of workers - check with Joel about a Topic's ability to be in a
// parallel stream.
child.setImplementingClass("Solr/Lucene");
child.setExpressionType(ExpressionType.DATASTORE);
ModifiableSolrParams mParams = new ModifiableSolrParams(params);
child.setExpression(mParams.getMap().entrySet().stream().map(e -> String.format(Locale.ROOT, "%s=%s", e.getKey(), e.getValue())).collect(Collectors.joining(",")));
child.setExpression(params.stream().map(e -> String.format(Locale.ROOT, "%s=%s", e.getKey(), Arrays.toString(e.getValue()))).collect(Collectors.joining(",")));
explanation.addChild(child);
return explanation;
}
public void setStreamContext(StreamContext context) {
streamContext = context;
this.context = context;
cache = context.getSolrClientCache();
}
public List<TupleStream> children() {
return new ArrayList<>();
return new ArrayList();
}
public void open() throws IOException {
ModifiableSolrParams paramsLoc = new ModifiableSolrParams(this.params);
addStats(paramsLoc, metrics);
paramsLoc.set("stats", "true");
paramsLoc.set("rows", "0");
if (streamContext.isLocal()) {
paramsLoc.set("distrib", "false");
}
Map<String, List<String>> shardsMap = (Map<String, List<String>>)streamContext.get("shards");
String json = getJsonFacetString(metrics);
ModifiableSolrParams paramsLoc = new ModifiableSolrParams(params);
paramsLoc.set("json.facet", json);
paramsLoc.set("rows", "0");
Map<String, List<String>> shardsMap = (Map<String, List<String>>)context.get("shards");
if(shardsMap == null) {
QueryRequest request = new QueryRequest(paramsLoc, SolrRequest.METHOD.POST);
CloudSolrClient cloudSolrClient = cache.getCloudSolrClient(zkHost);
cloudSolrClient = cache.getCloudSolrClient(zkHost);
try {
NamedList response = cloudSolrClient.request(request, collection);
this.tuple = getTuple(response);
getTuples(response, metrics);
} catch (Exception e) {
throw new IOException(e);
}
@ -228,7 +243,7 @@ public class StatsStream extends TupleStream implements Expressible {
QueryRequest request = new QueryRequest(paramsLoc, SolrRequest.METHOD.POST);
try {
NamedList response = client.request(request);
this.tuple = getTuple(response);
getTuples(response, metrics);
} catch (Exception e) {
throw new IOException(e);
}
@ -246,117 +261,98 @@ public class StatsStream extends TupleStream implements Expressible {
return builder.toString();
}
public void close() throws IOException {
}
public Tuple read() throws IOException {
if(!done) {
done = true;
if(index == 0) {
++index;
return tuple;
} else {
Map<String, Object> fields = new HashMap<>();
Map fields = new HashMap();
fields.put("EOF", true);
return new Tuple(fields);
Tuple tuple = new Tuple(fields);
return tuple;
}
}
public StreamComparator getStreamSort() {
return null;
private String getJsonFacetString(Metric[] _metrics) {
StringBuilder buf = new StringBuilder();
appendJson(buf, _metrics);
return "{"+buf.toString()+"}";
}
private void addStats(ModifiableSolrParams params, Metric[] _metrics) {
Map<String, List<String>> m = new HashMap<>();
private void appendJson(StringBuilder buf,
Metric[] _metrics) {
int metricCount = 0;
for(Metric metric : _metrics) {
String metricId = metric.getIdentifier();
if(metricId.contains("(")) {
metricId = metricId.substring(0, metricId.length()-1);
String[] parts = metricId.split("\\(");
String function = parts[0];
String column = parts[1];
List<String> stats = m.get(column);
if(stats == null) {
stats = new ArrayList<>();
String identifier = metric.getIdentifier();
if(!identifier.startsWith("count(")) {
if(metricCount>0) {
buf.append(",");
}
if(!column.equals("*")) {
m.put(column, stats);
}
if(function.equals("min")) {
stats.add("min");
} else if(function.equals("max")) {
stats.add("max");
} else if(function.equals("sum")) {
stats.add("sum");
} else if(function.equals("avg")) {
stats.add("mean");
} else if(function.equals("count")) {
this.doCount = true;
if(identifier.startsWith("per(")) {
buf.append("\"facet_").append(metricCount).append("\":\"").append(identifier.replaceFirst("per", "percentile")).append('"');
} else if(identifier.startsWith("std(")) {
buf.append("\"facet_").append(metricCount).append("\":\"").append(identifier.replaceFirst("std", "stddev")).append('"');
} else {
buf.append("\"facet_").append(metricCount).append("\":\"").append(identifier).append('"');
}
++metricCount;
}
}
for(Entry<String, List<String>> entry : m.entrySet()) {
StringBuilder buf = new StringBuilder();
List<String> stats = entry.getValue();
buf.append("{!");
for(String stat : stats) {
buf.append(stat).append("=").append("true ");
}
buf.append("}").append(entry.getKey());
params.add("stats.field", buf.toString());
}
//buf.append("}");
}
private Tuple getTuple(NamedList response) {
private void getTuples(NamedList response,
Metric[] metrics) {
Map<String, Object> map = new HashMap<>();
SolrDocumentList solrDocumentList = (SolrDocumentList) response.get("response");
this.tuple = new Tuple(new HashMap());
NamedList facets = (NamedList)response.get("facets");
System.out.println("###### Facets:"+facets);
fillTuple(tuple, facets, metrics);
}
long count = solrDocumentList.getNumFound();
private void fillTuple(Tuple t,
NamedList nl,
Metric[] _metrics) {
if(doCount) {
map.put("count(*)", count);
if(nl == null) {
return;
}
if(count != 0) {
NamedList stats = (NamedList)response.get("stats");
NamedList statsFields = (NamedList)stats.get("stats_fields");
for(int i=0; i<statsFields.size(); i++) {
String field = statsFields.getName(i);
NamedList theStats = (NamedList)statsFields.getVal(i);
for(int s=0; s<theStats.size(); s++) {
addStat(map, field, theStats.getName(s), theStats.getVal(s));
int m = 0;
for(Metric metric : _metrics) {
String identifier = metric.getIdentifier();
if(!identifier.startsWith("count(")) {
if(nl.get("facet_"+m) != null) {
Object d = nl.get("facet_" + m);
if(d instanceof Number) {
if (metric.outputLong) {
t.put(identifier, Math.round(((Number)d).doubleValue()));
} else {
t.put(identifier, ((Number)d).doubleValue());
}
} else {
t.put(identifier, d);
}
}
++m;
} else {
long l = ((Number)nl.get("count")).longValue();
t.put("count(*)", l);
}
}
return new Tuple(map);
}
public int getCost() {
return 0;
}
private void addStat(Map<String, Object> map, String field, String stat, Object val) {
if(stat.equals("mean")) {
String name = "avg("+field+")";
Metric m = metricMap.get(name);
if(m.outputLong) {
Number num = (Number) val;
map.put(name, Math.round(num.doubleValue()));
} else {
map.put(name, val);
}
} else {
map.put(stat+"("+field+")", val);
}
@Override
public StreamComparator getStreamSort() {
return null;
}
}

View File

@ -366,7 +366,13 @@ public class TimeSeriesStream extends TupleStream implements Expressible {
if(metricCount>0) {
buf.append(",");
}
buf.append("\"facet_").append(metricCount).append("\":\"").append(identifier).append('"');
if(identifier.startsWith("per(")) {
buf.append("\"facet_").append(metricCount).append("\":\"").append(identifier.replaceFirst("per", "percentile")).append('"');
} else if(identifier.startsWith("std(")) {
buf.append("\"facet_").append(metricCount).append("\":\"").append(identifier.replaceFirst("std", "stddev")).append('"');
} else {
buf.append("\"facet_").append(metricCount).append("\":\"").append(identifier).append('"');
}
++metricCount;
}
}

View File

@ -0,0 +1,84 @@
/*
* 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.metrics;
import java.io.IOException;
import java.util.Locale;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
public class PercentileMetric extends Metric {
private long longMax = -Long.MIN_VALUE;
private double doubleMax = -Double.MAX_VALUE;
private String columnName;
public PercentileMetric(String columnName, int percentile){
init("per", columnName, percentile);
}
public PercentileMetric(StreamExpression expression, StreamFactory factory) throws IOException{
// grab all parameters out
String functionName = expression.getFunctionName();
String columnName = factory.getValueOperand(expression, 0);
int percentile = Integer.parseInt(factory.getValueOperand(expression, 1));
// validate expression contains only what we want.
if(null == columnName){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expected %s(columnName)", expression, functionName));
}
if(2 != expression.getParameters().size()){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - unknown operands found", expression));
}
init(functionName, columnName, percentile);
}
private void init(String functionName, String columnName, int percentile){
this.columnName = columnName;
setFunctionName(functionName);
setIdentifier(functionName, "(", columnName, ","+percentile, ")");
}
public Number getValue() {
if(longMax == Long.MIN_VALUE) {
return doubleMax;
} else {
return longMax;
}
}
public String[] getColumns() {
return new String[]{columnName};
}
public void update(Tuple tuple) {
}
public Metric newInstance() {
return new MaxMetric(columnName);
}
@Override
public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
return new StreamExpression(getFunctionName()).withParameter(columnName);
}
}

View File

@ -0,0 +1,93 @@
/*
* 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.metrics;
import java.io.IOException;
import java.util.Locale;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
public class StdMetric extends Metric {
// How'd the MeanMetric get to be so mean?
// Maybe it was born with it.
// Maybe it was mayba-mean.
//
// I'll see myself out.
private String columnName;
private double doubleSum;
private long longSum;
private long count;
public StdMetric(String columnName){
init("std", columnName, false);
}
public StdMetric(String columnName, boolean outputLong){
init("std", columnName, outputLong);
}
public StdMetric(StreamExpression expression, StreamFactory factory) throws IOException{
// grab all parameters out
String functionName = expression.getFunctionName();
String columnName = factory.getValueOperand(expression, 0);
String outputLong = factory.getValueOperand(expression, 1);
// validate expression contains only what we want.
if(null == columnName){
throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expected %s(columnName)", expression, functionName));
}
boolean ol = false;
if(outputLong != null) {
ol = Boolean.parseBoolean(outputLong);
}
init(functionName, columnName, ol);
}
private void init(String functionName, String columnName, boolean outputLong){
this.columnName = columnName;
this.outputLong = outputLong;
setFunctionName(functionName);
setIdentifier(functionName, "(", columnName, ")");
}
public void update(Tuple tuple) {
}
public Metric newInstance() {
return new MeanMetric(columnName, outputLong);
}
public String[] getColumns() {
return new String[]{columnName};
}
public Number getValue() {
return null;
}
@Override
public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
return new StreamExpression(getFunctionName()).withParameter(columnName).withParameter(Boolean.toString(outputLong));
}
}

View File

@ -85,7 +85,7 @@ public class TestLang extends SolrTestCase {
"getSupportPoints", "pairSort", "log10", "plist", "recip", "pivot", "ltrim", "rtrim", "export",
"zplot", "natural", "repeat", "movingMAD", "hashRollup", "noop", "var", "stddev", "recNum", "isNull",
"notNull", "matches", "projectToBorder", "double", "long", "parseCSV", "parseTSV", "dateTime",
"split", "upper", "trim", "lower", "trunc", "cosine", "dbscan"};
"split", "upper", "trim", "lower", "trunc", "cosine", "dbscan", "per", "std"};
@Test
public void testLang() {

View File

@ -48,6 +48,8 @@ import org.apache.solr.client.solrj.io.stream.metrics.CountMetric;
import org.apache.solr.client.solrj.io.stream.metrics.MaxMetric;
import org.apache.solr.client.solrj.io.stream.metrics.MeanMetric;
import org.apache.solr.client.solrj.io.stream.metrics.MinMetric;
import org.apache.solr.client.solrj.io.stream.metrics.PercentileMetric;
import org.apache.solr.client.solrj.io.stream.metrics.StdMetric;
import org.apache.solr.client.solrj.io.stream.metrics.SumMetric;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
@ -762,7 +764,9 @@ public class StreamExpressionTest extends SolrCloudTestCase {
.withFunctionName("min", MinMetric.class)
.withFunctionName("max", MaxMetric.class)
.withFunctionName("avg", MeanMetric.class)
.withFunctionName("count", CountMetric.class);
.withFunctionName("count", CountMetric.class)
.withFunctionName("std", StdMetric.class)
.withFunctionName("per", PercentileMetric.class);
StreamExpression expression;
TupleStream stream;
@ -771,7 +775,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
SolrClientCache cache = new SolrClientCache();
try {
streamContext.setSolrClientCache(cache);
String expr = "stats(" + COLLECTIONORALIAS + ", q=*:*, sum(a_i), sum(a_f), min(a_i), min(a_f), max(a_i), max(a_f), avg(a_i), avg(a_f), count(*))";
String expr = "stats(" + COLLECTIONORALIAS + ", q=*:*, sum(a_i), sum(a_f), min(a_i), min(a_f), max(a_i), max(a_f), avg(a_i), avg(a_f), std(a_i), std(a_f), per(a_i, 50), per(a_f, 50), count(*))";
expression = StreamExpressionParser.parse(expr);
stream = factory.constructStream(expression);
stream.setStreamContext(streamContext);
@ -792,6 +796,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
Double maxf = tuple.getDouble("max(a_f)");
Double avgi = tuple.getDouble("avg(a_i)");
Double avgf = tuple.getDouble("avg(a_f)");
Double stdi = tuple.getDouble("std(a_i)");
Double stdf = tuple.getDouble("std(a_f)");
Double peri = tuple.getDouble("per(a_i,50)");
Double perf = tuple.getDouble("per(a_f,50)");
Double count = tuple.getDouble("count(*)");
assertTrue(sumi.longValue() == 70);
@ -802,11 +810,16 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(maxf.doubleValue() == 10.0D);
assertTrue(avgi.doubleValue() == 7.0D);
assertTrue(avgf.doubleValue() == 5.5D);
assertTrue(stdi.doubleValue() == 5.477225575051661D);
assertTrue(stdf.doubleValue() == 3.0276503540974917D);
assertTrue(peri.doubleValue() == 7.0D);
assertTrue(perf.doubleValue() == 5.5D);
assertTrue(count.doubleValue() == 10);
//Test without query
expr = "stats(" + COLLECTIONORALIAS + ", sum(a_i), sum(a_f), min(a_i), min(a_f), max(a_i), max(a_f), avg(a_i), avg(a_f), count(*))";
expr = "stats(" + COLLECTIONORALIAS + ", sum(a_i), sum(a_f), min(a_i), min(a_f), max(a_i), max(a_f), avg(a_i), avg(a_f), std(a_i), std(a_f), per(a_i, 50), per(a_f, 50), count(*))";
expression = StreamExpressionParser.parse(expr);
stream = factory.constructStream(expression);
stream.setStreamContext(streamContext);
@ -827,6 +840,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
maxf = tuple.getDouble("max(a_f)");
avgi = tuple.getDouble("avg(a_i)");
avgf = tuple.getDouble("avg(a_f)");
stdi = tuple.getDouble("std(a_i)");
stdf = tuple.getDouble("std(a_f)");
peri = tuple.getDouble("per(a_i,50)");
perf = tuple.getDouble("per(a_f,50)");
count = tuple.getDouble("count(*)");
assertTrue(sumi.longValue() == 70);
@ -837,12 +854,15 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(maxf.doubleValue() == 10.0D);
assertTrue(avgi.doubleValue() == 7.0D);
assertTrue(avgf.doubleValue() == 5.5D);
assertTrue(stdi.doubleValue() == 5.477225575051661D);
assertTrue(stdf.doubleValue() == 3.0276503540974917D);
assertTrue(peri.doubleValue() == 7.0D);
assertTrue(perf.doubleValue() == 5.5D);
assertTrue(count.doubleValue() == 10);
//Test with shards parameter
List<String> shardUrls = TupleStream.getShards(cluster.getZkServer().getZkAddress(), COLLECTIONORALIAS, streamContext);
expr = "stats(myCollection, q=*:*, sum(a_i), sum(a_f), min(a_i), min(a_f), max(a_i), max(a_f), avg(a_i), avg(a_f), count(*))";
expr = "stats(myCollection, q=*:*, sum(a_i), sum(a_f), min(a_i), min(a_f), max(a_i), max(a_f), avg(a_i), avg(a_f), std(a_i), std(a_f), per(a_i, 50), per(a_f, 50), count(*))";
Map<String, List<String>> shardsMap = new HashMap();
shardsMap.put("myCollection", shardUrls);
StreamContext context = new StreamContext();
@ -867,6 +887,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
maxf = tuple.getDouble("max(a_f)");
avgi = tuple.getDouble("avg(a_i)");
avgf = tuple.getDouble("avg(a_f)");
stdi = tuple.getDouble("std(a_i)");
stdf = tuple.getDouble("std(a_f)");
peri = tuple.getDouble("per(a_i,50)");
perf = tuple.getDouble("per(a_f,50)");
count = tuple.getDouble("count(*)");
assertTrue(sumi.longValue() == 70);
@ -877,6 +901,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(maxf.doubleValue() == 10.0D);
assertTrue(avgi.doubleValue() == 7.0D);
assertTrue(avgf.doubleValue() == 5.5D);
assertTrue(stdi.doubleValue() == 5.477225575051661D);
assertTrue(stdf.doubleValue() == 3.0276503540974917D);
assertTrue(peri.doubleValue() == 7.0D);
assertTrue(perf.doubleValue() == 5.5D);
assertTrue(count.doubleValue() == 10);
//Execersise the /stream hander
@ -1135,6 +1163,8 @@ public class StreamExpressionTest extends SolrCloudTestCase {
.withFunctionName("min", MinMetric.class)
.withFunctionName("max", MaxMetric.class)
.withFunctionName("avg", MeanMetric.class)
.withFunctionName("std", StdMetric.class)
.withFunctionName("per", PercentileMetric.class)
.withFunctionName("count", CountMetric.class);
// Basic test
@ -1150,6 +1180,8 @@ public class StreamExpressionTest extends SolrCloudTestCase {
+ "min(a_i), min(a_f), "
+ "max(a_i), max(a_f), "
+ "avg(a_i), avg(a_f), "
+ "std(a_i), std(a_f),"
+ "per(a_i, 50), per(a_f, 50),"
+ "count(*)"
+ ")";
@ -1158,7 +1190,6 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assert(tuples.size() == 3);
//Test Long and Double Sums
Tuple tuple = tuples.get(0);
String bucket = tuple.getString("a_s");
@ -1170,6 +1201,12 @@ public class StreamExpressionTest extends SolrCloudTestCase {
Double maxf = tuple.getDouble("max(a_f)");
Double avgi = tuple.getDouble("avg(a_i)");
Double avgf = tuple.getDouble("avg(a_f)");
Double stdi = tuple.getDouble("std(a_i)");
Double stdf = tuple.getDouble("std(a_f)");
Double peri = tuple.getDouble("per(a_i,50)");
Double perf = tuple.getDouble("per(a_f,50)");
Double count = tuple.getDouble("count(*)");
assertTrue(bucket.equals("hello4"));
@ -1182,6 +1219,11 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(avgi.doubleValue() == 7.5D);
assertTrue(avgf.doubleValue() == 5.5D);
assertTrue(count.doubleValue() == 2);
assertTrue(stdi.doubleValue() == 4.949747468305833D);
assertTrue(stdf.doubleValue() == 2.1213203435596424D);
assertTrue(peri.doubleValue() == 7.5D);
assertTrue(perf.doubleValue() == 5.5D);
tuple = tuples.get(1);
bucket = tuple.getString("a_s");
@ -1194,6 +1236,11 @@ public class StreamExpressionTest extends SolrCloudTestCase {
avgi = tuple.getDouble("avg(a_i)");
avgf = tuple.getDouble("avg(a_f)");
count = tuple.getDouble("count(*)");
stdi = tuple.getDouble("std(a_i)");
stdf = tuple.getDouble("std(a_f)");
peri = tuple.getDouble("per(a_i,50)");
perf = tuple.getDouble("per(a_f,50)");
assertTrue(bucket.equals("hello0"));
assertTrue(sumi.doubleValue() == 17.0D);
@ -1205,6 +1252,11 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(avgi.doubleValue() == 4.25D);
assertTrue(avgf.doubleValue() == 4.5D);
assertTrue(count.doubleValue() == 4);
assertTrue(stdi.doubleValue() == 6.551081335677848D);
assertTrue(stdf.doubleValue() == 4.041451884327381D);
assertTrue(peri.doubleValue() == 1.5D);
assertTrue(perf.doubleValue() == 3.5D);
tuple = tuples.get(2);
bucket = tuple.getString("a_s");
@ -1217,6 +1269,15 @@ public class StreamExpressionTest extends SolrCloudTestCase {
avgi = tuple.getDouble("avg(a_i)");
avgf = tuple.getDouble("avg(a_f)");
count = tuple.getDouble("count(*)");
stdi = tuple.getDouble("std(a_i)");
stdf = tuple.getDouble("std(a_f)");
peri = tuple.getDouble("per(a_i,50)");
perf = tuple.getDouble("per(a_f,50)");
System.out.println("STD and Per:"+stdi+":"+stdf+":"+peri+":"+perf);
//STD and Per:4.509249752822894:2.6457513110645907:11.0:7.0
//assert(false);
assertTrue(bucket.equals("hello3"));
assertTrue(sumi.doubleValue() == 38.0D);
@ -1228,6 +1289,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(avgi.doubleValue() == 9.5D);
assertTrue(avgf.doubleValue() == 6.5D);
assertTrue(count.doubleValue() == 4);
assertTrue(stdi.doubleValue() == 4.509249752822894D);
assertTrue(stdf.doubleValue() == 2.6457513110645907D);
assertTrue(peri.doubleValue() == 11.0D);
assertTrue(perf.doubleValue() == 7.0D);
//Reverse the Sort.
@ -1244,6 +1309,8 @@ public class StreamExpressionTest extends SolrCloudTestCase {
+ "min(a_i), min(a_f), "
+ "max(a_i), max(a_f), "
+ "avg(a_i), avg(a_f), "
+ "std(a_i), std(a_f),"
+ "per(a_i, 50), per(a_f, 50),"
+ "count(*)"
+ ")";
@ -1264,6 +1331,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
avgi = tuple.getDouble("avg(a_i)");
avgf = tuple.getDouble("avg(a_f)");
count = tuple.getDouble("count(*)");
stdi = tuple.getDouble("std(a_i)");
stdf = tuple.getDouble("std(a_f)");
peri = tuple.getDouble("per(a_i,50)");
perf = tuple.getDouble("per(a_f,50)");
assertTrue(bucket.equals("hello3"));
assertTrue(sumi.doubleValue() == 38.0D);
@ -1275,6 +1346,11 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(avgi.doubleValue() == 9.5D);
assertTrue(avgf.doubleValue() == 6.5D);
assertTrue(count.doubleValue() == 4);
assertTrue(stdi.doubleValue() == 4.509249752822894D);
assertTrue(stdf.doubleValue() == 2.6457513110645907D);
assertTrue(peri.doubleValue() == 11.0D);
assertTrue(perf.doubleValue() == 7.0D);
tuple = tuples.get(1);
bucket = tuple.getString("a_s");
@ -1287,6 +1363,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
avgi = tuple.getDouble("avg(a_i)");
avgf = tuple.getDouble("avg(a_f)");
count = tuple.getDouble("count(*)");
stdi = tuple.getDouble("std(a_i)");
stdf = tuple.getDouble("std(a_f)");
peri = tuple.getDouble("per(a_i,50)");
perf = tuple.getDouble("per(a_f,50)");
assertTrue(bucket.equals("hello0"));
assertTrue(sumi.doubleValue() == 17.0D);
@ -1298,6 +1378,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(avgi.doubleValue() == 4.25D);
assertTrue(avgf.doubleValue() == 4.5D);
assertTrue(count.doubleValue() == 4);
assertTrue(stdi.doubleValue() == 6.551081335677848D);
assertTrue(stdf.doubleValue() == 4.041451884327381D);
assertTrue(peri.doubleValue() == 1.5D);
assertTrue(perf.doubleValue() == 3.5D);
tuple = tuples.get(2);
bucket = tuple.getString("a_s");
@ -1310,6 +1394,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
avgi = tuple.getDouble("avg(a_i)");
avgf = tuple.getDouble("avg(a_f)");
count = tuple.getDouble("count(*)");
stdi = tuple.getDouble("std(a_i)");
stdf = tuple.getDouble("std(a_f)");
peri = tuple.getDouble("per(a_i,50)");
perf = tuple.getDouble("per(a_f,50)");
assertTrue(bucket.equals("hello4"));
assertTrue(sumi.longValue() == 15);
@ -1321,6 +1409,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(avgi.doubleValue() == 7.5D);
assertTrue(avgf.doubleValue() == 5.5D);
assertTrue(count.doubleValue() == 2);
assertTrue(stdi.doubleValue() == 4.949747468305833D);
assertTrue(stdf.doubleValue() == 2.1213203435596424D);
assertTrue(peri.doubleValue() == 7.5D);
assertTrue(perf.doubleValue() == 5.5D);
clause = "facet("
@ -1477,6 +1569,8 @@ public class StreamExpressionTest extends SolrCloudTestCase {
+ "min(a_i), min(a_f), "
+ "max(a_i), max(a_f), "
+ "avg(a_i), avg(a_f), "
+ "std(a_i), std(a_f),"
+ "per(a_i, 50), per(a_f, 50),"
+ "count(*)"
+ ")";
@ -1497,7 +1591,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
avgi = tuple.getDouble("avg(a_i)");
avgf = tuple.getDouble("avg(a_f)");
count = tuple.getDouble("count(*)");
stdi = tuple.getDouble("std(a_i)");
stdf = tuple.getDouble("std(a_f)");
peri = tuple.getDouble("per(a_i,50)");
perf = tuple.getDouble("per(a_f,50)");
assertTrue(bucket.equals("hello4"));
assertTrue(sumi.longValue() == 15);
@ -1509,7 +1606,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(avgi.doubleValue() == 7.5D);
assertTrue(avgf.doubleValue() == 5.5D);
assertTrue(count.doubleValue() == 2);
assertTrue(stdi.doubleValue() == 4.949747468305833D);
assertTrue(stdf.doubleValue() == 2.1213203435596424D);
assertTrue(peri.doubleValue() == 7.5D);
assertTrue(perf.doubleValue() == 5.5D);
tuple = tuples.get(1);
bucket = tuple.getString("a_s");
@ -1522,6 +1622,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
avgi = tuple.getDouble("avg(a_i)");
avgf = tuple.getDouble("avg(a_f)");
count = tuple.getDouble("count(*)");
stdi = tuple.getDouble("std(a_i)");
stdf = tuple.getDouble("std(a_f)");
peri = tuple.getDouble("per(a_i,50)");
perf = tuple.getDouble("per(a_f,50)");
assertTrue(bucket.equals("hello3"));
assertTrue(sumi.doubleValue() == 38.0D);
@ -1533,6 +1637,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(avgi.doubleValue() == 9.5D);
assertTrue(avgf.doubleValue() == 6.5D);
assertTrue(count.doubleValue() == 4);
assertTrue(stdi.doubleValue() == 4.509249752822894D);
assertTrue(stdf.doubleValue() == 2.6457513110645907D);
assertTrue(peri.doubleValue() == 11.0D);
assertTrue(perf.doubleValue() == 7.0D);
tuple = tuples.get(2);
@ -1546,6 +1654,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
avgi = tuple.getDouble("avg(a_i)");
avgf = tuple.getDouble("avg(a_f)");
count = tuple.getDouble("count(*)");
stdi = tuple.getDouble("std(a_i)");
stdf = tuple.getDouble("std(a_f)");
peri = tuple.getDouble("per(a_i,50)");
perf = tuple.getDouble("per(a_f,50)");
assertTrue(bucket.equals("hello0"));
assertTrue(sumi.doubleValue() == 17.0D);
@ -1557,6 +1669,11 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(avgi.doubleValue() == 4.25D);
assertTrue(avgf.doubleValue() == 4.5D);
assertTrue(count.doubleValue() == 4);
assertTrue(stdi.doubleValue() == 6.551081335677848D);
assertTrue(stdf.doubleValue() == 4.041451884327381D);
assertTrue(peri.doubleValue() == 1.5D);
assertTrue(perf.doubleValue() == 3.5D);
//Test index sort
@ -1810,14 +1927,16 @@ public class StreamExpressionTest extends SolrCloudTestCase {
.withFunctionName("min", MinMetric.class)
.withFunctionName("max", MaxMetric.class)
.withFunctionName("avg", MeanMetric.class)
.withFunctionName("count", CountMetric.class);
.withFunctionName("count", CountMetric.class)
.withFunctionName("std", StdMetric.class)
.withFunctionName("per", PercentileMetric.class);
// Basic test
clause = "facet("
+ "collection1, "
+ "q=\"*:*\", "
+ "buckets=\"level1_s, level2_s\", "
+ "bucketSorts=\"sum(a_i) desc, sum(a_i) desc)\", "
+ "bucketSorts=\"sum(a_i) desc, sum(a_i) desc\", "
+ "bucketSizeLimit=100, "
+ "sum(a_i), count(*)"
+ ")";
@ -1897,7 +2016,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
+ "collection1, "
+ "q=\"*:*\", "
+ "buckets=\"level1_s, level2_s\", "
+ "bucketSorts=\"level1_s desc, level2_s desc)\", "
+ "bucketSorts=\"level1_s desc, level2_s desc\", "
+ "bucketSizeLimit=100, "
+ "sum(a_i), count(*)"
+ ")";
@ -1972,6 +2091,89 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(bucket2.equals("a"));
assertTrue(sumi.longValue() == 2);
assertTrue(count.doubleValue() == 2);
//Add sorts for percentile
clause = "facet("
+ "collection1, "
+ "q=\"*:*\", "
+ "buckets=\"level1_s, level2_s\", "
+ "bucketSorts=\"per(a_i, 50) desc, std(a_i) desc\", "
+ "bucketSizeLimit=100, "
+ "std(a_i), per(a_i,50)"
+ ")";
stream = factory.constructStream(clause);
tuples = getTuples(stream);
assert(tuples.size() == 6);
tuple = tuples.get(0);
bucket1 = tuple.getString("level1_s");
bucket2 = tuple.getString("level2_s");
double stdi = tuple.getDouble("std(a_i)");
double peri = tuple.getDouble("per(a_i,50)");
assertTrue(bucket1.equals("hello3"));
assertTrue(bucket2.equals("b"));
assertTrue(stdi == 1.5275252316519468D);
assertTrue(peri == 12.0D);
tuple = tuples.get(1);
bucket1 = tuple.getString("level1_s");
bucket2 = tuple.getString("level2_s");
stdi = tuple.getDouble("std(a_i)");
peri = tuple.getDouble("per(a_i,50)");
assertTrue(bucket1.equals("hello4"));
assertTrue(bucket2.equals("b"));
assertTrue(stdi == 0.0D);
assertTrue(peri == 11.0);
tuple = tuples.get(2);
bucket1 = tuple.getString("level1_s");
bucket2 = tuple.getString("level2_s");
stdi = tuple.getDouble("std(a_i)");
peri = tuple.getDouble("per(a_i,50)");
assertTrue(bucket1.equals("hello0"));
assertTrue(bucket2.equals("b"));
assertTrue(stdi == 9.192388155425117D);
assertTrue(peri == 7.5D);
tuple = tuples.get(3);
bucket1 = tuple.getString("level1_s");
bucket2 = tuple.getString("level2_s");
stdi = tuple.getDouble("std(a_i)");
peri = tuple.getDouble("per(a_i,50)");
assertTrue(bucket1.equals("hello4"));
assertTrue(bucket2.equals("a"));
assertTrue(stdi == 0.0D);
assertTrue(peri == 4.0D);
tuple = tuples.get(4);
bucket1 = tuple.getString("level1_s");
bucket2 = tuple.getString("level2_s");
stdi = tuple.getDouble("std(a_i)");
peri = tuple.getDouble("per(a_i,50)");
assertTrue(bucket1.equals("hello3"));
assertTrue(bucket2.equals("a"));
assertTrue(stdi == 0.0D);
assertTrue(peri == 3.0D);
tuple = tuples.get(5);
bucket1 = tuple.getString("level1_s");
bucket2 = tuple.getString("level2_s");
stdi = tuple.getDouble("std(a_i)");
peri = tuple.getDouble("per(a_i,50)");
assertTrue(bucket1.equals("hello0"));
assertTrue(bucket2.equals("a"));
assertTrue(stdi == 1.4142135623730951D);
assertTrue(peri == 1.0D);
}
@Test
@ -2447,7 +2649,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
"end=\"2017-12-01T01:00:00.000Z\", " +
"gap=\"+1YEAR\", " +
"field=\"test_dt\", " +
"count(*), sum(price_f), max(price_f), min(price_f))";
"count(*), sum(price_f), max(price_f), min(price_f), avg(price_f), std(price_f), per(price_f, 50))";
ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
paramsLoc.set("expr", expr);
paramsLoc.set("qt", "/stream");
@ -2465,38 +2667,52 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(tuples.get(0).getDouble("sum(price_f)").equals(10000D));
assertTrue(tuples.get(0).getDouble("max(price_f)").equals(100D));
assertTrue(tuples.get(0).getDouble("min(price_f)").equals(100D));
assertTrue(tuples.get(0).getDouble("avg(price_f)").equals(100D));
assertTrue(tuples.get(0).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(0).getDouble("per(price_f,50)").equals(100D));
assertTrue(tuples.get(1).get("test_dt").equals("2014-01-01T01:00:00Z"));
assertTrue(tuples.get(1).getLong("count(*)").equals(50L));
assertTrue(tuples.get(1).getDouble("sum(price_f)").equals(25000D));
assertTrue(tuples.get(1).getDouble("max(price_f)").equals(500D));
assertTrue(tuples.get(1).getDouble("min(price_f)").equals(500D));
assertTrue(tuples.get(1).getDouble("avg(price_f)").equals(500D));
assertTrue(tuples.get(1).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(1).getDouble("per(price_f,50)").equals(500D));
assertTrue(tuples.get(2).get("test_dt").equals("2015-01-01T01:00:00Z"));
assertTrue(tuples.get(2).getLong("count(*)").equals(50L));
assertTrue(tuples.get(2).getDouble("sum(price_f)").equals(15000D));
assertTrue(tuples.get(2).getDouble("max(price_f)").equals(300D));
assertTrue(tuples.get(2).getDouble("min(price_f)").equals(300D));
assertTrue(tuples.get(2).getDouble("avg(price_f)").equals(300D));
assertTrue(tuples.get(2).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(2).getDouble("per(price_f,50)").equals(300D));
assertTrue(tuples.get(3).get("test_dt").equals("2016-01-01T01:00:00Z"));
assertTrue(tuples.get(3).getLong("count(*)").equals(50L));
assertTrue(tuples.get(3).getDouble("sum(price_f)").equals(20000D));
assertTrue(tuples.get(3).getDouble("max(price_f)").equals(400D));
assertTrue(tuples.get(3).getDouble("min(price_f)").equals(400D));
assertTrue(tuples.get(3).getDouble("avg(price_f)").equals(400D));
assertTrue(tuples.get(3).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(3).getDouble("per(price_f,50)").equals(400D));
assertTrue(tuples.get(4).get("test_dt").equals("2017-01-01T01:00:00Z"));
assertEquals((long)tuples.get(4).getLong("count(*)"), 0L);
assertEquals(tuples.get(4).getDouble("sum(price_f)"), 0D, 0);
assertEquals(tuples.get(4).getDouble("max(price_f)"),0D, 0);
assertEquals(tuples.get(4).getDouble("min(price_f)"), 0D, 0);
assertTrue(tuples.get(4).getDouble("avg(price_f)").equals(0D));
assertTrue(tuples.get(4).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(4).getDouble("per(price_f,50)").equals(0D));
expr = "timeseries("+COLLECTIONORALIAS+", q=\"*:*\", start=\"2013-01-01T01:00:00.000Z\", " +
"end=\"2016-12-01T01:00:00.000Z\", " +
"gap=\"+1YEAR\", " +
"field=\"test_dt\", " +
"format=\"yyyy\", " +
"count(*), sum(price_f), max(price_f), min(price_f))";
"count(*), sum(price_f), max(price_f), min(price_f), avg(price_f), std(price_f), per(price_f, 50))";
paramsLoc = new ModifiableSolrParams();
paramsLoc.set("expr", expr);
paramsLoc.set("qt", "/stream");
@ -2512,31 +2728,45 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(tuples.get(0).getDouble("sum(price_f)").equals(10000D));
assertTrue(tuples.get(0).getDouble("max(price_f)").equals(100D));
assertTrue(tuples.get(0).getDouble("min(price_f)").equals(100D));
assertTrue(tuples.get(0).getDouble("avg(price_f)").equals(100D));
assertTrue(tuples.get(0).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(0).getDouble("per(price_f,50)").equals(100D));
assertTrue(tuples.get(1).get("test_dt").equals("2014"));
assertTrue(tuples.get(1).getLong("count(*)").equals(50L));
assertTrue(tuples.get(1).getDouble("sum(price_f)").equals(25000D));
assertTrue(tuples.get(1).getDouble("max(price_f)").equals(500D));
assertTrue(tuples.get(1).getDouble("min(price_f)").equals(500D));
assertTrue(tuples.get(1).getDouble("avg(price_f)").equals(500D));
assertTrue(tuples.get(1).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(1).getDouble("per(price_f,50)").equals(500D));
assertTrue(tuples.get(2).get("test_dt").equals("2015"));
assertTrue(tuples.get(2).getLong("count(*)").equals(50L));
assertTrue(tuples.get(2).getDouble("sum(price_f)").equals(15000D));
assertTrue(tuples.get(2).getDouble("max(price_f)").equals(300D));
assertTrue(tuples.get(2).getDouble("min(price_f)").equals(300D));
assertTrue(tuples.get(2).getDouble("avg(price_f)").equals(300D));
assertTrue(tuples.get(2).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(2).getDouble("per(price_f,50)").equals(300D));
assertTrue(tuples.get(3).get("test_dt").equals("2016"));
assertTrue(tuples.get(3).getLong("count(*)").equals(50L));
assertTrue(tuples.get(3).getDouble("sum(price_f)").equals(20000D));
assertTrue(tuples.get(3).getDouble("max(price_f)").equals(400D));
assertTrue(tuples.get(3).getDouble("min(price_f)").equals(400D));
assertTrue(tuples.get(3).getDouble("avg(price_f)").equals(400D));
assertTrue(tuples.get(3).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(3).getDouble("per(price_f,50)").equals(400D));
expr = "timeseries("+COLLECTIONORALIAS+", q=\"*:*\", start=\"2013-01-01T01:00:00.000Z\", " +
"end=\"2016-12-01T01:00:00.000Z\", " +
"gap=\"+1YEAR\", " +
"field=\"test_dt\", " +
"format=\"yyyy-MM\", " +
"count(*), sum(price_f), max(price_f), min(price_f))";
"count(*), sum(price_f), max(price_f), min(price_f), avg(price_f), std(price_f), per(price_f, 50))";
paramsLoc = new ModifiableSolrParams();
paramsLoc.set("expr", expr);
paramsLoc.set("qt", "/stream");
@ -2552,24 +2782,36 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(tuples.get(0).getDouble("sum(price_f)").equals(10000D));
assertTrue(tuples.get(0).getDouble("max(price_f)").equals(100D));
assertTrue(tuples.get(0).getDouble("min(price_f)").equals(100D));
assertTrue(tuples.get(0).getDouble("avg(price_f)").equals(100D));
assertTrue(tuples.get(0).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(0).getDouble("per(price_f,50)").equals(100D));
assertTrue(tuples.get(1).get("test_dt").equals("2014-01"));
assertTrue(tuples.get(1).getLong("count(*)").equals(50L));
assertTrue(tuples.get(1).getDouble("sum(price_f)").equals(25000D));
assertTrue(tuples.get(1).getDouble("max(price_f)").equals(500D));
assertTrue(tuples.get(1).getDouble("min(price_f)").equals(500D));
assertTrue(tuples.get(1).getDouble("avg(price_f)").equals(500D));
assertTrue(tuples.get(1).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(1).getDouble("per(price_f,50)").equals(500D));
assertTrue(tuples.get(2).get("test_dt").equals("2015-01"));
assertTrue(tuples.get(2).getLong("count(*)").equals(50L));
assertTrue(tuples.get(2).getDouble("sum(price_f)").equals(15000D));
assertTrue(tuples.get(2).getDouble("max(price_f)").equals(300D));
assertTrue(tuples.get(2).getDouble("min(price_f)").equals(300D));
assertTrue(tuples.get(2).getDouble("avg(price_f)").equals(300D));
assertTrue(tuples.get(2).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(2).getDouble("per(price_f,50)").equals(300D));
assertTrue(tuples.get(3).get("test_dt").equals("2016-01"));
assertTrue(tuples.get(3).getLong("count(*)").equals(50L));
assertTrue(tuples.get(3).getDouble("sum(price_f)").equals(20000D));
assertTrue(tuples.get(3).getDouble("max(price_f)").equals(400D));
assertTrue(tuples.get(3).getDouble("min(price_f)").equals(400D));
assertTrue(tuples.get(3).getDouble("avg(price_f)").equals(400D));
assertTrue(tuples.get(3).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(3).getDouble("per(price_f,50)").equals(400D));
expr = "timeseries("+COLLECTIONORALIAS+", q=\"*:*\", start=\"2012-01-01T01:00:00.000Z\", " +
@ -2577,7 +2819,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
"gap=\"+1YEAR\", " +
"field=\"test_dt\", " +
"format=\"yyyy-MM\", " +
"count(*), sum(price_f), max(price_f), min(price_f))";
"count(*), sum(price_f), max(price_f), min(price_f), avg(price_f), std(price_f), per(price_f, 50))";
paramsLoc = new ModifiableSolrParams();
paramsLoc.set("expr", expr);
paramsLoc.set("qt", "/stream");
@ -2592,30 +2834,45 @@ public class StreamExpressionTest extends SolrCloudTestCase {
assertTrue(tuples.get(0).getDouble("sum(price_f)") == 0);
assertTrue(tuples.get(0).getDouble("max(price_f)") == 0);
assertTrue(tuples.get(0).getDouble("min(price_f)") == 0);
assertTrue(tuples.get(0).getDouble("avg(price_f)").equals(0D));
assertTrue(tuples.get(0).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(0).getDouble("per(price_f,50)").equals(0D));
assertTrue(tuples.get(1).get("test_dt").equals("2013-01"));
assertTrue(tuples.get(1).getLong("count(*)").equals(100L));
assertTrue(tuples.get(1).getDouble("sum(price_f)").equals(10000D));
assertTrue(tuples.get(1).getDouble("max(price_f)").equals(100D));
assertTrue(tuples.get(1).getDouble("min(price_f)").equals(100D));
assertTrue(tuples.get(1).getDouble("avg(price_f)").equals(100D));
assertTrue(tuples.get(1).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(1).getDouble("per(price_f,50)").equals(100D));
assertTrue(tuples.get(2).get("test_dt").equals("2014-01"));
assertTrue(tuples.get(2).getLong("count(*)").equals(50L));
assertTrue(tuples.get(2).getDouble("sum(price_f)").equals(25000D));
assertTrue(tuples.get(2).getDouble("max(price_f)").equals(500D));
assertTrue(tuples.get(2).getDouble("min(price_f)").equals(500D));
assertTrue(tuples.get(2).getDouble("avg(price_f)").equals(500D));
assertTrue(tuples.get(2).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(2).getDouble("per(price_f,50)").equals(500D));
assertTrue(tuples.get(3).get("test_dt").equals("2015-01"));
assertTrue(tuples.get(3).getLong("count(*)").equals(50L));
assertTrue(tuples.get(3).getDouble("sum(price_f)").equals(15000D));
assertTrue(tuples.get(3).getDouble("max(price_f)").equals(300D));
assertTrue(tuples.get(3).getDouble("min(price_f)").equals(300D));
assertTrue(tuples.get(3).getDouble("avg(price_f)").equals(300D));
assertTrue(tuples.get(3).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(3).getDouble("per(price_f,50)").equals(300D));
assertTrue(tuples.get(4).get("test_dt").equals("2016-01"));
assertTrue(tuples.get(4).getLong("count(*)").equals(50L));
assertTrue(tuples.get(4).getDouble("sum(price_f)").equals(20000D));
assertTrue(tuples.get(4).getDouble("max(price_f)").equals(400D));
assertTrue(tuples.get(4).getDouble("min(price_f)").equals(400D));
assertTrue(tuples.get(4).getDouble("avg(price_f)").equals(400D));
assertTrue(tuples.get(4).getDouble("std(price_f)").equals(0D));
assertTrue(tuples.get(4).getDouble("per(price_f,50)").equals(400D));
}
@Test