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);