Merge branch 'master' into feature/client_aggs_parsing

This commit is contained in:
Christoph Büscher 2017-05-16 11:54:02 +02:00
commit 059b23e92e
38 changed files with 474 additions and 165 deletions

View File

@ -224,10 +224,6 @@ public class MoreLikeThisQuery extends Query {
return likeText;
}
public void setLikeText(String likeText) {
setLikeText(new String[]{likeText});
}
public void setLikeText(String... likeText) {
this.likeText = likeText;
}
@ -236,7 +232,7 @@ public class MoreLikeThisQuery extends Query {
return likeFields;
}
public void setLikeText(Fields... likeFields) {
public void setLikeFields(Fields... likeFields) {
this.likeFields = likeFields;
}
@ -244,7 +240,7 @@ public class MoreLikeThisQuery extends Query {
setLikeText(likeText.toArray(Strings.EMPTY_ARRAY));
}
public void setUnlikeText(Fields... unlikeFields) {
public void setUnlikeFields(Fields... unlikeFields) {
this.unlikeFields = unlikeFields;
}

View File

@ -399,7 +399,7 @@ public final class ObjectParser<Value, Context> extends AbstractObjectParser<Val
LONG(VALUE_NUMBER, VALUE_STRING),
LONG_OR_NULL(VALUE_NUMBER, VALUE_STRING, VALUE_NULL),
INT(VALUE_NUMBER, VALUE_STRING),
BOOLEAN(VALUE_BOOLEAN),
BOOLEAN(VALUE_BOOLEAN, VALUE_STRING),
STRING_ARRAY(START_ARRAY, VALUE_STRING),
FLOAT_ARRAY(START_ARRAY, VALUE_NUMBER, VALUE_STRING),
DOUBLE_ARRAY(START_ARRAY, VALUE_NUMBER, VALUE_STRING),

View File

@ -178,6 +178,7 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQ
this.index = copy.index;
this.type = copy.type;
this.id = copy.id;
this.routing = copy.routing;
this.doc = copy.doc;
this.xContentType = copy.xContentType;
this.fields = copy.fields;
@ -343,7 +344,7 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQ
/**
* Convert this to a {@link TermVectorsRequest} for fetching the terms of the document.
*/
public TermVectorsRequest toTermVectorsRequest() {
TermVectorsRequest toTermVectorsRequest() {
TermVectorsRequest termVectorsRequest = new TermVectorsRequest(index, type, id)
.selectedFields(fields)
.routing(routing)
@ -1085,14 +1086,14 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQ
// fetching the items with multi-termvectors API
MultiTermVectorsResponse likeItemsResponse = fetchResponse(context.getClient(), likeItems);
// getting the Fields for liked items
mltQuery.setLikeText(getFieldsFor(likeItemsResponse));
mltQuery.setLikeFields(getFieldsFor(likeItemsResponse));
// getting the Fields for unliked items
if (unlikeItems.length > 0) {
MultiTermVectorsResponse unlikeItemsResponse = fetchResponse(context.getClient(), unlikeItems);
org.apache.lucene.index.Fields[] unlikeFields = getFieldsFor(unlikeItemsResponse);
if (unlikeFields.length > 0) {
mltQuery.setUnlikeText(unlikeFields);
mltQuery.setUnlikeFields(unlikeFields);
}
}

View File

@ -105,7 +105,7 @@ public class QueryRewriteContext {
return nowInMillis.getAsLong();
}
public BytesReference getTemplateBytes(Script template) {
public String getTemplateBytes(Script template) {
CompiledTemplate compiledTemplate = scriptService.compileTemplate(template, ScriptContext.Standard.SEARCH);
return compiledTemplate.run(template.getParams());
}

View File

@ -392,7 +392,7 @@ public class QueryShardContext extends QueryRewriteContext {
}
@Override
public final BytesReference getTemplateBytes(Script template) {
public final String getTemplateBytes(Script template) {
failIfFrozen();
return super.getTemplateBytes(template);
}

View File

@ -48,7 +48,7 @@ public class InternalTemplateService implements TemplateService {
return new Template() {
@Override
public String execute(Map<String, Object> model) {
return compiledTemplate.run(model).utf8ToString();
return compiledTemplate.run(model);
}
@Override

View File

@ -64,8 +64,8 @@ public final class ScriptContextRegistry {
/**
* @return <tt>true</tt> if the provided {@link ScriptContext} is supported, <tt>false</tt> otherwise
*/
boolean isSupportedContext(ScriptContext scriptContext) {
return scriptContexts.containsKey(scriptContext.getKey());
boolean isSupportedContext(String scriptContext) {
return scriptContexts.containsKey(scriptContext);
}
//script contexts can be used in fine-grained settings, we need to be careful with what we allow here

View File

@ -19,13 +19,19 @@
package org.elasticsearch.script;
import org.apache.lucene.util.SetOnce;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.function.Function;
/**
* Holds the boolean indicating the enabled mode for each of the different scripting languages available, each script source and each
@ -38,12 +44,55 @@ public class ScriptModes {
final Map<String, Boolean> scriptEnabled;
ScriptModes(ScriptSettings scriptSettings, Settings settings) {
private static final Setting<List<String>> TYPES_ALLOWED_SETTING =
Setting.listSetting("script.types_allowed", Collections.emptyList(), Function.identity(), Setting.Property.NodeScope);
private static final Setting<List<String>> CONTEXTS_ALLOWED_SETTING =
Setting.listSetting("script.contexts_allowed", Collections.emptyList(), Function.identity(), Setting.Property.NodeScope);
private final Set<String> typesAllowed;
private final Set<String> contextsAllowed;
ScriptModes(ScriptContextRegistry scriptContextRegistry, ScriptSettings scriptSettings, Settings settings) {
HashMap<String, Boolean> scriptModes = new HashMap<>();
for (Setting<Boolean> scriptModeSetting : scriptSettings.getScriptLanguageSettings()) {
scriptModes.put(scriptModeSetting.getKey(), scriptModeSetting.get(settings));
}
this.scriptEnabled = Collections.unmodifiableMap(scriptModes);
typesAllowed = TYPES_ALLOWED_SETTING.exists(settings) ? new HashSet<>() : null;
if (typesAllowed != null) {
for (String settingType : TYPES_ALLOWED_SETTING.get(settings)) {
boolean found = false;
for (ScriptType scriptType : ScriptType.values()) {
if (scriptType.getName().equals(settingType)) {
found = true;
typesAllowed.add(settingType);
break;
}
}
if (!found) {
throw new IllegalArgumentException(
"unknown script type [" + settingType + "] found in setting [" + TYPES_ALLOWED_SETTING.getKey() + "].");
}
}
}
contextsAllowed = CONTEXTS_ALLOWED_SETTING.exists(settings) ? new HashSet<>() : null;
if (contextsAllowed != null) {
for (String settingContext : CONTEXTS_ALLOWED_SETTING.get(settings)) {
if (scriptContextRegistry.isSupportedContext(settingContext)) {
contextsAllowed.add(settingContext);
} else {
throw new IllegalArgumentException(
"unknown script context [" + settingContext + "] found in setting [" + CONTEXTS_ALLOWED_SETTING.getKey() + "].");
}
}
}
}
/**
@ -60,6 +109,15 @@ public class ScriptModes {
if (NativeScriptEngine.NAME.equals(lang)) {
return true;
}
if (typesAllowed != null && typesAllowed.contains(scriptType.getName()) == false) {
throw new IllegalArgumentException("[" + scriptType.getName() + "] scripts cannot be executed");
}
if (contextsAllowed != null && contextsAllowed.contains(scriptContext.getKey()) == false) {
throw new IllegalArgumentException("[" + scriptContext.getKey() + "] scripts cannot be executed");
}
Boolean scriptMode = scriptEnabled.get(getKey(lang, scriptType, scriptContext));
if (scriptMode == null) {
throw new IllegalArgumentException("script mode not found for lang [" + lang + "], script_type [" + scriptType + "], operation [" + scriptContext.getKey() + "]");

View File

@ -152,7 +152,7 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust
this.scriptEnginesByLang = unmodifiableMap(enginesByLangBuilder);
this.scriptEnginesByExt = unmodifiableMap(enginesByExtBuilder);
this.scriptModes = new ScriptModes(scriptSettings, settings);
this.scriptModes = new ScriptModes(scriptContextRegistry, scriptSettings, settings);
// add file watcher for static scripts
scriptsDirectory = env.scriptsFile();
@ -325,7 +325,7 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust
/** Compiles a template. Note this will be moved to a separate TemplateService in the future. */
public CompiledTemplate compileTemplate(Script script, ScriptContext scriptContext) {
CompiledScript compiledScript = compile(script, scriptContext);
return params -> (BytesReference)executable(compiledScript, params).run();
return params -> (String)executable(compiledScript, params).run();
}
/**
@ -511,7 +511,7 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust
private boolean canExecuteScript(String lang, ScriptType scriptType, ScriptContext scriptContext) {
assert lang != null;
if (scriptContextRegistry.isSupportedContext(scriptContext) == false) {
if (scriptContextRegistry.isSupportedContext(scriptContext.getKey()) == false) {
throw new IllegalArgumentException("script context [" + scriptContext.getKey() + "] not supported");
}
return scriptModes.getScriptEnabled(lang, scriptType, scriptContext);

View File

@ -35,9 +35,10 @@ import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
public class InternalAdjacencyMatrix
extends InternalMultiBucketAggregation<InternalAdjacencyMatrix,InternalAdjacencyMatrix.InternalBucket>
public class InternalAdjacencyMatrix
extends InternalMultiBucketAggregation<InternalAdjacencyMatrix,InternalAdjacencyMatrix.InternalBucket>
implements AdjacencyMatrix {
public static class InternalBucket extends InternalMultiBucketAggregation.InternalBucket implements AdjacencyMatrix.Bucket {
@ -111,12 +112,31 @@ public class InternalAdjacencyMatrix
builder.endObject();
return builder;
}
@Override
public boolean equals(Object other) {
if (this == other) {
return true;
}
if (other == null || getClass() != other.getClass()) {
return false;
}
InternalBucket that = (InternalBucket) other;
return Objects.equals(key, that.key)
&& Objects.equals(docCount, that.docCount)
&& Objects.equals(aggregations, that.aggregations);
}
@Override
public int hashCode() {
return Objects.hash(getClass(), key, docCount, aggregations);
}
}
private final List<InternalBucket> buckets;
private Map<String, InternalBucket> bucketMap;
public InternalAdjacencyMatrix(String name, List<InternalBucket> buckets,
public InternalAdjacencyMatrix(String name, List<InternalBucket> buckets,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
super(name, pipelineAggregators, metaData);
this.buckets = buckets;
@ -193,13 +213,13 @@ public class InternalAdjacencyMatrix
ArrayList<InternalBucket> reducedBuckets = new ArrayList<>(bucketsMap.size());
for (List<InternalBucket> sameRangeList : bucketsMap.values()) {
InternalBucket reducedBucket = sameRangeList.get(0).reduce(sameRangeList, reduceContext);
if(reducedBucket.docCount >= 1){
if(reducedBucket.docCount >= 1){
reducedBuckets.add(reducedBucket);
}
}
Collections.sort(reducedBuckets, Comparator.comparing(InternalBucket::getKey));
InternalAdjacencyMatrix reduced = new InternalAdjacencyMatrix(name, reducedBuckets, pipelineAggregators(),
InternalAdjacencyMatrix reduced = new InternalAdjacencyMatrix(name, reducedBuckets, pipelineAggregators(),
getMetaData());
return reduced;
@ -215,4 +235,14 @@ public class InternalAdjacencyMatrix
return builder;
}
@Override
protected int doHashCode() {
return Objects.hash(buckets);
}
@Override
protected boolean doEquals(Object obj) {
InternalAdjacencyMatrix that = (InternalAdjacencyMatrix) obj;
return Objects.equals(buckets, that.buckets);
}
}

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.search.lookup;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReaderContext;
@ -26,6 +27,8 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.search.IndexSearcher;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.util.MinimalMap;
import java.io.IOException;
@ -64,7 +67,19 @@ public class LeafIndexLookup extends MinimalMap<String, IndexField> {
// computation is expensive
private int numDeletedDocs = -1;
private boolean deprecationEmitted = false;
private void logDeprecation() {
if (deprecationEmitted == false) {
Logger logger = Loggers.getLogger(getClass());
DeprecationLogger deprecationLogger = new DeprecationLogger(logger);
deprecationLogger.deprecated("Using _index is deprecated. Create a custom ScriptEngine to access index internals.");
deprecationEmitted = true;
}
}
public int numDocs() {
logDeprecation();
if (numDocs == -1) {
numDocs = parentReader.numDocs();
}
@ -72,6 +87,7 @@ public class LeafIndexLookup extends MinimalMap<String, IndexField> {
}
public int maxDoc() {
logDeprecation();
if (maxDoc == -1) {
maxDoc = parentReader.maxDoc();
}
@ -79,6 +95,7 @@ public class LeafIndexLookup extends MinimalMap<String, IndexField> {
}
public int numDeletedDocs() {
logDeprecation();
if (numDeletedDocs == -1) {
numDeletedDocs = parentReader.numDeletedDocs();
}
@ -127,6 +144,7 @@ public class LeafIndexLookup extends MinimalMap<String, IndexField> {
*/
@Override
public IndexField get(Object key) {
logDeprecation();
String stringField = (String) key;
IndexField indexField = indexFields.get(key);
if (indexField == null) {
@ -146,19 +164,23 @@ public class LeafIndexLookup extends MinimalMap<String, IndexField> {
* *
*/
public Fields termVectors() throws IOException {
logDeprecation();
assert reader != null;
return reader.getTermVectors(docId);
}
LeafReader getReader() {
logDeprecation();
return reader;
}
public int getDocId() {
logDeprecation();
return docId;
}
public IndexReader getParentReader() {
logDeprecation();
if (parentReader == null) {
return reader;
}
@ -166,10 +188,12 @@ public class LeafIndexLookup extends MinimalMap<String, IndexField> {
}
public IndexSearcher getIndexSearcher() {
logDeprecation();
return indexSearcher;
}
public IndexReaderContext getReaderContext() {
logDeprecation();
return getParentReader().getContext();
}
}

View File

@ -116,7 +116,7 @@ public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
vars.put(SUGGESTION_TEMPLATE_VAR_NAME, spare.toString());
QueryShardContext shardContext = suggestion.getShardContext();
final ExecutableScript executable = collateScript.apply(vars);
final BytesReference querySource = (BytesReference) executable.run();
final String querySource = (String) executable.run();
try (XContentParser parser = XContentFactory.xContent(querySource).createParser(shardContext.getXContentRegistry(),
querySource)) {
QueryBuilder innerQueryBuilder = shardContext.newParseContext(parser).parseInnerQueryBuilder();

View File

@ -31,5 +31,5 @@ import org.elasticsearch.script.ScriptType;
public interface CompiledTemplate {
/** Run a template and return the resulting string, encoded in utf8 bytes. */
BytesReference run(Map<String, Object> params);
String run(Map<String, Object> params);
}

View File

@ -31,6 +31,7 @@ import java.io.UncheckedIOException;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import static org.hamcrest.Matchers.hasSize;
@ -108,7 +109,6 @@ public class ObjectParserTests extends ESTestCase {
s = objectParser.parse(parser, null);
assertNotNull(s.object);
assertEquals(s.object.test, 0);
}
/**
@ -353,7 +353,11 @@ public class ObjectParserTests extends ESTestCase {
builder.field("string_array_field", "5");
}
boolean nullValue = randomBoolean();
builder.field("boolean_field", nullValue);
if (randomBoolean()) {
builder.field("boolean_field", nullValue);
} else {
builder.field("boolean_field", Boolean.toString(nullValue));
}
builder.field("string_or_null", nullValue ? null : "5");
builder.endObject();
XContentParser parser = createParser(JsonXContent.jsonXContent, builder.string());
@ -424,19 +428,19 @@ public class ObjectParserTests extends ESTestCase {
objectParser.declareStringOrNull(TestStruct::setString_or_null, new ParseField("string_or_null"));
objectParser.declareBoolean(TestStruct::setNull_value, new ParseField("boolean_field"));
TestStruct parse = objectParser.parse(parser, new TestStruct(), null);
assertArrayEquals(parse.double_array_field.toArray(), Arrays.asList(2.1d).toArray());
assertArrayEquals(parse.double_array_field.toArray(), Collections.singletonList(2.1d).toArray());
assertEquals(parse.double_field, 2.1d, 0.0d);
assertArrayEquals(parse.long_array_field.toArray(), Arrays.asList(4L).toArray());
assertArrayEquals(parse.long_array_field.toArray(), Collections.singletonList(4L).toArray());
assertEquals(parse.long_field, 4L);
assertArrayEquals(parse.string_array_field.toArray(), Arrays.asList("5").toArray());
assertArrayEquals(parse.string_array_field.toArray(), Collections.singletonList("5").toArray());
assertEquals(parse.string_field, "5");
assertArrayEquals(parse.int_array_field.toArray(), Arrays.asList(1).toArray());
assertArrayEquals(parse.int_array_field.toArray(), Collections.singletonList(1).toArray());
assertEquals(parse.int_field, 1);
assertArrayEquals(parse.float_array_field.toArray(), Arrays.asList(3.1f).toArray());
assertArrayEquals(parse.float_array_field.toArray(), Collections.singletonList(3.1f).toArray());
assertEquals(parse.float_field, 3.1f, 0.0f);
assertEquals(nullValue, parse.null_value);

View File

@ -23,6 +23,7 @@ import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.memory.MemoryIndex;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Query;
import org.elasticsearch.ElasticsearchException;
@ -61,6 +62,7 @@ import java.util.stream.Stream;
import static org.elasticsearch.index.query.QueryBuilders.moreLikeThisQuery;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.instanceOf;
public class MoreLikeThisQueryBuilderTests extends AbstractQueryTestCase<MoreLikeThisQueryBuilder> {
@ -264,6 +266,13 @@ public class MoreLikeThisQueryBuilderTests extends AbstractQueryTestCase<MoreLik
protected void doAssertLuceneQuery(MoreLikeThisQueryBuilder queryBuilder, Query query, SearchContext context) throws IOException {
if (queryBuilder.likeItems() != null && queryBuilder.likeItems().length > 0) {
assertThat(query, instanceOf(BooleanQuery.class));
BooleanQuery booleanQuery = (BooleanQuery) query;
for (BooleanClause booleanClause : booleanQuery) {
if (booleanClause.getQuery() instanceof MoreLikeThisQuery) {
MoreLikeThisQuery moreLikeThisQuery = (MoreLikeThisQuery) booleanClause.getQuery();
assertThat(moreLikeThisQuery.getLikeFields().length, greaterThan(0));
}
}
} else {
// we rely on integration tests for a deeper check here
assertThat(query, instanceOf(MoreLikeThisQuery.class));
@ -310,6 +319,12 @@ public class MoreLikeThisQueryBuilderTests extends AbstractQueryTestCase<MoreLik
assertEquals(expectedItem, newItem);
}
public void testItemCopy() throws IOException {
Item expectedItem = generateRandomItem();
Item newItem = new Item(expectedItem);
assertEquals(expectedItem, newItem);
}
public void testItemFromXContent() throws IOException {
Item expectedItem = generateRandomItem();
String json = expectedItem.toXContent(XContentFactory.jsonBuilder(), ToXContent.EMPTY_PARAMS).string();

View File

@ -77,6 +77,7 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
import static org.elasticsearch.node.RecoverySettingsChunkSizePlugin.CHUNK_SIZE_SETTING;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
@ -348,39 +349,28 @@ public class IndexRecoveryIT extends ESIntegTestCase {
assertRecoveryState(recoveryStates.get(0), 0, PeerRecoverySource.INSTANCE, true, Stage.DONE, nodeA, nodeB);
validateIndexRecoveryState(recoveryStates.get(0).getIndex());
statsResponse = client().admin().cluster().prepareNodesStats().clear().setIndices(new CommonStatsFlags(CommonStatsFlags.Flag.Recovery)).get();
assertThat(statsResponse.getNodes(), hasSize(2));
for (NodeStats nodeStats : statsResponse.getNodes()) {
Consumer<String> assertNodeHasThrottleTimeAndNoRecoveries = nodeName -> {
NodesStatsResponse nodesStatsResponse = client().admin().cluster().prepareNodesStats().setNodesIds(nodeName)
.clear().setIndices(new CommonStatsFlags(CommonStatsFlags.Flag.Recovery)).get();
assertThat(nodesStatsResponse.getNodes(), hasSize(1));
NodeStats nodeStats = nodesStatsResponse.getNodes().get(0);
final RecoveryStats recoveryStats = nodeStats.getIndices().getRecoveryStats();
assertThat(recoveryStats.currentAsSource(), equalTo(0));
assertThat(recoveryStats.currentAsTarget(), equalTo(0));
if (nodeStats.getNode().getName().equals(nodeA)) {
assertThat("node A throttling should be >0", recoveryStats.throttleTime().millis(), greaterThan(0L));
}
if (nodeStats.getNode().getName().equals(nodeB)) {
assertThat("node B throttling should be >0 ", recoveryStats.throttleTime().millis(), greaterThan(0L));
}
}
assertThat(nodeName + " throttling should be >0", recoveryStats.throttleTime().millis(), greaterThan(0L));
};
// we have to use assertBusy as recovery counters are decremented only when the last reference to the RecoveryTarget
// is decremented, which may happen after the recovery was done.
assertBusy(() -> assertNodeHasThrottleTimeAndNoRecoveries.accept(nodeA));
assertBusy(() -> assertNodeHasThrottleTimeAndNoRecoveries.accept(nodeB));
logger.info("--> bump replica count");
client().admin().indices().prepareUpdateSettings(INDEX_NAME)
.setSettings(Settings.builder().put("number_of_replicas", 1)).execute().actionGet();
ensureGreen();
statsResponse = client().admin().cluster().prepareNodesStats().clear().setIndices(new CommonStatsFlags(CommonStatsFlags.Flag.Recovery)).get();
assertThat(statsResponse.getNodes(), hasSize(2));
for (NodeStats nodeStats : statsResponse.getNodes()) {
final RecoveryStats recoveryStats = nodeStats.getIndices().getRecoveryStats();
assertThat(recoveryStats.currentAsSource(), equalTo(0));
assertThat(recoveryStats.currentAsTarget(), equalTo(0));
if (nodeStats.getNode().getName().equals(nodeA)) {
assertThat("node A throttling should be >0", recoveryStats.throttleTime().millis(), greaterThan(0L));
}
if (nodeStats.getNode().getName().equals(nodeB)) {
assertThat("node B throttling should be >0 ", recoveryStats.throttleTime().millis(), greaterThan(0L));
}
}
assertBusy(() -> assertNodeHasThrottleTimeAndNoRecoveries.accept(nodeA));
assertBusy(() -> assertNodeHasThrottleTimeAndNoRecoveries.accept(nodeB));
logger.info("--> start node C");
String nodeC = internalCluster().startNode();

View File

@ -97,14 +97,14 @@ public class ScriptModesTests extends ESTestCase {
}
public void testDefaultSettings() {
this.scriptModes = new ScriptModes(scriptSettings, Settings.EMPTY);
this.scriptModes = new ScriptModes(scriptContextRegistry, scriptSettings, Settings.EMPTY);
assertScriptModesAllOps(true, ScriptType.FILE);
assertScriptModesAllOps(false, ScriptType.STORED, ScriptType.INLINE);
}
public void testMissingSetting() {
assertAllSettingsWereChecked = false;
this.scriptModes = new ScriptModes(scriptSettings, Settings.EMPTY);
this.scriptModes = new ScriptModes(scriptContextRegistry, scriptSettings, Settings.EMPTY);
try {
scriptModes.getScriptEnabled("non_existing", randomFrom(ScriptType.values()), randomFrom(scriptContexts));
fail("Expected IllegalArgumentException");
@ -131,7 +131,7 @@ public class ScriptModesTests extends ESTestCase {
builder.put("script" + "." + randomScriptTypes[i].getName(), randomScriptModes[i]);
deprecated.add("script" + "." + randomScriptTypes[i].getName());
}
this.scriptModes = new ScriptModes(scriptSettings, builder.build());
this.scriptModes = new ScriptModes(scriptContextRegistry, scriptSettings, builder.build());
for (int i = 0; i < randomInt; i++) {
assertScriptModesAllOps(randomScriptModes[i], randomScriptTypes[i]);
@ -167,7 +167,7 @@ public class ScriptModesTests extends ESTestCase {
builder.put("script" + "." + randomScriptContexts[i].getKey(), randomScriptModes[i]);
deprecated.add("script" + "." + randomScriptContexts[i].getKey());
}
this.scriptModes = new ScriptModes(scriptSettings, builder.build());
this.scriptModes = new ScriptModes(scriptContextRegistry, scriptSettings, builder.build());
for (int i = 0; i < randomInt; i++) {
assertScriptModesAllTypes(randomScriptModes[i], randomScriptContexts[i]);
@ -187,7 +187,7 @@ public class ScriptModesTests extends ESTestCase {
.put("script.stored", "true")
.put("script.inline", "true");
//operations generic settings have precedence over script type generic settings
this.scriptModes = new ScriptModes(scriptSettings, builder.build());
this.scriptModes = new ScriptModes(scriptContextRegistry, scriptSettings, builder.build());
assertScriptModesAllTypes(false, scriptContext);
ScriptContext[] complementOf = complementOf(scriptContext);
assertScriptModes(true, new ScriptType[]{ScriptType.FILE, ScriptType.STORED}, complementOf);

View File

@ -216,6 +216,69 @@ public class ScriptServiceTests extends ESTestCase {
assertThat(compiledScript1.compiled(), sameInstance(compiledScript2.compiled()));
}
public void testAllowAllScriptTypeSettings() throws IOException {
buildScriptService(Settings.EMPTY);
assertCompileAccepted("painless", "script", ScriptType.INLINE, ScriptContext.Standard.SEARCH);
assertCompileAccepted("painless", "script", ScriptType.STORED, ScriptContext.Standard.SEARCH);
}
public void testAllowAllScriptContextSettings() throws IOException {
buildScriptService(Settings.EMPTY);
assertCompileAccepted("painless", "script", ScriptType.INLINE, ScriptContext.Standard.SEARCH);
assertCompileAccepted("painless", "script", ScriptType.INLINE, ScriptContext.Standard.AGGS);
assertCompileAccepted("painless", "script", ScriptType.INLINE, ScriptContext.Standard.UPDATE);
assertCompileAccepted("painless", "script", ScriptType.INLINE, ScriptContext.Standard.INGEST);
}
public void testAllowSomeScriptTypeSettings() throws IOException {
Settings.Builder builder = Settings.builder();
builder.put("script.types_allowed", "inline");
builder.put("script.engine.painless.stored", false);
buildScriptService(builder.build());
assertCompileAccepted("painless", "script", ScriptType.INLINE, ScriptContext.Standard.SEARCH);
assertCompileRejected("painless", "script", ScriptType.STORED, ScriptContext.Standard.SEARCH);
assertSettingDeprecationsAndWarnings(
ScriptSettingsTests.buildDeprecatedSettingsArray(scriptSettings, "script.engine.painless.stored"));
}
public void testAllowSomeScriptContextSettings() throws IOException {
Settings.Builder builder = Settings.builder();
builder.put("script.contexts_allowed", "search, aggs");
builder.put("script.update", false);
buildScriptService(builder.build());
assertCompileAccepted("painless", "script", ScriptType.INLINE, ScriptContext.Standard.SEARCH);
assertCompileAccepted("painless", "script", ScriptType.INLINE, ScriptContext.Standard.AGGS);
assertCompileRejected("painless", "script", ScriptType.INLINE, ScriptContext.Standard.UPDATE);
assertSettingDeprecationsAndWarnings(
ScriptSettingsTests.buildDeprecatedSettingsArray(scriptSettings, "script.update"));
}
public void testAllowNoScriptTypeSettings() throws IOException {
Settings.Builder builder = Settings.builder();
builder.put("script.types_allowed", "");
buildScriptService(builder.build());
assertCompileRejected("painless", "script", ScriptType.INLINE, ScriptContext.Standard.SEARCH);
assertCompileRejected("painless", "script", ScriptType.STORED, ScriptContext.Standard.SEARCH);
}
public void testAllowNoScriptContextSettings() throws IOException {
Settings.Builder builder = Settings.builder();
builder.put("script.contexts_allowed", "");
buildScriptService(builder.build());
assertCompileRejected("painless", "script", ScriptType.INLINE, ScriptContext.Standard.SEARCH);
assertCompileRejected("painless", "script", ScriptType.INLINE, ScriptContext.Standard.AGGS);
assertCompileRejected("painless", "script", ScriptType.INLINE, ScriptContext.Standard.UPDATE);
assertCompileRejected("painless", "script", ScriptType.INLINE, ScriptContext.Standard.INGEST);
}
public void testDefaultBehaviourFineGrainedSettings() throws IOException {
Settings.Builder builder = Settings.builder();
//rarely inject the default settings, which have no effect
@ -345,7 +408,7 @@ public class ScriptServiceTests extends ESTestCase {
do {
pluginName = randomAlphaOfLength(randomIntBetween(1, 10));
unknownContext = randomAlphaOfLength(randomIntBetween(1, 30));
} while(scriptContextRegistry.isSupportedContext(new ScriptContext.Plugin(pluginName, unknownContext)));
} while(scriptContextRegistry.isSupportedContext(new ScriptContext.Plugin(pluginName, unknownContext).getKey()));
String type = scriptEngine.getType();
try {
@ -491,8 +554,8 @@ public class ScriptServiceTests extends ESTestCase {
try {
scriptService.compile(new Script(scriptType, lang, script, Collections.emptyMap()), scriptContext);
fail("compile should have been rejected for lang [" + lang + "], script_type [" + scriptType + "], scripted_op [" + scriptContext + "]");
} catch(IllegalStateException e) {
//all good
} catch (IllegalArgumentException | IllegalStateException e) {
// pass
}
}

View File

@ -0,0 +1,92 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.search.aggregations.bucket.adjacency;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.test.InternalAggregationTestCase;
import org.junit.Before;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
public class InternalAdjacencyMatrixTests extends InternalAggregationTestCase<InternalAdjacencyMatrix> {
private List<String> keys;
@Override
@Before
public void setUp() throws Exception {
super.setUp();
keys = new ArrayList<>();
int numFilters = randomIntBetween(2, 4);
String[] filters = new String[numFilters];
for (int i = 0; i < numFilters; i++) {
filters[i] = randomAlphaOfLength(5);
}
for (int i = 0; i < filters.length; i++) {
keys.add(filters[i]);
for (int j = i + 1; j < filters.length; j++) {
if (filters[i].compareTo(filters[j]) <= 0) {
keys.add(filters[i] + "&" + filters[j]);
} else {
keys.add(filters[j] + "&" + filters[i]);
}
}
}
}
@Override
protected InternalAdjacencyMatrix createTestInstance(String name, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) {
final List<InternalAdjacencyMatrix.InternalBucket> buckets = new ArrayList<>();
for (int i = 0; i < keys.size(); ++i) {
String key = keys.get(i);
int docCount = randomIntBetween(0, 1000);
buckets.add(new InternalAdjacencyMatrix.InternalBucket(key, docCount, InternalAggregations.EMPTY));
}
return new InternalAdjacencyMatrix(name, buckets, pipelineAggregators, metaData);
}
@Override
protected void assertReduced(InternalAdjacencyMatrix reduced, List<InternalAdjacencyMatrix> inputs) {
final Map<String, Long> expectedCounts = new TreeMap<>();
for (InternalAdjacencyMatrix input : inputs) {
for (InternalAdjacencyMatrix.InternalBucket bucket : input.getBuckets()) {
expectedCounts.compute(bucket.getKeyAsString(),
(key, oldValue) -> (oldValue == null ? 0 : oldValue) + bucket.getDocCount());
}
}
final Map<String, Long> actualCounts = new TreeMap<>();
for (InternalAdjacencyMatrix.InternalBucket bucket : reduced.getBuckets()) {
actualCounts.compute(bucket.getKeyAsString(),
(key, oldValue) -> (oldValue == null ? 0 : oldValue) + bucket.getDocCount());
}
assertEquals(expectedCounts, actualCounts);
}
@Override
protected Reader<InternalAdjacencyMatrix> instanceReader() {
return InternalAdjacencyMatrix::new;
}
}

View File

@ -623,4 +623,18 @@ public class MoreLikeThisIT extends ESIntegTestCase {
assertSearchResponse(response);
assertHitCount(response, 1);
}
public void testWithRouting() throws IOException {
client().prepareIndex("index", "type", "1").setRouting("3").setSource("text", "this is a document").get();
client().prepareIndex("index", "type", "2").setRouting("1").setSource("text", "this is another document").get();
client().prepareIndex("index", "type", "3").setRouting("4").setSource("text", "this is yet another document").get();
refresh("index");
Item item = new Item("index", "type", "2").routing("1");
MoreLikeThisQueryBuilder moreLikeThisQueryBuilder = new MoreLikeThisQueryBuilder(new String[]{"text"}, null, new Item[]{item});
moreLikeThisQueryBuilder.minTermFreq(1);
moreLikeThisQueryBuilder.minDocFreq(1);
SearchResponse searchResponse = client().prepareSearch("index").setQuery(moreLikeThisQueryBuilder).get();
assertEquals(2, searchResponse.getHits().totalHits);
}
}

View File

@ -1156,7 +1156,7 @@ public class SuggestSearchIT extends ESIntegTestCase {
@Override
public Object run() {
return new BytesArray(result);
return result;
}
};
}

View File

@ -498,7 +498,7 @@ public class RemoteClusterConnectionTests extends ESTestCase {
barrier.await();
CountDownLatch latch = new CountDownLatch(numConnectionAttempts);
for (int i = 0; i < numConnectionAttempts; i++) {
AtomicReference<RuntimeException> executed = new AtomicReference<>();
AtomicReference<Exception> executed = new AtomicReference<>();
ActionListener<Void> listener = ActionListener.wrap(
x -> {
if (executed.compareAndSet(null, new RuntimeException())) {
@ -508,10 +508,21 @@ public class RemoteClusterConnectionTests extends ESTestCase {
}
},
x -> {
if (executed.compareAndSet(null, new RuntimeException())) {
if (executed.compareAndSet(null, x)) {
latch.countDown();
} else {
throw new AssertionError("shit's been called twice", executed.get());
final String message = x.getMessage();
if ((executed.get().getClass() == x.getClass()
&& "operation was cancelled reason [connect handler is closed]".equals(message)
&& message.equals(executed.get().getMessage())) == false) {
// we do cancel the operation and that means that if timing allows it, the caller
// of a blocking call as well as the handler will get the exception from the
// ExecutionCancelledException concurrently. unless that is the case we fail
// if we get called more than once!
AssertionError assertionError = new AssertionError("shit's been called twice", x);
assertionError.addSuppressed(executed.get());
throw assertionError;
}
}
if (x instanceof RejectedExecutionException || x instanceof AlreadyClosedException
|| x instanceof CancellableThreads.ExecutionCancelledException) {

View File

@ -77,7 +77,6 @@
# log4j 2
-Dlog4j.shutdownHookEnabled=false
-Dlog4j2.disable.jmx=true
-Dlog4j.skipJansi=true
## heap dumps

View File

@ -33,7 +33,7 @@ Response:
--------------------------------------------------
// TESTRESPONSE[s/\.\.\./"took": $body.took,"timed_out": false,"_shards": $body._shards,"hits": $body.hits,/]
The name of the aggregation (`grades_count` above) also serves as the key by which the aggregation result can be
The name of the aggregation (`types_count` above) also serves as the key by which the aggregation result can be
retrieved from the returned response.
==== Script
@ -65,7 +65,7 @@ This will interpret the `script` parameter as an `inline` script with the `painl
POST /sales/_search?size=0
{
"aggs" : {
"grades_count" : {
"types_count" : {
"value_count" : {
"script" : {
"file": "my_script",

View File

@ -12,3 +12,8 @@ elasticsearch 5.0 and have now been removed. Use painless instead.
milliseconds since epoch as a `long`. The same is true for
`doc.some_date_field[some_number]`. Use `doc.some_date_field.value.millis` to
fetch the milliseconds since epoch if you need it.
==== Script Settings
All of the existing scripting security settings have been deprecated. Instead
they are replaced with `script.allowed_types` and `script.allowed_contexts`.

View File

@ -1,6 +1,11 @@
[[breaking_60_settings_changes]]
=== Settings changes
==== Remove support for elasticsearch.json and elasticsearch.yaml configuration file
The configuration file found in the Elasticsearch config directory could previously have
a `.yml`, `.yaml` or `.json` extension. Only `elasticsearch.yml` is now supported.
==== Duplicate keys in configuration file
In previous versions of Elasticsearch, the configuration file was allowed to
@ -66,3 +71,8 @@ and `http.tcp.blocking_server` settings are not recognized anymore.
The `base` similarity is now ignored as coords and query normalization have
been removed. If provided, this setting will be ignored and issue a
deprecation warning.
==== Script Settings
All of the existing scripting security settings have been deprecated. Instead
they are replaced with `script.allowed_types` and `script.allowed_contexts`.

View File

@ -87,6 +87,51 @@ change from the defaults described above. You should be very, very careful
when allowing more than the defaults. Any extra permissions weakens the total
security of the Elasticsearch deployment.
[[allowed-script-types-setting]]
[float]
=== Allowed script types setting
By default all script types are allowed to be executed. This can be modified using the
setting `script.allowed_types`. Only the types specified as part of the setting will be
allowed to be executed.
[source,yaml]
----
script.allowed_types: inline <1>
----
<1> This will allow only inline scripts to be executed but not stored scripts
(or any other types).
[[allowed-script-contexts-setting]]
[float]
=== Allowed script contexts setting
By default all script contexts are allowed to be executed. This can be modified using the
setting `script.allowed_contexts`. Only the contexts specified as part of the setting will
be allowed to be executed.
[source,yaml]
----
script.allowed_contexts: search, update <1>
----
<1> This will allow only search and update scripts to be executed but not
aggs or plugin scripts (or any other contexts).
[[deprecated-script=settings]]
[float]
=== Deprecated script settings
The following settings have all been deprecated and will be removed in 6.0:
* <<security-script-source>>
* <<security-script-context>>
* <<security-script-fine>>
Use the following instead:
* <<allowed-script-types-setting>>
* <<allowed-script-contexts-setting>>
[[security-script-source]]
[float]
=== Script source settings

View File

@ -39,6 +39,7 @@ import org.elasticsearch.script.SearchScript;
import org.elasticsearch.search.lookup.SearchLookup;
import java.io.Reader;
import java.io.StringWriter;
import java.lang.ref.SoftReference;
import java.security.AccessController;
import java.security.PrivilegedAction;
@ -58,21 +59,6 @@ public final class MustacheScriptEngine implements ScriptEngine {
public static final String NAME = "mustache";
/** Thread local UTF8StreamWriter to store template execution results in, thread local to save object creation.*/
private static ThreadLocal<SoftReference<UTF8StreamWriter>> utf8StreamWriter = new ThreadLocal<>();
/** If exists, reset and return, otherwise create, reset and return a writer.*/
private static UTF8StreamWriter utf8StreamWriter() {
SoftReference<UTF8StreamWriter> ref = utf8StreamWriter.get();
UTF8StreamWriter writer = (ref == null) ? null : ref.get();
if (writer == null) {
writer = new UTF8StreamWriter(1024 * 4);
utf8StreamWriter.set(new SoftReference<>(writer));
}
writer.reset();
return writer;
}
/**
* Compile a template string to (in this case) a Mustache object than can
* later be re-used for execution to fill in missing parameter values.
@ -146,8 +132,8 @@ public final class MustacheScriptEngine implements ScriptEngine {
@Override
public Object run() {
final BytesStreamOutput result = new BytesStreamOutput();
try (UTF8StreamWriter writer = utf8StreamWriter().setOutput(result)) {
final StringWriter writer = new StringWriter();
try {
// crazy reflection here
SpecialPermission.check();
AccessController.doPrivileged((PrivilegedAction<Void>) () -> {
@ -158,7 +144,7 @@ public final class MustacheScriptEngine implements ScriptEngine {
logger.error((Supplier<?>) () -> new ParameterizedMessage("Error running {}", template), e);
throw new GeneralScriptException("Error running " + template, e);
}
return result.bytes();
return writer.toString();
}
}

View File

@ -26,6 +26,7 @@ import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
@ -102,11 +103,10 @@ public class TransportSearchTemplateAction extends HandledTransportAction<Search
Script script = new Script(searchTemplateRequest.getScriptType(), TEMPLATE_LANG, searchTemplateRequest.getScript(),
searchTemplateRequest.getScriptParams() == null ? Collections.emptyMap() : searchTemplateRequest.getScriptParams());
CompiledTemplate compiledScript = scriptService.compileTemplate(script, SEARCH);
BytesReference source = compiledScript.run(script.getParams());
response.setSource(source);
String source = compiledScript.run(script.getParams());
response.setSource(new BytesArray(source));
SearchRequest searchRequest = searchTemplateRequest.getRequest();
response.setSource(source);
if (searchTemplateRequest.isSimulate()) {
return null;
}

View File

@ -68,8 +68,7 @@ public class CustomMustacheFactoryTests extends ESTestCase {
CompiledScript compiled = new CompiledScript(INLINE, null, MustacheScriptEngine.NAME, script);
ExecutableScript executable = engine.executable(compiled, singletonMap("value", "a \"value\""));
BytesReference result = (BytesReference) executable.run();
assertThat(result.utf8ToString(), equalTo("{\"field\": \"a \\\"value\\\"\"}"));
assertThat(executable.run(), equalTo("{\"field\": \"a \\\"value\\\"\"}"));
}
public void testDefaultEncoder() {
@ -80,8 +79,7 @@ public class CustomMustacheFactoryTests extends ESTestCase {
CompiledScript compiled = new CompiledScript(INLINE, null, MustacheScriptEngine.NAME, script);
ExecutableScript executable = engine.executable(compiled, singletonMap("value", "a \"value\""));
BytesReference result = (BytesReference) executable.run();
assertThat(result.utf8ToString(), equalTo("{\"field\": \"a \"value\"\"}"));
assertThat(executable.run(), equalTo("{\"field\": \"a \"value\"\"}"));
}
public void testUrlEncoder() {
@ -92,7 +90,6 @@ public class CustomMustacheFactoryTests extends ESTestCase {
CompiledScript compiled = new CompiledScript(INLINE, null, MustacheScriptEngine.NAME, script);
ExecutableScript executable = engine.executable(compiled, singletonMap("value", "tilde~ AND date:[2016 FROM*]"));
BytesReference result = (BytesReference) executable.run();
assertThat(result.utf8ToString(), equalTo("{\"field\": \"tilde%7E+AND+date%3A%5B2016+FROM*%5D\"}"));
assertThat(executable.run(), equalTo("{\"field\": \"tilde%7E+AND+date%3A%5B2016+FROM*%5D\"}"));
}
}

View File

@ -58,11 +58,11 @@ public class MustacheScriptEngineTests extends ESTestCase {
+ "\"negative\": {\"term\": {\"body\": {\"value\": \"solr\"}" + "}}, \"negative_boost\": {{boost_val}} } }}";
Map<String, Object> vars = new HashMap<>();
vars.put("boost_val", "0.3");
BytesReference o = (BytesReference) qe.executable(new CompiledScript(ScriptType.INLINE, "", "mustache",
String o = (String) qe.executable(new CompiledScript(ScriptType.INLINE, "", "mustache",
qe.compile(null, template, compileParams)), vars).run();
assertEquals("GET _search {\"query\": {\"boosting\": {\"positive\": {\"match\": {\"body\": \"gift\"}},"
+ "\"negative\": {\"term\": {\"body\": {\"value\": \"solr\"}}}, \"negative_boost\": 0.3 } }}",
o.utf8ToString());
o);
}
{
String template = "GET _search {\"query\": " + "{\"boosting\": {" + "\"positive\": {\"match\": {\"body\": \"gift\"}},"
@ -70,11 +70,11 @@ public class MustacheScriptEngineTests extends ESTestCase {
Map<String, Object> vars = new HashMap<>();
vars.put("boost_val", "0.3");
vars.put("body_val", "\"quick brown\"");
BytesReference o = (BytesReference) qe.executable(new CompiledScript(ScriptType.INLINE, "", "mustache",
String o = (String) qe.executable(new CompiledScript(ScriptType.INLINE, "", "mustache",
qe.compile(null, template, compileParams)), vars).run();
assertEquals("GET _search {\"query\": {\"boosting\": {\"positive\": {\"match\": {\"body\": \"gift\"}},"
+ "\"negative\": {\"term\": {\"body\": {\"value\": \"\\\"quick brown\\\"\"}}}, \"negative_boost\": 0.3 } }}",
o.utf8ToString());
o);
}
}
@ -89,7 +89,7 @@ public class MustacheScriptEngineTests extends ESTestCase {
CompiledScript compiledScript = new CompiledScript(ScriptType.INLINE, null, "mustache",
qe.compile(null, script.getIdOrCode(), Collections.emptyMap()));
ExecutableScript executableScript = qe.executable(compiledScript, script.getParams());
assertThat(((BytesReference) executableScript.run()).utf8ToString(), equalTo("{\"match_all\":{}}"));
assertThat(executableScript.run(), equalTo("{\"match_all\":{}}"));
}
public void testParseTemplateAsSingleStringWithConditionalClause() throws IOException {
@ -105,7 +105,7 @@ public class MustacheScriptEngineTests extends ESTestCase {
CompiledScript compiledScript = new CompiledScript(ScriptType.INLINE, null, "mustache",
qe.compile(null, script.getIdOrCode(), Collections.emptyMap()));
ExecutableScript executableScript = qe.executable(compiledScript, script.getParams());
assertThat(((BytesReference) executableScript.run()).utf8ToString(), equalTo("{ \"match_all\":{} }"));
assertThat(executableScript.run(), equalTo("{ \"match_all\":{} }"));
}
public void testEscapeJson() throws IOException {

View File

@ -71,7 +71,7 @@ public class MustacheTests extends ESTestCase {
"Mustache templating broken",
"GET _search {\"query\": {\"boosting\": {\"positive\": {\"match\": {\"body\": \"gift\"}},"
+ "\"negative\": {\"term\": {\"body\": {\"value\": \"solr\"}}}, \"negative_boost\": 0.2 } }}",
((BytesReference) result.run()).utf8ToString()
result.run()
);
}
@ -83,22 +83,16 @@ public class MustacheTests extends ESTestCase {
new String[] { "foo", "bar" },
Arrays.asList("foo", "bar"));
vars.put("data", data);
Object output = engine.executable(mustache, vars).run();
assertThat(output, notNullValue());
assertThat(output, instanceOf(BytesReference.class));
BytesReference bytes = (BytesReference) output;
assertThat(bytes.utf8ToString(), equalTo("foo bar"));
assertThat(engine.executable(mustache, vars).run(), equalTo("foo bar"));
// Sets can come out in any order
Set<String> setData = new HashSet<>();
setData.add("foo");
setData.add("bar");
vars.put("data", setData);
output = engine.executable(mustache, vars).run();
assertThat(output, notNullValue());
assertThat(output, instanceOf(BytesReference.class));
bytes = (BytesReference) output;
assertThat(bytes.utf8ToString(), both(containsString("foo")).and(containsString("bar")));
Object output = engine.executable(mustache, vars).run();
assertThat(output, instanceOf(String.class));
assertThat((String)output, both(containsString("foo")).and(containsString("bar")));
}
public void testArrayInArrayAccess() throws Exception {
@ -111,11 +105,7 @@ public class MustacheTests extends ESTestCase {
singleton(new String[] { "foo", "bar" })
);
vars.put("data", data);
Object output = engine.executable(mustache, vars).run();
assertThat(output, notNullValue());
assertThat(output, instanceOf(BytesReference.class));
BytesReference bytes = (BytesReference) output;
assertThat(bytes.utf8ToString(), equalTo("foo bar"));
assertThat(engine.executable(mustache, vars).run(), equalTo("foo bar"));
}
public void testMapInArrayAccess() throws Exception {
@ -126,22 +116,16 @@ public class MustacheTests extends ESTestCase {
new Object[] { singletonMap("key", "foo"), singletonMap("key", "bar") },
Arrays.asList(singletonMap("key", "foo"), singletonMap("key", "bar")));
vars.put("data", data);
Object output = engine.executable(mustache, vars).run();
assertThat(output, notNullValue());
assertThat(output, instanceOf(BytesReference.class));
BytesReference bytes = (BytesReference) output;
assertThat(bytes.utf8ToString(), equalTo("foo bar"));
assertThat(engine.executable(mustache, vars).run(), equalTo("foo bar"));
// HashSet iteration order isn't fixed
Set<Object> setData = new HashSet<>();
setData.add(singletonMap("key", "foo"));
setData.add(singletonMap("key", "bar"));
vars.put("data", setData);
output = engine.executable(mustache, vars).run();
assertThat(output, notNullValue());
assertThat(output, instanceOf(BytesReference.class));
bytes = (BytesReference) output;
assertThat(bytes.utf8ToString(), both(containsString("foo")).and(containsString("bar")));
Object output = engine.executable(mustache, vars).run();
assertThat(output, instanceOf(String.class));
assertThat((String)output, both(containsString("foo")).and(containsString("bar")));
}
@ -156,14 +140,8 @@ public class MustacheTests extends ESTestCase {
data.put("list", randomList);
Map<String, Object> vars = new HashMap<>();
vars.put("data", data);
Object output = engine.executable(mustache, vars).run();
assertThat(output, notNullValue());
assertThat(output, instanceOf(BytesReference.class));
BytesReference bytes = (BytesReference) output;
String expectedString = String.format(Locale.ROOT, "%s %s", randomArrayValues.length, randomList.size());
assertThat(bytes.utf8ToString(), equalTo(expectedString));
assertThat(engine.executable(mustache, vars).run(), equalTo(expectedString));
}
public void testPrimitiveToJSON() throws Exception {
@ -399,9 +377,7 @@ public class MustacheTests extends ESTestCase {
private void assertScript(String script, Map<String, Object> vars, Matcher<Object> matcher) {
Object result = engine.executable(new CompiledScript(INLINE, "inline", "mustache", compile(script)), vars).run();
assertThat(result, notNullValue());
assertThat(result, instanceOf(BytesReference.class));
assertThat(((BytesReference) result).utf8ToString(), matcher);
assertThat(result, matcher);
}
private Object compile(String script) {

View File

@ -19,13 +19,22 @@
apply plugin: 'elasticsearch.vagrant'
dependencies {
// Collect all the plugins
for (Project subproj : project.rootProject.subprojects) {
if (subproj.path.startsWith(':plugins:')) {
List<String> plugins = []
for (Project subproj : project.rootProject.subprojects) {
if (subproj.path.startsWith(':plugins:')) {
// add plugin as a dep
dependencies {
bats project(path: "${subproj.path}", configuration: 'zip')
}
plugins.add(subproj.name)
}
}
plugins = plugins.toSorted()
tasks."vagrantCentos6#packagingTest".onlyIf { false } // fails, see https://github.com/elastic/elasticsearch/issues/24645
setupBats {
doFirst {
File expectedPlugins = file('build/plugins/expected')
expectedPlugins.parentFile.mkdirs()
expectedPlugins.setText(plugins.join('\n'), 'UTF-8')
}
}

View File

@ -173,8 +173,6 @@ install_and_check_plugin() {
# $2 description of the source of the plugin list
compare_plugins_list() {
cat $1 | sort > /tmp/plugins
ls /elasticsearch/plugins/*/build.gradle | cut -d '/' -f 4 |
sort > /tmp/expected
echo "Checking plugins from $2 (<) against expected plugins (>):"
diff /tmp/expected /tmp/plugins
diff -w /elasticsearch/qa/vagrant/build/plugins/expected /tmp/plugins
}

View File

@ -101,18 +101,6 @@ public class BootstrapForTesting {
// initialize paths the same exact way as bootstrap
Permissions perms = new Permissions();
Security.addClasspathPermissions(perms);
// crazy jython
for (URL url : JarHell.parseClassPath()) {
Path path = PathUtils.get(url.toURI());
// crazy jython...
String filename = path.getFileName().toString();
if (filename.contains("jython") && filename.endsWith(".jar")) {
// just enough so it won't fail when it does not exist
perms.add(new FilePermission(path.getParent().toString(), "read,readlink"));
perms.add(new FilePermission(path.getParent().resolve("Lib").toString(), "read,readlink"));
}
}
// java.io.tmpdir
Security.addPath(perms, "java.io.tmpdir", javaTmpDir, "read,readlink,write,delete");
// custom test config file

View File

@ -22,7 +22,6 @@ package org.elasticsearch.script;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Scorer;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.search.lookup.LeafSearchLookup;
import org.elasticsearch.search.lookup.SearchLookup;
@ -132,7 +131,7 @@ public class MockScriptEngine implements ScriptEngine {
if (vars != null) {
context.putAll(vars);
}
return new MockExecutableScript(context, script != null ? script : ctx -> new BytesArray(source));
return new MockExecutableScript(context, script != null ? script : ctx -> source);
}
public SearchScript createSearchScript(Map<String, Object> vars, SearchLookup lookup) {

View File

@ -177,7 +177,6 @@ public abstract class ESTestCase extends LuceneTestCase {
static {
System.setProperty("log4j.shutdownHookEnabled", "false");
System.setProperty("log4j2.disable.jmx", "true");
System.setProperty("log4j.skipJansi", "true"); // jython has this crazy shaded Jansi version that log4j2 tries to load
// shutdown hook so that when the test JVM exits, logging is shutdown too
Runtime.getRuntime().addShutdownHook(new Thread(() -> {