You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by rv...@apache.org on 2015/09/09 09:45:00 UTC

[49/50] [abbrv] jena git commit: Start implementing Leviathan aggregates (JENA-507)

Start implementing Leviathan aggregates (JENA-507)

- Adds initial implementations for all(), any() and none()
- Reworks AccumulatorExpr slightly to be more extensible
- Adds an AccumulatorShortCircuitExpr for aggregates that support short
  circuiting


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/7da685a9
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/7da685a9
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/7da685a9

Branch: refs/heads/JENA-507
Commit: 7da685a9434844653e283efb499d2400800d1fdd
Parents: f7a660e
Author: Rob Vesse <rv...@apache.org>
Authored: Mon Sep 7 14:50:05 2015 +0100
Committer: Rob Vesse <rv...@apache.org>
Committed: Mon Sep 7 14:50:05 2015 +0100

----------------------------------------------------------------------
 .../sparql/expr/aggregate/AccumulatorExpr.java  | 133 +++++++++++++------
 .../aggregate/AccumulatorShortCircuitExpr.java  |  85 ++++++++++++
 .../jena/sparql/expr/aggregate/AggAvg.java      |   3 +-
 .../library/leviathan/aggregates/AggAll.java    | 115 ++++++++++++++++
 .../library/leviathan/aggregates/AggAny.java    | 112 ++++++++++++++++
 .../library/leviathan/aggregates/AggNone.java   | 115 ++++++++++++++++
 6 files changed, 520 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/7da685a9/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AccumulatorExpr.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AccumulatorExpr.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AccumulatorExpr.java
index b94c98b..1203d6a 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AccumulatorExpr.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AccumulatorExpr.java
@@ -18,58 +18,109 @@
 
 package org.apache.jena.sparql.expr.aggregate;
 
-import org.apache.jena.sparql.engine.binding.Binding ;
-import org.apache.jena.sparql.expr.Expr ;
-import org.apache.jena.sparql.expr.ExprEvalException ;
-import org.apache.jena.sparql.expr.NodeValue ;
-import org.apache.jena.sparql.function.FunctionEnv ;
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.expr.Expr;
+import org.apache.jena.sparql.expr.ExprEvalException;
+import org.apache.jena.sparql.expr.NodeValue;
+import org.apache.jena.sparql.function.FunctionEnv;
 
 /** Accumulator that passes down every value of an expression */
-abstract class AccumulatorExpr implements Accumulator
-{
-    private long count = 0 ;
-    protected long errorCount = 0 ; 
-    private final Expr expr ;
-    
-    protected AccumulatorExpr(Expr expr)
-    {
-        this.expr = expr ;
+public abstract class AccumulatorExpr implements Accumulator {
+    private long count = 0;
+    private long errorCount = 0;
+    private final Expr expr;
+
+    protected AccumulatorExpr(Expr expr) {
+        this.expr = expr;
     }
-    
+
     @Override
-    final public void accumulate(Binding binding, FunctionEnv functionEnv)
-    {
-        try { 
-            NodeValue nv = expr.eval(binding, functionEnv) ;
-            accumulate(nv, binding, functionEnv) ;
-            count++ ;
-        } catch (ExprEvalException ex)
-        {
-            errorCount++ ;
-            accumulateError(binding, functionEnv) ;
+    public void accumulate(Binding binding, FunctionEnv functionEnv) {
+        try {
+            NodeValue nv = expr.eval(binding, functionEnv);
+            accumulate(nv, binding, functionEnv);
+            incrementCount();
+        } catch (ExprEvalException ex) {
+            incrementErrorCount();
+            accumulateError(binding, functionEnv);
         }
     }
     
-    
-    // Count(?v) is different
+    /**
+     * Gets the expression
+     * @return Expression
+     */
+    protected final Expr getExpr() {
+        return expr;
+    }
+
+    /**
+     * Increments the count of valid expression evaluations
+     */
+    protected final void incrementCount() {
+        count++;
+    }
+
+    /**
+     * Increments the count of error expression evaluations
+     */
+    protected final void incrementErrorCount() {
+        errorCount++;
+    }
+
+    /**
+     * Gets the value of the accumulator
+     */
     @Override
-    public NodeValue getValue()
-    {
-        if ( errorCount == 0 )
-            return getAccValue() ;  
-        return null ;
+    public NodeValue getValue() {
+        if (errorCount == 0)
+            return getAccValue();
+        return getErrorValue();
+    }
+
+    /**
+     * Gets the value that is returned in the event of any errors (defaults to
+     * {@code null})
+     * 
+     * @return Error value
+     */
+    protected NodeValue getErrorValue() {
+        return null;
+    }
+
+    /**
+     * Gets the count of valid expression evaluations
+     * 
+     * @return Valid expression evaluation count
+     */
+    protected final long getCount() {
+        return count;
+    }
+
+    /**
+     * Gets the count of expression evaluation errors encountered
+     * 
+     * @return Expression evaluation error count
+     */
+    protected final long getErrorCount() {
+        return errorCount;
     }
 
-    protected long getErrorCount() { return errorCount ; }
-    
     /** Called if no errors to get the accumulated result */
-    protected abstract NodeValue getAccValue() ; 
+    protected abstract NodeValue getAccValue();
+
+    /**
+     * Called when the expression being aggregated evaluates OK
+     * <p>
+     * Can throw {@link ExprEvalException} in which case the
+     * {@link #accumulateError} method is called
+     * </p>
+     */
+    protected abstract void accumulate(NodeValue nv, Binding binding, FunctionEnv functionEnv);
 
-    /** Called when the expression beeing aggregated evaluates OK.
-     * Can throw ExprEvalException - in which case the accumulateError is called */
-    protected abstract void accumulate(NodeValue nv, Binding binding, FunctionEnv functionEnv) ;
-    /** Called when an evaluation of the expression causes an error
-     * or when the accumulation step throws ExprEvalException  
+    /**
+     * Called when an evaluation of the expression causes an error or when the
+     * accumulation step throws {@link ExprEvalException}
      */
-    protected abstract void accumulateError(Binding binding, FunctionEnv functionEnv) ;
+    protected abstract void accumulateError(Binding binding, FunctionEnv functionEnv);
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/7da685a9/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AccumulatorShortCircuitExpr.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AccumulatorShortCircuitExpr.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AccumulatorShortCircuitExpr.java
new file mode 100644
index 0000000..cbaf053
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AccumulatorShortCircuitExpr.java
@@ -0,0 +1,85 @@
+/*
+ * 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.jena.sparql.expr.aggregate;
+
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.expr.Expr;
+import org.apache.jena.sparql.expr.ExprEvalException;
+import org.apache.jena.sparql.expr.NodeValue;
+import org.apache.jena.sparql.function.FunctionEnv;
+
+/**
+ * Accumulator that passes down the value of an expression but that allows for
+ * short circuiting.
+ * <p>
+ * Short circuiting handles the case where an accumulator can determine the
+ * final value ahead of seeing all the solutions. In this case it can indicate
+ * as such by calling the {@link #shortCircuit(NodeValue)} method after which
+ * point the evaluation of the expression is skipped for further solutions
+ * </p>
+ * 
+ */
+public abstract class AccumulatorShortCircuitExpr extends AccumulatorExpr {
+    private boolean shortCircuited = false;
+    private NodeValue shortCircuitValue;
+
+    protected AccumulatorShortCircuitExpr(Expr expr) {
+        super(expr);
+    }
+
+    @Override
+    public void accumulate(Binding binding, FunctionEnv functionEnv) {
+        // No need to do any further work if already short circuited
+        if (shortCircuited)
+            return;
+
+        try {
+            NodeValue nv = this.getExpr().eval(binding, functionEnv);
+            accumulate(nv, binding, functionEnv);
+            this.incrementCount();
+        } catch (ExprEvalException ex) {
+            this.incrementErrorCount();
+            accumulateError(binding, functionEnv);
+        }
+    }
+
+    // Count(?v) is different
+    @Override
+    public NodeValue getValue() {
+        if (shortCircuited)
+            return shortCircuitValue;
+        if (this.getErrorCount() == 0)
+            return getAccValue();
+        return getErrorValue();
+    }
+
+    /**
+     * Called when the derived class has been able to determine the final
+     * aggregate value without needing to process further values. Once set
+     * further expression evaluation is skipped and the final value will be the
+     * value given here
+     * 
+     * @param value
+     *            Final aggregate value
+     */
+    protected final void shortCircuit(NodeValue value) {
+        this.shortCircuited = true;
+        this.shortCircuitValue = value;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/7da685a9/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggAvg.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggAvg.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggAvg.java
index 57c32c7..d16bd63 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggAvg.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggAvg.java
@@ -100,8 +100,7 @@ public class AggAvg extends AggregatorBase
         public NodeValue getAccValue()
         {
             if ( count == 0 ) return noValuesToAvg ;
-            if ( super.errorCount != 0 )
-                //throw new ExprEvalException("avg: error in group") ; 
+            if ( this.getErrorCount() != 0 )
                 return null ;
             NodeValue nvCount = NodeValue.makeInteger(count) ;
             return XSDFuncOp.numDivide(total, nvCount) ;

http://git-wip-us.apache.org/repos/asf/jena/blob/7da685a9/jena-arq/src/main/java/org/apache/jena/sparql/function/library/leviathan/aggregates/AggAll.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/function/library/leviathan/aggregates/AggAll.java b/jena-arq/src/main/java/org/apache/jena/sparql/function/library/leviathan/aggregates/AggAll.java
new file mode 100644
index 0000000..1f499d6
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/function/library/leviathan/aggregates/AggAll.java
@@ -0,0 +1,115 @@
+/*
+ * 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.jena.sparql.function.library.leviathan.aggregates;
+
+import org.apache.jena.graph.Node;
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.expr.Expr;
+import org.apache.jena.sparql.expr.ExprList;
+import org.apache.jena.sparql.expr.NodeValue;
+import org.apache.jena.sparql.expr.aggregate.Accumulator;
+import org.apache.jena.sparql.expr.aggregate.AccumulatorShortCircuitExpr;
+import org.apache.jena.sparql.expr.aggregate.Aggregator;
+import org.apache.jena.sparql.expr.aggregate.AggregatorBase;
+import org.apache.jena.sparql.expr.nodevalue.NodeValueBoolean;
+import org.apache.jena.sparql.expr.nodevalue.XSDFuncOp;
+import org.apache.jena.sparql.function.FunctionEnv;
+import org.apache.jena.sparql.function.library.leviathan.LeviathanConstants;
+
+/**
+ * All aggregate, expression must evaluate to true for all solutions
+ */
+public class AggAll extends AggregatorBase {
+
+    /**
+     * @param expr
+     *            Expression
+     */
+    protected AggAll(Expr expr) {
+        super(LeviathanConstants.LeviathanFunctionLibraryURI + "all", false, expr);
+    }
+
+    @Override
+    public Aggregator copy(ExprList exprs) {
+        return new AggAll(exprs.get(0));
+    }
+
+    @Override
+    public boolean equals(Aggregator other, boolean bySyntax) {
+        if (other == null)
+            return false;
+        if (this == other)
+            return true;
+        if (!(other instanceof AggAll))
+            return false;
+        AggAll agg = (AggAll) other;
+        return getExpr().equals(agg.getExpr(), bySyntax);
+    }
+
+    @Override
+    public Accumulator createAccumulator() {
+        return new AccAll(this.getExpr());
+    }
+
+    @Override
+    public Node getValueEmpty() {
+        return NodeValue.TRUE.asNode();
+    }
+
+    @Override
+    public int hashCode() {
+        // TODO Auto-generated method stub
+        return 0;
+    }
+
+    private static class AccAll extends AccumulatorShortCircuitExpr {
+
+        /**
+         * @param expr
+         */
+        protected AccAll(Expr expr) {
+            super(expr);
+        }
+
+        @Override
+        protected NodeValue getAccValue() {
+            return NodeValue.TRUE;
+        }
+
+        @Override
+        protected NodeValue getErrorValue() {
+            return NodeValue.FALSE;
+        }
+
+        @Override
+        protected void accumulate(NodeValue nv, Binding binding, FunctionEnv functionEnv) {
+            // If true can't short circuit yet
+            if (XSDFuncOp.booleanEffectiveValue(nv))
+                return;
+
+            // Otherwise can short circuit
+            this.shortCircuit(NodeValue.FALSE);
+        }
+
+        @Override
+        protected void accumulateError(Binding binding, FunctionEnv functionEnv) {
+            this.shortCircuit(NodeValue.FALSE);
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/7da685a9/jena-arq/src/main/java/org/apache/jena/sparql/function/library/leviathan/aggregates/AggAny.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/function/library/leviathan/aggregates/AggAny.java b/jena-arq/src/main/java/org/apache/jena/sparql/function/library/leviathan/aggregates/AggAny.java
new file mode 100644
index 0000000..a462098
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/function/library/leviathan/aggregates/AggAny.java
@@ -0,0 +1,112 @@
+/*
+ * 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.jena.sparql.function.library.leviathan.aggregates;
+
+import org.apache.jena.graph.Node;
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.expr.Expr;
+import org.apache.jena.sparql.expr.ExprList;
+import org.apache.jena.sparql.expr.NodeValue;
+import org.apache.jena.sparql.expr.aggregate.Accumulator;
+import org.apache.jena.sparql.expr.aggregate.AccumulatorShortCircuitExpr;
+import org.apache.jena.sparql.expr.aggregate.Aggregator;
+import org.apache.jena.sparql.expr.aggregate.AggregatorBase;
+import org.apache.jena.sparql.expr.nodevalue.NodeValueBoolean;
+import org.apache.jena.sparql.expr.nodevalue.XSDFuncOp;
+import org.apache.jena.sparql.function.FunctionEnv;
+import org.apache.jena.sparql.function.library.leviathan.LeviathanConstants;
+
+/**
+ * All aggregate, expression must evaluate to true for at least one solution
+ */
+public class AggAny extends AggregatorBase {
+
+    /**
+     * @param expr
+     *            Expression
+     */
+    protected AggAny(Expr expr) {
+        super(LeviathanConstants.LeviathanFunctionLibraryURI + "any", false, expr);
+    }
+
+    @Override
+    public Aggregator copy(ExprList exprs) {
+        return new AggAny(exprs.get(0));
+    }
+
+    @Override
+    public boolean equals(Aggregator other, boolean bySyntax) {
+        if (other == null)
+            return false;
+        if (this == other)
+            return true;
+        if (!(other instanceof AggAny))
+            return false;
+        AggAny agg = (AggAny) other;
+        return getExpr().equals(agg.getExpr(), bySyntax);
+    }
+
+    @Override
+    public Accumulator createAccumulator() {
+        return new AccAny(this.getExpr());
+    }
+
+    @Override
+    public Node getValueEmpty() {
+        return NodeValue.TRUE.asNode();
+    }
+
+    @Override
+    public int hashCode() {
+        // TODO Auto-generated method stub
+        return 0;
+    }
+
+    private static class AccAny extends AccumulatorShortCircuitExpr {
+
+        /**
+         * @param expr
+         */
+        protected AccAny(Expr expr) {
+            super(expr);
+        }
+
+        @Override
+        protected NodeValue getAccValue() {
+            return NodeValue.FALSE;
+        }
+
+        @Override
+        protected NodeValue getErrorValue() {
+            return NodeValue.FALSE;
+        }
+
+        @Override
+        protected void accumulate(NodeValue nv, Binding binding, FunctionEnv functionEnv) {
+            // If true can short circuit yet
+            if (XSDFuncOp.booleanEffectiveValue(nv))
+                this.shortCircuit(NodeValue.TRUE);
+        }
+
+        @Override
+        protected void accumulateError(Binding binding, FunctionEnv functionEnv) {
+            // Can't short circuit errors
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/7da685a9/jena-arq/src/main/java/org/apache/jena/sparql/function/library/leviathan/aggregates/AggNone.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/function/library/leviathan/aggregates/AggNone.java b/jena-arq/src/main/java/org/apache/jena/sparql/function/library/leviathan/aggregates/AggNone.java
new file mode 100644
index 0000000..a155139
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/function/library/leviathan/aggregates/AggNone.java
@@ -0,0 +1,115 @@
+/*
+ * 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.jena.sparql.function.library.leviathan.aggregates;
+
+import org.apache.jena.graph.Node;
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.expr.Expr;
+import org.apache.jena.sparql.expr.ExprList;
+import org.apache.jena.sparql.expr.NodeValue;
+import org.apache.jena.sparql.expr.aggregate.Accumulator;
+import org.apache.jena.sparql.expr.aggregate.AccumulatorShortCircuitExpr;
+import org.apache.jena.sparql.expr.aggregate.Aggregator;
+import org.apache.jena.sparql.expr.aggregate.AggregatorBase;
+import org.apache.jena.sparql.expr.nodevalue.NodeValueBoolean;
+import org.apache.jena.sparql.expr.nodevalue.XSDFuncOp;
+import org.apache.jena.sparql.function.FunctionEnv;
+import org.apache.jena.sparql.function.library.leviathan.LeviathanConstants;
+
+/**
+ * All aggregate, expression must evaluate to false for all solutions
+ */
+public class AggNone extends AggregatorBase {
+
+    /**
+     * @param expr
+     *            Expression
+     */
+    protected AggNone(Expr expr) {
+        super(LeviathanConstants.LeviathanFunctionLibraryURI + "none", false, expr);
+    }
+
+    @Override
+    public Aggregator copy(ExprList exprs) {
+        return new AggNone(exprs.get(0));
+    }
+
+    @Override
+    public boolean equals(Aggregator other, boolean bySyntax) {
+        if (other == null)
+            return false;
+        if (this == other)
+            return true;
+        if (!(other instanceof AggNone))
+            return false;
+        AggNone agg = (AggNone) other;
+        return getExpr().equals(agg.getExpr(), bySyntax);
+    }
+
+    @Override
+    public Accumulator createAccumulator() {
+        return new AccNone(this.getExpr());
+    }
+
+    @Override
+    public Node getValueEmpty() {
+        return NodeValue.TRUE.asNode();
+    }
+
+    @Override
+    public int hashCode() {
+        // TODO Auto-generated method stub
+        return 0;
+    }
+
+    private static class AccNone extends AccumulatorShortCircuitExpr {
+
+        /**
+         * @param expr
+         */
+        protected AccNone(Expr expr) {
+            super(expr);
+        }
+
+        @Override
+        protected NodeValue getAccValue() {
+            return NodeValue.TRUE;
+        }
+
+        @Override
+        protected NodeValue getErrorValue() {
+            return NodeValue.FALSE;
+        }
+
+        @Override
+        protected void accumulate(NodeValue nv, Binding binding, FunctionEnv functionEnv) {
+            // If false can't short circuit yet
+            if (!XSDFuncOp.booleanEffectiveValue(nv))
+                return;
+
+            // Otherwise can short circuit
+            this.shortCircuit(NodeValue.FALSE);
+        }
+
+        @Override
+        protected void accumulateError(Binding binding, FunctionEnv functionEnv) {
+            this.shortCircuit(NodeValue.FALSE);
+        }
+
+    }
+}