You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cp...@apache.org on 2017/04/26 17:48:25 UTC
[11/14] lucene-solr:jira/solr-8668: SOLR-10566: Add timeseries
Streaming Expression
SOLR-10566: Add timeseries Streaming Expression
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/0e963f7a
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/0e963f7a
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/0e963f7a
Branch: refs/heads/jira/solr-8668
Commit: 0e963f7a8aeac0b8a831cd44fd48cd0c6bda11d2
Parents: 25f1dd2
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Apr 26 10:57:52 2017 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Wed Apr 26 11:17:43 2017 -0400
----------------------------------------------------------------------
.../org/apache/solr/handler/StreamHandler.java | 1 +
.../solrj/io/stream/TimeSeriesStream.java | 389 +++++++++++++++++++
.../solrj/io/stream/StreamExpressionTest.java | 76 +++-
3 files changed, 465 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0e963f7a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index a1f7993..c750ce9 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -165,6 +165,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
.withFunctionName("list", ListStream.class)
.withFunctionName("let", LetStream.class)
.withFunctionName("get", GetStream.class)
+ .withFunctionName("timeseries", TimeSeriesStream.class)
// metrics
.withFunctionName("min", MinMetric.class)
.withFunctionName("max", MaxMetric.class)
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0e963f7a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TimeSeriesStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TimeSeriesStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TimeSeriesStream.java
new file mode 100644
index 0000000..f548a46
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TimeSeriesStream.java
@@ -0,0 +1,389 @@
+/*
+ * 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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.CloudSolrClient.Builder;
+import org.apache.solr.client.solrj.io.SolrClientCache;
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.comp.ComparatorOrder;
+import org.apache.solr.client.solrj.io.comp.FieldComparator;
+import org.apache.solr.client.solrj.io.comp.MultipleFieldComparator;
+import org.apache.solr.client.solrj.io.comp.StreamComparator;
+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.StreamExpressionValue;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.apache.solr.client.solrj.io.stream.metrics.Bucket;
+import org.apache.solr.client.solrj.io.stream.metrics.Metric;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+
+/**
+ * The FacetStream abstracts the output from the JSON facet API as a Stream of Tuples. This provides an alternative to the
+ * RollupStream which uses Map/Reduce to perform aggregations.
+ **/
+
+public class TimeSeriesStream extends TupleStream implements Expressible {
+
+ private static final long serialVersionUID = 1;
+
+ private String start;
+ private String end;
+ private String gap;
+ private String field;
+
+ private Metric[] metrics;
+ private List<Tuple> tuples = new ArrayList();
+ private int index;
+ private String zkHost;
+ private SolrParams params;
+ private String collection;
+ protected transient SolrClientCache cache;
+ protected transient CloudSolrClient cloudSolrClient;
+
+ public TimeSeriesStream(String zkHost,
+ String collection,
+ SolrParams params,
+ Metric[] metrics,
+ String field,
+ String start,
+ String end,
+ String gap) throws IOException {
+ init(collection, params, field, metrics, start, end, gap, zkHost);
+ }
+
+ public TimeSeriesStream(StreamExpression expression, StreamFactory factory) throws IOException{
+ // grab all parameters out
+ String collectionName = factory.getValueOperand(expression, 0);
+ List<StreamExpressionNamedParameter> namedParams = factory.getNamedOperands(expression);
+ StreamExpressionNamedParameter startExpression = factory.getNamedOperand(expression, "start");
+ StreamExpressionNamedParameter endExpression = factory.getNamedOperand(expression, "end");
+ StreamExpressionNamedParameter fieldExpression = factory.getNamedOperand(expression, "field");
+ StreamExpressionNamedParameter gapExpression = factory.getNamedOperand(expression, "gap");
+ StreamExpressionNamedParameter zkHostExpression = factory.getNamedOperand(expression, "zkHost");
+ List<StreamExpression> metricExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, Metric.class);
+
+ String start = null;
+ if(startExpression != null) {
+ start = ((StreamExpressionValue)startExpression.getParameter()).getValue();
+ }
+
+ String end = null;
+ if(startExpression != null) {
+ end = ((StreamExpressionValue)endExpression.getParameter()).getValue();
+ }
+
+ String gap = null;
+ if(startExpression != null) {
+ gap = ((StreamExpressionValue)gapExpression.getParameter()).getValue();
+ }
+
+ String field = null;
+ if(startExpression != null) {
+ field = ((StreamExpressionValue)fieldExpression.getParameter()).getValue();
+ }
+
+ // Collection Name
+ if(null == collectionName){
+ throw new IOException(String.format(Locale.ROOT,"invalid expression %s - collectionName expected as first operand",expression));
+ }
+
+ // Named parameters - passed directly to solr as solrparams
+ if(0 == namedParams.size()){
+ throw new IOException(String.format(Locale.ROOT,"invalid expression %s - at least one named parameter expected. eg. 'q=*:*'",expression));
+ }
+
+ // Construct the metrics
+ Metric[] metrics = new Metric[metricExpressions.size()];
+ for(int idx = 0; idx < metricExpressions.size(); ++idx){
+ metrics[idx] = factory.constructMetric(metricExpressions.get(idx));
+ }
+
+ if(0 == metrics.length){
+ throw new IOException(String.format(Locale.ROOT,"invalid expression %s - at least one metric expected.",expression,collectionName));
+ }
+
+ // pull out known named params
+ ModifiableSolrParams params = new ModifiableSolrParams();
+ for(StreamExpressionNamedParameter namedParam : namedParams){
+ if(!namedParam.getName().equals("zkHost") && !namedParam.getName().equals("start") && !namedParam.getName().equals("end") && !namedParam.getName().equals("gap")){
+ params.add(namedParam.getName(), namedParam.getParameter().toString().trim());
+ }
+ }
+
+ // zkHost, optional - if not provided then will look into factory list to get
+ String zkHost = null;
+ if(null == zkHostExpression){
+ zkHost = factory.getCollectionZkHost(collectionName);
+ if(zkHost == null) {
+ zkHost = factory.getDefaultZkHost();
+ }
+ }
+ 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));
+ }
+
+ // We've got all the required items
+ init(collectionName, params, field, metrics, start, end, gap , zkHost);
+ }
+
+ public String getCollection() {
+ return this.collection;
+ }
+
+ private void init(String collection,
+ SolrParams params,
+ String field,
+ Metric[] metrics,
+ String start,
+ String end,
+ String gap,
+ String zkHost) throws IOException {
+ this.zkHost = zkHost;
+ this.collection = collection;
+ this.start = start;
+ this.gap = gap;
+ this.metrics = metrics;
+ this.field = field;
+ this.params = params;
+ this.end = end;
+ }
+
+ @Override
+ public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+ // function name
+ StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+ // collection
+ expression.addParameter(collection);
+
+ // 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));
+ }
+
+ expression.addParameter(new StreamExpressionNamedParameter("start", start));
+ expression.addParameter(new StreamExpressionNamedParameter("end", end));
+ expression.addParameter(new StreamExpressionNamedParameter("gap", gap));
+ expression.addParameter(new StreamExpressionNamedParameter("field", gap));
+
+ // zkHost
+ expression.addParameter(new StreamExpressionNamedParameter("zkHost", zkHost));
+
+ return expression;
+ }
+
+ @Override
+ public Explanation toExplanation(StreamFactory factory) throws IOException {
+
+ StreamExplanation explanation = new StreamExplanation(getStreamNodeId().toString());
+
+ explanation.setFunctionName(factory.getFunctionName(this.getClass()));
+ explanation.setImplementingClass(this.getClass().getName());
+ 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 (%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 tmpParams = new ModifiableSolrParams(SolrParams.toMultiMap(params.toNamedList()));
+
+ child.setExpression(tmpParams.getMap().entrySet().stream().map(e -> String.format(Locale.ROOT, "%s=%s", e.getKey(), e.getValue())).collect(Collectors.joining(",")));
+
+ explanation.addChild(child);
+
+ return explanation;
+ }
+
+ public void setStreamContext(StreamContext context) {
+ cache = context.getSolrClientCache();
+ }
+
+ public List<TupleStream> children() {
+ return new ArrayList();
+ }
+
+ public void open() throws IOException {
+ if(cache != null) {
+ cloudSolrClient = cache.getCloudSolrClient(zkHost);
+ } else {
+ cloudSolrClient = new Builder()
+ .withZkHost(zkHost)
+ .build();
+ }
+
+ String json = getJsonFacetString(field, metrics, start, end, gap);
+
+ ModifiableSolrParams paramsLoc = new ModifiableSolrParams(params);
+ paramsLoc.set("json.facet", json);
+ paramsLoc.set("rows", "0");
+
+ QueryRequest request = new QueryRequest(paramsLoc);
+ try {
+ NamedList response = cloudSolrClient.request(request, collection);
+ getTuples(response, field, metrics);
+ } catch (Exception e) {
+ throw new IOException(e);
+ }
+ }
+
+ public void close() throws IOException {
+ if(cache == null) {
+ cloudSolrClient.close();
+ }
+ }
+
+ public Tuple read() throws IOException {
+ if(index < tuples.size()) {
+ Tuple tuple = tuples.get(index);
+ ++index;
+ return tuple;
+ } else {
+ Map fields = new HashMap();
+ fields.put("EOF", true);
+ Tuple tuple = new Tuple(fields);
+ return tuple;
+ }
+ }
+
+ private String getJsonFacetString(String field, Metric[] _metrics, String start, String end, String gap) {
+ StringBuilder buf = new StringBuilder();
+ appendJson(buf, _metrics, field, start, end, gap);
+ return "{"+buf.toString()+"}";
+ }
+
+
+ private void appendJson(StringBuilder buf,
+ Metric[] _metrics,
+ String field,
+ String start,
+ String end,
+ String gap) {
+ buf.append('"');
+ buf.append("timeseries");
+ buf.append('"');
+ buf.append(":{");
+ buf.append("\"type\":\"range\"");
+ buf.append(",\"field\":\""+field+"\"");
+ buf.append(",\"start\":\""+start+"\"");
+ buf.append(",\"end\":\""+end+"\"");
+ buf.append(",\"gap\":\""+gap+"\"");
+
+ buf.append(",\"facet\":{");
+ int metricCount = 0;
+ for(Metric metric : _metrics) {
+ String identifier = metric.getIdentifier();
+ if(!identifier.startsWith("count(")) {
+ if(metricCount>0) {
+ buf.append(",");
+ }
+ buf.append("\"facet_" + metricCount + "\":\"" +identifier+"\"");
+ ++metricCount;
+ }
+ }
+ buf.append("}}");
+ }
+
+ private void getTuples(NamedList response,
+ String field,
+ Metric[] metrics) {
+
+ Tuple tuple = new Tuple(new HashMap());
+ NamedList facets = (NamedList)response.get("facets");
+ fillTuples(tuples, tuple, facets, field, metrics);
+ }
+
+ private void fillTuples(List<Tuple> tuples,
+ Tuple currentTuple,
+ NamedList facets,
+ String field,
+ Metric[] _metrics) {
+
+ NamedList nl = (NamedList)facets.get("timeseries");
+ if(nl == null) {
+ return;
+ }
+
+ List allBuckets = (List)nl.get("buckets");
+ for(int b=0; b<allBuckets.size(); b++) {
+ NamedList bucket = (NamedList)allBuckets.get(b);
+ Object val = bucket.get("val");
+ Tuple t = currentTuple.clone();
+ t.put(field, val);
+ int m = 0;
+ for(Metric metric : _metrics) {
+ String identifier = metric.getIdentifier();
+ if(!identifier.startsWith("count(")) {
+ double d = (double)bucket.get("facet_"+m);
+ if(metric.outputLong) {
+ t.put(identifier, Math.round(d));
+ } else {
+ t.put(identifier, d);
+ }
+ ++m;
+ } else {
+ long l = ((Number)bucket.get("count")).longValue();
+ t.put("count(*)", l);
+ }
+ }
+ tuples.add(t);
+ }
+ }
+
+ public int getCost() {
+ return 0;
+ }
+
+ @Override
+ public StreamComparator getStreamSort() {
+ return null;
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0e963f7a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index 51c5301..c459779 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -3850,7 +3850,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
false, true, TIMEOUT);
new UpdateRequest()
- .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0", "s_multi", "aaaa", "s_multi", "bbbb", "i_multi", "4", "i_multi", "7")
+ .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0", "s_multi", "aaaa", "s_multi", "bbbb", "i_multi", "4", "i_multi", "7")
.add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0", "s_multi", "aaaa1", "s_multi", "bbbb1", "i_multi", "44", "i_multi", "77")
.add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "s_multi", "aaaa2", "s_multi", "bbbb2", "i_multi", "444", "i_multi", "777")
.add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "s_multi", "aaaa3", "s_multi", "bbbb3", "i_multi", "4444", "i_multi", "7777")
@@ -5037,6 +5037,80 @@ public class StreamExpressionTest extends SolrCloudTestCase {
}
+ private String getDateString(String year, String month, String day) {
+ return year+"-"+month+"-"+day+"T00:00:00Z";
+
+ }
+
+ @Test
+ public void testTimeSeriesStream() throws Exception {
+ UpdateRequest updateRequest = new UpdateRequest();
+
+ int i=0;
+ while(i<50) {
+ updateRequest.add(id, "id_"+(++i),"test_dt", getDateString("2016", "5", "1"), "price_f", "400.00");
+ }
+
+ while(i<100) {
+ updateRequest.add(id, "id_"+(++i),"test_dt", getDateString("2015", "5", "1"), "price_f", "300.0");
+ }
+
+ while(i<150) {
+ updateRequest.add(id, "id_"+(++i),"test_dt", getDateString("2014", "5", "1"), "price_f", "500.0");
+ }
+
+ while(i<250) {
+ updateRequest.add(id, "id_"+(++i),"test_dt", getDateString("2013", "5", "1"), "price_f", "100.00");
+ }
+
+ updateRequest.commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+ String expr = "timeseries("+COLLECTIONORALIAS+", q=\"*:*\", start=\"2013-01-01T01:00:00.000Z\", " +
+ "end=\"2016-12-01T01:00:00.000Z\", " +
+ "gap=\"+1YEAR\", " +
+ "field=\"test_dt\", " +
+ "count(*), sum(price_f), max(price_f), min(price_f))";
+ ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
+ paramsLoc.set("expr", expr);
+ paramsLoc.set("qt", "/stream");
+
+ String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString()+"/"+COLLECTIONORALIAS;
+ TupleStream solrStream = new SolrStream(url, paramsLoc);
+
+ StreamContext context = new StreamContext();
+ solrStream.setStreamContext(context);
+ List<Tuple> tuples = getTuples(solrStream);
+ assertTrue(tuples.size() == 4);
+
+ assertTrue(tuples.get(0).get("test_dt").equals("2013-01-01T01:00:00Z"));
+ assertTrue(tuples.get(0).getLong("count(*)").equals(100L));
+ 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(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(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(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));
+
+ }
+
+
+
+
@Test
public void testListStream() throws Exception {
UpdateRequest updateRequest = new UpdateRequest();