You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jb...@apache.org on 2017/01/03 22:30:40 UTC

[1/5] lucene-solr:branch_6x: SOLR-8530: Add HavingStream to Streaming API and StreamingExpressions

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 6caefcda8 -> 0ab1ddb31


SOLR-8530: Add HavingStream to Streaming API and StreamingExpressions


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/2f7d6fc0
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/2f7d6fc0
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/2f7d6fc0

Branch: refs/heads/branch_6x
Commit: 2f7d6fc0fa3de7e2f1d09823d9ef4c6ee08e9d44
Parents: 6caefcd
Author: Joel Bernstein <jb...@apache.org>
Authored: Tue Jan 3 13:09:49 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Tue Jan 3 17:12:44 2017 -0500

----------------------------------------------------------------------
 .../solr/client/solrj/io/ops/AndOperation.java  | 101 ++++++++++
 .../client/solrj/io/ops/BooleanOperation.java   |  24 +++
 .../client/solrj/io/ops/EqualsOperation.java    |  70 +++++++
 .../io/ops/GreaterThanEqualToOperation.java     |  70 +++++++
 .../solrj/io/ops/GreaterThanOperation.java      |  70 +++++++
 .../solr/client/solrj/io/ops/LeafOperation.java |  59 ++++++
 .../solrj/io/ops/LessThanEqualToOperation.java  |  70 +++++++
 .../client/solrj/io/ops/LessThanOperation.java  |  70 +++++++
 .../solr/client/solrj/io/ops/NotOperation.java  |  87 +++++++++
 .../solr/client/solrj/io/ops/OrOperation.java   |  71 +++++++
 .../client/solrj/io/stream/HavingStream.java    | 190 +++++++++++++++++++
 11 files changed, 882 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2f7d6fc0/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/AndOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/AndOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/AndOperation.java
new file mode 100644
index 0000000..f095f63
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/AndOperation.java
@@ -0,0 +1,101 @@
+/*
+ * 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.ops;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.UUID;
+
+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.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+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 AndOperation implements BooleanOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  protected BooleanOperation leftOperand;
+  protected BooleanOperation rightOperand;
+
+  public void operate(Tuple tuple) {
+    leftOperand.operate(tuple);
+    rightOperand.operate(tuple);
+  }
+
+  public AndOperation(BooleanOperation leftOperand, BooleanOperation rightOperand) {
+    this.leftOperand = leftOperand;
+    this.rightOperand = rightOperand;
+  }
+
+  public AndOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+      List<StreamExpression> operationExpressions = factory.getExpressionOperandsRepresentingTypes(expression, BooleanOperation.class);
+      if(operationExpressions != null && operationExpressions.size() == 2) {
+        StreamExpression left = operationExpressions.get(0);
+        StreamOperation leftOp = factory.constructOperation(left);
+        if(leftOp instanceof BooleanOperation) {
+          leftOperand = (BooleanOperation) leftOp;
+        } else {
+          throw new IOException("The And/Or Operation requires a BooleanOperation.");
+        }
+
+        StreamExpression right = operationExpressions.get(1);
+        StreamOperation rightOp = factory.constructOperation(right);
+        if(rightOp instanceof BooleanOperation) {
+          rightOperand = (BooleanOperation) rightOp;
+        } else {
+          throw new IOException("The And/Or Operation requires a BooleanOperation.");
+        }
+      } else {
+        throw new IOException("The And/Or Operation requires a BooleanOperations.");
+      }
+  }
+
+  public boolean evaluate() {
+    return leftOperand.evaluate() && rightOperand.evaluate();
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    if(leftOperand instanceof Expressible) {
+      expression.addParameter(leftOperand.toExpression(factory));
+    } else {
+      throw new IOException("This left operand of the AndOperation contains a non-expressible operation - it cannot be converted to an expression");
+    }
+
+    if(rightOperand instanceof Expressible) {
+      expression.addParameter(rightOperand.toExpression(factory));
+    } else {
+      throw new IOException("This the right operand of the AndOperation contains a non-expressible operation - it cannot be converted to an expression");
+    }
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2f7d6fc0/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
new file mode 100644
index 0000000..609e4e1
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
@@ -0,0 +1,24 @@
+/*
+ * 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.ops;
+
+import org.apache.solr.client.solrj.io.Tuple;
+
+
+public interface BooleanOperation extends StreamOperation {
+  public abstract boolean evaluate();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2f7d6fc0/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java
new file mode 100644
index 0000000..1958551
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/EqualsOperation.java
@@ -0,0 +1,70 @@
+/*
+ * 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.ops;
+
+import java.io.IOException;
+import java.util.UUID;
+
+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.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class EqualsOperation extends LeafOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  public void operate(Tuple tuple) {
+    this.tuple = tuple;
+  }
+
+  public EqualsOperation(String field, double val) {
+    super(field, val);
+  }
+
+  public EqualsOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public boolean evaluate() {
+    Double d = tuple.getDouble(field);
+
+    if(d == null) {
+      return false;
+    }
+
+    return d.doubleValue() == val;
+  }
+
+  public StreamExpression toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    expression.addParameter(field);
+    expression.addParameter(Double.toString(val));
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2f7d6fc0/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java
new file mode 100644
index 0000000..87c8364
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanEqualToOperation.java
@@ -0,0 +1,70 @@
+/*
+ * 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.ops;
+
+import java.io.IOException;
+import java.util.UUID;
+
+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.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class GreaterThanEqualToOperation extends LeafOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  public void operate(Tuple tuple) {
+    this.tuple = tuple;
+  }
+
+  public GreaterThanEqualToOperation(String field, double val) {
+    super(field, val);
+  }
+
+  public GreaterThanEqualToOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public boolean evaluate() {
+    Double d = tuple.getDouble(field);
+
+    if(d == null) {
+      return false;
+    }
+
+    return d.doubleValue() >= val;
+  }
+
+  public StreamExpression toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    expression.addParameter(field);
+    expression.addParameter(Double.toString(val));
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2f7d6fc0/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java
new file mode 100644
index 0000000..664438a
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GreaterThanOperation.java
@@ -0,0 +1,70 @@
+/*
+ * 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.ops;
+
+import java.io.IOException;
+import java.util.UUID;
+
+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.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class GreaterThanOperation extends LeafOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  public void operate(Tuple tuple) {
+    this.tuple = tuple;
+  }
+
+  public GreaterThanOperation(String field, double val) {
+    super(field, val);
+  }
+
+  public GreaterThanOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public boolean evaluate() {
+    Double d = tuple.getDouble(field);
+
+    if(d == null) {
+      return false;
+    }
+
+    return d.doubleValue() > val;
+  }
+
+  public StreamExpression toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    expression.addParameter(field);
+    expression.addParameter(Double.toString(val));
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2f7d6fc0/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
new file mode 100644
index 0000000..b6ad897
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LeafOperation.java
@@ -0,0 +1,59 @@
+/*
+ * 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.ops;
+
+import java.io.IOException;
+import java.util.UUID;
+
+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.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public abstract class LeafOperation implements BooleanOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  protected String field;
+  protected Double val;
+  protected Tuple tuple;
+
+  public void operate(Tuple tuple) {
+    this.tuple = tuple;
+  }
+
+  public LeafOperation(String field, double val) {
+    this.field = field;
+    this.val = val;
+  }
+
+  public LeafOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    this.field = factory.getValueOperand(expression, 0);
+    this.val = Double.parseDouble(factory.getValueOperand(expression, 1));
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2f7d6fc0/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java
new file mode 100644
index 0000000..2da3274
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanEqualToOperation.java
@@ -0,0 +1,70 @@
+/*
+ * 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.ops;
+
+import java.io.IOException;
+import java.util.UUID;
+
+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.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class LessThanEqualToOperation extends LeafOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  public void operate(Tuple tuple) {
+    this.tuple = tuple;
+  }
+
+  public LessThanEqualToOperation(String field, double val) {
+    super(field, val);
+  }
+
+  public LessThanEqualToOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public boolean evaluate() {
+    Double d = tuple.getDouble(field);
+
+    if(d == null) {
+      return true;
+    }
+
+    return d.doubleValue() <= val;
+  }
+
+  public StreamExpression toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    expression.addParameter(field);
+    expression.addParameter(Double.toString(val));
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2f7d6fc0/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java
new file mode 100644
index 0000000..c1cec95
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/LessThanOperation.java
@@ -0,0 +1,70 @@
+/*
+ * 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.ops;
+
+import java.io.IOException;
+import java.util.UUID;
+
+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.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class LessThanOperation extends LeafOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  public void operate(Tuple tuple) {
+    this.tuple = tuple;
+  }
+
+  public LessThanOperation(String field, double val) {
+    super(field, val);
+  }
+
+  public LessThanOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public boolean evaluate() {
+    Double d = tuple.getDouble(field);
+
+    if(d == null) {
+      return true;
+    }
+    
+    return d.doubleValue() < val;
+  }
+
+  public StreamExpression toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    expression.addParameter(field);
+    expression.addParameter(Double.toString(val));
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2f7d6fc0/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/NotOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/NotOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/NotOperation.java
new file mode 100644
index 0000000..0e40b72
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/NotOperation.java
@@ -0,0 +1,87 @@
+/*
+ * 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.ops;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.UUID;
+
+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.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+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 NotOperation implements BooleanOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  protected BooleanOperation operand;
+
+  public void operate(Tuple tuple) {
+    operand.operate(tuple);
+  }
+
+  public NotOperation(BooleanOperation operand) {
+    this.operand = operand;
+  }
+
+  public NotOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    List<StreamExpression> operationExpressions = factory.getExpressionOperandsRepresentingTypes(expression, BooleanOperation.class);
+    if(operationExpressions != null && operationExpressions.size() == 1) {
+      StreamExpression op = operationExpressions.get(0);
+      StreamOperation streamOp = factory.constructOperation(op);
+      if(streamOp instanceof BooleanOperation) {
+        operand = (BooleanOperation) streamOp;
+      } else {
+        throw new IOException("The NotOperation requires a BooleanOperation.");
+      }
+
+    } else {
+      throw new IOException("The NotOperation requires a BooleanOperations.");
+    }
+  }
+
+  public boolean evaluate() {
+    return !operand.evaluate();
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    if(operand instanceof Expressible) {
+      expression.addParameter(operand.toExpression(factory));
+    } else {
+      throw new IOException("The operand of the NotOperation contains a non-expressible operation - it cannot be converted to an expression");
+    }
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2f7d6fc0/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/OrOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/OrOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/OrOperation.java
new file mode 100644
index 0000000..faac5cd
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/OrOperation.java
@@ -0,0 +1,71 @@
+/*
+ * 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.ops;
+
+import java.io.IOException;
+import java.util.UUID;
+
+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.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class OrOperation extends AndOperation {
+
+  private static final long serialVersionUID = 1;
+  private UUID operationNodeId = UUID.randomUUID();
+
+  public OrOperation(BooleanOperation leftOperand, BooleanOperation rightOperand) {
+    super(leftOperand, rightOperand);
+  }
+
+  public OrOperation(StreamExpression expression, StreamFactory factory) throws IOException {
+    super(expression, factory);
+  }
+
+  public boolean evaluate() {
+    return leftOperand.evaluate() || rightOperand.evaluate();
+  }
+
+  @Override
+  public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException {
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+    if(leftOperand instanceof Expressible) {
+      expression.addParameter(leftOperand.toExpression(factory));
+    } else {
+      throw new IOException("This left operand of the OrOperation contains a non-expressible operation - it cannot be converted to an expression");
+    }
+
+    if(rightOperand instanceof Expressible) {
+      expression.addParameter(rightOperand.toExpression(factory));
+    } else {
+      throw new IOException("This the right operand of the OrOperation contains a non-expressible operation - it cannot be converted to an expression");
+    }
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+    return new Explanation(operationNodeId.toString())
+        .withExpressionType(ExpressionType.OPERATION)
+        .withFunctionName(factory.getFunctionName(getClass()))
+        .withImplementingClass(getClass().getName())
+        .withExpression(toExpression(factory).toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/2f7d6fc0/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
new file mode 100644
index 0000000..36ca113
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
@@ -0,0 +1,190 @@
+/*
+ * 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.List;
+import java.util.Locale;
+
+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.MultipleFieldComparator;
+import org.apache.solr.client.solrj.io.comp.StreamComparator;
+import org.apache.solr.client.solrj.io.eq.FieldEqualitor;
+import org.apache.solr.client.solrj.io.eq.MultipleFieldEqualitor;
+import org.apache.solr.client.solrj.io.eq.StreamEqualitor;
+import org.apache.solr.client.solrj.io.ops.BooleanOperation;
+import org.apache.solr.client.solrj.io.ops.StreamOperation;
+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.StreamExpressionValue;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ *  Iterates over a TupleStream and buffers Tuples that are equal based on a comparator.
+ *  This allows tuples to be grouped by common field(s).
+ *
+ *  The read() method emits one tuple per group. The fields of the emitted Tuple reflect the first tuple
+ *  encountered in the group.
+ *
+ *  Use the Tuple.getMaps() method to return all the Tuples in the group. This method returns
+ *  a list of maps (including the group head), which hold the data for each Tuple in the group.
+ *
+ *  Note: The ReducerStream requires that it's underlying stream be sorted and partitioned by the same
+ *  fields as it's comparator.
+ *
+ **/
+
+public class HavingStream extends TupleStream implements Expressible {
+
+  private static final long serialVersionUID = 1;
+
+  private TupleStream stream;
+  private BooleanOperation op;
+
+  private transient Tuple currentGroupHead;
+
+  public HavingStream(TupleStream stream, BooleanOperation op) throws IOException {
+    init(stream, op);
+  }
+
+
+  public HavingStream(StreamExpression expression, StreamFactory factory) throws IOException{
+    // grab all parameters out
+    List<StreamExpression> streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class);
+    List<StreamExpression> operationExpressions = factory.getExpressionOperandsRepresentingTypes(expression, BooleanOperation.class);
+
+    // validate expression contains only what we want.
+    if(expression.getParameters().size() != streamExpressions.size() + 1){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - unknown operands found", expression));
+    }
+
+    if(1 != streamExpressions.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting a single stream but found %d",expression, streamExpressions.size()));
+    }
+
+
+    BooleanOperation booleanOperation = null;
+    if(operationExpressions != null && operationExpressions.size() == 1) {
+      StreamExpression ex = operationExpressions.get(0);
+      StreamOperation operation = factory.constructOperation(ex);
+      if(operation instanceof BooleanOperation) {
+        booleanOperation = (BooleanOperation) operation;
+      } else {
+        throw new IOException("The HavingStream requires a BooleanOperation. A StreamOperation was provided.");
+      }
+    } else {
+      throw new IOException("The HavingStream requires a BooleanOperation.");
+    }
+
+    init(factory.constructStream(streamExpressions.get(0)), booleanOperation);
+  }
+
+  private void init(TupleStream stream, BooleanOperation op) throws IOException{
+    this.stream = stream;
+    this.op = op;
+  }
+
+  @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()));
+
+    // stream
+    if(includeStreams){
+      expression.addParameter(((Expressible) stream).toExpression(factory));
+    }
+    else{
+      expression.addParameter("<stream>");
+    }
+
+    if(op instanceof Expressible) {
+      expression.addParameter(op.toExpression(factory));
+    } else {
+      throw new IOException("This ReducerStream contains a non-expressible operation - it cannot be converted to an expression");
+    }
+
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+
+    return 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())
+        .withHelpers(new Explanation[]{
+            op.toExplanation(factory)
+        });
+  }
+
+  public void setStreamContext(StreamContext context) {
+    this.stream.setStreamContext(context);
+  }
+
+  public List<TupleStream> children() {
+    List<TupleStream> l =  new ArrayList<TupleStream>();
+    l.add(stream);
+    return l;
+  }
+
+  public void open() throws IOException {
+    stream.open();
+  }
+
+  public void close() throws IOException {
+    stream.close();
+  }
+
+  public Tuple read() throws IOException {
+    while(true) {
+      Tuple tuple = stream.read();
+      if(tuple.EOF) {
+        return tuple;
+      }
+
+      op.operate(tuple);
+
+      if(op.evaluate()) {
+        return tuple;
+      }
+    }
+  }
+
+  /** Return the stream sort - ie, the order in which records are returned */
+  public StreamComparator getStreamSort(){
+    return stream.getStreamSort();
+  }
+
+  public int getCost() {
+    return 0;
+  }
+}
\ No newline at end of file


[3/5] lucene-solr:branch_6x: SOLR-8530: Fixed javadoc

Posted by jb...@apache.org.
SOLR-8530: Fixed javadoc


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/00af5fff
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/00af5fff
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/00af5fff

Branch: refs/heads/branch_6x
Commit: 00af5fff4d096000b0cde9066a599f68076c1862
Parents: 297b178
Author: Joel Bernstein <jb...@apache.org>
Authored: Tue Jan 3 14:52:30 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Tue Jan 3 17:13:35 2017 -0500

----------------------------------------------------------------------
 .../client/solrj/io/ops/BooleanOperation.java   |  6 ++++--
 .../client/solrj/io/stream/HavingStream.java    | 21 ++------------------
 2 files changed, 6 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00af5fff/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
index 609e4e1..d455999 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/BooleanOperation.java
@@ -16,8 +16,10 @@
  */
 package org.apache.solr.client.solrj.io.ops;
 
-import org.apache.solr.client.solrj.io.Tuple;
-
+/**
+ *  A BooleanOperation returns true or false for each tuple that it evaluates. The HavingStream applies a BooleanOperation to
+ *  determine which tuples to emit.
+ */
 
 public interface BooleanOperation extends StreamOperation {
   public abstract boolean evaluate();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00af5fff/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
index 36ca113..38c1a6b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/HavingStream.java
@@ -22,12 +22,7 @@ import java.util.List;
 import java.util.Locale;
 
 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.MultipleFieldComparator;
 import org.apache.solr.client.solrj.io.comp.StreamComparator;
-import org.apache.solr.client.solrj.io.eq.FieldEqualitor;
-import org.apache.solr.client.solrj.io.eq.MultipleFieldEqualitor;
-import org.apache.solr.client.solrj.io.eq.StreamEqualitor;
 import org.apache.solr.client.solrj.io.ops.BooleanOperation;
 import org.apache.solr.client.solrj.io.ops.StreamOperation;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation;
@@ -35,23 +30,11 @@ 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.StreamExpressionValue;
 import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 
 /**
- *  Iterates over a TupleStream and buffers Tuples that are equal based on a comparator.
- *  This allows tuples to be grouped by common field(s).
- *
- *  The read() method emits one tuple per group. The fields of the emitted Tuple reflect the first tuple
- *  encountered in the group.
- *
- *  Use the Tuple.getMaps() method to return all the Tuples in the group. This method returns
- *  a list of maps (including the group head), which hold the data for each Tuple in the group.
- *
- *  Note: The ReducerStream requires that it's underlying stream be sorted and partitioned by the same
- *  fields as it's comparator.
- *
+ * The HavingStream iterates over an internal stream and applies a BooleanOperation to each tuple. If the BooleanOperation
+ * evaluates to true then the HavingStream emits the tuple, if it returns false the tuple is not emitted.
  **/
 
 public class HavingStream extends TupleStream implements Expressible {


[2/5] lucene-solr:branch_6x: SOLR-8530: Add tests from the HavingStream

Posted by jb...@apache.org.
SOLR-8530: Add tests from the HavingStream


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/297b1789
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/297b1789
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/297b1789

Branch: refs/heads/branch_6x
Commit: 297b1789092f4f9b3a2cfb91da397e5034708486
Parents: 2f7d6fc
Author: Joel Bernstein <jb...@apache.org>
Authored: Fri Dec 16 11:26:55 2016 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Tue Jan 3 17:13:13 2017 -0500

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |  19 +-
 .../solrj/io/stream/StreamExpressionTest.java   | 201 +++++++++++++++++++
 2 files changed, 219 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/297b1789/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 639937d..dd79462 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -31,9 +31,17 @@ import org.apache.solr.client.solrj.io.Tuple;
 import org.apache.solr.client.solrj.io.comp.StreamComparator;
 import org.apache.solr.client.solrj.io.graph.GatherNodesStream;
 import org.apache.solr.client.solrj.io.graph.ShortestPathStream;
+import org.apache.solr.client.solrj.io.ops.AndOperation;
 import org.apache.solr.client.solrj.io.ops.ConcatOperation;
 import org.apache.solr.client.solrj.io.ops.DistinctOperation;
+import org.apache.solr.client.solrj.io.ops.EqualsOperation;
+import org.apache.solr.client.solrj.io.ops.GreaterThanEqualToOperation;
+import org.apache.solr.client.solrj.io.ops.GreaterThanOperation;
 import org.apache.solr.client.solrj.io.ops.GroupOperation;
+import org.apache.solr.client.solrj.io.ops.LessThanEqualToOperation;
+import org.apache.solr.client.solrj.io.ops.LessThanOperation;
+import org.apache.solr.client.solrj.io.ops.NotOperation;
+import org.apache.solr.client.solrj.io.ops.OrOperation;
 import org.apache.solr.client.solrj.io.ops.ReplaceOperation;
 import org.apache.solr.client.solrj.io.stream.*;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation;
@@ -155,7 +163,16 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
       
       // stream reduction operations
       .withFunctionName("group", GroupOperation.class)
-      .withFunctionName("distinct", DistinctOperation.class);
+      .withFunctionName("distinct", DistinctOperation.class)
+      .withFunctionName("having", HavingStream.class)
+      .withFunctionName("and", AndOperation.class)
+      .withFunctionName("or", OrOperation.class)
+      .withFunctionName("not", NotOperation.class)
+      .withFunctionName("gt", GreaterThanOperation.class)
+      .withFunctionName("lt", LessThanOperation.class)
+      .withFunctionName("eq", EqualsOperation.class)
+      .withFunctionName("lteq", LessThanEqualToOperation.class)
+      .withFunctionName("gteq", GreaterThanEqualToOperation.class);
 
      // This pulls all the overrides and additions from the config
      List<PluginInfo> pluginInfos = core.getSolrConfig().getPluginInfos(Expressible.class.getName());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/297b1789/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 1316af4..fd088f1 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
@@ -33,8 +33,16 @@ 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.ops.AndOperation;
 import org.apache.solr.client.solrj.io.ops.ConcatOperation;
+import org.apache.solr.client.solrj.io.ops.EqualsOperation;
+import org.apache.solr.client.solrj.io.ops.GreaterThanEqualToOperation;
+import org.apache.solr.client.solrj.io.ops.GreaterThanOperation;
 import org.apache.solr.client.solrj.io.ops.GroupOperation;
+import org.apache.solr.client.solrj.io.ops.LessThanEqualToOperation;
+import org.apache.solr.client.solrj.io.ops.LessThanOperation;
+import org.apache.solr.client.solrj.io.ops.NotOperation;
+import org.apache.solr.client.solrj.io.ops.OrOperation;
 import org.apache.solr.client.solrj.io.ops.ReplaceOperation;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParser;
@@ -803,6 +811,199 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
   }
 
+
+  @Test
+  public void testHavingStream() throws Exception {
+
+    SolrClientCache solrClientCache = new SolrClientCache();
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1", "subject", "blah blah blah 0")
+        .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2", "subject", "blah blah blah 2")
+        .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "subject", "blah blah blah 3")
+        .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "subject", "blah blah blah 4")
+        .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5", "subject", "blah blah blah 1")
+        .add(id, "5", "a_s", "hello3", "a_i", "5", "a_f", "6", "subject", "blah blah blah 5")
+        .add(id, "6", "a_s", "hello4", "a_i", "6", "a_f", "7", "subject", "blah blah blah 6")
+        .add(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "8", "subject", "blah blah blah 7")
+        .add(id, "8", "a_s", "hello3", "a_i", "8", "a_f", "9", "subject", "blah blah blah 8")
+        .add(id, "9", "a_s", "hello0", "a_i", "9", "a_f", "10", "subject", "blah blah blah 9")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
+        .withFunctionName("search", CloudSolrStream.class)
+        .withFunctionName("having", HavingStream.class)
+        .withFunctionName("and", AndOperation.class)
+        .withFunctionName("or", OrOperation.class)
+        .withFunctionName("not", NotOperation.class)
+        .withFunctionName("gt", GreaterThanOperation.class)
+        .withFunctionName("lt", LessThanOperation.class)
+        .withFunctionName("eq", EqualsOperation.class)
+        .withFunctionName("lteq", LessThanEqualToOperation.class)
+        .withFunctionName("gteq", GreaterThanEqualToOperation.class);
+
+    stream = factory.constructStream("having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), eq(a_i, 9))");
+    StreamContext context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 1);
+    Tuple t = tuples.get(0);
+    assertTrue(t.getString("id").equals("9"));
+
+    stream = factory.constructStream("having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), and(eq(a_i, 9),lt(a_i, 10)))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 1);
+    t = tuples.get(0);
+    assertTrue(t.getString("id").equals("9"));
+
+    stream = factory.constructStream("having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), or(eq(a_i, 9),eq(a_i, 8)))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 2);
+    t = tuples.get(0);
+    assertTrue(t.getString("id").equals("8"));
+
+    t = tuples.get(1);
+    assertTrue(t.getString("id").equals("9"));
+
+
+    stream = factory.constructStream("having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), and(eq(a_i, 9),not(eq(a_i, 9))))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 0);
+
+
+    stream = factory.constructStream("having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), and(lteq(a_i, 9), gteq(a_i, 8)))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    System.out.println("####Tuples:"+tuples.size());
+    assert(tuples.size() == 2);
+
+    t = tuples.get(0);
+    assertTrue(t.getString("id").equals("8"));
+
+    t = tuples.get(1);
+    assertTrue(t.getString("id").equals("9"));
+
+    solrClientCache.close();
+  }
+
+
+  @Test
+  public void testParallelHavingStream() throws Exception {
+
+    SolrClientCache solrClientCache = new SolrClientCache();
+
+    new UpdateRequest()
+        .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1", "subject", "blah blah blah 0")
+        .add(id, "2", "a_s", "hello0", "a_i", "2", "a_f", "2", "subject", "blah blah blah 2")
+        .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3", "subject", "blah blah blah 3")
+        .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4", "subject", "blah blah blah 4")
+        .add(id, "1", "a_s", "hello0", "a_i", "1", "a_f", "5", "subject", "blah blah blah 1")
+        .add(id, "5", "a_s", "hello3", "a_i", "5", "a_f", "6", "subject", "blah blah blah 5")
+        .add(id, "6", "a_s", "hello4", "a_i", "6", "a_f", "7", "subject", "blah blah blah 6")
+        .add(id, "7", "a_s", "hello3", "a_i", "7", "a_f", "8", "subject", "blah blah blah 7")
+        .add(id, "8", "a_s", "hello3", "a_i", "8", "a_f", "9", "subject", "blah blah blah 8")
+        .add(id, "9", "a_s", "hello0", "a_i", "9", "a_f", "10", "subject", "blah blah blah 9")
+        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
+
+    TupleStream stream;
+    List<Tuple> tuples;
+
+    StreamFactory factory = new StreamFactory()
+        .withCollectionZkHost(COLLECTIONORALIAS, cluster.getZkServer().getZkAddress())
+        .withFunctionName("search", CloudSolrStream.class)
+        .withFunctionName("having", HavingStream.class)
+        .withFunctionName("and", AndOperation.class)
+        .withFunctionName("or", OrOperation.class)
+        .withFunctionName("not", NotOperation.class)
+        .withFunctionName("gt", GreaterThanOperation.class)
+        .withFunctionName("lt", LessThanOperation.class)
+        .withFunctionName("eq", EqualsOperation.class)
+        .withFunctionName("lteq", LessThanEqualToOperation.class)
+        .withFunctionName("gteq", GreaterThanEqualToOperation.class)
+        .withFunctionName("parallel", ParallelStream.class);
+
+    stream = factory.constructStream("parallel(" + COLLECTIONORALIAS + ", workers=2, sort=\"a_f asc\", having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=id), eq(a_i, 9)))");
+    StreamContext context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 1);
+    Tuple t = tuples.get(0);
+    assertTrue(t.getString("id").equals("9"));
+
+    stream = factory.constructStream("parallel(" + COLLECTIONORALIAS + ", workers=2, sort=\"a_f asc\", having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=id), and(eq(a_i, 9),lt(a_i, 10))))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 1);
+    t = tuples.get(0);
+    assertTrue(t.getString("id").equals("9"));
+
+    stream = factory.constructStream("parallel(" + COLLECTIONORALIAS + ", workers=2, sort=\"a_f asc\",having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=id), or(eq(a_i, 9),eq(a_i, 8))))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 2);
+    t = tuples.get(0);
+    assertTrue(t.getString("id").equals("8"));
+
+    t = tuples.get(1);
+    assertTrue(t.getString("id").equals("9"));
+
+
+    stream = factory.constructStream("parallel(" + COLLECTIONORALIAS + ", workers=2, sort=\"a_f asc\", having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=id), and(eq(a_i, 9),not(eq(a_i, 9)))))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    assert(tuples.size() == 0);
+
+
+    stream = factory.constructStream("parallel(" + COLLECTIONORALIAS + ", workers=2, sort=\"a_f asc\",having(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\", partitionKeys=id), and(lteq(a_i, 9), gteq(a_i, 8))))");
+    context = new StreamContext();
+    context.setSolrClientCache(solrClientCache);
+    stream.setStreamContext(context);
+    tuples = getTuples(stream);
+
+    System.out.println("####Tuples:"+tuples.size());
+    assert(tuples.size() == 2);
+
+    t = tuples.get(0);
+    assertTrue(t.getString("id").equals("8"));
+
+    t = tuples.get(1);
+    assertTrue(t.getString("id").equals("9"));
+
+    solrClientCache.close();
+  }
+
   @Test
   public void testFetchStream() throws Exception {
 


[5/5] lucene-solr:branch_6x: Remove spurious import

Posted by jb...@apache.org.
Remove spurious import


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/0ab1ddb3
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/0ab1ddb3
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/0ab1ddb3

Branch: refs/heads/branch_6x
Commit: 0ab1ddb31e5bbde1c7956bb44114066d0c1aa163
Parents: d8a5814
Author: Joel Bernstein <jb...@apache.org>
Authored: Tue Jan 3 17:27:39 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Tue Jan 3 17:27:39 2017 -0500

----------------------------------------------------------------------
 .../src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java   | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/0ab1ddb3/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
index 7f991a4..0669cbe 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
@@ -19,7 +19,6 @@ package org.apache.solr.cloud;
 import java.io.File;
 import java.lang.invoke.MethodHandles;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.io.FileUtils;


[4/5] lucene-solr:branch_6x: SOLR-8530: Updated CHANGES.txt

Posted by jb...@apache.org.
SOLR-8530: Updated CHANGES.txt


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/d8a58146
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/d8a58146
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/d8a58146

Branch: refs/heads/branch_6x
Commit: d8a58146c3155a13f9bb8c46eb2d2878301426d3
Parents: 00af5ff
Author: Joel Bernstein <jb...@apache.org>
Authored: Tue Jan 3 17:09:21 2017 -0500
Committer: Joel Bernstein <jb...@apache.org>
Committed: Tue Jan 3 17:13:55 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d8a58146/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0429e17..6d0ecc8 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -150,6 +150,8 @@ New Features
 
 * SOLR-9854: Collect metrics for index merges and index store IO (ab)
 
+* SOLR-8530: Add HavingStream to Streaming API and StreamingExpressions (Joel Bernstein)
+
 Optimizations
 ----------------------
 * SOLR-9704: Facet Module / JSON Facet API: Optimize blockChildren facets that have