SQL: GROUP BY use composite instead of terms aggregation (elastic/x-pack-elasticsearch#4282)

Rewrote the GROUP BY to use composite aggregation instead of terms 
(and everything that comes with it) but instead rely on composite aggregation

This not only works better but simplifies the code complexity since
composite is a straight, two-level tree:

1. root/group-by/composite-keys
2. (metric) aggregations

This removes a lot of complexity from all stages that involve creating,
assembling and especially parsing the results.
By moving to composite agg, the aggregation/GROUP BY are now pageable
so the consumer/listener had to be extended to include a dedicated
cursor and specific (bucket) extractors inline with the scroll requests.

While at it, also improved the support for implicit GROUP BY by
formalizing it (previously it supported only counts and no other
agg).

In addition:
Fixed a JDBC bug that caused incorrect timeout to be passed
Improved the returned RowSet a bit and add better naming
Pick up @Nullable move from core
Make sure to specify the TimeZone for DateTimeHistogram extraction
Add missing javadoc
To avoid delegating NamedWriteableRegistry (NWR) and to keep the scope
clean, SQL writeables now handle their own serialization, keeping the
boundary between the Elasticsearch's NWR in place.
Pass NamedWriteableRegistry only when looking at the next page
To keep in line with the existing patter and simplify the code
bureaucracy, the deserialization happens directly.
Since the SearchSourceBuilder deserialization happens explicitly (and
it's otherwise opaque), the declarative invocation isn't necessary
anymore.
Add a bit more randomization in tests

Original commit: elastic/x-pack-elasticsearch@f5af046386
This commit is contained in:
Costin Leau 2018-04-12 20:20:10 +03:00 committed by GitHub
parent df61103b05
commit e5d18ad45c
126 changed files with 2747 additions and 2318 deletions

View File

@ -29,11 +29,12 @@ class JdbcStatement implements Statement, JdbcWrapper {
private boolean ignoreResultSetClose = false;
protected JdbcResultSet rs;
final RequestMeta requestMeta = new RequestMeta();
final RequestMeta requestMeta;
JdbcStatement(JdbcConnection jdbcConnection, JdbcConfiguration info) {
this.con = jdbcConnection;
this.cfg = info;
this.requestMeta = new RequestMeta(info.pageSize(), info.pageTimeout(), info.queryTimeout());
}
@Override
@ -111,7 +112,7 @@ class JdbcStatement implements Statement, JdbcWrapper {
@Override
public int getQueryTimeout() throws SQLException {
checkOpen();
return (int) TimeUnit.MILLISECONDS.toSeconds(requestMeta.timeoutInMs());
return (int) TimeUnit.MILLISECONDS.toSeconds(requestMeta.queryTimeoutInMs());
}
@Override
@ -120,7 +121,7 @@ class JdbcStatement implements Statement, JdbcWrapper {
if (seconds < 0) {
throw new SQLException("Query timeout must be positive");
}
requestMeta.timeout(TimeUnit.SECONDS.toMillis(seconds));
requestMeta.queryTimeout(TimeUnit.SECONDS.toMillis(seconds));
}
@Override
@ -153,7 +154,7 @@ class JdbcStatement implements Statement, JdbcWrapper {
return true;
}
// execute the query and handle the rs closing and initialization
// execute the query and handle the rs closing and initialization
protected void initResultSet(String sql, List<SqlTypedParamValue> params) throws SQLException {
// close previous result set
closeResultSet();
@ -190,7 +191,7 @@ class JdbcStatement implements Statement, JdbcWrapper {
@Override
public void setFetchDirection(int direction) throws SQLException {
checkOpen();
if (ResultSet.FETCH_REVERSE != direction
if (ResultSet.FETCH_REVERSE != direction
|| ResultSet.FETCH_FORWARD != direction
|| ResultSet.FETCH_UNKNOWN != direction) {
throw new SQLException("Invalid direction specified");
@ -215,14 +216,13 @@ class JdbcStatement implements Statement, JdbcWrapper {
@Override
public int getFetchSize() throws SQLException {
checkOpen();
int fetchSize = requestMeta.fetchSize();
// the spec is somewhat unclear. It looks like there are 3 states:
// unset (in this case -1 which the user cannot set) - in this case, the default fetch size is returned
// 0 meaning the hint is disabled (the user has called setFetch)
// >0 means actual hint
// tl;dr - if invalid, it means it was not set so return default - otherwise return the set value
return fetchSize < 0 ? cfg.pageSize() : fetchSize;
// tl;dr - unless the user set it, returning the default is fine
return requestMeta.fetchSize();
}
@Override

View File

@ -45,7 +45,7 @@ public class JdbcHttpClient {
public Cursor query(String sql, List<SqlTypedParamValue> params, RequestMeta meta) throws SQLException {
int fetch = meta.fetchSize() > 0 ? meta.fetchSize() : conCfg.pageSize();
SqlQueryRequest sqlRequest = new SqlQueryRequest(AbstractSqlRequest.Mode.JDBC, sql, params, null, DateTimeZone.UTC, fetch,
TimeValue.timeValueMillis(meta.timeoutInMs()), TimeValue.timeValueMillis(meta.timeoutInMs()), "");
TimeValue.timeValueMillis(meta.timeoutInMs()), TimeValue.timeValueMillis(meta.queryTimeoutInMs()), "");
SqlQueryResponse response = httpClient.query(sqlRequest);
return new DefaultCursor(this, response.cursor(), toJdbcColumnInfo(response.columns()), response.rows(), meta);
}
@ -55,11 +55,10 @@ public class JdbcHttpClient {
* the scroll id to use to fetch the next page.
*/
public Tuple<String, List<List<Object>>> nextPage(String cursor, RequestMeta meta) throws SQLException {
TimeValue timeValue = TimeValue.timeValueMillis(meta.timeoutInMs());
SqlQueryRequest sqlRequest = new SqlQueryRequest().cursor(cursor);
sqlRequest.mode(AbstractSqlRequest.Mode.JDBC);
sqlRequest.requestTimeout(timeValue);
sqlRequest.pageTimeout(timeValue);
sqlRequest.requestTimeout(TimeValue.timeValueMillis(meta.timeoutInMs()));
sqlRequest.pageTimeout(TimeValue.timeValueMillis(meta.queryTimeoutInMs()));
SqlQueryResponse response = httpClient.query(sqlRequest);
return new Tuple<>(response.cursor(), response.rows());
}

View File

@ -9,14 +9,17 @@ public class RequestMeta {
private int fetchSize;
private long timeoutInMs;
private long queryTimeoutInMs;
public RequestMeta() {
this(-1, 0);
}
public RequestMeta(int fetchSize, int timeout) {
public RequestMeta(int fetchSize, long timeout, long queryTimeoutInMs) {
this.fetchSize = fetchSize;
this.timeoutInMs = timeout;
this.queryTimeoutInMs = queryTimeoutInMs;
}
public RequestMeta queryTimeout(long timeout) {
this.queryTimeoutInMs = timeout;
return this;
}
public RequestMeta timeout(long timeout) {
@ -37,4 +40,7 @@ public class RequestMeta {
return timeoutInMs;
}
public long queryTimeoutInMs() {
return queryTimeoutInMs;
}
}

View File

@ -7,7 +7,7 @@ package org.elasticsearch.xpack.sql;
import org.elasticsearch.ElasticsearchException;
public class SqlException extends ElasticsearchException {
public abstract class SqlException extends ElasticsearchException {
public SqlException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
super(message, cause, enableSuppression, writableStackTrace);
}

View File

@ -5,13 +5,13 @@
*/
package org.elasticsearch.xpack.sql.analysis.analyzer;
import org.elasticsearch.common.util.Comparators;
import org.elasticsearch.xpack.sql.analysis.AnalysisException;
import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier.Failure;
import org.elasticsearch.xpack.sql.analysis.index.IndexResolution;
import org.elasticsearch.xpack.sql.capabilities.Resolvables;
import org.elasticsearch.xpack.sql.expression.Alias;
import org.elasticsearch.xpack.sql.expression.Attribute;
import org.elasticsearch.xpack.sql.expression.AttributeMap;
import org.elasticsearch.xpack.sql.expression.AttributeSet;
import org.elasticsearch.xpack.sql.expression.Expression;
import org.elasticsearch.xpack.sql.expression.Expressions;
@ -55,7 +55,6 @@ import org.joda.time.DateTimeZone;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Comparator;
import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
import java.util.List;
@ -110,7 +109,7 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
new ResolveFunctions(),
new ResolveAliases(),
new ProjectedAggregations(),
new ResolveAggsInHavingAndOrderBy()
new ResolveAggsInHaving()
//new ImplicitCasting()
);
// TODO: this might be removed since the deduplication happens already in ResolveFunctions
@ -159,7 +158,7 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
// Shared methods around the analyzer rules
//
private static Attribute resolveAgainstList(UnresolvedAttribute u, List<Attribute> attrList) {
private static Attribute resolveAgainstList(UnresolvedAttribute u, Collection<Attribute> attrList) {
List<Attribute> matches = new ArrayList<>();
// first take into account the qualified version
@ -190,7 +189,7 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
}
return u.withUnresolvedMessage("Reference [" + u.qualifiedName()
+ "] is ambiguous (to disambiguate use quotes or qualifiers); matches any of " +
+ "] is ambiguous (to disambiguate use quotes or qualifiers); matches any of " +
matches.stream()
.map(a -> "\"" + a.qualifier() + "\".\"" + a.name() + "\"")
.sorted()
@ -303,16 +302,15 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
if (!a.expressionsResolved() && Resolvables.resolved(a.aggregates())) {
List<Expression> groupings = a.groupings();
List<Expression> newGroupings = new ArrayList<>();
List<Attribute> resolved = Expressions.asAttributes(a.aggregates());
AttributeMap<Expression> resolved = Expressions.asAttributeMap(a.aggregates());
boolean changed = false;
for (int i = 0; i < groupings.size(); i++) {
Expression grouping = groupings.get(i);
for (Expression grouping : groupings) {
if (grouping instanceof UnresolvedAttribute) {
Attribute maybeResolved = resolveAgainstList((UnresolvedAttribute) grouping, resolved);
Attribute maybeResolved = resolveAgainstList((UnresolvedAttribute) grouping, resolved.keySet());
if (maybeResolved != null) {
changed = true;
// use the matched expression (not its attribute)
grouping = a.aggregates().get(i);
grouping = resolved.get(maybeResolved);
}
}
newGroupings.add(grouping);
@ -883,10 +881,10 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
};
//
// Handle aggs in HAVING and ORDER BY clause. To help folding any aggs not found in Aggregation
// Handle aggs in HAVING. To help folding any aggs not found in Aggregation
// will be pushed down to the Aggregate and then projected. This also simplifies the Verifier's job.
//
private class ResolveAggsInHavingAndOrderBy extends AnalyzeRule<LogicalPlan> {
private class ResolveAggsInHaving extends AnalyzeRule<LogicalPlan> {
@Override
protected boolean skipResolved() {

View File

@ -353,7 +353,7 @@ abstract class Verifier {
return true;
} else if (e instanceof Score) {
// Score can't be an aggretate function
// Score can't be an aggregate function
missing.put(e, source);
return true;
}

View File

@ -7,6 +7,7 @@ package org.elasticsearch.xpack.sql.execution;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.xpack.sql.analysis.analyzer.PreAnalyzer;
import org.elasticsearch.xpack.sql.analysis.index.IndexResolver;
@ -27,6 +28,7 @@ import java.util.List;
public class PlanExecutor {
private final Client client;
private final NamedWriteableRegistry writableRegistry;
private final FunctionRegistry functionRegistry;
@ -35,8 +37,10 @@ public class PlanExecutor {
private final Optimizer optimizer;
private final Planner planner;
public PlanExecutor(Client client, IndexResolver indexResolver) {
public PlanExecutor(Client client, IndexResolver indexResolver, NamedWriteableRegistry writeableRegistry) {
this.client = client;
this.writableRegistry = writeableRegistry;
this.indexResolver = indexResolver;
this.functionRegistry = new FunctionRegistry();
@ -45,6 +49,10 @@ public class PlanExecutor {
this.planner = new Planner();
}
public NamedWriteableRegistry writableRegistry() {
return writableRegistry;
}
private SqlSession newSession(Configuration cfg) {
return new SqlSession(cfg, client, functionRegistry, indexResolver, preAnalyzer, optimizer, planner);
}
@ -65,7 +73,7 @@ public class PlanExecutor {
}
public void nextPage(Configuration cfg, Cursor cursor, ActionListener<RowSet> listener) {
cursor.nextPage(cfg, client, listener);
cursor.nextPage(cfg, client, writableRegistry, listener);
}
public void cleanCursor(Configuration cfg, Cursor cursor, ActionListener<Boolean> listener) {

View File

@ -0,0 +1,22 @@
/*
* 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.sql.execution.search;
/**
* Reference to a ES aggregation (which can be either a GROUP BY or Metric agg).
*/
public abstract class AggRef implements FieldExtraction {
@Override
public void collectFields(SqlSourceBuilder sourceBuilder) {
// Aggregations do not need any special fields
}
@Override
public boolean supportedByAggsOnlyQuery() {
return true;
}
}

View File

@ -1,133 +0,0 @@
/*
* 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.sql.execution.search;
import java.util.Arrays;
import java.util.List;
/**
* Aggregations are returned by Elasticsearch in a tree structure where each
* nested level can have a different size. For example a group by a, b, c
* results in 3-level nested array where each level contains all the relevant
* values for its parent entry.
* <p>
* Assuming there's a total of 2 A's, 3 B's and 5 C's, the values will be
* <ul>
* <li>A-agg level = { A1, A2 }
* <li>B-agg level = { { A1B1, A1B2, A1B3 }, { A2B1, A2B2, A2B3 }
* <li>C-agg level = { { { A1B1C1, A1B1C2 ..}, { A1B2C1, etc... } } } and so on
* </ul>
* <p>
* Further more the columns are added in the order in which they are requested
* (0, 1, 2) eliminating the need for keys as these are implicit (their position
* in the list).
* <p>
* To help with the iteration, there are two dedicated counters:
* one that carries (increments) the counter for each level (indicated by the
* position inside the array) once the children reach their max a flat cursor
* to indicate the row.
*/
class AggValues {
private int row = 0;
private final List<Object[]> columns;
private int[] indexPerLevel;
private int size;
AggValues(List<Object[]> columns) {
this.columns = columns;
}
void init(int maxDepth, int limit) {
int sz = computeSize(columns, maxDepth);
size = limit > 0 ? Math.min(limit, sz) : sz;
indexPerLevel = new int[maxDepth + 1];
}
private static int computeSize(List<Object[]> columns, int maxDepth) {
// look only at arrays with the right depth (the others might be
// counters or other functions)
// then return the parent array to compute the actual returned results
Object[] leafArray = null;
for (int i = 0; i < columns.size() && leafArray == null; i++) {
Object[] col = columns.get(i);
Object o = col;
int level = 0;
Object[] parent = null;
// keep unwrapping until the desired level is reached
while (o instanceof Object[]) {
col = ((Object[]) o);
if (col.length > 0) {
if (level == maxDepth) {
leafArray = parent;
break;
} else {
parent = col;
level++;
o = col[0];
}
} else {
o = null;
}
}
}
if (leafArray == null) {
return columns.get(0).length;
}
int sz = 0;
for (Object leaf : leafArray) {
sz += ((Object[]) leaf).length;
}
return sz;
}
Object column(int column) {
Object o = columns.get(column);
for (int lvl = 0; o instanceof Object[]; lvl++) {
Object[] arr = (Object[]) o;
// the current branch is done
if (indexPerLevel[lvl] == arr.length) {
// reset the current branch
indexPerLevel[lvl] = 0;
// bump the parent - if it's too big it, the loop will restart
// again from that position
indexPerLevel[lvl - 1]++;
// restart the loop
lvl = -1;
o = columns.get(column);
} else {
o = arr[indexPerLevel[lvl]];
}
}
return o;
}
int size() {
return size;
}
void reset() {
row = 0;
Arrays.fill(indexPerLevel, 0);
}
boolean nextRow() {
if (row < size - 1) {
row++;
// increment leaf counter - the size check is done lazily while retrieving the columns
indexPerLevel[indexPerLevel.length - 1]++;
return true;
}
return false;
}
boolean hasCurrentRow() {
return row < size;
}
}

View File

@ -1,62 +0,0 @@
/*
* 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.sql.execution.search;
import org.elasticsearch.xpack.sql.session.AbstractRowSet;
import org.elasticsearch.xpack.sql.session.Cursor;
import org.elasticsearch.xpack.sql.session.SchemaRowSet;
import org.elasticsearch.xpack.sql.type.Schema;
import java.util.List;
import java.util.function.Supplier;
class AggsRowSet extends AbstractRowSet implements SchemaRowSet {
private final Schema schema;
private final AggValues agg;
private final List<Supplier<Object>> columns;
private int row = 0;
AggsRowSet(Schema schema, AggValues agg, List<Supplier<Object>> columns) {
this.schema = schema;
this.agg = agg;
this.columns = columns;
}
@Override
protected Object getColumn(int column) {
return columns.get(column).get();
}
@Override
protected boolean doHasCurrent() {
return row < size();
}
@Override
protected boolean doNext() {
return agg.nextRow();
}
@Override
protected void doReset() {
agg.reset();
}
@Override
public int size() {
return agg.size();
}
@Override
public Cursor nextPageCursor() {
return Cursor.EMPTY;
}
@Override
public Schema schema() {
return schema;
}
}

View File

@ -0,0 +1,208 @@
/*
* 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.sql.execution.search;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation;
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.elasticsearch.xpack.sql.execution.search.extractor.BucketExtractor;
import org.elasticsearch.xpack.sql.querydsl.agg.Aggs;
import org.elasticsearch.xpack.sql.session.Configuration;
import org.elasticsearch.xpack.sql.session.Cursor;
import org.elasticsearch.xpack.sql.session.RowSet;
import org.elasticsearch.xpack.sql.util.StringUtils;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Objects;
/**
* Cursor for composite aggregation (GROUP BY).
* Stores the query that gets updated/slides across requests.
*/
public class CompositeAggregationCursor implements Cursor {
private final Logger log = Loggers.getLogger(getClass());
public static final String NAME = "c";
private final String[] indices;
private final byte[] nextQuery;
private final List<BucketExtractor> extractors;
private final int limit;
CompositeAggregationCursor(byte[] next, List<BucketExtractor> exts, int remainingLimit, String... indices) {
this.indices = indices;
this.nextQuery = next;
this.extractors = exts;
this.limit = remainingLimit;
}
public CompositeAggregationCursor(StreamInput in) throws IOException {
indices = in.readStringArray();
nextQuery = in.readByteArray();
limit = in.readVInt();
extractors = in.readNamedWriteableList(BucketExtractor.class);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeStringArray(indices);
out.writeByteArray(nextQuery);
out.writeVInt(limit);
out.writeNamedWriteableList(extractors);
}
@Override
public String getWriteableName() {
return NAME;
}
String[] indices() {
return indices;
}
byte[] next() {
return nextQuery;
}
List<BucketExtractor> extractors() {
return extractors;
}
int limit() {
return limit;
}
@Override
public void nextPage(Configuration cfg, Client client, NamedWriteableRegistry registry, ActionListener<RowSet> listener) {
SearchSourceBuilder q;
try {
q = deserializeQuery(registry, nextQuery);
} catch (Exception ex) {
listener.onFailure(ex);
return;
}
SearchSourceBuilder query = q;
if (log.isTraceEnabled()) {
log.trace("About to execute composite query {} on {}", StringUtils.toString(query), indices);
}
SearchRequest search = Querier.prepareRequest(client, query, cfg.pageTimeout(), indices);
client.search(search, ActionListener.wrap(r -> {
updateCompositeAfterKey(r, query);
CompositeAggsRowSet rowSet = new CompositeAggsRowSet(extractors, r, limit,
serializeQuery(query), indices);
listener.onResponse(rowSet);
}, listener::onFailure));
}
static CompositeAggregation getComposite(SearchResponse response) {
Aggregation agg = response.getAggregations().get(Aggs.ROOT_GROUP_NAME);
if (agg == null) {
return null;
}
if (agg instanceof CompositeAggregation) {
return (CompositeAggregation) agg;
}
throw new SqlIllegalArgumentException("Unrecognized root group found; {}", agg.getClass());
}
static void updateCompositeAfterKey(SearchResponse r, SearchSourceBuilder next) {
CompositeAggregation composite = getComposite(r);
if (composite == null) {
throw new SqlIllegalArgumentException("Invalid server response; no group-by detected");
}
Map<String, Object> afterKey = composite.afterKey();
// a null after-key means done
if (afterKey != null) {
AggregationBuilder aggBuilder = next.aggregations().getAggregatorFactories().get(0);
// update after-key with the new value
if (aggBuilder instanceof CompositeAggregationBuilder) {
CompositeAggregationBuilder comp = (CompositeAggregationBuilder) aggBuilder;
comp.aggregateAfter(afterKey);
} else {
throw new SqlIllegalArgumentException("Invalid client request; expected a group-by but instead got {}", aggBuilder);
}
}
}
/**
* Deserializes the search source from a byte array.
*/
static SearchSourceBuilder deserializeQuery(NamedWriteableRegistry registry, byte[] source) throws IOException {
try (NamedWriteableAwareStreamInput in = new NamedWriteableAwareStreamInput(StreamInput.wrap(source), registry)) {
return new SearchSourceBuilder(in);
}
}
/**
* Serializes the search source to a byte array.
*/
static byte[] serializeQuery(SearchSourceBuilder source) throws IOException {
if (source == null) {
return new byte[0];
}
try (BytesStreamOutput out = new BytesStreamOutput()) {
source.writeTo(out);
return BytesReference.toBytes(out.bytes());
}
}
@Override
public void clear(Configuration cfg, Client client, ActionListener<Boolean> listener) {
listener.onResponse(true);
}
@Override
public int hashCode() {
return Objects.hash(Arrays.hashCode(indices), Arrays.hashCode(nextQuery), extractors, limit);
}
@Override
public boolean equals(Object obj) {
if (obj == null || obj.getClass() != getClass()) {
return false;
}
CompositeAggregationCursor other = (CompositeAggregationCursor) obj;
return Arrays.equals(indices, other.indices)
&& Arrays.equals(nextQuery, other.nextQuery)
&& Objects.equals(extractors, other.extractors)
&& Objects.equals(limit, other.limit);
}
@Override
public String toString() {
return "cursor for composite on index [" + Arrays.toString(indices) + "]";
}
}

View File

@ -0,0 +1,100 @@
/*
* 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.sql.execution.search;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregation;
import org.elasticsearch.xpack.sql.execution.search.extractor.BucketExtractor;
import org.elasticsearch.xpack.sql.session.AbstractRowSet;
import org.elasticsearch.xpack.sql.session.Cursor;
import org.elasticsearch.xpack.sql.session.RowSet;
import java.util.List;
import static java.util.Collections.emptyList;
/**
* {@link RowSet} specific to (GROUP BY) aggregation.
*/
class CompositeAggsRowSet extends AbstractRowSet {
private final List<BucketExtractor> exts;
private final List<? extends CompositeAggregation.Bucket> buckets;
private final Cursor cursor;
private final int size;
private int row = 0;
CompositeAggsRowSet(List<BucketExtractor> exts, SearchResponse response, int limit, byte[] next, String... indices) {
this.exts = exts;
CompositeAggregation composite = CompositeAggregationCursor.getComposite(response);
if (composite != null) {
buckets = composite.getBuckets();
} else {
buckets = emptyList();
}
// page size
size = limit < 0 ? buckets.size() : Math.min(buckets.size(), limit);
if (next == null) {
cursor = Cursor.EMPTY;
} else {
// compute remaining limit
int remainingLimit = limit - size;
// if the computed limit is zero, or the size is zero it means either there's nothing left or the limit has been reached
// note that a composite agg might be valid but return zero groups (since these can be filtered with HAVING/bucket selector)
// however the Querier takes care of that and keeps making requests until either the query is invalid or at least one response
// is returned
if (next == null || size == 0 || remainingLimit == 0) {
cursor = Cursor.EMPTY;
} else {
cursor = new CompositeAggregationCursor(next, exts, remainingLimit, indices);
}
}
}
@Override
protected Object getColumn(int column) {
return exts.get(column).extract(buckets.get(row));
}
@Override
public int columnCount() {
return exts.size();
}
@Override
protected boolean doHasCurrent() {
return row < size;
}
@Override
protected boolean doNext() {
if (row < size - 1) {
row++;
return true;
}
return false;
}
@Override
protected void doReset() {
row = 0;
}
@Override
public int size() {
return size;
}
@Override
public Cursor nextPageCursor() {
return cursor;
}
}

View File

@ -24,13 +24,4 @@ public interface FieldExtraction {
* ({@code true}) or should it force a scroll query ({@code false})?
*/
boolean supportedByAggsOnlyQuery();
/**
* Indicates the depth of the result. Used for counting the actual size of a
* result by knowing how many nested levels there are. Typically used by
* aggregations.
*
* @return depth of the result
*/
int depth();
}

View File

@ -0,0 +1,437 @@
/*
* 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.sql.execution.search;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
import org.elasticsearch.search.aggregations.bucket.filter.Filters;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.elasticsearch.xpack.sql.execution.search.extractor.BucketExtractor;
import org.elasticsearch.xpack.sql.execution.search.extractor.CompositeKeyExtractor;
import org.elasticsearch.xpack.sql.execution.search.extractor.ComputingExtractor;
import org.elasticsearch.xpack.sql.execution.search.extractor.ConstantExtractor;
import org.elasticsearch.xpack.sql.execution.search.extractor.FieldHitExtractor;
import org.elasticsearch.xpack.sql.execution.search.extractor.HitExtractor;
import org.elasticsearch.xpack.sql.execution.search.extractor.MetricAggExtractor;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition.AggExtractorInput;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition.AggPathInput;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition.HitExtractorInput;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition.ProcessorDefinition;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition.ReferenceInput;
import org.elasticsearch.xpack.sql.querydsl.agg.Aggs;
import org.elasticsearch.xpack.sql.querydsl.container.ComputedRef;
import org.elasticsearch.xpack.sql.querydsl.container.GlobalCountRef;
import org.elasticsearch.xpack.sql.querydsl.container.GroupByRef;
import org.elasticsearch.xpack.sql.querydsl.container.MetricAggRef;
import org.elasticsearch.xpack.sql.querydsl.container.QueryContainer;
import org.elasticsearch.xpack.sql.querydsl.container.ScriptFieldRef;
import org.elasticsearch.xpack.sql.querydsl.container.SearchHitFieldRef;
import org.elasticsearch.xpack.sql.session.Configuration;
import org.elasticsearch.xpack.sql.session.Rows;
import org.elasticsearch.xpack.sql.session.SchemaRowSet;
import org.elasticsearch.xpack.sql.type.Schema;
import org.elasticsearch.xpack.sql.util.StringUtils;
import java.io.IOException;
import java.util.ArrayList;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Set;
import static java.util.Collections.singletonList;
// TODO: add retry/back-off
public class Querier {
private final Logger log = Loggers.getLogger(getClass());
private final TimeValue keepAlive, timeout;
private final int size;
private final Client client;
@Nullable
private final QueryBuilder filter;
public Querier(Client client, Configuration cfg) {
this(client, cfg.requestTimeout(), cfg.pageTimeout(), cfg.filter(), cfg.pageSize());
}
public Querier(Client client, TimeValue keepAlive, TimeValue timeout, QueryBuilder filter, int size) {
this.client = client;
this.keepAlive = keepAlive;
this.timeout = timeout;
this.filter = filter;
this.size = size;
}
public void query(Schema schema, QueryContainer query, String index, ActionListener<SchemaRowSet> listener) {
// prepare the request
SearchSourceBuilder sourceBuilder = SourceGenerator.sourceBuilder(query, filter, size);
// set query timeout
if (timeout.getSeconds() > 0) {
sourceBuilder.timeout(timeout);
}
if (log.isTraceEnabled()) {
log.trace("About to execute query {} on {}", StringUtils.toString(sourceBuilder), index);
}
SearchRequest search = prepareRequest(client, sourceBuilder, timeout, index);
ActionListener<SearchResponse> l;
if (query.isAggsOnly()) {
if (query.aggs().useImplicitGroupBy()) {
l = new ImplicitGroupActionListener(listener, client, timeout, schema, query, search);
} else {
l = new CompositeActionListener(listener, client, timeout, schema, query, search);
}
} else {
search.scroll(keepAlive);
l = new ScrollActionListener(listener, client, timeout, schema, query);
}
client.search(search, l);
}
public static SearchRequest prepareRequest(Client client, SearchSourceBuilder source, TimeValue timeout, String... indices) {
SearchRequest search = client.prepareSearch(indices).setSource(source).setTimeout(timeout).request();
search.allowPartialSearchResults(false);
return search;
}
/**
* Dedicated listener for implicit/default group-by queries that return only _one_ result.
*/
static class ImplicitGroupActionListener extends BaseAggActionListener {
private static List<? extends Bucket> EMPTY_BUCKET = singletonList(new Bucket() {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
throw new SqlIllegalArgumentException("No group-by/aggs defined");
}
@Override
public Object getKey() {
throw new SqlIllegalArgumentException("No group-by/aggs defined");
}
@Override
public String getKeyAsString() {
throw new SqlIllegalArgumentException("No group-by/aggs defined");
}
@Override
public long getDocCount() {
throw new SqlIllegalArgumentException("No group-by/aggs defined");
}
@Override
public Aggregations getAggregations() {
throw new SqlIllegalArgumentException("No group-by/aggs defined");
}
});
ImplicitGroupActionListener(ActionListener<SchemaRowSet> listener, Client client, TimeValue keepAlive, Schema schema,
QueryContainer query, SearchRequest request) {
super(listener, client, keepAlive, schema, query, request);
}
@Override
protected void handleResponse(SearchResponse response, ActionListener<SchemaRowSet> listener) {
Aggregations aggs = response.getAggregations();
if (aggs != null) {
Aggregation agg = aggs.get(Aggs.ROOT_GROUP_NAME);
if (agg instanceof Filters) {
handleBuckets(((Filters) agg).getBuckets(), response);
} else {
throw new SqlIllegalArgumentException("Unrecognized root group found; {}", agg.getClass());
}
}
// can happen when only a count is requested which is derived from the response
else {
handleBuckets(EMPTY_BUCKET, response);
}
}
private void handleBuckets(List<? extends Bucket> buckets, SearchResponse response) {
if (buckets.size() == 1) {
Bucket implicitGroup = buckets.get(0);
List<BucketExtractor> extractors = initBucketExtractors(response);
Object[] values = new Object[extractors.size()];
for (int i = 0; i < values.length; i++) {
values[i] = extractors.get(i).extract(implicitGroup);
}
listener.onResponse(Rows.singleton(schema, values));
} else if (buckets.isEmpty()) {
listener.onResponse(Rows.empty(schema));
} else {
throw new SqlIllegalArgumentException("Too many groups returned by the implicit group; expected 1, received {}",
buckets.size());
}
}
}
/**
* Dedicated listener for composite aggs/group-by results.
*/
static class CompositeActionListener extends BaseAggActionListener {
CompositeActionListener(ActionListener<SchemaRowSet> listener, Client client, TimeValue keepAlive,
Schema schema, QueryContainer query, SearchRequest request) {
super(listener, client, keepAlive, schema, query, request);
}
@Override
protected void handleResponse(SearchResponse response, ActionListener<SchemaRowSet> listener) {
// there are some results
if (response.getAggregations().asList().size() > 0) {
CompositeAggregationCursor.updateCompositeAfterKey(response, request.source());
byte[] nextSearch = null;
try {
nextSearch = CompositeAggregationCursor.serializeQuery(request.source());
} catch (Exception ex) {
listener.onFailure(ex);
return;
}
listener.onResponse(
new SchemaCompositeAggsRowSet(schema, initBucketExtractors(response), response, query.limit(),
nextSearch,
request.indices()));
}
// no results
else {
listener.onResponse(Rows.empty(schema));
}
}
}
abstract static class BaseAggActionListener extends BaseActionListener {
final QueryContainer query;
final SearchRequest request;
BaseAggActionListener(ActionListener<SchemaRowSet> listener, Client client, TimeValue keepAlive, Schema schema,
QueryContainer query, SearchRequest request) {
super(listener, client, keepAlive, schema);
this.query = query;
this.request = request;
}
protected List<BucketExtractor> initBucketExtractors(SearchResponse response) {
// create response extractors for the first time
List<FieldExtraction> refs = query.columns();
List<BucketExtractor> exts = new ArrayList<>(refs.size());
for (FieldExtraction ref : refs) {
exts.add(createExtractor(ref, new ConstantExtractor(response.getHits().getTotalHits())));
}
return exts;
}
private BucketExtractor createExtractor(FieldExtraction ref, BucketExtractor totalCount) {
if (ref instanceof GroupByRef) {
GroupByRef r = (GroupByRef) ref;
return new CompositeKeyExtractor(r.key(), r.property(), r.timeZone());
}
if (ref instanceof MetricAggRef) {
MetricAggRef r = (MetricAggRef) ref;
return new MetricAggExtractor(r.name(), r.property(), r.innerKey());
}
if (ref == GlobalCountRef.INSTANCE) {
return totalCount;
}
if (ref instanceof ComputedRef) {
ProcessorDefinition proc = ((ComputedRef) ref).processor();
// wrap only agg inputs
proc = proc.transformDown(l -> {
BucketExtractor be = createExtractor(l.context(), totalCount);
return new AggExtractorInput(l.location(), l.expression(), l.action(), be);
}, AggPathInput.class);
return new ComputingExtractor(proc.asProcessor());
}
throw new SqlIllegalArgumentException("Unexpected value reference {}", ref.getClass());
}
}
/**
* Dedicated listener for column retrieval/non-grouped queries (scrolls).
*/
static class ScrollActionListener extends BaseActionListener {
private final QueryContainer query;
ScrollActionListener(ActionListener<SchemaRowSet> listener, Client client, TimeValue keepAlive,
Schema schema, QueryContainer query) {
super(listener, client, keepAlive, schema);
this.query = query;
}
@Override
protected void handleResponse(SearchResponse response, ActionListener<SchemaRowSet> listener) {
SearchHit[] hits = response.getHits().getHits();
// create response extractors for the first time
List<FieldExtraction> refs = query.columns();
List<HitExtractor> exts = new ArrayList<>(refs.size());
for (FieldExtraction ref : refs) {
exts.add(createExtractor(ref));
}
// there are some results
if (hits.length > 0) {
String scrollId = response.getScrollId();
// if there's an id, try to setup next scroll
if (scrollId != null &&
// is all the content already retrieved?
(Boolean.TRUE.equals(response.isTerminatedEarly()) || response.getHits().getTotalHits() == hits.length
// or maybe the limit has been reached
|| (hits.length >= query.limit() && query.limit() > -1))) {
// if so, clear the scroll
clear(response.getScrollId(), ActionListener.wrap(
succeeded -> listener.onResponse(new SchemaSearchHitRowSet(schema, exts, hits, query.limit(), null)),
listener::onFailure));
} else {
listener.onResponse(new SchemaSearchHitRowSet(schema, exts, hits, query.limit(), scrollId));
}
}
// no hits
else {
clear(response.getScrollId(), ActionListener.wrap(succeeded -> listener.onResponse(Rows.empty(schema)),
listener::onFailure));
}
}
private HitExtractor createExtractor(FieldExtraction ref) {
if (ref instanceof SearchHitFieldRef) {
SearchHitFieldRef f = (SearchHitFieldRef) ref;
return new FieldHitExtractor(f.name(), f.useDocValue(), f.hitName());
}
if (ref instanceof ScriptFieldRef) {
ScriptFieldRef f = (ScriptFieldRef) ref;
return new FieldHitExtractor(f.name(), true);
}
if (ref instanceof ComputedRef) {
ProcessorDefinition proc = ((ComputedRef) ref).processor();
// collect hitNames
Set<String> hitNames = new LinkedHashSet<>();
proc = proc.transformDown(l -> {
HitExtractor he = createExtractor(l.context());
hitNames.add(he.hitName());
if (hitNames.size() > 1) {
throw new SqlIllegalArgumentException("Multi-level nested fields [{}] not supported yet", hitNames);
}
return new HitExtractorInput(l.location(), l.expression(), he);
}, ReferenceInput.class);
String hitName = null;
if (hitNames.size() == 1) {
hitName = hitNames.iterator().next();
}
return new ComputingExtractor(proc.asProcessor(), hitName);
}
throw new SqlIllegalArgumentException("Unexpected value reference {}", ref.getClass());
}
}
/**
* Base listener class providing clean-up and exception handling.
* Handles both scroll queries (scan/scroll) and regular/composite-aggs queries.
*/
abstract static class BaseActionListener implements ActionListener<SearchResponse> {
final ActionListener<SchemaRowSet> listener;
final Client client;
final TimeValue keepAlive;
final Schema schema;
BaseActionListener(ActionListener<SchemaRowSet> listener, Client client, TimeValue keepAlive, Schema schema) {
this.listener = listener;
this.client = client;
this.keepAlive = keepAlive;
this.schema = schema;
}
// TODO: need to handle rejections plus check failures (shard size, etc...)
@Override
public void onResponse(final SearchResponse response) {
try {
ShardSearchFailure[] failure = response.getShardFailures();
if (!CollectionUtils.isEmpty(failure)) {
cleanup(response, new SqlIllegalArgumentException(failure[0].reason(), failure[0].getCause()));
} else {
handleResponse(response, ActionListener.wrap(listener::onResponse, e -> cleanup(response, e)));
}
} catch (Exception ex) {
cleanup(response, ex);
}
}
protected abstract void handleResponse(SearchResponse response, ActionListener<SchemaRowSet> listener);
// clean-up the scroll in case of exception
protected final void cleanup(SearchResponse response, Exception ex) {
if (response != null && response.getScrollId() != null) {
client.prepareClearScroll().addScrollId(response.getScrollId())
// in case of failure, report the initial exception instead of the one resulting from cleaning the scroll
.execute(ActionListener.wrap(r -> listener.onFailure(ex), e -> {
ex.addSuppressed(e);
listener.onFailure(ex);
}));
} else {
listener.onFailure(ex);
}
}
protected final void clear(String scrollId, ActionListener<Boolean> listener) {
if (scrollId != null) {
client.prepareClearScroll().addScrollId(scrollId).execute(
ActionListener.wrap(
clearScrollResponse -> listener.onResponse(clearScrollResponse.isSucceeded()),
listener::onFailure));
} else {
listener.onResponse(false);
}
}
@Override
public final void onFailure(Exception ex) {
listener.onFailure(ex);
}
}
}

View File

@ -0,0 +1,34 @@
/*
* 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.sql.execution.search;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.xpack.sql.execution.search.extractor.BucketExtractor;
import org.elasticsearch.xpack.sql.session.RowSet;
import org.elasticsearch.xpack.sql.session.SchemaRowSet;
import org.elasticsearch.xpack.sql.type.Schema;
import java.util.List;
/**
* Extension of the {@link RowSet} over a composite agg, extending it to provide its schema.
* Used for the initial response.
*/
class SchemaCompositeAggsRowSet extends CompositeAggsRowSet implements SchemaRowSet {
private final Schema schema;
SchemaCompositeAggsRowSet(Schema schema, List<BucketExtractor> exts, SearchResponse response, int limitAggs, byte[] next,
String... indices) {
super(exts, response, limitAggs, next, indices);
this.schema = schema;
}
@Override
public Schema schema() {
return schema;
}
}

View File

@ -14,13 +14,13 @@ import java.util.List;
/**
* Initial results from a scroll search. Distinct from the following pages
* because it has a {@link Schema} available. See {@link ScrolledSearchHitRowSet}
* because it has a {@link Schema} available. See {@link SearchHitRowSet}
* for the next pages.
*/
public class InitialSearchHitRowSet extends AbstractSearchHitRowSet implements SchemaRowSet {
class SchemaSearchHitRowSet extends SearchHitRowSet implements SchemaRowSet {
private final Schema schema;
public InitialSearchHitRowSet(Schema schema, List<HitExtractor> exts, SearchHit[] hits, int limitHits, String scrollId) {
SchemaSearchHitRowSet(Schema schema, List<HitExtractor> exts, SearchHit[] hits, int limitHits, String scrollId) {
super(exts, hits, limitHits, scrollId);
this.schema = schema;
}

View File

@ -6,14 +6,17 @@
package org.elasticsearch.xpack.sql.execution.search;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.ClearScrollRequest;
import org.elasticsearch.action.search.ClearScrollResponse;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchScrollRequest;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.xpack.sql.execution.search.extractor.HitExtractor;
import org.elasticsearch.xpack.sql.session.Configuration;
import org.elasticsearch.xpack.sql.session.Cursor;
@ -24,6 +27,9 @@ import java.util.List;
import java.util.Objects;
public class ScrollCursor implements Cursor {
private final Logger log = Loggers.getLogger(getClass());
public static final String NAME = "s";
private final String scrollId;
@ -38,28 +44,42 @@ public class ScrollCursor implements Cursor {
public ScrollCursor(StreamInput in) throws IOException {
scrollId = in.readString();
extractors = in.readNamedWriteableList(HitExtractor.class);
limit = in.readVInt();
extractors = in.readNamedWriteableList(HitExtractor.class);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(scrollId);
out.writeNamedWriteableList(extractors);
out.writeVInt(limit);
}
out.writeNamedWriteableList(extractors);
}
@Override
public String getWriteableName() {
return NAME;
}
String scrollId() {
return scrollId;
}
List<HitExtractor> extractors() {
return extractors;
}
int limit() {
return limit;
}
@Override
public void nextPage(Configuration cfg, Client client, ActionListener<RowSet> listener) {
public void nextPage(Configuration cfg, Client client, NamedWriteableRegistry registry, ActionListener<RowSet> listener) {
log.trace("About to execute scroll query {}", scrollId);
SearchScrollRequest request = new SearchScrollRequest(scrollId).scroll(cfg.pageTimeout());
client.searchScroll(request, ActionListener.wrap((SearchResponse response) -> {
ScrolledSearchHitRowSet rowSet = new ScrolledSearchHitRowSet(extractors, response.getHits().getHits(),
SearchHitRowSet rowSet = new SearchHitRowSet(extractors, response.getHits().getHits(),
limit, response.getScrollId());
if (rowSet.nextPageCursor() == Cursor.EMPTY ) {
// we are finished with this cursor, let's clean it before continuing
@ -104,5 +124,4 @@ public class ScrollCursor implements Cursor {
clearScrollRequest.addScrollId(scrollId);
client.clearScroll(clearScrollRequest, listener);
}
}
}

View File

@ -1,31 +0,0 @@
/*
* 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.sql.execution.search;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.xpack.sql.execution.search.extractor.HitExtractor;
import org.elasticsearch.xpack.sql.type.Schema;
import java.util.List;
/**
* "Next" page of results from a scroll search. Distinct from the first page
* because it no longer has the {@link Schema}. See {@link InitialSearchHitRowSet}
* for the initial results.
*/
public class ScrolledSearchHitRowSet extends AbstractSearchHitRowSet {
private final int columnCount;
public ScrolledSearchHitRowSet(List<HitExtractor> exts, SearchHit[] hits, int limitHits, String scrollId) {
super(exts, hits, limitHits, scrollId);
this.columnCount = exts.size();
}
@Override
public int columnCount() {
return columnCount;
}
}

View File

@ -1,379 +0,0 @@
/*
* 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.sql.execution.search;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
import org.elasticsearch.search.aggregations.support.AggregationPath;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.xpack.sql.SqlException;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.elasticsearch.xpack.sql.execution.search.extractor.ComputingHitExtractor;
import org.elasticsearch.xpack.sql.execution.search.extractor.FieldHitExtractor;
import org.elasticsearch.xpack.sql.execution.search.extractor.HitExtractor;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition.AggPathInput;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition.AggValueInput;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition.HitExtractorInput;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition.ProcessorDefinition;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition.ReferenceInput;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.Processor;
import org.elasticsearch.xpack.sql.querydsl.agg.AggPath;
import org.elasticsearch.xpack.sql.querydsl.container.AggRef;
import org.elasticsearch.xpack.sql.querydsl.container.ComputedRef;
import org.elasticsearch.xpack.sql.querydsl.container.QueryContainer;
import org.elasticsearch.xpack.sql.querydsl.container.ScriptFieldRef;
import org.elasticsearch.xpack.sql.querydsl.container.SearchHitFieldRef;
import org.elasticsearch.xpack.sql.querydsl.container.TotalCountRef;
import org.elasticsearch.xpack.sql.session.Configuration;
import org.elasticsearch.xpack.sql.session.Rows;
import org.elasticsearch.xpack.sql.session.SchemaRowSet;
import org.elasticsearch.xpack.sql.type.Schema;
import org.elasticsearch.xpack.sql.util.StringUtils;
import java.util.ArrayList;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.Supplier;
// TODO: add retry/back-off
public class Scroller {
private final Logger log = Loggers.getLogger(getClass());
private final TimeValue keepAlive, timeout;
private final int size;
private final Client client;
@Nullable
private final QueryBuilder filter;
public Scroller(Client client, Configuration cfg) {
this(client, cfg.requestTimeout(), cfg.pageTimeout(), cfg.filter(), cfg.pageSize());
}
public Scroller(Client client, TimeValue keepAlive, TimeValue timeout, QueryBuilder filter, int size) {
this.client = client;
this.keepAlive = keepAlive;
this.timeout = timeout;
this.filter = filter;
this.size = size;
}
public void scroll(Schema schema, QueryContainer query, String index, ActionListener<SchemaRowSet> listener) {
// prepare the request
SearchSourceBuilder sourceBuilder = SourceGenerator.sourceBuilder(query, filter, size);
if (log.isTraceEnabled()) {
log.trace("About to execute query {} on {}", StringUtils.toString(sourceBuilder), index);
}
SearchRequest search = client.prepareSearch(index).setSource(sourceBuilder).request();
ScrollerActionListener l;
if (query.isAggsOnly()) {
l = new AggsScrollActionListener(listener, client, timeout, schema, query);
} else {
search.scroll(keepAlive).source().timeout(timeout);
l = new HandshakeScrollActionListener(listener, client, timeout, schema, query);
}
client.search(search, l);
}
// dedicated scroll used for aggs-only/group-by results
static class AggsScrollActionListener extends ScrollerActionListener {
private final QueryContainer query;
AggsScrollActionListener(ActionListener<SchemaRowSet> listener, Client client, TimeValue keepAlive,
Schema schema, QueryContainer query) {
super(listener, client, keepAlive, schema);
this.query = query;
}
@Override
protected void handleResponse(SearchResponse response, ActionListener<SchemaRowSet> listener) {
final List<Object[]> extractedAggs = new ArrayList<>();
AggValues aggValues = new AggValues(extractedAggs);
List<Supplier<Object>> aggColumns = new ArrayList<>(query.columns().size());
// this method assumes the nested aggregation are all part of the same tree (the SQL group-by)
int maxDepth = -1;
List<FieldExtraction> cols = query.columns();
for (int index = 0; index < cols.size(); index++) {
FieldExtraction col = cols.get(index);
Supplier<Object> supplier = null;
if (col instanceof ComputedRef) {
ComputedRef pRef = (ComputedRef) col;
Processor processor = pRef.processor().transformUp(a -> {
Object[] value = extractAggValue(new AggRef(a.context()), response);
extractedAggs.add(value);
final int aggPosition = extractedAggs.size() - 1;
Supplier<Object> action = null;
if (a.action() != null) {
action = () -> a.action().process(aggValues.column(aggPosition));
}
else {
action = () -> aggValues.column(aggPosition);
}
return new AggValueInput(a.location(), a.expression(), action, a.innerKey());
}, AggPathInput.class).asProcessor();
// the input is provided through the value input above
supplier = () -> processor.process(null);
}
else {
extractedAggs.add(extractAggValue(col, response));
final int aggPosition = extractedAggs.size() - 1;
supplier = () -> aggValues.column(aggPosition);
}
aggColumns.add(supplier);
if (col.depth() > maxDepth) {
maxDepth = col.depth();
}
}
aggValues.init(maxDepth, query.limit());
clearScroll(response.getScrollId(), ActionListener.wrap(
succeeded -> listener.onResponse(new AggsRowSet(schema, aggValues, aggColumns)),
listener::onFailure));
}
private Object[] extractAggValue(FieldExtraction col, SearchResponse response) {
if (col == TotalCountRef.INSTANCE) {
return new Object[] { Long.valueOf(response.getHits().getTotalHits()) };
}
else if (col instanceof AggRef) {
Object[] arr;
AggRef ref = (AggRef) col;
String path = ref.path();
// yup, this is instance equality to make sure we only check the path used by the code
if (path == TotalCountRef.PATH) {
arr = new Object[] { Long.valueOf(response.getHits().getTotalHits()) };
}
else {
// workaround for elastic/elasticsearch/issues/23056
boolean formattedKey = AggPath.isBucketValueFormatted(path);
if (formattedKey) {
path = AggPath.bucketValueWithoutFormat(path);
}
Object value = getAggProperty(response.getAggregations(), path);
// unwrap nested map
if (ref.innerKey() != null) {
// needs changing when moving to Composite
if (value instanceof Object[]) {
Object[] val = (Object[]) value;
arr = new Object[val.length];
for (int i = 0; i < arr.length; i++) {
if (val[i] instanceof Map) {
arr[i] = ((Map<?, ?>) val[i]).get(ref.innerKey());
}
}
value = arr;
} else {
if (value instanceof Map) {
value = new Object[] { ((Map<?, ?>) value).get(ref.innerKey()) };
}
}
}
if (formattedKey) {
List<? extends Bucket> buckets = ((MultiBucketsAggregation) value).getBuckets();
arr = new Object[buckets.size()];
for (int i = 0; i < buckets.size(); i++) {
arr[i] = buckets.get(i).getKeyAsString();
}
} else {
arr = value instanceof Object[] ? (Object[]) value : new Object[] { value };
}
}
return arr;
}
throw new SqlIllegalArgumentException("Unexpected non-agg/grouped column specified; {}", col.getClass());
}
private static Object getAggProperty(Aggregations aggs, String path) {
List<String> list = AggregationPath.parse(path).getPathElementsAsStringList();
String aggName = list.get(0);
InternalAggregation agg = aggs.get(aggName);
if (agg == null) {
throw new SqlException("Cannot find an aggregation named {}", aggName);
}
return agg.getProperty(list.subList(1, list.size()));
}
}
// initial scroll used for parsing search hits (handles possible aggs)
static class HandshakeScrollActionListener extends ScrollerActionListener {
private final QueryContainer query;
HandshakeScrollActionListener(ActionListener<SchemaRowSet> listener, Client client, TimeValue keepAlive,
Schema schema, QueryContainer query) {
super(listener, client, keepAlive, schema);
this.query = query;
}
@Override
protected void handleResponse(SearchResponse response, ActionListener<SchemaRowSet> listener) {
SearchHit[] hits = response.getHits().getHits();
List<HitExtractor> exts = getExtractors();
// there are some results
if (hits.length > 0) {
String scrollId = response.getScrollId();
// if there's an id, try to setup next scroll
if (scrollId != null &&
// is all the content already retrieved?
(Boolean.TRUE.equals(response.isTerminatedEarly()) || response.getHits().getTotalHits() == hits.length
// or maybe the limit has been reached
|| (hits.length >= query.limit() && query.limit() > -1))) {
// if so, clear the scroll
clearScroll(response.getScrollId(), ActionListener.wrap(
succeeded -> listener.onResponse(new InitialSearchHitRowSet(schema, exts, hits, query.limit(), null)),
listener::onFailure));
} else {
listener.onResponse(new InitialSearchHitRowSet(schema, exts, hits, query.limit(), scrollId));
}
}
// no hits
else {
clearScroll(response.getScrollId(), ActionListener.wrap(succeeded -> listener.onResponse(Rows.empty(schema)),
listener::onFailure));
}
}
private List<HitExtractor> getExtractors() {
// create response extractors for the first time
List<FieldExtraction> refs = query.columns();
List<HitExtractor> exts = new ArrayList<>(refs.size());
for (FieldExtraction ref : refs) {
exts.add(createExtractor(ref));
}
return exts;
}
private HitExtractor createExtractor(FieldExtraction ref) {
if (ref instanceof SearchHitFieldRef) {
SearchHitFieldRef f = (SearchHitFieldRef) ref;
return new FieldHitExtractor(f.name(), f.useDocValue(), f.hitName());
}
if (ref instanceof ScriptFieldRef) {
ScriptFieldRef f = (ScriptFieldRef) ref;
return new FieldHitExtractor(f.name(), true);
}
if (ref instanceof ComputedRef) {
ProcessorDefinition proc = ((ComputedRef) ref).processor();
// collect hitNames
Set<String> hitNames = new LinkedHashSet<>();
proc = proc.transformDown(l -> {
HitExtractor he = createExtractor(l.context());
hitNames.add(he.hitName());
if (hitNames.size() > 1) {
throw new SqlIllegalArgumentException("Multi-level nested fields [{}] not supported yet", hitNames);
}
return new HitExtractorInput(l.location(), l.expression(), he);
}, ReferenceInput.class);
String hitName = null;
if (hitNames.size() == 1) {
hitName = hitNames.iterator().next();
}
return new ComputingHitExtractor(proc.asProcessor(), hitName);
}
throw new SqlIllegalArgumentException("Unexpected ValueReference {}", ref.getClass());
}
}
abstract static class ScrollerActionListener implements ActionListener<SearchResponse> {
final ActionListener<SchemaRowSet> listener;
final Client client;
final TimeValue keepAlive;
final Schema schema;
ScrollerActionListener(ActionListener<SchemaRowSet> listener, Client client, TimeValue keepAlive, Schema schema) {
this.listener = listener;
this.client = client;
this.keepAlive = keepAlive;
this.schema = schema;
}
// TODO: need to handle rejections plus check failures (shard size, etc...)
@Override
public void onResponse(final SearchResponse response) {
try {
ShardSearchFailure[] failure = response.getShardFailures();
if (!CollectionUtils.isEmpty(failure)) {
cleanupScroll(response, new SqlException(failure[0].reason(), failure[0].getCause()));
} else {
handleResponse(response, ActionListener.wrap(listener::onResponse, e -> cleanupScroll(response, e)));
}
} catch (Exception ex) {
cleanupScroll(response, ex);
}
}
protected abstract void handleResponse(SearchResponse response, ActionListener<SchemaRowSet> listener);
// clean-up the scroll in case of exception
protected final void cleanupScroll(SearchResponse response, Exception ex) {
if (response != null && response.getScrollId() != null) {
client.prepareClearScroll().addScrollId(response.getScrollId())
// in case of failure, report the initial exception instead of the one resulting from cleaning the scroll
.execute(ActionListener.wrap(r -> listener.onFailure(ex), e -> {
ex.addSuppressed(e);
listener.onFailure(ex);
}));
}
}
protected final void clearScroll(String scrollId, ActionListener<Boolean> listener) {
if (scrollId != null) {
client.prepareClearScroll().addScrollId(scrollId).execute(
ActionListener.wrap(
clearScrollResponse -> listener.onResponse(clearScrollResponse.isSucceeded()),
listener::onFailure));
} else {
listener.onResponse(false);
}
}
@Override
public final void onFailure(Exception ex) {
listener.onFailure(ex);
}
}
}

View File

@ -20,20 +20,20 @@ import java.util.Set;
/**
* Extracts rows from an array of {@link SearchHit}.
*/
abstract class AbstractSearchHitRowSet extends AbstractRowSet {
class SearchHitRowSet extends AbstractRowSet {
private final SearchHit[] hits;
private final Cursor cursor;
private final String scrollId;
private final List<HitExtractor> extractors;
private final Set<String> innerHits = new LinkedHashSet<>();
private final String innerHit;
private final int limit;
private final int size;
private final int[] indexPerLevel;
private int row = 0;
AbstractSearchHitRowSet(List<HitExtractor> exts, SearchHit[] hits, int limitHits, String scrollId) {
SearchHitRowSet(List<HitExtractor> exts, SearchHit[] hits, int limit, String scrollId) {
this.hits = hits;
this.scrollId = scrollId;
this.extractors = exts;
@ -60,8 +60,7 @@ abstract class AbstractSearchHitRowSet extends AbstractRowSet {
maxDepth = 1;
sz = 0;
for (int i = 0; i < hits.length; i++) {
SearchHit hit = hits[i];
for (SearchHit hit : hits) {
for (String ih : innerHits) {
SearchHits sh = hit.getInnerHits().get(ih);
if (sh != null) {
@ -70,10 +69,8 @@ abstract class AbstractSearchHitRowSet extends AbstractRowSet {
}
}
}
// overall limit
limit = limitHits;
// page size
size = limitHits < 0 ? sz : Math.min(sz, limitHits);
size = limit < 0 ? sz : Math.min(sz, limit);
indexPerLevel = new int[maxDepth + 1];
this.innerHit = innerHit;
@ -82,8 +79,8 @@ abstract class AbstractSearchHitRowSet extends AbstractRowSet {
* scroll but all results fit in the first page. */
cursor = Cursor.EMPTY;
} else {
// compute remaining limit
int remainingLimit = limit - size;
// compute remaining limit (only if the limit is specified - that is, positive).
int remainingLimit = limit < 0 ? limit : limit - size;
// if the computed limit is zero, or the size is zero it means either there's nothing left or the limit has been reached
if (size == 0 || remainingLimit == 0) {
cursor = Cursor.EMPTY;
@ -93,6 +90,11 @@ abstract class AbstractSearchHitRowSet extends AbstractRowSet {
}
}
@Override
public int columnCount() {
return extractors.size();
}
@Override
protected Object getColumn(int column) {
HitExtractor e = extractors.get(column);
@ -109,7 +111,7 @@ abstract class AbstractSearchHitRowSet extends AbstractRowSet {
hit = sh[indexPerLevel[lvl]];
}
return e.get(hit);
return e.extract(hit);
}
@Override
@ -170,4 +172,4 @@ abstract class AbstractSearchHitRowSet extends AbstractRowSet {
public Cursor nextPageCursor() {
return cursor;
}
}
}

View File

@ -9,7 +9,7 @@ import org.elasticsearch.index.query.BoolQueryBuilder;
import org.elasticsearch.index.query.ConstantScoreQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.PipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.fetch.StoredFieldsContext;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
@ -17,20 +17,14 @@ import org.elasticsearch.search.sort.FieldSortBuilder;
import org.elasticsearch.search.sort.NestedSortBuilder;
import org.elasticsearch.search.sort.ScriptSortBuilder.ScriptSortType;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.xpack.sql.expression.Attribute;
import org.elasticsearch.xpack.sql.expression.FieldAttribute;
import org.elasticsearch.xpack.sql.querydsl.agg.Aggs;
import org.elasticsearch.xpack.sql.querydsl.agg.GroupByColumnAgg;
import org.elasticsearch.xpack.sql.querydsl.agg.GroupingAgg;
import org.elasticsearch.xpack.sql.querydsl.container.AttributeSort;
import org.elasticsearch.xpack.sql.querydsl.container.QueryContainer;
import org.elasticsearch.xpack.sql.querydsl.container.ScoreSort;
import org.elasticsearch.xpack.sql.querydsl.container.ScriptSort;
import org.elasticsearch.xpack.sql.querydsl.container.Sort;
import org.elasticsearch.xpack.sql.querydsl.container.Sort.Direction;
import java.util.ArrayList;
import java.util.List;
import static java.util.Collections.singletonList;
@ -50,58 +44,45 @@ public abstract class SourceGenerator {
source.query(new ConstantScoreQueryBuilder(filter));
}
} else {
if (filter == null) {
source.query(container.query().asBuilder());
} else {
if (filter != null) {
source.query(new BoolQueryBuilder().must(container.query().asBuilder()).filter(filter));
} else {
source.query(container.query().asBuilder());
}
}
SqlSourceBuilder sortBuilder = new SqlSourceBuilder();
// Iterate through all the columns requested, collecting the fields that
// need to be retrieved from the result documents
// NB: the sortBuilder takes care of eliminating duplicates
container.columns().forEach(cr -> cr.collectFields(sortBuilder));
sortBuilder.build(source);
optimize(sortBuilder, source);
// add the aggs
Aggs aggs = container.aggs();
// add the aggs (if present)
AggregationBuilder aggBuilder = container.aggs().asAggBuilder();
// push limit onto group aggs
if (container.limit() > 0) {
List<GroupingAgg> groups = new ArrayList<>(aggs.groups());
if (groups.size() > 0) {
// get just the root agg
GroupingAgg mainAgg = groups.get(0);
if (mainAgg instanceof GroupByColumnAgg) {
groups.set(0, ((GroupByColumnAgg) mainAgg).withLimit(container.limit()));
aggs = aggs.with(groups);
}
}
}
for (AggregationBuilder builder : aggs.asAggBuilders()) {
source.aggregation(builder);
if (aggBuilder != null) {
source.aggregation(aggBuilder);
}
sorting(container, source);
// add the pipeline aggs
for (PipelineAggregationBuilder builder : aggs.asPipelineBuilders()) {
source.aggregation(builder);
// set page size
if (size != null) {
int sz = container.limit() > 0 ? Math.min(container.limit(), size) : size;
if (source.size() == -1) {
source.size(sz);
}
if (aggBuilder instanceof CompositeAggregationBuilder) {
((CompositeAggregationBuilder) aggBuilder).size(sz);
}
}
optimize(container, source);
// set size
if (size != null) {
if (source.size() == -1) {
int sz = container.limit() > 0 ? Math.min(container.limit(), size) : size;
source.size(sz);
}
}
return source;
}
@ -158,7 +139,7 @@ public abstract class SourceGenerator {
}
if (sortBuilder != null) {
sortBuilder.order(sortable.direction() == Direction.ASC ? SortOrder.ASC : SortOrder.DESC);
sortBuilder.order(sortable.direction().asOrder());
source.sort(sortBuilder);
}
}
@ -171,9 +152,10 @@ public abstract class SourceGenerator {
}
private static void optimize(QueryContainer query, SearchSourceBuilder builder) {
// if only aggs are needed, don't retrieve any docs
// if only aggs are needed, don't retrieve any docs and remove scoring
if (query.isAggsOnly()) {
builder.size(0);
builder.trackScores(false);
// disable source fetching (only doc values are used)
disableSource(builder);
}

View File

@ -0,0 +1,17 @@
/*
* 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.sql.execution.search.extractor;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
/**
* Extracts an aggregation value from a {@link Bucket}.
*/
public interface BucketExtractor extends NamedWriteable {
Object extract(Bucket bucket);
}

View File

@ -0,0 +1,30 @@
/*
* 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.sql.execution.search.extractor;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry.Entry;
import java.util.ArrayList;
import java.util.List;
public final class BucketExtractors {
private BucketExtractors() {}
/**
* All of the named writeables needed to deserialize the instances of
* {@linkplain BucketExtractor}s.
*/
public static List<NamedWriteableRegistry.Entry> getNamedWriteables() {
List<NamedWriteableRegistry.Entry> entries = new ArrayList<>();
entries.add(new Entry(BucketExtractor.class, CompositeKeyExtractor.NAME, CompositeKeyExtractor::new));
entries.add(new Entry(BucketExtractor.class, ComputingExtractor.NAME, ComputingExtractor::new));
entries.add(new Entry(BucketExtractor.class, MetricAggExtractor.NAME, MetricAggExtractor::new));
entries.add(new Entry(BucketExtractor.class, ConstantExtractor.NAME, ConstantExtractor::new));
return entries;
}
}

View File

@ -0,0 +1,121 @@
/*
* 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.sql.execution.search.extractor;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.elasticsearch.xpack.sql.querydsl.container.GroupByRef.Property;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import java.io.IOException;
import java.util.Map;
import java.util.Objects;
public class CompositeKeyExtractor implements BucketExtractor {
/**
* Key or Komposite extractor.
*/
static final String NAME = "k";
private final String key;
private final Property property;
private final DateTimeZone timeZone;
/**
* Constructs a new <code>CompositeKeyExtractor</code> instance.
* The time-zone parameter is used to indicate a date key.
*/
public CompositeKeyExtractor(String key, Property property, DateTimeZone timeZone) {
this.key = key;
this.property = property;
this.timeZone = timeZone;
}
CompositeKeyExtractor(StreamInput in) throws IOException {
key = in.readString();
property = in.readEnum(Property.class);
timeZone = in.readOptionalTimeZone();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(key);
out.writeEnum(property);
out.writeOptionalTimeZone(timeZone);
}
String key() {
return key;
}
Property property() {
return property;
}
DateTimeZone timeZone() {
return timeZone;
}
@Override
public String getWriteableName() {
return NAME;
}
@Override
public Object extract(Bucket bucket) {
if (property == Property.COUNT) {
return bucket.getDocCount();
}
// get the composite value
Object m = bucket.getKey();
if (!(m instanceof Map)) {
throw new SqlIllegalArgumentException("Unexpected bucket returned: {}", m);
}
Object object = ((Map<?, ?>) m).get(key);
if (timeZone != null) {
if (object instanceof Long) {
object = new DateTime(((Long) object).longValue(), timeZone);
} else {
throw new SqlIllegalArgumentException("Invalid date key returned: {}", object);
}
}
return object;
}
@Override
public int hashCode() {
return Objects.hash(key, property, timeZone);
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
CompositeKeyExtractor other = (CompositeKeyExtractor) obj;
return Objects.equals(key, other.key)
&& Objects.equals(property, other.property)
&& Objects.equals(timeZone, other.timeZone);
}
@Override
public String toString() {
return "|" + key + "|";
}
}

View File

@ -8,6 +8,7 @@ package org.elasticsearch.xpack.sql.execution.search.extractor;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.HitExtractorProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.Processor;
@ -15,7 +16,7 @@ import java.io.IOException;
import java.util.Objects;
/**
* HitExtractor that delegates to a processor. The difference between this class
* Hit/BucketExtractor that delegates to a processor. The difference between this class
* and {@link HitExtractorProcessor} is that the latter is used inside a
* {@link Processor} tree as a leaf (and thus can effectively parse the
* {@link SearchHit} while this class is used when scrolling and passing down
@ -25,7 +26,7 @@ import java.util.Objects;
* to reduce API complexity (and keep the {@link HitExtractor} only as an
* internal implementation detail).
*/
public class ComputingHitExtractor implements HitExtractor {
public class ComputingExtractor implements HitExtractor, BucketExtractor {
/**
* Stands for {@code comPuting}. We try to use short names for {@link HitExtractor}s
* to save a few bytes when when we send them back to the user.
@ -34,12 +35,16 @@ public class ComputingHitExtractor implements HitExtractor {
private final Processor processor;
private final String hitName;
public ComputingHitExtractor(Processor processor, String hitName) {
public ComputingExtractor(Processor processor) {
this(processor, null);
}
public ComputingExtractor(Processor processor, String hitName) {
this.processor = processor;
this.hitName = hitName;
}
ComputingHitExtractor(StreamInput in) throws IOException {
ComputingExtractor(StreamInput in) throws IOException {
processor = in.readNamedWriteable(Processor.class);
hitName = in.readOptionalString();
}
@ -59,8 +64,17 @@ public class ComputingHitExtractor implements HitExtractor {
return processor;
}
public Object extract(Object input) {
return processor.process(input);
}
@Override
public Object get(SearchHit hit) {
public Object extract(Bucket bucket) {
return processor.process(bucket);
}
@Override
public Object extract(SearchHit hit) {
return processor.process(hit);
}
@ -74,7 +88,7 @@ public class ComputingHitExtractor implements HitExtractor {
if (obj == null || obj.getClass() != getClass()) {
return false;
}
ComputingHitExtractor other = (ComputingHitExtractor) obj;
ComputingExtractor other = (ComputingExtractor) obj;
return Objects.equals(processor, other.processor)
&& Objects.equals(hitName, other.hitName);
}

View File

@ -8,6 +8,7 @@ package org.elasticsearch.xpack.sql.execution.search.extractor;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
import java.io.IOException;
import java.util.Objects;
@ -15,7 +16,7 @@ import java.util.Objects;
/**
* Returns the a constant for every search hit against which it is run.
*/
public class ConstantExtractor implements HitExtractor {
public class ConstantExtractor implements HitExtractor, BucketExtractor {
/**
* Stands for {@code constant}. We try to use short names for {@link HitExtractor}s
* to save a few bytes when when we send them back to the user.
@ -42,7 +43,12 @@ public class ConstantExtractor implements HitExtractor {
}
@Override
public Object get(SearchHit hit) {
public Object extract(SearchHit hit) {
return constant;
}
@Override
public Object extract(Bucket bucket) {
return constant;
}
@ -69,4 +75,4 @@ public class ConstantExtractor implements HitExtractor {
public String toString() {
return "^" + constant;
}
}
}

View File

@ -10,7 +10,6 @@ import org.elasticsearch.common.document.DocumentField;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.xpack.sql.SqlException;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.joda.time.ReadableDateTime;
@ -83,7 +82,7 @@ public class FieldHitExtractor implements HitExtractor {
}
@Override
public Object get(SearchHit hit) {
public Object extract(SearchHit hit) {
Object value = null;
if (useDocValue) {
DocumentField field = hit.field(fieldName);
@ -111,18 +110,18 @@ public class FieldHitExtractor implements HitExtractor {
if (ARRAYS_LENIENCY || list.size() == 1) {
return unwrapMultiValue(list.get(0));
} else {
throw new SqlException("Arrays (returned by [{}]) are not supported", fieldName);
throw new SqlIllegalArgumentException("Arrays (returned by [{}]) are not supported", fieldName);
}
}
}
if (values instanceof Map) {
throw new SqlException("Objects (returned by [{}]) are not supported", fieldName);
throw new SqlIllegalArgumentException("Objects (returned by [{}]) are not supported", fieldName);
}
if (values instanceof Long || values instanceof Double || values instanceof String || values instanceof Boolean
|| values instanceof ReadableDateTime) {
return values;
}
throw new SqlException("Type {} (returned by [{}]) is not supported", values.getClass().getSimpleName(), fieldName);
throw new SqlIllegalArgumentException("Type {} (returned by [{}]) is not supported", values.getClass().getSimpleName(), fieldName);
}
@SuppressWarnings("unchecked")
@ -137,7 +136,7 @@ public class FieldHitExtractor implements HitExtractor {
first = false;
value = ((Map<String, Object>) value).get(node);
} else {
throw new SqlException("Cannot extract value [{}] from source", fieldName);
throw new SqlIllegalArgumentException("Cannot extract value [{}] from source", fieldName);
}
}
return unwrapMultiValue(value);

View File

@ -16,7 +16,7 @@ public interface HitExtractor extends NamedWriteable {
/**
* Extract the value from a hit.
*/
Object get(SearchHit hit);
Object extract(SearchHit hit);
/**
* Name of the inner hit needed by this extractor if it needs one, {@code null} otherwise.

View File

@ -7,12 +7,14 @@ package org.elasticsearch.xpack.sql.execution.search.extractor;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry.Entry;
import org.elasticsearch.xpack.sql.expression.function.scalar.Processors;
import java.util.ArrayList;
import java.util.List;
public abstract class HitExtractors {
public final class HitExtractors {
private HitExtractors() {}
/**
* All of the named writeables needed to deserialize the instances of
* {@linkplain HitExtractor}.
@ -21,9 +23,8 @@ public abstract class HitExtractors {
List<NamedWriteableRegistry.Entry> entries = new ArrayList<>();
entries.add(new Entry(HitExtractor.class, ConstantExtractor.NAME, ConstantExtractor::new));
entries.add(new Entry(HitExtractor.class, FieldHitExtractor.NAME, FieldHitExtractor::new));
entries.add(new Entry(HitExtractor.class, ComputingHitExtractor.NAME, ComputingHitExtractor::new));
entries.add(new Entry(HitExtractor.class, ComputingExtractor.NAME, ComputingExtractor::new));
entries.add(new Entry(HitExtractor.class, ScoreExtractor.NAME, in -> ScoreExtractor.INSTANCE));
entries.addAll(Processors.getNamedWriteables());
return entries;
}
}

View File

@ -0,0 +1,108 @@
/*
* 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.sql.execution.search.extractor;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.elasticsearch.xpack.sql.querydsl.agg.Aggs;
import java.io.IOException;
import java.util.Map;
import java.util.Objects;
public class MetricAggExtractor implements BucketExtractor {
static final String NAME = "m";
private final String name;
private final String property;
private final String innerKey;
public MetricAggExtractor(String name, String property, String innerKey) {
this.name = name;
this.property = property;
this.innerKey = innerKey;
}
MetricAggExtractor(StreamInput in) throws IOException {
name = in.readString();
property = in.readString();
innerKey = in.readOptionalString();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
out.writeString(property);
out.writeOptionalString(innerKey);
}
String name() {
return name;
}
String property() {
return property;
}
String innerKey() {
return innerKey;
}
@Override
public String getWriteableName() {
return NAME;
}
@Override
public Object extract(Bucket bucket) {
InternalAggregation agg = bucket.getAggregations().get(name);
if (agg == null) {
throw new SqlIllegalArgumentException("Cannot find an aggregation named {}", name);
}
if (agg instanceof InternalNumericMetricsAggregation.MultiValue) {
//TODO: need to investigate when this can be not-null
//if (innerKey == null) {
// throw new SqlIllegalArgumentException("Invalid innerKey {} specified for aggregation {}", innerKey, name);
//}
return ((InternalNumericMetricsAggregation.MultiValue) agg).value(property);
}
Object v = agg.getProperty(property);
return innerKey != null && v instanceof Map ? ((Map<?, ?>) v).get(innerKey) : v;
}
@Override
public int hashCode() {
return Objects.hash(name, property, innerKey);
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
MetricAggExtractor other = (MetricAggExtractor) obj;
return Objects.equals(name, other.name)
&& Objects.equals(property, other.property)
&& Objects.equals(innerKey, other.innerKey);
}
@Override
public String toString() {
String i = innerKey != null ? "[" + innerKey + "]" : "";
return Aggs.ROOT_GROUP_NAME + ">" + name + "." + property + i;
}
}

View File

@ -34,7 +34,7 @@ public class ScoreExtractor implements HitExtractor {
}
@Override
public Object get(SearchHit hit) {
public Object extract(SearchHit hit) {
return hit.getScore();
}

View File

@ -13,6 +13,7 @@ import java.util.List;
import java.util.function.Predicate;
import static java.util.Collections.emptyList;
import static java.util.Collections.emptyMap;
import static java.util.stream.Collectors.toList;
public abstract class Expressions {
@ -38,6 +39,18 @@ public abstract class Expressions {
return list;
}
public static AttributeMap<Expression> asAttributeMap(List<? extends NamedExpression> named) {
if (named.isEmpty()) {
return new AttributeMap<>(emptyMap());
}
AttributeMap<Expression> map = new AttributeMap<>();
for (NamedExpression exp : named) {
map.add(exp.toAttribute(), exp);
}
return map;
}
public static boolean anyMatch(List<? extends Expression> exps, Predicate<? super Expression> predicate) {
for (Expression exp : exps) {
if (exp.anyMatch(predicate)) {

View File

@ -7,7 +7,6 @@ package org.elasticsearch.xpack.sql.expression.function.aggregate;
import org.elasticsearch.xpack.sql.expression.Expression;
import org.elasticsearch.xpack.sql.expression.function.Function;
import org.elasticsearch.xpack.sql.querydsl.agg.AggPath;
import org.elasticsearch.xpack.sql.tree.Location;
import org.elasticsearch.xpack.sql.tree.NodeInfo;
import org.elasticsearch.xpack.sql.type.DataType;
@ -78,7 +77,12 @@ public class InnerAggregate extends AggregateFunction {
public AggregateFunctionAttribute toAttribute() {
// this is highly correlated with QueryFolder$FoldAggregate#addFunction (regarding the function name within the querydsl)
return new AggregateFunctionAttribute(location(), name(), dataType(), outer.id(), functionId(),
AggPath.metricValue(functionId(), innerId));
aggMetricValue(functionId(), innerId));
}
public static String aggMetricValue(String aggPath, String valueName) {
// handle aggPath inconsistency (for percentiles and percentileRanks) percentile[99.9] (valid) vs percentile.99.9 (invalid)
return aggPath + "[" + valueName + "]";
}
@Override

View File

@ -11,7 +11,6 @@ import org.elasticsearch.xpack.sql.expression.Foldables;
import org.elasticsearch.xpack.sql.tree.Location;
import org.elasticsearch.xpack.sql.tree.NodeInfo;
import org.elasticsearch.xpack.sql.type.DataType;
import org.elasticsearch.xpack.sql.type.DataTypes;
import java.util.List;
@ -61,6 +60,6 @@ public class Percentile extends NumericAggregate implements EnclosedAgg {
@Override
public String innerName() {
return "[" + Double.toString(Foldables.doubleValueOf(percent)) + "]";
return Double.toString(Foldables.doubleValueOf(percent));
}
}

View File

@ -11,7 +11,6 @@ import org.elasticsearch.xpack.sql.expression.Foldables;
import org.elasticsearch.xpack.sql.tree.Location;
import org.elasticsearch.xpack.sql.tree.NodeInfo;
import org.elasticsearch.xpack.sql.type.DataType;
import org.elasticsearch.xpack.sql.type.DataTypes;
import java.util.List;
@ -61,6 +60,6 @@ public class PercentileRank extends AggregateFunction implements EnclosedAgg {
@Override
public String innerName() {
return "[" + Double.toString(Foldables.doubleValueOf(value)) + "]";
return Double.toString(Foldables.doubleValueOf(value));
}
}

View File

@ -11,16 +11,18 @@ import org.elasticsearch.xpack.sql.expression.function.scalar.arithmetic.BinaryA
import org.elasticsearch.xpack.sql.expression.function.scalar.arithmetic.UnaryArithmeticProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTimeProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.math.MathProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.BucketExtractorProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.ChainingProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.ConstantProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.HitExtractorProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.MatrixFieldProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.Processor;
import java.util.ArrayList;
import java.util.List;
public abstract class Processors {
public final class Processors {
private Processors() {}
/**
* All of the named writeables needed to deserialize the instances of
@ -31,9 +33,9 @@ public abstract class Processors {
// base
entries.add(new Entry(Processor.class, ConstantProcessor.NAME, ConstantProcessor::new));
entries.add(new Entry(Processor.class, HitExtractorProcessor.NAME, HitExtractorProcessor::new));
entries.add(new Entry(Processor.class, BucketExtractorProcessor.NAME, BucketExtractorProcessor::new));
entries.add(new Entry(Processor.class, CastProcessor.NAME, CastProcessor::new));
entries.add(new Entry(Processor.class, ChainingProcessor.NAME, ChainingProcessor::new));
entries.add(new Entry(Processor.class, MatrixFieldProcessor.NAME, MatrixFieldProcessor::new));
// arithmetic
entries.add(new Entry(Processor.class, BinaryArithmeticProcessor.NAME, BinaryArithmeticProcessor::new));

View File

@ -0,0 +1,51 @@
/*
* 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.sql.expression.function.scalar.processor.definition;
import org.elasticsearch.xpack.sql.execution.search.SqlSourceBuilder;
import org.elasticsearch.xpack.sql.execution.search.extractor.BucketExtractor;
import org.elasticsearch.xpack.sql.expression.Expression;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.BucketExtractorProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.ChainingProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.Processor;
import org.elasticsearch.xpack.sql.tree.Location;
import org.elasticsearch.xpack.sql.tree.NodeInfo;
public class AggExtractorInput extends LeafInput<BucketExtractor> {
private final Processor chained;
public AggExtractorInput(Location location, Expression expression, Processor processor, BucketExtractor context) {
super(location, expression, context);
this.chained = processor;
}
@Override
protected NodeInfo<AggExtractorInput> info() {
return NodeInfo.create(this, AggExtractorInput::new, expression(), chained, context());
}
@Override
public Processor asProcessor() {
Processor proc = new BucketExtractorProcessor(context());
return chained != null ? new ChainingProcessor(proc, chained) : proc;
}
@Override
public final boolean supportedByAggsOnlyQuery() {
return true;
}
@Override
public ProcessorDefinition resolveAttributes(AttributeResolver resolver) {
return this;
}
@Override
public final void collectFields(SqlSourceBuilder sourceBuilder) {
// Nothing to collect
}
}

View File

@ -5,39 +5,37 @@
*/
package org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition;
import org.elasticsearch.xpack.sql.execution.search.AggRef;
import org.elasticsearch.xpack.sql.expression.Expression;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.Processor;
import org.elasticsearch.xpack.sql.tree.Location;
import org.elasticsearch.xpack.sql.tree.NodeInfo;
import java.util.Objects;
public class AggPathInput extends CommonNonExecutableInput<String> {
public class AggPathInput extends CommonNonExecutableInput<AggRef> {
private final String innerKey;
// used in case the agg itself is not returned in a suitable format (like date aggs)
private final Processor action;
public AggPathInput(Expression expression, String context) {
this(Location.EMPTY, expression, context, null, null);
public AggPathInput(Expression expression, AggRef context) {
this(Location.EMPTY, expression, context, null);
}
public AggPathInput(Expression expression, String context, String innerKey) {
this(Location.EMPTY, expression, context, innerKey, null);
}
public AggPathInput(Location location, Expression expression, String context, String innerKey, Processor action) {
/**
*
* Constructs a new <code>AggPathInput</code> instance.
* The action is used for handling corner-case results such as date histogram which returns
* a full date object for year which requires additional extraction.
*/
public AggPathInput(Location location, Expression expression, AggRef context, Processor action) {
super(location, expression, context);
this.innerKey = innerKey;
this.action = action;
}
@Override
protected NodeInfo<AggPathInput> info() {
return NodeInfo.create(this, AggPathInput::new, expression(), context(), innerKey, action);
}
public String innerKey() {
return innerKey;
return NodeInfo.create(this, AggPathInput::new, expression(), context(), action);
}
public Processor action() {
@ -56,7 +54,7 @@ public class AggPathInput extends CommonNonExecutableInput<String> {
@Override
public int hashCode() {
return Objects.hash(context(), innerKey);
return Objects.hash(context(), action);
}
@Override
@ -71,7 +69,6 @@ public class AggPathInput extends CommonNonExecutableInput<String> {
AggPathInput other = (AggPathInput) obj;
return Objects.equals(context(), other.context())
&& Objects.equals(innerKey, other.innerKey)
&& Objects.equals(action, other.action);
}
}

View File

@ -1,82 +0,0 @@
/*
* 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.sql.expression.function.scalar.processor.definition;
import org.elasticsearch.xpack.sql.execution.search.SqlSourceBuilder;
import org.elasticsearch.xpack.sql.expression.Expression;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.MatrixFieldProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.Processor;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.SuppliedProcessor;
import org.elasticsearch.xpack.sql.tree.Location;
import org.elasticsearch.xpack.sql.tree.NodeInfo;
import java.util.Objects;
import java.util.function.Supplier;
public class AggValueInput extends LeafInput<Supplier<Object>> {
private final String innerKey;
private final Processor matrixProcessor;
public AggValueInput(Location location, Expression expression, Supplier<Object> context, String innerKey) {
super(location, expression, context);
this.innerKey = innerKey;
this.matrixProcessor = innerKey != null ? new MatrixFieldProcessor(innerKey) : null;
}
@Override
protected NodeInfo<AggValueInput> info() {
return NodeInfo.create(this, AggValueInput::new, expression(), context(), innerKey);
}
public String innerKey() {
return innerKey;
}
@Override
public Processor asProcessor() {
return new SuppliedProcessor(() -> matrixProcessor != null ? matrixProcessor.process(context().get()) : context().get());
}
@Override
public final boolean supportedByAggsOnlyQuery() {
return true;
}
@Override
public ProcessorDefinition resolveAttributes(AttributeResolver resolver) {
return this;
}
@Override
public final void collectFields(SqlSourceBuilder sourceBuilder) {
// Nothing to collect
}
@Override
public final int depth() {
return 0;
}
@Override
public int hashCode() {
return Objects.hash(context(), innerKey);
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
AggValueInput other = (AggValueInput) obj;
return Objects.equals(context(), other.context())
&& Objects.equals(innerKey, other.innerKey);
}
}

View File

@ -39,9 +39,4 @@ public class AttributeInput extends NonExecutableInput<Attribute> {
public final void collectFields(SqlSourceBuilder sourceBuilder) {
// Nothing to extract
}
@Override
public final int depth() {
return 0;
}
}

View File

@ -8,6 +8,7 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.processor.definit
import org.elasticsearch.xpack.sql.execution.search.SqlSourceBuilder;
import org.elasticsearch.xpack.sql.expression.Expression;
import org.elasticsearch.xpack.sql.tree.Location;
import java.util.Arrays;
import java.util.List;
@ -52,11 +53,6 @@ public abstract class BinaryProcessorDefinition extends ProcessorDefinition {
return replaceChildren(newLeft, newRight);
}
@Override
public final int depth() {
return Math.max(left.depth(), right.depth());
}
/**
* Build a copy of this object with new left and right children. Used by
* {@link #resolveAttributes(AttributeResolver)}.

View File

@ -34,9 +34,4 @@ abstract class CommonNonExecutableInput<T> extends NonExecutableInput<T> {
public final void collectFields(SqlSourceBuilder sourceBuilder) {
// Nothing to extract
}
@Override
public final int depth() {
return 0;
}
}

View File

@ -42,9 +42,4 @@ public class ConstantInput extends LeafInput<Object> {
public final void collectFields(SqlSourceBuilder sourceBuilder) {
// Nothing to collect
}
@Override
public final int depth() {
return 0;
}
}

View File

@ -43,9 +43,4 @@ public class HitExtractorInput extends LeafInput<HitExtractor> {
public final void collectFields(SqlSourceBuilder sourceBuilder) {
// No fields to collect
}
@Override
public final int depth() {
return 0;
}
}

View File

@ -7,12 +7,12 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.processor.definit
import org.elasticsearch.xpack.sql.expression.Expression;
import org.elasticsearch.xpack.sql.tree.Location;
import java.util.List;
import java.util.Objects;
import static java.util.Collections.emptyList;
import java.util.List;
public abstract class LeafInput<T> extends ProcessorDefinition {
private T context;
@ -52,7 +52,7 @@ public abstract class LeafInput<T> extends ProcessorDefinition {
}
LeafInput<?> other = (LeafInput<?>) obj;
return Objects.equals(context(), other.context())
return Objects.equals(context(), other.context())
&& Objects.equals(expression(), other.expression());
}
}

View File

@ -35,9 +35,4 @@ public class ReferenceInput extends NonExecutableInput<FieldExtraction> {
public final void collectFields(SqlSourceBuilder sourceBuilder) {
context().collectFields(sourceBuilder);
}
@Override
public int depth() {
return context().depth();
}
}

View File

@ -8,15 +8,15 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.processor.definit
import org.elasticsearch.xpack.sql.execution.search.SqlSourceBuilder;
import org.elasticsearch.xpack.sql.execution.search.extractor.ScoreExtractor;
import org.elasticsearch.xpack.sql.expression.Expression;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.HitExtractorProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.Processor;
import org.elasticsearch.xpack.sql.tree.Location;
import org.elasticsearch.xpack.sql.tree.NodeInfo;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.HitExtractorProcessor;
import static java.util.Collections.emptyList;
import java.util.List;
import static java.util.Collections.emptyList;
public class ScoreProcessorDefinition extends ProcessorDefinition {
public ScoreProcessorDefinition(Location location, Expression expression) {
super(location, expression, emptyList());
@ -56,9 +56,4 @@ public class ScoreProcessorDefinition extends ProcessorDefinition {
public void collectFields(SqlSourceBuilder sourceBuilder) {
sourceBuilder.trackScores();
}
@Override
public int depth() {
return 0;
}
}

View File

@ -11,12 +11,12 @@ import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.runtime.Processor;
import org.elasticsearch.xpack.sql.tree.Location;
import org.elasticsearch.xpack.sql.tree.NodeInfo;
import java.util.List;
import java.util.Objects;
import static java.util.Collections.singletonList;
import java.util.List;
public final class UnaryProcessorDefinition extends ProcessorDefinition {
private final ProcessorDefinition child;
@ -78,11 +78,6 @@ public final class UnaryProcessorDefinition extends ProcessorDefinition {
child.collectFields(sourceBuilder);
}
@Override
public int depth() {
return child.depth();
}
@Override
public int hashCode() {
return Objects.hash(expression(), child, action);

View File

@ -0,0 +1,76 @@
/*
* 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.sql.expression.function.scalar.processor.runtime;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.elasticsearch.xpack.sql.execution.search.extractor.BucketExtractor;
import java.io.IOException;
import java.util.Objects;
/**
* Processor wrapping an {@link BucketExtractor}, essentially being a source/leaf of a
* Processor tree.
*/
public class BucketExtractorProcessor implements Processor {
public static final String NAME = "a";
private final BucketExtractor extractor;
public BucketExtractorProcessor(BucketExtractor extractor) {
this.extractor = extractor;
}
public BucketExtractorProcessor(StreamInput in) throws IOException {
extractor = in.readNamedWriteable(BucketExtractor.class);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeNamedWriteable(extractor);
}
@Override
public String getWriteableName() {
return NAME;
}
@Override
public Object process(Object input) {
if (!(input instanceof Bucket)) {
throw new SqlIllegalArgumentException("Expected an agg bucket but received {}", input);
}
return extractor.extract((Bucket) input);
}
@Override
public int hashCode() {
return Objects.hash(extractor);
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
BucketExtractorProcessor other = (BucketExtractorProcessor) obj;
return Objects.equals(extractor, other.extractor);
}
@Override
public String toString() {
return extractor.toString();
}
}

View File

@ -15,7 +15,7 @@ import java.io.IOException;
import java.util.Objects;
/**
* Processor wrapping a HitExtractor esentially being a source/leaf of a
* Processor wrapping a {@link HitExtractor}, essentially being a source/leaf of a
* Processor tree.
*/
public class HitExtractorProcessor implements Processor {
@ -47,7 +47,7 @@ public class HitExtractorProcessor implements Processor {
if (!(input instanceof SearchHit)) {
throw new SqlIllegalArgumentException("Expected a SearchHit but received {}", input);
}
return extractor.get((SearchHit) input);
return extractor.extract((SearchHit) input);
}
@Override

View File

@ -1,69 +0,0 @@
/*
* 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.sql.expression.function.scalar.processor.runtime;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
import java.util.Map;
import java.util.Objects;
public class MatrixFieldProcessor implements Processor {
public static final String NAME = "mat";
private final String key;
public MatrixFieldProcessor(String key) {
this.key = key;
}
public MatrixFieldProcessor(StreamInput in) throws IOException {
key = in.readString();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(key);
}
@Override
public String getWriteableName() {
return NAME;
}
String key() {
return key;
}
@Override
public Object process(Object r) {
return r instanceof Map ? ((Map<?, ?>) r).get(key) : r;
}
@Override
public int hashCode() {
return Objects.hashCode(key);
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
MatrixFieldProcessor other = (MatrixFieldProcessor) obj;
return Objects.equals(key, other.key);
}
public String toString() {
return "[" + key + "]";
}
}

View File

@ -1,37 +0,0 @@
/*
* 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.sql.expression.function.scalar.processor.runtime;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import java.io.IOException;
import java.util.function.Supplier;
public class SuppliedProcessor implements Processor {
private final Supplier<Object> action;
public SuppliedProcessor(Supplier<Object> action) {
this.action = action;
}
@Override
public String getWriteableName() {
throw new SqlIllegalArgumentException("transient");
}
@Override
public void writeTo(StreamOutput out) throws IOException {
throw new SqlIllegalArgumentException("transient");
}
@Override
public Object process(Object input) {
return action.get();
}
}

View File

@ -6,7 +6,7 @@
package org.elasticsearch.xpack.sql.plan.physical;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.xpack.sql.execution.search.Scroller;
import org.elasticsearch.xpack.sql.execution.search.Querier;
import org.elasticsearch.xpack.sql.expression.Attribute;
import org.elasticsearch.xpack.sql.querydsl.container.QueryContainer;
import org.elasticsearch.xpack.sql.session.Rows;
@ -56,8 +56,8 @@ public class EsQueryExec extends LeafExec {
@Override
public void execute(SqlSession session, ActionListener<SchemaRowSet> listener) {
Scroller scroller = new Scroller(session.client(), session.settings());
scroller.scroll(Rows.schema(output), queryContainer, index, listener);
Querier scroller = new Querier(session.client(), session.settings());
scroller.query(Rows.schema(output), queryContainer, index, listener);
}
@Override

View File

@ -6,6 +6,7 @@
package org.elasticsearch.xpack.sql.planner;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.xpack.sql.execution.search.AggRef;
import org.elasticsearch.xpack.sql.expression.Alias;
import org.elasticsearch.xpack.sql.expression.Attribute;
import org.elasticsearch.xpack.sql.expression.Expression;
@ -22,6 +23,7 @@ import org.elasticsearch.xpack.sql.expression.function.aggregate.Count;
import org.elasticsearch.xpack.sql.expression.function.aggregate.InnerAggregate;
import org.elasticsearch.xpack.sql.expression.function.scalar.ScalarFunction;
import org.elasticsearch.xpack.sql.expression.function.scalar.ScalarFunctionAttribute;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTimeFunction;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTimeHistogramFunction;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition.AggPathInput;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition.ProcessorDefinition;
@ -39,23 +41,26 @@ import org.elasticsearch.xpack.sql.plan.physical.ProjectExec;
import org.elasticsearch.xpack.sql.planner.QueryTranslator.GroupingContext;
import org.elasticsearch.xpack.sql.planner.QueryTranslator.QueryTranslation;
import org.elasticsearch.xpack.sql.querydsl.agg.AggFilter;
import org.elasticsearch.xpack.sql.querydsl.agg.AggPath;
import org.elasticsearch.xpack.sql.querydsl.agg.Aggs;
import org.elasticsearch.xpack.sql.querydsl.agg.GroupingAgg;
import org.elasticsearch.xpack.sql.querydsl.agg.GroupByKey;
import org.elasticsearch.xpack.sql.querydsl.agg.LeafAgg;
import org.elasticsearch.xpack.sql.querydsl.container.AttributeSort;
import org.elasticsearch.xpack.sql.querydsl.container.ComputedRef;
import org.elasticsearch.xpack.sql.querydsl.container.GlobalCountRef;
import org.elasticsearch.xpack.sql.querydsl.container.GroupByRef;
import org.elasticsearch.xpack.sql.querydsl.container.GroupByRef.Property;
import org.elasticsearch.xpack.sql.querydsl.container.MetricAggRef;
import org.elasticsearch.xpack.sql.querydsl.container.QueryContainer;
import org.elasticsearch.xpack.sql.querydsl.container.ScoreSort;
import org.elasticsearch.xpack.sql.querydsl.container.ScriptSort;
import org.elasticsearch.xpack.sql.querydsl.container.Sort.Direction;
import org.elasticsearch.xpack.sql.querydsl.container.TotalCountRef;
import org.elasticsearch.xpack.sql.querydsl.query.Query;
import org.elasticsearch.xpack.sql.rule.Rule;
import org.elasticsearch.xpack.sql.rule.RuleExecutor;
import org.elasticsearch.xpack.sql.session.EmptyExecutable;
import org.elasticsearch.xpack.sql.type.DataType;
import org.elasticsearch.xpack.sql.util.Check;
import org.elasticsearch.xpack.sql.util.StringUtils;
import org.joda.time.DateTimeZone;
import java.util.Arrays;
import java.util.LinkedHashMap;
@ -65,8 +70,10 @@ import java.util.concurrent.atomic.AtomicReference;
import static org.elasticsearch.xpack.sql.planner.QueryTranslator.and;
import static org.elasticsearch.xpack.sql.planner.QueryTranslator.toAgg;
import static org.elasticsearch.xpack.sql.planner.QueryTranslator.toQuery;
import static org.elasticsearch.xpack.sql.util.CollectionUtils.combine;
/**
* Folds the PhysicalPlan into a {@link Query}.
*/
class QueryFolder extends RuleExecutor<PhysicalPlan> {
PhysicalPlan fold(PhysicalPlan plan) {
return execute(plan);
@ -175,51 +182,8 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
if (filter == null) {
return qContainer.aggs();
}
// find the relevant groups and compute the shortest path (the highest group in the hierarchy)
Map<String, GroupingAgg> groupPaths = new LinkedHashMap<>();
// root group
String shortestPath = null;
GroupingAgg targetGroup = null;
for (String refId : filter.aggRefs()) {
// is it root group or agg property (_count)
if (refId == null) {
shortestPath = StringUtils.EMPTY;
}
else {
// find function group
GroupingAgg groupAgg = qContainer.findGroupForAgg(refId);
if (groupAgg == null) {
groupAgg = qContainer.pseudoFunctions().get(refId);
}
if (groupAgg == null) {
// Weird ctor call to make sure we don't interpret the message as a pattern
throw new FoldingException(fexec, "Cannot find group for agg " + refId
+ " referrenced by agg filter " + filter.name() + "(" + filter + ")", (Exception) null);
}
String path = groupAgg.asParentPath();
if (shortestPath == null || shortestPath.length() > path.length()) {
shortestPath = path;
targetGroup = groupAgg;
}
groupPaths.put(refId, groupAgg);
}
}
// and finally update the agg groups
if (targetGroup == GroupingAgg.DEFAULT_GROUP) {
throw new PlanningException("Aggregation filtering not supported (yet) without explicit grouping");
//aggs = aggs.addAgg(null, filter);
}
if (targetGroup == null) {
throw new PlanningException("Cannot determine group column; likely an invalid query - please report");
}
else {
aggs = aggs.updateGroup(targetGroup.withPipelines(combine(targetGroup.subPipelines(), filter)));
aggs = aggs.addAgg(filter);
}
return aggs;
@ -296,7 +260,7 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
// get the backing expression and check if it belongs to a agg group or whether it's
// an expression in the first place
Expression exp = p.expression();
GroupingAgg matchingGroup = null;
GroupByKey matchingGroup = null;
if (groupingContext != null) {
// is there a group (aggregation) for this expression ?
matchingGroup = groupingContext.groupFor(exp);
@ -315,6 +279,7 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
if (matchingGroup != null) {
if (exp instanceof Attribute || exp instanceof ScalarFunction) {
Processor action = null;
DateTimeZone tz = null;
/*
* special handling of dates since aggs return the typed Date object which needs
* extraction instead of handling this in the scroller, the folder handles this
@ -322,8 +287,9 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
*/
if (exp instanceof DateTimeHistogramFunction) {
action = ((UnaryProcessorDefinition) p).action();
tz = ((DateTimeFunction) exp).timeZone();
}
return new AggPathInput(exp.location(), exp, matchingGroup.propertyPath(), null, action);
return new AggPathInput(exp.location(), exp, new GroupByRef(matchingGroup.id(), null, tz), action);
}
}
// or found an aggregate expression (which has to work on an attribute used for grouping)
@ -350,35 +316,41 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
// already used in the aggpath)
//aliases.put(as.toAttribute(), sf.toAttribute());
}
// apply the same logic above (for function inputs) to non-scalar functions with small variantions:
// apply the same logic above (for function inputs) to non-scalar functions with small variations:
// instead of adding things as input, add them as full blown column
else {
GroupingAgg matchingGroup = null;
GroupByKey matchingGroup = null;
if (groupingContext != null) {
// is there a group (aggregation) for this expression ?
matchingGroup = groupingContext.groupFor(child);
}
// attributes can only refer to declared groups
if (child instanceof Attribute) {
Check.notNull(matchingGroup, "Cannot find group '%s'", Expressions.name(child));
queryC = queryC.addAggColumn(matchingGroup.propertyPath());
Check.notNull(matchingGroup, "Cannot find group [{}]", Expressions.name(child));
// check if the field is a date - if so mark it as such to interpret the long as a date
// UTC is used since that's what the server uses and there's no conversion applied
// (like for date histograms)
DateTimeZone dt = DataType.DATE == child.dataType() ? DateTimeZone.UTC : null;
queryC = queryC.addColumn(new GroupByRef(matchingGroup.id(), null, dt));
}
else {
// the only thing left is agg function
Check.isTrue(Functions.isAggregate(child),
"Expected aggregate function inside alias; got %s", child.nodeString());
"Expected aggregate function inside alias; got [{}]", child.nodeString());
Tuple<QueryContainer, AggPathInput> withAgg = addAggFunction(matchingGroup,
(AggregateFunction) child, compoundAggMap, queryC);
queryC = withAgg.v1().addAggColumn(withAgg.v2().context(), withAgg.v2().innerKey());
queryC = withAgg.v1().addColumn(withAgg.v2().context());
}
}
// not an Alias or Function means it's an Attribute so apply the same logic as above
} else {
GroupingAgg matchingGroup = null;
GroupByKey matchingGroup = null;
if (groupingContext != null) {
matchingGroup = groupingContext.groupFor(ne);
Check.notNull(matchingGroup, "Cannot find group '%s'", Expressions.name(ne));
queryC = queryC.addAggColumn(matchingGroup.propertyPath());
Check.notNull(matchingGroup, "Cannot find group [{}]", Expressions.name(ne));
DateTimeZone dt = DataType.DATE == ne.dataType() ? DateTimeZone.UTC : null;
queryC = queryC.addColumn(new GroupByRef(matchingGroup.id(), null, dt));
}
}
}
@ -393,26 +365,25 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
return a;
}
private Tuple<QueryContainer, AggPathInput> addAggFunction(GroupingAgg parentAgg, AggregateFunction f,
private Tuple<QueryContainer, AggPathInput> addAggFunction(GroupByKey groupingAgg, AggregateFunction f,
Map<CompoundNumericAggregate, String> compoundAggMap, QueryContainer queryC) {
String functionId = f.functionId();
// handle count as a special case agg
if (f instanceof Count) {
Count c = (Count) f;
if (!c.distinct()) {
String path = parentAgg == null ? TotalCountRef.PATH : AggPath.bucketCount(parentAgg.asParentPath());
Map<String, GroupingAgg> pseudoFunctions = new LinkedHashMap<>(queryC.pseudoFunctions());
pseudoFunctions.put(functionId, parentAgg);
return new Tuple<>(queryC.withPseudoFunctions(pseudoFunctions), new AggPathInput(f, path));
AggRef ref = groupingAgg == null ?
GlobalCountRef.INSTANCE :
new GroupByRef(groupingAgg.id(), Property.COUNT, null);
Map<String, GroupByKey> pseudoFunctions = new LinkedHashMap<>(queryC.pseudoFunctions());
pseudoFunctions.put(functionId, groupingAgg);
return new Tuple<>(queryC.withPseudoFunctions(pseudoFunctions), new AggPathInput(f, ref));
}
}
AggPathInput aggInput = null;
// otherwise translate the function into an actual agg
String parentPath = parentAgg != null ? parentAgg.asParentPath() : null;
String groupId = parentAgg != null ? parentAgg.id() : null;
if (f instanceof InnerAggregate) {
InnerAggregate ia = (InnerAggregate) f;
CompoundNumericAggregate outer = ia.outer();
@ -420,22 +391,22 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
// the compound agg hasn't been seen before so initialize it
if (cAggPath == null) {
LeafAgg leafAgg = toAgg(parentPath, functionId, outer);
cAggPath = leafAgg.propertyPath();
LeafAgg leafAgg = toAgg(outer.functionId(), outer);
cAggPath = leafAgg.id();
compoundAggMap.put(outer, cAggPath);
// add the agg (without any reference)
queryC = queryC.with(queryC.aggs().addAgg(leafAgg));
}
String aggPath = AggPath.metricValue(cAggPath, ia.innerId());
// FIXME: concern leak - hack around MatrixAgg which is not
// generalized (afaik)
aggInput = new AggPathInput(f, aggPath, ia.innerKey() != null ? QueryTranslator.nameOf(ia.innerKey()) : null);
aggInput = new AggPathInput(f,
new MetricAggRef(cAggPath, ia.innerId(), ia.innerKey() != null ? QueryTranslator.nameOf(ia.innerKey()) : null));
}
else {
LeafAgg leafAgg = toAgg(parentPath, functionId, f);
aggInput = new AggPathInput(f, leafAgg.propertyPath());
queryC = queryC.with(queryC.aggs().addAgg(groupId, leafAgg));
LeafAgg leafAgg = toAgg(functionId, f);
aggInput = new AggPathInput(f, new MetricAggRef(leafAgg.id()));
queryC = queryC.with(queryC.aggs().addAgg(leafAgg));
}
return new Tuple<>(queryC, aggInput);
@ -457,17 +428,17 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
// check whether there's an alias (occurs with scalar functions which are not named)
attr = qContainer.aliases().getOrDefault(attr, attr);
String lookup = attr.id().toString();
GroupingAgg group = qContainer.findGroupForAgg(lookup);
GroupByKey group = qContainer.findGroupForAgg(lookup);
// TODO: might need to validate whether the target field or group actually exist
if (group != null && group != GroupingAgg.DEFAULT_GROUP) {
if (group != null && group != Aggs.IMPLICIT_GROUP_KEY) {
// check whether the lookup matches a group
if (group.id().equals(lookup)) {
qContainer = qContainer.updateGroup(group.with(direction));
}
// else it's a leafAgg
else {
qContainer = qContainer.updateGroup(group.with(lookup, direction));
qContainer = qContainer.updateGroup(group.with(direction));
}
}
else {

View File

@ -53,15 +53,14 @@ import org.elasticsearch.xpack.sql.expression.regex.Like;
import org.elasticsearch.xpack.sql.expression.regex.LikePattern;
import org.elasticsearch.xpack.sql.expression.regex.RLike;
import org.elasticsearch.xpack.sql.querydsl.agg.AggFilter;
import org.elasticsearch.xpack.sql.querydsl.agg.AggPath;
import org.elasticsearch.xpack.sql.querydsl.agg.AndAggFilter;
import org.elasticsearch.xpack.sql.querydsl.agg.AvgAgg;
import org.elasticsearch.xpack.sql.querydsl.agg.CardinalityAgg;
import org.elasticsearch.xpack.sql.querydsl.agg.ExtendedStatsAgg;
import org.elasticsearch.xpack.sql.querydsl.agg.GroupByColumnAgg;
import org.elasticsearch.xpack.sql.querydsl.agg.GroupByDateAgg;
import org.elasticsearch.xpack.sql.querydsl.agg.GroupByScriptAgg;
import org.elasticsearch.xpack.sql.querydsl.agg.GroupingAgg;
import org.elasticsearch.xpack.sql.querydsl.agg.GroupByColumnKey;
import org.elasticsearch.xpack.sql.querydsl.agg.GroupByDateKey;
import org.elasticsearch.xpack.sql.querydsl.agg.GroupByKey;
import org.elasticsearch.xpack.sql.querydsl.agg.GroupByScriptKey;
import org.elasticsearch.xpack.sql.querydsl.agg.LeafAgg;
import org.elasticsearch.xpack.sql.querydsl.agg.MatrixStatsAgg;
import org.elasticsearch.xpack.sql.querydsl.agg.MaxAgg;
@ -89,9 +88,7 @@ import org.elasticsearch.xpack.sql.type.DataType;
import org.elasticsearch.xpack.sql.util.Check;
import org.elasticsearch.xpack.sql.util.ReflectionUtils;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
@ -165,10 +162,10 @@ abstract class QueryTranslator {
throw new UnsupportedOperationException(format(Locale.ROOT, "Don't know how to translate %s %s", e.nodeName(), e));
}
static LeafAgg toAgg(String parent, String id, Function f) {
static LeafAgg toAgg(String id, Function f) {
for (AggTranslator<?> translator : AGG_TRANSLATORS) {
LeafAgg agg = translator.apply(id, parent, f);
LeafAgg agg = translator.apply(id, f);
if (agg != null) {
return agg;
}
@ -178,44 +175,26 @@ abstract class QueryTranslator {
}
static class GroupingContext {
final GroupingAgg head;
final GroupingAgg tail;
final ExpressionId headAggId;
final Map<ExpressionId, GroupByKey> groupMap;
final GroupByKey tail;
final Map<ExpressionId, GroupingAgg> groupMap;
final List<String> aggNames;
final String groupPath;
GroupingContext(Map<ExpressionId, GroupingAgg> groupMap, String propertyPath) {
GroupingContext(Map<ExpressionId, GroupByKey> groupMap) {
this.groupMap = groupMap;
this.groupPath = propertyPath;
aggNames = new ArrayList<>(groupMap.size());
for (GroupingAgg gAgg : groupMap.values()) {
aggNames.add(gAgg.id());
GroupByKey lastAgg = null;
for (Entry<ExpressionId, GroupByKey> entry : groupMap.entrySet()) {
lastAgg = entry.getValue();
}
Iterator<Entry<ExpressionId, GroupingAgg>> iterator = groupMap.entrySet().iterator();
Entry<ExpressionId, GroupingAgg> entry = iterator.next();
headAggId = entry.getKey();
head = entry.getValue();
GroupingAgg lastAgg = head;
while (iterator.hasNext()) {
lastAgg = iterator.next().getValue();
}
tail = lastAgg;
}
GroupingAgg groupFor(Expression exp) {
GroupByKey groupFor(Expression exp) {
if (Functions.isAggregate(exp)) {
AggregateFunction f = (AggregateFunction) exp;
// if there's at least one agg in the tree
if (groupPath != null) {
GroupingAgg matchingGroup = null;
if (!groupMap.isEmpty()) {
GroupByKey matchingGroup = null;
// group found - finding the dedicated agg
if (f.field() instanceof NamedExpression) {
matchingGroup = groupMap.get(((NamedExpression) f.field()).id());
@ -239,18 +218,16 @@ abstract class QueryTranslator {
}
}
// creates a tree of GroupBy aggs plus some extra information
// useful for tree validation/group referencing
/**
* Creates the list of GroupBy keys
*/
static GroupingContext groupBy(List<? extends Expression> groupings) {
if (groupings.isEmpty()) {
return null;
}
Map<ExpressionId, GroupingAgg> aggMap = new LinkedHashMap<>();
Map<ExpressionId, GroupByKey> aggMap = new LinkedHashMap<>();
// nested the aggs but also
// identify each agg by an expression for later referencing
String propertyPath = "";
for (Expression exp : groupings) {
String aggId;
if (exp instanceof NamedExpression) {
@ -265,38 +242,36 @@ abstract class QueryTranslator {
}
aggId = ne.id().toString();
propertyPath = AggPath.path(propertyPath, aggId);
GroupingAgg agg = null;
GroupByKey key = null;
// handle functions differently
if (exp instanceof Function) {
// dates are handled differently because of date histograms
if (exp instanceof DateTimeHistogramFunction) {
DateTimeHistogramFunction dthf = (DateTimeHistogramFunction) exp;
agg = new GroupByDateAgg(aggId, AggPath.bucketValue(propertyPath), nameOf(exp),
dthf.interval(), dthf.timeZone());
key = new GroupByDateKey(aggId, nameOf(exp), dthf.interval(), dthf.timeZone());
}
// all other scalar functions become a script
else if (exp instanceof ScalarFunction) {
ScalarFunction sf = (ScalarFunction) exp;
agg = new GroupByScriptAgg(aggId, AggPath.bucketValue(propertyPath), nameOf(exp), sf.asScript());
key = new GroupByScriptKey(aggId, nameOf(exp), sf.asScript());
}
// bumped into into an invalid function (which should be caught by the verifier)
else {
throw new SqlIllegalArgumentException("Cannot GROUP BY function {}", exp);
}
}
else {
agg = new GroupByColumnAgg(aggId, AggPath.bucketValue(propertyPath), ne.name());
key = new GroupByColumnKey(aggId, ne.name());
}
aggMap.put(ne.id(), agg);
aggMap.put(ne.id(), key);
}
else {
throw new SqlIllegalArgumentException("Don't know how to group on {}", exp.nodeString());
}
}
return new GroupingContext(aggMap, propertyPath);
return new GroupingContext(aggMap);
}
static QueryTranslation and(Location loc, QueryTranslation left, QueryTranslation right) {
@ -630,7 +605,7 @@ abstract class QueryTranslator {
return new TermQuery(loc, name, value);
}
Check.isTrue(false, "don't know how to translate binary comparison %s in %s", bc.right().nodeString(), bc);
Check.isTrue(false, "don't know how to translate binary comparison [{}] in [{}]", bc.right().nodeString(), bc);
return null;
}
}
@ -739,91 +714,91 @@ abstract class QueryTranslator {
static class DistinctCounts extends SingleValueAggTranslator<Count> {
@Override
protected LeafAgg toAgg(String id, String path, Count c) {
protected LeafAgg toAgg(String id, Count c) {
if (!c.distinct()) {
return null;
}
return new CardinalityAgg(id, path, field(c));
return new CardinalityAgg(id, field(c));
}
}
static class Sums extends SingleValueAggTranslator<Sum> {
@Override
protected LeafAgg toAgg(String id, String path, Sum s) {
return new SumAgg(id, path, field(s));
protected LeafAgg toAgg(String id, Sum s) {
return new SumAgg(id, field(s));
}
}
static class Avgs extends SingleValueAggTranslator<Avg> {
@Override
protected LeafAgg toAgg(String id, String path, Avg a) {
return new AvgAgg(id, path, field(a));
protected LeafAgg toAgg(String id, Avg a) {
return new AvgAgg(id, field(a));
}
}
static class Maxes extends SingleValueAggTranslator<Max> {
@Override
protected LeafAgg toAgg(String id, String path, Max m) {
return new MaxAgg(id, path, field(m));
protected LeafAgg toAgg(String id, Max m) {
return new MaxAgg(id, field(m));
}
}
static class Mins extends SingleValueAggTranslator<Min> {
@Override
protected LeafAgg toAgg(String id, String path, Min m) {
return new MinAgg(id, path, field(m));
protected LeafAgg toAgg(String id, Min m) {
return new MinAgg(id, field(m));
}
}
static class StatsAggs extends CompoundAggTranslator<Stats> {
@Override
protected LeafAgg toAgg(String id, String path, Stats s) {
return new StatsAgg(id, path, field(s));
protected LeafAgg toAgg(String id, Stats s) {
return new StatsAgg(id, field(s));
}
}
static class ExtendedStatsAggs extends CompoundAggTranslator<ExtendedStats> {
@Override
protected LeafAgg toAgg(String id, String path, ExtendedStats e) {
return new ExtendedStatsAgg(id, path, field(e));
protected LeafAgg toAgg(String id, ExtendedStats e) {
return new ExtendedStatsAgg(id, field(e));
}
}
static class MatrixStatsAggs extends CompoundAggTranslator<MatrixStats> {
@Override
protected LeafAgg toAgg(String id, String path, MatrixStats m) {
return new MatrixStatsAgg(id, path, singletonList(field(m)));
protected LeafAgg toAgg(String id, MatrixStats m) {
return new MatrixStatsAgg(id, singletonList(field(m)));
}
}
static class PercentilesAggs extends CompoundAggTranslator<Percentiles> {
@Override
protected LeafAgg toAgg(String id, String path, Percentiles p) {
return new PercentilesAgg(id, path, field(p), doubleValuesOf(p.percents()));
protected LeafAgg toAgg(String id, Percentiles p) {
return new PercentilesAgg(id, field(p), doubleValuesOf(p.percents()));
}
}
static class PercentileRanksAggs extends CompoundAggTranslator<PercentileRanks> {
@Override
protected LeafAgg toAgg(String id, String path, PercentileRanks p) {
return new PercentileRanksAgg(id, path, field(p), doubleValuesOf(p.values()));
protected LeafAgg toAgg(String id, PercentileRanks p) {
return new PercentileRanksAgg(id, field(p), doubleValuesOf(p.values()));
}
}
static class DateTimes extends SingleValueAggTranslator<Min> {
@Override
protected LeafAgg toAgg(String id, String path, Min m) {
return new MinAgg(id, path, field(m));
protected LeafAgg toAgg(String id, Min m) {
return new MinAgg(id, field(m));
}
}
@ -832,33 +807,31 @@ abstract class QueryTranslator {
private final Class<?> typeToken = ReflectionUtils.detectSuperTypeForRuleLike(getClass());
@SuppressWarnings("unchecked")
public final LeafAgg apply(String id, String parent, Function f) {
return (typeToken.isInstance(f) ? asAgg(id, parent, (F) f) : null);
public final LeafAgg apply(String id, Function f) {
return (typeToken.isInstance(f) ? asAgg(id, (F) f) : null);
}
protected abstract LeafAgg asAgg(String id, String parent, F f);
protected abstract LeafAgg asAgg(String id, F f);
}
abstract static class SingleValueAggTranslator<F extends Function> extends AggTranslator<F> {
@Override
protected final LeafAgg asAgg(String id, String parent, F function) {
String path = parent == null ? id : AggPath.path(parent, id);
return toAgg(id, AggPath.metricValue(path), function);
protected final LeafAgg asAgg(String id, F function) {
return toAgg(id, function);
}
protected abstract LeafAgg toAgg(String id, String path, F f);
protected abstract LeafAgg toAgg(String id, F f);
}
abstract static class CompoundAggTranslator<C extends CompoundNumericAggregate> extends AggTranslator<C> {
@Override
protected final LeafAgg asAgg(String id, String parent, C function) {
String path = parent == null ? id : AggPath.path(parent, id);
return toAgg(id, path, function);
protected final LeafAgg asAgg(String id, C function) {
return toAgg(id, function);
}
protected abstract LeafAgg toAgg(String id, String path, C f);
protected abstract LeafAgg toAgg(String id, C f);
}
@ -883,4 +856,4 @@ abstract class QueryTranslator {
return query;
}
}
}
}

View File

@ -7,6 +7,7 @@ package org.elasticsearch.xpack.sql.plugin;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.xpack.sql.session.Configuration;
@ -57,8 +58,8 @@ public class CliFormatterCursor implements Cursor {
}
@Override
public void nextPage(Configuration cfg, Client client, ActionListener<RowSet> listener) {
delegate.nextPage(cfg, client, listener);
public void nextPage(Configuration cfg, Client client, NamedWriteableRegistry registry, ActionListener<RowSet> listener) {
delegate.nextPage(cfg, client, registry, listener);
}
@Override
@ -73,8 +74,12 @@ public class CliFormatterCursor implements Cursor {
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
CliFormatterCursor that = (CliFormatterCursor) o;
return Objects.equals(delegate, that.delegate) &&
Objects.equals(formatter, that.formatter);

View File

@ -20,6 +20,7 @@ import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.rest.action.RestResponseListener;
import org.elasticsearch.rest.action.RestToXContentListener;
import org.elasticsearch.xpack.sql.session.Cursor;
import org.elasticsearch.xpack.sql.session.Cursors;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
@ -63,7 +64,7 @@ public class RestSqlQueryAction extends BaseRestHandler {
return channel -> client.execute(SqlQueryAction.INSTANCE, sqlRequest, new RestResponseListener<SqlQueryResponse>(channel) {
@Override
public RestResponse buildResponse(SqlQueryResponse response) throws Exception {
Cursor cursor = Cursor.decodeFromString(sqlRequest.cursor());
Cursor cursor = Cursors.decodeFromString(sqlRequest.cursor());
final String data = text.format(cursor, request, response);
RestResponse restResponse = new BytesRestResponse(RestStatus.OK, text.contentType(request),
@ -72,7 +73,7 @@ public class RestSqlQueryAction extends BaseRestHandler {
Cursor responseCursor = text.wrapCursor(cursor, response);
if (responseCursor != Cursor.EMPTY) {
restResponse.addHeader("Cursor", Cursor.encodeToString(Version.CURRENT, responseCursor));
restResponse.addHeader("Cursor", Cursors.encodeToString(Version.CURRENT, responseCursor));
}
restResponse.addHeader("Took-nanos", Long.toString(System.nanoTime() - startNanos));

View File

@ -45,7 +45,6 @@ public class SqlPlugin extends Plugin implements ActionPlugin {
private final boolean enabled;
private final SqlLicenseChecker sqlLicenseChecker;
private IndexResolver indexResolver;
SqlPlugin(boolean enabled, SqlLicenseChecker sqlLicenseChecker) {
this.enabled = enabled;
@ -79,18 +78,19 @@ public class SqlPlugin extends Plugin implements ActionPlugin {
ResourceWatcherService resourceWatcherService, ScriptService scriptService,
NamedXContentRegistry xContentRegistry, Environment environment,
NodeEnvironment nodeEnvironment, NamedWriteableRegistry namedWriteableRegistry) {
return createComponents(client, clusterService.getClusterName().value());
return createComponents(client, clusterService.getClusterName().value(), namedWriteableRegistry);
}
/**
* Create components used by the sql plugin.
*/
Collection<Object> createComponents(Client client, String clusterName) {
Collection<Object> createComponents(Client client, String clusterName, NamedWriteableRegistry namedWriteableRegistry) {
if (false == enabled) {
return emptyList();
}
indexResolver = new IndexResolver(client, clusterName);
return Arrays.asList(sqlLicenseChecker, indexResolver, new PlanExecutor(client, indexResolver));
IndexResolver indexResolver = new IndexResolver(client, clusterName);
return Arrays.asList(sqlLicenseChecker, indexResolver, new PlanExecutor(client, indexResolver, namedWriteableRegistry));
}
@Override

View File

@ -8,6 +8,7 @@ package org.elasticsearch.xpack.sql.plugin;
import org.elasticsearch.common.Strings;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.xpack.sql.session.Cursor;
import org.elasticsearch.xpack.sql.session.Cursors;
import org.elasticsearch.xpack.sql.util.StringUtils;
import java.util.List;
@ -46,7 +47,7 @@ enum TextFormat {
@Override
Cursor wrapCursor(Cursor oldCursor, SqlQueryResponse response) {
CliFormatter formatter = (oldCursor instanceof CliFormatterCursor) ?
CliFormatter formatter = (oldCursor instanceof CliFormatterCursor) ?
((CliFormatterCursor) oldCursor).getCliFormatter() : new CliFormatter(response);
return CliFormatterCursor.wrap(super.wrapCursor(oldCursor, response), formatter);
}
@ -232,7 +233,7 @@ enum TextFormat {
}
Cursor wrapCursor(Cursor oldCursor, SqlQueryResponse response) {
return Cursor.decodeFromString(response.cursor());
return Cursors.decodeFromString(response.cursor());
}
static TextFormat fromMediaTypeOrFormat(RestRequest request) {

View File

@ -16,6 +16,7 @@ import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.sql.execution.PlanExecutor;
import org.elasticsearch.xpack.sql.session.Configuration;
import org.elasticsearch.xpack.sql.session.Cursor;
import org.elasticsearch.xpack.sql.session.Cursors;
import static org.elasticsearch.xpack.sql.plugin.SqlClearCursorAction.NAME;
@ -38,7 +39,12 @@ public class TransportSqlClearCursorAction extends HandledTransportAction<SqlCle
@Override
protected void doExecute(SqlClearCursorRequest request, ActionListener<SqlClearCursorResponse> listener) {
sqlLicenseChecker.checkIfSqlAllowed(request.mode());
Cursor cursor = Cursor.decodeFromString(request.getCursor());
operation(planExecutor, request, listener);
}
public static void operation(PlanExecutor planExecutor, SqlClearCursorRequest request,
ActionListener<SqlClearCursorResponse> listener) {
Cursor cursor = Cursors.decodeFromString(request.getCursor());
planExecutor.cleanCursor(Configuration.DEFAULT, cursor, ActionListener.wrap(
success -> listener.onResponse(new SqlClearCursorResponse(success)), listener::onFailure));
}

View File

@ -17,7 +17,7 @@ import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.sql.execution.PlanExecutor;
import org.elasticsearch.xpack.sql.session.Configuration;
import org.elasticsearch.xpack.sql.session.Cursor;
import org.elasticsearch.xpack.sql.session.Cursors;
import org.elasticsearch.xpack.sql.session.RowSet;
import org.elasticsearch.xpack.sql.session.SchemaRowSet;
import org.elasticsearch.xpack.sql.type.Schema;
@ -64,7 +64,7 @@ public class TransportSqlQueryAction extends HandledTransportAction<SqlQueryRequ
planExecutor.sql(cfg, request.query(), request.params(),
ActionListener.wrap(rowSet -> listener.onResponse(createResponse(request, rowSet)), listener::onFailure));
} else {
planExecutor.nextPage(cfg, Cursor.decodeFromString(request.cursor()),
planExecutor.nextPage(cfg, Cursors.decodeFromString(request.cursor()),
ActionListener.wrap(rowSet -> listener.onResponse(createResponse(rowSet, null)), listener::onFailure));
}
}
@ -92,7 +92,7 @@ public class TransportSqlQueryAction extends HandledTransportAction<SqlQueryRequ
});
return new SqlQueryResponse(
Cursor.encodeToString(Version.CURRENT, rowSet.nextPageCursor()),
Cursors.encodeToString(Version.CURRENT, rowSet.nextPageCursor()),
columns,
rows);
}

View File

@ -8,22 +8,18 @@ package org.elasticsearch.xpack.sql.querydsl.agg;
import java.util.Locale;
import java.util.Objects;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.xpack.sql.util.StringUtils;
import static java.lang.String.format;
/**
* Base class holding common properties for Elasticsearch aggregations.
*/
public abstract class Agg {
private final String id;
private final String fieldName;
private final String propertyPath;
private final String asParentPath;
Agg(String id, String propertyPath, String fieldName) {
Agg(String id, String fieldName) {
this.id = id;
this.propertyPath = propertyPath;
int index = propertyPath.lastIndexOf(".");
this.asParentPath = index > 0 ? propertyPath.substring(0, index) : StringUtils.EMPTY;
this.fieldName = fieldName;
}
@ -31,23 +27,13 @@ public abstract class Agg {
return id;
}
public String propertyPath() {
return propertyPath;
}
public String asParentPath() {
return asParentPath;
}
public String fieldName() {
return fieldName;
}
abstract AggregationBuilder toBuilder();
@Override
public int hashCode() {
return Objects.hash(id, propertyPath);
return Objects.hash(id, fieldName);
}
@Override
@ -61,13 +47,12 @@ public abstract class Agg {
}
Agg other = (Agg) obj;
return Objects.equals(id, other.id)
&& Objects.equals(fieldName, other.fieldName)
&& Objects.equals(propertyPath, other.propertyPath);
return Objects.equals(id, other.id)
&& Objects.equals(fieldName, other.fieldName);
}
@Override
public String toString() {
return format(Locale.ROOT, "%s(%s)#%s", getClass().getSimpleName(), fieldName, propertyPath);
return format(Locale.ROOT, "%s(%s)", getClass().getSimpleName(), fieldName);
}
}

View File

@ -1,102 +0,0 @@
/*
* 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.sql.querydsl.agg;
import org.elasticsearch.common.Strings;
import org.elasticsearch.xpack.sql.util.StringUtils;
// utility class around constructing and parsing ES specific aggregation path
// see org.elasticsearch.search.aggregations.support.AggregationPath
public abstract class AggPath {
private static final char PATH_DELIMITER_CHAR = '>';
private static final String PATH_DELIMITER = String.valueOf(PATH_DELIMITER_CHAR);
private static final String VALUE_DELIMITER = ".";
private static final String PATH_BUCKET_VALUE = "._key";
private static final String PATH_BUCKET_COUNT = "._count";
private static final String PATH_BUCKET_VALUE_FORMATTED = "._key_as_string";
private static final String PATH_DEFAULT_VALUE = ".value";
public static String bucketCount(String aggPath) {
return aggPath + PATH_BUCKET_COUNT;
}
public static String bucketValue(String aggPath) {
return aggPath + PATH_BUCKET_VALUE;
}
public static boolean isBucketValueFormatted(String path) {
return path.endsWith(PATH_BUCKET_VALUE_FORMATTED);
}
public static String bucketValueWithoutFormat(String path) {
return path.substring(0, path.length() - PATH_BUCKET_VALUE_FORMATTED.length());
}
public static String metricValue(String aggPath) {
return aggPath + PATH_DEFAULT_VALUE;
}
public static String metricValue(String aggPath, String valueName) {
// handle aggPath inconsistency (for percentiles and percentileRanks) percentile[99.9] (valid) vs percentile.99.9 (invalid)
return valueName.startsWith("[") ? aggPath + valueName : aggPath + VALUE_DELIMITER + valueName;
}
public static String path(String parent, String child) {
return (Strings.hasLength(parent) ? parent + PATH_DELIMITER : StringUtils.EMPTY) + child;
}
/*
* The depth indicates the level of an agg excluding the root agg
* (because all aggs in SQL require a group). However all other
* bucket aggs are counted.
*
* Since the path does not indicate the type of agg used, to
* differentiate between metric properties and bucket properties,
* the bucket value is considered. This is needed since one might
* refer to the keys or count of a bucket path. As the opposite
* side there are metric aggs which have the same level as their
* parent (their nesting is an ES implementation detail).
*
* Examples:
*
* agg1 = 0; agg1 = default/root group
* agg1>agg2._count = 1; ._count indicates agg2 is a bucket
* agg and thus it counted - agg1 (default group),
* depth=0, agg2 (bucketed), depth=1
* agg1>agg2>agg3.value = 1; agg3.value indicates a metric bucket
* thus only agg1 and agg2 are counted -> depth=2. In
* other words, agg3.value has the same depth as
* agg2._count
* agg1>agg2>agg3._count = 2; ._count indicates agg3 is a
* bucket agg, so count it for depth -> depth = 2
* agg1>agg2>agg3.sum = 1; .sum indicates agg3 is a metric agg,
* only agg1 and agg2 are bucket and with agg1 being the
* default group -> depth = 1
*/
public static int depth(String path) {
int depth = countCharIn(path, PATH_DELIMITER_CHAR);
// a metric value always has .foo while a bucket prop with ._foo
int dot = path.lastIndexOf(".");
if (depth > 0 && dot > 0) {
String prop = path.substring(dot + 1);
if (!prop.startsWith("_")) {
return Math.max(0, depth - 1);
}
}
return depth;
}
private static int countCharIn(CharSequence sequence, char c) {
int count = 0;
for (int i = 0; i < sequence.length(); i++) {
if (c == sequence.charAt(i)) {
count++;
}
}
return count;
}
}

View File

@ -6,8 +6,11 @@
package org.elasticsearch.xpack.sql.querydsl.agg;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.PipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.composite.CompositeValuesSourceBuilder;
import org.elasticsearch.search.aggregations.bucket.filter.FiltersAggregationBuilder;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.elasticsearch.xpack.sql.querydsl.container.Sort.Direction;
import java.util.ArrayList;
import java.util.Collection;
@ -15,185 +18,128 @@ import java.util.List;
import java.util.Objects;
import static java.util.Collections.emptyList;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.xpack.sql.util.CollectionUtils.combine;
import static org.elasticsearch.xpack.sql.util.StringUtils.EMPTY;
// Aggregations associated with a query.
//
// This maps the SQL target namely:
// - aggregations are either at the root (default or nameless group) or belong to a named group
// - groups are chained in a linear fashion (group1 -> group2 -> group3)
//
// Aggregations are of two types - concrete metric aggs or pipeline/filtering aggs
//
// Pipeline Notes:
//
// As pipeline aggs can refers to aggs across different groups (different tree levels)
// one needs to pay attention attention before adding them to properly find the lowest level and
// make everything relative to that.
//
// For example sum > 100 && count < 10 might imply sum referring on level 2 while count is on level 3
// In this case, the pipeline would be added to group/level 2 (and refer to sum) while count will be
// made relative to this group/level 3 namely group3>count
/**
* SQL Aggregations associated with a query.
*
* This class maps the SQL GroupBy's (and co) to ES composite agg.
* While the composite agg doesn't require a dedicated structure, for folding purposes, this structure
* tracks the relationship between each key and its sub-aggs or pipelines.
*
* Since sub-aggs can only refer to their group key and these are on the root-level, the tree can have at most
* 2 levels - the grouping and its sub-aggs.
*
* In case no group is specified (which maps to the default group in SQL), due to ES nature a 'dummy' filter agg
* is used.
*/
public class Aggs {
private final List<GroupingAgg> groups;
private final List<Agg> rootAggs;
private final List<PipelineAgg> rootPipelineAggs;
public static final String ROOT_GROUP_NAME = "groupby";
public static final GroupByKey IMPLICIT_GROUP_KEY = new GroupByKey(ROOT_GROUP_NAME, EMPTY, null) {
@Override
public CompositeValuesSourceBuilder<?> asValueSource() {
throw new SqlIllegalArgumentException("Default group does not translate to an aggregation");
}
@Override
protected GroupByKey copy(String id, String fieldName, Direction direction) {
return this;
}
};
private final List<GroupByKey> groups;
private final List<LeafAgg> metricAggs;
private final List<PipelineAgg> pipelineAggs;
public Aggs() {
this(emptyList(), emptyList(), emptyList());
}
public Aggs(List<Agg> rootAggs, List<PipelineAgg> rootPipelineAggs, List<GroupingAgg> groups) {
this.rootAggs = rootAggs;
this.rootPipelineAggs = rootPipelineAggs;
public Aggs(List<GroupByKey> groups, List<LeafAgg> metricAggs, List<PipelineAgg> pipelineAggs) {
this.groups = groups;
this.metricAggs = metricAggs;
this.pipelineAggs = pipelineAggs;
}
public List<GroupingAgg> groups() {
public List<GroupByKey> groups() {
return groups;
}
public List<AggregationBuilder> asAggBuilders() {
List<AggregationBuilder> aggBuilders = new ArrayList<>(rootAggs.size() + 1);
public AggregationBuilder asAggBuilder() {
AggregationBuilder rootGroup = null;
for (Agg agg : rootAggs) {
aggBuilders.add(agg.toBuilder());
if (groups.isEmpty() && metricAggs.isEmpty()) {
return null;
}
// if there's a group, move everything under the composite agg
if (!groups.isEmpty()) {
AggregationBuilder root = null, last = null;
for (GroupingAgg groupBy : groups) {
AggregationBuilder builder = groupBy.toBuilder();
if (root == null) {
root = builder;
}
// chain each group
if (last != null) {
last.subAggregation(builder);
}
last = builder;
List<CompositeValuesSourceBuilder<?>> keys = new ArrayList<>(groups.size());
// first iterate to compute the sources
for (GroupByKey key : groups) {
keys.add(key.asValueSource());
}
aggBuilders.add(root);
rootGroup = new CompositeAggregationBuilder(ROOT_GROUP_NAME, keys);
} else {
rootGroup = new FiltersAggregationBuilder(ROOT_GROUP_NAME, matchAllQuery());
}
return aggBuilders;
for (LeafAgg agg : metricAggs) {
rootGroup.subAggregation(agg.toBuilder());
}
for (PipelineAgg agg : pipelineAggs) {
rootGroup.subAggregation(agg.toBuilder());
}
return rootGroup;
}
public List<PipelineAggregationBuilder> asPipelineBuilders() {
List<PipelineAggregationBuilder> aggBuilders = new ArrayList<>(rootPipelineAggs.size() + 1);
for (PipelineAgg agg : rootPipelineAggs) {
aggBuilders.add(agg.toBuilder());
}
return aggBuilders;
public boolean useImplicitGroupBy() {
return groups.isEmpty();
}
public Aggs addGroups(Collection<GroupingAgg> groups) {
return new Aggs(rootAggs, rootPipelineAggs, combine(this.groups, groups));
public Aggs addGroups(Collection<GroupByKey> groups) {
return new Aggs(combine(this.groups, groups), metricAggs, pipelineAggs);
}
public Aggs addAgg(Agg agg) {
if (agg instanceof GroupingAgg) {
GroupingAgg group = (GroupingAgg) agg;
return with(combine(this.groups, group));
}
if (agg instanceof LeafAgg) {
LeafAgg leaf = (LeafAgg) agg;
if (groups.isEmpty()) {
return new Aggs(combine(rootAggs, leaf), rootPipelineAggs, this.groups);
}
else {
// copy the groups and replace the tail
List<GroupingAgg> groups = new ArrayList<>(this.groups);
GroupingAgg tail = groups.get(groups.size() - 1);
groups.set(groups.size() - 1, tail.withAggs(combine(tail.subAggs(), leaf)));
return with(groups);
}
}
throw new SqlIllegalArgumentException("Does not know how to handle type {}", agg);
public Aggs addAgg(LeafAgg agg) {
return new Aggs(groups, combine(metricAggs, agg), pipelineAggs);
}
public Aggs addAgg(PipelineAgg pipelineAgg) {
if (groups.isEmpty()) {
return new Aggs(rootAggs, combine(rootPipelineAggs, pipelineAgg), this.groups);
}
else {
// copy the groups and replace the tail
List<GroupingAgg> groups = new ArrayList<>(this.groups);
GroupingAgg tail = groups.get(groups.size() - 1);
groups.set(groups.size() - 1, tail.withPipelines(combine(tail.subPipelines(), pipelineAgg)));
return with(groups);
}
return new Aggs(groups, metricAggs, combine(pipelineAggs, pipelineAgg));
}
public Aggs addAgg(String groupId, LeafAgg child) {
// it's a root agg
if (groupId == null) {
return new Aggs(combine(rootAggs, child), rootPipelineAggs, groups());
}
List<GroupingAgg> groups = new ArrayList<>(this.groups);
for (int i = 0; i < groups.size(); i++) {
GroupingAgg group = groups.get(i);
if (groupId.equals(group.id())) {
groups.set(i, group.withAggs(combine(group.subAggs(), child)));
return with(groups);
}
}
throw new SqlIllegalArgumentException("Could not find group named {}", groupId);
}
public Aggs addAgg(String groupId, PipelineAgg child) {
// it's a root agg
if (groupId == null) {
return new Aggs(rootAggs, combine(rootPipelineAggs, child), groups());
}
List<GroupingAgg> groups = new ArrayList<>(this.groups);
for (int i = 0; i < groups.size(); i++) {
GroupingAgg group = groups.get(i);
if (groupId.equals(group.id())) {
groups.set(i, group.withPipelines(combine(group.subPipelines(), child)));
return with(groups);
}
}
throw new SqlIllegalArgumentException("Could not find group named {}", groupId);
}
public GroupingAgg findGroupForAgg(String groupOrAggId) {
for (GroupingAgg group : this.groups) {
public GroupByKey findGroupForAgg(String groupOrAggId) {
for (GroupByKey group : this.groups) {
if (groupOrAggId.equals(group.id())) {
return group;
}
for (LeafAgg subAgg : group.subAggs()) {
if (groupOrAggId.equals(subAgg.id())) {
return group;
}
}
}
// is it maybe a root agg
for (Agg agg : rootAggs) {
// maybe it's the default group agg ?
for (Agg agg : metricAggs) {
if (groupOrAggId.equals(agg.id())) {
return GroupingAgg.DEFAULT_GROUP;
return IMPLICIT_GROUP_KEY;
}
}
return null;
}
public Aggs updateGroup(GroupingAgg group) {
List<GroupingAgg> groups = new ArrayList<>(this.groups);
public Aggs updateGroup(GroupByKey group) {
List<GroupByKey> groups = new ArrayList<>(this.groups);
for (int i = 0; i < groups.size(); i++) {
GroupingAgg g = groups.get(i);
GroupByKey g = groups.get(i);
if (group.id().equals(g.id())) {
groups.set(i, group);
return with(groups);
@ -202,13 +148,13 @@ public class Aggs {
throw new SqlIllegalArgumentException("Could not find group named {}", group.id());
}
public Aggs with(List<GroupingAgg> groups) {
return new Aggs(rootAggs, rootPipelineAggs, groups);
public Aggs with(List<GroupByKey> groups) {
return new Aggs(groups, metricAggs, pipelineAggs);
}
@Override
public int hashCode() {
return Objects.hash(rootAggs, rootPipelineAggs, groups);
return Objects.hash(groups, metricAggs, pipelineAggs);
}
@Override
@ -222,8 +168,9 @@ public class Aggs {
}
Aggs other = (Aggs) obj;
return Objects.equals(rootAggs, other.rootAggs)
&& Objects.equals(rootPipelineAggs, other.rootPipelineAggs)
&& Objects.equals(groups, other.groups);
return Objects.equals(groups, other.groups)
&& Objects.equals(metricAggs, other.metricAggs)
&& Objects.equals(pipelineAggs, other.pipelineAggs);
}
}
}

View File

@ -11,8 +11,8 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.avg;
public class AvgAgg extends LeafAgg {
public AvgAgg(String id, String propertyPath, String fieldName) {
super(id, propertyPath, fieldName);
public AvgAgg(String id, String fieldName) {
super(id, fieldName);
}
@Override

View File

@ -11,8 +11,8 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.cardinal
public class CardinalityAgg extends LeafAgg {
public CardinalityAgg(String id, String propertyPath, String fieldName) {
super(id, propertyPath, fieldName);
public CardinalityAgg(String id, String fieldName) {
super(id, fieldName);
}
@Override AggregationBuilder toBuilder() {

View File

@ -11,8 +11,8 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.extended
public class ExtendedStatsAgg extends LeafAgg {
public ExtendedStatsAgg(String id, String propertyPath, String fieldName) {
super(id, propertyPath, fieldName);
public ExtendedStatsAgg(String id, String fieldName) {
super(id, fieldName);
}
@Override

View File

@ -1,86 +0,0 @@
/*
* 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.sql.querydsl.agg;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.BucketOrder;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
import org.elasticsearch.xpack.sql.querydsl.container.Sort;
import org.elasticsearch.xpack.sql.querydsl.container.Sort.Direction;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import static java.util.Collections.emptyList;
import static java.util.Collections.emptyMap;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
public class GroupByColumnAgg extends GroupingAgg {
private static final int DEFAULT_LIMIT = 512;
private final int limit;
public GroupByColumnAgg(String id, String propertyPath, String fieldName) {
this(id, propertyPath, fieldName, emptyList(), emptyList(), emptyMap(), -1);
}
public GroupByColumnAgg(String id, String propertyPath, String fieldName, List<LeafAgg> subAggs,
List<PipelineAgg> subPipelines, Map<String, Direction> order, int limit) {
super(id, propertyPath, fieldName, subAggs, subPipelines, order);
this.limit = limit < 0 ? DEFAULT_LIMIT : Math.min(limit, DEFAULT_LIMIT);
}
public int limit() {
return limit;
}
@Override
protected AggregationBuilder toGroupingAgg() {
// TODO: the size should be configurable
TermsAggregationBuilder terms = termsTarget(terms(id()).size(limit));
List<BucketOrder> termOrders = emptyList();
if (!order().isEmpty()) {
termOrders = new ArrayList<>();
for (Entry<String, Sort.Direction> entry : order().entrySet()) {
String key = entry.getKey();
boolean asc = entry.getValue() == Direction.ASC;
BucketOrder o = null;
// special cases
if (GROUP_KEY_SORTING.equals(key)) {
o = BucketOrder.key(asc);
}
else if (GROUP_COUNT_SORTING.equals(key)) {
o = BucketOrder.count(asc);
}
else {
o = BucketOrder.aggregation(key, asc);
}
termOrders.add(o);
}
terms.order(termOrders);
}
terms.minDocCount(1);
return terms;
}
protected TermsAggregationBuilder termsTarget(TermsAggregationBuilder builder) {
return builder.field(fieldName());
}
@Override
protected GroupByColumnAgg copy(String id, String propertyPath, String fieldName, List<LeafAgg> subAggs,
List<PipelineAgg> subPipelines, Map<String, Direction> order) {
return new GroupByColumnAgg(id, propertyPath, fieldName, subAggs, subPipelines, order, limit);
}
public GroupByColumnAgg withLimit(int limit) {
return new GroupByColumnAgg(id(), propertyPath(), fieldName(), subAggs(), subPipelines(), order(), limit);
}
}

View File

@ -0,0 +1,35 @@
/*
* 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.sql.querydsl.agg;
import org.elasticsearch.search.aggregations.bucket.composite.TermsValuesSourceBuilder;
import org.elasticsearch.xpack.sql.querydsl.container.Sort.Direction;
/**
* GROUP BY key for regular fields.
*/
public class GroupByColumnKey extends GroupByKey {
public GroupByColumnKey(String id, String fieldName) {
this(id, fieldName, null);
}
public GroupByColumnKey(String id, String fieldName, Direction direction) {
super(id, fieldName, direction);
}
@Override
public TermsValuesSourceBuilder asValueSource() {
return new TermsValuesSourceBuilder(id())
.field(fieldName())
.order(direction().asOrder());
}
@Override
protected GroupByKey copy(String id, String fieldName, Direction direction) {
return new GroupByColumnKey(id, fieldName, direction);
}
}

View File

@ -1,77 +0,0 @@
/*
* 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.sql.querydsl.agg;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.BucketOrder;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
import org.elasticsearch.xpack.sql.querydsl.container.Sort;
import org.elasticsearch.xpack.sql.querydsl.container.Sort.Direction;
import org.joda.time.DateTimeZone;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import static java.util.Collections.emptyList;
import static java.util.Collections.emptyMap;
import static org.elasticsearch.search.aggregations.AggregationBuilders.dateHistogram;
public class GroupByDateAgg extends GroupingAgg {
private final String interval;
private final DateTimeZone timeZone;
public GroupByDateAgg(String id, String propertyPath, String fieldName, String interval, DateTimeZone timeZone) {
this(id, propertyPath, fieldName, interval, timeZone, emptyList(), emptyList(), emptyMap());
}
public GroupByDateAgg(String id, String propertyPath, String fieldName, String interval, DateTimeZone timeZone,
List<LeafAgg> subAggs, List<PipelineAgg> subPipelines, Map<String, Direction> order) {
super(id, propertyPath, fieldName, subAggs, subPipelines, order);
this.interval = interval;
this.timeZone = timeZone;
}
public String interval() {
return interval;
}
@Override
protected AggregationBuilder toGroupingAgg() {
DateHistogramAggregationBuilder dhab = dateHistogram(id())
.field(fieldName())
.timeZone(timeZone)
.dateHistogramInterval(new DateHistogramInterval(interval));
if (!order().isEmpty()) {
for (Entry<String, Sort.Direction> entry : order().entrySet()) {
String key = entry.getKey();
boolean asc = entry.getValue() == Direction.ASC;
// special cases
if (GROUP_KEY_SORTING.equals(key)) {
dhab.order(BucketOrder.key(asc));
}
else if (GROUP_COUNT_SORTING.equals(key)) {
dhab.order(BucketOrder.count(asc));
}
else {
dhab.order(BucketOrder.aggregation(key, asc));
}
}
}
dhab.minDocCount(1);
return dhab;
}
@Override
protected GroupingAgg copy(String id, String propertyPath, String fieldName, List<LeafAgg> subAggs,
List<PipelineAgg> subPipelines, Map<String, Direction> order) {
return new GroupByDateAgg(id, propertyPath, fieldName, interval, timeZone, subAggs, subPipelines, order);
}
}

View File

@ -0,0 +1,68 @@
/*
* 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.sql.querydsl.agg;
import org.elasticsearch.search.aggregations.bucket.composite.DateHistogramValuesSourceBuilder;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
import org.elasticsearch.xpack.sql.querydsl.container.Sort.Direction;
import org.joda.time.DateTimeZone;
import java.util.Objects;
/**
* GROUP BY key specific for date fields.
*/
public class GroupByDateKey extends GroupByKey {
private final String interval;
private final DateTimeZone timeZone;
public GroupByDateKey(String id, String fieldName, String interval, DateTimeZone timeZone) {
this(id, fieldName, null, interval, timeZone);
}
public GroupByDateKey(String id, String fieldName, Direction direction, String interval, DateTimeZone timeZone) {
super(id, fieldName, direction);
this.interval = interval;
this.timeZone = timeZone;
}
public String interval() {
return interval;
}
public DateTimeZone timeZone() {
return timeZone;
}
@Override
public DateHistogramValuesSourceBuilder asValueSource() {
return new DateHistogramValuesSourceBuilder(id())
.field(fieldName())
.dateHistogramInterval(new DateHistogramInterval(interval))
.timeZone(timeZone);
}
@Override
protected GroupByKey copy(String id, String fieldName, Direction direction) {
return new GroupByDateKey(id, fieldName, direction, interval, timeZone);
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), interval, timeZone);
}
@Override
public boolean equals(Object obj) {
if (super.equals(obj)) {
GroupByDateKey other = (GroupByDateKey) obj;
return Objects.equals(interval, other.interval)
&& Objects.equals(timeZone, other.timeZone);
}
return false;
}
}

View File

@ -0,0 +1,47 @@
/*
* 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.sql.querydsl.agg;
import org.elasticsearch.search.aggregations.bucket.composite.CompositeValuesSourceBuilder;
import org.elasticsearch.xpack.sql.querydsl.container.Sort.Direction;
import java.util.Objects;
/**
* A key for a SQL GroupBy which maps to value source for composite aggregation.
*/
public abstract class GroupByKey extends Agg {
private final Direction direction;
GroupByKey(String id, String fieldName, Direction direction) {
super(id, fieldName);
// ASC is the default order of CompositeValueSource
this.direction = direction == null ? Direction.ASC : direction;
}
public Direction direction() {
return direction;
}
public abstract CompositeValuesSourceBuilder<?> asValueSource();
protected abstract GroupByKey copy(String id, String fieldName, Direction direction);
public GroupByKey with(Direction direction) {
return this.direction == direction ? this : copy(id(), fieldName(), direction);
}
@Override
public int hashCode() {
return Objects.hash(id(), fieldName(), direction);
}
@Override
public boolean equals(Object obj) {
return super.equals(obj) && Objects.equals(direction, ((GroupByKey) obj).direction);
}
}

View File

@ -1,57 +0,0 @@
/*
* 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.sql.querydsl.agg;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.xpack.sql.expression.function.scalar.script.ScriptTemplate;
import org.elasticsearch.xpack.sql.querydsl.container.Sort.Direction;
import java.util.List;
import java.util.Map;
import static java.util.Collections.emptyList;
import static java.util.Collections.emptyMap;
public class GroupByScriptAgg extends GroupByColumnAgg {
private final ScriptTemplate script;
public GroupByScriptAgg(String id, String propertyPath, String fieldName, ScriptTemplate script) {
this(id, propertyPath, fieldName, script, emptyList(), emptyList(), emptyMap(), -1);
}
public GroupByScriptAgg(String id, String propertyPath, String fieldName, ScriptTemplate script, List<LeafAgg> subAggs,
List<PipelineAgg> subPipelines, Map<String, Direction> order, int limit) {
super(id, propertyPath, fieldName, subAggs, subPipelines, order, limit);
this.script = script;
}
public ScriptTemplate script() {
return script;
}
@Override
protected TermsAggregationBuilder termsTarget(TermsAggregationBuilder builder) {
builder.script(script.toPainless());
if (script.outputType().isNumeric()) {
builder.valueType(ValueType.NUMBER);
}
return builder;
}
@Override
protected GroupByScriptAgg copy(String id, String propertyPath, String fieldName, List<LeafAgg> subAggs,
List<PipelineAgg> subPipelines, Map<String, Direction> order) {
return new GroupByScriptAgg(id, propertyPath, fieldName, script, subAggs, subPipelines, order, limit());
}
@Override
public GroupByScriptAgg withLimit(int limit) {
return new GroupByScriptAgg(id(), propertyPath(), fieldName(), script, subAggs(), subPipelines(), order(), limit);
}
}

View File

@ -0,0 +1,62 @@
/*
* 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.sql.querydsl.agg;
import org.elasticsearch.search.aggregations.bucket.composite.TermsValuesSourceBuilder;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.xpack.sql.expression.function.scalar.script.ScriptTemplate;
import org.elasticsearch.xpack.sql.querydsl.container.Sort.Direction;
import java.util.Objects;
/**
* GROUP BY key for scripts (typically caused by functions).
*/
public class GroupByScriptKey extends GroupByKey {
private final ScriptTemplate script;
public GroupByScriptKey(String id, String fieldName, ScriptTemplate script) {
this(id, fieldName, null, script);
}
public GroupByScriptKey(String id, String fieldName, Direction direction, ScriptTemplate script) {
super(id, fieldName, direction);
this.script = script;
}
public ScriptTemplate script() {
return script;
}
@Override
public TermsValuesSourceBuilder asValueSource() {
TermsValuesSourceBuilder builder = new TermsValuesSourceBuilder(id())
.script(script.toPainless())
.order(direction().asOrder());
if (script.outputType().isNumeric()) {
builder.valueType(ValueType.NUMBER);
}
return builder;
}
@Override
protected GroupByKey copy(String id, String fieldName, Direction direction) {
return new GroupByScriptKey(id, fieldName, direction, script);
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), script);
}
@Override
public boolean equals(Object obj) {
return super.equals(obj) && Objects.equals(script, ((GroupByScriptKey) obj).script);
}
}

View File

@ -1,133 +0,0 @@
/*
* 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.sql.querydsl.agg;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.elasticsearch.xpack.sql.querydsl.container.Sort.Direction;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import static java.util.Collections.emptyList;
import static org.elasticsearch.xpack.sql.util.StringUtils.EMPTY;
public abstract class GroupingAgg extends Agg {
protected static final String GROUP_KEY_SORTING = "_key";
protected static final String GROUP_COUNT_SORTING = "_count";
public static final GroupingAgg DEFAULT_GROUP = new GroupingAgg(EMPTY, EMPTY, EMPTY, emptyList(), emptyList(), null) {
@Override
protected AggregationBuilder toGroupingAgg() {
throw new SqlIllegalArgumentException("Default group cannot build aggregations");
}
@Override
protected GroupingAgg copy(String id, String propertyPath, String fieldName, List<LeafAgg> subAggs,
List<PipelineAgg> subPipelines, Map<String, Direction> order) {
throw new SqlIllegalArgumentException("Default group cannot be cloned");
}
};
private final List<LeafAgg> subAggs;
private final List<PipelineAgg> subPipelines;
private final Map<String, Direction> order;
GroupingAgg(String id, String propertyPath, String fieldName, List<LeafAgg> subAggs,
List<PipelineAgg> subPipelines, Map<String, Direction> order) {
super(id, propertyPath, fieldName);
this.subAggs = subAggs;
this.subPipelines = subPipelines;
this.order = order;
}
public List<LeafAgg> subAggs() {
return subAggs;
}
public List<PipelineAgg> subPipelines() {
return subPipelines;
}
public Map<String, Direction> order() {
return order;
}
@Override
AggregationBuilder toBuilder() {
AggregationBuilder groupingAgg = toGroupingAgg();
for (LeafAgg leafAgg : subAggs) {
groupingAgg.subAggregation(leafAgg.toBuilder());
}
for (PipelineAgg aggFilter : subPipelines) {
groupingAgg.subAggregation(aggFilter.toBuilder());
}
return groupingAgg;
}
protected abstract AggregationBuilder toGroupingAgg();
public GroupingAgg withAggs(List<LeafAgg> subAggs) {
return copy(id(), propertyPath(), fieldName(), subAggs, subPipelines, order);
}
public GroupingAgg withPipelines(List<PipelineAgg> subPipelines) {
return copy(id(), propertyPath(), fieldName(), subAggs, subPipelines, order);
}
public GroupingAgg with(String id) {
return Objects.equals(id(), id) ? this : copy(id, propertyPath(), fieldName(), subAggs, subPipelines, order);
}
public GroupingAgg with(Direction order) {
return with(GROUP_KEY_SORTING, order);
}
public GroupingAgg with(String leafAggId, Direction order) {
if (Objects.equals(this.order.get(leafAggId), order)) {
return this;
}
Map<String, Direction> newOrder = new LinkedHashMap<>(this.order);
newOrder.put(leafAggId, order);
return copy(id(), propertyPath(), fieldName(), subAggs, subPipelines, newOrder);
}
protected abstract GroupingAgg copy(String id, String propertyPath, String fieldName, List<LeafAgg> subAggs,
List<PipelineAgg> subPipelines, Map<String, Direction> order);
@Override
public int hashCode() {
return Objects.hash(order, id(), propertyPath(), fieldName(), subAggs(), subPipelines());
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
GroupByColumnAgg other = (GroupByColumnAgg) obj;
return Objects.equals(id(), other.id())
&& Objects.equals(propertyPath(), other.propertyPath())
&& Objects.equals(fieldName(), other.fieldName())
&& Objects.equals(subAggs(), other.subAggs())
&& Objects.equals(subPipelines(), other.subPipelines());
}
@Override
public String toString() {
return super.toString() + "=" + subAggs() + "|" + subPipelines();
}
}

View File

@ -5,9 +5,13 @@
*/
package org.elasticsearch.xpack.sql.querydsl.agg;
import org.elasticsearch.search.aggregations.AggregationBuilder;
public abstract class LeafAgg extends Agg {
LeafAgg(String id, String propertyPath, String fieldName) {
super(id, propertyPath, fieldName);
LeafAgg(String id, String fieldName) {
super(id, fieldName);
}
abstract AggregationBuilder toBuilder();
}

View File

@ -15,8 +15,8 @@ public class MatrixStatsAgg extends LeafAgg {
private final List<String> fields;
public MatrixStatsAgg(String id, String propertyPath, List<String> fields) {
super(id, propertyPath, "<multi-field>");
public MatrixStatsAgg(String id, List<String> fields) {
super(id, "<multi-field>");
this.fields = fields;
}

View File

@ -11,8 +11,8 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.max;
public class MaxAgg extends LeafAgg {
public MaxAgg(String id, String propertyPath, String fieldName) {
super(id, propertyPath, fieldName);
public MaxAgg(String id, String fieldName) {
super(id, fieldName);
}
@Override

View File

@ -11,8 +11,8 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.min;
public class MinAgg extends LeafAgg {
public MinAgg(String id, String propertyPath, String fieldName) {
super(id, propertyPath, fieldName);
public MinAgg(String id, String fieldName) {
super(id, fieldName);
}
@Override

View File

@ -15,8 +15,8 @@ public class PercentileRanksAgg extends LeafAgg {
private final List<Double> values;
public PercentileRanksAgg(String id, String propertyPath, String fieldName, List<Double> values) {
super(id, propertyPath, fieldName);
public PercentileRanksAgg(String id, String fieldName, List<Double> values) {
super(id, fieldName);
this.values = values;
}

View File

@ -15,8 +15,8 @@ public class PercentilesAgg extends LeafAgg {
private final List<Double> percents;
public PercentilesAgg(String id, String propertyPath, String fieldName, List<Double> percents) {
super(id, propertyPath, fieldName);
public PercentilesAgg(String id, String fieldName, List<Double> percents) {
super(id, fieldName);
this.percents = percents;
}

View File

@ -11,8 +11,8 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.stats;
public class StatsAgg extends LeafAgg {
public StatsAgg(String id, String propertyPath, String fieldName) {
super(id, propertyPath, fieldName);
public StatsAgg(String id, String fieldName) {
super(id, fieldName);
}
@Override

View File

@ -11,8 +11,8 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
public class SumAgg extends LeafAgg {
public SumAgg(String id, String propertyPath, String fieldName) {
super(id, propertyPath, fieldName);
public SumAgg(String id, String fieldName) {
super(id, fieldName);
}
@Override AggregationBuilder toBuilder() {

View File

@ -1,55 +0,0 @@
/*
* 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.sql.querydsl.container;
import org.elasticsearch.xpack.sql.execution.search.FieldExtraction;
import org.elasticsearch.xpack.sql.execution.search.SqlSourceBuilder;
import org.elasticsearch.xpack.sql.querydsl.agg.AggPath;
public class AggRef implements FieldExtraction {
private final String path;
private final int depth;
private final String innerKey;
public AggRef(String path) {
this(path, null);
}
public AggRef(String path, String innerKey) {
this.path = path;
depth = AggPath.depth(path);
this.innerKey = innerKey;
}
@Override
public String toString() {
return path;
}
@Override
public int depth() {
return depth;
}
public String path() {
return path;
}
public String innerKey() {
return innerKey;
}
@Override
public void collectFields(SqlSourceBuilder sourceBuilder) {
// Aggregations do not need any special fields
}
@Override
public boolean supportedByAggsOnlyQuery() {
return true;
}
}

View File

@ -21,11 +21,6 @@ public class ComputedRef implements FieldExtraction {
return processor;
}
@Override
public int depth() {
return processor.depth();
}
@Override
public boolean supportedByAggsOnlyQuery() {
return processor.supportedByAggsOnlyQuery();

View File

@ -15,11 +15,6 @@ public abstract class FieldReference implements FieldExtraction {
*/
public abstract String name();
@Override
public final int depth() {
return 0;
}
@Override
public final boolean supportedByAggsOnlyQuery() {
return false;

View File

@ -0,0 +1,21 @@
/*
* 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.sql.querydsl.container;
import org.elasticsearch.xpack.sql.execution.search.AggRef;
/**
* Aggregation reference pointing to the (so called) global count, meaning
* COUNT over the entire data set.
*/
public final class GlobalCountRef extends AggRef {
public static final GlobalCountRef INSTANCE = new GlobalCountRef();
@Override
public String toString() {
return "#_Total_Hits_#";
}
}

View File

@ -0,0 +1,50 @@
/*
* 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.sql.querydsl.container;
import org.elasticsearch.xpack.sql.execution.search.AggRef;
import org.joda.time.DateTimeZone;
/**
* Reference to a GROUP BY agg (typically this gets translated to a composite key).
*/
public class GroupByRef extends AggRef {
public enum Property {
VALUE, COUNT;
}
private final String key;
private final Property property;
private final DateTimeZone timeZone;
public GroupByRef(String key) {
this(key, null, null);
}
public GroupByRef(String key, Property property, DateTimeZone timeZone) {
this.key = key;
this.property = property == null ? Property.VALUE : property;
this.timeZone = timeZone;
}
public String key() {
return key;
}
public Property property() {
return property;
}
public DateTimeZone timeZone() {
return timeZone;
}
@Override
public String toString() {
return "|" + key + (property == Property.COUNT ? ".count" : "") + "|";
}
}

View File

@ -0,0 +1,52 @@
/*
* 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.sql.querydsl.container;
import org.elasticsearch.xpack.sql.execution.search.AggRef;
import org.elasticsearch.xpack.sql.querydsl.agg.Aggs;
/**
* Reference to a sub/nested/metric aggregation.
* Due to how ES query works, this is _always_ a child aggregation with the grouping (composite agg) as the parent.
*/
public class MetricAggRef extends AggRef {
private final String name;
private final String property;
private final String innerKey;
public MetricAggRef(String name) {
this(name, "value");
}
public MetricAggRef(String name, String property) {
this(name, property, null);
}
public MetricAggRef(String name, String property, String innerKey) {
this.name = name;
this.property = property;
this.innerKey = innerKey;
}
public String name() {
return name;
}
public String property() {
return property;
}
public String innerKey() {
return innerKey;
}
@Override
public String toString() {
String i = innerKey != null ? "[" + innerKey + "]" : "";
return Aggs.ROOT_GROUP_NAME + ">" + name + "." + property + i;
}
}

View File

@ -21,10 +21,10 @@ import org.elasticsearch.xpack.sql.expression.function.ScoreAttribute;
import org.elasticsearch.xpack.sql.expression.function.scalar.ScalarFunctionAttribute;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition.ProcessorDefinition;
import org.elasticsearch.xpack.sql.expression.function.scalar.processor.definition.ScoreProcessorDefinition;
import org.elasticsearch.xpack.sql.querydsl.agg.AggPath;
import org.elasticsearch.xpack.sql.querydsl.agg.Aggs;
import org.elasticsearch.xpack.sql.querydsl.agg.GroupingAgg;
import org.elasticsearch.xpack.sql.querydsl.agg.GroupByKey;
import org.elasticsearch.xpack.sql.querydsl.agg.LeafAgg;
import org.elasticsearch.xpack.sql.querydsl.container.GroupByRef.Property;
import org.elasticsearch.xpack.sql.querydsl.query.BoolQuery;
import org.elasticsearch.xpack.sql.querydsl.query.MatchAll;
import org.elasticsearch.xpack.sql.querydsl.query.NestedQuery;
@ -60,7 +60,7 @@ public class QueryContainer {
private final Map<Attribute, Attribute> aliases;
// pseudo functions (like count) - that are 'extracted' from other aggs
private final Map<String, GroupingAgg> pseudoFunctions;
private final Map<String, GroupByKey> pseudoFunctions;
// scalar function processors - recorded as functions get folded;
// at scrolling, their inputs (leaves) get updated
@ -71,14 +71,13 @@ public class QueryContainer {
// computed
private final boolean aggsOnly;
private final int aggDepth;
public QueryContainer() {
this(null, null, null, null, null, null, null, -1);
}
public QueryContainer(Query query, Aggs aggs, List<FieldExtraction> refs, Map<Attribute, Attribute> aliases,
Map<String, GroupingAgg> pseudoFunctions,
Map<String, GroupByKey> pseudoFunctions,
Map<Attribute, ProcessorDefinition> scalarFunctions,
Set<Sort> sort, int limit) {
this.query = query;
@ -90,7 +89,6 @@ public class QueryContainer {
this.sort = sort == null || sort.isEmpty() ? emptySet() : sort;
this.limit = limit;
aggsOnly = columns.stream().allMatch(FieldExtraction::supportedByAggsOnlyQuery);
aggDepth = columns.stream().mapToInt(FieldExtraction::depth).max().orElse(0);
}
public Query query() {
@ -109,7 +107,7 @@ public class QueryContainer {
return aliases;
}
public Map<String, GroupingAgg> pseudoFunctions() {
public Map<String, GroupByKey> pseudoFunctions() {
return pseudoFunctions;
}
@ -125,10 +123,6 @@ public class QueryContainer {
return aggsOnly;
}
public int aggDepth() {
return aggDepth;
}
public boolean hasColumns() {
return !columns.isEmpty();
}
@ -149,7 +143,7 @@ public class QueryContainer {
return new QueryContainer(query, aggs, columns, a, pseudoFunctions, scalarFunctions, sort, limit);
}
public QueryContainer withPseudoFunctions(Map<String, GroupingAgg> p) {
public QueryContainer withPseudoFunctions(Map<String, GroupByKey> p) {
return new QueryContainer(query, aggs, columns, aliases, p, scalarFunctions, sort, limit);
}
@ -299,39 +293,28 @@ public class QueryContainer {
//
// agg methods
//
public QueryContainer addAggColumn(String aggPath) {
return with(combine(columns, new AggRef(aggPath)));
}
public QueryContainer addAggColumn(String aggPath, String innerKey) {
return with(combine(columns, new AggRef(aggPath, innerKey)));
}
public QueryContainer addAggCount(GroupingAgg parentGroup, String functionId) {
FieldExtraction ref = parentGroup == null ? TotalCountRef.INSTANCE : new AggRef(AggPath.bucketCount(parentGroup.asParentPath()));
Map<String, GroupingAgg> pseudoFunctions = new LinkedHashMap<>(this.pseudoFunctions);
pseudoFunctions.put(functionId, parentGroup);
public QueryContainer addAggCount(GroupByKey group, String functionId) {
FieldExtraction ref = group == null ? GlobalCountRef.INSTANCE : new GroupByRef(group.id(), Property.COUNT, null);
Map<String, GroupByKey> pseudoFunctions = new LinkedHashMap<>(this.pseudoFunctions);
pseudoFunctions.put(functionId, group);
return new QueryContainer(query, aggs, combine(columns, ref), aliases, pseudoFunctions, scalarFunctions, sort, limit);
}
public QueryContainer addAgg(String groupId, LeafAgg agg) {
return addAgg(groupId, agg, agg.propertyPath());
return with(aggs.addAgg(agg));
}
public QueryContainer addAgg(String groupId, LeafAgg agg, String aggRefPath) {
return new QueryContainer(query, aggs.addAgg(groupId, agg), columns, aliases, pseudoFunctions, scalarFunctions, sort, limit);
}
public QueryContainer addGroups(Collection<GroupingAgg> values) {
public QueryContainer addGroups(Collection<GroupByKey> values) {
return with(aggs.addGroups(values));
}
public QueryContainer updateGroup(GroupingAgg group) {
return with(aggs.updateGroup(group));
public GroupByKey findGroupForAgg(String aggId) {
return aggs.findGroupForAgg(aggId);
}
public GroupingAgg findGroupForAgg(String aggId) {
return aggs.findGroupForAgg(aggId);
public QueryContainer updateGroup(GroupByKey group) {
return with(aggs.updateGroup(group));
}
//
@ -372,4 +355,4 @@ public class QueryContainer {
throw new RuntimeException("error rendering", e);
}
}
}
}

View File

@ -5,6 +5,7 @@
*/
package org.elasticsearch.xpack.sql.querydsl.container;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.xpack.sql.expression.Order.OrderDirection;
public class Sort {
@ -15,6 +16,10 @@ public class Sort {
public static Direction from(OrderDirection dir) {
return dir == null || dir == OrderDirection.ASC ? ASC : DESC;
}
public SortOrder asOrder() {
return this == Direction.ASC ? SortOrder.ASC : SortOrder.DESC;
}
}
private final Direction direction;

View File

@ -1,24 +0,0 @@
/*
* 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.sql.querydsl.container;
import org.elasticsearch.xpack.sql.util.StringUtils;
// somewhat of a fake agg (since it gets optimized and it gets its value from the response)
public final class TotalCountRef extends AggRef {
public static final TotalCountRef INSTANCE = new TotalCountRef();
public static final String PATH = "#_count_#";
TotalCountRef() {
super(StringUtils.EMPTY);
}
@Override
public String toString() {
return "TotalCountRef";
}
}

View File

@ -5,97 +5,24 @@
*/
package org.elasticsearch.xpack.sql.session;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.io.stream.InputStreamStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.OutputStreamStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.xpack.sql.execution.search.ScrollCursor;
import org.elasticsearch.xpack.sql.execution.search.extractor.HitExtractors;
import org.elasticsearch.xpack.sql.plugin.CliFormatterCursor;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Base64;
import java.util.List;
/**
* Information required to access the next page of response.
*/
public interface Cursor extends NamedWriteable {
NamedWriteableRegistry CURSOR_REGISTRY = new NamedWriteableRegistry(Cursor.getNamedWriteables());
Cursor EMPTY = EmptyCursor.INSTANCE;
/**
* Request the next page of data.
*/
void nextPage(Configuration cfg, Client client, ActionListener<RowSet> listener);
void nextPage(Configuration cfg, Client client, NamedWriteableRegistry registry, ActionListener<RowSet> listener);
/**
* Cleans the resources associated with the cursor
*/
void clear(Configuration cfg, Client client, ActionListener<Boolean> listener);
/**
* The {@link NamedWriteable}s required to deserialize {@link Cursor}s.
*/
static List<NamedWriteableRegistry.Entry> getNamedWriteables() {
List<NamedWriteableRegistry.Entry> entries = new ArrayList<>();
entries.addAll(HitExtractors.getNamedWriteables());
entries.add(new NamedWriteableRegistry.Entry(Cursor.class, EmptyCursor.NAME, in -> EMPTY));
entries.add(new NamedWriteableRegistry.Entry(Cursor.class, ScrollCursor.NAME, ScrollCursor::new));
entries.add(new NamedWriteableRegistry.Entry(Cursor.class, CliFormatterCursor.NAME, CliFormatterCursor::new));
return entries;
}
/**
* Write a {@linkplain Cursor} to a string for serialization across xcontent.
*/
static String encodeToString(Version version, Cursor info) {
if(info == EMPTY) {
return "";
}
try (ByteArrayOutputStream os = new ByteArrayOutputStream()) {
try (OutputStream base64 = Base64.getEncoder().wrap(os);
StreamOutput out = new OutputStreamStreamOutput(base64)) {
Version.writeVersion(version, out);
out.writeNamedWriteable(info);
}
return os.toString(StandardCharsets.UTF_8.name());
} catch (IOException ex) {
throw new RuntimeException("unexpected failure converting next page info to a string", ex);
}
}
/**
* Read a {@linkplain Cursor} from a string.
*/
static Cursor decodeFromString(String info) {
if (info.isEmpty()) {
return EMPTY;
}
byte[] bytes = info.getBytes(StandardCharsets.UTF_8);
try (StreamInput delegate = new InputStreamStreamInput(Base64.getDecoder().wrap(new ByteArrayInputStream(bytes)));
StreamInput in = new NamedWriteableAwareStreamInput(delegate, CURSOR_REGISTRY)) {
Version version = Version.readVersion(in);
if (version.after(Version.CURRENT)) {
throw new RuntimeException("Unsupported scroll version " + version);
}
in.setVersion(version);
return in.readNamedWriteable(Cursor.class);
} catch (IOException ex) {
throw new RuntimeException("unexpected failure deconding cursor", ex);
}
}
}

View File

@ -0,0 +1,99 @@
/*
* 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.sql.session;
import org.elasticsearch.Version;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.OutputStreamStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.elasticsearch.xpack.sql.execution.search.CompositeAggregationCursor;
import org.elasticsearch.xpack.sql.execution.search.ScrollCursor;
import org.elasticsearch.xpack.sql.execution.search.extractor.BucketExtractors;
import org.elasticsearch.xpack.sql.execution.search.extractor.HitExtractors;
import org.elasticsearch.xpack.sql.expression.function.scalar.Processors;
import org.elasticsearch.xpack.sql.plugin.CliFormatterCursor;
import java.io.ByteArrayOutputStream;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Base64;
import java.util.List;
/**
* Registry and utilities around {@link Cursor}s.
*/
public final class Cursors {
private static final NamedWriteableRegistry WRITEABLE_REGISTRY = new NamedWriteableRegistry(getNamedWriteables());
private Cursors() {};
/**
* The {@link NamedWriteable}s required to deserialize {@link Cursor}s.
*/
public static List<NamedWriteableRegistry.Entry> getNamedWriteables() {
List<NamedWriteableRegistry.Entry> entries = new ArrayList<>();
// cursors
entries.add(new NamedWriteableRegistry.Entry(Cursor.class, EmptyCursor.NAME, in -> Cursor.EMPTY));
entries.add(new NamedWriteableRegistry.Entry(Cursor.class, ScrollCursor.NAME, ScrollCursor::new));
entries.add(new NamedWriteableRegistry.Entry(Cursor.class, CompositeAggregationCursor.NAME, CompositeAggregationCursor::new));
entries.add(new NamedWriteableRegistry.Entry(Cursor.class, CliFormatterCursor.NAME, CliFormatterCursor::new));
// plus all their dependencies
entries.addAll(Processors.getNamedWriteables());
entries.addAll(HitExtractors.getNamedWriteables());
entries.addAll(BucketExtractors.getNamedWriteables());
return entries;
}
/**
* Write a {@linkplain Cursor} to a string for serialization across xcontent.
*/
public static String encodeToString(Version version, Cursor info) {
if (info == Cursor.EMPTY) {
return "";
}
try (ByteArrayOutputStream os = new ByteArrayOutputStream()) {
try (OutputStream base64 = Base64.getEncoder().wrap(os); StreamOutput out = new OutputStreamStreamOutput(base64)) {
Version.writeVersion(version, out);
out.writeNamedWriteable(info);
}
return os.toString(StandardCharsets.UTF_8.name());
} catch (Exception ex) {
throw new SqlIllegalArgumentException("Unexpected failure retriving next page", ex);
}
}
/**
* Read a {@linkplain Cursor} from a string.
*/
public static Cursor decodeFromString(String info) {
if (info.isEmpty()) {
return Cursor.EMPTY;
}
byte[] bytes = info.getBytes(StandardCharsets.UTF_8);
try (StreamInput in = new NamedWriteableAwareStreamInput(StreamInput.wrap(Base64.getDecoder().decode(bytes)), WRITEABLE_REGISTRY)) {
Version version = Version.readVersion(in);
if (version.after(Version.CURRENT)) {
throw new SqlIllegalArgumentException("Unsupported cursor version " + version);
}
in.setVersion(version);
return in.readNamedWriteable(Cursor.class);
} catch (SqlIllegalArgumentException ex) {
throw ex;
} catch (Exception ex) {
throw new SqlIllegalArgumentException("Unexpected failure decoding cursor", ex);
}
}
}

View File

@ -7,6 +7,7 @@ package org.elasticsearch.xpack.sql.session;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
@ -30,7 +31,7 @@ class EmptyCursor implements Cursor {
}
@Override
public void nextPage(Configuration cfg, Client client, ActionListener<RowSet> listener) {
public void nextPage(Configuration cfg, Client client, NamedWriteableRegistry registry, ActionListener<RowSet> listener) {
throw new IllegalArgumentException("there is no next page");
}

View File

@ -5,14 +5,14 @@
*/
package org.elasticsearch.xpack.sql.session;
import java.util.ArrayList;
import java.util.List;
import org.elasticsearch.xpack.sql.expression.Attribute;
import org.elasticsearch.xpack.sql.type.DataType;
import org.elasticsearch.xpack.sql.type.Schema;
import org.elasticsearch.xpack.sql.util.Check;
import java.util.ArrayList;
import java.util.List;
public abstract class Rows {
public static Schema schema(List<Attribute> attr) {
@ -40,8 +40,12 @@ public abstract class Rows {
}
public static SchemaRowSet singleton(List<Attribute> attrs, Object... values) {
Check.isTrue(attrs.size() == values.length, "Schema %s and values %s are out of sync", attrs, values);
return new SingletonRowSet(schema(attrs), values);
return singleton(schema(attrs), values);
}
public static SchemaRowSet singleton(Schema schema, Object... values) {
Check.isTrue(schema.size() == values.length, "Schema {} and values {} are out of sync", schema, values);
return new SingletonRowSet(schema, values);
}
public static SchemaRowSet empty(Schema schema) {

View File

@ -17,7 +17,7 @@ public class SingletonExecutable implements Executable {
private final Object[] values;
public SingletonExecutable(List<Attribute> output, Object... values) {
Check.isTrue(output.size() == values.length, "Output %s and values %s are out of sync", output, values);
Check.isTrue(output.size() == values.length, "Attributes {} and values {} are out of sync", output, values);
this.output = output;
this.values = values;
}

View File

@ -162,4 +162,4 @@ public class SqlSession {
public Configuration settings() {
return settings;
}
}
}

View File

@ -5,6 +5,9 @@
*/
package org.elasticsearch.xpack.sql.type;
import org.elasticsearch.xpack.sql.type.Schema.Entry;
import org.elasticsearch.xpack.sql.util.Check;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
@ -13,9 +16,6 @@ import java.util.Spliterators;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
import org.elasticsearch.xpack.sql.type.Schema.Entry;
import org.elasticsearch.xpack.sql.util.Check;
import static java.util.Collections.emptyList;
public class Schema implements Iterable<Entry> {
@ -51,7 +51,7 @@ public class Schema implements Iterable<Entry> {
private final List<DataType> types;
public Schema(List<String> names, List<DataType> types) {
Check.isTrue(names.size() == types.size(), "Different # of names %s vs types %s", names, types);
Check.isTrue(names.size() == types.size(), "Different # of names {} vs types {}", names, types);
this.types = types;
this.names = names;
}

View File

@ -0,0 +1,70 @@
/*
* 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.sql.execution.search;
import org.elasticsearch.Version;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.test.AbstractWireSerializingTestCase;
import org.elasticsearch.xpack.sql.execution.search.extractor.BucketExtractor;
import org.elasticsearch.xpack.sql.execution.search.extractor.CompositeKeyExtractorTests;
import org.elasticsearch.xpack.sql.execution.search.extractor.ConstantExtractorTests;
import org.elasticsearch.xpack.sql.execution.search.extractor.MetricAggExtractorTests;
import org.elasticsearch.xpack.sql.session.Cursors;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.function.Supplier;
public class CompositeAggregationCursorTests extends AbstractWireSerializingTestCase<CompositeAggregationCursor> {
public static CompositeAggregationCursor randomCompositeCursor() {
int extractorsSize = between(1, 20);
List<BucketExtractor> extractors = new ArrayList<>(extractorsSize);
for (int i = 0; i < extractorsSize; i++) {
extractors.add(randomBucketExtractor());
}
return new CompositeAggregationCursor(new byte[randomInt(256)], extractors, randomIntBetween(10, 1024), randomAlphaOfLength(5));
}
static BucketExtractor randomBucketExtractor() {
List<Supplier<BucketExtractor>> options = new ArrayList<>();
options.add(ConstantExtractorTests::randomConstantExtractor);
options.add(MetricAggExtractorTests::randomMetricAggExtractor);
options.add(CompositeKeyExtractorTests::randomCompositeKeyExtractor);
return randomFrom(options).get();
}
@Override
protected CompositeAggregationCursor mutateInstance(CompositeAggregationCursor instance) throws IOException {
return new CompositeAggregationCursor(instance.next(), instance.extractors(), randomIntBetween(1, 512), instance.indices());
}
@Override
protected NamedWriteableRegistry getNamedWriteableRegistry() {
return new NamedWriteableRegistry(Cursors.getNamedWriteables());
}
@Override
protected CompositeAggregationCursor createTestInstance() {
return randomCompositeCursor();
}
@Override
protected Reader<CompositeAggregationCursor> instanceReader() {
return CompositeAggregationCursor::new;
}
@Override
protected CompositeAggregationCursor copyInstance(CompositeAggregationCursor instance, Version version) throws IOException {
/* Randomly choose between internal protocol round trip and String based
* round trips used to toXContent. */
if (randomBoolean()) {
return super.copyInstance(instance, version);
}
return (CompositeAggregationCursor) Cursors.decodeFromString(Cursors.encodeToString(version, instance));
}
}

Some files were not shown because too many files have changed in this diff Show More