EQL: Introduce filter pipe (#61805)

Allow filtering through a pipe, across events and sequences.
Filter pipes are pushed down to base queries.
For now filtering after limit (head/tail) is forbidden as the
semantics are still up for debate.

Fix #59763

(cherry picked from commit 80569a388b76cecb5f55037fe989c8b6f140761b)
This commit is contained in:
Costin Leau 2020-09-02 15:47:06 +03:00 committed by Costin Leau
parent a0e4331c49
commit e6dc8054a5
22 changed files with 379 additions and 208 deletions

View File

@ -248,16 +248,6 @@ case_insensitive = true
query = 'process where process_name >= "SYSTE" and process_name <= "systex"'
expected_event_ids = [1, 2]
[[queries]]
name = "processWithStringEqualityCaseInsensitive1"
case_insensitive = true
query = '''
process where process_name == "VMACTHLP.exe" and unique_pid == 12
| filter true
'''
expected_event_ids = [12]
[[queries]]
name = "processNameIN"
query = '''

View File

@ -46,35 +46,6 @@ name = "equalsNullHead"
expected_event_ids = [1, 2, 3, 4, 5]
query = 'process where bad_field == null | head 5'
[[queries]]
name = "lteAndGtWithFilter"
tags = ["comparisons", "pipes"]
query = '''
process where serial_event_id <= 8 and serial_event_id > 7
| filter serial_event_id == 8
'''
expected_event_ids = [8]
[[queries]]
name = "filtersLteAndGt"
query = '''
process where true
| filter serial_event_id <= 10
| filter serial_event_id > 6
'''
expected_event_ids = [7, 8, 9, 10]
[[queries]]
name = "filterLteAndGtWithHead"
query = '''
process where true
| filter serial_event_id <= 10
| filter serial_event_id > 6
| head 2
'''
expected_event_ids = [7, 8]
[[queries]]
name = "headWithFiltersAndTail"
query = '''
@ -129,7 +100,6 @@ case_insensitive = true
query = 'process where process_name >= "SYSTE" and process_name <= "systex"'
expected_event_ids = [1, 2]
[[queries]]
name = "processWithStringEqualityCaseInsensitive1"
case_insensitive = true

View File

@ -0,0 +1,74 @@
/*
* 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.analysis;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.xpack.eql.plan.logical.KeyedFilter;
import org.elasticsearch.xpack.eql.plan.logical.LimitWithOffset;
import org.elasticsearch.xpack.eql.session.EqlConfiguration;
import org.elasticsearch.xpack.ql.expression.Literal;
import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
import org.elasticsearch.xpack.ql.plan.logical.Project;
import org.elasticsearch.xpack.ql.tree.NodeUtils;
import org.elasticsearch.xpack.ql.tree.Source;
import org.elasticsearch.xpack.ql.type.DataTypes;
import org.elasticsearch.xpack.ql.util.Holder;
import static java.util.Collections.emptyList;
import static org.elasticsearch.xpack.ql.tree.Source.synthetic;
/**
* Post processor of the user query once it got analyzed and verified.
* The purpose of this class is to add implicit blocks to the query based on the user request
* that help with the query execution not its semantics.
*
* This could have been done in the optimizer however due to its wrapping nature (which is clunky to do with rules)
* and since the optimized is not parameterized, making this a separate step (similar to the pre-analyzer) is more natural.
*/
public class PostAnalyzer {
private static final Logger log = LogManager.getLogger(PostAnalyzer.class);
public LogicalPlan postAnalyze(LogicalPlan plan, EqlConfiguration configuration) {
LogicalPlan initial = plan;
if (plan.analyzed()) {
// implicit limit
// implicit sequence fetch size
// implicit project + fetch size (if needed)
Holder<Boolean> hasJoin = new Holder<>(Boolean.FALSE);
Source projectCtx = synthetic("<implicit-project>");
// first per KeyedFilter
plan = plan.transformUp(k -> {
hasJoin.set(Boolean.TRUE);
Project p = new Project(projectCtx, k.child(), k.extractionAttributes());
// TODO: this could be incorporated into the query generation
LogicalPlan fetchSize = new LimitWithOffset(synthetic("<fetch-size>"),
new Literal(synthetic("<fetch-value>"), configuration.fetchSize(), DataTypes.INTEGER),
p);
return new KeyedFilter(k.source(), fetchSize, k.keys(), k.timestamp(), k.tiebreaker());
}, KeyedFilter.class);
// in case of event queries, filter everything
if (hasJoin.get() == false) {
plan = new Project(projectCtx, plan, emptyList());
}
}
if (log.isTraceEnabled()) {
log.trace("Applied post-analysys\n{}", NodeUtils.diffString(initial, plan));
}
return plan;
}
}

View File

@ -8,6 +8,7 @@ package org.elasticsearch.xpack.eql.analysis;
import org.elasticsearch.xpack.eql.plan.logical.Head;
import org.elasticsearch.xpack.eql.plan.logical.Join;
import org.elasticsearch.xpack.eql.plan.logical.LimitWithOffset;
import org.elasticsearch.xpack.eql.plan.logical.Sequence;
import org.elasticsearch.xpack.eql.plan.logical.Tail;
import org.elasticsearch.xpack.eql.stats.FeatureMetric;
@ -17,10 +18,8 @@ import org.elasticsearch.xpack.ql.common.Failure;
import org.elasticsearch.xpack.ql.expression.Attribute;
import org.elasticsearch.xpack.ql.expression.UnresolvedAttribute;
import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
import org.elasticsearch.xpack.ql.plan.logical.Project;
import org.elasticsearch.xpack.ql.tree.Node;
import org.elasticsearch.xpack.ql.type.DataTypes;
import org.elasticsearch.xpack.ql.util.Holder;
import org.elasticsearch.xpack.ql.util.StringUtils;
import java.util.ArrayList;
@ -34,7 +33,6 @@ import java.util.Set;
import static java.util.stream.Collectors.toMap;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.EVENT;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.JOIN;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.JOIN_UNTIL;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.JOIN_KEYS_FIVE_OR_MORE;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.JOIN_KEYS_FOUR;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.JOIN_KEYS_ONE;
@ -44,15 +42,16 @@ import static org.elasticsearch.xpack.eql.stats.FeatureMetric.JOIN_QUERIES_FIVE_
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.JOIN_QUERIES_FOUR;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.JOIN_QUERIES_THREE;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.JOIN_QUERIES_TWO;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.JOIN_UNTIL;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.PIPE_HEAD;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.PIPE_TAIL;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.SEQUENCE;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.SEQUENCE_UNTIL;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.SEQUENCE_MAXSPAN;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.SEQUENCE_QUERIES_FIVE_OR_MORE;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.SEQUENCE_QUERIES_FOUR;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.SEQUENCE_QUERIES_THREE;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.SEQUENCE_QUERIES_TWO;
import static org.elasticsearch.xpack.eql.stats.FeatureMetric.SEQUENCE_UNTIL;
import static org.elasticsearch.xpack.ql.common.Failure.fail;
/**
@ -140,15 +139,33 @@ public class Verifier {
failures.addAll(localFailures);
});
// Concrete verifications
// if there are no (major) unresolved failures, do more in-depth analysis
if (failures.isEmpty()) {
plan.forEachDown(p -> {
Set<Failure> localFailures = new LinkedHashSet<>();
checkNoPipesAfterLimit(p, localFailures);
failures.addAll(localFailures);
// mark the plan as analyzed
// if everything checks out
if (failures.isEmpty()) {
p.setAnalyzed();
}
});
}
// gather metrics
if (failures.isEmpty()) {
BitSet b = new BitSet(FeatureMetric.values().length);
Holder<Boolean> isLikelyAnEventQuery = new Holder<>(false);
plan.forEachDown(p -> {
if (p instanceof Project) {
isLikelyAnEventQuery.set(true);
} else if (p instanceof Head) {
if (p instanceof Head) {
b.set(PIPE_HEAD.ordinal());
} else if (p instanceof Tail) {
b.set(PIPE_TAIL.ordinal());
@ -212,7 +229,7 @@ public class Verifier {
}
});
if (isLikelyAnEventQuery.get() && b.get(SEQUENCE.ordinal()) == false && b.get(JOIN.ordinal()) == false) {
if (b.get(SEQUENCE.ordinal()) == false && b.get(JOIN.ordinal()) == false) {
b.set(EVENT.ordinal());
}
@ -223,4 +240,12 @@ public class Verifier {
return failures;
}
private void checkNoPipesAfterLimit(LogicalPlan p, Set<Failure> localFailures) {
if ((p instanceof LimitWithOffset) == false) {
if (p.anyMatch(LimitWithOffset.class::isInstance)) {
localFailures.add(fail(p, "Pipe [{}] not allowed (yet) after head/tail", p.sourceText()));
}
}
}
}

View File

@ -9,6 +9,7 @@ package org.elasticsearch.xpack.eql.execution;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.xpack.eql.analysis.PostAnalyzer;
import org.elasticsearch.xpack.eql.analysis.PreAnalyzer;
import org.elasticsearch.xpack.eql.analysis.Verifier;
import org.elasticsearch.xpack.eql.expression.function.EqlFunctionRegistry;
@ -33,6 +34,7 @@ public class PlanExecutor {
private final FunctionRegistry functionRegistry;
private final PreAnalyzer preAnalyzer;
private final PostAnalyzer postAnalyzer;
private final Verifier verifier;
private final Optimizer optimizer;
private final Planner planner;
@ -50,13 +52,14 @@ public class PlanExecutor {
this.metrics = new Metrics();
this.preAnalyzer = new PreAnalyzer();
this.postAnalyzer = new PostAnalyzer();
this.verifier = new Verifier(metrics);
this.optimizer = new Optimizer();
this.planner = new Planner();
}
private EqlSession newSession(EqlConfiguration cfg) {
return new EqlSession(client, cfg, indexResolver, preAnalyzer, functionRegistry, verifier, optimizer, planner, this);
return new EqlSession(client, cfg, indexResolver, preAnalyzer, postAnalyzer, functionRegistry, verifier, optimizer, planner, this);
}
public void eql(EqlConfiguration cfg, String eql, ParserParams parserParams, ActionListener<Results> listener) {

View File

@ -19,6 +19,7 @@ import org.elasticsearch.xpack.ql.expression.Literal;
import org.elasticsearch.xpack.ql.expression.Order;
import org.elasticsearch.xpack.ql.expression.Order.NullsPosition;
import org.elasticsearch.xpack.ql.expression.Order.OrderDirection;
import org.elasticsearch.xpack.ql.expression.predicate.logical.And;
import org.elasticsearch.xpack.ql.expression.predicate.logical.Not;
import org.elasticsearch.xpack.ql.expression.predicate.nulls.IsNotNull;
import org.elasticsearch.xpack.ql.expression.predicate.nulls.IsNull;
@ -38,6 +39,7 @@ import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.ReplaceSurrogateFunct
import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.SetAsOptimized;
import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.TransformDirection;
import org.elasticsearch.xpack.ql.plan.logical.Filter;
import org.elasticsearch.xpack.ql.plan.logical.LeafPlan;
import org.elasticsearch.xpack.ql.plan.logical.Limit;
import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
import org.elasticsearch.xpack.ql.plan.logical.OrderBy;
@ -51,6 +53,7 @@ import java.util.Arrays;
import java.util.List;
import static java.util.Collections.singletonList;
import static java.util.stream.Collectors.toList;
public class Optimizer extends RuleExecutor<LogicalPlan> {
@ -60,7 +63,8 @@ public class Optimizer extends RuleExecutor<LogicalPlan> {
@Override
protected Iterable<RuleExecutor<LogicalPlan>.Batch> batches() {
Batch substitutions = new Batch("Operator Replacement", Limiter.ONCE, new ReplaceSurrogateFunction());
Batch substitutions = new Batch("Substitution", Limiter.ONCE,
new ReplaceSurrogateFunction());
Batch operators = new Batch("Operator Optimization",
new ConstantFolding(),
@ -76,7 +80,8 @@ public class Optimizer extends RuleExecutor<LogicalPlan> {
// prune/elimination
new PruneFilters(),
new PruneLiteralsInOrderBy(),
new CombineLimits());
new CombineLimits(),
new PushDownFilterPipe());
Batch ordering = new Batch("Implicit Order",
new SortByLimit(),
@ -92,8 +97,7 @@ public class Optimizer extends RuleExecutor<LogicalPlan> {
return Arrays.asList(substitutions, operators, ordering, local, label);
}
private static class ReplaceWildcards extends OptimizerRule<Filter> {
private static boolean isWildcard(Expression expr) {
@ -237,6 +241,42 @@ public class Optimizer extends RuleExecutor<LogicalPlan> {
}
}
/**
* Push down filter pipes.
*/
static class PushDownFilterPipe extends OptimizerRule<Filter> {
@Override
protected LogicalPlan rule(Filter filter) {
LogicalPlan child = filter.child();
// can't push it down further
if (child instanceof LeafPlan) {
return filter;
}
// combine filters if possible
if (child instanceof Filter) {
Filter f = (Filter) child;
return new Filter(f.source(), f.child(), new And(filter.source(), f.condition(), filter.condition()));
}
// treat Join separately to avoid pushing the filter on until
if (child instanceof Join) {
Join j = (Join) child;
return j.with(j.queries().stream()
.map(q -> {
Filter f = new Filter(filter.source(), q.child(), filter.condition());
return new KeyedFilter(q.source(), f, q.keys(), q.timestamp(), q.tiebreaker());
})
.collect(toList()), j.until(), j.direction());
}
// otherwise keep pushing it down
return child.replaceChildren(child.children().stream()
.map(c -> new Filter(filter.source(), c, filter.condition()))
.collect(toList()));
}
}
/**
* Align the implicit order with the limit (head means ASC or tail means DESC).
*/

View File

@ -42,7 +42,6 @@ import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equal
import org.elasticsearch.xpack.ql.plan.logical.Filter;
import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
import org.elasticsearch.xpack.ql.plan.logical.OrderBy;
import org.elasticsearch.xpack.ql.plan.logical.Project;
import org.elasticsearch.xpack.ql.plan.logical.UnresolvedRelation;
import org.elasticsearch.xpack.ql.tree.Source;
import org.elasticsearch.xpack.ql.type.DataTypes;
@ -54,14 +53,16 @@ import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import static java.util.Arrays.asList;
import static java.util.Collections.emptyList;
import static java.util.Collections.singletonList;
import static org.elasticsearch.xpack.ql.tree.Source.synthetic;
public abstract class LogicalPlanBuilder extends ExpressionBuilder {
private static final Set<String> SUPPORTED_PIPES = Sets.newHashSet("count", "filter", "head", "sort", "tail", "unique", "unique_count");
private static final String FILTER_PIPE = "filter", HEAD_PIPE = "head", TAIL_PIPE = "tail";
private static final Set<String> SUPPORTED_PIPES = Sets.newHashSet("count", FILTER_PIPE, HEAD_PIPE, "sort", TAIL_PIPE, "unique",
"unique_count");
private final UnresolvedRelation RELATION = new UnresolvedRelation(synthetic("<relation>"), null, "", false, "");
private final EmptyAttribute UNSPECIFIED_FIELD = new EmptyAttribute(synthetic("<unspecified>"));
@ -87,7 +88,12 @@ public abstract class LogicalPlanBuilder extends ExpressionBuilder {
public Object visitStatement(StatementContext ctx) {
LogicalPlan plan = plan(ctx.query());
//
// Add implicit blocks
//
// the first pipe will be the implicit order
// declared here for resolving any possible tie-breakers
boolean asc = defaultDirection() == OrderDirection.ASC;
NullsPosition position = asc ? NullsPosition.FIRST : NullsPosition.LAST;
@ -136,7 +142,7 @@ public abstract class LogicalPlanBuilder extends ExpressionBuilder {
@Override
public LogicalPlan visitEventQuery(EqlBaseParser.EventQueryContext ctx) {
return new Project(source(ctx), visitEventFilter(ctx.eventFilter()), defaultProjection());
return visitEventFilter(ctx.eventFilter());
}
@Override
@ -208,22 +214,7 @@ public abstract class LogicalPlanBuilder extends ExpressionBuilder {
private KeyedFilter keyedFilter(List<Attribute> joinKeys, ParseTree ctx, JoinKeysContext joinCtx, SubqueryContext subqueryCtx) {
List<Attribute> keys = CollectionUtils.combine(joinKeys, visitJoinKeys(joinCtx));
LogicalPlan eventQuery = visitEventFilter(subqueryCtx.eventFilter());
// add fetch size as a limit so it gets propagated into the resulting query
LogicalPlan fetchSize = new LimitWithOffset(synthetic("<fetch-size>"),
new Literal(synthetic("<fetch-value>"), params.fetchSize(), DataTypes.INTEGER),
eventQuery);
// filter fields
LogicalPlan child = new Project(source(ctx), fetchSize, CollectionUtils.combine(keys, defaultProjection()));
return new KeyedFilter(source(ctx), child, keys, fieldTimestamp(), fieldTiebreaker());
}
private List<Attribute> defaultProjection() {
Attribute fieldTieBreaker = fieldTiebreaker();
if (Expressions.isPresent(fieldTieBreaker)) {
return asList(fieldTimestamp(), fieldTiebreaker());
}
return singletonList(fieldTimestamp());
return new KeyedFilter(source(ctx), eventQuery, keys, fieldTimestamp(), fieldTiebreaker());
}
@Override
@ -347,11 +338,14 @@ public abstract class LogicalPlanBuilder extends ExpressionBuilder {
}
switch (name) {
case "head":
case FILTER_PIPE:
return new Filter(source(ctx), plan, onlyOnePipeArgument(source(ctx), name, ctx.booleanExpression()));
case HEAD_PIPE:
Expression headLimit = pipeIntArgument(source(ctx), name, ctx.booleanExpression());
return new Head(source(ctx), headLimit, plan);
case "tail":
case TAIL_PIPE:
Expression tailLimit = pipeIntArgument(source(ctx), name, ctx.booleanExpression());
// negate the limit
return new Tail(source(ctx), tailLimit, plan);
@ -361,16 +355,19 @@ public abstract class LogicalPlanBuilder extends ExpressionBuilder {
}
}
private Expression pipeIntArgument(Source source, String pipeName, List<BooleanExpressionContext> exps) {
private Expression onlyOnePipeArgument(Source source, String pipeName, List<BooleanExpressionContext> exps) {
int size = CollectionUtils.isEmpty(exps) ? 0 : exps.size();
if (size != 1) {
throw new ParsingException(source, "Pipe [{}] expects exactly one argument but found [{}]", pipeName, size);
}
BooleanExpressionContext limitCtx = exps.get(0);
Expression expression = expression(limitCtx);
return expression(exps.get(0));
}
private Expression pipeIntArgument(Source source, String pipeName, List<BooleanExpressionContext> exps) {
Expression expression = onlyOnePipeArgument(source, pipeName, exps);
if (expression.dataType().isInteger() == false || expression.foldable() == false || (int) expression.fold() < 0) {
throw new ParsingException(source(limitCtx), "Pipe [{}] expects a positive integer but found [{}]", pipeName, expression
throw new ParsingException(expression.source(), "Pipe [{}] expects a positive integer but found [{}]", pipeName, expression
.sourceText());
}

View File

@ -8,12 +8,14 @@ package org.elasticsearch.xpack.eql.plan.logical;
import org.elasticsearch.xpack.ql.capabilities.Resolvables;
import org.elasticsearch.xpack.ql.expression.Attribute;
import org.elasticsearch.xpack.ql.expression.Expressions;
import org.elasticsearch.xpack.ql.expression.NamedExpression;
import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan;
import org.elasticsearch.xpack.ql.tree.NodeInfo;
import org.elasticsearch.xpack.ql.tree.Source;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
@ -56,6 +58,18 @@ public class KeyedFilter extends UnaryPlan {
return tiebreaker;
}
public List<? extends NamedExpression> extractionAttributes() {
List<NamedExpression> out = new ArrayList<>();
out.add(timestamp);
if (Expressions.isPresent(tiebreaker)) {
out.add(tiebreaker);
}
out.addAll(keys);
return out;
}
@Override
public boolean expressionsResolved() {
return Resolvables.resolved(keys) && timestamp.resolved() && tiebreaker.resolved();

View File

@ -119,7 +119,7 @@ public class TransportEqlSearchAction extends HandledTransportAction<EqlSearchRe
.fetchSize(request.fetchSize());
EqlConfiguration cfg = new EqlConfiguration(request.indices(), zoneId, username, clusterName, filter, timeout, includeFrozen,
request.isCaseSensitive(), clientId, new TaskId(nodeId, task.getId()), task);
request.isCaseSensitive(), request.fetchSize(), clientId, new TaskId(nodeId, task.getId()), task);
planExecutor.eql(cfg, request.query(), params, wrap(r -> listener.onResponse(createResponse(r, task.getExecutionId())),
listener::onFailure));
}

View File

@ -24,12 +24,14 @@ public class EqlConfiguration extends org.elasticsearch.xpack.ql.session.Configu
private final TaskId taskId;
private final EqlSearchTask task;
private final boolean isCaseSensitive;
private final int fetchSize;
@Nullable
private final QueryBuilder filter;
public EqlConfiguration(String[] indices, ZoneId zi, String username, String clusterName, QueryBuilder filter, TimeValue requestTimeout,
boolean includeFrozen, boolean isCaseSensitive, String clientId, TaskId taskId, EqlSearchTask task) {
boolean includeFrozen, boolean isCaseSensitive, int fetchSize, String clientId, TaskId taskId,
EqlSearchTask task) {
super(zi, username, clusterName);
this.indices = indices;
@ -40,6 +42,7 @@ public class EqlConfiguration extends org.elasticsearch.xpack.ql.session.Configu
this.isCaseSensitive = isCaseSensitive;
this.taskId = taskId;
this.task = task;
this.fetchSize = fetchSize;
}
public String[] indices() {
@ -54,6 +57,10 @@ public class EqlConfiguration extends org.elasticsearch.xpack.ql.session.Configu
return requestTimeout;
}
public int fetchSize() {
return fetchSize;
}
public QueryBuilder filter() {
return filter;
}

View File

@ -11,6 +11,7 @@ import org.elasticsearch.client.Client;
import org.elasticsearch.client.ParentTaskAssigningClient;
import org.elasticsearch.tasks.TaskCancelledException;
import org.elasticsearch.xpack.eql.analysis.Analyzer;
import org.elasticsearch.xpack.eql.analysis.PostAnalyzer;
import org.elasticsearch.xpack.eql.analysis.PreAnalyzer;
import org.elasticsearch.xpack.eql.analysis.Verifier;
import org.elasticsearch.xpack.eql.execution.PlanExecutor;
@ -32,17 +33,20 @@ public class EqlSession {
private final IndexResolver indexResolver;
private final PreAnalyzer preAnalyzer;
private final PostAnalyzer postAnalyzer;
private final Analyzer analyzer;
private final Optimizer optimizer;
private final Planner planner;
public EqlSession(Client client, EqlConfiguration cfg, IndexResolver indexResolver, PreAnalyzer preAnalyzer,
FunctionRegistry functionRegistry, Verifier verifier, Optimizer optimizer, Planner planner, PlanExecutor planExecutor) {
public EqlSession(Client client, EqlConfiguration cfg, IndexResolver indexResolver, PreAnalyzer preAnalyzer, PostAnalyzer postAnalyzer,
FunctionRegistry functionRegistry, Verifier verifier, Optimizer optimizer, Planner planner,
PlanExecutor planExecutor) {
this.client = new ParentTaskAssigningClient(client, cfg.getTaskId());
this.configuration = cfg;
this.indexResolver = indexResolver;
this.preAnalyzer = preAnalyzer;
this.postAnalyzer = postAnalyzer;
this.analyzer = new Analyzer(cfg, functionRegistry, verifier);
this.optimizer = optimizer;
this.planner = planner;
@ -87,7 +91,7 @@ public class EqlSession {
return;
}
preAnalyze(parsed, wrap(p -> listener.onResponse(analyzer.analyze(p)), listener::onFailure));
preAnalyze(parsed, wrap(p -> listener.onResponse(postAnalyze(analyzer.analyze(p))), listener::onFailure));
}
private <T> void preAnalyze(LogicalPlan parsed, ActionListener<LogicalPlan> listener) {
@ -101,6 +105,10 @@ public class EqlSession {
}, listener::onFailure));
}
private LogicalPlan postAnalyze(LogicalPlan verified) {
return postAnalyzer.postAnalyze(verified, configuration);
}
private LogicalPlan doParse(String eql, ParserParams params) {
return new EqlParser().createStatement(eql, params);
}

View File

@ -17,6 +17,7 @@ import java.util.Collections;
import static org.elasticsearch.test.ESTestCase.randomAlphaOfLength;
import static org.elasticsearch.test.ESTestCase.randomBoolean;
import static org.elasticsearch.test.ESTestCase.randomIntBetween;
import static org.elasticsearch.test.ESTestCase.randomLong;
import static org.elasticsearch.test.ESTestCase.randomNonNegativeLong;
import static org.elasticsearch.test.ESTestCase.randomZone;
@ -27,12 +28,12 @@ public final class EqlTestUtils {
}
public static final EqlConfiguration TEST_CFG_CASE_INSENSITIVE = new EqlConfiguration(new String[] {"none"},
org.elasticsearch.xpack.ql.util.DateUtils.UTC, "nobody", "cluster", null, TimeValue.timeValueSeconds(30), false, false,
"", new TaskId("test", 123), null);
org.elasticsearch.xpack.ql.util.DateUtils.UTC, "nobody", "cluster", null, TimeValue.timeValueSeconds(30), false, false,
123, "", new TaskId("test", 123), null);
public static final EqlConfiguration TEST_CFG_CASE_SENSITIVE = new EqlConfiguration(new String[] {"none"},
org.elasticsearch.xpack.ql.util.DateUtils.UTC, "nobody", "cluster", null, TimeValue.timeValueSeconds(30), false, true,
"", new TaskId("test", 123), null);
org.elasticsearch.xpack.ql.util.DateUtils.UTC, "nobody", "cluster", null, TimeValue.timeValueSeconds(30), false, true,
123, "", new TaskId("test", 123), null);
public static EqlConfiguration randomConfiguration() {
return internalRandomConfiguration(randomBoolean());
@ -51,6 +52,7 @@ public final class EqlTestUtils {
new TimeValue(randomNonNegativeLong()),
randomBoolean(),
isCaseSensitive,
randomIntBetween(1, 1000),
randomAlphaOfLength(16),
new TaskId(randomAlphaOfLength(10), randomNonNegativeLong()),
randomTask());

View File

@ -332,4 +332,8 @@ public class VerifierTests extends ESTestCase {
"define one or use MATCH/QUERY instead",
error(idxr, "process where string(multi_field.english) == 'foo'"));
}
}
public void testLimitWithFilter() {
assertEquals("1:30: Pipe [| filter false] not allowed (yet) after head/tail", error("process where true | head 10 | filter false"));
}
}

View File

@ -9,6 +9,7 @@ package org.elasticsearch.xpack.eql.optimizer;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.eql.analysis.Analyzer;
import org.elasticsearch.xpack.eql.analysis.PostAnalyzer;
import org.elasticsearch.xpack.eql.analysis.PreAnalyzer;
import org.elasticsearch.xpack.eql.analysis.Verifier;
import org.elasticsearch.xpack.eql.expression.function.EqlFunctionRegistry;
@ -21,6 +22,7 @@ import org.elasticsearch.xpack.eql.plan.physical.LocalRelation;
import org.elasticsearch.xpack.eql.stats.Metrics;
import org.elasticsearch.xpack.ql.expression.Attribute;
import org.elasticsearch.xpack.ql.expression.EmptyAttribute;
import org.elasticsearch.xpack.ql.expression.Expression;
import org.elasticsearch.xpack.ql.expression.FieldAttribute;
import org.elasticsearch.xpack.ql.expression.Literal;
import org.elasticsearch.xpack.ql.expression.Order;
@ -30,6 +32,7 @@ import org.elasticsearch.xpack.ql.expression.predicate.logical.And;
import org.elasticsearch.xpack.ql.expression.predicate.logical.Not;
import org.elasticsearch.xpack.ql.expression.predicate.nulls.IsNotNull;
import org.elasticsearch.xpack.ql.expression.predicate.nulls.IsNull;
import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals;
import org.elasticsearch.xpack.ql.expression.predicate.regex.Like;
import org.elasticsearch.xpack.ql.index.EsIndex;
import org.elasticsearch.xpack.ql.index.IndexResolution;
@ -73,8 +76,10 @@ public class OptimizerTests extends ESTestCase {
private LogicalPlan accept(IndexResolution resolution, String eql) {
PreAnalyzer preAnalyzer = new PreAnalyzer();
PostAnalyzer postAnalyzer = new PostAnalyzer();
Analyzer analyzer = new Analyzer(TEST_CFG_CASE_INSENSITIVE, new EqlFunctionRegistry(), new Verifier(new Metrics()));
return optimizer.optimize(analyzer.analyze(preAnalyzer.preAnalyze(parser.createStatement(eql), resolution)));
return optimizer.optimize(postAnalyzer.postAnalyze(analyzer.analyze(preAnalyzer.preAnalyze(parser.createStatement(eql),
resolution)), TEST_CFG_CASE_INSENSITIVE));
}
private LogicalPlan accept(String eql) {
@ -89,9 +94,6 @@ public class OptimizerTests extends ESTestCase {
for (String q : tests) {
LogicalPlan plan = defaultPipes(accept(q));
assertTrue(plan instanceof Project);
plan = ((Project) plan).child();
assertTrue(plan instanceof Filter);
Filter filter = (Filter) plan;
@ -110,8 +112,6 @@ public class OptimizerTests extends ESTestCase {
for (String q : tests) {
LogicalPlan plan = defaultPipes(accept(q));
assertTrue(plan instanceof Project);
plan = ((Project) plan).child();
assertTrue(plan instanceof Filter);
Filter filter = (Filter) plan;
@ -131,8 +131,6 @@ public class OptimizerTests extends ESTestCase {
for (String q : tests) {
LogicalPlan plan = defaultPipes(accept(q));
assertTrue(plan instanceof Project);
plan = ((Project) plan).child();
assertTrue(plan instanceof Filter);
Filter filter = (Filter) plan;
@ -155,8 +153,6 @@ public class OptimizerTests extends ESTestCase {
for (String q : tests) {
LogicalPlan plan = defaultPipes(accept(q));
assertTrue(plan instanceof Project);
plan = ((Project) plan).child();
assertTrue(plan instanceof Filter);
@ -175,8 +171,6 @@ public class OptimizerTests extends ESTestCase {
public void testWildcardEscapes() {
LogicalPlan plan = defaultPipes(accept("foo where command_line == '* %bar_ * \\\\ \\n \\r \\t'"));
assertTrue(plan instanceof Project);
plan = ((Project) plan).child();
assertTrue(plan instanceof Filter);
Filter filter = (Filter) plan;
@ -231,6 +225,8 @@ public class OptimizerTests extends ESTestCase {
}
private void checkOffsetAndLimit(LogicalPlan plan, int offset, int limit) {
assertTrue(plan instanceof Project);
plan = ((Project) plan).child();
assertTrue(plan instanceof LimitWithOffset);
LimitWithOffset lo = (LimitWithOffset) plan;
assertEquals("Incorrect offset", offset, lo.offset());
@ -299,9 +295,62 @@ public class OptimizerTests extends ESTestCase {
}
private LogicalPlan defaultPipes(LogicalPlan plan) {
assertTrue(plan instanceof Project);
plan = ((Project) plan).child();
assertTrue(plan instanceof LimitWithOffset);
plan = ((LimitWithOffset) plan).child();
assertTrue(plan instanceof OrderBy);
return ((OrderBy) plan).child();
}
public void testFilterPipePushdownEventQuery() {
Filter filter = new Filter(EMPTY, rel(), new IsNull(EMPTY, Literal.TRUE));
Filter pipe = new Filter(EMPTY, filter, new Equals(EMPTY, timestamp(), Literal.TRUE));
LogicalPlan optimized = new Optimizer.PushDownFilterPipe().rule(pipe);
assertEquals(Filter.class, optimized.getClass());
Filter f = (Filter) optimized;
Expression exp = f.condition();
assertEquals(And.class, exp.getClass());
And and = (And) exp;
assertEquals(filter.condition(), and.left());
assertEquals(pipe.condition(), and.right());
}
public void testFilterPipePushdownSequence() {
Filter filter = new Filter(EMPTY, rel(), new IsNull(EMPTY, Literal.TRUE));
KeyedFilter rule1 = keyedFilter(filter);
KeyedFilter rule2 = keyedFilter(filter);
KeyedFilter until = keyedFilter(filter);
Sequence s = new Sequence(EMPTY, asList(rule1, rule2), until, TimeValue.MINUS_ONE, timestamp(), tiebreaker(), OrderDirection.ASC);
Filter pipe = new Filter(EMPTY, s, new Equals(EMPTY, timestamp(), Literal.TRUE));
// apply it once to push down the filter
LogicalPlan optimized = new Optimizer.PushDownFilterPipe().apply(pipe);
// second to combine the filters
optimized = new Optimizer.PushDownFilterPipe().apply(optimized);
assertEquals(Sequence.class, optimized.getClass());
Sequence seq = (Sequence) optimized;
assertEquals(filter.condition(), condition(seq.until()));
Expression rule1Condition = condition(seq.children().get(0));
And and = (And) rule1Condition;
assertEquals(filter.condition(), and.left());
assertEquals(pipe.condition(), and.right());
Expression rule2Condition = condition(seq.children().get(1));
and = (And) rule2Condition;
assertEquals(filter.condition(), and.left());
assertEquals(pipe.condition(), and.right());
}
private Expression condition(LogicalPlan plan) {
assertEquals(KeyedFilter.class, plan.getClass());
KeyedFilter kf = (KeyedFilter) plan;
assertEquals(Filter.class, kf.child().getClass());
return ((Filter) kf.child()).condition();
}
}

View File

@ -45,7 +45,7 @@ public class ExpressionTests extends ESTestCase {
}
private List<Expression> exprs(String... sources) {
List<Expression> results = new ArrayList<Expression>(sources.length);
List<Expression> results = new ArrayList<>(sources.length);
for (String s : sources) {
results.add(expr(s));
}

View File

@ -26,7 +26,6 @@ import org.elasticsearch.xpack.ql.expression.UnresolvedAttribute;
import org.elasticsearch.xpack.ql.plan.logical.Filter;
import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
import org.elasticsearch.xpack.ql.plan.logical.OrderBy;
import org.elasticsearch.xpack.ql.plan.logical.Project;
import org.elasticsearch.xpack.ql.plan.logical.UnresolvedRelation;
import org.elasticsearch.xpack.ql.tree.Source;
import org.elasticsearch.xpack.ql.type.DataTypes;
@ -79,8 +78,7 @@ public class LogicalPlanTests extends ESTestCase {
private LogicalPlan wrapFilter(Expression exp) {
LogicalPlan filter = new Filter(Source.EMPTY, relation(), exp);
Order order = new Order(Source.EMPTY, timestamp(), OrderDirection.ASC, NullsPosition.FIRST);
LogicalPlan project = new Project(Source.EMPTY, filter, singletonList(timestamp()));
LogicalPlan sorted = new OrderBy(Source.EMPTY, project, singletonList(order));
LogicalPlan sorted = new OrderBy(Source.EMPTY, filter, singletonList(order));
LogicalPlan head = new Head(Source.EMPTY, new Literal(Source.EMPTY, RequestDefaults.SIZE, DataTypes.INTEGER), sorted);
return head;
}

View File

@ -9,6 +9,7 @@ package org.elasticsearch.xpack.eql.planner;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.eql.EqlTestUtils;
import org.elasticsearch.xpack.eql.analysis.Analyzer;
import org.elasticsearch.xpack.eql.analysis.PostAnalyzer;
import org.elasticsearch.xpack.eql.analysis.PreAnalyzer;
import org.elasticsearch.xpack.eql.analysis.Verifier;
import org.elasticsearch.xpack.eql.expression.function.EqlFunctionRegistry;
@ -25,6 +26,7 @@ import static org.elasticsearch.xpack.ql.type.TypesTests.loadMapping;
public abstract class AbstractQueryFolderTestCase extends ESTestCase {
protected EqlParser parser = new EqlParser();
protected PreAnalyzer preAnalyzer = new PreAnalyzer();
protected PostAnalyzer postAnalyzer = new PostAnalyzer();
protected EqlConfiguration configuration = EqlTestUtils.randomConfiguration();
protected Analyzer analyzer = new Analyzer(configuration, new EqlFunctionRegistry(), new Verifier(new Metrics()));
protected Optimizer optimizer = new Optimizer();
@ -33,7 +35,8 @@ public abstract class AbstractQueryFolderTestCase extends ESTestCase {
protected IndexResolution index = IndexResolution.valid(new EsIndex("test", loadMapping("mapping-default.json", true)));
protected PhysicalPlan plan(IndexResolution resolution, String eql) {
return planner.plan(optimizer.optimize(analyzer.analyze(preAnalyzer.preAnalyze(parser.createStatement(eql), resolution))));
return planner.plan(optimizer.optimize(postAnalyzer.postAnalyze(analyzer.analyze(preAnalyzer.preAnalyze(parser.createStatement(eql),
resolution)), configuration)));
}
protected PhysicalPlan plan(String eql) {

View File

@ -121,7 +121,7 @@ public class QueryFolderOkTests extends AbstractQueryFolderTestCase {
PhysicalPlan p = plan(query);
assertEquals(EsQueryExec.class, p.getClass());
EsQueryExec eqe = (EsQueryExec) p;
assertEquals(1, eqe.output().size());
assertEquals(0, eqe.output().size());
final String query = eqe.queryContainer().toString().replaceAll("\\s+", "");

View File

@ -54,18 +54,18 @@ public class VerifierMetricsTests extends ESTestCase {
}
public void testSequenceQuery() {
Counters c = eql("sequence\r\n" +
" [process where serial_event_id = 1]\r\n" +
Counters c = eql("sequence\r\n" +
" [process where serial_event_id = 1]\r\n" +
" [process where serial_event_id = 2]");
assertCounters(c, unmodifiableSet(new HashSet<>(Arrays.asList(SEQUENCE, PIPE_HEAD, SEQUENCE_QUERIES_TWO))));
}
@AwaitsFix(bugUrl = "waiting for the join implementation")
public void testJoinQuery() {
Counters c = eql("join\r\n" +
" [file where file_name=\"*.exe\"] by ppid\r\n" +
" [file where file_name=\"*.com\"] by pid\r\n" +
"until [process where opcode=1] by ppid\r\n" +
Counters c = eql("join\r\n" +
" [file where file_name=\"*.exe\"] by ppid\r\n" +
" [file where file_name=\"*.com\"] by pid\r\n" +
"until [process where opcode=1] by ppid\r\n" +
"| head 1");
assertCounters(c, unmodifiableSet(new HashSet<>(Arrays.asList(JOIN, PIPE_HEAD, JOIN_UNTIL, JOIN_QUERIES_TWO, JOIN_KEYS_ONE))));
}
@ -81,81 +81,81 @@ public class VerifierMetricsTests extends ESTestCase {
}
public void testSequenceMaxSpanQuery() {
Counters c = eql("sequence with maxspan=1d\r\n" +
" [process where serial_event_id < 4] by exit_code\r\n" +
" [process where opcode == 1] by user\r\n" +
" [process where opcode == 2] by user\r\n" +
Counters c = eql("sequence with maxspan=1d\r\n" +
" [process where serial_event_id < 4] by exit_code\r\n" +
" [process where opcode == 1] by user\r\n" +
" [process where opcode == 2] by user\r\n" +
" [file where parent_process_name == \"file_delete_event\"] by exit_code\r\n" +
"until [process where opcode=1] by ppid\r\n" +
"| head 4\r\n" +
"until [process where opcode=1] by ppid\r\n" +
"| head 4\r\n" +
"| tail 2");
assertCounters(c, unmodifiableSet(new HashSet<>(Arrays.asList(SEQUENCE, PIPE_HEAD, PIPE_TAIL, SEQUENCE_MAXSPAN, SEQUENCE_UNTIL,
SEQUENCE_QUERIES_FOUR, JOIN_KEYS_ONE))));
}
public void testSequenceWithTwoQueries() {
Counters c = eql("sequence with maxspan=1d\r\n" +
" [process where serial_event_id < 4] by exit_code\r\n" +
" [process where opcode == 1] by user\r\n" +
"until [process where opcode=1] by ppid\r\n" +
"| head 4\r\n" +
Counters c = eql("sequence with maxspan=1d\r\n" +
" [process where serial_event_id < 4] by exit_code\r\n" +
" [process where opcode == 1] by user\r\n" +
"until [process where opcode=1] by ppid\r\n" +
"| head 4\r\n" +
"| tail 2");
assertCounters(c, unmodifiableSet(new HashSet<>(Arrays.asList(SEQUENCE, PIPE_HEAD, PIPE_TAIL, SEQUENCE_MAXSPAN, SEQUENCE_UNTIL,
SEQUENCE_QUERIES_TWO, JOIN_KEYS_ONE))));
}
public void testSequenceWithThreeQueries() {
Counters c = eql("sequence with maxspan=1d\r\n" +
" [process where serial_event_id < 4] by exit_code\r\n" +
" [process where opcode == 1] by user\r\n" +
" [file where parent_process_name == \"file_delete_event\"] by exit_code\r\n" +
Counters c = eql("sequence with maxspan=1d\r\n" +
" [process where serial_event_id < 4] by exit_code\r\n" +
" [process where opcode == 1] by user\r\n" +
" [file where parent_process_name == \"file_delete_event\"] by exit_code\r\n" +
"| head 4");
assertCounters(c, unmodifiableSet(new HashSet<>(Arrays.asList(SEQUENCE, PIPE_HEAD, SEQUENCE_MAXSPAN, SEQUENCE_QUERIES_THREE,
JOIN_KEYS_ONE))));
}
public void testSequenceWithFiveQueries() {
Counters c = eql("sequence with maxspan=1d\r\n" +
" [process where serial_event_id < 4] by exit_code\r\n" +
" [process where opcode == 1] by user\r\n" +
Counters c = eql("sequence with maxspan=1d\r\n" +
" [process where serial_event_id < 4] by exit_code\r\n" +
" [process where opcode == 1] by user\r\n" +
" [file where parent_process_name == \"file_delete_event\"] by exit_code\r\n" +
" [process where serial_event_id < 4] by exit_code\r\n" +
" [process where opcode == 1] by user\r\n" +
" [process where serial_event_id < 4] by exit_code\r\n" +
" [process where opcode == 1] by user\r\n" +
"| head 4");
assertCounters(c, unmodifiableSet(new HashSet<>(Arrays.asList(SEQUENCE, PIPE_HEAD, SEQUENCE_MAXSPAN, SEQUENCE_QUERIES_FIVE_OR_MORE,
JOIN_KEYS_ONE))));
}
public void testSequenceWithSevenQueries() {
Counters c = eql("sequence by exit_code, user\r\n" +
" [process where serial_event_id < 4]\r\n" +
" [process where opcode == 1]\r\n" +
Counters c = eql("sequence by exit_code, user\r\n" +
" [process where serial_event_id < 4]\r\n" +
" [process where opcode == 1]\r\n" +
" [file where parent_process_name == \"file_delete_event\"]\r\n" +
" [process where serial_event_id < 4]\r\n" +
" [process where opcode == 1]\r\n" +
" [process where true]\r\n" +
" [process where true]\r\n" +
" [process where serial_event_id < 4]\r\n" +
" [process where opcode == 1]\r\n" +
" [process where true]\r\n" +
" [process where true]\r\n" +
"| tail 1");
assertCounters(c, unmodifiableSet(new HashSet<>(Arrays.asList(SEQUENCE, PIPE_TAIL, SEQUENCE_QUERIES_FIVE_OR_MORE, JOIN_KEYS_TWO))));
}
public void testSequenceWithThreeKeys() {
Counters c = eql("sequence by exit_code, user, serial_event_id\r\n" +
" [process where serial_event_id < 4]\r\n" +
Counters c = eql("sequence by exit_code, user, serial_event_id\r\n" +
" [process where serial_event_id < 4]\r\n" +
" [process where opcode == 1]\r\n");
assertCounters(c, unmodifiableSet(new HashSet<>(Arrays.asList(SEQUENCE, PIPE_HEAD, SEQUENCE_QUERIES_TWO, JOIN_KEYS_THREE))));
}
public void testSequenceWithFourKeys() {
Counters c = eql("sequence by exit_code, user, serial_event_id, pid\r\n" +
" [process where serial_event_id < 4]\r\n" +
Counters c = eql("sequence by exit_code, user, serial_event_id, pid\r\n" +
" [process where serial_event_id < 4]\r\n" +
" [process where opcode == 1]\r\n");
assertCounters(c, unmodifiableSet(new HashSet<>(Arrays.asList(SEQUENCE, PIPE_HEAD, SEQUENCE_QUERIES_TWO, JOIN_KEYS_FOUR))));
}
public void testSequenceWithFiveKeys() {
Counters c = eql("sequence by exit_code, user, serial_event_id, pid, ppid\r\n" +
" [process where serial_event_id < 4]\r\n" +
Counters c = eql("sequence by exit_code, user, serial_event_id, pid, ppid\r\n" +
" [process where serial_event_id < 4]\r\n" +
" [process where opcode == 1]\r\n");
assertCounters(c, unmodifiableSet(new HashSet<>(Arrays.asList(SEQUENCE, PIPE_HEAD, SEQUENCE_QUERIES_TWO, JOIN_KEYS_FIVE_OR_MORE))));
}
@ -173,7 +173,7 @@ public class VerifierMetricsTests extends ESTestCase {
return eql(query, null);
}
private Counters eql(String query, Verifier v) {
private Counters eql(String query, Verifier v) {
Verifier verifier = v;
Metrics metrics = null;
if (v == null) {

View File

@ -90,11 +90,24 @@ process where serial_event_id = 1;
process where serial_event_id < 4;
process where false;
process where missing_field != null;
process where serial_event_id <= 8 and serial_event_id > 7
| filter serial_event_id == 8;
process where true
| filter serial_event_id <= 10
| filter serial_event_id > 6;
process where true
| filter serial_event_id <= 10
| filter serial_event_id > 6
| head 2;
process where process_name == "VMACTHLP.exe" and unique_pid == 12 | filter true;
process where process_name == "impossible name" or (serial_event_id < 4.5 and serial_event_id >= 3.1)
;
@ -570,7 +583,6 @@ sequence by pid with maxspan=2h
// Pipes
//
security where event_id == 4624
| tail 10
;
@ -583,11 +595,15 @@ process where not (exit_code > -1)
| head 10
;
process where not (exit_code > -1) | head 7;
process where not (-1 < exit_code) | head 7;
process where true | filter true;
process where 1==1 | filter abc == def;
process where 1==1 | filter abc == def and 1 != 2;
file where true
| tail 3;
@ -686,4 +702,19 @@ process where fake_field != "*"
| head 4;
process where not (fake_field == "*")
| head 4;
| head 4;
process where 'net.EXE' == original_file_name
| filter process_name="net*.exe"
;
process where process_name == original_file_name
| filter process_name='net*.exe'
;
process where original_file_name == process_name
| filter length(original_file_name) > 0
;
process where process_name != original_file_name
| filter length(original_file_name) > 0;

View File

@ -42,21 +42,9 @@ process where 1==1 | count user_name, unique_pid, concat(field2,a,bc);
process where 1==1 | unique user_name, concat(field2,a,bc), field2;
process where true | filter true;
process where 1==1 | filter abc == def;
process where 1==1 | filter abc == def and 1 != 2;
process where 1==1 | count process_name | filter percent > 0.5;
any where true | unique a, b, c | sort a, b, c | count;
any where true | unique a, b, c | sort a, b, c | count;
@ -122,32 +110,6 @@ sequence by pid with maxspan=1.0075d
* https://raw.githubusercontent.com/endgameinc/eql/master/eql/etc/test_queries.toml
*/
process where serial_event_id <= 8 and serial_event_id > 7
| filter serial_event_id == 8;
process where true
| filter serial_event_id <= 10
| filter serial_event_id > 6;
process where true
| filter serial_event_id <= 10
| filter serial_event_id > 6
| head 2;
process where true
| head 1000
| filter serial_event_id <= 10
| filter serial_event_id > 6
| tail 2
;
process where process_name == "VMACTHLP.exe" and unique_pid == 12 | filter true;
process where process_name in ("python.exe", "SMSS.exe", "explorer.exe")
| unique process_name;
@ -297,24 +259,6 @@ sequence
| tail 1
;
process where 'net.EXE' == original_file_name
| filter process_name="net*.exe"
;
process where process_name == original_file_name
| filter process_name='net*.exe'
;
process where original_file_name == process_name
| filter length(original_file_name) > 0
;
process where process_name != original_file_name
| filter length(original_file_name) > 0;
any where process_name == "svchost.exe"
| unique_count event_type_full, process_name;

View File

@ -565,3 +565,15 @@ process where true | tail 10 | head 7
;
"size":7,
;
eventQueryWithFilterPipe
process where true | filter serial_event_id <= 4
;
"range":{"serial_event_id":{"from":null,"to":4,"include_lower":false,"include_upper":true
;
eventQueryWithFilterAndFilterPipe
process where serial_event_id > 1 | filter serial_event_id <= 4
;
"range":{"serial_event_id":{"from":1,"to":4,"include_lower":false,"include_upper":true
;