nashorn js aggregators
diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregator.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregator.java
index 4f85c1d..d89cc1e 100644
--- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregator.java
+++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregator.java
@@ -26,16 +26,6 @@
 
 public class JavaScriptAggregator implements Aggregator
 {
-  static interface ScriptAggregator
-  {
-    public double aggregate(double current, ObjectColumnSelector[] selectorList);
-
-    public double combine(double a, double b);
-
-    public double reset();
-
-    public void close();
-  }
 
   private final String name;
   private final ObjectColumnSelector[] selectorList;
diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java
index 0c1e4d2..408068b 100644
--- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java
+++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java
@@ -28,20 +28,11 @@
 import com.google.common.primitives.Doubles;
 import io.druid.segment.ColumnSelectorFactory;
 import io.druid.segment.ObjectColumnSelector;
-import org.mozilla.javascript.Context;
-import org.mozilla.javascript.ContextAction;
-import org.mozilla.javascript.ContextFactory;
-import org.mozilla.javascript.Function;
-import org.mozilla.javascript.NativeArray;
-import org.mozilla.javascript.Scriptable;
-import org.mozilla.javascript.ScriptableObject;
 
 import javax.annotation.Nullable;
-import java.lang.reflect.Array;
 import java.nio.ByteBuffer;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
-import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
 
@@ -56,7 +47,7 @@
   private final String fnCombine;
 
 
-  private final JavaScriptAggregator.ScriptAggregator compiledScript;
+  private final ScriptAggregator compiledScript;
 
   @JsonCreator
   public JavaScriptAggregatorFactory(
@@ -80,7 +71,7 @@
     this.fnReset = fnReset;
     this.fnCombine = fnCombine;
 
-    this.compiledScript = compileScript(fnAggregate, fnReset, fnCombine);
+    this.compiledScript = new RhinoScriptAggregatorFactory(fnAggregate, fnReset, fnCombine).compileScript();
   }
 
   @Override
@@ -242,104 +233,6 @@
            '}';
   }
 
-  public static JavaScriptAggregator.ScriptAggregator compileScript(
-      final String aggregate,
-      final String reset,
-      final String combine
-  )
-  {
-    final ContextFactory contextFactory = ContextFactory.getGlobal();
-    Context context = contextFactory.enterContext();
-    context.setOptimizationLevel(9);
-
-    final ScriptableObject scope = context.initStandardObjects();
-
-    final Function fnAggregate = context.compileFunction(scope, aggregate, "aggregate", 1, null);
-    final Function fnReset = context.compileFunction(scope, reset, "reset", 1, null);
-    final Function fnCombine = context.compileFunction(scope, combine, "combine", 1, null);
-    Context.exit();
-
-    return new JavaScriptAggregator.ScriptAggregator()
-    {
-      @Override
-      public double aggregate(final double current, final ObjectColumnSelector[] selectorList)
-      {
-        Context cx = Context.getCurrentContext();
-        if (cx == null) {
-          cx = contextFactory.enterContext();
-
-          // Disable primitive wrapping- we want Java strings and primitives to behave like JS entities.
-          cx.getWrapFactory().setJavaPrimitiveWrap(false);
-        }
-
-        final int size = selectorList.length;
-        final Object[] args = new Object[size + 1];
-
-        args[0] = current;
-        for (int i = 0 ; i < size ; i++) {
-          final ObjectColumnSelector selector = selectorList[i];
-          if (selector != null) {
-            final Object arg = selector.get();
-            if (arg != null && arg.getClass().isArray()) {
-              // Context.javaToJS on an array sort of works, although it returns false for Array.isArray(...) and
-              // may have other issues too. Let's just copy the array and wrap that.
-              final Object[] arrayAsObjectArray = new Object[Array.getLength(arg)];
-              for (int j = 0; j < Array.getLength(arg); j++) {
-                arrayAsObjectArray[j] = Array.get(arg, j);
-              }
-              args[i + 1] = cx.newArray(scope, arrayAsObjectArray);
-            } else {
-              args[i + 1] = Context.javaToJS(arg, scope);
-            }
-          }
-        }
-
-        final Object res = fnAggregate.call(cx, scope, scope, args);
-        return Context.toNumber(res);
-      }
-
-      @Override
-      public double combine(final double a, final double b)
-      {
-        final Object res = contextFactory.call(
-            new ContextAction()
-            {
-              @Override
-              public Object run(final Context cx)
-              {
-                return fnCombine.call(cx, scope, scope, new Object[]{a, b});
-              }
-            }
-        );
-        return Context.toNumber(res);
-      }
-
-      @Override
-      public double reset()
-      {
-        final Object res = contextFactory.call(
-            new ContextAction()
-            {
-              @Override
-              public Object run(final Context cx)
-              {
-                return fnReset.call(cx, scope, scope, new Object[]{});
-              }
-            }
-        );
-        return Context.toNumber(res);
-      }
-
-      @Override
-      public void close()
-      {
-        if (Context.getCurrentContext() != null) {
-          Context.exit();
-        }
-      }
-    };
-  }
-
   @Override
   public boolean equals(Object o)
   {
diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java
index ad723c5..8851af0 100644
--- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java
+++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java
@@ -28,11 +28,11 @@
 public class JavaScriptBufferAggregator implements BufferAggregator
 {
   private final ObjectColumnSelector[] selectorList;
-  private final JavaScriptAggregator.ScriptAggregator script;
+  private final ScriptAggregator script;
 
   public JavaScriptBufferAggregator(
       List<ObjectColumnSelector> selectorList,
-      JavaScriptAggregator.ScriptAggregator script
+      ScriptAggregator script
   )
   {
     this.selectorList = Lists.newArrayList(selectorList).toArray(new ObjectColumnSelector[]{});
diff --git a/processing/src/main/java/io/druid/query/aggregation/Nashorn2ScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/Nashorn2ScriptAggregatorFactory.java
new file mode 100644
index 0000000..9653033
--- /dev/null
+++ b/processing/src/main/java/io/druid/query/aggregation/Nashorn2ScriptAggregatorFactory.java
@@ -0,0 +1,92 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright (C) 2012, 2013, 2014  Metamarkets Group Inc.
+ *
+ * This program is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU General Public License
+ * as published by the Free Software Foundation; either version 2
+ * of the License, or (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with this program; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301, USA.
+ */
+
+package io.druid.query.aggregation;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import io.druid.segment.ObjectColumnSelector;
+
+import javax.script.Invocable;
+import javax.script.ScriptEngine;
+import javax.script.ScriptEngineManager;
+import javax.script.ScriptException;
+
+public class Nashorn2ScriptAggregatorFactory
+{
+  public static interface JSScriptAggregator {
+    public double aggregate(double current, Object... values);
+
+    public double combine(double a, double b);
+
+    public double reset();
+  }
+
+  private String aggregate;
+  private String reset;
+  private String combine;
+
+  public Nashorn2ScriptAggregatorFactory(String aggregate, String reset, String combine)
+  {
+    this.aggregate = aggregate;
+    this.reset = reset;
+    this.combine = combine;
+  }
+
+  public ScriptAggregator compileScript()
+  {
+    Preconditions.checkNotNull(aggregate, "Aggregate script must not be null");
+    Preconditions.checkNotNull(combine, "Combining script must not be null");
+    Preconditions.checkNotNull(reset, "Reset script must not be null");
+
+    ScriptEngine engine = new ScriptEngineManager().getEngineByName("nashorn");
+    final Invocable invocable = (Invocable) engine;
+
+
+    final String script =   "function aggregate(current, selectorList) {"
+                          + "  return scriptAggregate(current, selectorList[0].get());"
+                          + "}"
+                          + ""
+                          + "function combine(a, b) {"
+                          + "  return scriptCombine(a, b);"
+                          + "}"
+                          + ""
+                          + "function reset() {"
+                          + "  return scriptReset();"
+                          + "}"
+                          + ""
+                          + "function close() {}"
+                          + "";
+    try {
+      // this is a little janky, but faster than using `var aggregate = function(...) { ... }`
+      engine.eval(aggregate.replaceFirst("function", "function scriptAggregate"));
+      engine.eval(reset.replaceFirst("function", "function scriptReset"));
+      engine.eval(combine.replaceFirst("function", "function scriptCombine"));
+      engine.eval(script);
+    }
+    catch (ScriptException e) {
+      Throwables.propagate(e);
+    }
+
+    final ScriptAggregator js = invocable.getInterface(ScriptAggregator.class);
+    if(js == null) throw new RuntimeException("Unable to find appropriate js functions");
+
+    return js;
+  }
+}
diff --git a/processing/src/main/java/io/druid/query/aggregation/NashornScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/NashornScriptAggregatorFactory.java
new file mode 100644
index 0000000..85af2ac
--- /dev/null
+++ b/processing/src/main/java/io/druid/query/aggregation/NashornScriptAggregatorFactory.java
@@ -0,0 +1,123 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright (C) 2012, 2013, 2014  Metamarkets Group Inc.
+ *
+ * This program is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU General Public License
+ * as published by the Free Software Foundation; either version 2
+ * of the License, or (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with this program; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301, USA.
+ */
+
+package io.druid.query.aggregation;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import io.druid.segment.ObjectColumnSelector;
+
+import javax.script.Invocable;
+import javax.script.ScriptEngine;
+import javax.script.ScriptEngineManager;
+import javax.script.ScriptException;
+
+public class NashornScriptAggregatorFactory
+{
+  public static interface JSScriptAggregator {
+    public double aggregate(double current, Object... values);
+
+    public double combine(double a, double b);
+
+    public double reset();
+  }
+
+  private String aggregate;
+  private String reset;
+  private String combine;
+
+  public NashornScriptAggregatorFactory(String aggregate, String reset, String combine)
+  {
+    this.aggregate = aggregate;
+    this.reset = reset;
+    this.combine = combine;
+  }
+
+  public ScriptAggregator compileScript()
+  {
+    Preconditions.checkNotNull(aggregate, "Aggregate script must not be null");
+    Preconditions.checkNotNull(combine, "Combining script must not be null");
+    Preconditions.checkNotNull(reset, "Reset script must not be null");
+
+    ScriptEngine engine = new ScriptEngineManager().getEngineByName("nashorn");
+    final Invocable invocable = (Invocable) engine;
+
+    try {
+      // this is a little janky, but seems faster than using `var aggregate = function(...) { ... }`
+      engine.eval(aggregate.replace("function", "function aggregate"));
+      engine.eval(reset.replace("function", "function reset"));
+      engine.eval(combine.replace("function", "function combine"));
+    }
+    catch (ScriptException e) {
+      Throwables.propagate(e);
+    }
+
+    final JSScriptAggregator js = invocable.getInterface(JSScriptAggregator.class);
+    if(js == null) throw new RuntimeException("Unable to find appropriate js functions");
+
+    return new ScriptAggregator()
+    {
+      @Override
+      public double aggregate(final double current, final ObjectColumnSelector[] selectorList)
+      {
+        final int size = selectorList.length;
+        final Object[] args = new Object[size + 1];
+
+        args[0] = current;
+        for (int i = 0; i < size; i++) {
+          final ObjectColumnSelector selector = selectorList[i];
+          if (selector != null) {
+            final Object arg = selector.get();
+//            if (arg != null && arg.getClass().isArray()) {
+//              // Context.javaToJS on an array sort of works, although it returns false for Array.isArray(...) and
+//              // may have other issues too. Let's just copy the array and wrap that.
+//              final Object[] arrayAsObjectArray = new Object[Array.getLength(arg)];
+//              for (int j = 0; j < Array.getLength(arg); j++) {
+//                arrayAsObjectArray[j] = Array.get(arg, j);
+//              }
+//              args[i + 1] = cx.newArray(scope, arrayAsObjectArray);
+//            } else {
+            args[i + 1] = arg;
+//            }
+          }
+        }
+
+        return js.aggregate(current, args);
+      }
+
+      @Override
+      public double combine(final double a, final double b)
+      {
+        return js.combine(a, b);
+      }
+
+      @Override
+      public double reset()
+      {
+        return js.reset();
+      }
+
+      @Override
+      public void close()
+      {
+        // TODO: figure out how to close engine resources
+      }
+    };
+  }
+}
diff --git a/processing/src/main/java/io/druid/query/aggregation/RhinoScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/RhinoScriptAggregatorFactory.java
new file mode 100644
index 0000000..7c9392f
--- /dev/null
+++ b/processing/src/main/java/io/druid/query/aggregation/RhinoScriptAggregatorFactory.java
@@ -0,0 +1,142 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright (C) 2012, 2013, 2014  Metamarkets Group Inc.
+ *
+ * This program is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU General Public License
+ * as published by the Free Software Foundation; either version 2
+ * of the License, or (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with this program; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301, USA.
+ */
+
+package io.druid.query.aggregation;
+
+import com.google.common.base.Preconditions;
+import io.druid.segment.ObjectColumnSelector;
+import org.mozilla.javascript.Context;
+import org.mozilla.javascript.ContextAction;
+import org.mozilla.javascript.ContextFactory;
+import org.mozilla.javascript.Function;
+import org.mozilla.javascript.ScriptableObject;
+
+import java.lang.reflect.Array;
+
+public class RhinoScriptAggregatorFactory
+{
+  private String aggregate;
+  private String reset;
+  private String combine;
+
+  public RhinoScriptAggregatorFactory(String aggregate, String reset, String combine)
+  {
+    this.aggregate = aggregate;
+    this.reset = reset;
+    this.combine = combine;
+  }
+
+  public ScriptAggregator compileScript()
+  {
+    Preconditions.checkNotNull(aggregate, "Aggregate script must not be null");
+    Preconditions.checkNotNull(combine, "Combining script must not be null");
+    Preconditions.checkNotNull(reset, "Reset script must not be null");
+
+    final ContextFactory contextFactory = ContextFactory.getGlobal();
+    Context context = contextFactory.enterContext();
+    context.setOptimizationLevel(9);
+
+    final ScriptableObject scope = context.initStandardObjects();
+
+    final Function fnAggregate = context.compileFunction(scope, aggregate, "aggregate", 1, null);
+    final Function fnReset = context.compileFunction(scope, reset, "reset", 1, null);
+    final Function fnCombine = context.compileFunction(scope, combine, "combine", 1, null);
+    Context.exit();
+
+    return new ScriptAggregator()
+    {
+      @Override
+      public double aggregate(final double current, final ObjectColumnSelector[] selectorList)
+      {
+        Context cx = Context.getCurrentContext();
+        if (cx == null) {
+          cx = contextFactory.enterContext();
+
+          // Disable primitive wrapping- we want Java strings and primitives to behave like JS entities.
+          cx.getWrapFactory().setJavaPrimitiveWrap(false);
+        }
+
+        final int size = selectorList.length;
+        final Object[] args = new Object[size + 1];
+
+        args[0] = current;
+        for (int i = 0 ; i < size ; i++) {
+          final ObjectColumnSelector selector = selectorList[i];
+          if (selector != null) {
+            final Object arg = selector.get();
+            if (arg != null && arg.getClass().isArray()) {
+              // Context.javaToJS on an array sort of works, although it returns false for Array.isArray(...) and
+              // may have other issues too. Let's just copy the array and wrap that.
+              final Object[] arrayAsObjectArray = new Object[Array.getLength(arg)];
+              for (int j = 0; j < Array.getLength(arg); j++) {
+                arrayAsObjectArray[j] = Array.get(arg, j);
+              }
+              args[i + 1] = cx.newArray(scope, arrayAsObjectArray);
+            } else {
+              args[i + 1] = Context.javaToJS(arg, scope);
+            }
+          }
+        }
+
+        final Object res = fnAggregate.call(cx, scope, scope, args);
+        return Context.toNumber(res);
+      }
+
+      @Override
+      public double combine(final double a, final double b)
+      {
+        final Object res = contextFactory.call(
+            new ContextAction()
+            {
+              @Override
+              public Object run(final Context cx)
+              {
+                return fnCombine.call(cx, scope, scope, new Object[]{a, b});
+              }
+            }
+        );
+        return Context.toNumber(res);
+      }
+
+      @Override
+      public double reset()
+      {
+        final Object res = contextFactory.call(
+            new ContextAction()
+            {
+              @Override
+              public Object run(final Context cx)
+              {
+                return fnReset.call(cx, scope, scope, new Object[]{});
+              }
+            }
+        );
+        return Context.toNumber(res);
+      }
+
+      @Override
+      public void close()
+      {
+        if (Context.getCurrentContext() != null) {
+          Context.exit();
+        }
+      }
+    };
+  }
+}
diff --git a/processing/src/main/java/io/druid/query/aggregation/ScriptAggregator.java b/processing/src/main/java/io/druid/query/aggregation/ScriptAggregator.java
new file mode 100644
index 0000000..6a84592
--- /dev/null
+++ b/processing/src/main/java/io/druid/query/aggregation/ScriptAggregator.java
@@ -0,0 +1,33 @@
+/*
+ * Druid - a distributed column store.
+ * Copyright (C) 2012, 2013, 2014  Metamarkets Group Inc.
+ *
+ * This program is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU General Public License
+ * as published by the Free Software Foundation; either version 2
+ * of the License, or (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+ * GNU General Public License for more details.
+ *
+ * You should have received a copy of the GNU General Public License
+ * along with this program; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301, USA.
+ */
+
+package io.druid.query.aggregation;
+
+import io.druid.segment.ObjectColumnSelector;
+
+public interface ScriptAggregator
+{
+  public double aggregate(double current, ObjectColumnSelector[] selectorList);
+
+  public double combine(double a, double b);
+
+  public double reset();
+
+  public void close();
+}
diff --git a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorBenchmark.java b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorBenchmark.java
index 8c9b10a..b5e133f 100644
--- a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorBenchmark.java
+++ b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorBenchmark.java
@@ -29,21 +29,23 @@
 
 public class JavaScriptAggregatorBenchmark extends SimpleBenchmark
 {
-
   protected static final Map<String, String> scriptDoubleSum = Maps.newHashMap();
   static {
-    scriptDoubleSum.put("fnAggregate", "function aggregate(current, a) { return current + a }");
-    scriptDoubleSum.put("fnReset", "function reset() { return 0 }");
-    scriptDoubleSum.put("fnCombine", "function combine(a,b) { return a + b }");
+    scriptDoubleSum.put("fnAggregate", "function(current, a) { return current + a; }");
+    scriptDoubleSum.put("fnReset", "function() { return 0; }");
+    scriptDoubleSum.put("fnCombine", "function(a,b) { return a + b; }");
   }
 
+  public static final int COUNT = 20_000;
+
   private static void aggregate(TestFloatColumnSelector selector, Aggregator agg)
   {
     agg.aggregate();
     selector.increment();
   }
 
-  private JavaScriptAggregator jsAggregator;
+  private JavaScriptAggregator nashornAggregator;
+  private JavaScriptAggregator rhinoAggregator;
   private DoubleSumAggregator doubleAgg;
   final LoopingFloatColumnSelector selector = new LoopingFloatColumnSelector(new float[]{42.12f, 9f});
 
@@ -52,24 +54,47 @@
   {
     Map<String, String> script = scriptDoubleSum;
 
-    jsAggregator = new JavaScriptAggregator(
+    rhinoAggregator = new JavaScriptAggregator(
         "billy",
         Lists.asList(MetricSelectorUtils.wrap(selector), new ObjectColumnSelector[]{}),
-        JavaScriptAggregatorFactory.compileScript(
+        new RhinoScriptAggregatorFactory(
             script.get("fnAggregate"),
             script.get("fnReset"),
             script.get("fnCombine")
-        )
+        ).compileScript()
+    );
+
+    nashornAggregator = new JavaScriptAggregator(
+        "billy",
+        Lists.asList(MetricSelectorUtils.wrap(selector), new ObjectColumnSelector[]{}),
+        new Nashorn2ScriptAggregatorFactory(
+            script.get("fnAggregate"),
+            script.get("fnReset"),
+            script.get("fnCombine")
+        ).compileScript()
     );
 
     doubleAgg = new DoubleSumAggregator("billy", selector);
   }
 
-  public double timeJavaScriptDoubleSum(int reps)
+  public double timeNashornScriptDoubleSum(int reps)
   {
     double val = 0;
     for(int i = 0; i < reps; ++i) {
-      aggregate(selector, jsAggregator);
+      for(int k = 0; k < COUNT; ++k) {
+        aggregate(selector, nashornAggregator);
+      }
+    }
+    return val;
+  }
+
+  public double timeRhinoScriptDoubleSum(int reps)
+  {
+    double val = 0;
+    for(int i = 0; i < reps; ++i) {
+      for(int k = 0; k < COUNT; ++k) {
+        aggregate(selector, rhinoAggregator);
+      }
     }
     return val;
   }
@@ -78,7 +103,9 @@
   {
     double val = 0;
     for(int i = 0; i < reps; ++i) {
-      aggregate(selector, doubleAgg);
+      for(int k = 0; k < COUNT; ++k) {
+        aggregate(selector, doubleAgg);
+      }
     }
     return val;
   }
diff --git a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java
index c509d8b..0872faa 100644
--- a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java
+++ b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java
@@ -36,13 +36,13 @@
   protected static final Map<String, String> scriptDoubleSum = Maps.newHashMap();
 
   static {
-    sumLogATimesBPlusTen.put("fnAggregate", "function aggregate(current, a, b) { return current + (Math.log(a) * b) }");
-    sumLogATimesBPlusTen.put("fnReset", "function reset()                  { return 10 }");
-    sumLogATimesBPlusTen.put("fnCombine", "function combine(a,b)             { return a + b }");
+    sumLogATimesBPlusTen.put("fnAggregate", "function aggregate(current, a, b) { return current + (Math.log(a) * b); }");
+    sumLogATimesBPlusTen.put("fnReset", "function reset()                  { return 10; }");
+    sumLogATimesBPlusTen.put("fnCombine", "function combine(a,b)             { return a + b; }");
 
-    scriptDoubleSum.put("fnAggregate", "function aggregate(current, a) { return current + a }");
-    scriptDoubleSum.put("fnReset", "function reset()               { return 0 }");
-    scriptDoubleSum.put("fnCombine", "function combine(a,b)          { return a + b }");
+    scriptDoubleSum.put("fnAggregate", "function aggregate(current, a) { return current + a; }");
+    scriptDoubleSum.put("fnReset", "function reset()               { return 0; }");
+    scriptDoubleSum.put("fnCombine", "function combine(a,b)          { return a + b; }");
   }
 
   private static void aggregate(TestFloatColumnSelector selector1, TestFloatColumnSelector selector2, Aggregator agg)
@@ -85,9 +85,11 @@
     JavaScriptAggregator agg = new JavaScriptAggregator(
       "billy",
       Arrays.<ObjectColumnSelector>asList(MetricSelectorUtils.wrap(selector1), MetricSelectorUtils.wrap(selector2)),
-      JavaScriptAggregatorFactory.compileScript(script.get("fnAggregate"),
-                                                script.get("fnReset"),
-                                                script.get("fnCombine"))
+      new RhinoScriptAggregatorFactory(
+          script.get("fnAggregate"),
+          script.get("fnReset"),
+          script.get("fnCombine")
+      ).compileScript()
     );
 
     agg.reset();
@@ -121,9 +123,11 @@
     Map<String, String> script = sumLogATimesBPlusTen;
     JavaScriptBufferAggregator agg = new JavaScriptBufferAggregator(
       Arrays.<ObjectColumnSelector>asList(MetricSelectorUtils.wrap(selector1), MetricSelectorUtils.wrap(selector2)),
-      JavaScriptAggregatorFactory.compileScript(script.get("fnAggregate"),
-                                                script.get("fnReset"),
-                                                script.get("fnCombine"))
+      new RhinoScriptAggregatorFactory(
+          script.get("fnAggregate"),
+          script.get("fnReset"),
+          script.get("fnCombine")
+      ).compileScript()
     );
 
     ByteBuffer buf = ByteBuffer.allocateDirect(32);
@@ -156,9 +160,11 @@
     JavaScriptAggregator agg = new JavaScriptAggregator(
         "billy",
         Collections.<ObjectColumnSelector>singletonList(null),
-        JavaScriptAggregatorFactory.compileScript(script.get("fnAggregate"),
-                                                  script.get("fnReset"),
-                                                  script.get("fnCombine"))
+        new RhinoScriptAggregatorFactory(
+            script.get("fnAggregate"),
+            script.get("fnReset"),
+            script.get("fnCombine")
+        ).compileScript()
     );
 
     final double val = 0;
@@ -188,11 +194,11 @@
     final JavaScriptAggregator agg = new JavaScriptAggregator(
         "billy",
         Collections.<ObjectColumnSelector>singletonList(ocs),
-        JavaScriptAggregatorFactory.compileScript(
+        new Nashorn2ScriptAggregatorFactory(
             "function aggregate(current, a) { if (Array.isArray(a)) { return current + a.length; } else if (typeof a === 'string') { return current + 1; } else { return current; } }",
             scriptDoubleSum.get("fnReset"),
             scriptDoubleSum.get("fnCombine")
-        )
+        ).compileScript()
     );
 
     agg.reset();
@@ -246,9 +252,11 @@
     JavaScriptAggregator aggRhino = new JavaScriptAggregator(
       "billy",
       Lists.asList(MetricSelectorUtils.wrap(selector), new ObjectColumnSelector[]{}),
-      JavaScriptAggregatorFactory.compileScript(script.get("fnAggregate"),
-                                                script.get("fnReset"),
-                                                script.get("fnCombine"))
+      new RhinoScriptAggregatorFactory(
+          script.get("fnAggregate"),
+          script.get("fnReset"),
+          script.get("fnCombine")
+      ).compileScript()
     );
 
     DoubleSumAggregator doubleAgg = new DoubleSumAggregator("billy", selector);