Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

EQL: Introduce support for sequences #56300

Merged
merged 9 commits into from
May 13, 2020
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -7,11 +7,13 @@
package org.elasticsearch.test.eql;

import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;

import org.elasticsearch.Build;
import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.client.EqlClient;
import org.elasticsearch.client.RequestOptions;
import org.elasticsearch.client.RestHighLevelClient;
import org.elasticsearch.client.eql.EqlSearchRequest;
Expand All @@ -32,7 +34,9 @@
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;

import static java.util.stream.Collectors.toList;
import static org.hamcrest.Matchers.instanceOf;

public abstract class CommonEqlActionTestCase extends ESRestTestCase {
Expand Down Expand Up @@ -121,14 +125,15 @@ public void cleanup() throws Exception {

@ParametersFactory(shuffle = false, argumentFormatting = PARAM_FORMATTING)
public static List<Object[]> readTestSpecs() throws Exception {
List<Object[]> testSpecs = new ArrayList<>();

// Load EQL validation specs
List<EqlSpec> specs = EqlSpecLoader.load("/test_queries.toml", true);
specs.addAll(EqlSpecLoader.load("/test_queries_supported.toml", true));
List<EqlSpec> unsupportedSpecs = EqlSpecLoader.load("/test_queries_unsupported.toml", false);

// Validate only currently supported specs
List<EqlSpec> filteredSpecs = new ArrayList<>();

for (EqlSpec spec : specs) {
boolean supported = true;
// Check if spec is supported, simple iteration, cause the list is short.
Expand All @@ -140,18 +145,25 @@ public static List<Object[]> readTestSpecs() throws Exception {
}

if (supported) {
String name = spec.description();
if (Strings.isNullOrEmpty(name)) {
name = spec.note();
}
if (Strings.isNullOrEmpty(name)) {
name = spec.query();
}

testSpecs.add(new Object[]{++counter, name, spec});
filteredSpecs.add(spec);
}
}
return testSpecs;
counter = specs.size();
return asArray(filteredSpecs);
}

public static List<Object[]> asArray(List<EqlSpec> specs) {
AtomicInteger counter = new AtomicInteger();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why AtomicInteger where int will do just fine?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

int doesn't work since the stream pipeline requires a final/non-enclosed object. The AtomicLong is essentially a hack to increment the counter while keeping the same reference.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's a bit of misuse for AtomicInteger only for a Holder functionality but since it's a test it's not important.

return specs.stream().map(spec -> {
String name = spec.description();
if (Strings.isNullOrEmpty(name)) {
name = spec.note();
}
if (Strings.isNullOrEmpty(name)) {
name = spec.query();
}
return new Object[] { counter.incrementAndGet(), name, spec };
}).collect(toList());
}

private final int num;
Expand All @@ -165,9 +177,20 @@ public CommonEqlActionTestCase(int num, String name, EqlSpec spec) {
}

public void test() throws Exception {
EqlSearchRequest request = new EqlSearchRequest(testIndexName, spec.query());
EqlSearchResponse response = highLevelClient().eql().search(request, RequestOptions.DEFAULT);
assertSpec(response.hits().events());
assertResponse(runQuery(testIndexName, spec.query()));
}

protected void assertResponse(EqlSearchResponse response) {
assertSearchHits(response.hits().events());
}

protected EqlSearchResponse runQuery(String index, String query) throws Exception {
EqlSearchRequest request = new EqlSearchRequest(testIndexName, query);
return eqlClient().search(request, RequestOptions.DEFAULT);
}

protected EqlClient eqlClient() {
return highLevelClient().eql();
}

private static long[] extractIds(List<SearchHit> events) {
Expand All @@ -179,7 +202,7 @@ private static long[] extractIds(List<SearchHit> events) {
return ids;
}

private void assertSpec(List<SearchHit> events) {
protected void assertSearchHits(List<SearchHit> events) {
assertNotNull(events);
assertArrayEquals("unexpected result for spec: [" + spec.toString() + "]", spec.expectedEventIds(), extractIds(events));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,3 +118,26 @@ query = "file where 66.0 / serial_event_id == 1"
[[queries]]
expected_event_ids = [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 46]
query = "process where serial_event_id + ((1 + 3) * 2 / (3 - 1)) * 2 == 54 or 70 + serial_event_id < 100"

[[queries]]
query = '''
sequence
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Apparently these are getting dropped by CommonEqlActionTestCase.readTestSpecs.
Any idea why @aleksmaus?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, they still need to be removed from test_queries_unsupported.toml

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

And since these are duplicates of test_queries.toml, you can remove them here too

[process where serial_event_id = 1]
[process where serial_event_id = 2]
'''
expected_event_ids = [1, 2]

[[queries]]
query = '''
sequence
Copy link
Contributor

@rw-access rw-access May 11, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we also add another test where the by is after sequence?

sequence by unique_ppid
  [process where serial_event_id=1]
  [process where true]

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Clearly there needs to be more testing in the future and polishing but that's future work. the PR is already large enough - a current issue with the current dataset is that it relies on a custom tie-breaker something not yet implemented.

[process where serial_event_id=1] by unique_pid
[process where true] by unique_ppid'''
expected_event_ids = [1, 2]

[[queries]]
query = '''
sequence
[process where serial_event_id<3] by unique_pid
[process where true] by unique_ppid
'''
expected_event_ids = [1, 2, 2, 3]
rw-access marked this conversation as resolved.
Show resolved Hide resolved
Original file line number Diff line number Diff line change
Expand Up @@ -13,10 +13,10 @@
import org.elasticsearch.xpack.ql.type.InvalidMappedField;
import org.elasticsearch.xpack.ql.type.UnsupportedEsField;

import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.LinkedHashSet;
import java.util.Objects;
import java.util.Set;

import static java.util.stream.Collectors.toList;

Expand All @@ -32,7 +32,7 @@ static Attribute resolveAgainstList(UnresolvedAttribute u, Collection<Attribute>
}

static Attribute resolveAgainstList(UnresolvedAttribute u, Collection<Attribute> attrList, boolean allowCompound) {
List<Attribute> matches = new ArrayList<>();
Set<Attribute> matches = new LinkedHashSet<>();

// first take into account the qualified version
boolean qualified = u.qualifier() != null;
Expand All @@ -57,7 +57,7 @@ static Attribute resolveAgainstList(UnresolvedAttribute u, Collection<Attribute>
}

if (matches.size() == 1) {
return handleSpecialFields(u, matches.get(0), allowCompound);
return handleSpecialFields(u, matches.iterator().next(), allowCompound);
}

return u.withUnresolvedMessage(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,8 @@
import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
import org.elasticsearch.xpack.ql.rule.RuleExecutor;

import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.LinkedHashSet;

import static java.util.Arrays.asList;
import static org.elasticsearch.xpack.eql.analysis.AnalysisUtils.resolveAgainstList;
Expand Down Expand Up @@ -72,7 +71,7 @@ protected LogicalPlan rule(LogicalPlan plan) {
return plan.transformExpressionsUp(e -> {
if (e instanceof UnresolvedAttribute) {
UnresolvedAttribute u = (UnresolvedAttribute) e;
List<Attribute> childrenOutput = new ArrayList<>();
Collection<Attribute> childrenOutput = new LinkedHashSet<>();
for (LogicalPlan child : plan.children()) {
childrenOutput.addAll(child.output());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,9 @@ public class PreAnalyzer {

public LogicalPlan preAnalyze(LogicalPlan plan, IndexResolution indices) {
if (plan.analyzed() == false) {
final EsRelation esRelation = new EsRelation(plan.source(), indices.get(), false);
// FIXME: includeFrozen needs to be set already
plan = plan.transformUp(r -> new EsRelation(r.source(), indices.get(), false), UnresolvedRelation.class);
plan = plan.transformUp(r -> esRelation, UnresolvedRelation.class);
plan.forEachUp(LogicalPlan::setPreAnalyzed);
}
return plan;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/

package org.elasticsearch.xpack.eql.execution.assembler;

import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.xpack.ql.execution.search.extractor.HitExtractor;

import java.util.List;

public class Criterion {

private final SearchSourceBuilder searchSource;
private final List<HitExtractor> keyExtractors;
private final HitExtractor timestampExtractor;

public Criterion(SearchSourceBuilder searchSource, List<HitExtractor> searchAfterExractors, HitExtractor timestampExtractor) {
this.searchSource = searchSource;
this.keyExtractors = searchAfterExractors;
this.timestampExtractor = timestampExtractor;
}

public SearchSourceBuilder searchSource() {
return searchSource;
}

public List<HitExtractor> keyExtractors() {
return keyExtractors;
}

public HitExtractor timestampExtractor() {
return timestampExtractor;
}

public void fromTimestamp(long timestampMarker) {
// TODO: this is likely to be rewritten afterwards
searchSource.searchAfter(new Object[] { timestampMarker });
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/

package org.elasticsearch.xpack.eql.execution.assembler;

import org.elasticsearch.action.ActionListener;
import org.elasticsearch.xpack.eql.session.Results;

public interface Executable {

void execute(ActionListener<Results> resultsListener);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,122 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/

package org.elasticsearch.xpack.eql.execution.assembler;

import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.tasks.TaskCancelledException;
import org.elasticsearch.xpack.eql.EqlIllegalArgumentException;
import org.elasticsearch.xpack.eql.execution.listener.RuntimeUtils;
import org.elasticsearch.xpack.eql.execution.payload.Payload;
import org.elasticsearch.xpack.eql.execution.payload.SearchResponsePayload;
import org.elasticsearch.xpack.eql.execution.search.Querier;
import org.elasticsearch.xpack.eql.execution.search.SourceGenerator;
import org.elasticsearch.xpack.eql.execution.search.extractor.FieldHitExtractor;
import org.elasticsearch.xpack.eql.execution.search.extractor.TimestampFieldHitExtractor;
import org.elasticsearch.xpack.eql.plan.physical.EsQueryExec;
import org.elasticsearch.xpack.eql.plan.physical.PhysicalPlan;
import org.elasticsearch.xpack.eql.plan.physical.SequenceExec;
import org.elasticsearch.xpack.eql.querydsl.container.FieldExtractorRegistry;
import org.elasticsearch.xpack.eql.querydsl.container.QueryContainer;
import org.elasticsearch.xpack.eql.session.Configuration;
import org.elasticsearch.xpack.eql.session.EqlSession;
import org.elasticsearch.xpack.ql.execution.search.extractor.HitExtractor;
import org.elasticsearch.xpack.ql.expression.Attribute;
import org.elasticsearch.xpack.ql.expression.Expression;
import org.elasticsearch.xpack.ql.util.Check;
import org.elasticsearch.xpack.ql.util.StringUtils;

import java.util.ArrayList;
import java.util.List;

import static org.elasticsearch.action.ActionListener.wrap;
import static org.elasticsearch.xpack.eql.execution.listener.RuntimeUtils.prepareRequest;

public class ExecutionManager implements QueryClient {

private static final Logger log = LogManager.getLogger(Querier.class);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

did you mean ExecutionManager.class here?


private final Configuration cfg;
private final Client client;
private final TimeValue keepAlive;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You don't seem to do anything with keepAlive. I assume it will be added later?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes

private final String indices;

public ExecutionManager(EqlSession eqlSession) {
this.cfg = eqlSession.configuration();
this.client = eqlSession.client();
this.keepAlive = cfg.requestTimeout();
this.indices = cfg.indexAsWildcard();
}

public Executable from(SequenceExec seqExec) {
FieldExtractorRegistry extractorRegistry = new FieldExtractorRegistry();

List<List<Attribute>> listOfKeys = seqExec.keys();
List<PhysicalPlan> plans = seqExec.children();
List<Criterion> criteria = new ArrayList<>(plans.size() - 1);

// build a criterion for each query
for (int i = 0; i < plans.size() - 1; i++) {
List<Attribute> keys = listOfKeys.get(i);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we need to assert the assumption on sizes of the plans and listOfKeys here? not obvious in the scope of this function

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is handled inside LogicalPlan.visitJoin/visitSequence when the keys between the top-level and potentially individual queries are merged in a unified way.

// fields
HitExtractor tsExtractor = timestampExtractor(hitExtractor(seqExec.timestamp(), extractorRegistry));
List<HitExtractor> keyExtractors = hitExtractors(keys, extractorRegistry);

PhysicalPlan query = plans.get(i);
// search query
// TODO: this could be generalized into an exec only query
Check.isTrue(query instanceof EsQueryExec, "Expected a query but got [{}]", query.getClass());
QueryContainer container = ((EsQueryExec) query).queryContainer();
SearchSourceBuilder searchSource = SourceGenerator.sourceBuilder(container, cfg.filter(), cfg.size());

criteria.add(new Criterion(searchSource, keyExtractors, tsExtractor));
}
return new SequenceRuntime(criteria, this);
}

private HitExtractor timestampExtractor(HitExtractor hitExtractor) {
if (hitExtractor instanceof FieldHitExtractor) {
FieldHitExtractor fe = (FieldHitExtractor) hitExtractor;
return (fe instanceof TimestampFieldHitExtractor) ? hitExtractor : new TimestampFieldHitExtractor(fe);
}
throw new EqlIllegalArgumentException("Unexpected extractor [{}]", hitExtractor);
}

private HitExtractor hitExtractor(Expression exp, FieldExtractorRegistry registry) {
return RuntimeUtils.createExtractor(registry.fieldExtraction(exp), cfg);
}

private List<HitExtractor> hitExtractors(List<? extends Expression> exps, FieldExtractorRegistry registry) {
List<HitExtractor> extractors = new ArrayList<>(exps.size());
for (Expression exp : exps) {
extractors.add(hitExtractor(exp, registry));
}
return extractors;
}

@Override
public void query(SearchSourceBuilder searchSource, ActionListener<Payload<SearchHit>> listener) {
// set query timeout
searchSource.timeout(cfg.requestTimeout());

if (log.isTraceEnabled()) {
log.trace("About to execute query {} on {}", StringUtils.toString(searchSource), indices);
}
if (cfg.isCancelled()) {
throw new TaskCancelledException("cancelled");
}

SearchRequest search = prepareRequest(client, searchSource, false, indices);
client.search(search, wrap(sr -> listener.onResponse(new SearchResponsePayload(sr)), listener::onFailure));
}
}
Loading