-
Notifications
You must be signed in to change notification settings - Fork 25k
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
Changes from 1 commit
70d7ce8
9d1ead0
efb8239
3515f88
2b608e4
e6386bf
d3e2671
45250f8
f58a5a8
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Apparently these are getting dropped by CommonEqlActionTestCase.readTestSpecs. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ah, they still need to be removed from There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should we also add another test where the
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 |
---|---|---|
@@ -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); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. did you mean |
||
|
||
private final Configuration cfg; | ||
private final Client client; | ||
private final TimeValue keepAlive; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You don't seem to do anything with There was a problem hiding this comment. Choose a reason for hiding this commentThe 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); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is handled inside |
||
// 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)); | ||
} | ||
} |
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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.There was a problem hiding this comment.
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.