From e8096980a017aef3a3318f113d161e3dc48f1686 Mon Sep 17 00:00:00 2001 From: penghuo Date: Mon, 13 Jan 2020 10:47:01 -0800 Subject: [PATCH 1/5] update --- .../sql/domain/ColumnTypeProvider.java | 92 ++++++ .../sql/domain/QueryActionRequest.java | 31 +++ .../executor/QueryActionElasticExecutor.java | 12 + .../adapter/QueryPlanQueryAction.java | 50 ++++ .../adapter/QueryPlanRequestBuilder.java | 63 +++++ .../sql/plugin/RestSqlAction.java | 28 +- .../sql/plugin/SearchDao.java | 8 +- .../sql/query/ESActionFactory.java | 94 ++++++- .../converter/SQLAggregationParser.java | 262 ++++++++++++++++++ .../SQLExprToExpressionConverter.java | 107 +++++++ .../converter/SQLToOperatorConverter.java | 100 +++++++ .../core/BindingTupleQueryPlanner.java | 89 ++++++ .../sql/query/planner/core/ColumnNode.java | 37 +++ .../physical/node/scroll/BindingTupleRow.java | 47 ++++ .../sql/esintgtest/CsvFormatResponseIT.java | 59 ++-- .../esintgtest/PrettyFormatResponseIT.java | 13 +- .../sql/esintgtest/SQLFunctionsIT.java | 56 ++-- .../sql/unittest/JSONRequestTest.java | 23 +- .../domain/ColumnTypeProviderTest.java | 54 ++++ .../unittest/planner/ESActionFactoryTest.java | 77 +++++ 20 files changed, 1212 insertions(+), 90 deletions(-) create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/domain/ColumnTypeProvider.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/domain/QueryActionRequest.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/adapter/QueryPlanQueryAction.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/adapter/QueryPlanRequestBuilder.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLAggregationParser.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLExprToExpressionConverter.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLToOperatorConverter.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/core/BindingTupleQueryPlanner.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/core/ColumnNode.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/BindingTupleRow.java create mode 100644 src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/domain/ColumnTypeProviderTest.java create mode 100644 src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/ESActionFactoryTest.java diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/domain/ColumnTypeProvider.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/domain/ColumnTypeProvider.java new file mode 100644 index 0000000000..3640cf6da9 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/domain/ColumnTypeProvider.java @@ -0,0 +1,92 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.domain; + +import com.amazon.opendistroforelasticsearch.sql.antlr.semantic.types.Type; +import com.amazon.opendistroforelasticsearch.sql.antlr.semantic.types.base.ESDataType; +import com.amazon.opendistroforelasticsearch.sql.antlr.semantic.types.special.Product; +import com.amazon.opendistroforelasticsearch.sql.executor.format.Schema; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * The definition of column type provider + */ +public class ColumnTypeProvider { + private final List typeList; + + private static final Map TYPE_MAP = + new ImmutableMap.Builder() + .put(ESDataType.SHORT, Schema.Type.SHORT) + .put(ESDataType.LONG, Schema.Type.LONG) + .put(ESDataType.INTEGER, Schema.Type.INTEGER) + .put(ESDataType.FLOAT, Schema.Type.FLOAT) + .put(ESDataType.DOUBLE, Schema.Type.DOUBLE) + .put(ESDataType.KEYWORD, Schema.Type.KEYWORD) + .put(ESDataType.TEXT, Schema.Type.TEXT) + .put(ESDataType.STRING, Schema.Type.TEXT) + .put(ESDataType.DATE, Schema.Type.DATE) + .put(ESDataType.BOOLEAN, Schema.Type.BOOLEAN) + .put(ESDataType.UNKNOWN, Schema.Type.DOUBLE) + .build(); + public static final Schema.Type COLUMN_DEFAULT_TYPE = Schema.Type.DOUBLE; + + public ColumnTypeProvider(Type type) { + this.typeList = convertOutputColumnType(type); + } + + public ColumnTypeProvider() { + this.typeList = new ArrayList<>(); + } + + /** + * Get the type of column by index. + * + * @param index column index. + * @return column type. + */ + public Schema.Type get(int index) { + if (typeList.isEmpty()) { + return COLUMN_DEFAULT_TYPE; + } else { + return typeList.get(index); + } + } + + private List convertOutputColumnType(Type type) { + if (type instanceof Product) { + List types = ((Product) type).getTypes(); + return types.stream().map(t -> convertType(t)).collect(Collectors.toList()); + } else if (type instanceof ESDataType) { + return ImmutableList.of(convertType(type)); + } else { + return ImmutableList.of(COLUMN_DEFAULT_TYPE); + } + } + + private Schema.Type convertType(Type type) { + try { + return TYPE_MAP.getOrDefault(type, COLUMN_DEFAULT_TYPE); + } catch (Exception e) { + return COLUMN_DEFAULT_TYPE; + } + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/domain/QueryActionRequest.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/domain/QueryActionRequest.java new file mode 100644 index 0000000000..ed0e478eeb --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/domain/QueryActionRequest.java @@ -0,0 +1,31 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.domain; + +import com.amazon.opendistroforelasticsearch.sql.executor.Format; +import lombok.Getter; +import lombok.RequiredArgsConstructor; + +/** + * The definition of QueryActionRequest. + */ +@Getter +@RequiredArgsConstructor +public class QueryActionRequest { + private final String sql; + private final ColumnTypeProvider typeProvider; + private final Format format; +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/QueryActionElasticExecutor.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/QueryActionElasticExecutor.java index e5205239d2..cf98346643 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/QueryActionElasticExecutor.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/QueryActionElasticExecutor.java @@ -18,6 +18,9 @@ import com.amazon.opendistroforelasticsearch.sql.exception.SqlParseException; import com.amazon.opendistroforelasticsearch.sql.executor.join.ElasticJoinExecutor; import com.amazon.opendistroforelasticsearch.sql.executor.multi.MultiRequestExecutorFactory; +import com.amazon.opendistroforelasticsearch.sql.executor.adapter.QueryPlanQueryAction; +import com.amazon.opendistroforelasticsearch.sql.executor.adapter.QueryPlanRequestBuilder; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; import com.amazon.opendistroforelasticsearch.sql.query.AggregationQueryAction; import com.amazon.opendistroforelasticsearch.sql.query.DefaultQueryAction; import com.amazon.opendistroforelasticsearch.sql.query.DeleteQueryAction; @@ -36,6 +39,7 @@ import org.elasticsearch.search.aggregations.Aggregations; import java.io.IOException; +import java.util.List; /** * Created by Eliran on 3/10/2015. @@ -60,6 +64,11 @@ public static Aggregations executeAggregationAction(AggregationQueryAction aggre return ((SearchResponse) select.get()).getAggregations(); } + public static List executeQueryPlanQueryAction(QueryPlanQueryAction queryPlanQueryAction) { + QueryPlanRequestBuilder select = (QueryPlanRequestBuilder) queryPlanQueryAction.explain(); + return select.execute(); + } + public static ActionResponse executeShowQueryAction(ShowQueryAction showQueryAction) { return showQueryAction.explain().get(); } @@ -89,6 +98,9 @@ public static Object executeAnyAction(Client client, QueryAction queryAction) if (queryAction instanceof AggregationQueryAction) { return executeAggregationAction((AggregationQueryAction) queryAction); } + if (queryAction instanceof QueryPlanQueryAction) { + return executeQueryPlanQueryAction((QueryPlanQueryAction) queryAction); + } if (queryAction instanceof ShowQueryAction) { return executeShowQueryAction((ShowQueryAction) queryAction); } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/adapter/QueryPlanQueryAction.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/adapter/QueryPlanQueryAction.java new file mode 100644 index 0000000000..6f3491186d --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/adapter/QueryPlanQueryAction.java @@ -0,0 +1,50 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.executor.adapter; + +import com.amazon.opendistroforelasticsearch.sql.query.QueryAction; +import com.amazon.opendistroforelasticsearch.sql.query.SqlElasticRequestBuilder; +import com.google.common.base.Strings; + +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * The definition of QueryPlan of QueryAction which works as the adapter to the current QueryAction framework. + */ +public class QueryPlanQueryAction extends QueryAction { + private final QueryPlanRequestBuilder requestBuilder; + + public QueryPlanQueryAction(QueryPlanRequestBuilder requestBuilder) { + super(null, null); + this.requestBuilder = requestBuilder; + } + + @Override + public SqlElasticRequestBuilder explain() { + return requestBuilder; + } + + @Override + public Optional> getFieldNames() { + List fieldNames = ((QueryPlanRequestBuilder) requestBuilder).outputColumns() + .stream() + .map(node -> Strings.isNullOrEmpty(node.getAlias()) ? node.getName() : node.getAlias()) + .collect(Collectors.toList()); + return Optional.of(fieldNames); + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/adapter/QueryPlanRequestBuilder.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/adapter/QueryPlanRequestBuilder.java new file mode 100644 index 0000000000..105c8d3145 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/adapter/QueryPlanRequestBuilder.java @@ -0,0 +1,63 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.executor.adapter; + +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColNode; +import com.amazon.opendistroforelasticsearch.sql.query.planner.core.BindingTupleQueryPlanner; +import com.amazon.opendistroforelasticsearch.sql.query.SqlElasticRequestBuilder; +import lombok.RequiredArgsConstructor; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.action.ActionResponse; + +import java.util.List; + +/** + * The definition of QueryPlan SqlElasticRequestBuilder. + */ +@RequiredArgsConstructor +public class QueryPlanRequestBuilder implements SqlElasticRequestBuilder { + private final BindingTupleQueryPlanner queryPlanner; + + public List execute() { + return queryPlanner.execute(); + } + + public List outputColumns() { + return queryPlanner.getColNodes(); + } + + @Override + public String explain() { + return queryPlanner.explain(); + } + + @Override + public ActionRequest request() { + throw new RuntimeException("unsupported operation"); + } + + @Override + public ActionResponse get() { + throw new RuntimeException("unsupported operation"); + } + + @Override + public ActionRequestBuilder getBuilder() { + throw new RuntimeException("unsupported operation"); + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/RestSqlAction.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/RestSqlAction.java index f5e823944f..8c175c943d 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/RestSqlAction.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/RestSqlAction.java @@ -19,10 +19,14 @@ import com.amazon.opendistroforelasticsearch.sql.antlr.OpenDistroSqlAnalyzer; import com.amazon.opendistroforelasticsearch.sql.antlr.SqlAnalysisConfig; import com.amazon.opendistroforelasticsearch.sql.antlr.SqlAnalysisException; +import com.amazon.opendistroforelasticsearch.sql.antlr.semantic.types.Type; +import com.amazon.opendistroforelasticsearch.sql.domain.ColumnTypeProvider; +import com.amazon.opendistroforelasticsearch.sql.domain.QueryActionRequest; import com.amazon.opendistroforelasticsearch.sql.esdomain.LocalClusterState; import com.amazon.opendistroforelasticsearch.sql.exception.SQLFeatureDisabledException; import com.amazon.opendistroforelasticsearch.sql.exception.SqlParseException; import com.amazon.opendistroforelasticsearch.sql.executor.ActionRequestRestExecutorFactory; +import com.amazon.opendistroforelasticsearch.sql.executor.Format; import com.amazon.opendistroforelasticsearch.sql.executor.RestExecutor; import com.amazon.opendistroforelasticsearch.sql.executor.format.ErrorMessage; import com.amazon.opendistroforelasticsearch.sql.metrics.MetricName; @@ -52,6 +56,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.function.Predicate; import java.util.regex.Pattern; @@ -111,7 +116,8 @@ protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient cli final SqlRequest sqlRequest = SqlRequestFactory.getSqlRequest(request); LOG.info("[{}] Incoming request {}: {}", LogUtils.getRequestId(), request.uri(), sqlRequest.getSql()); - final QueryAction queryAction = explainRequest(client, sqlRequest); + final QueryAction queryAction = + explainRequest(client, sqlRequest, SqlRequestParam.getFormat(request.params())); return channel -> executeSqlRequest(request, queryAction, client, channel); } catch (Exception e) { logAndPublishMetrics(e); @@ -136,18 +142,19 @@ private static void logAndPublishMetrics(final Exception e) { } } - private static QueryAction explainRequest(final NodeClient client, final SqlRequest sqlRequest) + private static QueryAction explainRequest(final NodeClient client, final SqlRequest sqlRequest, Format format) throws SQLFeatureNotSupportedException, SqlParseException { - performAnalysis(sqlRequest.getSql()); + ColumnTypeProvider typeProvider = performAnalysis(sqlRequest.getSql()); - final QueryAction queryAction = new SearchDao(client).explain(sqlRequest.getSql()); + final QueryAction queryAction = new SearchDao(client) + .explain(new QueryActionRequest(sqlRequest.getSql(), typeProvider, format)); queryAction.setSqlRequest(sqlRequest); return queryAction; } - private void executeSqlRequest(final RestRequest request, final QueryAction queryAction, - final Client client, final RestChannel channel) throws Exception { + private void executeSqlRequest(final RestRequest request, final QueryAction queryAction, final Client client, + final RestChannel channel) throws Exception { Map params = request.params(); if (isExplainRequest(request)) { final String jsonExplanation = queryAction.explain().explain(); @@ -202,7 +209,7 @@ private boolean isSQLFeatureEnabled() { return allowExplicitIndex && isSqlEnabled; } - private static void performAnalysis(String sql) { + private static ColumnTypeProvider performAnalysis(String sql) { LocalClusterState clusterState = LocalClusterState.state(); SqlAnalysisConfig config = new SqlAnalysisConfig( clusterState.getSettingValue(QUERY_ANALYSIS_ENABLED), @@ -211,6 +218,11 @@ private static void performAnalysis(String sql) { ); OpenDistroSqlAnalyzer analyzer = new OpenDistroSqlAnalyzer(config); - analyzer.analyze(sql, clusterState); + Optional outputColumnType = analyzer.analyze(sql, clusterState); + if (outputColumnType.isPresent()) { + return new ColumnTypeProvider(outputColumnType.get()); + } else { + return new ColumnTypeProvider(); + } } } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/SearchDao.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/SearchDao.java index 3ead2ec359..56f56903bc 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/SearchDao.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/plugin/SearchDao.java @@ -15,6 +15,7 @@ package com.amazon.opendistroforelasticsearch.sql.plugin; +import com.amazon.opendistroforelasticsearch.sql.domain.QueryActionRequest; import com.amazon.opendistroforelasticsearch.sql.exception.SqlParseException; import com.amazon.opendistroforelasticsearch.sql.query.ESActionFactory; import com.amazon.opendistroforelasticsearch.sql.query.QueryAction; @@ -51,11 +52,12 @@ public Client getClient() { * Prepare action And transform sql * into ES ActionRequest * - * @param sql SQL query to execute. + * @param queryActionRequest SQL query action request to execute. * @return ES request * @throws SqlParseException */ - public QueryAction explain(String sql) throws SqlParseException, SQLFeatureNotSupportedException { - return ESActionFactory.create(client, sql); + public QueryAction explain(QueryActionRequest queryActionRequest) + throws SqlParseException, SQLFeatureNotSupportedException { + return ESActionFactory.create(client, queryActionRequest); } } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/ESActionFactory.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/ESActionFactory.java index 03e6e4c401..0bff89a659 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/ESActionFactory.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/ESActionFactory.java @@ -16,24 +16,35 @@ package com.amazon.opendistroforelasticsearch.sql.query; import com.alibaba.druid.sql.ast.SQLExpr; +import com.alibaba.druid.sql.ast.expr.SQLAggregateExpr; +import com.alibaba.druid.sql.ast.expr.SQLAllColumnExpr; +import com.alibaba.druid.sql.ast.expr.SQLMethodInvokeExpr; import com.alibaba.druid.sql.ast.expr.SQLQueryExpr; import com.alibaba.druid.sql.ast.statement.SQLDeleteStatement; import com.alibaba.druid.sql.ast.statement.SQLJoinTableSource; +import com.alibaba.druid.sql.ast.statement.SQLSelectGroupByClause; +import com.alibaba.druid.sql.ast.statement.SQLSelectItem; import com.alibaba.druid.sql.ast.statement.SQLUnionQuery; import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlSelectQueryBlock; import com.alibaba.druid.sql.dialect.mysql.parser.MySqlStatementParser; +import com.alibaba.druid.sql.dialect.mysql.visitor.MySqlASTVisitorAdapter; import com.alibaba.druid.sql.parser.ParserException; import com.alibaba.druid.sql.parser.SQLExprParser; import com.alibaba.druid.sql.parser.SQLStatementParser; import com.alibaba.druid.sql.parser.Token; +import com.amazon.opendistroforelasticsearch.sql.domain.ColumnTypeProvider; import com.amazon.opendistroforelasticsearch.sql.domain.Delete; import com.amazon.opendistroforelasticsearch.sql.domain.IndexStatement; import com.amazon.opendistroforelasticsearch.sql.domain.JoinSelect; +import com.amazon.opendistroforelasticsearch.sql.domain.QueryActionRequest; import com.amazon.opendistroforelasticsearch.sql.domain.Select; import com.amazon.opendistroforelasticsearch.sql.esdomain.LocalClusterState; import com.amazon.opendistroforelasticsearch.sql.exception.SqlParseException; import com.amazon.opendistroforelasticsearch.sql.executor.ElasticResultHandler; +import com.amazon.opendistroforelasticsearch.sql.executor.Format; import com.amazon.opendistroforelasticsearch.sql.executor.QueryActionElasticExecutor; +import com.amazon.opendistroforelasticsearch.sql.executor.adapter.QueryPlanQueryAction; +import com.amazon.opendistroforelasticsearch.sql.executor.adapter.QueryPlanRequestBuilder; import com.amazon.opendistroforelasticsearch.sql.parser.ElasticLexer; import com.amazon.opendistroforelasticsearch.sql.parser.ElasticSqlExprParser; import com.amazon.opendistroforelasticsearch.sql.parser.SqlParser; @@ -41,9 +52,10 @@ import com.amazon.opendistroforelasticsearch.sql.query.join.ESJoinQueryActionFactory; import com.amazon.opendistroforelasticsearch.sql.query.multi.MultiQueryAction; import com.amazon.opendistroforelasticsearch.sql.query.multi.MultiQuerySelect; +import com.amazon.opendistroforelasticsearch.sql.query.planner.core.BindingTupleQueryPlanner; import com.amazon.opendistroforelasticsearch.sql.rewriter.RewriteRuleExecutor; -import com.amazon.opendistroforelasticsearch.sql.rewriter.identifier.UnquoteIdentifierRule; import com.amazon.opendistroforelasticsearch.sql.rewriter.alias.TableAliasPrefixRemoveRule; +import com.amazon.opendistroforelasticsearch.sql.rewriter.identifier.UnquoteIdentifierRule; import com.amazon.opendistroforelasticsearch.sql.rewriter.join.JoinRewriteRule; import com.amazon.opendistroforelasticsearch.sql.rewriter.matchtoterm.TermFieldRewriter; import com.amazon.opendistroforelasticsearch.sql.rewriter.matchtoterm.TermFieldRewriter.TermRewriterFilter; @@ -51,6 +63,7 @@ import com.amazon.opendistroforelasticsearch.sql.rewriter.ordinal.OrdinalRewriterRule; import com.amazon.opendistroforelasticsearch.sql.rewriter.parent.SQLExprParentSetterRule; import com.amazon.opendistroforelasticsearch.sql.rewriter.subquery.SubQueryRewriteRule; +import com.google.common.annotations.VisibleForTesting; import org.elasticsearch.client.Client; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; @@ -63,6 +76,11 @@ public class ESActionFactory { + public static QueryAction create(Client client, String sql) + throws SqlParseException, SQLFeatureNotSupportedException { + return create(client, new QueryActionRequest(sql, new ColumnTypeProvider(), Format.JSON)); + } + /** * Create the compatible Query object * based on the SQL query. @@ -70,9 +88,9 @@ public class ESActionFactory { * @param sql The SQL query. * @return Query object. */ - public static QueryAction create(Client client, String sql) throws SqlParseException, - SQLFeatureNotSupportedException { - + public static QueryAction create(Client client, QueryActionRequest request) + throws SqlParseException, SQLFeatureNotSupportedException { + String sql = request.getSql(); // Linebreak matcher sql = sql.replaceAll("\\R", " ").trim(); @@ -92,8 +110,8 @@ public static QueryAction create(Client client, String sql) throws SqlParseExcep if (isMulti(sqlExpr)) { sqlExpr.accept(new TermFieldRewriter(client, TermRewriterFilter.MULTI_QUERY)); - MultiQuerySelect multiSelect = new SqlParser() - .parseMultiSelect((SQLUnionQuery) sqlExpr.getSubQuery().getQuery()); + MultiQuerySelect multiSelect = + new SqlParser().parseMultiSelect((SQLUnionQuery) sqlExpr.getSubQuery().getQuery()); return new MultiQueryAction(client, multiSelect); } else if (isJoin(sqlExpr, sql)) { new JoinRewriteRule(LocalClusterState.state()).rewrite(sqlExpr); @@ -102,6 +120,11 @@ public static QueryAction create(Client client, String sql) throws SqlParseExcep return ESJoinQueryActionFactory.createJoinAction(client, joinSelect); } else { sqlExpr.accept(new TermFieldRewriter(client)); + // migrate aggregation to query planner framework. + if (shouldMigrateToQueryPlan(sqlExpr, request.getFormat())) { + return new QueryPlanQueryAction(new QueryPlanRequestBuilder( + new BindingTupleQueryPlanner(client, sqlExpr, request.getTypeProvider()))); + } Select select = new SqlParser().parseSelect(sqlExpr); return handleSelect(client, select); } @@ -124,16 +147,15 @@ public static QueryAction create(Client client, String sql) throws SqlParseExcep private static String getFirstWord(String sql) { int endOfFirstWord = sql.indexOf(' '); - return sql - .substring(0, endOfFirstWord > 0 ? endOfFirstWord : sql.length()) - .toUpperCase(); + return sql.substring(0, endOfFirstWord > 0 ? endOfFirstWord : sql.length()).toUpperCase(); } private static boolean isMulti(SQLQueryExpr sqlExpr) { return sqlExpr.getSubQuery().getQuery() instanceof SQLUnionQuery; } - private static void executeAndFillSubQuery(Client client, SubQueryExpression subQueryExpression, + private static void executeAndFillSubQuery(Client client, + SubQueryExpression subQueryExpression, QueryAction queryAction) throws SqlParseException { List values = new ArrayList<>(); Object queryResult; @@ -172,7 +194,7 @@ private static SQLStatementParser createSqlStatementParser(String sql) { private static boolean isJoin(SQLQueryExpr sqlExpr, String sql) { MySqlSelectQueryBlock query = (MySqlSelectQueryBlock) sqlExpr.getSubQuery().getQuery(); return query.getFrom() instanceof SQLJoinTableSource - && ((SQLJoinTableSource) query.getFrom()).getJoinType() != SQLJoinTableSource.JoinType.COMMA; + && ((SQLJoinTableSource) query.getFrom()).getJoinType() != SQLJoinTableSource.JoinType.COMMA; } private static SQLExpr toSqlExpr(String sql) { @@ -184,4 +206,54 @@ private static SQLExpr toSqlExpr(String sql) { } return expr; } + + @VisibleForTesting + public static boolean shouldMigrateToQueryPlan(SQLQueryExpr expr, Format format) { + // The JSON format will return the Elasticsearch aggregation result, which is not supported by the QueryPlanner. + if (format == Format.JSON) { + return false; + } + QueryPlannerScopeDecider decider = new QueryPlannerScopeDecider(); + return decider.isInScope(expr); + } + + private static class QueryPlannerScopeDecider extends MySqlASTVisitorAdapter { + private boolean hasAggregationFunc = false; + private boolean hasNestedFunction = false; + private boolean hasGroupBy = false; + private boolean hasAllColumnExpr = false; + + public boolean isInScope(SQLQueryExpr expr) { + expr.accept(this); + return !hasAllColumnExpr && !hasNestedFunction && (hasGroupBy || hasAggregationFunc); + } + + @Override + public boolean visit(SQLSelectItem expr) { + if (expr.getExpr() instanceof SQLAllColumnExpr) { + hasAllColumnExpr = true; + } + return super.visit(expr); + } + + @Override + public boolean visit(SQLSelectGroupByClause expr) { + hasGroupBy = true; + return super.visit(expr); + } + + @Override + public boolean visit(SQLAggregateExpr expr) { + hasAggregationFunc = true; + return super.visit(expr); + } + + @Override + public boolean visit(SQLMethodInvokeExpr expr) { + if (expr.getMethodName().equalsIgnoreCase("nested")) { + hasNestedFunction = true; + } + return super.visit(expr); + } + } } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLAggregationParser.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLAggregationParser.java new file mode 100644 index 0000000000..24fc45ff00 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLAggregationParser.java @@ -0,0 +1,262 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.query.planner.converter; + +import com.alibaba.druid.sql.ast.SQLExpr; +import com.alibaba.druid.sql.ast.expr.SQLAggregateExpr; +import com.alibaba.druid.sql.ast.expr.SQLCastExpr; +import com.alibaba.druid.sql.ast.expr.SQLIdentifierExpr; +import com.alibaba.druid.sql.ast.expr.SQLMethodInvokeExpr; +import com.alibaba.druid.sql.ast.statement.SQLSelectItem; +import com.alibaba.druid.sql.dialect.mysql.ast.expr.MySqlSelectGroupByExpr; +import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.alibaba.druid.sql.dialect.mysql.visitor.MySqlASTVisitorAdapter; +import com.amazon.opendistroforelasticsearch.sql.domain.ColumnTypeProvider; +import com.amazon.opendistroforelasticsearch.sql.expression.core.Expression; +import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColumnNode; +import com.google.common.base.Strings; +import lombok.Getter; +import lombok.RequiredArgsConstructor; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.var; + +/** + * The definition of SQL Aggregation Converter which will parse the query to project column node list and + * aggregation list + * e.g. parse the query: SELECT age, MAX(balance) - MIN(balance) FROM T GROUP BY age. + * will generate the + * node list: age, max_0 - min_0 + * aggregation list: age, max(balance) as max_0, min(balance) as min_0 + * + */ +@RequiredArgsConstructor +public class SQLAggregationParser { + private final ColumnTypeProvider columnTypeProvider; + private Context context; + @Getter + private List columnNodes = new ArrayList<>(); + + public void parse(MySqlSelectQueryBlock queryBlock) { + context = new Context(constructSQLExprAliasMapFromSelect(queryBlock)); + + //2. find all GroupKeyExpr from GroupBy expression. + findAllGroupKeyExprFromGroupByAndSelect(queryBlock); + findAllAggregationExprFromSelect(queryBlock); + + //3. parse the select list to expression + parseExprInSelectList(queryBlock, new SQLExprToExpressionConverter(context)); + } + + public List selectItemList() { + List sqlSelectItems = new ArrayList<>(); + context.getGroupKeyExprMap().entrySet().forEach(entry -> sqlSelectItems + .add(new SQLSelectItem(entry.getKey(), entry.getValue().getExpression().toString()))); + context.getAggregationExprMap().entrySet().forEach(entry -> sqlSelectItems + .add(new SQLSelectItem(entry.getKey(), entry.getValue().getExpression().toString()))); + return sqlSelectItems; + } + + private Map constructSQLExprAliasMapFromSelect(MySqlSelectQueryBlock queryBlock) { + return queryBlock.getSelectList().stream().filter(item -> !Strings.isNullOrEmpty(item.getAlias())) + .collect(Collectors.toMap(SQLSelectItem::getExpr, SQLSelectItem::getAlias)); + } + + /** + * The SQL-92 require nonaggregated name column in the select list must appear in the GROUP BY, But the + * existing uses cases violate this require. e.g. AggregationIT. countGroupByDateTest + * Ref the https://dev.mysql.com/doc/refman/8.0/en/group-by-handling.html for detail information + */ + private void findAllGroupKeyExprFromGroupByAndSelect(MySqlSelectQueryBlock queryBlock) { + if (queryBlock.getGroupBy() == null) { + return; + } + // 1. fetch the expr from groupby clause. + List groupByKeyExprList = + queryBlock.getGroupBy().getItems().stream().map(item -> ((MySqlSelectGroupByExpr) item).getExpr()) + .collect(Collectors.toList()); + + // 2. find the group expr from select. + for (SQLSelectItem selectItem : queryBlock.getSelectList()) { + SQLExpr selectItemExpr = selectItem.getExpr(); + // extension, group key in select could not in group by. + if (selectItemExpr instanceof SQLIdentifierExpr) { + context.addGroupKeyExpr(selectItemExpr); + } else { + for (SQLExpr groupByExpr : groupByKeyExprList) { + // SQL-92,nonaggregated name column in the select list must appear in the GROUP BY + if (compareSelectExprAndGroupByExpr(selectItemExpr, selectItem.getAlias(), groupByExpr)) { + context.addGroupKeyExpr(selectItemExpr); + } else if (groupByExpr instanceof SQLIdentifierExpr) { + // support expression over group key, e.g. SELECT log(G), max(A) FROM T GROUP BY G. + String groupByName = ((SQLIdentifierExpr) groupByExpr).getName(); + selectItemExpr.accept(new MySqlASTVisitorAdapter() { + @Override + public boolean visit(SQLAggregateExpr x) { + return false; + } + + @Override + public boolean visit(SQLIdentifierExpr expr) { + if (groupByName.equalsIgnoreCase(expr.getName())) { + expr.setParent(selectItem.getParent()); + context.addGroupKeyExpr(expr); + } + return false; + } + }); + } + } + } + } + } + + private boolean compareSelectExprAndGroupByExpr(SQLExpr selectItemExpr, String alias, SQLExpr groupByExpr) { + if (groupByExpr.equals(selectItemExpr)) { + return true; + } else if (groupByExpr instanceof SQLIdentifierExpr + && ((SQLIdentifierExpr) groupByExpr).getName().equalsIgnoreCase(alias)) { + return true; + } + return false; + } + + private void findAllAggregationExprFromSelect(MySqlSelectQueryBlock queryBlock) { + queryBlock.getSelectList().forEach(selectItem -> selectItem.accept(new MySqlASTVisitorAdapter() { + @Override + public boolean visit(SQLAggregateExpr expr) { + context.addAggregationExpr(expr); + return true; + } + })); + } + + private void parseExprInSelectList(MySqlSelectQueryBlock queryBlock, SQLExprToExpressionConverter exprConverter) { + List selectItems = queryBlock.getSelectList(); + for (int i = 0; i < selectItems.size(); i++) { + Expression expression = exprConverter.convert(selectItems.get(i).getExpr()); + ColumnNode columnNode = ColumnNode.builder() + .name(nameOfSelectItem(selectItems.get(i))) + .alias(selectItems.get(i).getAlias()) + .type(columnTypeProvider.get(i)) + .expr(expression) + .build(); + columnNodes.add(columnNode); + } + } + + private String nameOfSelectItem(SQLSelectItem selectItem) { + return Strings.isNullOrEmpty(selectItem.getAlias()) ? Context + .nameOfExpr(selectItem.getExpr()) : selectItem.getAlias(); + } + + @RequiredArgsConstructor + public static class Context { + private final AliasGenerator aliasGenerator = new AliasGenerator(); + + private final Map selectSQLExprAliasMap; + + @Getter + private final Map groupKeyExprMap = new LinkedHashMap<>(); + @Getter + private final Map aggregationExprMap = new LinkedHashMap<>(); + + Optional resolve(SQLExpr expr) { + if (groupKeyExprMap.containsKey(expr)) { + return Optional.of(groupKeyExprMap.get(expr).getExpression()); + } else if (aggregationExprMap.containsKey(expr)) { + return Optional.of(aggregationExprMap.get(expr).getExpression()); + } else { + return Optional.empty(); + } + } + + public void addGroupKeyExpr(SQLExpr groupKeyExpr) { + if (!groupKeyExprMap.containsKey(groupKeyExpr)) { + groupKeyExprMap.put(groupKeyExpr, new GroupKeyExpr(groupKeyExpr)); + } + } + + public void addAggregationExpr(SQLAggregateExpr aggregationExpr) { + if (!aggregationExprMap.containsKey(aggregationExpr)) { + aggregationExprMap.put(aggregationExpr, new AggregationExpr(aggregationExpr)); + } + } + + @Getter + public class GroupKeyExpr { + private final SQLExpr expr; + private final Expression expression; + + public GroupKeyExpr(SQLExpr expr) { + this.expr = expr; + String exprName = nameOfExpr(expr).replace(".", "#"); + if (expr instanceof SQLIdentifierExpr + && selectSQLExprAliasMap.values().contains(((SQLIdentifierExpr) expr).getName())) { + exprName = ((SQLIdentifierExpr) expr).getName(); + } + this.expression = var(selectSQLExprAliasMap.getOrDefault(expr, exprName)); + } + } + + @Getter + public class AggregationExpr { + private final SQLAggregateExpr expr; + private final Expression expression; + + public AggregationExpr(SQLAggregateExpr expr) { + this.expr = expr; + this.expression = + var(selectSQLExprAliasMap.getOrDefault(expr, aliasGenerator.nextAlias(expr.getMethodName()))); + } + } + + public static String nameOfExpr(SQLExpr expr) { + String exprName = expr.toString().toLowerCase(); + if (expr instanceof SQLAggregateExpr) { + exprName = String.format("%s(%s)", ((SQLAggregateExpr) expr).getMethodName(), + ((SQLAggregateExpr) expr).getArguments().get(0)); + } else if (expr instanceof SQLMethodInvokeExpr) { + exprName = String.format("%s(%s)", ((SQLMethodInvokeExpr) expr).getMethodName(), + ((SQLMethodInvokeExpr) expr).getParameters().get(0)); + } else if (expr instanceof SQLIdentifierExpr) { + exprName = ((SQLIdentifierExpr) expr).getName(); + } else if (expr instanceof SQLCastExpr) { + exprName = String.format("CAST(%s AS %s)", ((SQLCastExpr) expr).getExpr(), + ((SQLCastExpr) expr).getDataType().getName()); + } + return exprName; + } + + static class AliasGenerator { + private int aliasSuffix = 0; + + private String nextAlias(String name) { + return String.format("%s_%d", name, next()); + } + + private Integer next() { + return aliasSuffix++; + } + } + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLExprToExpressionConverter.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLExprToExpressionConverter.java new file mode 100644 index 0000000000..5dbd47c10f --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLExprToExpressionConverter.java @@ -0,0 +1,107 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.query.planner.converter; + +import com.alibaba.druid.sql.ast.SQLExpr; +import com.alibaba.druid.sql.ast.expr.SQLBinaryOpExpr; +import com.alibaba.druid.sql.ast.expr.SQLBinaryOperator; +import com.alibaba.druid.sql.ast.expr.SQLCastExpr; +import com.alibaba.druid.sql.ast.expr.SQLMethodInvokeExpr; +import com.amazon.opendistroforelasticsearch.sql.expression.core.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory; +import com.google.common.collect.ImmutableMap; +import lombok.Getter; +import lombok.RequiredArgsConstructor; + +import java.util.Map; +import java.util.Optional; +import java.util.function.BiFunction; +import java.util.function.Function; + +import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.cast; +import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.log; + +/** + * The definition of {@link SQLExpr} to {@link Expression} converter. + */ +@RequiredArgsConstructor +public class SQLExprToExpressionConverter { + private final SQLAggregationParser.Context context; + + /** + * Convert the {@link SQLExpr} to {@link Expression} + * @param expr {@link SQLExpr} + * @return expression {@link Expression} + */ + public Expression convert(SQLExpr expr) { + Optional resolvedExpression = context.resolve(expr); + if (resolvedExpression.isPresent()) { + return resolvedExpression.get(); + } else { + if (expr instanceof SQLBinaryOpExpr) { + return binaryOperatorToExpression((SQLBinaryOpExpr) expr, this::convert); + } else if (expr instanceof SQLMethodInvokeExpr) { + return methodToExpression((SQLMethodInvokeExpr) expr, this::convert); + } else if (expr instanceof SQLCastExpr) { + return cast(convert(((SQLCastExpr) expr).getExpr())); + } else { + throw new RuntimeException("unsupported expr: " + expr); + } + } + } + + private Expression binaryOperatorToExpression(SQLBinaryOpExpr expr, + Function converter) { + return BinaryExpression.of(expr.getOperator()) + .map(binaryExpression -> binaryExpression.getExpressionBuilder() + .apply(converter.apply((expr.getLeft())), + converter.apply(expr.getRight()))) + .orElseThrow(() -> new RuntimeException( + "unsupported operator in select: " + (expr.getOperator()))); + } + + private Expression methodToExpression(SQLMethodInvokeExpr expr, Function converter) { + if (("log").equalsIgnoreCase(expr.getMethodName())) { + return log(converter.apply(expr.getParameters().get(0))); + } else { + throw new RuntimeException("unsupported operator in select: " + expr.getMethodName()); + } + } + + @Getter + @RequiredArgsConstructor + private enum BinaryExpression { + ADD(SQLBinaryOperator.Add, ExpressionFactory::add), + SUB(SQLBinaryOperator.Subtract, ExpressionFactory::sub); + + private final SQLBinaryOperator sqlBinaryOperator; + private final BiFunction expressionBuilder; + + private static final Map BINARY_OPERATOR_MAP; + + static { + ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); + for (BinaryExpression operator : BinaryExpression.values()) { + builder.put(operator.getSqlBinaryOperator(), operator); + } + BINARY_OPERATOR_MAP = builder.build(); + } + + public static Optional of(SQLBinaryOperator operator) { + return Optional.ofNullable(BINARY_OPERATOR_MAP.get(operator)); + } + } +} \ No newline at end of file diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLToOperatorConverter.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLToOperatorConverter.java new file mode 100644 index 0000000000..e492d76171 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLToOperatorConverter.java @@ -0,0 +1,100 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.query.planner.converter; + +import com.alibaba.druid.sql.ast.expr.SQLMethodInvokeExpr; +import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.alibaba.druid.sql.dialect.mysql.visitor.MySqlASTVisitorAdapter; +import com.amazon.opendistroforelasticsearch.sql.domain.ColumnTypeProvider; +import com.amazon.opendistroforelasticsearch.sql.domain.Select; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.parser.SqlParser; +import com.amazon.opendistroforelasticsearch.sql.query.AggregationQueryAction; +import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColumnNode; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.PhysicalOperator; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.project.PhysicalProject; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll.PhysicalScroll; +import lombok.Getter; +import lombok.SneakyThrows; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.client.Client; + +import java.util.List; + +/** + * Definition of SQL to PhysicalOperator converter. + */ +public class SQLToOperatorConverter extends MySqlASTVisitorAdapter { + private static final Logger LOG = LogManager.getLogger(SQLToOperatorConverter.class); + + private final Client client; + private final SQLAggregationParser aggregationParser; + + @Getter + private PhysicalOperator physicalOperator; + + public SQLToOperatorConverter(Client client, ColumnTypeProvider columnTypeProvider) { + this.client = client; + this.aggregationParser = new SQLAggregationParser(columnTypeProvider); + } + + @Override + public boolean visit(MySqlSelectQueryBlock query) { + //1. rewrite all the function name to lower case. + rewriteFunctionNameToLowerCase(query); + + //2. parse the aggregation + aggregationParser.parse(query); + + + //3. construct the PhysicalOperator + physicalOperator = project( + scroll(query)); + return false; + } + + /** + * Get list of {@link ColumnNode}. + * + * @return list of {@link ColumnNode}. + */ + public List getColumnNodes() { + return aggregationParser.getColumnNodes(); + } + + private void rewriteFunctionNameToLowerCase(MySqlSelectQueryBlock query) { + query.accept(new MySqlASTVisitorAdapter() { + @Override + public boolean visit(SQLMethodInvokeExpr x) { + x.setMethodName(x.getMethodName().toLowerCase()); + return true; + } + }); + } + + private PhysicalOperator project(PhysicalOperator input) { + return new PhysicalProject(input, aggregationParser.getColumnNodes()); + } + + @SneakyThrows + private PhysicalOperator scroll(MySqlSelectQueryBlock query) { + query.getSelectList().clear(); + query.getSelectList().addAll(aggregationParser.selectItemList()); + Select select = new SqlParser().parseSelect(query); + return new PhysicalScroll(new AggregationQueryAction(client, select)); + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/core/BindingTupleQueryPlanner.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/core/BindingTupleQueryPlanner.java new file mode 100644 index 0000000000..c922feb747 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/core/BindingTupleQueryPlanner.java @@ -0,0 +1,89 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.query.planner.core; + +import com.alibaba.druid.sql.ast.expr.SQLQueryExpr; +import com.amazon.opendistroforelasticsearch.sql.domain.ColumnTypeProvider; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.query.planner.converter.SQLToOperatorConverter; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.PhysicalOperator; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll.PhysicalScroll; +import lombok.Getter; +import org.elasticsearch.client.Client; + +import java.util.ArrayList; +import java.util.List; + +/** + * The definition of QueryPlanner which return the {@link BindingTuple} as result. + */ +public class BindingTupleQueryPlanner { + private PhysicalOperator physicalOperator; + @Getter + private List columnNodes; + + public BindingTupleQueryPlanner(Client client, SQLQueryExpr sqlExpr, ColumnTypeProvider columnTypeProvider) { + SQLToOperatorConverter converter = new SQLToOperatorConverter(client, columnTypeProvider); + sqlExpr.accept(converter); + this.physicalOperator = converter.getPhysicalOperator(); + this.columnNodes = converter.getColumnNodes(); + } + + /** + * Execute the QueryPlanner. + * @return list of {@link BindingTuple}. + */ + public List execute() { + PhysicalOperator op = physicalOperator; + List tuples = new ArrayList<>(); + try { + op.open(null); + } catch (Exception e) { + throw new RuntimeException(e); + } + + while (op.hasNext()) { + tuples.add(op.next().data()); + } + return tuples; + } + + /** + * Explain the physical execution plan. + * @return execution plan. + */ + public String explain() { + Explanation explanation = new Explanation(); + physicalOperator.accept(explanation); + return explanation.explain(); + } + + private static class Explanation implements PlanNode.Visitor { + private String explain; + + public String explain() { + return explain; + } + + @Override + public boolean visit(PlanNode planNode) { + if (planNode instanceof PhysicalScroll) { + explain = planNode.toString(); + } + return true; + } + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/core/ColumnNode.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/core/ColumnNode.java new file mode 100644 index 0000000000..4e5755aab1 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/core/ColumnNode.java @@ -0,0 +1,37 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.query.planner.core; + +import com.amazon.opendistroforelasticsearch.sql.executor.format.Schema; +import com.amazon.opendistroforelasticsearch.sql.expression.core.Expression; +import lombok.Builder; +import lombok.Getter; +import lombok.Setter; +import lombok.ToString; + +/** + * The definition of column node. + */ +@Builder +@Setter +@Getter +@ToString +public class ColumnNode { + private String name; + private String alias; + private Schema.Type type; + private Expression expr; +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/BindingTupleRow.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/BindingTupleRow.java new file mode 100644 index 0000000000..8593e3e641 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/BindingTupleRow.java @@ -0,0 +1,47 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll; + +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.Row; +import lombok.RequiredArgsConstructor; + +import java.util.Map; + +@RequiredArgsConstructor +public class BindingTupleRow implements Row { + private final BindingTuple bindingTuple; + + @Override + public RowKey key(String[] colNames) { + return null; + } + + @Override + public Row combine(Row otherRow) { + throw new RuntimeException("unsupported operation"); + } + + @Override + public void retain(Map colNameAlias) { + // do nothing + } + + @Override + public BindingTuple data() { + return bindingTuple; + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/esintgtest/CsvFormatResponseIT.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/esintgtest/CsvFormatResponseIT.java index 5fc7f427ce..4a4662c1fa 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/sql/esintgtest/CsvFormatResponseIT.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/esintgtest/CsvFormatResponseIT.java @@ -44,10 +44,10 @@ import static com.amazon.opendistroforelasticsearch.sql.esintgtest.TestsConstants.TEST_INDEX_ONLINE; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; -import static org.hamcrest.Matchers.hasItems; import static org.hamcrest.Matchers.not; import static org.hamcrest.core.StringStartsWith.startsWith; @@ -371,15 +371,12 @@ public void aggAfterTermsGroupBy() throws Exception { TEST_INDEX_ACCOUNT); CSVResult csvResult = executeCsvRequest(query, false); List headers = csvResult.getHeaders(); - Assert.assertEquals(2, headers.size()); - Assert.assertEquals("gender", headers.get(0)); - Assert.assertEquals("COUNT(*)", headers.get(1)); + Assert.assertEquals(1, headers.size()); + assertThat(headers, contains(equalTo("COUNT(*)"))); List lines = csvResult.getLines(); Assert.assertEquals(2, lines.size()); - Assert.assertTrue("m,507.0", lines.contains("m,507.0")); - Assert.assertTrue("f,493.0", lines.contains("f,493.0")); - + assertThat(lines, containsInAnyOrder(equalTo("507.0"), equalTo("493.0"))); } @Test @@ -389,18 +386,16 @@ public void aggAfterTwoTermsGroupBy() throws Exception { TEST_INDEX_ACCOUNT); CSVResult csvResult = executeCsvRequest(query, false); List headers = csvResult.getHeaders(); - Assert.assertEquals(3, headers.size()); - Assert.assertEquals("gender", headers.get(0)); - Assert.assertEquals("age", headers.get(1)); - Assert.assertEquals("COUNT(*)", headers.get(2)); + Assert.assertEquals(1, headers.size()); + assertThat(headers, contains(equalTo("COUNT(*)"))); List lines = csvResult.getLines(); Assert.assertEquals(4, lines.size()); - Assert.assertTrue("m,36,31.0", lines.contains("m,36,31.0")); - Assert.assertTrue("m,35,28.0", lines.contains("m,36,31.0")); - Assert.assertTrue("f,36,21.0", lines.contains("f,36,21.0")); - Assert.assertTrue("f,35,24.0", lines.contains("f,35,24.0")); - + assertThat(lines, containsInAnyOrder( + equalTo("31.0"), + equalTo("28.0"), + equalTo("21.0"), + equalTo("24.0"))); } @Test @@ -410,23 +405,16 @@ public void multipleAggAfterTwoTermsGroupBy() throws Exception { TEST_INDEX_ACCOUNT); CSVResult csvResult = executeCsvRequest(query, false); List headers = csvResult.getHeaders(); - Assert.assertEquals(4, headers.size()); - Assert.assertEquals("gender", headers.get(0)); - Assert.assertEquals("age", headers.get(1)); - Assert.assertTrue(headers.get(2).equals("COUNT(*)") || headers.get(2).equals("SUM(balance)")); - Assert.assertTrue(headers.get(3).equals("COUNT(*)") || headers.get(3).equals("SUM(balance)")); + Assert.assertEquals(2, headers.size()); + assertThat(headers, contains(equalTo("COUNT(*)"), equalTo("SUM(balance)"))); List lines = csvResult.getLines(); Assert.assertEquals(4, lines.size()); - Assert.assertTrue("m,36,31.0,647425.0", - lines.contains("m,36,31.0,647425.0") || lines.contains("m,36,647425.0,31.0")); - Assert.assertTrue("m,35,28.0,678337.0", - lines.contains("m,35,28.0,678337.0") || lines.contains("m,35,678337.0,28.0")); - Assert.assertTrue("f,36,21.0,505660.0", - lines.contains("f,36,21.0,505660.0") || lines.contains("f,36,505660.0,21.0")); - Assert.assertTrue("f,35,24.0,472771.0", - lines.contains("f,35,24.0,472771.0") || lines.contains("f,35,472771.0,24.0")); - + assertThat(lines, containsInAnyOrder( + equalTo("31.0,647425.0"), + equalTo("28.0,678337.0"), + equalTo("21.0,505660.0"), + equalTo("24.0,472771.0"))); } @Test @@ -435,14 +423,15 @@ public void dateHistogramTest() throws Exception { " group by date_histogram('field'='insert_time','interval'='4d','alias'='days')", TEST_INDEX_ONLINE); CSVResult csvResult = executeCsvRequest(query, false); List headers = csvResult.getHeaders(); - Assert.assertEquals(2, headers.size()); - Assert.assertEquals("days", headers.get(0)); - Assert.assertEquals("COUNT(*)", headers.get(1)); + Assert.assertEquals(1, headers.size()); + assertThat(headers, contains(equalTo("COUNT(*)"))); List lines = csvResult.getLines(); Assert.assertEquals(3, lines.size()); - Assert.assertThat(lines, - hasItems("2014-08-14 00:00:00,477.0", "2014-08-18 00:00:00,5664.0", "2014-08-22 00:00:00,3795.0")); + assertThat(lines, containsInAnyOrder( + equalTo("477.0"), + equalTo("5664.0"), + equalTo("3795.0"))); } @Test diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/esintgtest/PrettyFormatResponseIT.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/esintgtest/PrettyFormatResponseIT.java index eda5aecc95..bd2d152054 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/sql/esintgtest/PrettyFormatResponseIT.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/esintgtest/PrettyFormatResponseIT.java @@ -313,8 +313,7 @@ public void aggregationFunctionInSelect() throws IOException { String.format(Locale.ROOT, "SELECT COUNT(*) FROM %s GROUP BY age", TestsConstants.TEST_INDEX_ACCOUNT)); - String count = "COUNT(*)"; - List fields = Arrays.asList("age", count); + List fields = Arrays.asList("COUNT(*)"); assertContainsColumns(getSchema(response), fields); JSONArray dataRows = getDataRows(response); @@ -332,8 +331,7 @@ public void aggregationFunctionInSelectCaseCheck() throws IOException { String.format(Locale.ROOT, "SELECT count(*) FROM %s GROUP BY age", TestsConstants.TEST_INDEX_ACCOUNT)); - String count = "COUNT(*)"; - List fields = Arrays.asList("age", count); + List fields = Arrays.asList("COUNT(*)"); assertContainsColumns(getSchema(response), fields); JSONArray dataRows = getDataRows(response); @@ -351,8 +349,7 @@ public void aggregationFunctionInSelectWithAlias() throws IOException { String.format(Locale.ROOT, "SELECT COUNT(*) AS total FROM %s GROUP BY age", TestsConstants.TEST_INDEX_ACCOUNT)); - String count = "total"; - List fields = Arrays.asList("age", count); + List fields = Arrays.asList("total"); assertContainsColumns(getSchema(response), fields); JSONArray dataRows = getDataRows(response); @@ -370,7 +367,7 @@ public void aggregationFunctionInSelectGroupByMultipleFields() throws IOExceptio String.format(Locale.ROOT, "SELECT SUM(age) FROM %s GROUP BY age, state.keyword", TestsConstants.TEST_INDEX_ACCOUNT)); - List fields = Arrays.asList("age", "state.keyword", "SUM(age)"); + List fields = Arrays.asList("SUM(age)"); assertContainsColumns(getSchema(response), fields); assertContainsData(getDataRows(response), fields); } @@ -398,7 +395,7 @@ public void multipleAggregationFunctionsInSelect() throws IOException { String.format(Locale.ROOT, "SELECT COUNT(*), AVG(age) FROM %s GROUP BY age", TestsConstants.TEST_INDEX_ACCOUNT)); - List fields = Arrays.asList("age", "COUNT(*)", "AVG(age)"); + List fields = Arrays.asList("COUNT(*)", "AVG(age)"); assertContainsColumns(getSchema(response), fields); assertContainsData(getDataRows(response), fields); } diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/esintgtest/SQLFunctionsIT.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/esintgtest/SQLFunctionsIT.java index e72224fd2f..a227b49c46 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/sql/esintgtest/SQLFunctionsIT.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/esintgtest/SQLFunctionsIT.java @@ -262,10 +262,12 @@ public void castIntFieldToFloatWithoutAliasJdbcFormatGroupByTest() { "SELECT CAST(balance AS FLOAT) FROM " + TestsConstants.TEST_INDEX_ACCOUNT + " GROUP BY balance DESC LIMIT 5"); - String float_type_cast = "{\"name\":\"balance\",\"type\":\"long\"}"; - assertEquals(response.getJSONArray("schema").get(0).toString(), float_type_cast); - Float[] expectedOutput = new Float[] {22026.0F, 23285.0F, 36038.0F, 39063.0F, 45493.0F}; + assertEquals("CAST(balance AS FLOAT)", response.query("/schema/0/name")); + assertNull(response.query("/schema/0/alias")); + assertEquals("float", response.query("/schema/0/type")); + Float[] expectedOutput = new Float[] {22026.0F, 23285.0F, 36038.0F, 39063.0F, 45493.0F}; + assertEquals(5, response.getJSONArray("datarows").length()); for (int i = 0; i < response.getJSONArray("datarows").length(); ++i) { Assert.assertThat( response.getJSONArray("datarows") @@ -280,10 +282,12 @@ public void castIntFieldToFloatWithAliasJdbcFormatGroupByTest() { "SELECT CAST(balance AS FLOAT) AS jdbc_float_alias " + "FROM " + TestsConstants.TEST_INDEX_ACCOUNT + " GROUP BY jdbc_float_alias ASC LIMIT 5"); - String float_type_cast = "{\"name\":\"jdbc_float_alias\",\"type\":\"float\"}"; - assertEquals(response.getJSONArray("schema").get(0).toString(), float_type_cast); - Float[] expectedOutput = new Float[] {22026.0F, 23285.0F, 36038.0F, 39063.0F, 45493.0F}; + assertEquals("jdbc_float_alias", response.query("/schema/0/name")); + assertEquals("jdbc_float_alias", response.query("/schema/0/alias")); + assertEquals("float", response.query("/schema/0/type")); + Float[] expectedOutput = new Float[] {22026.0F, 23285.0F, 36038.0F, 39063.0F, 45493.0F}; + assertEquals(5, response.getJSONArray("datarows").length()); for (int i = 0; i < response.getJSONArray("datarows").length(); ++i) { Assert.assertThat( response.getJSONArray("datarows") @@ -298,10 +302,12 @@ public void castIntFieldToDoubleWithAliasJdbcFormatGroupByTest() { "SELECT CAST(age AS DOUBLE) AS jdbc_double_alias " + "FROM " + TestsConstants.TEST_INDEX_ACCOUNT + " GROUP BY jdbc_double_alias DESC LIMIT 5"); - String float_type_cast = "{\"name\":\"jdbc_double_alias\",\"type\":\"double\"}"; - assertEquals(response.getJSONArray("schema").get(0).toString(), float_type_cast); - Double[] expectedOutput = new Double[] {31.0, 39.0, 26.0, 32.0, 35.0}; + assertEquals("jdbc_double_alias", response.query("/schema/0/name")); + assertEquals("jdbc_double_alias", response.query("/schema/0/alias")); + assertEquals("double", response.query("/schema/0/type")); + Double[] expectedOutput = new Double[] {31.0, 39.0, 26.0, 32.0, 35.0}; + assertEquals(5, response.getJSONArray("datarows").length()); for (int i = 0; i < response.getJSONArray("datarows").length(); ++i) { Assert.assertThat( response.getJSONArray("datarows") @@ -510,11 +516,12 @@ public void right() throws IOException { @Test public void ifFuncShouldPassJDBC() { - assertThat( - executeQuery("SELECT IF(age > 30, 'True', 'False') AS Ages FROM " + TEST_INDEX_ACCOUNT - + " WHERE age IS NOT NULL GROUP BY Ages", "jdbc"), - containsString("\"type\": \"keyword\"") - ); + JSONObject response = executeJdbcRequest( + "SELECT IF(age > 30, 'True', 'False') AS Ages FROM " + TEST_INDEX_ACCOUNT + + " WHERE age IS NOT NULL GROUP BY Ages"); + assertEquals("Ages", response.query("/schema/0/name")); + assertEquals("Ages", response.query("/schema/0/alias")); + assertEquals("double", response.query("/schema/0/type")); } @Test @@ -547,12 +554,12 @@ public void ifFuncWithNullInputAsConditionTest() throws IOException { @Test public void ifnullShouldPassJDBC() throws IOException { - assertThat( - executeQuery("SELECT IFNULL(lastname, 'unknown') AS name FROM " + TEST_INDEX_ACCOUNT - + " GROUP BY name", "jdbc"), - containsString("\"type\": \"keyword\"") - ); - + JSONObject response = executeJdbcRequest( + "SELECT IFNULL(lastname, 'unknown') AS name FROM " + TEST_INDEX_ACCOUNT + + " GROUP BY name"); + assertEquals("name", response.query("/schema/0/name")); + assertEquals("name", response.query("/schema/0/alias")); + assertEquals("double", response.query("/schema/0/type")); } @Test @@ -577,10 +584,11 @@ public void ifnullWithNullInputTest() throws IOException { @Test public void isnullShouldPassJDBC() { - assertThat( - executeQuery("SELECT ISNULL(lastname) AS name FROM " + TEST_INDEX_ACCOUNT + " GROUP BY name", "jdbc"), - containsString("\"type\": \"keyword\"") - ); + JSONObject response = + executeJdbcRequest("SELECT ISNULL(lastname) AS name FROM " + TEST_INDEX_ACCOUNT + " GROUP BY name"); + assertEquals("name", response.query("/schema/0/name")); + assertEquals("name", response.query("/schema/0/alias")); + assertEquals("integer", response.query("/schema/0/type")); } @Test diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/JSONRequestTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/JSONRequestTest.java index 90e0874d3c..c801e4dd55 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/JSONRequestTest.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/JSONRequestTest.java @@ -16,8 +16,12 @@ package com.amazon.opendistroforelasticsearch.sql.unittest; import com.alibaba.druid.sql.parser.ParserException; +import com.amazon.opendistroforelasticsearch.sql.domain.ColumnTypeProvider; +import com.amazon.opendistroforelasticsearch.sql.domain.QueryActionRequest; import com.amazon.opendistroforelasticsearch.sql.esintgtest.TestsConstants; import com.amazon.opendistroforelasticsearch.sql.exception.SqlParseException; +import com.amazon.opendistroforelasticsearch.sql.executor.Format; +import com.amazon.opendistroforelasticsearch.sql.executor.format.Schema; import com.amazon.opendistroforelasticsearch.sql.query.ESActionFactory; import com.amazon.opendistroforelasticsearch.sql.query.QueryAction; import com.amazon.opendistroforelasticsearch.sql.query.SqlElasticRequestBuilder; @@ -26,21 +30,37 @@ import com.google.common.io.Files; import org.elasticsearch.client.Client; import org.json.JSONObject; +import org.junit.Before; import org.junit.Ignore; import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; import org.mockito.Mockito; +import org.mockito.runners.MockitoJUnitRunner; import java.io.File; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.sql.SQLFeatureNotSupportedException; +import java.util.List; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Mockito.when; +@RunWith(MockitoJUnitRunner.class) public class JSONRequestTest { + @Mock + private ColumnTypeProvider columnTypeProvider; + + @Before + public void setup() { + when(columnTypeProvider.get(anyInt())).thenReturn(Schema.Type.DOUBLE); + } + @Test public void aggWithoutWhere() { String explainSQL = explainSQL("SELECT name, COUNT(nested(projects, 'projects')) AS c " + @@ -345,7 +365,8 @@ private String explain(String request) { private String translate(String sql, JSONObject jsonRequest) throws SQLFeatureNotSupportedException, SqlParseException { Client mockClient = Mockito.mock(Client.class); CheckScriptContents.stubMockClient(mockClient); - QueryAction queryAction = ESActionFactory.create(mockClient, sql); + QueryAction queryAction = + ESActionFactory.create(mockClient, new QueryActionRequest(sql, columnTypeProvider, Format.JDBC)); SqlRequest sqlRequest = new SqlRequest(sql, jsonRequest); queryAction.setSqlRequest(sqlRequest); diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/domain/ColumnTypeProviderTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/domain/ColumnTypeProviderTest.java new file mode 100644 index 0000000000..be4a3087b7 --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/domain/ColumnTypeProviderTest.java @@ -0,0 +1,54 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.unittest.domain; + +import com.amazon.opendistroforelasticsearch.sql.antlr.semantic.types.base.ESDataType; +import com.amazon.opendistroforelasticsearch.sql.antlr.semantic.types.operator.SetOperator; +import com.amazon.opendistroforelasticsearch.sql.antlr.semantic.types.special.Product; +import com.amazon.opendistroforelasticsearch.sql.domain.ColumnTypeProvider; +import com.amazon.opendistroforelasticsearch.sql.executor.format.Schema; +import com.google.common.collect.ImmutableList; +import org.junit.Test; + +import static com.amazon.opendistroforelasticsearch.sql.domain.ColumnTypeProvider.COLUMN_DEFAULT_TYPE; +import static org.junit.Assert.assertEquals; + +public class ColumnTypeProviderTest { + @Test + public void singleESDataTypeShouldReturnCorrectSchemaType() { + assertEquals(Schema.Type.LONG, new ColumnTypeProvider(ESDataType.LONG).get(0)); + } + + @Test + public void productTypeShouldReturnCorrectSchemaType() { + ColumnTypeProvider columnTypeProvider = + new ColumnTypeProvider(new Product(ImmutableList.of(ESDataType.LONG, ESDataType.SHORT))); + assertEquals(Schema.Type.LONG, columnTypeProvider.get(0)); + assertEquals(Schema.Type.SHORT, columnTypeProvider.get(1)); + } + + @Test + public void unSupportedTypeShouldReturnDefaultSchemaType() { + ColumnTypeProvider columnTypeProvider = new ColumnTypeProvider(SetOperator.UNION); + assertEquals(COLUMN_DEFAULT_TYPE, columnTypeProvider.get(0)); + } + + @Test + public void providerWithoutColumnTypeShouldReturnDefaultSchemaType() { + ColumnTypeProvider columnTypeProvider = new ColumnTypeProvider(); + assertEquals(COLUMN_DEFAULT_TYPE, columnTypeProvider.get(0)); + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/ESActionFactoryTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/ESActionFactoryTest.java new file mode 100644 index 0000000000..16a8033b63 --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/ESActionFactoryTest.java @@ -0,0 +1,77 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.unittest.planner; + +import com.amazon.opendistroforelasticsearch.sql.executor.Format; +import com.amazon.opendistroforelasticsearch.sql.query.ESActionFactory; +import com.amazon.opendistroforelasticsearch.sql.util.SqlParserUtils; +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ESActionFactoryTest { + @Test + public void josnOutputRequestShouldNotMigrateToQueryPlan() { + String sql = "SELECT age, MAX(balance) " + + "FROM account " + + "GROUP BY age"; + + assertFalse(ESActionFactory.shouldMigrateToQueryPlan(SqlParserUtils.parse(sql), Format.JSON)); + } + + @Test + public void nestQueryShouldNotMigrateToQueryPlan() { + String sql = "SELECT age, nested(balance) " + + "FROM account " + + "GROUP BY age"; + + assertFalse(ESActionFactory.shouldMigrateToQueryPlan(SqlParserUtils.parse(sql), Format.JDBC)); + } + + @Test + public void nonAggregationQueryShouldNotMigrateToQueryPlan() { + String sql = "SELECT age " + + "FROM account "; + + assertFalse(ESActionFactory.shouldMigrateToQueryPlan(SqlParserUtils.parse(sql), Format.JDBC)); + } + + @Test + public void aggregationQueryWithoutGroupByShouldMigrateToQueryPlan() { + String sql = "SELECT age, COUNT(balance) " + + "FROM account "; + + assertTrue(ESActionFactory.shouldMigrateToQueryPlan(SqlParserUtils.parse(sql), Format.JDBC)); + } + + @Test + public void aggregationQueryWithExpressionByShouldMigrateToQueryPlan() { + String sql = "SELECT age, MAX(balance) - MIN(balance) " + + "FROM account "; + + assertTrue(ESActionFactory.shouldMigrateToQueryPlan(SqlParserUtils.parse(sql), Format.JDBC)); + } + + @Test + public void queryOnlyHasGroupByShouldMigrateToQueryPlan() { + String sql = "SELECT CAST(age AS DOUBLE) as alias " + + "FROM account " + + "GROUP BY alias"; + + assertTrue(ESActionFactory.shouldMigrateToQueryPlan(SqlParserUtils.parse(sql), Format.JDBC)); + } +} From 200f6039341b468d6d42b0a18804e60a4ea98269 Mon Sep 17 00:00:00 2001 From: penghuo Date: Mon, 13 Jan 2020 13:26:00 -0800 Subject: [PATCH 2/5] update --- .../adapter/QueryPlanRequestBuilder.java | 6 +- .../format/BindingTupleResultSet.java | 58 +++ .../node/project/PhysicalProject.java | 65 ++++ .../physical/node/scroll/PhysicalScroll.java | 103 ++++++ .../SearchAggregationResponseHelper.java | 101 +++++ .../BindingTupleQueryPlannerExecuteTest.java | 124 +++++++ .../converter/SQLAggregationParserTest.java | 298 +++++++++++++++ .../SQLExprToExpressionConverterTest.java | 138 +++++++ .../converter/SQLToOperatorConverterTest.java | 84 +++++ .../SearchAggregationResponseHelperTest.java | 346 ++++++++++++++++++ .../sql/util/AggregationUtils.java | 93 +++++ 11 files changed, 1413 insertions(+), 3 deletions(-) create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/BindingTupleResultSet.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/project/PhysicalProject.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/PhysicalScroll.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/SearchAggregationResponseHelper.java create mode 100644 src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/BindingTupleQueryPlannerExecuteTest.java create mode 100644 src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLAggregationParserTest.java create mode 100644 src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLExprToExpressionConverterTest.java create mode 100644 src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLToOperatorConverterTest.java create mode 100644 src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/physical/SearchAggregationResponseHelperTest.java create mode 100644 src/test/java/com/amazon/opendistroforelasticsearch/sql/util/AggregationUtils.java diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/adapter/QueryPlanRequestBuilder.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/adapter/QueryPlanRequestBuilder.java index 105c8d3145..1980876a4c 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/adapter/QueryPlanRequestBuilder.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/adapter/QueryPlanRequestBuilder.java @@ -16,7 +16,7 @@ package com.amazon.opendistroforelasticsearch.sql.executor.adapter; import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; -import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColNode; +import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColumnNode; import com.amazon.opendistroforelasticsearch.sql.query.planner.core.BindingTupleQueryPlanner; import com.amazon.opendistroforelasticsearch.sql.query.SqlElasticRequestBuilder; import lombok.RequiredArgsConstructor; @@ -37,8 +37,8 @@ public List execute() { return queryPlanner.execute(); } - public List outputColumns() { - return queryPlanner.getColNodes(); + public List outputColumns() { + return queryPlanner.getColumnNodes(); } @Override diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/BindingTupleResultSet.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/BindingTupleResultSet.java new file mode 100644 index 0000000000..da96577b89 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/BindingTupleResultSet.java @@ -0,0 +1,58 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.executor.format; + +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.expression.model.SSValue; +import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColumnNode; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class BindingTupleResultSet extends ResultSet { + + public BindingTupleResultSet(List columnNodes, List bindingTuples) { + this.schema = buildSchema(columnNodes); + this.dataRows = buildDataRows(bindingTuples); + } + + + public static Schema buildSchema(List columnNodes) { + List columnList = columnNodes.stream() + .map(node -> new Schema.Column( + node.getName(), + node.getAlias(), + node.getType())) + .collect(Collectors.toList()); + return new Schema("dummy", "dummy", columnList); + } + + public static DataRows buildDataRows(List bindingTuples) { + List rowList = bindingTuples.stream().map(tuple -> { + Map bindingMap = tuple.getBindingMap(); + Map rowMap = new HashMap<>(); + for (String s : bindingMap.keySet()) { + rowMap.put(s, bindingMap.get(s).value()); + } + return new DataRows.Row(rowMap); + }).collect(Collectors.toList()); + + return new DataRows(bindingTuples.size(), bindingTuples.size(), rowList); + } +} + diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/project/PhysicalProject.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/project/PhysicalProject.java new file mode 100644 index 0000000000..3ce2049872 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/project/PhysicalProject.java @@ -0,0 +1,65 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.project; + +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.expression.model.SSValue; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll.BindingTupleRow; +import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColumnNode; +import com.amazon.opendistroforelasticsearch.sql.query.planner.core.PlanNode; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.PhysicalOperator; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.Row; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.estimation.Cost; +import lombok.RequiredArgsConstructor; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * The definition of Project Operator. + */ +@RequiredArgsConstructor +public class PhysicalProject implements PhysicalOperator { + private final PhysicalOperator next; + private final List fields; + + @Override + public Cost estimate() { + return null; + } + + @Override + public PlanNode[] children() { + return new PlanNode[]{next}; + } + + @Override + public boolean hasNext() { + return next.hasNext(); + } + + @Override + public Row next() { + BindingTuple input = next.next().data(); + Map output = new HashMap<>(); + for (ColumnNode field : fields) { + SSValue ssValue = field.getExpr().valueOf(input); + output.put(field.getName(), ssValue); + } + return new BindingTupleRow(BindingTuple.builder().bindingMap(output).build()); + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/PhysicalScroll.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/PhysicalScroll.java new file mode 100644 index 0000000000..b4ee975ba1 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/PhysicalScroll.java @@ -0,0 +1,103 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll; + +import com.amazon.opendistroforelasticsearch.sql.exception.SqlParseException; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.expression.model.SSValue; +import com.amazon.opendistroforelasticsearch.sql.query.AggregationQueryAction; +import com.amazon.opendistroforelasticsearch.sql.query.QueryAction; +import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ExecuteParams; +import com.amazon.opendistroforelasticsearch.sql.query.planner.core.PlanNode; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.PhysicalOperator; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.Row; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.estimation.Cost; +import lombok.RequiredArgsConstructor; +import lombok.SneakyThrows; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.search.aggregations.Aggregations; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static com.amazon.opendistroforelasticsearch.sql.expression.model.SSValueFactory.fromJson; +import static com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll.SearchAggregationResponseHelper.flatten; + +/** + * The definition of Scroll Operator. + */ +@RequiredArgsConstructor +public class PhysicalScroll implements PhysicalOperator { + private final QueryAction queryAction; + + private Iterator rowIterator; + + @Override + public Cost estimate() { + return null; + } + + @Override + public PlanNode[] children() { + return new PlanNode[0]; + } + + @Override + public boolean hasNext() { + return rowIterator.hasNext(); + } + + @Override + public Row next() { + return rowIterator.next(); + } + + @Override + public void open(ExecuteParams params) { + try { + ActionResponse response = queryAction.explain().get(); + if (queryAction instanceof AggregationQueryAction) { + populateSearchAggregationResponse(((SearchResponse) response).getAggregations()); + } else { + throw new IllegalStateException("Not support QueryAction type: " + queryAction.getClass()); + } + } catch (SqlParseException e) { + throw new RuntimeException(e); + } + } + + private void populateSearchAggregationResponse(Aggregations aggs) { + List> flatten = flatten(aggs); + List bindingTupleList = flatten.stream().map(map -> { + Map ssValueMap = new HashMap<>(); + for (Map.Entry entry : map.entrySet()) { + ssValueMap.put(entry.getKey(), fromJson(entry.getValue())); + } + return BindingTuple.builder().bindingMap(ssValueMap).build(); + }).map(bindingTuple -> new BindingTupleRow(bindingTuple)).collect(Collectors.toList()); + rowIterator = bindingTupleList.iterator(); + } + + @SneakyThrows + @Override + public String toString() { + return queryAction.explain().toString(); + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/SearchAggregationResponseHelper.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/SearchAggregationResponseHelper.java new file mode 100644 index 0000000000..c1951bdfbd --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/SearchAggregationResponseHelper.java @@ -0,0 +1,101 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll; + +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTupleFactory; +import com.google.common.annotations.VisibleForTesting; +import org.elasticsearch.search.aggregations.Aggregation; +import org.elasticsearch.search.aggregations.Aggregations; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; +import org.elasticsearch.search.aggregations.bucket.terms.Terms; +import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregation; +import org.elasticsearch.search.aggregations.metrics.Percentile; +import org.elasticsearch.search.aggregations.metrics.Percentiles; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +/** + * The definition of Search {@link Aggregations} parser helper class. + */ +public class SearchAggregationResponseHelper { + public static List populateSearchAggregationResponse(Aggregations aggs) { + List> flatten = flatten(aggs); + List bindingTupleList = flatten.stream() + .map(BindingTupleFactory::from) + .map(bindingTuple -> new BindingTupleRow(bindingTuple)) + .collect(Collectors.toList()); + return bindingTupleList; + } + + @VisibleForTesting + public static List> flatten(Aggregations aggregations) { + List aggregationList = aggregations.asList(); + List> resultList = new ArrayList<>(); + Map resultMap = new HashMap<>(); + for (Aggregation aggregation : aggregationList) { + if (aggregation instanceof Terms) { + for (Terms.Bucket bucket : ((Terms) aggregation).getBuckets()) { + List> internalBucketList = flatten(bucket.getAggregations()); + fillResultListWithInternalBucket(resultList, internalBucketList, aggregation.getName(), + bucket.getKey()); + } + } else if (aggregation instanceof NumericMetricsAggregation.SingleValue) { + resultMap.put(aggregation.getName(), ((NumericMetricsAggregation.SingleValue) aggregation).value()); + } else if (aggregation instanceof Percentiles) { + Percentiles percentiles = (Percentiles) aggregation; + resultMap.putAll((Map) StreamSupport.stream(percentiles.spliterator(), false) + .collect(Collectors.toMap( + (percentile) -> String.format("%s_%s", percentiles.getName(), percentile.getPercent()), + Percentile::getValue, (v1, v2) -> { + throw new IllegalArgumentException( + String.format("Duplicate key for values %s and %s", v1, v2)); + }, HashMap::new))); + } else if (aggregation instanceof Histogram) { + for (Histogram.Bucket bucket : ((Histogram) aggregation).getBuckets()) { + List> internalBucketList = flatten(bucket.getAggregations()); + fillResultListWithInternalBucket(resultList, internalBucketList, aggregation.getName(), + bucket.getKeyAsString()); + } + } else { + throw new RuntimeException("unsupported aggregation type " + aggregation.getType()); + } + } + if (!resultMap.isEmpty()) { + resultList.add(resultMap); + } + return resultList; + } + + private static void fillResultListWithInternalBucket(List> resultList, + List> internalBucketList, + String aggregationName, Object bucketKey) { + if (internalBucketList.isEmpty()) { + resultList.add(new HashMap() {{ + put(aggregationName, bucketKey); + }}); + } else { + for (Map map : internalBucketList) { + map.put(aggregationName, bucketKey); + } + resultList.addAll(internalBucketList); + } + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/BindingTupleQueryPlannerExecuteTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/BindingTupleQueryPlannerExecuteTest.java new file mode 100644 index 0000000000..0613cb0023 --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/BindingTupleQueryPlannerExecuteTest.java @@ -0,0 +1,124 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.unittest.planner; + +import com.amazon.opendistroforelasticsearch.sql.domain.ColumnTypeProvider; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTupleFactory; +import com.amazon.opendistroforelasticsearch.sql.query.planner.core.BindingTupleQueryPlanner; +import com.amazon.opendistroforelasticsearch.sql.util.AggregationUtils; +import com.amazon.opendistroforelasticsearch.sql.util.SqlParserUtils; +import com.google.common.collect.ImmutableMap; +import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.search.aggregations.Aggregations; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.stubbing.Answer; + +import java.util.List; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class BindingTupleQueryPlannerExecuteTest { + @Mock + private Client client; + @Mock + private SearchResponse aggResponse; + @Mock + private ColumnTypeProvider columnTypeProvider; + + @Before + public void init() { + MockitoAnnotations.initMocks(this); + + ActionFuture mockFuture = mock(ActionFuture.class); + when(client.execute(any(), any())).thenReturn(mockFuture); + when(mockFuture.actionGet()).thenAnswer(invocationOnMock -> aggResponse); + } + + @Test + public void testAggregationShouldPass() { + assertThat(query("SELECT gender, MAX(age) as max, MIN(age) as min FROM accounts GROUP BY gender", + mockSearchAggregation()), + containsInAnyOrder( + BindingTupleFactory.from(ImmutableMap.of("gender", "m", "max", 20d, "min", 10d)), + BindingTupleFactory.from(ImmutableMap.of("gender", "f", "max", 40d, "min", 20d)))); + } + + + protected List query(String sql, MockSearchAggregation mockAgg) { + doAnswer(mockAgg).when(aggResponse).getAggregations(); + + BindingTupleQueryPlanner queryPlanner = + new BindingTupleQueryPlanner(client, SqlParserUtils.parse(sql), columnTypeProvider); + return queryPlanner.execute(); + } + + private MockSearchAggregation mockSearchAggregation() { + return new MockSearchAggregation("{\n" + + " \"sterms#gender\": {\n" + + " \"buckets\": [\n" + + " {\n" + + " \"key\": \"m\",\n" + + " \"doc_count\": 507,\n" + + " \"min#min\": {\n" + + " \"value\": 10\n" + + " },\n" + + " \"max#max\": {\n" + + " \"value\": 20\n" + + " }\n" + + " },\n" + + " {\n" + + " \"key\": \"f\",\n" + + " \"doc_count\": 493,\n" + + " \"min#min\": {\n" + + " \"value\": 20\n" + + " },\n" + + " \"max#max\": {\n" + + " \"value\": 40\n" + + " }\n" + + " }\n" + + " ]\n" + + " }\n" + + "}"); + } + + protected static class MockSearchAggregation implements Answer { + private final Aggregations aggregation; + + public MockSearchAggregation(String agg) { + aggregation = AggregationUtils.fromJson(agg); + } + + @Override + public Aggregations answer(InvocationOnMock invocationOnMock) throws Throwable { + return aggregation; + } + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLAggregationParserTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLAggregationParserTest.java new file mode 100644 index 0000000000..7a38d092c0 --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLAggregationParserTest.java @@ -0,0 +1,298 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.unittest.planner.converter; + +import com.alibaba.druid.sql.SQLUtils; +import com.alibaba.druid.sql.ast.expr.SQLAggregateExpr; +import com.alibaba.druid.sql.ast.expr.SQLQueryExpr; +import com.alibaba.druid.sql.ast.statement.SQLSelectItem; +import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlSelectQueryBlock; +import com.alibaba.druid.util.JdbcConstants; +import com.amazon.opendistroforelasticsearch.sql.domain.ColumnTypeProvider; +import com.amazon.opendistroforelasticsearch.sql.expression.core.Expression; +import com.amazon.opendistroforelasticsearch.sql.query.planner.converter.SQLAggregationParser; +import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColumnNode; +import org.hamcrest.Description; +import org.hamcrest.TypeSafeMatcher; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.runners.MockitoJUnitRunner; + +import java.util.List; + +import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.add; +import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.cast; +import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.log; +import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.var; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; + +@RunWith(MockitoJUnitRunner.class) +public class SQLAggregationParserTest { + @Rule + public ExpectedException exceptionRule = ExpectedException.none(); + + @Test + public void parseAggWithoutExpressionShouldPass() { + String sql = "SELECT dayOfWeek, max(FlightDelayMin), MIN(FlightDelayMin) as min " + + "FROM kibana_sample_data_flights " + + "GROUP BY dayOfWeek"; + SQLAggregationParser parser = new SQLAggregationParser(new ColumnTypeProvider()); + parser.parse(mYSqlSelectQueryBlock(sql)); + List sqlSelectItems = parser.selectItemList(); + List columnNodes = parser.getColumnNodes(); + + assertThat(sqlSelectItems, containsInAnyOrder(group("dayOfWeek", "dayOfWeek"), + agg("MAX", "FlightDelayMin", "MAX_0"), + agg("MIN", "FlightDelayMin", "min"))); + + assertThat(columnNodes, containsInAnyOrder(columnNode("dayOfWeek", null, var("dayOfWeek")), + columnNode("MAX(FlightDelayMin)", null, var("MAX_0")), + columnNode("min", "min", var("min")))); + } + + @Test + public void parseAggWithFunctioniWithoutExpressionShouldPass() { + String sql = "SELECT dayOfWeek, max(FlightDelayMin), MIN(FlightDelayMin) as min " + + "FROM kibana_sample_data_flights " + + "GROUP BY dayOfWeek"; + SQLAggregationParser parser = new SQLAggregationParser(new ColumnTypeProvider()); + parser.parse(mYSqlSelectQueryBlock(sql)); + List sqlSelectItems = parser.selectItemList(); + List columnNodes = parser.getColumnNodes(); + + assertThat(sqlSelectItems, containsInAnyOrder(group("dayOfWeek", "dayOfWeek"), + agg("MAX", "FlightDelayMin", "MAX_0"), + agg("MIN", "FlightDelayMin", "min"))); + + assertThat(columnNodes, containsInAnyOrder(columnNode("dayOfWeek", null, var("dayOfWeek")), + columnNode("MAX(FlightDelayMin)", null, var("MAX_0")), + columnNode("min", "min", var("min")))); + } + + @Test + public void parseAggWithExpressionShouldPass() { + String sql = "SELECT dayOfWeek, max(FlightDelayMin) + MIN(FlightDelayMin) as sub " + + "FROM kibana_sample_data_flights " + + "GROUP BY dayOfWeek"; + SQLAggregationParser parser = new SQLAggregationParser(new ColumnTypeProvider()); + parser.parse(mYSqlSelectQueryBlock(sql)); + List sqlSelectItems = parser.selectItemList(); + List columnNodes = parser.getColumnNodes(); + + assertThat(sqlSelectItems, containsInAnyOrder(group("dayOfWeek", "dayOfWeek"), + agg("MAX", "FlightDelayMin", "MAX_0"), + agg("MIN", "FlightDelayMin", "MIN_1"))); + + assertThat(columnNodes, containsInAnyOrder(columnNode("dayOfWeek", null, var("dayOfWeek")), + columnNode("sub", "sub", add(var("MAX_0"), var("MIN_1"))))); + } + + @Test + public void functionOverFiledShouldPass() { + String sql = "SELECT dayOfWeek, max(FlightDelayMin) + MIN(FlightDelayMin) as sub " + + "FROM kibana_sample_data_flights " + + "GROUP BY dayOfWeek"; + SQLAggregationParser parser = new SQLAggregationParser(new ColumnTypeProvider()); + parser.parse(mYSqlSelectQueryBlock(sql)); + List sqlSelectItems = parser.selectItemList(); + List columnNodes = parser.getColumnNodes(); + + assertThat(sqlSelectItems, containsInAnyOrder(group("dayOfWeek", "dayOfWeek"), + agg("MAX", "FlightDelayMin", "MAX_0"), + agg("MIN", "FlightDelayMin", "MIN_1"))); + + assertThat(columnNodes, containsInAnyOrder(columnNode("dayOfWeek", null, var("dayOfWeek")), + columnNode("sub", "sub", add(var("MAX_0"), var("MIN_1"))))); + } + + @Test + public void parseCompoundAggWithExpressionShouldPass() { + String sql = "SELECT ASCII(dayOfWeek), log(max(FlightDelayMin) + MIN(FlightDelayMin)) as log " + + "FROM kibana_sample_data_flights " + + "GROUP BY ASCII(dayOfWeek)"; + SQLAggregationParser parser = new SQLAggregationParser(new ColumnTypeProvider()); + parser.parse(mYSqlSelectQueryBlock(sql)); + List sqlSelectItems = parser.selectItemList(); + List columnNodes = parser.getColumnNodes(); + + assertThat(sqlSelectItems, containsInAnyOrder(group("ASCII(dayOfWeek)", "ASCII(dayOfWeek)"), + agg("MAX", "FlightDelayMin", "MAX_0"), + agg("MIN", "FlightDelayMin", "MIN_1"))); + + assertThat(columnNodes, containsInAnyOrder(columnNode("ASCII(dayOfWeek)", null, var("ASCII(dayOfWeek)")), + columnNode("log", "log", log(add(var("MAX_0"), var("MIN_1")))))); + } + + @Test + public void parseFunctionGroupColumnOverShouldPass() { + String sql = "SELECT CAST(balance AS FLOAT) FROM accounts GROUP BY balance"; + SQLAggregationParser parser = new SQLAggregationParser(new ColumnTypeProvider()); + parser.parse(mYSqlSelectQueryBlock(sql)); + List sqlSelectItems = parser.selectItemList(); + List columnNodes = parser.getColumnNodes(); + + assertThat(sqlSelectItems, containsInAnyOrder(group("balance", "balance"))); + assertThat(columnNodes, containsInAnyOrder(columnNode("CAST(balance AS FLOAT)", null, cast(var("balance"))))); + } + + @Test + public void withoutAggregationShouldPass() { + String sql = "SELECT age, gender FROM accounts GROUP BY age, gender"; + SQLAggregationParser parser = new SQLAggregationParser(new ColumnTypeProvider()); + parser.parse(mYSqlSelectQueryBlock(sql)); + List sqlSelectItems = parser.selectItemList(); + List columnNodes = parser.getColumnNodes(); + + assertThat(sqlSelectItems, containsInAnyOrder( + group("age", "age"), + group("gender", "gender"))); + assertThat(columnNodes, containsInAnyOrder( + columnNode("age", null, var("age")), + columnNode("gender", null, var("gender")))); + } + + @Test + public void groupKeyInSelectWithFunctionShouldPass() { + String sql = "SELECT log(age), max(balance) FROM accounts GROUP BY age"; + SQLAggregationParser parser = new SQLAggregationParser(new ColumnTypeProvider()); + parser.parse(mYSqlSelectQueryBlock(sql)); + List sqlSelectItems = parser.selectItemList(); + List columnNodes = parser.getColumnNodes(); + + assertThat(sqlSelectItems, containsInAnyOrder( + group("age", "age"), + agg("max", "balance", "max_0"))); + assertThat(columnNodes, containsInAnyOrder( + columnNode("log(age)", null, log(var("age"))), + columnNode("max(balance)", null, var("max_0")))); + } + + @Test + public void theDotInFieldNameShouldBeReplaceWithSharp() { + String sql = "SELECT name.lastname, max(balance) FROM accounts GROUP BY name.lastname"; + SQLAggregationParser parser = new SQLAggregationParser(new ColumnTypeProvider()); + parser.parse(mYSqlSelectQueryBlock(sql)); + List sqlSelectItems = parser.selectItemList(); + List columnNodes = parser.getColumnNodes(); + + assertThat(sqlSelectItems, containsInAnyOrder( + group("name.lastname", "name#lastname"), + agg("max", "balance", "max_0"))); + assertThat(columnNodes, containsInAnyOrder( + columnNode("name.lastname", null, var("name#lastname")), + columnNode("max(balance)", null, var("max_0")))); + } + + @Test + public void noGroupKeyInSelectShouldPass() { + String sql = "SELECT AVG(age) FROM t GROUP BY age"; + SQLAggregationParser parser = new SQLAggregationParser(new ColumnTypeProvider()); + parser.parse(mYSqlSelectQueryBlock(sql)); + List sqlSelectItems = parser.selectItemList(); + List columnNodes = parser.getColumnNodes(); + + assertThat(sqlSelectItems, containsInAnyOrder( + agg("avg", "age", "avg_0"))); + assertThat(columnNodes, containsInAnyOrder( + columnNode("avg(age)", null, var("avg_0")))); + } + + /** + * TermQueryExplainIT.testNestedSingleGroupBy + */ + @Test + public void aggregationWithNestedShouldThrowException() { + exceptionRule.expect(RuntimeException.class); + exceptionRule.expectMessage("unsupported operator in select: nested"); + + String sql = "SELECT nested(projects.name, 'projects'),id " + + "FROM t " + + "GROUP BY nested(projects.name.keyword, 'projects')"; + SQLAggregationParser parser = new SQLAggregationParser(new ColumnTypeProvider()); + parser.parse(mYSqlSelectQueryBlock(sql)); + } + + private MySqlSelectQueryBlock mYSqlSelectQueryBlock(String sql) { + String dbType = JdbcConstants.MYSQL; + SQLQueryExpr sqlQueryExpr = (SQLQueryExpr) SQLUtils.toSQLExpr(sql, dbType); + return ((MySqlSelectQueryBlock) sqlQueryExpr.getSubQuery().getQuery()); + } + + private TypeSafeMatcher columnNode(String name, String alias, Expression expr) { + return new TypeSafeMatcher() { + @Override + public void describeTo(Description description) { + description.appendText(String.format("(name=%s,alias=%s,expression=%s)", name, alias, expr)); + } + + @Override + protected boolean matchesSafely(ColumnNode item) { + if (name == null) { + return false; + } + if (alias == null && item.getAlias() != null) { + return false; + } + + return name.equalsIgnoreCase(item.getName()) && + ((alias == null && item.getAlias() == null) || alias.equals(item.getAlias())) && + expr.toString().equalsIgnoreCase(item.getExpr().toString()); + } + }; + } + + private TypeSafeMatcher agg(String methodName, String name, String alias) { + return new TypeSafeMatcher() { + @Override + public void describeTo(Description description) { + description.appendText(String.format("(methodName=%s, name=%s, alias=%s)", methodName, name, alias)); + } + + @Override + protected boolean matchesSafely(SQLSelectItem item) { + if (item.getExpr() instanceof SQLAggregateExpr) { + return ((SQLAggregateExpr) item.getExpr()).getMethodName().equalsIgnoreCase(methodName) && + ((SQLAggregateExpr) item.getExpr()).getArguments() + .get(0) + .toString() + .equalsIgnoreCase(name) && + ((item.getAlias() == null && alias == null) || item.getAlias().equalsIgnoreCase(alias)); + } else { + return false; + } + } + }; + } + + private TypeSafeMatcher group(String name, String alias) { + return new TypeSafeMatcher() { + @Override + public void describeTo(Description description) { + description.appendText(String.format("(name=%s, alias=%s)", name, alias)); + } + + @Override + protected boolean matchesSafely(SQLSelectItem item) { + boolean b = item.getExpr().toString().equalsIgnoreCase(name) && + ((item.getAlias() == null && alias == null) || item.getAlias().equalsIgnoreCase(alias)); + return b; + } + }; + } +} \ No newline at end of file diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLExprToExpressionConverterTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLExprToExpressionConverterTest.java new file mode 100644 index 0000000000..62decdaa5b --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLExprToExpressionConverterTest.java @@ -0,0 +1,138 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.unittest.planner.converter; + +import com.alibaba.druid.sql.ast.expr.SQLAggregateExpr; +import com.alibaba.druid.sql.ast.expr.SQLBinaryOpExpr; +import com.alibaba.druid.sql.ast.expr.SQLBinaryOperator; +import com.alibaba.druid.sql.ast.expr.SQLIdentifierExpr; +import com.alibaba.druid.sql.ast.expr.SQLMethodInvokeExpr; +import com.amazon.opendistroforelasticsearch.sql.expression.core.Expression; +import com.amazon.opendistroforelasticsearch.sql.query.planner.converter.SQLAggregationParser; +import com.amazon.opendistroforelasticsearch.sql.query.planner.converter.SQLExprToExpressionConverter; +import com.google.common.collect.ImmutableMap; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.runners.MockitoJUnitRunner; + +import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.add; +import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.log; +import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.var; +import static org.junit.Assert.assertEquals; + +@RunWith(MockitoJUnitRunner.class) +public class SQLExprToExpressionConverterTest { + @Rule + public ExpectedException exceptionRule = ExpectedException.none(); + + private SQLExprToExpressionConverter converter; + private SQLAggregationParser.Context context; + private final SQLAggregateExpr maxA = new SQLAggregateExpr("MAX"); + private final SQLAggregateExpr maxB = new SQLAggregateExpr("MAX"); + private final SQLAggregateExpr minA = new SQLAggregateExpr("MIN"); + private final SQLIdentifierExpr groupG = new SQLIdentifierExpr("A"); + private final SQLIdentifierExpr aggA = new SQLIdentifierExpr("A"); + private final SQLIdentifierExpr aggB = new SQLIdentifierExpr("B"); + + @Before + public void setup() { + maxA.getArguments().add(aggA); + maxB.getArguments().add(aggB); + minA.getArguments().add(aggA); + context = new SQLAggregationParser.Context(ImmutableMap.of()); + converter = new SQLExprToExpressionConverter(context); + } + + @Test + public void identifierShouldReturnVarExpression() { + context.addGroupKeyExpr(groupG); + Expression expression = converter.convert(groupG); + + assertEquals(var("A").toString(), expression.toString()); + } + + @Test + public void binaryOperatorAddShouldReturnAddExpression() { + context.addAggregationExpr(maxA); + context.addAggregationExpr(minA); + + Expression expression = converter.convert(new SQLBinaryOpExpr(maxA, SQLBinaryOperator.Add, minA)); + assertEquals(add(var("MAX_0"), var("MIN_1")).toString(), expression.toString()); + } + + @Test + public void compoundBinaryOperatorShouldReturnCorrectExpression() { + context.addAggregationExpr(maxA); + context.addAggregationExpr(minA); + + Expression expression = converter.convert(new SQLBinaryOpExpr(maxA, SQLBinaryOperator.Add, + new SQLBinaryOpExpr(maxA, SQLBinaryOperator.Add, + minA))); + assertEquals(add(var("MAX_0"), add(var("MAX_0"), var("MIN_1"))).toString(), expression.toString()); + } + + @Test + public void functionOverCompoundBinaryOperatorShouldReturnCorrectExpression() { + context.addAggregationExpr(maxA); + context.addAggregationExpr(minA); + + SQLMethodInvokeExpr methodInvokeExpr = new SQLMethodInvokeExpr("LOG"); + methodInvokeExpr.addParameter(new SQLBinaryOpExpr(maxA, SQLBinaryOperator.Add, + new SQLBinaryOpExpr(maxA, SQLBinaryOperator.Add, + minA))); + + Expression expression = converter.convert(methodInvokeExpr); + assertEquals(log(add(var("MAX_0"), add(var("MAX_0"), var("MIN_1")))).toString(), expression.toString()); + } + + @Test + public void functionOverGroupColumn() { + context.addAggregationExpr(maxA); + context.addAggregationExpr(minA); + + SQLMethodInvokeExpr methodInvokeExpr = new SQLMethodInvokeExpr("LOG"); + methodInvokeExpr.addParameter(new SQLBinaryOpExpr(maxA, SQLBinaryOperator.Add, + new SQLBinaryOpExpr(maxA, SQLBinaryOperator.Add, + minA))); + + Expression expression = converter.convert(methodInvokeExpr); + assertEquals(log(add(var("MAX_0"), add(var("MAX_0"), var("MIN_1")))).toString(), expression.toString()); + } + + @Test + public void unknownIdentifierShouldThrowException() { + context.addAggregationExpr(maxA); + context.addAggregationExpr(minA); + + exceptionRule.expect(RuntimeException.class); + exceptionRule.expectMessage("unsupported expr"); + converter.convert(new SQLBinaryOpExpr(maxA, SQLBinaryOperator.Add, maxB)); + } + + @Test + public void unsupportOperationShouldThrowException() { + exceptionRule.expect(RuntimeException.class); + exceptionRule.expectMessage("unsupported operator in select: LOG10"); + + context.addAggregationExpr(maxA); + SQLMethodInvokeExpr methodInvokeExpr = new SQLMethodInvokeExpr("LOG10"); + methodInvokeExpr.addParameter(maxA); + converter.convert(methodInvokeExpr); + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLToOperatorConverterTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLToOperatorConverterTest.java new file mode 100644 index 0000000000..6d40daceae --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLToOperatorConverterTest.java @@ -0,0 +1,84 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.unittest.planner.converter; + +import com.alibaba.druid.sql.SQLUtils; +import com.alibaba.druid.sql.ast.expr.SQLQueryExpr; +import com.alibaba.druid.util.JdbcConstants; +import com.amazon.opendistroforelasticsearch.sql.domain.ColumnTypeProvider; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.query.planner.converter.SQLToOperatorConverter; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.PhysicalOperator; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.project.PhysicalProject; +import org.elasticsearch.client.Client; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import static org.junit.Assert.assertTrue; + +@RunWith(MockitoJUnitRunner.class) +public class SQLToOperatorConverterTest { + @Mock + private Client client; + + private SQLToOperatorConverter converter; + + @Before + public void setup() { + converter = new SQLToOperatorConverter(client, new ColumnTypeProvider()); + } + + @Test + public void convertAggShouldPass() { + String sql = "SELECT dayOfWeek, max(FlightDelayMin), MIN(FlightDelayMin) as min " + + "FROM kibana_sample_data_flights " + + "GROUP BY dayOfWeek"; + toExpr(sql).accept(converter); + PhysicalOperator physicalOperator = converter.getPhysicalOperator(); + + assertTrue(physicalOperator instanceof PhysicalProject); + } + + @Test + public void convertMaxMinusMinShouldPass() { + String sql = "SELECT dayOfWeek, max(FlightDelayMin) - MIN(FlightDelayMin) as diff " + + "FROM kibana_sample_data_flights " + + "GROUP BY dayOfWeek"; + toExpr(sql).accept(converter); + PhysicalOperator physicalOperator = converter.getPhysicalOperator(); + + assertTrue(physicalOperator instanceof PhysicalProject); + } + + @Test + public void convertDistinctPass() { + String sql = "SELECT dayOfWeek, max(FlightDelayMin) - MIN(FlightDelayMin) as diff " + + "FROM kibana_sample_data_flights " + + "GROUP BY dayOfWeek"; + toExpr(sql).accept(converter); + PhysicalOperator physicalOperator = converter.getPhysicalOperator(); + + assertTrue(physicalOperator instanceof PhysicalProject); + } + + private SQLQueryExpr toExpr(String sql) { + String dbType = JdbcConstants.MYSQL; + return (SQLQueryExpr) SQLUtils.toSQLExpr(sql, dbType); + } +} \ No newline at end of file diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/physical/SearchAggregationResponseHelperTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/physical/SearchAggregationResponseHelperTest.java new file mode 100644 index 0000000000..3cf5a2aa1d --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/physical/SearchAggregationResponseHelperTest.java @@ -0,0 +1,346 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.unittest.planner.physical; + +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTupleFactory; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll.BindingTupleRow; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll.SearchAggregationResponseHelper; +import com.amazon.opendistroforelasticsearch.sql.util.AggregationUtils; +import com.google.common.collect.ImmutableMap; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeMatcher; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.runners.MockitoJUnitRunner; + +import java.util.List; +import java.util.Map; + +import static com.amazon.opendistroforelasticsearch.sql.util.MatcherUtils.featureValueOf; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasEntry; +import static org.junit.Assert.assertEquals; + +@RunWith(MockitoJUnitRunner.class) +public class SearchAggregationResponseHelperTest { + /** + * SELECT MAX(age) as max + * FROM accounts + */ + @Test + public void noBucketOneMetricShouldPass() { + String json = "{\n" + + " \"max#max\": {\n" + + " \"value\": 40\n" + + " }\n" + + "}"; + List> result = SearchAggregationResponseHelper.flatten(AggregationUtils.fromJson(json)); + assertThat(result, contains(allOf(hasEntry("max", 40d)))); + } + + /** + * SELECT MAX(age) as max, MIN(age) as min + * FROM accounts + */ + @Test + public void noBucketMultipleMetricShouldPass() { + String json = "{\n" + + " \"max#max\": {\n" + + " \"value\": 40\n" + + " },\n" + + " \"min#min\": {\n" + + " \"value\": 20\n" + + " }\n" + + "}"; + List> result = SearchAggregationResponseHelper.flatten(AggregationUtils.fromJson(json)); + assertThat(result, contains(allOf(hasEntry("max", 40d), hasEntry("min", 20d)))); + } + + /** + * SELECT gender, MAX(age) as max, MIN(age) as min + * FROM accounts + * GROUP BY gender + */ + @Test + public void oneBucketMultipleMetricShouldPass() { + String json = "{\n" + + " \"sterms#gender\": {\n" + + " \"buckets\": [\n" + + " {\n" + + " \"key\": \"m\",\n" + + " \"doc_count\": 507,\n" + + " \"min#min\": {\n" + + " \"value\": 10\n" + + " },\n" + + " \"max#max\": {\n" + + " \"value\": 20\n" + + " }\n" + + " },\n" + + " {\n" + + " \"key\": \"f\",\n" + + " \"doc_count\": 493,\n" + + " \"min#min\": {\n" + + " \"value\": 20\n" + + " },\n" + + " \"max#max\": {\n" + + " \"value\": 40\n" + + " }\n" + + " }\n" + + " ]\n" + + " }\n" + + "}"; + List> result = SearchAggregationResponseHelper.flatten(AggregationUtils.fromJson(json)); + assertThat(result, contains(allOf(hasEntry("gender", (Object) "m"), hasEntry("min", 10d), hasEntry("max", 20d)), + allOf(hasEntry("gender", (Object) "f"), hasEntry("min", 20d), + hasEntry("max", 40d)))); + } + + /** + * SELECT gender, state, MAX(age) as max, MIN(age) as min + * FROM accounts + * GROUP BY gender, state + */ + @Test + public void multipleBucketMultipleMetricShouldPass() { + String json = "{\n" + + " \"sterms#gender\": {\n" + + " \"buckets\": [\n" + + " {\n" + + " \"key\": \"m\",\n" + + " \"sterms#state\": {\n" + + " \"buckets\": [\n" + + " {\n" + + " \"key\": \"MD\",\n" + + " \"min#min\": {\n" + + " \"value\": 22\n" + + " },\n" + + " \"max#max\": {\n" + + " \"value\": 39\n" + + " }\n" + + " },\n" + + " {\n" + + " \"key\": \"ID\",\n" + + " \"min#min\": {\n" + + " \"value\": 23\n" + + " },\n" + + " \"max#max\": {\n" + + " \"value\": 40\n" + + " }\n" + + " }\n" + + " ]\n" + + " }\n" + + " },\n" + + " {\n" + + " \"key\": \"f\",\n" + + " \"sterms#state\": {\n" + + " \"buckets\": [\n" + + " {\n" + + " \"key\": \"TX\",\n" + + " \"min#min\": {\n" + + " \"value\": 20\n" + + " },\n" + + " \"max#max\": {\n" + + " \"value\": 38\n" + + " }\n" + + " },\n" + + " {\n" + + " \"key\": \"MI\",\n" + + " \"min#min\": {\n" + + " \"value\": 22\n" + + " },\n" + + " \"max#max\": {\n" + + " \"value\": 40\n" + + " }\n" + + " }\n" + + " ]\n" + + " }\n" + + " }\n" + + " ]\n" + + " }\n" + + "}"; + List> result = SearchAggregationResponseHelper.flatten(AggregationUtils.fromJson(json)); + assertThat(result, contains( + allOf(hasEntry("gender", (Object) "m"), hasEntry("state", (Object) "MD"), hasEntry("min", 22d), + hasEntry("max", 39d)), + allOf(hasEntry("gender", (Object) "m"), hasEntry("state", (Object) "ID"), hasEntry("min", 23d), + hasEntry("max", 40d)), + allOf(hasEntry("gender", (Object) "f"), hasEntry("state", (Object) "TX"), hasEntry("min", 20d), + hasEntry("max", 38d)), + allOf(hasEntry("gender", (Object) "f"), hasEntry("state", (Object) "MI"), hasEntry("min", 22d), + hasEntry("max", 40d)))); + } + + /** + * SELECT age, gender FROM accounts GROUP BY age, gender + */ + @Test + public void multipleBucketWithoutMetricShouldPass() { + String json = "{\n" + + " \"lterms#age\": {\n" + + " \"buckets\": [\n" + + " {\n" + + " \"key\": 31,\n" + + " \"doc_count\": 61,\n" + + " \"sterms#gender\": {\n" + + " \"buckets\": [\n" + + " {\n" + + " \"key\": \"m\",\n" + + " \"doc_count\": 35\n" + + " },\n" + + " {\n" + + " \"key\": \"f\",\n" + + " \"doc_count\": 26\n" + + " }\n" + + " ]\n" + + " }\n" + + " },\n" + + " {\n" + + " \"key\": 39,\n" + + " \"doc_count\": 60,\n" + + " \"sterms#gender\": {\n" + + " \"buckets\": [\n" + + " {\n" + + " \"key\": \"f\",\n" + + " \"doc_count\": 38\n" + + " },\n" + + " {\n" + + " \"key\": \"m\",\n" + + " \"doc_count\": 22\n" + + " }\n" + + " ]\n" + + " }\n" + + " }\n" + + " ]\n" + + " }\n" + + "}"; + List> result = SearchAggregationResponseHelper.flatten(AggregationUtils.fromJson(json)); + assertThat(result, containsInAnyOrder( + allOf(hasEntry("age", (Object) 31L), hasEntry("gender","m")), + allOf(hasEntry("age", (Object) 31L), hasEntry("gender","f")), + allOf(hasEntry("age", (Object) 39L), hasEntry("gender","m")), + allOf(hasEntry("age", (Object) 39L), hasEntry("gender","f")))); + } + + /** + * SELECT PERCENTILES(age) FROM accounts + */ + @Test + public void noBucketPercentilesShouldPass() { + String json = "{\n" + + " \"percentiles_bucket#age\": {\n" + + " \"values\": {\n" + + " \"1.0\": 20,\n" + + " \"5.0\": 21,\n" + + " \"25.0\": 25,\n" + + " \"50.0\": 30.90909090909091,\n" + + " \"75.0\": 35,\n" + + " \"95.0\": 39,\n" + + " \"99.0\": 40\n" + + " }\n" + + " }\n" + + "}"; + List> result = SearchAggregationResponseHelper.flatten(AggregationUtils.fromJson(json)); + assertThat(result, contains(allOf(hasEntry("age_1.0", 20d)))); + } + + /** + * SELECT count(*) from online + * GROUP BY date_histogram('field'='insert_time','interval'='4d','alias'='days') + */ + @Test + public void populateShouldPass() { + String json = "{\n" + + " \"date_histogram#days\": {\n" + + " \"buckets\": [\n" + + " {\n" + + " \"key_as_string\": \"2014-08-14 00:00:00\",\n" + + " \"key\": 1407974400000,\n" + + " \"doc_count\": 477,\n" + + " \"value_count#COUNT_0\": {\n" + + " \"value\": 477\n" + + " }\n" + + " }\n" + + " ]\n" + + " }\n" + + "}"; + List> result = SearchAggregationResponseHelper.flatten(AggregationUtils.fromJson(json)); + assertThat(result, containsInAnyOrder( + allOf(hasEntry("days", (Object) "2014-08-14 00:00:00"), hasEntry("COUNT_0",477d)))); + } + + /** + * SELECT s + */ + @Test + public void populateSearchAggregationResponeShouldPass() { + String json = "{\n" + + " \"lterms#age\": {\n" + + " \"buckets\": [\n" + + " {\n" + + " \"key\": 31,\n" + + " \"doc_count\": 61,\n" + + " \"sterms#gender\": {\n" + + " \"buckets\": [\n" + + " {\n" + + " \"key\": \"m\",\n" + + " \"doc_count\": 35\n" + + " },\n" + + " {\n" + + " \"key\": \"f\",\n" + + " \"doc_count\": 26\n" + + " }\n" + + " ]\n" + + " }\n" + + " },\n" + + " {\n" + + " \"key\": 39,\n" + + " \"doc_count\": 60,\n" + + " \"sterms#gender\": {\n" + + " \"buckets\": [\n" + + " {\n" + + " \"key\": \"f\",\n" + + " \"doc_count\": 38\n" + + " },\n" + + " {\n" + + " \"key\": \"m\",\n" + + " \"doc_count\": 22\n" + + " }\n" + + " ]\n" + + " }\n" + + " }\n" + + " ]\n" + + " }\n" + + "}"; + List bindingTupleRows = + SearchAggregationResponseHelper.populateSearchAggregationResponse(AggregationUtils.fromJson(json)); + assertEquals(4, bindingTupleRows.size()); + assertThat(bindingTupleRows, containsInAnyOrder( + bindingTupleRow(BindingTupleFactory.from(ImmutableMap.of("age", 31L, "gender", "m"))), + bindingTupleRow(BindingTupleFactory.from(ImmutableMap.of("age", 31L, "gender", "f"))), + bindingTupleRow(BindingTupleFactory.from(ImmutableMap.of("age", 39L, "gender", "m"))), + bindingTupleRow(BindingTupleFactory.from(ImmutableMap.of("age", 39L, "gender", "f"))))); + } + + private static Matcher bindingTupleRow(BindingTuple bindingTuple) { + return featureValueOf("BindingTuple", equalTo(bindingTuple), BindingTupleRow::data); + } +} \ No newline at end of file diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/util/AggregationUtils.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/util/AggregationUtils.java new file mode 100644 index 0000000000..165a8ed31e --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/util/AggregationUtils.java @@ -0,0 +1,93 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.util; + +import com.google.common.collect.ImmutableMap; +import lombok.SneakyThrows; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.ContextParser; +import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContent; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.search.aggregations.Aggregation; +import org.elasticsearch.search.aggregations.Aggregations; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.histogram.ParsedDateHistogram; +import org.elasticsearch.search.aggregations.bucket.terms.DoubleTerms; +import org.elasticsearch.search.aggregations.bucket.terms.LongTerms; +import org.elasticsearch.search.aggregations.bucket.terms.ParsedDoubleTerms; +import org.elasticsearch.search.aggregations.bucket.terms.ParsedLongTerms; +import org.elasticsearch.search.aggregations.bucket.terms.ParsedStringTerms; +import org.elasticsearch.search.aggregations.bucket.terms.StringTerms; +import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.ParsedAvg; +import org.elasticsearch.search.aggregations.metrics.ParsedMax; +import org.elasticsearch.search.aggregations.metrics.ParsedMin; +import org.elasticsearch.search.aggregations.metrics.ParsedSum; +import org.elasticsearch.search.aggregations.metrics.ParsedValueCount; +import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; +import org.elasticsearch.search.aggregations.pipeline.ParsedPercentilesBucket; +import org.elasticsearch.search.aggregations.pipeline.PercentilesBucketPipelineAggregationBuilder; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; + +public class AggregationUtils { + private final static List entryList = + new ImmutableMap.Builder>().put( + MinAggregationBuilder.NAME, (p, c) -> ParsedMin.fromXContent(p, (String) c)) + .put(MaxAggregationBuilder.NAME, (p, c) -> ParsedMax.fromXContent(p, (String) c)) + .put(SumAggregationBuilder.NAME, (p, c) -> ParsedSum.fromXContent(p, (String) c)) + .put(AvgAggregationBuilder.NAME, (p, c) -> ParsedAvg.fromXContent(p, (String) c)) + .put(StringTerms.NAME, (p, c) -> ParsedStringTerms.fromXContent(p, (String) c)) + .put(LongTerms.NAME, (p, c) -> ParsedLongTerms.fromXContent(p, (String) c)) + .put(DoubleTerms.NAME, (p, c) -> ParsedDoubleTerms.fromXContent(p, (String) c)) + .put(ValueCountAggregationBuilder.NAME, (p, c) -> ParsedValueCount.fromXContent(p, (String) c)) + .put(PercentilesBucketPipelineAggregationBuilder.NAME, + (p, c) -> ParsedPercentilesBucket.fromXContent(p, (String) c)) + .put(DateHistogramAggregationBuilder.NAME, (p, c) -> ParsedDateHistogram.fromXContent(p, (String) c)) + .build() + .entrySet() + .stream() + .map(entry -> new NamedXContentRegistry.Entry(Aggregation.class, new ParseField(entry.getKey()), + entry.getValue())) + .collect(Collectors.toList()); + private final static NamedXContentRegistry namedXContentRegistry = new NamedXContentRegistry(entryList); + private final static XContent xContent = XContentFactory.xContent(XContentType.JSON); + + /** + * Populate {@link Aggregations} from JSON string. + * @param json json string + * @return {@link Aggregations} + */ + public static Aggregations fromJson(String json) { + try { + XContentParser xContentParser = + xContent.createParser(namedXContentRegistry, LoggingDeprecationHandler.INSTANCE, json); + xContentParser.nextToken(); + return Aggregations.fromXContent(xContentParser); + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} From caf3846bf43db7311b8a70d1bea67693eae44711 Mon Sep 17 00:00:00 2001 From: penghuo Date: Mon, 13 Jan 2020 15:48:42 -0800 Subject: [PATCH 3/5] update --- .../sql/executor/csv/CSVResultsExtractor.java | 21 +++++++ .../format/BindingTupleResultSet.java | 7 ++- .../format/PrettyFormatRestExecutor.java | 2 +- .../sql/executor/format/Protocol.java | 18 +++++- .../format/BindingTupleResultSetTest.java | 58 +++++++++++++++++++ .../format/CSVResultsExtractorTest.java | 50 ++++++++++++++++ 6 files changed, 152 insertions(+), 4 deletions(-) create mode 100644 src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/BindingTupleResultSetTest.java create mode 100644 src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/CSVResultsExtractorTest.java diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/csv/CSVResultsExtractor.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/csv/CSVResultsExtractor.java index 21cf1310a1..beac591d09 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/csv/CSVResultsExtractor.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/csv/CSVResultsExtractor.java @@ -15,6 +15,8 @@ package com.amazon.opendistroforelasticsearch.sql.executor.csv; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.expression.model.SSValue; import com.amazon.opendistroforelasticsearch.sql.utils.Util; import com.google.common.base.Joiner; import org.elasticsearch.common.document.DocumentField; @@ -38,6 +40,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; /** * Created by Eliran on 27/12/2015. @@ -84,6 +87,24 @@ public CSVResult extractResults(Object queryResult, boolean flat, String separat return new CSVResult(headers, csvLines); } + // Handle List result. + if (queryResult instanceof List) { + List bindingTuples = (List) queryResult; + List csvLines = bindingTuples.stream().map(tuple -> { + Map bindingMap = tuple.getBindingMap(); + List rowValues = new ArrayList<>(); + for (String fieldName : fieldNames) { + if (bindingMap.containsKey(fieldName)) { + rowValues.add(bindingMap.get(fieldName).value()); + } else { + rowValues.add(""); + } + } + return Joiner.on(separator).join(rowValues); + }).collect(Collectors.toList()); + + return new CSVResult(fieldNames, csvLines); + } return null; } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/BindingTupleResultSet.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/BindingTupleResultSet.java index da96577b89..6bbec17b16 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/BindingTupleResultSet.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/BindingTupleResultSet.java @@ -18,12 +18,16 @@ import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; import com.amazon.opendistroforelasticsearch.sql.expression.model.SSValue; import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColumnNode; +import com.google.common.annotations.VisibleForTesting; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +/** + * The definition of BindingTuple ResultSet. + */ public class BindingTupleResultSet extends ResultSet { public BindingTupleResultSet(List columnNodes, List bindingTuples) { @@ -31,7 +35,7 @@ public BindingTupleResultSet(List columnNodes, List bi this.dataRows = buildDataRows(bindingTuples); } - + @VisibleForTesting public static Schema buildSchema(List columnNodes) { List columnList = columnNodes.stream() .map(node -> new Schema.Column( @@ -42,6 +46,7 @@ public static Schema buildSchema(List columnNodes) { return new Schema("dummy", "dummy", columnList); } + @VisibleForTesting public static DataRows buildDataRows(List bindingTuples) { List rowList = bindingTuples.stream().map(tuple -> { Map bindingMap = tuple.getBindingMap(); diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/PrettyFormatRestExecutor.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/PrettyFormatRestExecutor.java index 478749653e..4c7a693db2 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/PrettyFormatRestExecutor.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/PrettyFormatRestExecutor.java @@ -67,7 +67,7 @@ public String execute(Client client, Map params, QueryAction que try { Object queryResult = QueryActionElasticExecutor.executeAnyAction(client, queryAction); - protocol = new Protocol(client, queryAction.getQueryStatement(), queryResult, format); + protocol = new Protocol(client, queryAction, queryResult, format); } catch (Exception e) { LOG.error("Error happened in pretty formatter", e); protocol = new Protocol(e); diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/Protocol.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/Protocol.java index 6b1376c0a4..031d2728e5 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/Protocol.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/Protocol.java @@ -20,10 +20,16 @@ import com.amazon.opendistroforelasticsearch.sql.domain.QueryStatement; import com.amazon.opendistroforelasticsearch.sql.executor.format.DataRows.Row; import com.amazon.opendistroforelasticsearch.sql.executor.format.Schema.Column; +import com.amazon.opendistroforelasticsearch.sql.executor.adapter.QueryPlanQueryAction; +import com.amazon.opendistroforelasticsearch.sql.executor.adapter.QueryPlanRequestBuilder; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColumnNode; +import com.amazon.opendistroforelasticsearch.sql.query.QueryAction; import org.elasticsearch.client.Client; import org.json.JSONArray; import org.json.JSONObject; +import java.util.List; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -40,10 +46,15 @@ public class Protocol { private long total; private ResultSet resultSet; private ErrorMessage error; + private List columnNodeList; - public Protocol(Client client, QueryStatement query, Object queryResult, String formatType) { + public Protocol(Client client, QueryAction queryAction, Object queryResult, String formatType) { + if (queryAction instanceof QueryPlanQueryAction) { + this.columnNodeList = + ((QueryPlanRequestBuilder) (((QueryPlanQueryAction) queryAction).explain())).outputColumns(); + } this.formatType = formatType; - + QueryStatement query = queryAction.getQueryStatement(); this.status = OK_STATUS; this.resultSet = loadResultSet(client, query, queryResult); this.size = resultSet.getDataRows().getSize(); @@ -57,6 +68,9 @@ public Protocol(Exception e) { } private ResultSet loadResultSet(Client client, QueryStatement queryStatement, Object queryResult) { + if (queryResult instanceof List) { + return new BindingTupleResultSet(columnNodeList, (List) queryResult); + } if (queryStatement instanceof Query) { return new SelectResultSet(client, (Query) queryStatement, queryResult); } else if (queryStatement instanceof IndexStatement) { diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/BindingTupleResultSetTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/BindingTupleResultSetTest.java new file mode 100644 index 0000000000..6700050238 --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/BindingTupleResultSetTest.java @@ -0,0 +1,58 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.unittest.executor.format; + +import com.amazon.opendistroforelasticsearch.sql.executor.format.BindingTupleResultSet; +import com.amazon.opendistroforelasticsearch.sql.executor.format.DataRows; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTupleFactory; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.hamcrest.Matcher; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static com.amazon.opendistroforelasticsearch.sql.util.MatcherUtils.featureValueOf; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.hasEntry; + +public class BindingTupleResultSetTest { + + @Test + public void buildDataRowsFromBindingTupleShouldPass() { + assertThat(row(Arrays.asList(BindingTupleFactory.from(ImmutableMap.of("age", 31, "gender", "m")), + BindingTupleFactory.from(ImmutableMap.of("age", 31, "gender", "f")), + BindingTupleFactory.from(ImmutableMap.of("age", 39, "gender", "m")), + BindingTupleFactory.from(ImmutableMap.of("age", 39, "gender", "f")))), + containsInAnyOrder(rowContents(allOf(hasEntry("age", 31), hasEntry("gender", (Object) "m"))), + rowContents(allOf(hasEntry("age", 31), hasEntry("gender", (Object) "f"))), + rowContents(allOf(hasEntry("age", 39), hasEntry("gender", (Object) "m"))), + rowContents(allOf(hasEntry("age", 39), hasEntry("gender", (Object) "f"))))); + } + + private static Matcher rowContents(Matcher> matcher) { + return featureValueOf("DataRows.Row", matcher, DataRows.Row::getContents); + } + + private List row(List bindingTupleList) { + return ImmutableList.copyOf(BindingTupleResultSet.buildDataRows(bindingTupleList).iterator()); + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/CSVResultsExtractorTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/CSVResultsExtractorTest.java new file mode 100644 index 0000000000..adcc3b7164 --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/CSVResultsExtractorTest.java @@ -0,0 +1,50 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.unittest.executor.format; + +import com.amazon.opendistroforelasticsearch.sql.executor.csv.CSVResult; +import com.amazon.opendistroforelasticsearch.sql.executor.csv.CSVResultsExtractor; +import com.amazon.opendistroforelasticsearch.sql.executor.csv.CsvExtractorException; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTupleFactory; +import com.google.common.collect.ImmutableMap; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; + +public class CSVResultsExtractorTest { + private final CSVResultsExtractor csvResultsExtractor = new CSVResultsExtractor(false, false, false); + + @Test + public void extractResultsFromBindingTupleListShouldPass() throws CsvExtractorException { + CSVResult csvResult = csv(Arrays.asList(BindingTupleFactory.from(ImmutableMap.of("age", 31, "gender", "m")), + BindingTupleFactory.from(ImmutableMap.of("age", 31, "gender", "f")), + BindingTupleFactory.from(ImmutableMap.of("age", 39, "gender", "m")), + BindingTupleFactory.from(ImmutableMap.of("age", 39, "gender", "f"))), + Arrays.asList("age", "gender")); + + assertThat(csvResult.getHeaders(), contains("age", "gender")); + assertThat(csvResult.getLines(), contains("31,m", "31,f", "39,m", "39,f")); + } + + private CSVResult csv(List bindingTupleList, List fieldNames) throws CsvExtractorException { + return csvResultsExtractor.extractResults(bindingTupleList, false, ",", fieldNames); + } +} From e865ef5a155f75e26dae2badad52e39bd1002311 Mon Sep 17 00:00:00 2001 From: penghuo Date: Tue, 14 Jan 2020 06:50:14 -0800 Subject: [PATCH 4/5] update --- .../sql/executor/csv/CSVResultsExtractor.java | 4 +- .../format/BindingTupleResultSet.java | 4 +- .../sql/expression/core/Expression.java | 32 ++++ .../expression/core/ExpressionFactory.java | 143 ++++++++++++++++++ .../sql/expression/domain/BindingTuple.java | 83 ++++++++++ .../expression/model/ExprBooleanValue.java | 48 ++++++ .../expression/model/ExprCollectionValue.java | 53 +++++++ .../sql/expression/model/ExprDoubleValue.java | 45 ++++++ .../expression/model/ExprIntegerValue.java | 45 ++++++ .../expression/model/ExprMissingValue.java | 26 ++++ .../sql/expression/model/ExprStringValue.java | 45 ++++++ .../sql/expression/model/ExprTupleValue.java | 56 +++++++ .../sql/expression/model/ExprValue.java | 63 ++++++++ .../expression/model/ExprValueFactory.java | 82 ++++++++++ .../converter/SQLAggregationParser.java | 8 +- .../node/project/PhysicalProject.java | 8 +- .../physical/node/scroll/PhysicalScroll.java | 6 +- .../SearchAggregationResponseHelper.java | 4 +- .../format/BindingTupleResultSetTest.java | 9 +- .../format/CSVResultsExtractorTest.java | 9 +- .../core/ArithmeticExpressionTest.java | 56 +++++++ .../expression/core/ExpressionTest.java | 41 +++++ .../expression/core/RefExpressionTest.java | 63 ++++++++ .../expression/model/ExprValueTest.java | 47 ++++++ .../BindingTupleQueryPlannerExecuteTest.java | 5 +- .../converter/SQLAggregationParserTest.java | 50 +++--- .../SQLExprToExpressionConverterTest.java | 15 +- .../SearchAggregationResponseHelperTest.java | 11 +- 28 files changed, 997 insertions(+), 64 deletions(-) create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/core/Expression.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/core/ExpressionFactory.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/domain/BindingTuple.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprBooleanValue.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprCollectionValue.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprDoubleValue.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprIntegerValue.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprMissingValue.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprStringValue.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprTupleValue.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprValue.java create mode 100644 src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprValueFactory.java create mode 100644 src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/core/ArithmeticExpressionTest.java create mode 100644 src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/core/ExpressionTest.java create mode 100644 src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/core/RefExpressionTest.java create mode 100644 src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/model/ExprValueTest.java diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/csv/CSVResultsExtractor.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/csv/CSVResultsExtractor.java index beac591d09..3f05a5df52 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/csv/CSVResultsExtractor.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/csv/CSVResultsExtractor.java @@ -16,7 +16,7 @@ package com.amazon.opendistroforelasticsearch.sql.executor.csv; import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; -import com.amazon.opendistroforelasticsearch.sql.expression.model.SSValue; +import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValue; import com.amazon.opendistroforelasticsearch.sql.utils.Util; import com.google.common.base.Joiner; import org.elasticsearch.common.document.DocumentField; @@ -91,7 +91,7 @@ public CSVResult extractResults(Object queryResult, boolean flat, String separat if (queryResult instanceof List) { List bindingTuples = (List) queryResult; List csvLines = bindingTuples.stream().map(tuple -> { - Map bindingMap = tuple.getBindingMap(); + Map bindingMap = tuple.getBindingMap(); List rowValues = new ArrayList<>(); for (String fieldName : fieldNames) { if (bindingMap.containsKey(fieldName)) { diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/BindingTupleResultSet.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/BindingTupleResultSet.java index 6bbec17b16..a028db1ebc 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/BindingTupleResultSet.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/executor/format/BindingTupleResultSet.java @@ -16,7 +16,7 @@ package com.amazon.opendistroforelasticsearch.sql.executor.format; import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; -import com.amazon.opendistroforelasticsearch.sql.expression.model.SSValue; +import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValue; import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColumnNode; import com.google.common.annotations.VisibleForTesting; @@ -49,7 +49,7 @@ public static Schema buildSchema(List columnNodes) { @VisibleForTesting public static DataRows buildDataRows(List bindingTuples) { List rowList = bindingTuples.stream().map(tuple -> { - Map bindingMap = tuple.getBindingMap(); + Map bindingMap = tuple.getBindingMap(); Map rowMap = new HashMap<>(); for (String s : bindingMap.keySet()) { rowMap.put(s, bindingMap.get(s).value()); diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/core/Expression.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/core/Expression.java new file mode 100644 index 0000000000..9b643ef325 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/core/Expression.java @@ -0,0 +1,32 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.expression.core; + + +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValue; + +/** + * The definition of the Expression. + */ +public interface Expression { + /** + * Evaluate the result on the BindingTuple context. + * @param tuple BindingTuple + * @return ExprValue + */ + ExprValue valueOf(BindingTuple tuple); +} \ No newline at end of file diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/core/ExpressionFactory.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/core/ExpressionFactory.java new file mode 100644 index 0000000000..4b460b6c68 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/core/ExpressionFactory.java @@ -0,0 +1,143 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.expression.core; + + +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValueFactory; +import lombok.RequiredArgsConstructor; + +import java.util.function.BiFunction; + +import static com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValueFactory.doubleValue; + +/** + * The definition of Expression factory. + */ +public class ExpressionFactory { + /** + * Reference + */ + public static Expression ref(String bindingName) { + return new Expression() { + @Override + public String toString() { + return String.format("%s", bindingName); + } + + @Override + public ExprValue valueOf(BindingTuple tuple) { + return tuple.resolve(bindingName); + } + }; + } + + @RequiredArgsConstructor + enum ArithmeticOperation { + ADD(Integer::sum, Double::sum), + SUB((arg1, arg2) -> arg1 - arg2, + (arg1, arg2) -> arg1 - arg2); + + private final BiFunction integerFunc; + private final BiFunction doubleFunc; + } + + public static Expression add(Expression left, Expression right) { + return new Expression() { + @Override + public ExprValue valueOf(BindingTuple tuple) { + return arithmetic(ArithmeticOperation.ADD, left.valueOf(tuple), right.valueOf(tuple)); + } + + @Override + public String toString() { + return String.format("add(%s,%s)", left, right); + } + }; + } + + public static Expression sub(Expression left, Expression right) { + return new Expression() { + @Override + public ExprValue valueOf(BindingTuple tuple) { + return arithmetic(ArithmeticOperation.ADD, left.valueOf(tuple), right.valueOf(tuple)); + } + + @Override + public String toString() { + return String.format("sub(%s,%s)", left, right); + } + }; + } + + public static Expression log(Expression expr) { + return new Expression() { + @Override + public ExprValue valueOf(BindingTuple tuple) { + final ExprValue exprValue = expr.valueOf(tuple); + switch (exprValue.kind()) { + case INTEGER_VALUE: + return doubleValue(Math.log(exprValue.numberValue().intValue())); + case DOUBLE_VALUE: + return doubleValue(Math.log(exprValue.numberValue().doubleValue())); + default: + throw new RuntimeException("unsupported log operand: " + exprValue.kind()); + } + } + + @Override + public String toString() { + return String.format("log(%s)", expr); + } + }; + } + + public static Expression cast(Expression expr) { + return new Expression() { + @Override + public ExprValue valueOf(BindingTuple tuple) { + return expr.valueOf(tuple); + } + + @Override + public String toString() { + return String.format("cast(%s)", expr); + } + }; + } + + private static ExprValue arithmetic(ArithmeticOperation op, ExprValue v1, ExprValue v2) { + if (v1.kind() != v2.kind()) { + throw new RuntimeException( + String.format("operation with different type is unsupported: %s(%s, %s)", op.name(), v1.kind(), + v2.kind())); + } else { + switch (v1.kind()) { + case DOUBLE_VALUE: + return ExprValueFactory.doubleValue( + op.doubleFunc.apply(v1.numberValue().doubleValue(), v2.numberValue().doubleValue())); + case INTEGER_VALUE: + return ExprValueFactory + .integerValue( + op.integerFunc.apply(v1.numberValue().intValue(), v2.numberValue().intValue())); + default: + throw new RuntimeException(String.format("unsupported operation: %s(%s, %s)", op.name(), v1.kind(), + v2.kind())); + } + } + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/domain/BindingTuple.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/domain/BindingTuple.java new file mode 100644 index 0000000000..0589e98973 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/domain/BindingTuple.java @@ -0,0 +1,83 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.expression.domain; + + +import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprMissingValue; +import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValue; +import lombok.Builder; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.Singular; +import org.json.JSONObject; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValueFactory.fromJson; + +/** + * BindingTuple represents the a relationship between bindingName and ExprValue. + * e.g. The operation output column name is bindingName, the value is the ExprValue. + */ +@Builder +@Getter +@EqualsAndHashCode +public class BindingTuple { + @Singular("binding") + private Map bindingMap; + + /** + * Resolve the Binding Name in BindingTuple context. + * @param bindingName binding name. + * @return binding value. + */ + public ExprValue resolve(String bindingName) { + return bindingMap.getOrDefault(bindingName, new ExprMissingValue()); + } + + @Override + public String toString() { + final StringBuffer sb = new StringBuffer("<"); + final List list = bindingMap.entrySet() + .stream() + .map(entry -> String.format("%s:%s", entry.getKey(), entry.getValue())) + .collect(Collectors.toList()); + sb.append(String.join(",", list)); + sb.append('>'); + return sb.toString(); + } + + public static BindingTuple from(Map map) { + Map ssValueMap = new HashMap<>(); + for (Map.Entry entry : map.entrySet()) { + ssValueMap.put(entry.getKey(), fromJson(entry.getValue())); + } + return BindingTuple.builder() + .bindingMap(ssValueMap) + .build(); + } + + public static BindingTuple from(JSONObject json) { + Map valueMap = new HashMap<>(); + for (String s : json.keySet()) { + valueMap.put(s, fromJson(json.get(s))); + } + return new BindingTuple(valueMap); + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprBooleanValue.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprBooleanValue.java new file mode 100644 index 0000000000..c4309a8019 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprBooleanValue.java @@ -0,0 +1,48 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.expression.model; + +import lombok.EqualsAndHashCode; +import lombok.RequiredArgsConstructor; + +@EqualsAndHashCode +@RequiredArgsConstructor +public class ExprBooleanValue implements ExprValue { + private final Boolean value; + + @Override + public Object value() { + return value; + } + + @Override + public Boolean booleanValue() { + return value; + } + + @Override + public ExprValueKind kind() { + return ExprValueKind.BOOLEAN_VALUE; + } + + @Override + public String toString() { + final StringBuffer sb = new StringBuffer("SSBooleanValue{"); + sb.append("value=").append(value); + sb.append('}'); + return sb.toString(); + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprCollectionValue.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprCollectionValue.java new file mode 100644 index 0000000000..7db1e69caa --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprCollectionValue.java @@ -0,0 +1,53 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.expression.model; + +import lombok.EqualsAndHashCode; +import lombok.RequiredArgsConstructor; + +import java.util.List; +import java.util.stream.Collectors; + +import static com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValue.ExprValueKind.COLLECTION_VALUE; + +@EqualsAndHashCode +@RequiredArgsConstructor +public class ExprCollectionValue implements ExprValue { + private final List valueList; + + @Override + public Object value() { + return valueList; + } + + @Override + public List collectionValue() { + return valueList; + } + + @Override + public ExprValueKind kind() { + return COLLECTION_VALUE; + } + + @Override + public String toString() { + final StringBuffer sb = new StringBuffer("["); + sb.append(String.join(",", valueList.stream().map(tuple -> tuple.toString()).collect(Collectors.toList()))); + sb.append("]"); + return sb.toString(); + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprDoubleValue.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprDoubleValue.java new file mode 100644 index 0000000000..05278823a1 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprDoubleValue.java @@ -0,0 +1,45 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.expression.model; + +import lombok.EqualsAndHashCode; +import lombok.RequiredArgsConstructor; + +@EqualsAndHashCode +@RequiredArgsConstructor +public class ExprDoubleValue implements ExprValue { + private final Double value; + + @Override + public Object value() { + return value; + } + + @Override + public Number numberValue() { + return value; + } + + @Override + public ExprValueKind kind() { + return ExprValueKind.DOUBLE_VALUE; + } + + @Override + public String toString() { + return value.toString(); + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprIntegerValue.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprIntegerValue.java new file mode 100644 index 0000000000..5d6b9407c1 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprIntegerValue.java @@ -0,0 +1,45 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.expression.model; + +import lombok.EqualsAndHashCode; +import lombok.RequiredArgsConstructor; + +@EqualsAndHashCode +@RequiredArgsConstructor +public class ExprIntegerValue implements ExprValue { + private final Integer value; + + @Override + public Object value() { + return value; + } + + @Override + public Number numberValue() { + return value; + } + + @Override + public ExprValueKind kind() { + return ExprValueKind.INTEGER_VALUE; + } + + @Override + public String toString() { + return value.toString(); + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprMissingValue.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprMissingValue.java new file mode 100644 index 0000000000..379a22168a --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprMissingValue.java @@ -0,0 +1,26 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.expression.model; + +/** + * The definition of the missing value. + */ +public class ExprMissingValue implements ExprValue { + @Override + public ExprValueKind kind() { + return ExprValueKind.MISSING_VALUE; + } +} \ No newline at end of file diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprStringValue.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprStringValue.java new file mode 100644 index 0000000000..7744515872 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprStringValue.java @@ -0,0 +1,45 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.expression.model; + +import lombok.EqualsAndHashCode; +import lombok.RequiredArgsConstructor; + +@EqualsAndHashCode +@RequiredArgsConstructor +public class ExprStringValue implements ExprValue { + private final String value; + + @Override + public Object value() { + return value; + } + + @Override + public String stringValue() { + return value; + } + + @Override + public ExprValueKind kind() { + return ExprValueKind.STRING_VALUE; + } + + @Override + public String toString() { + return value; + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprTupleValue.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprTupleValue.java new file mode 100644 index 0000000000..34ff2b1d7c --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprTupleValue.java @@ -0,0 +1,56 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.expression.model; + +import lombok.EqualsAndHashCode; +import lombok.RequiredArgsConstructor; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +@EqualsAndHashCode +@RequiredArgsConstructor +public class ExprTupleValue implements ExprValue { + private final Map valueMap; + + @Override + public Object value() { + return valueMap; + } + + @Override + public Map tupleValue() { + return valueMap; + } + + @Override + public ExprValueKind kind() { + return ExprValueKind.TUPLE_VALUE; + } + + @Override + public String toString() { + final StringBuffer sb = new StringBuffer("{"); + final List list = valueMap.entrySet() + .stream() + .map(entry -> String.format("%s:%s", entry.getKey(), entry.getValue())) + .collect(Collectors.toList()); + sb.append(String.join(",", list)); + sb.append('}'); + return sb.toString(); + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprValue.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprValue.java new file mode 100644 index 0000000000..018eaa6761 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprValue.java @@ -0,0 +1,63 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.expression.model; + +import java.util.List; +import java.util.Map; + +/** + * The definition of the Value used in the Expression + */ +public interface ExprValue { + default Object value() { + throw new IllegalStateException("invalid value operation on " + kind()); + } + + default List collectionValue() { + throw new IllegalStateException("invalid collectionValue operation on " + kind()); + } + + default Map tupleValue() { + throw new IllegalStateException("invalid tupleValue on " + kind()); + } + + default Number numberValue() { + throw new IllegalStateException("invalid numberValue operation on " + kind()); + } + + default Boolean booleanValue() { + throw new IllegalStateException("invalid booleanValue operation on " + kind()); + } + + default String stringValue() { + throw new IllegalStateException("invalid stringValue operation on " + kind()); + } + + default ExprValueKind kind() { + throw new IllegalStateException("invalid kind operation"); + } + + enum ExprValueKind { + TUPLE_VALUE, + COLLECTION_VALUE, + MISSING_VALUE, + + BOOLEAN_VALUE, + INTEGER_VALUE, + DOUBLE_VALUE, + STRING_VALUE + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprValueFactory.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprValueFactory.java new file mode 100644 index 0000000000..b1d6ba01d0 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprValueFactory.java @@ -0,0 +1,82 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.expression.model; + +import org.json.JSONArray; +import org.json.JSONObject; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * The definition of {@link ExprValue} factory. + */ +public class ExprValueFactory { + + public static ExprValue booleanValue(Boolean value) { + return new ExprBooleanValue(value); + } + + public static ExprValue integerValue(Integer value) { + return new ExprIntegerValue(value); + } + + public static ExprValue doubleValue(Double value) { + return new ExprDoubleValue(value); + } + + public static ExprValue stringValue(String value) { + return new ExprStringValue(value); + } + + public static ExprValue tupleValue(JSONObject jsonObject) { + Map valueMap = new HashMap<>(); + for (String s : jsonObject.keySet()) { + valueMap.put(s, fromJson(jsonObject.get(s))); + } + return new ExprTupleValue(valueMap); + } + + public static ExprValue collectionValue(JSONArray array) { + List valueList = new ArrayList<>(); + for (Object o : array) { + valueList.add(fromJson(o)); + } + return new ExprCollectionValue(valueList); + } + + public static ExprValue fromJson(Object o) { + if (o instanceof JSONObject) { + return tupleValue((JSONObject) o); + } else if (o instanceof JSONArray) { + return collectionValue(((JSONArray) o)); + } else if (o instanceof Integer) { + return integerValue((Integer) o); + } else if (o instanceof Long) { + return integerValue(((Long) o).intValue()); + } else if (o instanceof Boolean) { + return booleanValue((Boolean) o); + } else if (o instanceof Double) { + return doubleValue((Double) o); + } else if (o instanceof String) { + return stringValue((String) o); + } else { + throw new IllegalStateException("unsupported type " + o.getClass()); + } + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLAggregationParser.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLAggregationParser.java index 24fc45ff00..8983f79507 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLAggregationParser.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/converter/SQLAggregationParser.java @@ -26,6 +26,7 @@ import com.alibaba.druid.sql.dialect.mysql.visitor.MySqlASTVisitorAdapter; import com.amazon.opendistroforelasticsearch.sql.domain.ColumnTypeProvider; import com.amazon.opendistroforelasticsearch.sql.expression.core.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory; import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColumnNode; import com.google.common.base.Strings; import lombok.Getter; @@ -38,8 +39,6 @@ import java.util.Optional; import java.util.stream.Collectors; -import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.var; - /** * The definition of SQL Aggregation Converter which will parse the query to project column node list and * aggregation list @@ -214,7 +213,7 @@ public GroupKeyExpr(SQLExpr expr) { && selectSQLExprAliasMap.values().contains(((SQLIdentifierExpr) expr).getName())) { exprName = ((SQLIdentifierExpr) expr).getName(); } - this.expression = var(selectSQLExprAliasMap.getOrDefault(expr, exprName)); + this.expression = ExpressionFactory.ref(selectSQLExprAliasMap.getOrDefault(expr, exprName)); } } @@ -226,7 +225,8 @@ public class AggregationExpr { public AggregationExpr(SQLAggregateExpr expr) { this.expr = expr; this.expression = - var(selectSQLExprAliasMap.getOrDefault(expr, aliasGenerator.nextAlias(expr.getMethodName()))); + ExpressionFactory.ref(selectSQLExprAliasMap.getOrDefault(expr, aliasGenerator + .nextAlias(expr.getMethodName()))); } } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/project/PhysicalProject.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/project/PhysicalProject.java index 3ce2049872..454ada2d4a 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/project/PhysicalProject.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/project/PhysicalProject.java @@ -16,7 +16,7 @@ package com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.project; import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; -import com.amazon.opendistroforelasticsearch.sql.expression.model.SSValue; +import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValue; import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll.BindingTupleRow; import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColumnNode; import com.amazon.opendistroforelasticsearch.sql.query.planner.core.PlanNode; @@ -55,10 +55,10 @@ public boolean hasNext() { @Override public Row next() { BindingTuple input = next.next().data(); - Map output = new HashMap<>(); + Map output = new HashMap<>(); for (ColumnNode field : fields) { - SSValue ssValue = field.getExpr().valueOf(input); - output.put(field.getName(), ssValue); + ExprValue exprValue = field.getExpr().valueOf(input); + output.put(field.getName(), exprValue); } return new BindingTupleRow(BindingTuple.builder().bindingMap(output).build()); } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/PhysicalScroll.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/PhysicalScroll.java index b4ee975ba1..20e1fa492d 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/PhysicalScroll.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/PhysicalScroll.java @@ -17,7 +17,7 @@ import com.amazon.opendistroforelasticsearch.sql.exception.SqlParseException; import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; -import com.amazon.opendistroforelasticsearch.sql.expression.model.SSValue; +import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValue; import com.amazon.opendistroforelasticsearch.sql.query.AggregationQueryAction; import com.amazon.opendistroforelasticsearch.sql.query.QueryAction; import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ExecuteParams; @@ -37,7 +37,7 @@ import java.util.Map; import java.util.stream.Collectors; -import static com.amazon.opendistroforelasticsearch.sql.expression.model.SSValueFactory.fromJson; +import static com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValueFactory.fromJson; import static com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll.SearchAggregationResponseHelper.flatten; /** @@ -86,7 +86,7 @@ public void open(ExecuteParams params) { private void populateSearchAggregationResponse(Aggregations aggs) { List> flatten = flatten(aggs); List bindingTupleList = flatten.stream().map(map -> { - Map ssValueMap = new HashMap<>(); + Map ssValueMap = new HashMap<>(); for (Map.Entry entry : map.entrySet()) { ssValueMap.put(entry.getKey(), fromJson(entry.getValue())); } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/SearchAggregationResponseHelper.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/SearchAggregationResponseHelper.java index c1951bdfbd..1d3bb4bdcf 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/SearchAggregationResponseHelper.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/SearchAggregationResponseHelper.java @@ -15,7 +15,7 @@ package com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll; -import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTupleFactory; +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; import com.google.common.annotations.VisibleForTesting; import org.elasticsearch.search.aggregations.Aggregation; import org.elasticsearch.search.aggregations.Aggregations; @@ -39,7 +39,7 @@ public class SearchAggregationResponseHelper { public static List populateSearchAggregationResponse(Aggregations aggs) { List> flatten = flatten(aggs); List bindingTupleList = flatten.stream() - .map(BindingTupleFactory::from) + .map(BindingTuple::from) .map(bindingTuple -> new BindingTupleRow(bindingTuple)) .collect(Collectors.toList()); return bindingTupleList; diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/BindingTupleResultSetTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/BindingTupleResultSetTest.java index 6700050238..cf51d5cd44 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/BindingTupleResultSetTest.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/BindingTupleResultSetTest.java @@ -18,7 +18,6 @@ import com.amazon.opendistroforelasticsearch.sql.executor.format.BindingTupleResultSet; import com.amazon.opendistroforelasticsearch.sql.executor.format.DataRows; import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; -import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTupleFactory; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.hamcrest.Matcher; @@ -38,10 +37,10 @@ public class BindingTupleResultSetTest { @Test public void buildDataRowsFromBindingTupleShouldPass() { - assertThat(row(Arrays.asList(BindingTupleFactory.from(ImmutableMap.of("age", 31, "gender", "m")), - BindingTupleFactory.from(ImmutableMap.of("age", 31, "gender", "f")), - BindingTupleFactory.from(ImmutableMap.of("age", 39, "gender", "m")), - BindingTupleFactory.from(ImmutableMap.of("age", 39, "gender", "f")))), + assertThat(row(Arrays.asList(BindingTuple.from(ImmutableMap.of("age", 31, "gender", "m")), + BindingTuple.from(ImmutableMap.of("age", 31, "gender", "f")), + BindingTuple.from(ImmutableMap.of("age", 39, "gender", "m")), + BindingTuple.from(ImmutableMap.of("age", 39, "gender", "f")))), containsInAnyOrder(rowContents(allOf(hasEntry("age", 31), hasEntry("gender", (Object) "m"))), rowContents(allOf(hasEntry("age", 31), hasEntry("gender", (Object) "f"))), rowContents(allOf(hasEntry("age", 39), hasEntry("gender", (Object) "m"))), diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/CSVResultsExtractorTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/CSVResultsExtractorTest.java index adcc3b7164..2f46945453 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/CSVResultsExtractorTest.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/executor/format/CSVResultsExtractorTest.java @@ -19,7 +19,6 @@ import com.amazon.opendistroforelasticsearch.sql.executor.csv.CSVResultsExtractor; import com.amazon.opendistroforelasticsearch.sql.executor.csv.CsvExtractorException; import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; -import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTupleFactory; import com.google.common.collect.ImmutableMap; import org.junit.Test; @@ -34,10 +33,10 @@ public class CSVResultsExtractorTest { @Test public void extractResultsFromBindingTupleListShouldPass() throws CsvExtractorException { - CSVResult csvResult = csv(Arrays.asList(BindingTupleFactory.from(ImmutableMap.of("age", 31, "gender", "m")), - BindingTupleFactory.from(ImmutableMap.of("age", 31, "gender", "f")), - BindingTupleFactory.from(ImmutableMap.of("age", 39, "gender", "m")), - BindingTupleFactory.from(ImmutableMap.of("age", 39, "gender", "f"))), + CSVResult csvResult = csv(Arrays.asList(BindingTuple.from(ImmutableMap.of("age", 31, "gender", "m")), + BindingTuple.from(ImmutableMap.of("age", 31, "gender", "f")), + BindingTuple.from(ImmutableMap.of("age", 39, "gender", "m")), + BindingTuple.from(ImmutableMap.of("age", 39, "gender", "f"))), Arrays.asList("age", "gender")); assertThat(csvResult.getHeaders(), contains("age", "gender")); diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/core/ArithmeticExpressionTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/core/ArithmeticExpressionTest.java new file mode 100644 index 0000000000..29e68af377 --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/core/ArithmeticExpressionTest.java @@ -0,0 +1,56 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.unittest.expression.core; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.runners.MockitoJUnitRunner; + +import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.add; +import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.ref; +import static org.junit.Assert.assertEquals; + +@RunWith(MockitoJUnitRunner.class) +public class ArithmeticExpressionTest extends ExpressionTest { + @Rule + public ExpectedException exceptionRule = ExpectedException.none(); + + @Test + public void addIntegerValueShouldPass() { + assertEquals(2, add(ref("intValue"), ref("intValue")).valueOf(bindingTuple()).numberValue()); + } + + @Test + public void multipleAddIntegerValueShouldPass() { + assertEquals(3, + add(ref("intValue"), add(ref("intValue"), ref("intValue"))).valueOf(bindingTuple()).numberValue()); + } + + @Test + public void addDoubleValueShouldPass() { + assertEquals(4d, add(ref("doubleValue"), ref("doubleValue")).valueOf(bindingTuple()).numberValue()); + } + + @Test + public void addDoubleAndIntegerShouldThrowException() { + exceptionRule.expect(RuntimeException.class); + exceptionRule.expectMessage("operation with different type is unsupported: ADD(DOUBLE_VALUE, INTEGER_VALUE)"); + + add(ref("doubleValue"), ref("intValue")).valueOf(bindingTuple()); + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/core/ExpressionTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/core/ExpressionTest.java new file mode 100644 index 0000000000..d36de0766b --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/core/ExpressionTest.java @@ -0,0 +1,41 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.unittest.expression.core; + +import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; +import org.json.JSONObject; + +public class ExpressionTest { + protected BindingTuple bindingTuple() { + String json = "{\n" + + " \"intValue\": 1,\n" + + " \"doubleValue\": 2.0,\n" + + " \"stringValue\": \"string\",\n" + + " \"booleanValue\": true,\n" + + " \"tupleValue\": {\n" + + " \"intValue\": 1,\n" + + " \"doubleValue\": 2.0,\n" + + " \"stringValue\": \"string\"\n" + + " },\n" + + " \"collectValue\": [\n" + + " 1,\n" + + " 2,\n" + + " 3\n" + + " ]\n" + + "}"; + return BindingTuple.from(new JSONObject(json)); + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/core/RefExpressionTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/core/RefExpressionTest.java new file mode 100644 index 0000000000..86fc54c78b --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/core/RefExpressionTest.java @@ -0,0 +1,63 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.unittest.expression.core; + +import org.junit.Test; + +import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.ref; +import static com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValueFactory.doubleValue; +import static com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValueFactory.integerValue; +import static com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValueFactory.stringValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.hasEntry; +import static org.junit.Assert.assertEquals; + +public class RefExpressionTest extends ExpressionTest { + @Test + public void refIntegerValueShouldPass() { + assertEquals(1, ref("intValue").valueOf(bindingTuple()).numberValue()); + } + + @Test + public void refDoubleValueShouldPass() { + assertEquals(2.0d, ref("doubleValue").valueOf(bindingTuple()).numberValue()); + } + + @Test + public void refStringValueShouldPass() { + assertEquals("string", ref("stringValue").valueOf(bindingTuple()).stringValue()); + } + + @Test + public void refBooleanValueShouldPass() { + assertEquals(true, ref("booleanValue").valueOf(bindingTuple()).booleanValue()); + } + + @Test + public void refTupleValueShouldPass() { + assertThat(ref("tupleValue").valueOf(bindingTuple()).tupleValue(), + allOf(hasEntry("intValue", integerValue(1)), hasEntry("doubleValue", doubleValue(2d)), + hasEntry("stringValue", stringValue("string")))); + } + + @Test + public void refCollectValueShouldPass() { + assertThat(ref("collectValue").valueOf(bindingTuple()).collectionValue(), + contains(integerValue(1), integerValue(2), integerValue(3))); + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/model/ExprValueTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/model/ExprValueTest.java new file mode 100644 index 0000000000..7f317b041b --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/expression/model/ExprValueTest.java @@ -0,0 +1,47 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.sql.unittest.expression.model; + +import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprIntegerValue; +import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValue; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.runners.MockitoJUnitRunner; + +import static org.junit.Assert.assertEquals; + +@RunWith(MockitoJUnitRunner.class) +public class ExprValueTest { + @Rule + public ExpectedException exceptionRule = ExpectedException.none(); + + @Test + public void interValueShouldReturnCorrectValue() { + ExprValue value = new ExprIntegerValue(1); + assertEquals(1, value.numberValue()); + } + + @Test + public void stringValueOnInterValueShouldThrowException() { + exceptionRule.expect(IllegalStateException.class); + exceptionRule.expectMessage("invalid stringValue operation on INTEGER_VALUE"); + + ExprValue value = new ExprIntegerValue(1); + value.stringValue(); + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/BindingTupleQueryPlannerExecuteTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/BindingTupleQueryPlannerExecuteTest.java index 0613cb0023..478f41d877 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/BindingTupleQueryPlannerExecuteTest.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/BindingTupleQueryPlannerExecuteTest.java @@ -17,7 +17,6 @@ import com.amazon.opendistroforelasticsearch.sql.domain.ColumnTypeProvider; import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; -import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTupleFactory; import com.amazon.opendistroforelasticsearch.sql.query.planner.core.BindingTupleQueryPlanner; import com.amazon.opendistroforelasticsearch.sql.util.AggregationUtils; import com.amazon.opendistroforelasticsearch.sql.util.SqlParserUtils; @@ -67,8 +66,8 @@ public void testAggregationShouldPass() { assertThat(query("SELECT gender, MAX(age) as max, MIN(age) as min FROM accounts GROUP BY gender", mockSearchAggregation()), containsInAnyOrder( - BindingTupleFactory.from(ImmutableMap.of("gender", "m", "max", 20d, "min", 10d)), - BindingTupleFactory.from(ImmutableMap.of("gender", "f", "max", 40d, "min", 20d)))); + BindingTuple.from(ImmutableMap.of("gender", "m", "max", 20d, "min", 10d)), + BindingTuple.from(ImmutableMap.of("gender", "f", "max", 40d, "min", 20d)))); } diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLAggregationParserTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLAggregationParserTest.java index 7a38d092c0..9679ff4e5a 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLAggregationParserTest.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLAggregationParserTest.java @@ -23,6 +23,7 @@ import com.alibaba.druid.util.JdbcConstants; import com.amazon.opendistroforelasticsearch.sql.domain.ColumnTypeProvider; import com.amazon.opendistroforelasticsearch.sql.expression.core.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory; import com.amazon.opendistroforelasticsearch.sql.query.planner.converter.SQLAggregationParser; import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColumnNode; import org.hamcrest.Description; @@ -38,7 +39,7 @@ import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.add; import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.cast; import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.log; -import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.var; +import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.ref; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -61,9 +62,10 @@ public void parseAggWithoutExpressionShouldPass() { agg("MAX", "FlightDelayMin", "MAX_0"), agg("MIN", "FlightDelayMin", "min"))); - assertThat(columnNodes, containsInAnyOrder(columnNode("dayOfWeek", null, var("dayOfWeek")), - columnNode("MAX(FlightDelayMin)", null, var("MAX_0")), - columnNode("min", "min", var("min")))); + assertThat(columnNodes, containsInAnyOrder(columnNode("dayOfWeek", null, ExpressionFactory.ref("dayOfWeek")), + columnNode("MAX(FlightDelayMin)", null, ExpressionFactory + .ref("MAX_0")), + columnNode("min", "min", ExpressionFactory.ref("min")))); } @Test @@ -80,9 +82,10 @@ public void parseAggWithFunctioniWithoutExpressionShouldPass() { agg("MAX", "FlightDelayMin", "MAX_0"), agg("MIN", "FlightDelayMin", "min"))); - assertThat(columnNodes, containsInAnyOrder(columnNode("dayOfWeek", null, var("dayOfWeek")), - columnNode("MAX(FlightDelayMin)", null, var("MAX_0")), - columnNode("min", "min", var("min")))); + assertThat(columnNodes, containsInAnyOrder(columnNode("dayOfWeek", null, ExpressionFactory.ref("dayOfWeek")), + columnNode("MAX(FlightDelayMin)", null, ExpressionFactory + .ref("MAX_0")), + columnNode("min", "min", ExpressionFactory.ref("min")))); } @Test @@ -99,8 +102,9 @@ public void parseAggWithExpressionShouldPass() { agg("MAX", "FlightDelayMin", "MAX_0"), agg("MIN", "FlightDelayMin", "MIN_1"))); - assertThat(columnNodes, containsInAnyOrder(columnNode("dayOfWeek", null, var("dayOfWeek")), - columnNode("sub", "sub", add(var("MAX_0"), var("MIN_1"))))); + assertThat(columnNodes, containsInAnyOrder(columnNode("dayOfWeek", null, ExpressionFactory.ref("dayOfWeek")), + columnNode("sub", "sub", add(ExpressionFactory.ref("MAX_0"), ExpressionFactory + .ref("MIN_1"))))); } @Test @@ -117,8 +121,9 @@ public void functionOverFiledShouldPass() { agg("MAX", "FlightDelayMin", "MAX_0"), agg("MIN", "FlightDelayMin", "MIN_1"))); - assertThat(columnNodes, containsInAnyOrder(columnNode("dayOfWeek", null, var("dayOfWeek")), - columnNode("sub", "sub", add(var("MAX_0"), var("MIN_1"))))); + assertThat(columnNodes, containsInAnyOrder(columnNode("dayOfWeek", null, ExpressionFactory.ref("dayOfWeek")), + columnNode("sub", "sub", add(ExpressionFactory.ref("MAX_0"), ExpressionFactory + .ref("MIN_1"))))); } @Test @@ -135,8 +140,10 @@ public void parseCompoundAggWithExpressionShouldPass() { agg("MAX", "FlightDelayMin", "MAX_0"), agg("MIN", "FlightDelayMin", "MIN_1"))); - assertThat(columnNodes, containsInAnyOrder(columnNode("ASCII(dayOfWeek)", null, var("ASCII(dayOfWeek)")), - columnNode("log", "log", log(add(var("MAX_0"), var("MIN_1")))))); + assertThat(columnNodes, containsInAnyOrder(columnNode("ASCII(dayOfWeek)", null, ExpressionFactory + .ref("ASCII(dayOfWeek)")), + columnNode("log", "log", log(add(ExpressionFactory.ref("MAX_0"), ExpressionFactory + .ref("MIN_1")))))); } @Test @@ -148,7 +155,8 @@ public void parseFunctionGroupColumnOverShouldPass() { List columnNodes = parser.getColumnNodes(); assertThat(sqlSelectItems, containsInAnyOrder(group("balance", "balance"))); - assertThat(columnNodes, containsInAnyOrder(columnNode("CAST(balance AS FLOAT)", null, cast(var("balance"))))); + assertThat(columnNodes, containsInAnyOrder(columnNode("CAST(balance AS FLOAT)", null, cast( + ExpressionFactory.ref("balance"))))); } @Test @@ -163,8 +171,8 @@ public void withoutAggregationShouldPass() { group("age", "age"), group("gender", "gender"))); assertThat(columnNodes, containsInAnyOrder( - columnNode("age", null, var("age")), - columnNode("gender", null, var("gender")))); + columnNode("age", null, ExpressionFactory.ref("age")), + columnNode("gender", null, ExpressionFactory.ref("gender")))); } @Test @@ -179,8 +187,8 @@ public void groupKeyInSelectWithFunctionShouldPass() { group("age", "age"), agg("max", "balance", "max_0"))); assertThat(columnNodes, containsInAnyOrder( - columnNode("log(age)", null, log(var("age"))), - columnNode("max(balance)", null, var("max_0")))); + columnNode("log(age)", null, log(ExpressionFactory.ref("age"))), + columnNode("max(balance)", null, ExpressionFactory.ref("max_0")))); } @Test @@ -195,8 +203,8 @@ public void theDotInFieldNameShouldBeReplaceWithSharp() { group("name.lastname", "name#lastname"), agg("max", "balance", "max_0"))); assertThat(columnNodes, containsInAnyOrder( - columnNode("name.lastname", null, var("name#lastname")), - columnNode("max(balance)", null, var("max_0")))); + columnNode("name.lastname", null, ExpressionFactory.ref("name#lastname")), + columnNode("max(balance)", null, ExpressionFactory.ref("max_0")))); } @Test @@ -210,7 +218,7 @@ public void noGroupKeyInSelectShouldPass() { assertThat(sqlSelectItems, containsInAnyOrder( agg("avg", "age", "avg_0"))); assertThat(columnNodes, containsInAnyOrder( - columnNode("avg(age)", null, var("avg_0")))); + columnNode("avg(age)", null, ExpressionFactory.ref("avg_0")))); } /** diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLExprToExpressionConverterTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLExprToExpressionConverterTest.java index 62decdaa5b..4b6d1d694f 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLExprToExpressionConverterTest.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/converter/SQLExprToExpressionConverterTest.java @@ -21,6 +21,7 @@ import com.alibaba.druid.sql.ast.expr.SQLIdentifierExpr; import com.alibaba.druid.sql.ast.expr.SQLMethodInvokeExpr; import com.amazon.opendistroforelasticsearch.sql.expression.core.Expression; +import com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory; import com.amazon.opendistroforelasticsearch.sql.query.planner.converter.SQLAggregationParser; import com.amazon.opendistroforelasticsearch.sql.query.planner.converter.SQLExprToExpressionConverter; import com.google.common.collect.ImmutableMap; @@ -33,7 +34,6 @@ import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.add; import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.log; -import static com.amazon.opendistroforelasticsearch.sql.expression.core.ExpressionFactory.var; import static org.junit.Assert.assertEquals; @RunWith(MockitoJUnitRunner.class) @@ -64,7 +64,7 @@ public void identifierShouldReturnVarExpression() { context.addGroupKeyExpr(groupG); Expression expression = converter.convert(groupG); - assertEquals(var("A").toString(), expression.toString()); + assertEquals(ExpressionFactory.ref("A").toString(), expression.toString()); } @Test @@ -73,7 +73,7 @@ public void binaryOperatorAddShouldReturnAddExpression() { context.addAggregationExpr(minA); Expression expression = converter.convert(new SQLBinaryOpExpr(maxA, SQLBinaryOperator.Add, minA)); - assertEquals(add(var("MAX_0"), var("MIN_1")).toString(), expression.toString()); + assertEquals(add(ExpressionFactory.ref("MAX_0"), ExpressionFactory.ref("MIN_1")).toString(), expression.toString()); } @Test @@ -84,7 +84,8 @@ public void compoundBinaryOperatorShouldReturnCorrectExpression() { Expression expression = converter.convert(new SQLBinaryOpExpr(maxA, SQLBinaryOperator.Add, new SQLBinaryOpExpr(maxA, SQLBinaryOperator.Add, minA))); - assertEquals(add(var("MAX_0"), add(var("MAX_0"), var("MIN_1"))).toString(), expression.toString()); + assertEquals(add(ExpressionFactory.ref("MAX_0"), add(ExpressionFactory.ref("MAX_0"), ExpressionFactory + .ref("MIN_1"))).toString(), expression.toString()); } @Test @@ -98,7 +99,8 @@ public void functionOverCompoundBinaryOperatorShouldReturnCorrectExpression() { minA))); Expression expression = converter.convert(methodInvokeExpr); - assertEquals(log(add(var("MAX_0"), add(var("MAX_0"), var("MIN_1")))).toString(), expression.toString()); + assertEquals(log(add(ExpressionFactory.ref("MAX_0"), add(ExpressionFactory.ref("MAX_0"), ExpressionFactory + .ref("MIN_1")))).toString(), expression.toString()); } @Test @@ -112,7 +114,8 @@ public void functionOverGroupColumn() { minA))); Expression expression = converter.convert(methodInvokeExpr); - assertEquals(log(add(var("MAX_0"), add(var("MAX_0"), var("MIN_1")))).toString(), expression.toString()); + assertEquals(log(add(ExpressionFactory.ref("MAX_0"), add(ExpressionFactory.ref("MAX_0"), ExpressionFactory + .ref("MIN_1")))).toString(), expression.toString()); } @Test diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/physical/SearchAggregationResponseHelperTest.java b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/physical/SearchAggregationResponseHelperTest.java index 3cf5a2aa1d..6f9348b1ac 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/physical/SearchAggregationResponseHelperTest.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/sql/unittest/planner/physical/SearchAggregationResponseHelperTest.java @@ -16,14 +16,11 @@ package com.amazon.opendistroforelasticsearch.sql.unittest.planner.physical; import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; -import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTupleFactory; import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll.BindingTupleRow; import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll.SearchAggregationResponseHelper; import com.amazon.opendistroforelasticsearch.sql.util.AggregationUtils; import com.google.common.collect.ImmutableMap; -import org.hamcrest.Description; import org.hamcrest.Matcher; -import org.hamcrest.TypeSafeMatcher; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.runners.MockitoJUnitRunner; @@ -334,10 +331,10 @@ public void populateSearchAggregationResponeShouldPass() { SearchAggregationResponseHelper.populateSearchAggregationResponse(AggregationUtils.fromJson(json)); assertEquals(4, bindingTupleRows.size()); assertThat(bindingTupleRows, containsInAnyOrder( - bindingTupleRow(BindingTupleFactory.from(ImmutableMap.of("age", 31L, "gender", "m"))), - bindingTupleRow(BindingTupleFactory.from(ImmutableMap.of("age", 31L, "gender", "f"))), - bindingTupleRow(BindingTupleFactory.from(ImmutableMap.of("age", 39L, "gender", "m"))), - bindingTupleRow(BindingTupleFactory.from(ImmutableMap.of("age", 39L, "gender", "f"))))); + bindingTupleRow(BindingTuple.from(ImmutableMap.of("age", 31L, "gender", "m"))), + bindingTupleRow(BindingTuple.from(ImmutableMap.of("age", 31L, "gender", "f"))), + bindingTupleRow(BindingTuple.from(ImmutableMap.of("age", 39L, "gender", "m"))), + bindingTupleRow(BindingTuple.from(ImmutableMap.of("age", 39L, "gender", "f"))))); } private static Matcher bindingTupleRow(BindingTuple bindingTuple) { From 58c195ee1b5c3d3234a6076448dad9067632155c Mon Sep 17 00:00:00 2001 From: penghuo Date: Fri, 17 Jan 2020 17:14:18 -0800 Subject: [PATCH 5/5] update --- .../sql/expression/domain/BindingTuple.java | 33 ++++++------------- .../expression/model/ExprCollectionValue.java | 7 ++-- .../sql/expression/model/ExprTupleValue.java | 13 +++----- .../expression/model/ExprValueFactory.java | 25 +++++--------- .../node/project/PhysicalProject.java | 14 +++----- .../physical/node/scroll/PhysicalScroll.java | 25 ++------------ 6 files changed, 33 insertions(+), 84 deletions(-) diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/domain/BindingTuple.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/domain/BindingTuple.java index 0589e98973..6e4435c3ee 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/domain/BindingTuple.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/domain/BindingTuple.java @@ -18,19 +18,16 @@ import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprMissingValue; import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValue; +import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValueFactory; import lombok.Builder; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.Singular; import org.json.JSONObject; -import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import static com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValueFactory.fromJson; - /** * BindingTuple represents the a relationship between bindingName and ExprValue. * e.g. The operation output column name is bindingName, the value is the ExprValue. @@ -40,10 +37,11 @@ @EqualsAndHashCode public class BindingTuple { @Singular("binding") - private Map bindingMap; + private final Map bindingMap; /** * Resolve the Binding Name in BindingTuple context. + * * @param bindingName binding name. * @return binding value. */ @@ -53,31 +51,20 @@ public ExprValue resolve(String bindingName) { @Override public String toString() { - final StringBuffer sb = new StringBuffer("<"); - final List list = bindingMap.entrySet() + return bindingMap.entrySet() .stream() .map(entry -> String.format("%s:%s", entry.getKey(), entry.getValue())) - .collect(Collectors.toList()); - sb.append(String.join(",", list)); - sb.append('>'); - return sb.toString(); + .collect(Collectors.joining(",", "<", ">")); } public static BindingTuple from(Map map) { - Map ssValueMap = new HashMap<>(); - for (Map.Entry entry : map.entrySet()) { - ssValueMap.put(entry.getKey(), fromJson(entry.getValue())); - } - return BindingTuple.builder() - .bindingMap(ssValueMap) - .build(); + return from(new JSONObject(map)); } public static BindingTuple from(JSONObject json) { - Map valueMap = new HashMap<>(); - for (String s : json.keySet()) { - valueMap.put(s, fromJson(json.get(s))); - } - return new BindingTuple(valueMap); + Map map = json.toMap(); + BindingTupleBuilder bindingTupleBuilder = BindingTuple.builder(); + map.forEach((key, value) -> bindingTupleBuilder.binding(key, ExprValueFactory.from(value))); + return bindingTupleBuilder.build(); } } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprCollectionValue.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprCollectionValue.java index 7db1e69caa..7335ff31c5 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprCollectionValue.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprCollectionValue.java @@ -45,9 +45,8 @@ public ExprValueKind kind() { @Override public String toString() { - final StringBuffer sb = new StringBuffer("["); - sb.append(String.join(",", valueList.stream().map(tuple -> tuple.toString()).collect(Collectors.toList()))); - sb.append("]"); - return sb.toString(); + return valueList.stream() + .map(Object::toString) + .collect(Collectors.joining(",", "[", "]")); } } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprTupleValue.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprTupleValue.java index 34ff2b1d7c..08b08e53a6 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprTupleValue.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprTupleValue.java @@ -18,7 +18,6 @@ import lombok.EqualsAndHashCode; import lombok.RequiredArgsConstructor; -import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -44,13 +43,9 @@ public ExprValueKind kind() { @Override public String toString() { - final StringBuffer sb = new StringBuffer("{"); - final List list = valueMap.entrySet() - .stream() - .map(entry -> String.format("%s:%s", entry.getKey(), entry.getValue())) - .collect(Collectors.toList()); - sb.append(String.join(",", list)); - sb.append('}'); - return sb.toString(); + return valueMap.entrySet() + .stream() + .map(entry -> String.format("%s:%s", entry.getKey(), entry.getValue())) + .collect(Collectors.joining(",", "{", "}")); } } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprValueFactory.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprValueFactory.java index b1d6ba01d0..7254e4f3bf 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprValueFactory.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/expression/model/ExprValueFactory.java @@ -15,9 +15,6 @@ package com.amazon.opendistroforelasticsearch.sql.expression.model; -import org.json.JSONArray; -import org.json.JSONObject; - import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -44,27 +41,23 @@ public static ExprValue stringValue(String value) { return new ExprStringValue(value); } - public static ExprValue tupleValue(JSONObject jsonObject) { + public static ExprValue tupleValue(Map map) { Map valueMap = new HashMap<>(); - for (String s : jsonObject.keySet()) { - valueMap.put(s, fromJson(jsonObject.get(s))); - } + map.forEach((k, v) -> valueMap.put(k, from(v))); return new ExprTupleValue(valueMap); } - public static ExprValue collectionValue(JSONArray array) { + public static ExprValue collectionValue(List list) { List valueList = new ArrayList<>(); - for (Object o : array) { - valueList.add(fromJson(o)); - } + list.forEach(o -> valueList.add(from(o))); return new ExprCollectionValue(valueList); } - public static ExprValue fromJson(Object o) { - if (o instanceof JSONObject) { - return tupleValue((JSONObject) o); - } else if (o instanceof JSONArray) { - return collectionValue(((JSONArray) o)); + public static ExprValue from(Object o) { + if (o instanceof Map) { + return tupleValue((Map) o); + } else if (o instanceof List) { + return collectionValue(((List) o)); } else if (o instanceof Integer) { return integerValue((Integer) o); } else if (o instanceof Long) { diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/project/PhysicalProject.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/project/PhysicalProject.java index 454ada2d4a..d4f79d5717 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/project/PhysicalProject.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/project/PhysicalProject.java @@ -16,18 +16,15 @@ package com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.project; import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; -import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValue; -import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll.BindingTupleRow; import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ColumnNode; import com.amazon.opendistroforelasticsearch.sql.query.planner.core.PlanNode; import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.PhysicalOperator; import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.Row; import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.estimation.Cost; +import com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll.BindingTupleRow; import lombok.RequiredArgsConstructor; -import java.util.HashMap; import java.util.List; -import java.util.Map; /** * The definition of Project Operator. @@ -55,11 +52,8 @@ public boolean hasNext() { @Override public Row next() { BindingTuple input = next.next().data(); - Map output = new HashMap<>(); - for (ColumnNode field : fields) { - ExprValue exprValue = field.getExpr().valueOf(input); - output.put(field.getName(), exprValue); - } - return new BindingTupleRow(BindingTuple.builder().bindingMap(output).build()); + BindingTuple.BindingTupleBuilder outputBindingTupleBuilder = BindingTuple.builder(); + fields.forEach(field -> outputBindingTupleBuilder.binding(field.getName(), field.getExpr().valueOf(input))); + return new BindingTupleRow(outputBindingTupleBuilder.build()); } } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/PhysicalScroll.java b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/PhysicalScroll.java index 20e1fa492d..2b63f68850 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/PhysicalScroll.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/sql/query/planner/physical/node/scroll/PhysicalScroll.java @@ -17,7 +17,6 @@ import com.amazon.opendistroforelasticsearch.sql.exception.SqlParseException; import com.amazon.opendistroforelasticsearch.sql.expression.domain.BindingTuple; -import com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValue; import com.amazon.opendistroforelasticsearch.sql.query.AggregationQueryAction; import com.amazon.opendistroforelasticsearch.sql.query.QueryAction; import com.amazon.opendistroforelasticsearch.sql.query.planner.core.ExecuteParams; @@ -29,16 +28,8 @@ import lombok.SneakyThrows; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.search.aggregations.Aggregations; -import java.util.HashMap; import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static com.amazon.opendistroforelasticsearch.sql.expression.model.ExprValueFactory.fromJson; -import static com.amazon.opendistroforelasticsearch.sql.query.planner.physical.node.scroll.SearchAggregationResponseHelper.flatten; /** * The definition of Scroll Operator. @@ -74,7 +65,9 @@ public void open(ExecuteParams params) { try { ActionResponse response = queryAction.explain().get(); if (queryAction instanceof AggregationQueryAction) { - populateSearchAggregationResponse(((SearchResponse) response).getAggregations()); + rowIterator = SearchAggregationResponseHelper + .populateSearchAggregationResponse(((SearchResponse) response).getAggregations()) + .iterator(); } else { throw new IllegalStateException("Not support QueryAction type: " + queryAction.getClass()); } @@ -83,18 +76,6 @@ public void open(ExecuteParams params) { } } - private void populateSearchAggregationResponse(Aggregations aggs) { - List> flatten = flatten(aggs); - List bindingTupleList = flatten.stream().map(map -> { - Map ssValueMap = new HashMap<>(); - for (Map.Entry entry : map.entrySet()) { - ssValueMap.put(entry.getKey(), fromJson(entry.getValue())); - } - return BindingTuple.builder().bindingMap(ssValueMap).build(); - }).map(bindingTuple -> new BindingTupleRow(bindingTuple)).collect(Collectors.toList()); - rowIterator = bindingTupleList.iterator(); - } - @SneakyThrows @Override public String toString() {