- Notifications
You must be signed in to change notification settings - Fork 25.6k
Eql Sampling #85206
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Eql Sampling #85206
Conversation
costin left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Left some comments.
| final long[] ids = new long[len]; | ||
| for (int i = 0; i < len; i++) { | ||
| Object field = events.get(i).sourceAsMap().get(tiebreaker()); | ||
| Object field = events.get(i).sourceAsMap().get(tiebreaker() == null ? idField() : tiebreaker()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
tiebreaker() could be cached and the condition evaluated outside the loop - just like the previous change.
| import static java.util.Collections.singletonList; | ||
| | ||
| public class Join extends LogicalPlan { | ||
| public class Join extends Sampling { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Join and Sampling share common properties so it makes sense to reuse code between them however due to the difference in semantics, it's better the classes don't extend one another.
Sequence extends Join so if Join extends Sampling it means Sequence is a type of sampling.
Also Join currently exists as a logical plan without being exposed in the query .
My suggestion is to extract the common join properties into a separate class (say AbstractJoin) and keep Sequence/Join separately from Sample.
| : sequence | ||
| | join | ||
| | eventQuery | ||
| | sampling |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sample I think works better than sampling - we use sequence as oppose to sequencing, join instead of joining.
| return new SearchSourceBuilder(in); | ||
| } | ||
| } catch (IOException e) { | ||
| throw new UncheckedIOException(e); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Use EQL exceptions to wrap the underlying exception - adds consistency and proper handling.
| * Sets keys / terms to filter on in the final stage filtering (where actual events are gathered). | ||
| * Can be removed through null. | ||
| */ | ||
| public void singleKeysPair(final List<Object> compositeKeyValues, int maxStages) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
singleKeyPair
| * Sets keys / terms to filter on in an intermediate stage filtering. | ||
| * Can be removed through null. | ||
| */ | ||
| public void multipleKeysPairs(List<Map<String, Object>> values, List<String> previousCriterionKeys) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
multipleKeyPairs
| /* | ||
| * Not a great way of getting a copy of a SearchSourceBuilder | ||
| */ | ||
| private SearchSourceBuilder copySource() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Better to make this static and parameterize it - not for performance but to indicate that it copies any given source as oppose to copying the existing source and then returning a new instance.
Make the copy/modify/reassign code flow better.
| | ||
| import java.util.List; | ||
| | ||
| public class SamplingCriterion<Q extends QueryRequest> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
A common base Criterion class which contains the keys, key extraction and the key size is useful not just in sharing code but also in ExecutionManager.
Add EqlSampleDataLoader Introduce AbstractJoin having Join and Sample as subclasses Rename Sampling to Sample Rename some methods Parametrize copySource
…o sampling_in_eql
costin left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Left another round of comments.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
A better name might be samples (since sampling means the process of selecting the sample).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why is this relevant only for the Sequence but not for Sample?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Because in a Sample there is no notion of limit (so far at least) and what this is doing is to add a limit plan to the tree.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
does that mean that fetch_size is ignored for sample? Is there a global limit for join keys then or will sample just always return all the keys?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
fetch_size is different. What you probably meant is size. And yes, for now, there is no limit and it will return all keys.
I will add fetch_size support in a new commit. This one is about the size of the results page returned by the composite aggregation and it has a very similar meaning in sequences as well.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes, I had size in mind. Isn't support for size somewhat crucial? Otherwise a query might easily OOM due to too many samples.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 on addressing limit/size in a separate PR if it's non-simplistic - this one is already quite big.
Potentially open a separate branch from sample to avoid any impact on releases.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
SampleCriterion extends Criterion<AggregatedQueryRequest>
Since the AggregatedQueryRequest is not used anywhere else it could be renamed to SampleQueryRequest.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
SequenceCriterion extends Criterion<BoxedQueryRequest>
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
final var hits = ..
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
could use better names to avoid the comments - int reponseIndex / currentResponse, int groupIndex, currentGroup
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
var next = page.size() == MAX_PAGE_SIZE ? page : stack.pop(); log.trace("Final stage... getting next page of the " + (next == page ? "current" : "previous") + " page"); nextPage(listener, next); There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Does it make sense to introduce a new type for Sample?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It may make sense in the future, not sure. I added it. But the payload is still a List of Sequences (the result of a Sample has an identical structure to a Sequence) and it's a bit more involved to add a Sample payload.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This seems better suited as a logical optimization rule as a oppose to a folding one.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually, this is a leftover from one of the initial ideas on how to introduce bucket extractors. The name of this rule is a misnomer, it doesn't actually propagate the composite keys, but more creating extractors. I've removed this rule, the logic that's adding the extractors is in ExecutionManager.
4b8cdf9 to 8075b2b Compare | Pinging @elastic/es-ql (Team:QL) |
| Hi @astefan, I've created a changelog YAML for you. |
| @elasticmachine update branch |
| | ||
| import java.util.List; | ||
| | ||
| public class Criterion<Q extends QueryRequest> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This class does not seem to have much meaning after this refactoring, probably we can just remove it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually, I've moved to this class a common part of SampleCriterion and SequenceCriterion - the keySize.
Luegg left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Lots to digest here so I'm only leaving some preliminary comments for now.
| String[] splitNames = index.split(","); | ||
| int i = 0; | ||
| | ||
| while (shouldLoadData && i < splitNames.length) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think Stream.allMatch would help a lot here to communicate the intent. As in:
boolean shouldLoadData = Arrays.stream(index.split(",")) .allMatch( indexName -> provisioningClient.performRequest(new Request("HEAD", "/" + unqualifiedIndexName(indexName))) .getStatusLine() .getStatusCode() == 200 ); | private long[] extractIds(List<Map<String, Object>> events) { | ||
| final int len = events.size(); | ||
| final long[] ids = new long[len]; | ||
| String idField = tiebreaker() == null ? idField() : tiebreaker(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think this could be clearer if this logic is pushed to idField(). The default implementation can return tiebreaker() and EqlSampleTestCase.idField() stays as is.
| | ||
| @Override | ||
| protected String tiebreaker() { | ||
| return null; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why is tiebreaker not used for sampling queries?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In general, a sample has no sense of chronological status. A sample is an unordered sequence of events and, in a way, is a less restrictive sequence.
In this particular case (EqlSampleTestCase) the id (which should be unique for all documents in the testing scenario) makes more sense for identifying the matching documents than a tiebreaker.
| @@ -0,0 +1,252 @@ | |||
| [[queries]] | |||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
One case I'm missing (and currently fails) is a sample without a join key as in sample [any where true] [any where true].
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
another one is composition with head/tail. It looks like these two are ignored.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You're right. I'll add them.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@Luegg thinking about this some more, samples do not make sense without a join key. Without the restriction of chronological order and without an element that ties the two events together (the join key), the returned events are just a collection of random (technically not random) events.
I've added 0-join keys as a restriction.
Same for head and tail (or pipes in general) - a limitation for the moment. head and tail may be part of the size/limit story but these details haven't been ironed out yet.
| }); | ||
| } | ||
| | ||
| hasJoin.set(hasJoin.get() || plan.anyMatch(Sample.class::isInstance)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| hasJoin.set(hasJoin.get() || plan.anyMatch(Sample.class::isInstance)); | |
| boolean hasJoin = plan.anyMatch(AbstractJoin.class::isInstance) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I've done it the way it is in the PR because the first anyMatch is already traversing the tree and I wanted to take advantage of that step and not do the traversal twice in all cases.
| // get results through search (to keep using PIT) | ||
| client.fetchHits(hits(samples), ActionListeners.map(listener, listOfHits -> { | ||
| SamplePayload payload = new SamplePayload(samples, listOfHits, false, timeTook()); | ||
| return payload; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
payload can be inlined and the curly braces of the lambda can be dropped
| * Starting point of the iterator, which also goes through all the criterions and gathers initial results pages. | ||
| */ | ||
| private void advance(ActionListener<Payload> listener) { | ||
| int currentStage = stack.size(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
"stage" and "criteria" seems to be used interchangeably. I think it would be easier to understand if only one of the two terms is used in the code base. Stage is quite an overloaded term.
| * Sets keys / terms to filter on in an intermediate stage filtering. | ||
| * Can be removed through null. | ||
| */ | ||
| public void multipleKeyPairs(List<Map<String, Object>> values, List<String> previousCriterionKeys) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The use of "key" and "value" is somewhat inconsistent which makes it hard to follow what's being passed. As far as I understand, values is a list of join keys and previousCriterionKeys is the list of join key field names. Also, on the call site both arguments are referred to as just "keys": request.multipleKeyPairs(previousCriterion.keys(previousResults.hits), previousResults.keys).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In general, "key" is the name of the field, "value" is its value.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
does that mean that fetch_size is ignored for sample? Is there a global limit for join keys then or will sample just always return all the keys?
| } | ||
| } | ||
| | ||
| SearchSourceBuilder newSource = copySource(searchSource); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe it would be easier to create new search source builders from scratch instead of hacky copy. This way it would also be more explicit what needs to end up in the source for the according query. E.g. currently it also inherits the fields from the original query which is not really needed as far as I understand.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
When I took the decision on hacky copying the source, I realized it was close to impossible to copy a SearchSourceBuilder because of queryBuilder (which has many implementations). You'd have to copy a BoolQueryBuilder and ExistsQueryBuilder and other possible such builders we use in functions and operators.
| @elasticmachine update branch |
| @elasticmachine run elasticsearch-ci/part-2 |
| @elasticmachine run elasticsearch-ci/docs-check |
luigidellaquila left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Adding some comments
| request = criterion.firstQuery(); | ||
| } | ||
| | ||
| final SampleQueryRequest rr = request; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This can be avoided by making request final (it's never reassigned)
| InternalComposite composite = (InternalComposite) a; | ||
| log.trace("Advancing.... found [{}] hits", composite.getBuckets().size()); | ||
| Page nextPage = new Page(composite, rr); | ||
| if (nextPage != null && nextPage.size() > 0) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| if (nextPage != null && nextPage.size() > 0) { | |
| if (nextPage.size() > 0) { |
nextPage cannot be null here
| final SampleQueryRequest rr = request; | ||
| log.trace("Querying stage [{}] {}", currentStage, request); | ||
| client.query(request, wrap(r -> { | ||
| Aggregation a = r.getAggregations().get(COMPOSITE_AGG_NAME); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If I'm not wrong, this logic is exactly the same as nextPage() (apart from the log message), probably it's worth extracting it as a method and use it in both contexts
| | ||
| class SamplePayload extends AbstractPayload { | ||
| | ||
| private final List<org.elasticsearch.xpack.eql.action.EqlSearchResponse.Sequence> values; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Are values conceptually Sequences?
Structurally, for transport purposes, Sequence does the job, even though the events are supposed to be a Set (not a List).
Just wondering if it makes sense to keep the two concepts separate and have a specialized EqlSearchResponse.Sample
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, the response of a "sample" is identical in structure with the one of a "sequence".
| public void testMatchSample() { | ||
| assertEquals( | ||
| asSearchHitsList(2, 1, 3), | ||
| matchSample(asList(asSearchHitsList(1, 1, 2), asSearchHitsList(1, 1, 1), asSearchHitsList(1, 1, 3)), 3) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can these search hits contain duplicates in practice?
It makes sense to test the algorithm in this case as well, just wondering if the algorithm can be simplified if it's not true (ie. avoid the backtracking by sorting the searchHits lists by size)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I thought a bit about this, IMHO we should not test duplicates here. They are not allowed by construction (each query in the final reduce step should not produce duplicates), and they would also invalidate the general approach (ie. if you had duplicates, you could not rely on size=n and terminate_after=n in the final queries, without losing results).
The backtracking algorithm tolerates duplicate results, but IMHO it can be improved, and this test could fail with a different, yet valid, algorithm that does not tolerate duplicates.
| int initialSize = samples.size(); | ||
| client.multiQuery(searches, ActionListener.wrap(r -> { | ||
| List<List<SearchHit>> finalSamples = new ArrayList<>(); | ||
| List<List<SearchHit>> sample = new ArrayList<>(maxStages); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Probably we can use List<Set<SearchHit>> here. It would make some operations faster (ie. contains()) and it would make the intention more clear.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think a Set can be used here. The order of each sub-list of SearchHits should be kept. Each sub-list matches one filter of the query. The position of the SearchHit should correspond to the position of the query filter.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Isn't the order of requests kept in the first List...? The Set contains the results for a single query filter, that should not be so relevant as long as you can decide which filter you run first
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You're right. That order shouldn't matter.
costin left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM! This looks great!
| assert400BadRequest(test[0], test[1]); | ||
| } | ||
| | ||
| bulkIndex(""" |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
👍 for using """
| @Override | ||
| protected int requestFetchSize() { | ||
| // a more relevant fetch_size value for Samples, from algorithm point of view, so we'll mostly test this value | ||
| if (frequently()) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit - the ternary expression makes this a one-liner: return frequently() ? 2 : super.requestFetchSize()
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 on addressing limit/size in a separate PR if it's non-simplistic - this one is already quite big.
Potentially open a separate branch from sample to avoid any impact on releases.
| // search query | ||
| if (query instanceof EsQueryExec esQueryExec) { | ||
| SampleQueryRequest firstQuery = new SampleQueryRequest( | ||
| () -> wrapAsFilter(esQueryExec.source(session, false)), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please create an issue for it so it doesn't get lost.
| s.direction(), | ||
| s.maxSpan() | ||
| ); | ||
| if (p instanceof Sequence sequence) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit, switch the check: check on Sample and return it otherwise fallback to the existing code - less code modified.
| [any where port > 100] by op_sys | ||
| [any where bool == true] by os | ||
| ''' | ||
| expected_event_ids = [17,26,16, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
hey @astefan! it's been a while! 🙂
cool feature, looks similar to Endgame EQL join but without all the baggage that word brings, so I'm glad to see the name change.
I think it behaves how I would intuitively expect, just want to double check one property: how are the resulting events ordered? Is it (1) in declaration order, parallel to the structure of the query? Or is it (2) chronological order, not mirroring the structure.
I can understand arguments for both, and I think (1) makes the most intuitive sense to me and that was how the Endgame join syntax behaved. From a quick glance, it looks like this is the current behavior, which sounds like the right call.
Glad to see this happen, it's exciting!
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
hey @rw-access. Nice to see you in the PR :-) and thank you for the interest.
The events mirror the structure of the query. The position of one event corresponds to the same position of the filter it is matching.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
excellent! that's what it looked like. nice work with the feature!
bpintea left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Impressive work, LGTM!
| | ||
| queries.add(joinTerm); | ||
| int numberOfQueries = queries.size(); | ||
| if (numberOfQueries > 5) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: might be nice to extract this value into a constant. Also, I guess a doc PR will follow.
Then, given the "narrowing" nature of the keys-searching algorithm and capped composite page size (vs. just permutations search), it might be too conservative. But I guess that's future work.
| @SuppressWarnings("unchecked") | ||
| public List<Sequence> sequences() { | ||
| return type == Type.SEQUENCE ? (List<Sequence>) results : null; | ||
| return (type == Type.SEQUENCE || type == Type.SAMPLE) ? (List<Sequence>) results : null; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not strictly related to this line: I understand that the sample has practically the same format as a sequence and that a sample is a list of Sequences, but the result isn't a sequence at all - the order of events is influenced by the order of the rules, but that's an implementation detail - so I'd find it natural for the result to reflect that: i.e. the result's hits should contain samples instead of sequences, imo. Otoh, keeping it like this might be easier to implement by the clients.
| | ||
| public class Sample extends AbstractJoin { | ||
| | ||
| public Sample(Source source, List<KeyedFilter> queries, KeyedFilter... query) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The varadic argument is provided nowhere.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice catch! I've removed it.
| public void setup() throws Exception { | ||
| RestClient provisioningClient = provisioningClient(); | ||
| if (provisioningClient.performRequest(new Request("HEAD", "/" + unqualifiedIndexName())).getStatusLine().getStatusCode() == 404) { | ||
| boolean shouldLoadData = false == Arrays.stream(index.split(",")) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Optional alternative: do away with false == ... by renaming var to dataLoaded.
| assertThat(response.getHeader("Content-Type"), containsString(contentType)); | ||
| assertThat(EntityUtils.toString(response.getEntity()), containsString(test[1])); | ||
| assertThat(response.getStatusLine().getStatusCode(), is(400)); | ||
| assert400BadRequest(test[0], test[1]); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: method's only called here, could be replaced by assertBadRequest(..., 400)
| } | ||
| | ||
| /* | ||
| * Starting point of the iterator, which also goes through all the criterions and gathers initial results pages. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
criteria
luigidellaquila left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM, great work!
| @elasticmachine update branch |
A sample searches for events matching the declared filters in all possible permutations. The result of a sample is identical in structure with the one of a sequence, but for each combination of join key values, if there is at least one match, the result will contain only one events combination matching the sample (as opposed to sequences where all results are returned).