EQL: Introduce support for sequence maxspan (#58635)
EQL sequences can specify now a maximum time allowed for their span (computed between the first and the last matching event). (cherry picked from commit 747c3592244192a2e25a092f62aec91a899afc83)
This commit is contained in:
parent
773f3574a9
commit
3a546f1f51
|
@ -31,6 +31,9 @@ import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
import static org.hamcrest.Matchers.instanceOf;
|
||||||
|
import static org.junit.Assert.assertThat;
|
||||||
|
|
||||||
public class DataLoader {
|
public class DataLoader {
|
||||||
|
|
||||||
private static final String TEST_DATA = "/test_data.json";
|
private static final String TEST_DATA = "/test_data.json";
|
||||||
|
@ -64,7 +67,10 @@ public class DataLoader {
|
||||||
try (XContentParser parser = p.apply(JsonXContent.jsonXContent, DataLoader.class.getResourceAsStream(TEST_DATA))) {
|
try (XContentParser parser = p.apply(JsonXContent.jsonXContent, DataLoader.class.getResourceAsStream(TEST_DATA))) {
|
||||||
List<Object> list = parser.list();
|
List<Object> list = parser.list();
|
||||||
for (Object item : list) {
|
for (Object item : list) {
|
||||||
bulk.add(new IndexRequest(testIndexName).source((Map<String, Object>) item, XContentType.JSON));
|
assertThat(item, instanceOf(Map.class));
|
||||||
|
Map<String, Object> entry = (Map<String, Object>) item;
|
||||||
|
transformDataset(entry);
|
||||||
|
bulk.add(new IndexRequest(testIndexName).source(entry, XContentType.JSON));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -78,6 +84,23 @@ public class DataLoader {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static void transformDataset(Map<String, Object> entry) {
|
||||||
|
Object object = entry.get("timestamp");
|
||||||
|
assertThat(object, instanceOf(Long.class));
|
||||||
|
Long ts = (Long) object;
|
||||||
|
// currently this is windows filetime
|
||||||
|
entry.put("@timestamp", winFileTimeToUnix(ts));
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private static final long FILETIME_EPOCH_DIFF = 11644473600000L;
|
||||||
|
private static final long FILETIME_ONE_MILLISECOND = 10 * 1000;
|
||||||
|
|
||||||
|
public static long winFileTimeToUnix(final long filetime) {
|
||||||
|
long ts = (filetime / FILETIME_ONE_MILLISECOND);
|
||||||
|
return ts - FILETIME_EPOCH_DIFF;
|
||||||
|
}
|
||||||
|
|
||||||
private static XContentParser createParser(XContent xContent, InputStream data) throws IOException {
|
private static XContentParser createParser(XContent xContent, InputStream data) throws IOException {
|
||||||
NamedXContentRegistry contentRegistry = new NamedXContentRegistry(ClusterModule.getNamedXWriteables());
|
NamedXContentRegistry contentRegistry = new NamedXContentRegistry(ClusterModule.getNamedXWriteables());
|
||||||
return xContent.createParser(contentRegistry, LoggingDeprecationHandler.INSTANCE, data);
|
return xContent.createParser(contentRegistry, LoggingDeprecationHandler.INSTANCE, data);
|
||||||
|
|
|
@ -46,8 +46,7 @@
|
||||||
"type" : "date"
|
"type" : "date"
|
||||||
},
|
},
|
||||||
"@timestamp" : {
|
"@timestamp" : {
|
||||||
"type" : "alias",
|
"type" : "date"
|
||||||
"path" : "timestamp"
|
|
||||||
},
|
},
|
||||||
"user" : {
|
"user" : {
|
||||||
"type" : "keyword"
|
"type" : "keyword"
|
||||||
|
|
|
@ -336,51 +336,6 @@ until
|
||||||
'''
|
'''
|
||||||
expected_event_ids = [54, 55, 61, 67]
|
expected_event_ids = [54, 55, 61, 67]
|
||||||
|
|
||||||
[[queries]]
|
|
||||||
query = '''
|
|
||||||
sequence
|
|
||||||
[process where opcode == 1] by unique_pid, process_path
|
|
||||||
[file where opcode == 0] by unique_pid, process_path
|
|
||||||
[file where opcode == 0] by unique_pid, process_path
|
|
||||||
[file where opcode == 0] by unique_pid, process_path
|
|
||||||
until
|
|
||||||
[file where opcode == 200] by unique_pid, process_path
|
|
||||||
'''
|
|
||||||
|
|
||||||
|
|
||||||
[[queries]]
|
|
||||||
note = "Sequence: non-field based join."
|
|
||||||
query = '''
|
|
||||||
sequence
|
|
||||||
[process where serial_event_id<3] by unique_pid * 2
|
|
||||||
[process where true] by unique_ppid * 2
|
|
||||||
'''
|
|
||||||
expected_event_ids = [1, 2,
|
|
||||||
2, 3]
|
|
||||||
|
|
||||||
|
|
||||||
[[queries]]
|
|
||||||
note = "Sequence: multiple non-field based joins."
|
|
||||||
query = '''
|
|
||||||
sequence
|
|
||||||
[process where serial_event_id<3] by unique_pid * 2, length(unique_pid), string(unique_pid)
|
|
||||||
[process where true] by unique_ppid * 2, length(unique_ppid), string(unique_ppid)
|
|
||||||
'''
|
|
||||||
expected_event_ids = [1, 2,
|
|
||||||
2, 3]
|
|
||||||
|
|
||||||
[[queries]]
|
|
||||||
query = '''
|
|
||||||
sequence with maxspan=500ms
|
|
||||||
[file where event_subtype_full == "file_create_event"] by file_path
|
|
||||||
[process where opcode == 1] by process_path
|
|
||||||
[process where opcode == 2] by process_path
|
|
||||||
[file where event_subtype_full == "file_delete_event"] by file_path
|
|
||||||
| head 4
|
|
||||||
| tail 2
|
|
||||||
'''
|
|
||||||
expected_event_ids = []
|
|
||||||
|
|
||||||
[[queries]]
|
[[queries]]
|
||||||
query = '''
|
query = '''
|
||||||
sequence
|
sequence
|
||||||
|
@ -1026,6 +981,28 @@ query = '''
|
||||||
registry where arrayContains(bytes_written_string_list, "missing", "en-US")
|
registry where arrayContains(bytes_written_string_list, "missing", "en-US")
|
||||||
'''
|
'''
|
||||||
|
|
||||||
|
|
||||||
|
[[queries]]
|
||||||
|
note = "Sequence: non-field based join."
|
||||||
|
query = '''
|
||||||
|
sequence
|
||||||
|
[process where serial_event_id<3] by unique_pid * 2
|
||||||
|
[process where true] by unique_ppid * 2
|
||||||
|
'''
|
||||||
|
expected_event_ids = [1, 2,
|
||||||
|
2, 3]
|
||||||
|
|
||||||
|
|
||||||
|
[[queries]]
|
||||||
|
note = "Sequence: multiple non-field based joins."
|
||||||
|
query = '''
|
||||||
|
sequence
|
||||||
|
[process where serial_event_id<3] by unique_pid * 2, length(unique_pid), string(unique_pid)
|
||||||
|
[process where true] by unique_ppid * 2, length(unique_ppid), string(unique_ppid)
|
||||||
|
'''
|
||||||
|
expected_event_ids = [1, 2,
|
||||||
|
2, 3]
|
||||||
|
|
||||||
# TODO: update toggles for this function
|
# TODO: update toggles for this function
|
||||||
[[queries]]
|
[[queries]]
|
||||||
case_sensitive = true
|
case_sensitive = true
|
||||||
|
|
|
@ -10,6 +10,8 @@ import org.elasticsearch.search.SearchHit;
|
||||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||||
import org.elasticsearch.xpack.eql.EqlIllegalArgumentException;
|
import org.elasticsearch.xpack.eql.EqlIllegalArgumentException;
|
||||||
import org.elasticsearch.xpack.eql.execution.search.QueryRequest;
|
import org.elasticsearch.xpack.eql.execution.search.QueryRequest;
|
||||||
|
import org.elasticsearch.xpack.eql.execution.sequence.Ordinal;
|
||||||
|
import org.elasticsearch.xpack.eql.util.ReversedIterator;
|
||||||
import org.elasticsearch.xpack.ql.execution.search.extractor.HitExtractor;
|
import org.elasticsearch.xpack.ql.execution.search.extractor.HitExtractor;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -22,12 +24,17 @@ public class Criterion implements QueryRequest {
|
||||||
private final HitExtractor tiebreakerExtractor;
|
private final HitExtractor tiebreakerExtractor;
|
||||||
|
|
||||||
// search after markers
|
// search after markers
|
||||||
private Object[] startMarker;
|
private Ordinal startMarker;
|
||||||
private Object[] stopMarker;
|
private Ordinal stopMarker;
|
||||||
|
|
||||||
|
private boolean reverse;
|
||||||
|
|
||||||
//TODO: should accept QueryRequest instead of another SearchSourceBuilder
|
//TODO: should accept QueryRequest instead of another SearchSourceBuilder
|
||||||
public Criterion(SearchSourceBuilder searchSource, List<HitExtractor> searchAfterExractors, HitExtractor timestampExtractor,
|
public Criterion(SearchSourceBuilder searchSource,
|
||||||
HitExtractor tiebreakerExtractor) {
|
List<HitExtractor> searchAfterExractors,
|
||||||
|
HitExtractor timestampExtractor,
|
||||||
|
HitExtractor tiebreakerExtractor,
|
||||||
|
boolean reverse) {
|
||||||
this.searchSource = searchSource;
|
this.searchSource = searchSource;
|
||||||
this.keyExtractors = searchAfterExractors;
|
this.keyExtractors = searchAfterExractors;
|
||||||
this.timestampExtractor = timestampExtractor;
|
this.timestampExtractor = timestampExtractor;
|
||||||
|
@ -35,6 +42,7 @@ public class Criterion implements QueryRequest {
|
||||||
|
|
||||||
this.startMarker = null;
|
this.startMarker = null;
|
||||||
this.stopMarker = null;
|
this.stopMarker = null;
|
||||||
|
this.reverse = reverse;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -54,54 +62,45 @@ public class Criterion implements QueryRequest {
|
||||||
return tiebreakerExtractor;
|
return tiebreakerExtractor;
|
||||||
}
|
}
|
||||||
|
|
||||||
public long timestamp(SearchHit hit) {
|
|
||||||
Object ts = timestampExtractor.extract(hit);
|
|
||||||
if (ts instanceof Number) {
|
|
||||||
return ((Number) ts).longValue();
|
|
||||||
}
|
|
||||||
throw new EqlIllegalArgumentException("Expected timestamp as long but got {}", ts);
|
|
||||||
}
|
|
||||||
|
|
||||||
@SuppressWarnings({ "unchecked" })
|
@SuppressWarnings({ "unchecked" })
|
||||||
public Comparable<Object> tiebreaker(SearchHit hit) {
|
public Ordinal ordinal(SearchHit hit) {
|
||||||
if (tiebreakerExtractor == null) {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
Object tb = tiebreakerExtractor.extract(hit);
|
|
||||||
if (tb instanceof Comparable) {
|
|
||||||
return (Comparable<Object>) tb;
|
|
||||||
}
|
|
||||||
throw new EqlIllegalArgumentException("Expected tiebreaker to be Comparable but got {}", tb);
|
|
||||||
}
|
|
||||||
|
|
||||||
public Object[] startMarker() {
|
Object ts = timestampExtractor.extract(hit);
|
||||||
return startMarker;
|
if (ts instanceof Number == false) {
|
||||||
}
|
throw new EqlIllegalArgumentException("Expected timestamp as long but got {}", ts);
|
||||||
|
|
||||||
public Object[] stopMarker() {
|
|
||||||
return stopMarker;
|
|
||||||
}
|
|
||||||
|
|
||||||
private Object[] marker(SearchHit hit) {
|
|
||||||
long timestamp = timestamp(hit);
|
|
||||||
Object tiebreaker = null;
|
|
||||||
if (tiebreakerExtractor() != null) {
|
|
||||||
tiebreaker = tiebreaker(hit);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return tiebreaker != null ? new Object[] { timestamp, tiebreaker } : new Object[] { timestamp };
|
long timestamp = ((Number) ts).longValue();
|
||||||
|
Comparable<Object> tiebreaker = null;
|
||||||
|
|
||||||
|
if (tiebreakerExtractor != null) {
|
||||||
|
Object tb = tiebreakerExtractor.extract(hit);
|
||||||
|
if (tb instanceof Comparable == false) {
|
||||||
|
throw new EqlIllegalArgumentException("Expected tiebreaker to be Comparable but got {}", tb);
|
||||||
|
}
|
||||||
|
tiebreaker = (Comparable<Object>) tb;
|
||||||
|
}
|
||||||
|
return new Ordinal(timestamp, tiebreaker);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void startMarker(SearchHit hit) {
|
public void startMarker(Ordinal ordinal) {
|
||||||
startMarker = marker(hit);
|
startMarker = ordinal;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void stopMarker(SearchHit hit) {
|
public void stopMarker(Ordinal ordinal) {
|
||||||
stopMarker = marker(hit);
|
stopMarker = ordinal;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Criterion useMarker(Object[] marker) {
|
public Ordinal nextMarker() {
|
||||||
searchSource.searchAfter(marker);
|
return startMarker.compareTo(stopMarker) < 1 ? startMarker : stopMarker;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Criterion useMarker(Ordinal marker) {
|
||||||
|
searchSource.searchAfter(marker.toArray());
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Iterable<SearchHit> iterable(List<SearchHit> hits) {
|
||||||
|
return () -> reverse ? new ReversedIterator<>(hits) : hits.iterator();
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -6,6 +6,7 @@
|
||||||
|
|
||||||
package org.elasticsearch.xpack.eql.execution.assembler;
|
package org.elasticsearch.xpack.eql.execution.assembler;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.xpack.eql.EqlIllegalArgumentException;
|
import org.elasticsearch.xpack.eql.EqlIllegalArgumentException;
|
||||||
import org.elasticsearch.xpack.eql.execution.search.BasicQueryClient;
|
import org.elasticsearch.xpack.eql.execution.search.BasicQueryClient;
|
||||||
import org.elasticsearch.xpack.eql.execution.search.Limit;
|
import org.elasticsearch.xpack.eql.execution.search.Limit;
|
||||||
|
@ -43,11 +44,14 @@ public class ExecutionManager {
|
||||||
Attribute timestamp,
|
Attribute timestamp,
|
||||||
Attribute tiebreaker,
|
Attribute tiebreaker,
|
||||||
OrderDirection direction,
|
OrderDirection direction,
|
||||||
|
TimeValue maxSpan,
|
||||||
Limit limit) {
|
Limit limit) {
|
||||||
FieldExtractorRegistry extractorRegistry = new FieldExtractorRegistry();
|
FieldExtractorRegistry extractorRegistry = new FieldExtractorRegistry();
|
||||||
|
|
||||||
List<Criterion> criteria = new ArrayList<>(plans.size() - 1);
|
List<Criterion> criteria = new ArrayList<>(plans.size() - 1);
|
||||||
|
|
||||||
|
boolean descending = direction == OrderDirection.DESC;
|
||||||
|
|
||||||
// build a criterion for each query
|
// build a criterion for each query
|
||||||
for (int i = 0; i < plans.size() - 1; i++) {
|
for (int i = 0; i < plans.size() - 1; i++) {
|
||||||
List<Attribute> keys = listOfKeys.get(i);
|
List<Attribute> keys = listOfKeys.get(i);
|
||||||
|
@ -61,9 +65,10 @@ public class ExecutionManager {
|
||||||
// TODO: this could be generalized into an exec only query
|
// TODO: this could be generalized into an exec only query
|
||||||
Check.isTrue(query instanceof EsQueryExec, "Expected a query but got [{}]", query.getClass());
|
Check.isTrue(query instanceof EsQueryExec, "Expected a query but got [{}]", query.getClass());
|
||||||
QueryRequest request = ((EsQueryExec) query).queryRequest(session);
|
QueryRequest request = ((EsQueryExec) query).queryRequest(session);
|
||||||
criteria.add(new Criterion(request.searchSource(), keyExtractors, tsExtractor, tbExtractor));
|
// base query remains descending, the rest need to flip
|
||||||
|
criteria.add(new Criterion(request.searchSource(), keyExtractors, tsExtractor, tbExtractor, i > 0 && descending));
|
||||||
}
|
}
|
||||||
return new SequenceRuntime(criteria, new BasicQueryClient(session), direction == OrderDirection.DESC, limit);
|
return new SequenceRuntime(criteria, new BasicQueryClient(session), maxSpan, limit);
|
||||||
}
|
}
|
||||||
|
|
||||||
private HitExtractor timestampExtractor(HitExtractor hitExtractor) {
|
private HitExtractor timestampExtractor(HitExtractor hitExtractor) {
|
||||||
|
|
|
@ -6,24 +6,23 @@
|
||||||
|
|
||||||
package org.elasticsearch.xpack.eql.execution.assembler;
|
package org.elasticsearch.xpack.eql.execution.assembler;
|
||||||
|
|
||||||
|
import org.elasticsearch.xpack.eql.execution.sequence.Ordinal;
|
||||||
import org.elasticsearch.xpack.eql.execution.sequence.SequenceKey;
|
import org.elasticsearch.xpack.eql.execution.sequence.SequenceKey;
|
||||||
|
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
|
||||||
class KeyAndOrdinal {
|
class KeyAndOrdinal {
|
||||||
final SequenceKey key;
|
final SequenceKey key;
|
||||||
final long timestamp;
|
final Ordinal ordinal;
|
||||||
final Comparable<Object> tiebreaker;
|
|
||||||
|
|
||||||
KeyAndOrdinal(SequenceKey key, long timestamp, Comparable<Object> tiebreaker) {
|
KeyAndOrdinal(SequenceKey key, Ordinal ordinal) {
|
||||||
this.key = key;
|
this.key = key;
|
||||||
this.timestamp = timestamp;
|
this.ordinal = ordinal;
|
||||||
this.tiebreaker = tiebreaker;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode() {
|
public int hashCode() {
|
||||||
return Objects.hash(key, timestamp, tiebreaker);
|
return Objects.hash(key, ordinal);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -38,12 +37,11 @@ class KeyAndOrdinal {
|
||||||
|
|
||||||
KeyAndOrdinal other = (KeyAndOrdinal) obj;
|
KeyAndOrdinal other = (KeyAndOrdinal) obj;
|
||||||
return Objects.equals(key, other.key)
|
return Objects.equals(key, other.key)
|
||||||
&& Objects.equals(timestamp, other.timestamp)
|
&& Objects.equals(ordinal, other.ordinal);
|
||||||
&& Objects.equals(tiebreaker, other.tiebreaker);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return key + "[" + timestamp + "][" + (tiebreaker != null ? Objects.toString(tiebreaker) : "") + "]";
|
return key.toString() + ordinal.toString();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -10,18 +10,23 @@ import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.xpack.eql.execution.payload.AbstractPayload;
|
import org.elasticsearch.xpack.eql.execution.payload.AbstractPayload;
|
||||||
import org.elasticsearch.xpack.eql.execution.sequence.Sequence;
|
import org.elasticsearch.xpack.eql.execution.sequence.Sequence;
|
||||||
import org.elasticsearch.xpack.eql.session.Results.Type;
|
import org.elasticsearch.xpack.eql.session.Results.Type;
|
||||||
|
import org.elasticsearch.xpack.eql.util.ReversedIterator;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
class SequencePayload extends AbstractPayload {
|
class SequencePayload extends AbstractPayload {
|
||||||
|
|
||||||
private final List<org.elasticsearch.xpack.eql.action.EqlSearchResponse.Sequence> sequences;
|
private final List<org.elasticsearch.xpack.eql.action.EqlSearchResponse.Sequence> sequences;
|
||||||
|
|
||||||
SequencePayload(List<Sequence> seq, boolean timedOut, TimeValue timeTook, Object[] nextKeys) {
|
SequencePayload(List<Sequence> seq, boolean timedOut, TimeValue timeTook) {
|
||||||
super(timedOut, timeTook, nextKeys);
|
super(timedOut, timeTook);
|
||||||
sequences = new ArrayList<>(seq.size());
|
sequences = new ArrayList<>(seq.size());
|
||||||
for (Sequence s : seq) {
|
boolean needsReversal = seq.size() > 1 && (seq.get(0).ordinal().compareTo(seq.get(1).ordinal()) > 0);
|
||||||
|
|
||||||
|
for (Iterator<Sequence> it = needsReversal ? new ReversedIterator<>(seq) : seq.iterator(); it.hasNext();) {
|
||||||
|
Sequence s = it.next();
|
||||||
sequences.add(new org.elasticsearch.xpack.eql.action.EqlSearchResponse.Sequence(s.key().asStringList(), s.hits()));
|
sequences.add(new org.elasticsearch.xpack.eql.action.EqlSearchResponse.Sequence(s.key().asStringList(), s.hits()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -11,17 +11,15 @@ import org.apache.logging.log4j.Logger;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.search.SearchHit;
|
import org.elasticsearch.search.SearchHit;
|
||||||
import org.elasticsearch.xpack.eql.execution.payload.ReversePayload;
|
|
||||||
import org.elasticsearch.xpack.eql.execution.search.Limit;
|
import org.elasticsearch.xpack.eql.execution.search.Limit;
|
||||||
import org.elasticsearch.xpack.eql.execution.search.QueryClient;
|
import org.elasticsearch.xpack.eql.execution.search.QueryClient;
|
||||||
|
import org.elasticsearch.xpack.eql.execution.sequence.Ordinal;
|
||||||
import org.elasticsearch.xpack.eql.execution.sequence.Sequence;
|
import org.elasticsearch.xpack.eql.execution.sequence.Sequence;
|
||||||
import org.elasticsearch.xpack.eql.execution.sequence.SequenceKey;
|
import org.elasticsearch.xpack.eql.execution.sequence.SequenceKey;
|
||||||
import org.elasticsearch.xpack.eql.execution.sequence.SequenceStateMachine;
|
import org.elasticsearch.xpack.eql.execution.sequence.SequenceStateMachine;
|
||||||
import org.elasticsearch.xpack.eql.session.Payload;
|
import org.elasticsearch.xpack.eql.session.Payload;
|
||||||
import org.elasticsearch.xpack.eql.util.ReversedIterator;
|
|
||||||
import org.elasticsearch.xpack.ql.execution.search.extractor.HitExtractor;
|
import org.elasticsearch.xpack.ql.execution.search.extractor.HitExtractor;
|
||||||
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import static org.elasticsearch.action.ActionListener.wrap;
|
import static org.elasticsearch.action.ActionListener.wrap;
|
||||||
|
@ -38,18 +36,14 @@ class SequenceRuntime implements Executable {
|
||||||
private final int numberOfStages;
|
private final int numberOfStages;
|
||||||
private final SequenceStateMachine stateMachine;
|
private final SequenceStateMachine stateMachine;
|
||||||
private final QueryClient queryClient;
|
private final QueryClient queryClient;
|
||||||
private final boolean descending;
|
|
||||||
|
|
||||||
private long startTime;
|
private long startTime;
|
||||||
|
|
||||||
SequenceRuntime(List<Criterion> criteria, QueryClient queryClient, boolean descending, Limit limit) {
|
SequenceRuntime(List<Criterion> criteria, QueryClient queryClient, TimeValue maxSpan, Limit limit) {
|
||||||
this.criteria = criteria;
|
this.criteria = criteria;
|
||||||
this.numberOfStages = criteria.size();
|
this.numberOfStages = criteria.size();
|
||||||
this.queryClient = queryClient;
|
this.queryClient = queryClient;
|
||||||
boolean hasTiebreaker = criteria.get(0).tiebreakerExtractor() != null;
|
this.stateMachine = new SequenceStateMachine(numberOfStages, maxSpan, limit);
|
||||||
this.stateMachine = new SequenceStateMachine(numberOfStages, hasTiebreaker, limit);
|
|
||||||
|
|
||||||
this.descending = descending;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -73,9 +67,8 @@ class SequenceRuntime implements Executable {
|
||||||
// narrow by the previous stage timestamp marker
|
// narrow by the previous stage timestamp marker
|
||||||
|
|
||||||
Criterion previous = criteria.get(stage - 1);
|
Criterion previous = criteria.get(stage - 1);
|
||||||
// if DESC, flip the markers (the stop becomes the start due to the reverse order), otherwise keep it accordingly
|
// pass the next marker along
|
||||||
Object[] marker = descending && stage == 1 ? previous.stopMarker() : previous.startMarker();
|
currentCriterion.useMarker(previous.nextMarker());
|
||||||
currentCriterion.useMarker(marker);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
log.info("Querying stage {}", stage);
|
log.info("Querying stage {}", stage);
|
||||||
|
@ -95,34 +88,39 @@ class SequenceRuntime implements Executable {
|
||||||
}
|
}
|
||||||
|
|
||||||
// hits are guaranteed to be non-empty
|
// hits are guaranteed to be non-empty
|
||||||
private void findMatches(int currentStage, List<SearchHit> hits) {
|
private void findMatches(int stage, List<SearchHit> hits) {
|
||||||
// update criterion
|
// update criterion
|
||||||
Criterion criterion = criteria.get(currentStage);
|
Criterion criterion = criteria.get(stage);
|
||||||
criterion.startMarker(hits.get(0));
|
|
||||||
criterion.stopMarker(hits.get(hits.size() - 1));
|
|
||||||
|
|
||||||
// break the results per key
|
// break the results per key
|
||||||
// when dealing with descending order, queries outside the base are ASC (search_before)
|
// when dealing with descending order, queries outside the base are ASC (search_before)
|
||||||
// so look at the data in reverse (that is DESC)
|
// so look at the data in reverse (that is DESC)
|
||||||
for (Iterator<SearchHit> it = descending ? new ReversedIterator<>(hits) : hits.iterator(); it.hasNext();) {
|
Ordinal firstOrdinal = null, ordinal = null;
|
||||||
SearchHit hit = it.next();
|
for (SearchHit hit : criterion.iterable(hits)) {
|
||||||
|
|
||||||
KeyAndOrdinal ko = key(hit, criterion);
|
KeyAndOrdinal ko = key(hit, criterion);
|
||||||
if (currentStage == 0) {
|
|
||||||
Sequence seq = new Sequence(ko.key, numberOfStages, ko.timestamp, ko.tiebreaker, hit);
|
ordinal = ko.ordinal;
|
||||||
long tStart = (long) criterion.startMarker()[0];
|
|
||||||
long tStop = (long) criterion.stopMarker()[0];
|
if (firstOrdinal == null) {
|
||||||
stateMachine.trackSequence(seq, tStart, tStop);
|
firstOrdinal = ordinal;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (stage == 0) {
|
||||||
|
Sequence seq = new Sequence(ko.key, numberOfStages, ordinal, hit);
|
||||||
|
stateMachine.trackSequence(seq);
|
||||||
} else {
|
} else {
|
||||||
stateMachine.match(currentStage, ko.key, ko.timestamp, ko.tiebreaker, hit);
|
stateMachine.match(stage, ko.key, ordinal, hit);
|
||||||
|
|
||||||
// early skip in case of reaching the limit
|
// early skip in case of reaching the limit
|
||||||
// check the last stage to avoid calling the state machine in other stages
|
// check the last stage to avoid calling the state machine in other stages
|
||||||
if (stateMachine.reachedLimit()) {
|
if (stateMachine.reachedLimit()) {
|
||||||
return;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
criterion.startMarker(firstOrdinal);
|
||||||
|
criterion.stopMarker(ordinal);
|
||||||
}
|
}
|
||||||
|
|
||||||
private KeyAndOrdinal key(SearchHit hit, Criterion criterion) {
|
private KeyAndOrdinal key(SearchHit hit, Criterion criterion) {
|
||||||
|
@ -139,14 +137,13 @@ class SequenceRuntime implements Executable {
|
||||||
key = new SequenceKey(docKeys);
|
key = new SequenceKey(docKeys);
|
||||||
}
|
}
|
||||||
|
|
||||||
return new KeyAndOrdinal(key, criterion.timestamp(hit), criterion.tiebreaker(hit));
|
return new KeyAndOrdinal(key, criterion.ordinal(hit));
|
||||||
}
|
}
|
||||||
|
|
||||||
private Payload sequencePayload() {
|
private Payload sequencePayload() {
|
||||||
List<Sequence> completed = stateMachine.completeSequences();
|
List<Sequence> completed = stateMachine.completeSequences();
|
||||||
TimeValue tookTime = new TimeValue(System.currentTimeMillis() - startTime);
|
TimeValue tookTime = new TimeValue(System.currentTimeMillis() - startTime);
|
||||||
SequencePayload payload = new SequencePayload(completed, false, tookTime, null);
|
return new SequencePayload(completed, false, tookTime);
|
||||||
return descending ? new ReversePayload(payload) : payload;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean hasFinished(int stage) {
|
private boolean hasFinished(int stage) {
|
||||||
|
|
|
@ -13,12 +13,10 @@ public abstract class AbstractPayload implements Payload {
|
||||||
|
|
||||||
private final boolean timedOut;
|
private final boolean timedOut;
|
||||||
private final TimeValue timeTook;
|
private final TimeValue timeTook;
|
||||||
private final Object[] nextKeys;
|
|
||||||
|
|
||||||
protected AbstractPayload(boolean timedOut, TimeValue timeTook, Object[] nextKeys) {
|
protected AbstractPayload(boolean timedOut, TimeValue timeTook) {
|
||||||
this.timedOut = timedOut;
|
this.timedOut = timedOut;
|
||||||
this.timeTook = timeTook;
|
this.timeTook = timeTook;
|
||||||
this.nextKeys = nextKeys;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -30,9 +28,4 @@ public abstract class AbstractPayload implements Payload {
|
||||||
public TimeValue timeTook() {
|
public TimeValue timeTook() {
|
||||||
return timeTook;
|
return timeTook;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object[] nextKeys() {
|
|
||||||
return nextKeys;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -37,11 +37,6 @@ public class ReversePayload implements Payload {
|
||||||
return delegate.timeTook();
|
return delegate.timeTook();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object[] nextKeys() {
|
|
||||||
return delegate.nextKeys();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <V> List<V> values() {
|
public <V> List<V> values() {
|
||||||
return delegate.values();
|
return delegate.values();
|
||||||
|
|
|
@ -18,7 +18,7 @@ public class SearchResponsePayload extends AbstractPayload {
|
||||||
private final List<SearchHit> hits;
|
private final List<SearchHit> hits;
|
||||||
|
|
||||||
public SearchResponsePayload(SearchResponse response) {
|
public SearchResponsePayload(SearchResponse response) {
|
||||||
super(response.isTimedOut(), response.getTook(), null);
|
super(response.isTimedOut(), response.getTook());
|
||||||
hits = Arrays.asList(response.getHits().getHits());
|
hits = Arrays.asList(response.getHits().getHits());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -15,22 +15,16 @@ import java.util.Objects;
|
||||||
*/
|
*/
|
||||||
class Match {
|
class Match {
|
||||||
|
|
||||||
private final long timestamp;
|
private final Ordinal ordinal;
|
||||||
private final Comparable<Object> tiebreaker;
|
|
||||||
private final SearchHit hit;
|
private final SearchHit hit;
|
||||||
|
|
||||||
Match(long timestamp, Comparable<Object> tiebreaker, SearchHit hit) {
|
Match(Ordinal ordinal, SearchHit hit) {
|
||||||
this.timestamp = timestamp;
|
this.ordinal = ordinal;
|
||||||
this.tiebreaker = tiebreaker;
|
|
||||||
this.hit = hit;
|
this.hit = hit;
|
||||||
}
|
}
|
||||||
|
|
||||||
long timestamp() {
|
Ordinal ordinal() {
|
||||||
return timestamp;
|
return ordinal;
|
||||||
}
|
|
||||||
|
|
||||||
Comparable<Object> tiebreaker() {
|
|
||||||
return tiebreaker;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
SearchHit hit() {
|
SearchHit hit() {
|
||||||
|
@ -39,7 +33,7 @@ class Match {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode() {
|
public int hashCode() {
|
||||||
return Objects.hash(timestamp, tiebreaker, hit);
|
return Objects.hash(ordinal, hit);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -53,13 +47,12 @@ class Match {
|
||||||
}
|
}
|
||||||
|
|
||||||
Match other = (Match) obj;
|
Match other = (Match) obj;
|
||||||
return Objects.equals(timestamp, other.timestamp)
|
return Objects.equals(ordinal, other.ordinal)
|
||||||
&& Objects.equals(tiebreaker, other.tiebreaker)
|
|
||||||
&& Objects.equals(hit, other.hit);
|
&& Objects.equals(hit, other.hit);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return timestamp + "[" + (tiebreaker != null ? tiebreaker : "") + "]->" + hit.getId();
|
return ordinal.toString() + "->" + hit.getId();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,75 @@
|
||||||
|
/*
|
||||||
|
* 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.sequence;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class Ordinal implements Comparable<Ordinal> {
|
||||||
|
|
||||||
|
final long timestamp;
|
||||||
|
final Comparable<Object> tiebreaker;
|
||||||
|
|
||||||
|
public Ordinal(long timestamp, Comparable<Object> tiebreaker) {
|
||||||
|
this.timestamp = timestamp;
|
||||||
|
this.tiebreaker = tiebreaker;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return Objects.hash(timestamp, tiebreaker);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object obj) {
|
||||||
|
if (this == obj) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (obj == null || getClass() != obj.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
Ordinal other = (Ordinal) obj;
|
||||||
|
return Objects.equals(timestamp, other.timestamp)
|
||||||
|
&& Objects.equals(tiebreaker, other.tiebreaker);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "[" + timestamp + "][" + (tiebreaker != null ? tiebreaker.toString() : "") + "]";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compareTo(Ordinal o) {
|
||||||
|
if (timestamp < o.timestamp) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
if (timestamp == o.timestamp) {
|
||||||
|
if (tiebreaker != null) {
|
||||||
|
if (o.tiebreaker != null) {
|
||||||
|
return tiebreaker.compareTo(o.tiebreaker);
|
||||||
|
}
|
||||||
|
// nulls are first - lower than any other value
|
||||||
|
// other tiebreaker is null this one isn't, fall through 1
|
||||||
|
}
|
||||||
|
// null tiebreaker
|
||||||
|
else {
|
||||||
|
if (o.tiebreaker != null) {
|
||||||
|
return -1;
|
||||||
|
} else {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// if none of the branches above matched, this ordinal is greater than o
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Object[] toArray() {
|
||||||
|
return tiebreaker != null ? new Object[] { timestamp, tiebreaker } : new Object[] { timestamp };
|
||||||
|
}
|
||||||
|
}
|
|
@ -31,19 +31,19 @@ public class Sequence {
|
||||||
|
|
||||||
private int currentStage = 0;
|
private int currentStage = 0;
|
||||||
|
|
||||||
public Sequence(SequenceKey key, int stages, long timestamp, Comparable<Object> tiebreaker, SearchHit firstHit) {
|
public Sequence(SequenceKey key, int stages, Ordinal ordinal, SearchHit firstHit) {
|
||||||
Check.isTrue(stages >= 2, "A sequence requires at least 2 criteria, given [{}]", stages);
|
Check.isTrue(stages >= 2, "A sequence requires at least 2 criteria, given [{}]", stages);
|
||||||
this.key = key;
|
this.key = key;
|
||||||
this.stages = stages;
|
this.stages = stages;
|
||||||
this.matches = new Match[stages];
|
this.matches = new Match[stages];
|
||||||
this.matches[0] = new Match(timestamp, tiebreaker, firstHit);
|
this.matches[0] = new Match(ordinal, firstHit);
|
||||||
}
|
}
|
||||||
|
|
||||||
public int putMatch(int stage, SearchHit hit, long timestamp, Comparable<Object> tiebreaker) {
|
public int putMatch(int stage, SearchHit hit, Ordinal ordinal) {
|
||||||
if (stage == currentStage + 1) {
|
if (stage == currentStage + 1) {
|
||||||
int previousStage = currentStage;
|
int previousStage = currentStage;
|
||||||
currentStage = stage;
|
currentStage = stage;
|
||||||
matches[currentStage] = new Match(timestamp, tiebreaker, hit);
|
matches[currentStage] = new Match(ordinal, hit);
|
||||||
return previousStage;
|
return previousStage;
|
||||||
}
|
}
|
||||||
throw new EqlIllegalArgumentException("Incorrect stage [{}] specified for Sequence[key={}, stage=]", stage, key, currentStage);
|
throw new EqlIllegalArgumentException("Incorrect stage [{}] specified for Sequence[key={}, stage=]", stage, key, currentStage);
|
||||||
|
@ -53,24 +53,12 @@ public class Sequence {
|
||||||
return key;
|
return key;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int currentStage() {
|
public Ordinal ordinal() {
|
||||||
return currentStage;
|
return matches[currentStage].ordinal();
|
||||||
}
|
}
|
||||||
|
|
||||||
public long currentTimestamp() {
|
public long startTimestamp() {
|
||||||
return matches[currentStage].timestamp();
|
return matches[0].ordinal().timestamp;
|
||||||
}
|
|
||||||
|
|
||||||
public Comparable<Object> currentTiebreaker() {
|
|
||||||
return matches[currentStage].tiebreaker();
|
|
||||||
}
|
|
||||||
|
|
||||||
public long timestamp(int stage) {
|
|
||||||
// stages not initialized yet return an out-of-band value to have no impact on the interval range
|
|
||||||
if (stage > currentStage) {
|
|
||||||
return Long.MAX_VALUE;
|
|
||||||
}
|
|
||||||
return matches[stage].timestamp();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<SearchHit> hits() {
|
public List<SearchHit> hits() {
|
||||||
|
@ -110,7 +98,7 @@ public class Sequence {
|
||||||
StringBuilder sb = new StringBuilder();
|
StringBuilder sb = new StringBuilder();
|
||||||
sb.append(format(null, "[Seq<{}>[{}/{}]]",
|
sb.append(format(null, "[Seq<{}>[{}/{}]]",
|
||||||
key,
|
key,
|
||||||
nf.format(currentStage()),
|
nf.format(currentStage),
|
||||||
nf.format(stages - 1)));
|
nf.format(stages - 1)));
|
||||||
|
|
||||||
for (int i = 0; i < matches.length; i++) {
|
for (int i = 0; i < matches.length; i++) {
|
||||||
|
|
|
@ -10,6 +10,7 @@ import org.elasticsearch.common.collect.Tuple;
|
||||||
|
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.function.Predicate;
|
||||||
|
|
||||||
import static org.elasticsearch.common.logging.LoggerMessageFormat.format;
|
import static org.elasticsearch.common.logging.LoggerMessageFormat.format;
|
||||||
|
|
||||||
|
@ -21,27 +22,21 @@ public class SequenceFrame {
|
||||||
// timestamp compression (whose range is known for the current frame).
|
// timestamp compression (whose range is known for the current frame).
|
||||||
private final List<Sequence> sequences = new LinkedList<>();
|
private final List<Sequence> sequences = new LinkedList<>();
|
||||||
|
|
||||||
// time frame being/end
|
private Ordinal start, stop;
|
||||||
private long tBegin = Long.MAX_VALUE, tEnd = Long.MIN_VALUE;
|
|
||||||
private long min = tBegin, max = tEnd;
|
|
||||||
|
|
||||||
public void add(Sequence sequence) {
|
public void add(Sequence sequence) {
|
||||||
sequences.add(sequence);
|
sequences.add(sequence);
|
||||||
long ts = sequence.currentTimestamp();
|
Ordinal ordinal = sequence.ordinal();
|
||||||
if (min > ts) {
|
if (start == null) {
|
||||||
min = ts;
|
start = ordinal;
|
||||||
}
|
stop = ordinal;
|
||||||
if (max < ts) {
|
} else {
|
||||||
max = ts;
|
if (start.compareTo(ordinal) > 0) {
|
||||||
}
|
start = ordinal;
|
||||||
}
|
}
|
||||||
|
if (stop.compareTo(ordinal) < 0) {
|
||||||
public void setTimeFrame(long begin, long end) {
|
stop = ordinal;
|
||||||
if (tBegin > begin) {
|
}
|
||||||
tBegin = begin;
|
|
||||||
}
|
|
||||||
if (tEnd < end) {
|
|
||||||
tEnd = end;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -49,53 +44,27 @@ public class SequenceFrame {
|
||||||
* Returns the latest Sequence from the group that has its timestamp
|
* Returns the latest Sequence from the group that has its timestamp
|
||||||
* less than the given argument alongside its position in the list.
|
* less than the given argument alongside its position in the list.
|
||||||
*/
|
*/
|
||||||
public Tuple<Sequence, Integer> before(long timestamp, Comparable<Object> tiebreaker) {
|
public Tuple<Sequence, Integer> before(Ordinal ordinal) {
|
||||||
Sequence matchSeq = null;
|
return find(o -> o.compareTo(ordinal) < 0);
|
||||||
int matchPos = -1;
|
|
||||||
int position = -1;
|
|
||||||
for (Sequence sequence : sequences) {
|
|
||||||
position++;
|
|
||||||
// ts only comparison
|
|
||||||
if (sequence.currentTimestamp() < timestamp) {
|
|
||||||
matchSeq = sequence;
|
|
||||||
matchPos = position;
|
|
||||||
}
|
|
||||||
// apply tiebreaker (null first, that is null is less than any value)
|
|
||||||
else if (tiebreaker != null && sequence.currentTimestamp() == timestamp) {
|
|
||||||
Comparable<Object> tb = sequence.currentTiebreaker();
|
|
||||||
if (tb == null || tb.compareTo(tiebreaker) < 0) {
|
|
||||||
matchSeq = sequence;
|
|
||||||
matchPos = position;
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return matchSeq != null ? new Tuple<>(matchSeq, matchPos) : null;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the first Sequence from the group that has its timestamp
|
* Returns the first Sequence from the group that has its timestamp
|
||||||
* greater than the given argument alongside its position in the list.
|
* greater than the given argument alongside its position in the list.
|
||||||
*/
|
*/
|
||||||
public Tuple<Sequence, Integer> after(long timestamp, Comparable<Object> tiebreaker) {
|
public Tuple<Sequence, Integer> after(Ordinal ordinal) {
|
||||||
|
return find(o -> o.compareTo(ordinal) > 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Tuple<Sequence, Integer> find(Predicate<Ordinal> predicate) {
|
||||||
Sequence matchSeq = null;
|
Sequence matchSeq = null;
|
||||||
int matchPos = -1;
|
int matchPos = -1;
|
||||||
int position = -1;
|
int position = -1;
|
||||||
for (Sequence sequence : sequences) {
|
for (Sequence sequence : sequences) {
|
||||||
position++;
|
position++;
|
||||||
// ts only comparison
|
if (predicate.test(sequence.ordinal())) {
|
||||||
if (sequence.currentTimestamp() > timestamp) {
|
|
||||||
matchSeq = sequence;
|
matchSeq = sequence;
|
||||||
matchPos = position;
|
matchPos = position;
|
||||||
}
|
|
||||||
// apply tiebreaker (null first, that is null is less than any value)
|
|
||||||
else if (tiebreaker != null && sequence.currentTimestamp() == timestamp) {
|
|
||||||
Comparable<Object> tb = sequence.currentTiebreaker();
|
|
||||||
if (tb == null || tb.compareTo(tiebreaker) > 0) {
|
|
||||||
matchSeq = sequence;
|
|
||||||
matchPos = position;
|
|
||||||
}
|
|
||||||
} else {
|
} else {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -112,9 +81,9 @@ public class SequenceFrame {
|
||||||
|
|
||||||
// update min time
|
// update min time
|
||||||
if (sequences.isEmpty() == false) {
|
if (sequences.isEmpty() == false) {
|
||||||
min = sequences.get(0).currentTimestamp();
|
start = sequences.get(0).ordinal();
|
||||||
} else {
|
} else {
|
||||||
min = Long.MAX_VALUE;
|
stop = null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -124,6 +93,6 @@ public class SequenceFrame {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return format(null, "[{}-{}]({} seqs)", tBegin, tEnd, sequences.size());
|
return format(null, "[{}-{}]({} seqs)", start, stop, sequences.size());
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -7,6 +7,7 @@
|
||||||
package org.elasticsearch.xpack.eql.execution.sequence;
|
package org.elasticsearch.xpack.eql.execution.sequence;
|
||||||
|
|
||||||
import org.elasticsearch.common.collect.Tuple;
|
import org.elasticsearch.common.collect.Tuple;
|
||||||
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.search.SearchHit;
|
import org.elasticsearch.search.SearchHit;
|
||||||
import org.elasticsearch.xpack.eql.execution.search.Limit;
|
import org.elasticsearch.xpack.eql.execution.search.Limit;
|
||||||
|
|
||||||
|
@ -24,33 +25,25 @@ public class SequenceStateMachine {
|
||||||
/** Current keys on each stage */
|
/** Current keys on each stage */
|
||||||
private final StageToKeys stageToKeys;
|
private final StageToKeys stageToKeys;
|
||||||
|
|
||||||
/** minimum timestamp per stage */
|
|
||||||
/** this ignores the key */
|
|
||||||
private final long[] timestampMarkers;
|
|
||||||
|
|
||||||
private final Comparable<?>[] tiebreakerMarkers;
|
|
||||||
private final boolean hasTieBreaker;
|
|
||||||
|
|
||||||
private final int completionStage;
|
private final int completionStage;
|
||||||
|
|
||||||
/** list of completed sequences - separate to avoid polluting the other stages */
|
/** list of completed sequences - separate to avoid polluting the other stages */
|
||||||
private final List<Sequence> completed;
|
private final List<Sequence> completed;
|
||||||
|
private final long maxSpanInMillis;
|
||||||
|
|
||||||
private int offset = 0;
|
private int offset = 0;
|
||||||
private int limit = -1;
|
private int limit = -1;
|
||||||
private boolean limitReached = false;
|
private boolean limitReached = false;
|
||||||
|
|
||||||
@SuppressWarnings("rawtypes")
|
@SuppressWarnings("rawtypes")
|
||||||
public SequenceStateMachine(int stages, boolean hasTiebreaker, Limit limit) {
|
public SequenceStateMachine(int stages, TimeValue maxSpan, Limit limit) {
|
||||||
this.completionStage = stages - 1;
|
this.completionStage = stages - 1;
|
||||||
|
|
||||||
this.stageToKeys = new StageToKeys(completionStage);
|
this.stageToKeys = new StageToKeys(completionStage);
|
||||||
this.keyToSequences = new KeyToSequences(completionStage);
|
this.keyToSequences = new KeyToSequences(completionStage);
|
||||||
this.timestampMarkers = new long[completionStage];
|
|
||||||
this.tiebreakerMarkers = new Comparable[completionStage];
|
|
||||||
this.completed = new LinkedList<>();
|
this.completed = new LinkedList<>();
|
||||||
|
|
||||||
this.hasTieBreaker = hasTiebreaker;
|
this.maxSpanInMillis = maxSpan.millis();
|
||||||
|
|
||||||
// limit && offset
|
// limit && offset
|
||||||
if (limit != null) {
|
if (limit != null) {
|
||||||
|
@ -63,34 +56,11 @@ public class SequenceStateMachine {
|
||||||
return completed;
|
return completed;
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getTimestampMarker(int stage) {
|
public void trackSequence(Sequence sequence) {
|
||||||
return timestampMarkers[stage];
|
|
||||||
}
|
|
||||||
|
|
||||||
public Comparable<?> getTiebreakerMarker(int stage) {
|
|
||||||
return tiebreakerMarkers[stage];
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setTimestampMarker(int stage, long timestamp) {
|
|
||||||
timestampMarkers[stage] = timestamp;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setTiebreakerMarker(int stage, Comparable<Object> tiebreaker) {
|
|
||||||
tiebreakerMarkers[stage] = tiebreaker;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Object[] getMarkers(int stage) {
|
|
||||||
long ts = timestampMarkers[stage];
|
|
||||||
Comparable<?> tb = tiebreakerMarkers[stage];
|
|
||||||
return hasTieBreaker ? new Object[] { ts, tb } : new Object[] { ts };
|
|
||||||
}
|
|
||||||
|
|
||||||
public void trackSequence(Sequence sequence, long tStart, long tStop) {
|
|
||||||
SequenceKey key = sequence.key();
|
SequenceKey key = sequence.key();
|
||||||
|
|
||||||
stageToKeys.keys(0).add(key);
|
stageToKeys.keys(0).add(key);
|
||||||
SequenceFrame frame = keyToSequences.frame(0, key);
|
SequenceFrame frame = keyToSequences.frame(0, key);
|
||||||
frame.setTimeFrame(tStart, tStop);
|
|
||||||
frame.add(sequence);
|
frame.add(sequence);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -98,23 +68,27 @@ public class SequenceStateMachine {
|
||||||
* Match the given hit (based on key and timestamp and potential tiebreaker) with any potential sequence from the previous
|
* Match the given hit (based on key and timestamp and potential tiebreaker) with any potential sequence from the previous
|
||||||
* given stage. If that's the case, update the sequence and the rest of the references.
|
* given stage. If that's the case, update the sequence and the rest of the references.
|
||||||
*/
|
*/
|
||||||
public boolean match(int stage, SequenceKey key, long timestamp, Comparable<Object> tiebreaker, SearchHit hit) {
|
public boolean match(int stage, SequenceKey key, Ordinal ordinal, SearchHit hit) {
|
||||||
int previousStage = stage - 1;
|
int previousStage = stage - 1;
|
||||||
// check key presence to avoid creating a collection
|
// check key presence to avoid creating a collection
|
||||||
SequenceFrame frame = keyToSequences.frameIfPresent(previousStage, key);
|
SequenceFrame frame = keyToSequences.frameIfPresent(previousStage, key);
|
||||||
if (frame == null || frame.isEmpty()) {
|
if (frame == null || frame.isEmpty()) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
// pick the sequence with the highest (for ASC) / lowest (for DESC) timestamp lower than current match timestamp
|
Tuple<Sequence, Integer> before = frame.before(ordinal);
|
||||||
Tuple<Sequence, Integer> neighbour = frame.before(timestamp, tiebreaker);
|
if (before == null) {
|
||||||
if (neighbour == null) {
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
Sequence sequence = neighbour.v1();
|
Sequence sequence = before.v1();
|
||||||
// eliminate the match and all previous values from the frame
|
// eliminate the match and all previous values from the frame
|
||||||
frame.trim(neighbour.v2() + 1);
|
frame.trim(before.v2() + 1);
|
||||||
// update sequence
|
|
||||||
sequence.putMatch(stage, hit, timestamp, tiebreaker);
|
// check maxspan before continuing the sequence
|
||||||
|
if (maxSpanInMillis > 0 && (ordinal.timestamp - sequence.startTimestamp() >= maxSpanInMillis)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
sequence.putMatch(stage, hit, ordinal);
|
||||||
|
|
||||||
// remove the frame and keys early (as the key space is large)
|
// remove the frame and keys early (as the key space is large)
|
||||||
if (frame.isEmpty()) {
|
if (frame.isEmpty()) {
|
||||||
|
|
|
@ -7,6 +7,7 @@
|
||||||
package org.elasticsearch.xpack.eql.plan.physical;
|
package org.elasticsearch.xpack.eql.plan.physical;
|
||||||
|
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.xpack.eql.EqlIllegalArgumentException;
|
import org.elasticsearch.xpack.eql.EqlIllegalArgumentException;
|
||||||
import org.elasticsearch.xpack.eql.execution.assembler.ExecutionManager;
|
import org.elasticsearch.xpack.eql.execution.assembler.ExecutionManager;
|
||||||
import org.elasticsearch.xpack.eql.execution.search.Limit;
|
import org.elasticsearch.xpack.eql.execution.search.Limit;
|
||||||
|
@ -33,6 +34,7 @@ public class SequenceExec extends PhysicalPlan {
|
||||||
private final Attribute tiebreaker;
|
private final Attribute tiebreaker;
|
||||||
private final Limit limit;
|
private final Limit limit;
|
||||||
private final OrderDirection direction;
|
private final OrderDirection direction;
|
||||||
|
private final TimeValue maxSpan;
|
||||||
|
|
||||||
public SequenceExec(Source source,
|
public SequenceExec(Source source,
|
||||||
List<List<Attribute>> keys,
|
List<List<Attribute>> keys,
|
||||||
|
@ -41,8 +43,9 @@ public class SequenceExec extends PhysicalPlan {
|
||||||
PhysicalPlan until,
|
PhysicalPlan until,
|
||||||
Attribute timestamp,
|
Attribute timestamp,
|
||||||
Attribute tiebreaker,
|
Attribute tiebreaker,
|
||||||
OrderDirection direction) {
|
OrderDirection direction,
|
||||||
this(source, combine(matches, until), combine(keys, singletonList(untilKeys)), timestamp, tiebreaker, null, direction);
|
TimeValue maxSpan) {
|
||||||
|
this(source, combine(matches, until), combine(keys, singletonList(untilKeys)), timestamp, tiebreaker, null, direction, maxSpan);
|
||||||
}
|
}
|
||||||
|
|
||||||
private SequenceExec(Source source,
|
private SequenceExec(Source source,
|
||||||
|
@ -51,18 +54,20 @@ public class SequenceExec extends PhysicalPlan {
|
||||||
Attribute ts,
|
Attribute ts,
|
||||||
Attribute tb,
|
Attribute tb,
|
||||||
Limit limit,
|
Limit limit,
|
||||||
OrderDirection direction) {
|
OrderDirection direction,
|
||||||
|
TimeValue maxSpan) {
|
||||||
super(source, children);
|
super(source, children);
|
||||||
this.keys = keys;
|
this.keys = keys;
|
||||||
this.timestamp = ts;
|
this.timestamp = ts;
|
||||||
this.tiebreaker = tb;
|
this.tiebreaker = tb;
|
||||||
this.limit = limit;
|
this.limit = limit;
|
||||||
this.direction = direction;
|
this.direction = direction;
|
||||||
|
this.maxSpan = maxSpan;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected NodeInfo<SequenceExec> info() {
|
protected NodeInfo<SequenceExec> info() {
|
||||||
return NodeInfo.create(this, SequenceExec::new, children(), keys, timestamp, tiebreaker, limit, direction);
|
return NodeInfo.create(this, SequenceExec::new, children(), keys, timestamp, tiebreaker, limit, direction, maxSpan);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -72,7 +77,7 @@ public class SequenceExec extends PhysicalPlan {
|
||||||
children().size(),
|
children().size(),
|
||||||
newChildren.size());
|
newChildren.size());
|
||||||
}
|
}
|
||||||
return new SequenceExec(source(), newChildren, keys, timestamp, tiebreaker, limit, direction);
|
return new SequenceExec(source(), newChildren, keys, timestamp, tiebreaker, limit, direction, maxSpan);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -109,12 +114,13 @@ public class SequenceExec extends PhysicalPlan {
|
||||||
}
|
}
|
||||||
|
|
||||||
public SequenceExec with(Limit limit) {
|
public SequenceExec with(Limit limit) {
|
||||||
return new SequenceExec(source(), children(), keys(), timestamp(), tiebreaker(), limit, direction);
|
return new SequenceExec(source(), children(), keys(), timestamp(), tiebreaker(), limit, direction, maxSpan);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void execute(EqlSession session, ActionListener<Payload> listener) {
|
public void execute(EqlSession session, ActionListener<Payload> listener) {
|
||||||
new ExecutionManager(session).assemble(keys(), children(), timestamp(), tiebreaker(), direction, limit()).execute(listener);
|
new ExecutionManager(session).assemble(keys(), children(), timestamp(), tiebreaker(), direction, maxSpan, limit()).execute(
|
||||||
|
listener);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -78,7 +78,7 @@ class Mapper extends RuleExecutor<PhysicalPlan> {
|
||||||
map(s.until().child()),
|
map(s.until().child()),
|
||||||
s.timestamp(),
|
s.timestamp(),
|
||||||
s.tiebreaker(),
|
s.tiebreaker(),
|
||||||
s.direction());
|
s.direction(), s.maxSpan());
|
||||||
}
|
}
|
||||||
|
|
||||||
if (p instanceof LocalRelation) {
|
if (p instanceof LocalRelation) {
|
||||||
|
|
|
@ -36,11 +36,6 @@ public class EmptyPayload implements Payload {
|
||||||
return TimeValue.ZERO;
|
return TimeValue.ZERO;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object[] nextKeys() {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public <V> List<V> values() {
|
public <V> List<V> values() {
|
||||||
return emptyList();
|
return emptyList();
|
||||||
|
|
|
@ -22,7 +22,5 @@ public interface Payload {
|
||||||
|
|
||||||
TimeValue timeTook();
|
TimeValue timeTook();
|
||||||
|
|
||||||
Object[] nextKeys();
|
|
||||||
|
|
||||||
<V> List<V> values();
|
<V> List<V> values();
|
||||||
}
|
}
|
||||||
|
|
|
@ -80,7 +80,7 @@ public class SequenceRuntimeTests extends ESTestCase {
|
||||||
private final int ordinal;
|
private final int ordinal;
|
||||||
|
|
||||||
TestCriterion(int ordinal) {
|
TestCriterion(int ordinal) {
|
||||||
super(SearchSourceBuilder.searchSource().size(ordinal), keyExtractors, tsExtractor, tbExtractor);
|
super(SearchSourceBuilder.searchSource().size(ordinal), keyExtractors, tsExtractor, tbExtractor, false);
|
||||||
this.ordinal = ordinal;
|
this.ordinal = ordinal;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -144,11 +144,6 @@ public class SequenceRuntimeTests extends ESTestCase {
|
||||||
return TimeValue.ZERO;
|
return TimeValue.ZERO;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Object[] nextKeys() {
|
|
||||||
return new Object[0];
|
|
||||||
}
|
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
@Override
|
@Override
|
||||||
public <V> List<V> values() {
|
public <V> List<V> values() {
|
||||||
|
@ -191,7 +186,7 @@ public class SequenceRuntimeTests extends ESTestCase {
|
||||||
SequenceRuntime runtime = new SequenceRuntime(criteria, (r, l) -> {
|
SequenceRuntime runtime = new SequenceRuntime(criteria, (r, l) -> {
|
||||||
Map<Integer, Tuple<String, String>> evs = events.get(r.searchSource().size());
|
Map<Integer, Tuple<String, String>> evs = events.get(r.searchSource().size());
|
||||||
l.onResponse(new TestPayload(evs));
|
l.onResponse(new TestPayload(evs));
|
||||||
}, false, null);
|
}, TimeValue.MINUS_ONE, null);
|
||||||
|
|
||||||
// finally make the assertion at the end of the listener
|
// finally make the assertion at the end of the listener
|
||||||
runtime.execute(wrap(this::checkResults, ex -> {
|
runtime.execute(wrap(this::checkResults, ex -> {
|
||||||
|
|
|
@ -145,6 +145,5 @@ public class LogicalPlanTests extends ESTestCase {
|
||||||
|
|
||||||
TimeValue maxSpan = seq.maxSpan();
|
TimeValue maxSpan = seq.maxSpan();
|
||||||
assertEquals(new TimeValue(2, TimeUnit.SECONDS), maxSpan);
|
assertEquals(new TimeValue(2, TimeUnit.SECONDS), maxSpan);
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
Loading…
Reference in New Issue