mirror of https://github.com/apache/druid.git
JavaScript: Disable now affects worker selection and router strategy too. (#3458)
This commit is contained in:
parent
e0e28866ee
commit
7a2a4bc6de
|
@ -350,7 +350,7 @@ the following properties.
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.javascript.disabled`|Set to "true" to disable JavaScript functionality. This affects the JavaScript parser, filter, extractionFn, aggregator, and post-aggregator.|false|
|
||||
|`druid.javascript.disabled`|Set to "true" to disable JavaScript functionality. This affects the JavaScript parser, filter, extractionFn, aggregator, post-aggregator, router strategy, and worker selection strategy.|false|
|
||||
|
||||
<div class="note info">
|
||||
Please refer to the Druid <a href="../development/javascript.html">JavaScript programming guide</a> for guidelines
|
||||
|
|
|
@ -19,16 +19,18 @@
|
|||
|
||||
package io.druid.indexing.overlord.setup;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.metamx.common.ISE;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.indexing.overlord.ImmutableWorkerInfo;
|
||||
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
|
||||
import io.druid.indexing.overlord.config.WorkerTaskRunnerConfig;
|
||||
import io.druid.js.JavaScriptConfig;
|
||||
|
||||
import javax.script.Compilable;
|
||||
import javax.script.Invocable;
|
||||
|
@ -47,9 +49,17 @@ public class JavaScriptWorkerSelectStrategy implements WorkerSelectStrategy
|
|||
private final String function;
|
||||
|
||||
@JsonCreator
|
||||
public JavaScriptWorkerSelectStrategy(@JsonProperty("function") String fn)
|
||||
public JavaScriptWorkerSelectStrategy(
|
||||
@JsonProperty("function") String fn,
|
||||
@JacksonInject JavaScriptConfig config
|
||||
)
|
||||
{
|
||||
Preconditions.checkNotNull(fn, "function must not be null");
|
||||
|
||||
if (config.isDisabled()) {
|
||||
throw new ISE("JavaScript is disabled");
|
||||
}
|
||||
|
||||
final ScriptEngine engine = new ScriptEngineManager().getEngineByName("javascript");
|
||||
try {
|
||||
((Compilable) engine).compile("var apply = " + fn).eval();
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
|
||||
package io.druid.indexing.overlord.setup;
|
||||
|
||||
import com.fasterxml.jackson.databind.InjectableValues;
|
||||
import com.fasterxml.jackson.databind.JsonMappingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
|
@ -26,15 +28,21 @@ import io.druid.indexing.common.task.Task;
|
|||
import io.druid.indexing.overlord.ImmutableWorkerInfo;
|
||||
import io.druid.indexing.overlord.TestRemoteTaskRunnerConfig;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.js.JavaScriptConfig;
|
||||
import org.easymock.EasyMock;
|
||||
import org.hamcrest.CoreMatchers;
|
||||
import org.joda.time.Period;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
public class JavaScriptWorkerSelectStrategyTest
|
||||
{
|
||||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
final JavaScriptWorkerSelectStrategy strategy = new JavaScriptWorkerSelectStrategy(
|
||||
private final JavaScriptWorkerSelectStrategy STRATEGY = new JavaScriptWorkerSelectStrategy(
|
||||
"function (config, zkWorkers, task) {\n"
|
||||
+ "var batch_workers = new java.util.ArrayList();\n"
|
||||
+ "batch_workers.add(\"10.0.0.1\");\n"
|
||||
|
@ -60,22 +68,50 @@ public class JavaScriptWorkerSelectStrategyTest
|
|||
+ " }\n"
|
||||
+ "}\n"
|
||||
+ "return null;\n"
|
||||
+ "}"
|
||||
+ "}",
|
||||
JavaScriptConfig.getDefault()
|
||||
);
|
||||
|
||||
@Test
|
||||
public void testSerde() throws Exception
|
||||
{
|
||||
ObjectMapper mapper = new DefaultObjectMapper();
|
||||
mapper.setInjectableValues(
|
||||
new InjectableValues.Std().addValue(
|
||||
JavaScriptConfig.class,
|
||||
JavaScriptConfig.getDefault()
|
||||
)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
strategy,
|
||||
STRATEGY,
|
||||
mapper.readValue(
|
||||
mapper.writeValueAsString(strategy),
|
||||
mapper.writeValueAsString(STRATEGY),
|
||||
JavaScriptWorkerSelectStrategy.class
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDisabled() throws Exception
|
||||
{
|
||||
ObjectMapper mapper = new DefaultObjectMapper();
|
||||
mapper.setInjectableValues(
|
||||
new InjectableValues.Std().addValue(
|
||||
JavaScriptConfig.class,
|
||||
new JavaScriptConfig(true)
|
||||
)
|
||||
);
|
||||
|
||||
final String strategyString = mapper.writeValueAsString(STRATEGY);
|
||||
|
||||
expectedException.expect(JsonMappingException.class);
|
||||
expectedException.expectCause(CoreMatchers.<Throwable>instanceOf(IllegalStateException.class));
|
||||
expectedException.expectMessage("JavaScript is disabled");
|
||||
|
||||
mapper.readValue(strategyString, JavaScriptWorkerSelectStrategy.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFindWorkerForTask()
|
||||
{
|
||||
|
@ -86,7 +122,7 @@ public class JavaScriptWorkerSelectStrategyTest
|
|||
"10.0.0.3", worker2
|
||||
);
|
||||
|
||||
ImmutableWorkerInfo workerForBatchTask = strategy.findWorkerForTask(
|
||||
ImmutableWorkerInfo workerForBatchTask = STRATEGY.findWorkerForTask(
|
||||
new TestRemoteTaskRunnerConfig(new Period("PT1S")),
|
||||
workerMap,
|
||||
createMockTask("index_hadoop")
|
||||
|
@ -94,7 +130,7 @@ public class JavaScriptWorkerSelectStrategyTest
|
|||
// batch tasks should be sent to worker1
|
||||
Assert.assertEquals(worker1, workerForBatchTask);
|
||||
|
||||
ImmutableWorkerInfo workerForOtherTask = strategy.findWorkerForTask(
|
||||
ImmutableWorkerInfo workerForOtherTask = STRATEGY.findWorkerForTask(
|
||||
new TestRemoteTaskRunnerConfig(new Period("PT1S")),
|
||||
workerMap,
|
||||
createMockTask("other_type")
|
||||
|
@ -110,7 +146,7 @@ public class JavaScriptWorkerSelectStrategyTest
|
|||
"10.0.0.1", createMockWorker(1, true, true),
|
||||
"10.0.0.2", createMockWorker(1, true, true)
|
||||
);
|
||||
Optional<ImmutableWorkerInfo> workerForOtherTask = strategy.findWorkerForTask(
|
||||
Optional<ImmutableWorkerInfo> workerForOtherTask = STRATEGY.findWorkerForTask(
|
||||
new TestRemoteTaskRunnerConfig(new Period("PT1S")),
|
||||
workerMap,
|
||||
createMockTask("other_type")
|
||||
|
@ -125,14 +161,14 @@ public class JavaScriptWorkerSelectStrategyTest
|
|||
"10.0.0.1", createMockWorker(1, true, false),
|
||||
"10.0.0.4", createMockWorker(1, true, false)
|
||||
);
|
||||
Optional<ImmutableWorkerInfo> workerForBatchTask = strategy.findWorkerForTask(
|
||||
Optional<ImmutableWorkerInfo> workerForBatchTask = STRATEGY.findWorkerForTask(
|
||||
new TestRemoteTaskRunnerConfig(new Period("PT1S")),
|
||||
workerMap,
|
||||
createMockTask("index_hadoop")
|
||||
);
|
||||
Assert.assertFalse(workerForBatchTask.isPresent());
|
||||
|
||||
Optional<ImmutableWorkerInfo> workerForOtherTask = strategy.findWorkerForTask(
|
||||
Optional<ImmutableWorkerInfo> workerForOtherTask = STRATEGY.findWorkerForTask(
|
||||
new TestRemoteTaskRunnerConfig(new Period("PT1S")),
|
||||
workerMap,
|
||||
createMockTask("otherTask")
|
||||
|
@ -148,14 +184,14 @@ public class JavaScriptWorkerSelectStrategyTest
|
|||
"10.0.0.1", createMockWorker(1, false, true),
|
||||
"10.0.0.4", createMockWorker(1, false, true)
|
||||
);
|
||||
Optional<ImmutableWorkerInfo> workerForBatchTask = strategy.findWorkerForTask(
|
||||
Optional<ImmutableWorkerInfo> workerForBatchTask = STRATEGY.findWorkerForTask(
|
||||
new TestRemoteTaskRunnerConfig(new Period("PT1S")),
|
||||
workerMap,
|
||||
createMockTask("index_hadoop")
|
||||
);
|
||||
Assert.assertFalse(workerForBatchTask.isPresent());
|
||||
|
||||
Optional<ImmutableWorkerInfo> workerForOtherTask = strategy.findWorkerForTask(
|
||||
Optional<ImmutableWorkerInfo> workerForOtherTask = STRATEGY.findWorkerForTask(
|
||||
new TestRemoteTaskRunnerConfig(new Period("PT1S")),
|
||||
workerMap,
|
||||
createMockTask("otherTask")
|
||||
|
@ -172,7 +208,7 @@ public class JavaScriptWorkerSelectStrategyTest
|
|||
"10.0.0.1", createMockWorker(1, true, true),
|
||||
"10.0.0.2", createMockWorker(5, true, true)
|
||||
);
|
||||
Optional<ImmutableWorkerInfo> workerForBatchTask = strategy.findWorkerForTask(
|
||||
Optional<ImmutableWorkerInfo> workerForBatchTask = STRATEGY.findWorkerForTask(
|
||||
new TestRemoteTaskRunnerConfig(new Period("PT1S")),
|
||||
workerMap,
|
||||
createMockTask("index_hadoop")
|
||||
|
|
|
@ -19,11 +19,14 @@
|
|||
|
||||
package io.druid.server.router;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.metamx.common.ISE;
|
||||
import io.druid.js.JavaScriptConfig;
|
||||
import io.druid.query.Query;
|
||||
|
||||
import javax.script.Compilable;
|
||||
|
@ -43,10 +46,17 @@ public class JavaScriptTieredBrokerSelectorStrategy implements TieredBrokerSelec
|
|||
private final String function;
|
||||
|
||||
@JsonCreator
|
||||
public JavaScriptTieredBrokerSelectorStrategy(@JsonProperty("function") String fn)
|
||||
public JavaScriptTieredBrokerSelectorStrategy(
|
||||
@JsonProperty("function") String fn,
|
||||
@JacksonInject JavaScriptConfig config
|
||||
)
|
||||
{
|
||||
Preconditions.checkNotNull(fn, "function must not be null");
|
||||
|
||||
if (config.isDisabled()) {
|
||||
throw new ISE("JavaScript is disabled");
|
||||
}
|
||||
|
||||
final ScriptEngine engine = new ScriptEngineManager().getEngineByName("javascript");
|
||||
try {
|
||||
((Compilable)engine).compile("var apply = " + fn).eval();
|
||||
|
|
|
@ -19,40 +19,78 @@
|
|||
|
||||
package io.druid.server.router;
|
||||
|
||||
import com.fasterxml.jackson.databind.InjectableValues;
|
||||
import com.fasterxml.jackson.databind.JsonMappingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.js.JavaScriptConfig;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.query.topn.TopNQueryBuilder;
|
||||
import org.hamcrest.CoreMatchers;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.internal.matchers.ThrowableMessageMatcher;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
import java.util.LinkedHashMap;
|
||||
|
||||
public class JavaScriptTieredBrokerSelectorStrategyTest
|
||||
{
|
||||
final TieredBrokerSelectorStrategy jsStrategy = new JavaScriptTieredBrokerSelectorStrategy(
|
||||
"function (config, query) { if (query.getAggregatorSpecs && query.getDimensionSpec && query.getDimensionSpec().getDimension() == 'bigdim' && query.getAggregatorSpecs().size() >= 3) { var size = config.getTierToBrokerMap().values().size(); if (size > 0) { return config.getTierToBrokerMap().values().toArray()[size-1] } else { return config.getDefaultBrokerServiceName() } } else { return null } }"
|
||||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
private final TieredBrokerSelectorStrategy STRATEGY = new JavaScriptTieredBrokerSelectorStrategy(
|
||||
"function (config, query) { if (query.getAggregatorSpecs && query.getDimensionSpec && query.getDimensionSpec().getDimension() == 'bigdim' && query.getAggregatorSpecs().size() >= 3) { var size = config.getTierToBrokerMap().values().size(); if (size > 0) { return config.getTierToBrokerMap().values().toArray()[size-1] } else { return config.getDefaultBrokerServiceName() } } else { return null } }",
|
||||
JavaScriptConfig.getDefault()
|
||||
);
|
||||
|
||||
@Test
|
||||
public void testSerde() throws Exception
|
||||
{
|
||||
ObjectMapper mapper = new DefaultObjectMapper();
|
||||
mapper.setInjectableValues(
|
||||
new InjectableValues.Std().addValue(
|
||||
JavaScriptConfig.class,
|
||||
JavaScriptConfig.getDefault()
|
||||
)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
jsStrategy,
|
||||
STRATEGY,
|
||||
mapper.readValue(
|
||||
mapper.writeValueAsString(jsStrategy),
|
||||
mapper.writeValueAsString(STRATEGY),
|
||||
JavaScriptTieredBrokerSelectorStrategy.class
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDisabled() throws Exception
|
||||
{
|
||||
ObjectMapper mapper = new DefaultObjectMapper();
|
||||
mapper.setInjectableValues(
|
||||
new InjectableValues.Std().addValue(
|
||||
JavaScriptConfig.class,
|
||||
new JavaScriptConfig(true)
|
||||
)
|
||||
);
|
||||
|
||||
final String strategyString = mapper.writeValueAsString(STRATEGY);
|
||||
|
||||
expectedException.expect(JsonMappingException.class);
|
||||
expectedException.expectCause(CoreMatchers.<Throwable>instanceOf(IllegalStateException.class));
|
||||
expectedException.expectMessage("JavaScript is disabled");
|
||||
|
||||
mapper.readValue(strategyString, JavaScriptTieredBrokerSelectorStrategy.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetBrokerServiceName() throws Exception
|
||||
{
|
||||
|
@ -77,19 +115,19 @@ public class JavaScriptTieredBrokerSelectorStrategyTest
|
|||
};
|
||||
|
||||
final TopNQueryBuilder queryBuilder = new TopNQueryBuilder().dataSource("test")
|
||||
.intervals("2014/2015")
|
||||
.dimension("bigdim")
|
||||
.metric("count")
|
||||
.threshold(1)
|
||||
.aggregators(
|
||||
ImmutableList.<AggregatorFactory>of(
|
||||
new CountAggregatorFactory("count")
|
||||
)
|
||||
);
|
||||
.intervals("2014/2015")
|
||||
.dimension("bigdim")
|
||||
.metric("count")
|
||||
.threshold(1)
|
||||
.aggregators(
|
||||
ImmutableList.<AggregatorFactory>of(
|
||||
new CountAggregatorFactory("count")
|
||||
)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
Optional.absent(),
|
||||
jsStrategy.getBrokerServiceName(
|
||||
STRATEGY.getBrokerServiceName(
|
||||
tieredBrokerConfig,
|
||||
queryBuilder.build()
|
||||
)
|
||||
|
@ -98,7 +136,7 @@ public class JavaScriptTieredBrokerSelectorStrategyTest
|
|||
|
||||
Assert.assertEquals(
|
||||
Optional.absent(),
|
||||
jsStrategy.getBrokerServiceName(
|
||||
STRATEGY.getBrokerServiceName(
|
||||
tieredBrokerConfig,
|
||||
Druids.newTimeBoundaryQueryBuilder().dataSource("test").bound("maxTime").build()
|
||||
)
|
||||
|
@ -106,7 +144,7 @@ public class JavaScriptTieredBrokerSelectorStrategyTest
|
|||
|
||||
Assert.assertEquals(
|
||||
Optional.of("druid/slowBroker"),
|
||||
jsStrategy.getBrokerServiceName(
|
||||
STRATEGY.getBrokerServiceName(
|
||||
tieredBrokerConfig,
|
||||
queryBuilder.aggregators(
|
||||
ImmutableList.of(
|
||||
|
@ -122,7 +160,7 @@ public class JavaScriptTieredBrokerSelectorStrategyTest
|
|||
tierBrokerMap.clear();
|
||||
Assert.assertEquals(
|
||||
Optional.of("druid/broker"),
|
||||
jsStrategy.getBrokerServiceName(
|
||||
STRATEGY.getBrokerServiceName(
|
||||
tieredBrokerConfig,
|
||||
queryBuilder.aggregators(
|
||||
ImmutableList.of(
|
||||
|
|
Loading…
Reference in New Issue