You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by an...@apache.org on 2016/03/13 18:27:51 UTC

[05/16] jena git commit: JENA-1156: add agg:stdev, agg:stdev_samp, agg:stdev_pop and variance.

JENA-1156: add agg:stdev, agg:stdev_samp, agg:stdev_pop and variance.

And DISINCT versions: URI + "d"

Contributes to JENA-803.

Includes refactoring in the gramamr for SPARQL 1.1.
Includes fixes and changes to the grammar for the ARQ language.


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

Branch: refs/heads/master
Commit: 9538e47368e886d3710f22b41ff9f409d1a86626
Parents: afd4ed6
Author: Andy Seaborne <an...@apache.org>
Authored: Fri Mar 11 18:05:01 2016 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Sun Mar 13 17:21:00 2016 +0000

----------------------------------------------------------------------
 jena-arq/Grammar/arq.jj                         |  63 +-
 jena-arq/Grammar/master.jj                      |  62 +-
 jena-arq/Grammar/sparql_11.jj                   |  46 +-
 .../jena/sparql/expr/aggregate/Accumulator.java |   1 -
 .../expr/aggregate/AccumulatorFactory.java      |   6 +-
 .../jena/sparql/expr/aggregate/AggCustom.java   |   8 +-
 .../expr/aggregate/AggregateRegistry.java       |  12 +-
 .../jena/sparql/expr/aggregate/Aggregator.java  |   2 +-
 .../expr/aggregate/AggregatorFactory.java       |   8 +-
 .../apache/jena/sparql/expr/aggregate/Args.java |  28 +
 .../sparql/expr/aggregate/lib/AccStatBase.java  |  99 +++
 .../sparql/expr/aggregate/lib/AccStatLib.java   |  46 +
 .../aggregate/lib/AccStatStdDevPopulation.java  |  34 +
 .../expr/aggregate/lib/AccStatStdDevSample.java |  34 +
 .../aggregate/lib/AccStatVarPopulation.java     |  34 +
 .../expr/aggregate/lib/AccStatVarSample.java    |  34 +
 .../aggregate/lib/StandardCustomAggregates.java |  64 ++
 .../jena/sparql/function/StandardFunctions.java |   3 +
 .../apache/jena/sparql/lang/arq/ARQParser.java  | 866 ++++++++-----------
 .../sparql/lang/sparql_11/SPARQLParser11.java   |  39 +-
 .../jena/sparql/sse/builders/BuilderExpr.java   |   2 +-
 .../org/apache/jena/sparql/expr/TS_Expr.java    |   1 +
 .../jena/sparql/expr/TestCustomAggregates.java  |   4 +-
 .../sparql/expr/TestStatisticsAggregates.java   | 215 +++++
 24 files changed, 1074 insertions(+), 637 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/Grammar/arq.jj
----------------------------------------------------------------------
diff --git a/jena-arq/Grammar/arq.jj b/jena-arq/Grammar/arq.jj
index 562c5d4..1d30d0f 100644
--- a/jena-arq/Grammar/arq.jj
+++ b/jena-arq/Grammar/arq.jj
@@ -754,7 +754,7 @@ Expr Constraint() : { Expr c ; }
   )
   { return c ; }
 }
-Expr FunctionCall() : { String fname ; ExprList a ; }
+Expr FunctionCall() : { String fname ; Args a ; }
 {
   fname = iri()
   a = ArgList()
@@ -762,21 +762,20 @@ Expr FunctionCall() : { String fname ; ExprList a ; }
      if ( AggregateRegistry.isRegistered(fname) ) {
          if ( ! getAllowAggregatesInExpressions() )
             throwParseException("Aggregate expression not legal at this point : "+fname, -1, -1) ;
-         Aggregator agg = AggregatorFactory.createCustom(true, false, fname, a) ;
+         Aggregator agg = AggregatorFactory.createCustom(fname, a) ;
          Expr exprAgg = getQuery().allocAggregate(agg) ;
          return exprAgg ;
      }
      return new E_Function(fname, a) ;
   }
 }
-ExprList ArgList() : { Expr expr ; boolean distinct = false ;
-                      ExprList args = new ExprList() ; Token t ; }
+Args ArgList() : { Expr expr ; Args args = new Args() ; Token t ; }
 {
   (
     <NIL>
   |
     <LPAREN>
-      (t = <DISTINCT> { distinct = true ; }
+      (t = <DISTINCT> { args.distinct = true ; }
       { int beginLine = t.beginLine; int beginColumn = t.beginColumn; t = null; }
         {
           if ( ! getAllowAggregatesInExpressions() )
@@ -790,17 +789,17 @@ ExprList ArgList() : { Expr expr ; boolean distinct = false ;
    )
     { return args ; }
 }
-ExprList ExpressionList() : { Expr expr = null ; ExprList args = new ExprList() ;}
+ExprList ExpressionList() : { Expr expr = null ; ExprList exprList = new ExprList() ;}
 {
   (
     <NIL>
   |
     <LPAREN>
-    expr = Expression() { args.add(expr) ; }
-      (<COMMA> expr = Expression() { args.add(expr) ; } )*
+    expr = Expression() { exprList.add(expr) ; }
+      (<COMMA> expr = Expression() { exprList.add(expr) ; } )*
     <RPAREN>
   )
-  { return args ; }
+  { return exprList ; }
 }
 Template ConstructTemplate() : { QuadAcc acc = new QuadAcc() ;
                                  Template t = new Template (acc);}
@@ -1429,9 +1428,8 @@ Expr NotExistsFunc() : { Element el ; }
    { return createExprNotExists(el) ; }
 }
 Expr Aggregate() : { Aggregator agg = null ; String sep = null ;
-                     boolean distinct = false ;
                      Expr expr = null ; Expr expr2 = null ;
-                     ExprList a = new ExprList() ;
+                     boolean distinct = false ;
                      ExprList ordered = new ExprList() ;
                      Token t ; }
 {
@@ -1455,7 +1453,7 @@ Expr Aggregate() : { Aggregator agg = null ; String sep = null ;
   | t = <GROUP_CONCAT>
     <LPAREN>
     (t = <DISTINCT> { distinct = true ; })?
-    expr = Expression() { a.add(expr) ; }
+    expr = Expression()
     (LOOKAHEAD(2)
        ( <SEMICOLON> <SEPARATOR> <EQ> sep=String()
            (<SEMICOLON> <ORDER><BY> expr2 = Expression() { ordered.add(expr2) ; })?
@@ -1465,29 +1463,22 @@ Expr Aggregate() : { Aggregator agg = null ; String sep = null ;
     )?
     <RPAREN>
     { agg = AggregatorFactory.createGroupConcat(distinct, expr, sep, ordered) ; }
-    | t = <AGG>
-          { String iri ; }
-          iri = iri()
-          <LPAREN>
-          ( <DISTINCT> { distinct = true ; } )?
-          (
-             expr = Expression() { a.add(expr) ; }
-             (<COMMA> expr = Expression() { a.add(expr) ; } )*
-          )?
-          <RPAREN>
-      { agg = AggregatorFactory.createCustom(false, distinct, iri, a) ; }
-   )
-   {
-     if ( ! getAllowAggregatesInExpressions() )
-            throwParseException("Aggregate expression not legal at this point",
-                                 t.beginLine, t.beginColumn) ;
-   }
-   { Expr exprAgg = getQuery().allocAggregate(agg) ;
-     return exprAgg ; }
-}
-Expr iriOrFunction() : { String iri ; ExprList a = null ;
-                         ExprList params = null ;
-                         boolean distinct = false ; }
+  | t = <AGG>
+    { String iri ; }
+    iri = iri()
+    { Args a = new Args() ; }
+    a = ArgList()
+    { agg = AggregatorFactory.createCustom(iri, a) ; }
+  )
+  {
+    if ( ! getAllowAggregatesInExpressions() )
+           throwParseException("Aggregate expression not legal at this point",
+                                t.beginLine, t.beginColumn) ;
+  }
+  { Expr exprAgg = getQuery().allocAggregate(agg) ;
+    return exprAgg ; }
+}
+Expr iriOrFunction() : { String iri ; Args a = null ; }
 {
   iri = iri()
   (a = ArgList())?
@@ -1496,7 +1487,7 @@ Expr iriOrFunction() : { String iri ; ExprList a = null ;
     if ( AggregateRegistry.isRegistered(iri) ) {
          if ( ! getAllowAggregatesInExpressions() )
             throwParseException("Aggregate expression not legal at this point : "+iri, -1, -1) ;
-         Aggregator agg = AggregatorFactory.createCustom(true, false, iri, a) ;
+         Aggregator agg = AggregatorFactory.createCustom(iri, a) ;
          Expr exprAgg = getQuery().allocAggregate(agg) ;
          return exprAgg ;
       }

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/Grammar/master.jj
----------------------------------------------------------------------
diff --git a/jena-arq/Grammar/master.jj b/jena-arq/Grammar/master.jj
index 8d1adcb..c04d1ca 100644
--- a/jena-arq/Grammar/master.jj
+++ b/jena-arq/Grammar/master.jj
@@ -1058,17 +1058,16 @@ Expr Constraint() : { Expr c ; }
   { return c ; }
 }
 
-Expr FunctionCall() : { String fname ; ExprList a ; }
+Expr FunctionCall() : { String fname ; Args a ; }
 {
   fname = iri()
   a = ArgList()
-
   {
      if ( AggregateRegistry.isRegistered(fname) ) {
          // aggregates
          if ( ! getAllowAggregatesInExpressions() )
             throwParseException("Aggregate expression not legal at this point : "+fname, -1, -1) ;
-         Aggregator agg = AggregatorFactory.createCustom(true, false, fname, a) ;
+         Aggregator agg = AggregatorFactory.createCustom(fname, a) ;
          Expr exprAgg = getQuery().allocAggregate(agg) ;
          return exprAgg ;
      }
@@ -1076,14 +1075,13 @@ Expr FunctionCall() : { String fname ; ExprList a ; }
   }
 }
 
-ExprList ArgList() : { Expr expr ; boolean distinct = false ;
-                      ExprList args = new ExprList() ; Token t ; }
+Args ArgList() : { Expr expr ; Args args = new Args() ; Token t ; }
 { 
   (
     <NIL>
   |
     <LPAREN>
-      (t = <DISTINCT> { distinct = true ; }
+      (t = <DISTINCT> { args.distinct = true ; }
       { int beginLine = t.beginLine; int beginColumn = t.beginColumn; t = null; }
         {
           if ( ! getAllowAggregatesInExpressions() )
@@ -1098,17 +1096,17 @@ ExprList ArgList() : { Expr expr ; boolean distinct = false ;
     { return args ; }
 }
 
-ExprList ExpressionList() : { Expr expr = null ; ExprList args = new ExprList() ;}
+ExprList ExpressionList() : { Expr expr = null ; ExprList exprList = new ExprList() ;}
 {
   (
     <NIL>
   |
     <LPAREN>
-    expr = Expression() { args.add(expr) ; } 
-      (<COMMA> expr = Expression() { args.add(expr) ; } )* 
+    expr = Expression() { exprList.add(expr) ; } 
+      (<COMMA> expr = Expression() { exprList.add(expr) ; } )* 
     <RPAREN>
   )
-  { return args ; }
+  { return exprList ; }
 }
 
 
@@ -1983,9 +1981,8 @@ Expr NotExistsFunc() : { Element el ; }
 }
 
 Expr Aggregate() : { Aggregator agg = null ; String sep = null ;
-                     boolean distinct = false ;
                      Expr expr = null ;  Expr expr2 = null ;
-                     ExprList a = new ExprList() ;
+                     boolean distinct = false ;
                      ExprList ordered = new ExprList() ;
                      Token t ; }
 {
@@ -2018,7 +2015,7 @@ Expr Aggregate() : { Aggregator agg = null ; String sep = null ;
   | t = <GROUP_CONCAT>
     <LPAREN>
     (t = <DISTINCT> { distinct = true ; })?
-    expr = Expression() { a.add(expr) ; } 
+    expr = Expression()
 #ifdef SPARQL_11
     // Single arg version
     (<SEMICOLON> <SEPARATOR> <EQ> sep=String())?
@@ -2038,36 +2035,29 @@ Expr Aggregate() : { Aggregator agg = null ; String sep = null ;
 
 #ifdef ARQ
     /* Explicit syntax (aggregate even if not registered) */
-    | t = <AGG>
-          { String iri ; }
-          iri = iri()
-          <LPAREN>
-          ( <DISTINCT> { distinct = true ; } )?
-          (
-             expr = Expression() { a.add(expr) ; } 
-             (<COMMA> expr = Expression() { a.add(expr) ; } )*            
-          )?
-          <RPAREN>
-      { agg = AggregatorFactory.createCustom(false, distinct, iri, a) ; }
+  | t = <AGG>
+    { String iri ; }
+    iri = iri()
+    { Args a = new Args() ; }
+    a = ArgList()
+    { agg = AggregatorFactory.createCustom(iri, a) ; }
 #endif
-   )
+  )
 
-   {
-     if ( ! getAllowAggregatesInExpressions() )
-            throwParseException("Aggregate expression not legal at this point",
-                                 t.beginLine, t.beginColumn) ;
-   }
-   { Expr exprAgg = getQuery().allocAggregate(agg) ;
-     return exprAgg ; }
+  {
+    if ( ! getAllowAggregatesInExpressions() )
+           throwParseException("Aggregate expression not legal at this point",
+                                t.beginLine, t.beginColumn) ;
+  }
+  { Expr exprAgg = getQuery().allocAggregate(agg) ;
+    return exprAgg ; }
 }
 
 // See also FunctionCall.
 // The case of "q:name()" or "q:agg()" or just "q:name"
 // by expanding out FunctionCall()
 
-Expr iriOrFunction() : { String iri ; ExprList a = null ; 
-                         ExprList params = null ; 
-                         boolean distinct = false ; }
+Expr iriOrFunction() : { String iri ; Args a = null ; }
 {
   iri = iri()
   (a = ArgList())?
@@ -2077,7 +2067,7 @@ Expr iriOrFunction() : { String iri ; ExprList a = null ;
          // aggregates
          if ( ! getAllowAggregatesInExpressions() )
             throwParseException("Aggregate expression not legal at this point : "+iri, -1, -1) ;
-         Aggregator agg = AggregatorFactory.createCustom(true, false, iri, a) ;
+         Aggregator agg = AggregatorFactory.createCustom(iri, a) ;
          Expr exprAgg = getQuery().allocAggregate(agg) ;
          return exprAgg ;
       }

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/Grammar/sparql_11.jj
----------------------------------------------------------------------
diff --git a/jena-arq/Grammar/sparql_11.jj b/jena-arq/Grammar/sparql_11.jj
index 4e14be5..7f19c46 100644
--- a/jena-arq/Grammar/sparql_11.jj
+++ b/jena-arq/Grammar/sparql_11.jj
@@ -679,7 +679,7 @@ Expr Constraint() : { Expr c ; }
   )
   { return c ; }
 }
-Expr FunctionCall() : { String fname ; ExprList a ; }
+Expr FunctionCall() : { String fname ; Args a ; }
 {
   fname = iri()
   a = ArgList()
@@ -687,21 +687,20 @@ Expr FunctionCall() : { String fname ; ExprList a ; }
      if ( AggregateRegistry.isRegistered(fname) ) {
          if ( ! getAllowAggregatesInExpressions() )
             throwParseException("Aggregate expression not legal at this point : "+fname, -1, -1) ;
-         Aggregator agg = AggregatorFactory.createCustom(true, false, fname, a) ;
+         Aggregator agg = AggregatorFactory.createCustom(fname, a) ;
          Expr exprAgg = getQuery().allocAggregate(agg) ;
          return exprAgg ;
      }
      return new E_Function(fname, a) ;
   }
 }
-ExprList ArgList() : { Expr expr ; boolean distinct = false ;
-                      ExprList args = new ExprList() ; Token t ; }
+Args ArgList() : { Expr expr ; Args args = new Args() ; Token t ; }
 {
   (
     <NIL>
   |
     <LPAREN>
-      (t = <DISTINCT> { distinct = true ; }
+      (t = <DISTINCT> { args.distinct = true ; }
       { int beginLine = t.beginLine; int beginColumn = t.beginColumn; t = null; }
         {
           if ( ! getAllowAggregatesInExpressions() )
@@ -715,17 +714,17 @@ ExprList ArgList() : { Expr expr ; boolean distinct = false ;
    )
     { return args ; }
 }
-ExprList ExpressionList() : { Expr expr = null ; ExprList args = new ExprList() ;}
+ExprList ExpressionList() : { Expr expr = null ; ExprList exprList = new ExprList() ;}
 {
   (
     <NIL>
   |
     <LPAREN>
-    expr = Expression() { args.add(expr) ; }
-      (<COMMA> expr = Expression() { args.add(expr) ; } )*
+    expr = Expression() { exprList.add(expr) ; }
+      (<COMMA> expr = Expression() { exprList.add(expr) ; } )*
     <RPAREN>
   )
-  { return args ; }
+  { return exprList ; }
 }
 Template ConstructTemplate() : { TripleCollectorBGP acc = new TripleCollectorBGP();
                                  Template t = new Template(acc.getBGP()) ; }
@@ -1298,9 +1297,8 @@ Expr NotExistsFunc() : { Element el ; }
    { return createExprNotExists(el) ; }
 }
 Expr Aggregate() : { Aggregator agg = null ; String sep = null ;
-                     boolean distinct = false ;
                      Expr expr = null ; Expr expr2 = null ;
-                     ExprList a = new ExprList() ;
+                     boolean distinct = false ;
                      ExprList ordered = new ExprList() ;
                      Token t ; }
 {
@@ -1324,22 +1322,20 @@ Expr Aggregate() : { Aggregator agg = null ; String sep = null ;
   | t = <GROUP_CONCAT>
     <LPAREN>
     (t = <DISTINCT> { distinct = true ; })?
-    expr = Expression() { a.add(expr) ; }
+    expr = Expression()
     (<SEMICOLON> <SEPARATOR> <EQ> sep=String())?
     <RPAREN>
     { agg = AggregatorFactory.createGroupConcat(distinct, expr, sep, ordered) ; }
-   )
-   {
-     if ( ! getAllowAggregatesInExpressions() )
-            throwParseException("Aggregate expression not legal at this point",
-                                 t.beginLine, t.beginColumn) ;
-   }
-   { Expr exprAgg = getQuery().allocAggregate(agg) ;
-     return exprAgg ; }
-}
-Expr iriOrFunction() : { String iri ; ExprList a = null ;
-                         ExprList params = null ;
-                         boolean distinct = false ; }
+  )
+  {
+    if ( ! getAllowAggregatesInExpressions() )
+           throwParseException("Aggregate expression not legal at this point",
+                                t.beginLine, t.beginColumn) ;
+  }
+  { Expr exprAgg = getQuery().allocAggregate(agg) ;
+    return exprAgg ; }
+}
+Expr iriOrFunction() : { String iri ; Args a = null ; }
 {
   iri = iri()
   (a = ArgList())?
@@ -1348,7 +1344,7 @@ Expr iriOrFunction() : { String iri ; ExprList a = null ;
     if ( AggregateRegistry.isRegistered(iri) ) {
          if ( ! getAllowAggregatesInExpressions() )
             throwParseException("Aggregate expression not legal at this point : "+iri, -1, -1) ;
-         Aggregator agg = AggregatorFactory.createCustom(true, false, iri, a) ;
+         Aggregator agg = AggregatorFactory.createCustom(iri, a) ;
          Expr exprAgg = getQuery().allocAggregate(agg) ;
          return exprAgg ;
       }

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/Accumulator.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/Accumulator.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/Accumulator.java
index 1004716..974dbfb 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/Accumulator.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/Accumulator.java
@@ -30,5 +30,4 @@ public interface Accumulator
 {
     public void accumulate(Binding binding, FunctionEnv functionEnv) ;
     public NodeValue getValue() ;
-    //public QueryIterator results() ;
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AccumulatorFactory.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AccumulatorFactory.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AccumulatorFactory.java
index ed398e3..a9b9438 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AccumulatorFactory.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AccumulatorFactory.java
@@ -22,10 +22,6 @@ package org.apache.jena.sparql.expr.aggregate;
  * processor for one group key for one execution.
  */
 public interface AccumulatorFactory {
-    /**
-     * 
-     * @return Accumulator
-     */
-    public Accumulator createAccumulator(AggCustom agg) ;
+    public Accumulator createAccumulator(AggCustom agg, boolean distinct) ;
 }
 

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggCustom.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggCustom.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggCustom.java
index 9a2b2ec..40e8ed3 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggCustom.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggCustom.java
@@ -42,13 +42,13 @@ public class AggCustom extends AggregatorBase
     
     private final String iri ;
 
-    public AggCustom(String iri, ExprList exprs) { 
-        super("AGG", false, exprs) ;
+    public AggCustom(String iri, boolean distinct, ExprList exprs) { 
+        super("AGG", distinct, exprs) ;
         this.iri = iri ; 
     } 
     
     @Override
-    public Aggregator copy(ExprList exprs) { return new AggCustom(iri, exprs) ; }
+    public Aggregator copy(ExprList exprs) { return new AggCustom(iri, isDistinct, exprs) ; }
     
     @Override
     public String asSparqlExpr(SerializationContext sCxt) {
@@ -104,7 +104,7 @@ public class AggCustom extends AggregatorBase
         AccumulatorFactory f = AggregateRegistry.getAccumulatorFactory(iri) ;
         if ( f == null )
             throw new QueryExecException("Unregistered aggregate: "+iri) ;
-        return f.createAccumulator(this) ;
+        return f.createAccumulator(this, isDistinct) ;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggregateRegistry.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggregateRegistry.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggregateRegistry.java
index cd63e76..322a489 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggregateRegistry.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggregateRegistry.java
@@ -29,8 +29,8 @@ import org.apache.jena.graph.Node ;
  */
 public class AggregateRegistry {
     
-    private static Map<String, AccumulatorFactory> registry = new HashMap<>() ;
-    private static Map<String, Node>               noGroupValues = new HashMap<>() ;
+    private static Map<String, AccumulatorFactory> registry         = new HashMap<>() ;
+    private static Map<String, Node>               noGroupValues    = new HashMap<>() ;
     
     /**
      * Register a custom aggregate, with its associated factory for accumulators.
@@ -44,9 +44,7 @@ public class AggregateRegistry {
         noGroupValues.put(uri, noGroupValue) ;
     }
 
-    /**
-     * Remove a registration.
-     */
+    /** Remove a registration. */
     public static void unregister(String uri) {
         registry.remove(uri) ;
         noGroupValues.remove(uri) ;
@@ -57,12 +55,12 @@ public class AggregateRegistry {
         return registry.get(uri) ;
     }
     
-    /** Return the AccumulatorFactory for a registered custom aggregate. */
+    /** Return the registered "no groups" value */ 
     public static Node getNoGroupValue(String uri) {
         return noGroupValues.get(uri) ;
     }
 
-    /** Return the AccumulatorFactory for a registered custom aggregate. */
+    /** Is the URI registered as an aggregate function? */
     public static boolean isRegistered(String uri) {
         return registry.containsKey(uri) ;
     }

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/Aggregator.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/Aggregator.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/Aggregator.java
index 0449913..6bd02dd 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/Aggregator.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/Aggregator.java
@@ -34,7 +34,7 @@ public interface Aggregator
     /** Create an accumulator for this aggregator */ 
     public Accumulator createAccumulator() ;
     
-    /** Value if there are no elements in any group : return null for no result */
+    /** Value if there are no groups : return null for no result */
     public Node getValueEmpty() ;
     
     public String toPrefixString()  ;

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggregatorFactory.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggregatorFactory.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggregatorFactory.java
index a268f49..d055929 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggregatorFactory.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/AggregatorFactory.java
@@ -63,9 +63,13 @@ public class AggregatorFactory {
         return new AggNull() ;
     }
 
-    public static Aggregator createCustom(boolean usedAsFunction, boolean distinct, String iri, ExprList a) {
+    public static Aggregator createCustom(String iri, Args a) {
+        return createCustom(iri, a.distinct, ExprList.copy(a)) ;
+    }
+    
+    public static Aggregator createCustom(String iri, boolean distinct, ExprList exprs) {
         if ( ! AggregateRegistry.isRegistered(iri) )
             Log.warn(AggregatorFactory.class, "Not registered: custom aggregate <"+iri+">") ;
-        return new AggCustom(iri, a) ;
+        return new AggCustom(iri, distinct, exprs) ;
     }
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/Args.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/Args.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/Args.java
new file mode 100644
index 0000000..4a4d5db
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/Args.java
@@ -0,0 +1,28 @@
+/*
+ * 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.expr.ExprList ;
+
+/** An Args is an ExprList with a flag for DISTINCT.
+ *  Only used for rule ArgList in the SPARQl grammar.
+ */  
+public class Args extends ExprList {
+    public boolean distinct = false ;
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatBase.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatBase.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatBase.java
new file mode 100644
index 0000000..ade562d
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatBase.java
@@ -0,0 +1,99 @@
+/*
+ * 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.lib;
+
+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.expr.aggregate.AccumulatorExpr ;
+import org.apache.jena.sparql.function.FunctionEnv ;
+
+/** Base for statistics aggregations */ 
+abstract class AccStatBase extends AccumulatorExpr {
+    // Could also be used for AVG and SUM but those came before this.
+    
+    private static final NodeValue noValuesToAvg = NodeValue.nvZERO ; // null? NaN?
+    private static final NodeValue errorValue    = null ;
+
+    public AccStatBase(Expr expr, boolean distinct) {
+        // Merge "distinct" into AccumulatorExpr
+        super(expr, distinct);
+    }
+    protected long   count          = 0 ;
+    protected double K              = 0 ;
+    // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
+    // Var = (SumSq − (Sum × Sum) / n) / (n − 1)
+    // These are offset by K.
+    protected double sum          = 0 ;
+    protected double sumSquared   = 0 ;
+
+    @Override
+    final protected NodeValue getAccValue() {
+        if ( super.errorCount != 0 )
+            //throw new ExprEvalException("avg: error in group") ; 
+            return null ;
+        if ( count <= 0 ) return noValuesToAvg ;
+        try {
+            double x1 = calc() ;
+            return NodeValue.makeDouble(x1) ;
+        } catch (ExprEvalException ex) { return errorValue ; }
+    }
+
+    abstract protected double calc() ;
+
+    /** Calculate the variance (sample) */ 
+    final protected double calcVarianceSample() {
+        // (N*sum(?x*?x) - sum(?x) ) / N*(N-1) 
+        return AccStatLib.calcVarianceSample(sumSquared, sum, count) ; 
+    }
+    
+    /** Calculate the variance (population) */ 
+    final protected double calcVariancePop() {
+        // (N*sum(?x*?x) - sum(?x) ) / N*N 
+        return AccStatLib.calcVariancePopulation(sumSquared, sum, count) ;
+    }
+    
+    @Override
+    protected void accumulate(NodeValue nv, Binding binding, FunctionEnv functionEnv) {
+        // shifted_data_variance
+        if ( nv.isNumber() ) {
+            double d = nv.getDouble() ;
+            count++ ;
+            if ( count == 1 ) {
+                K = d ;
+                sum = (d-K) ; // == 0 of K set.
+                sumSquared = (d-K)*(d-K) ; // == 0
+                return ; 
+            }
+            else {
+                double dk = (d-K) ;
+                double dk2 = dk * dk ;
+                sum = sum + dk ;
+                sumSquared = sumSquared + dk2 ;
+            }
+        }
+        else
+            throw new ExprEvalException("Not a number: "+nv) ;
+    }
+
+    @Override
+    protected void accumulateError(Binding binding, FunctionEnv functionEnv)
+    {}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatLib.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatLib.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatLib.java
new file mode 100644
index 0000000..bfd3e3c
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatLib.java
@@ -0,0 +1,46 @@
+/*
+ * 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.lib;
+
+import org.apache.jena.sparql.expr.ExprEvalException ;
+
+public class AccStatLib {
+    /** Calculate the variance (sample) */ 
+    public static double calcVarianceSample(double sumSquared, double sum, long count) {
+        // (N*sum(?x*?x) - sum(?x) ) / N*(N-1) 
+        return calcVariance$(sumSquared, sum, count, count-1) ;
+    }
+
+    /** Calculate the variance (population) */ 
+    public static double calcVariancePopulation(double sumSquared, double sum, long N) {
+        return calcVariance$(sumSquared, sum, N, N) ;
+    }
+    
+    // Engine.
+    static private double calcVariance$(double sumSquared, double sum, long N, long N1) {
+//        System.out.printf("sum = %f, sumSq = %f, N=%d\n", sum, sumSquared, N) ;
+        if ( N <= 0 )
+            throw new ExprEvalException("N= "+N) ;
+        if ( N1 == 0 )
+            throw new ExprEvalException("Sample size one") ;
+        double x = sumSquared - (sum*sum)/N ;
+        x = x / N1 ;
+        return x ;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatStdDevPopulation.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatStdDevPopulation.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatStdDevPopulation.java
new file mode 100644
index 0000000..2058147
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatStdDevPopulation.java
@@ -0,0 +1,34 @@
+/*
+ * 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.lib;
+
+import org.apache.jena.sparql.expr.Expr ;
+
+/** Aggregate - statistical standard deviation - population (i.e. N) */ 
+public class AccStatStdDevPopulation extends AccStatVarPopulation {
+
+    public AccStatStdDevPopulation(Expr expr, boolean distinct) {
+        super(expr, distinct);
+    }
+    
+    @Override
+    protected double calc() {
+        return Math.sqrt(super.calc());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatStdDevSample.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatStdDevSample.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatStdDevSample.java
new file mode 100644
index 0000000..940d5da
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatStdDevSample.java
@@ -0,0 +1,34 @@
+/*
+ * 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.lib;
+
+import org.apache.jena.sparql.expr.Expr ;
+
+/** Aggregate - statistical standard deviation - sample (i.e. N-1) */ 
+public class AccStatStdDevSample extends AccStatVarSample {
+
+    public AccStatStdDevSample(Expr expr, boolean distinct) {
+        super(expr, distinct);
+    }
+
+    @Override
+    protected double calc() {
+        return Math.sqrt(super.calc());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatVarPopulation.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatVarPopulation.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatVarPopulation.java
new file mode 100644
index 0000000..8172baf
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatVarPopulation.java
@@ -0,0 +1,34 @@
+/*
+ * 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.lib;
+
+import org.apache.jena.sparql.expr.Expr ;
+
+/** Aggregate - statistical variance - population (i.e. N) */ 
+public class AccStatVarPopulation extends AccStatVarSample {
+
+    public AccStatVarPopulation(Expr expr, boolean distinct) {
+        super(expr, distinct);
+    }
+
+    @Override
+    protected double calc() {
+        return super.calcVariancePop();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatVarSample.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatVarSample.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatVarSample.java
new file mode 100644
index 0000000..8f5e1a8
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/AccStatVarSample.java
@@ -0,0 +1,34 @@
+/*
+ * 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.lib;
+
+import org.apache.jena.sparql.expr.Expr ;
+
+/** Aggregate - statistical variance - sample (i.e. N-1) */ 
+public class AccStatVarSample extends AccStatBase {
+
+    public AccStatVarSample(Expr expr, boolean distinct) {
+        super(expr, distinct);
+    }
+
+    @Override
+    protected double calc() {
+        return super.calcVarianceSample() ;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/StandardCustomAggregates.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/StandardCustomAggregates.java b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/StandardCustomAggregates.java
new file mode 100644
index 0000000..0a396d4
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/expr/aggregate/lib/StandardCustomAggregates.java
@@ -0,0 +1,64 @@
+/*
+ * 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.lib;
+
+import org.apache.jena.sparql.ARQConstants ;
+import org.apache.jena.sparql.expr.aggregate.AccumulatorFactory ;
+import org.apache.jena.sparql.expr.aggregate.AggregateRegistry ;
+
+/** Register custom aggregates.
+ * This includes the statistics aggregate functions as custom aggregates, 
+ * as {@code <http://jena.apache.org/ARQ/function#stdev} etc.
+ * This is commonly abbreviated {@code afn:stdev}.
+ */ 
+public class StandardCustomAggregates {
+
+    public static String BASE = ARQConstants.ARQFunctionLibraryURI ;
+
+    public static void register() {
+        // Non-DISTINCT
+        AccumulatorFactory f_VarP =     (agg, distinct) -> new AccStatVarPopulation(agg.getExpr(), distinct) ;
+        AccumulatorFactory f_Var =      (agg, distinct) -> new AccStatVarSample(agg.getExpr(), distinct) ;
+        AccumulatorFactory f_StdevP =   (agg, distinct) -> new AccStatStdDevPopulation(agg.getExpr(), distinct) ;
+        AccumulatorFactory f_Stdev =    (agg, distinct) -> new AccStatStdDevSample(agg.getExpr(), distinct) ;
+        
+//        AggregateRegistry.register(BASE+"stddevp", f_StdevP, null) ;     // Common? typo
+//        AggregateRegistry.register(BASE+"stddev", f_Stdev, null) ;       // Common? typo
+        AggregateRegistry.register(BASE+"stdevp", f_StdevP, null) ;      // SQL spelling
+        AggregateRegistry.register(BASE+"stdev", f_Stdev, null) ;        // SQL spelling
+        
+//        AggregateRegistry.register(BASE+"variancep", f_VarP, null) ;     // Longer
+//        AggregateRegistry.register(BASE+"variance", f_Var, null) ;       // Longer
+        AggregateRegistry.register(BASE+"varp", f_VarP, null) ;          // SQL spelling
+        AggregateRegistry.register(BASE+"var", f_Var, null) ;            // SQL spelling
+        
+        // DISTINCT versions for 
+        // 
+        AccumulatorFactory f_VarP_d =     (agg, distinct) -> new AccStatVarPopulation(agg.getExpr(), true) ;
+        AccumulatorFactory f_Var_d =      (agg, distinct) -> new AccStatVarSample(agg.getExpr(), true) ;
+        AccumulatorFactory f_StdevP_d =   (agg, distinct) -> new AccStatStdDevPopulation(agg.getExpr(), true) ;
+        AccumulatorFactory f_Stdev_d =    (agg, distinct) -> new AccStatStdDevSample(agg.getExpr(), true) ;
+
+        AggregateRegistry.register(BASE+"varpd", f_VarP_d, null) ;
+        AggregateRegistry.register(BASE+"vard", f_Var_d, null) ;
+        AggregateRegistry.register(BASE+"stdevpd", f_StdevP_d, null) ;
+        AggregateRegistry.register(BASE+"stdevd", f_Stdev_d, null) ;
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/9538e473/jena-arq/src/main/java/org/apache/jena/sparql/function/StandardFunctions.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/function/StandardFunctions.java b/jena-arq/src/main/java/org/apache/jena/sparql/function/StandardFunctions.java
index c860f47..1d5d9f4 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/function/StandardFunctions.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/function/StandardFunctions.java
@@ -20,6 +20,7 @@ package org.apache.jena.sparql.function;
 
 import org.apache.jena.datatypes.xsd.XSDDatatype ;
 import org.apache.jena.sparql.ARQConstants ;
+import org.apache.jena.sparql.expr.aggregate.lib.StandardCustomAggregates ;
 import org.apache.jena.sparql.function.library.* ;
 import org.apache.jena.sparql.function.library.sqrt ;
 import org.apache.jena.sparql.function.library.leviathan.* ;
@@ -34,6 +35,8 @@ public class StandardFunctions
         String math = ARQConstants.mathPrefix ;
         String sparqlfn = ARQConstants.fnSparql ;
         
+        StandardCustomAggregates.register();
+        
         // See http://www.w3.org/TR/xpath-datamodel/#types-hierarchy
         // No durations here