Skip to content
This repository has been archived by the owner on Aug 2, 2022. It is now read-only.

Commit

Permalink
Add new sql module (#527)
Browse files Browse the repository at this point in the history
* Add new SQL module with support for SELECT literal

* Add support for more data types

* Route request to new frontend

* Add more UT

* Fix doctest

* Add more UT

* Add more UT

* Fix checkstyle and jacoco

* Address PR

* Add more docs

* Use new syntax check exception

* Unregister new handler and add UT

* Support fetch size 0 which is default request by JDBC driver

* Address PR comments
  • Loading branch information
dai-chen authored Jun 26, 2020
1 parent 0fad590 commit 087da88
Show file tree
Hide file tree
Showing 50 changed files with 2,514 additions and 212 deletions.
1 change: 1 addition & 0 deletions config/checkstyle/suppressions.xml
Original file line number Diff line number Diff line change
Expand Up @@ -7,5 +7,6 @@

<suppress files="com[\\/]amazon[\\/]opendistroforelasticsearch[\\/]sql[\\/]legacy" checks=".*"/>
<suppress files="com[\\/]amazon[\\/]opendistroforelasticsearch[\\/]sql[\\/]ppl[\\/]antlr[\\/]parser" checks=".*"/>
<suppress files="com[\\/]amazon[\\/]opendistroforelasticsearch[\\/]sql[\\/]sql[\\/]antlr[\\/]parser" checks=".*"/>

</suppressions>
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import com.amazon.opendistroforelasticsearch.sql.ast.expression.Argument;
import com.amazon.opendistroforelasticsearch.sql.ast.expression.Field;
import com.amazon.opendistroforelasticsearch.sql.ast.expression.Let;
import com.amazon.opendistroforelasticsearch.sql.ast.expression.Literal;
import com.amazon.opendistroforelasticsearch.sql.ast.expression.Map;
import com.amazon.opendistroforelasticsearch.sql.ast.expression.UnresolvedExpression;
import com.amazon.opendistroforelasticsearch.sql.ast.tree.Aggregation;
Expand All @@ -31,10 +32,12 @@
import com.amazon.opendistroforelasticsearch.sql.ast.tree.Sort;
import com.amazon.opendistroforelasticsearch.sql.ast.tree.Sort.SortOption;
import com.amazon.opendistroforelasticsearch.sql.ast.tree.UnresolvedPlan;
import com.amazon.opendistroforelasticsearch.sql.ast.tree.Values;
import com.amazon.opendistroforelasticsearch.sql.data.model.ExprMissingValue;
import com.amazon.opendistroforelasticsearch.sql.exception.SemanticCheckException;
import com.amazon.opendistroforelasticsearch.sql.expression.DSL;
import com.amazon.opendistroforelasticsearch.sql.expression.Expression;
import com.amazon.opendistroforelasticsearch.sql.expression.LiteralExpression;
import com.amazon.opendistroforelasticsearch.sql.expression.ReferenceExpression;
import com.amazon.opendistroforelasticsearch.sql.expression.aggregation.Aggregator;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalAggregation;
Expand All @@ -47,12 +50,14 @@
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalRemove;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalRename;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalSort;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalValues;
import com.amazon.opendistroforelasticsearch.sql.storage.StorageEngine;
import com.amazon.opendistroforelasticsearch.sql.storage.Table;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableList.Builder;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;
import lombok.RequiredArgsConstructor;
Expand Down Expand Up @@ -145,18 +150,23 @@ public LogicalPlan visitAggregation(Aggregation node, AnalysisContext context) {
@Override
public LogicalPlan visitProject(Project node, AnalysisContext context) {
LogicalPlan child = node.getChild().get(0).accept(this, context);
List<ReferenceExpression> referenceExpressions =
node.getProjectList().stream()
.map(expr -> (ReferenceExpression) expressionAnalyzer.analyze(expr, context))
.collect(Collectors.toList());

if (node.hasArgument()) {
Argument argument = node.getArgExprList().get(0);
Boolean exclude = (Boolean) argument.getValue().getValue();
if (exclude) {
List<ReferenceExpression> referenceExpressions =
node.getProjectList().stream()
.map(expr -> (ReferenceExpression) expressionAnalyzer.analyze(expr, context))
.collect(Collectors.toList());
return new LogicalRemove(child, ImmutableSet.copyOf(referenceExpressions));
}
}
return new LogicalProject(child, referenceExpressions);

List<Expression> expressions = node.getProjectList().stream()
.map(expr -> expressionAnalyzer.analyze(expr, context))
.collect(Collectors.toList());
return new LogicalProject(child, expressions);
}

/**
Expand Down Expand Up @@ -222,4 +232,17 @@ public LogicalPlan visitDedupe(Dedupe node, AnalysisContext context) {
keepEmpty,
consecutive);
}

@Override
public LogicalPlan visitValues(Values node, AnalysisContext context) {
List<List<Literal>> values = node.getValues();
List<List<LiteralExpression>> valueExprs = new ArrayList<>();
for (List<Literal> value : values) {
valueExprs.add(value.stream()
.map(val -> (LiteralExpression) expressionAnalyzer.analyze(val, context))
.collect(Collectors.toList()));
}
return new LogicalValues(valueExprs);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
import com.amazon.opendistroforelasticsearch.sql.ast.tree.Relation;
import com.amazon.opendistroforelasticsearch.sql.ast.tree.Rename;
import com.amazon.opendistroforelasticsearch.sql.ast.tree.Sort;
import com.amazon.opendistroforelasticsearch.sql.ast.tree.Values;

/**
* AST nodes visitor Defines the traverse path.
Expand Down Expand Up @@ -173,4 +174,8 @@ public T visitSort(Sort node, C context) {
public T visitDedupe(Dedupe node, C context) {
return visitChildren(node, context);
}

public T visitValues(Values node, C context) {
return visitChildren(node, context);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,8 @@
import com.amazon.opendistroforelasticsearch.sql.ast.tree.Rename;
import com.amazon.opendistroforelasticsearch.sql.ast.tree.Sort;
import com.amazon.opendistroforelasticsearch.sql.ast.tree.UnresolvedPlan;
import com.amazon.opendistroforelasticsearch.sql.ast.tree.Values;
import com.google.common.collect.ImmutableList;
import java.util.Arrays;
import java.util.List;
import lombok.experimental.UtilityClass;
Expand Down Expand Up @@ -86,6 +88,16 @@ public static UnresolvedPlan rename(UnresolvedPlan input, Map... maps) {
return new Rename(Arrays.asList(maps), input);
}

/**
* Initialize Values node by rows of literals.
* @param values rows in which each row is a list of literal values
* @return Values node
*/
@SafeVarargs
public UnresolvedPlan values(List<Literal>... values) {
return new Values(Arrays.asList(values));
}

public static UnresolvedExpression qualifiedName(String... parts) {
return new QualifiedName(Arrays.asList(parts));
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* Copyright 2020 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.ast.tree;

import com.amazon.opendistroforelasticsearch.sql.ast.AbstractNodeVisitor;
import com.amazon.opendistroforelasticsearch.sql.ast.Node;
import com.amazon.opendistroforelasticsearch.sql.ast.expression.Literal;
import com.google.common.collect.ImmutableList;
import java.util.List;
import lombok.EqualsAndHashCode;
import lombok.Getter;
import lombok.RequiredArgsConstructor;
import lombok.ToString;

/**
* AST node class for a sequence of literal values.
*/
@ToString
@Getter
@EqualsAndHashCode(callSuper = false)
@RequiredArgsConstructor
public class Values extends UnresolvedPlan {

private final List<List<Literal>> values;

@Override
public UnresolvedPlan attach(UnresolvedPlan child) {
throw new UnsupportedOperationException("Values node is supposed to have no child node");
}

@Override
public <T, C> T accept(AbstractNodeVisitor<T, C> nodeVisitor, C context) {
return nodeVisitor.visitValues(this, context);
}

@Override
public List<? extends Node> getChild() {
return ImmutableList.of();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,14 @@ public static LiteralExpression literal(Double value) {
return new LiteralExpression(ExprValueUtils.doubleValue(value));
}

public static LiteralExpression literal(String value) {
return new LiteralExpression(ExprValueUtils.stringValue(value));
}

public static LiteralExpression literal(Boolean value) {
return new LiteralExpression(ExprValueUtils.booleanValue(value));
}

public static LiteralExpression literal(ExprValue value) {
return new LiteralExpression(value);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
/*
* Copyright 2020 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.planner;

import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalAggregation;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalDedupe;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalEval;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalFilter;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlan;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanNodeVisitor;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalProject;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalRelation;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalRemove;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalRename;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalSort;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalValues;
import com.amazon.opendistroforelasticsearch.sql.planner.physical.AggregationOperator;
import com.amazon.opendistroforelasticsearch.sql.planner.physical.DedupeOperator;
import com.amazon.opendistroforelasticsearch.sql.planner.physical.EvalOperator;
import com.amazon.opendistroforelasticsearch.sql.planner.physical.FilterOperator;
import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlan;
import com.amazon.opendistroforelasticsearch.sql.planner.physical.ProjectOperator;
import com.amazon.opendistroforelasticsearch.sql.planner.physical.RemoveOperator;
import com.amazon.opendistroforelasticsearch.sql.planner.physical.RenameOperator;
import com.amazon.opendistroforelasticsearch.sql.planner.physical.SortOperator;
import com.amazon.opendistroforelasticsearch.sql.planner.physical.ValuesOperator;

/**
* Default implementor for implementing logical to physical translation. "Default" here means all
* logical operator will be translated to correspondent physical operator to pipeline operations
* in post-processing style in memory.
* Different storage can override methods here to optimize default pipelining operator, for example
* a storage has the flexibility to override visitFilter and visitRelation to push down filtering
* operation and return a single physical index scan operator.
*
* @param <C> context type
*/
public class DefaultImplementor<C> extends LogicalPlanNodeVisitor<PhysicalPlan, C> {

@Override
public PhysicalPlan visitDedupe(LogicalDedupe node, C context) {
return new DedupeOperator(
visitChild(node, context),
node.getDedupeList(),
node.getAllowedDuplication(),
node.getKeepEmpty(),
node.getConsecutive());
}

@Override
public PhysicalPlan visitProject(LogicalProject node, C context) {
return new ProjectOperator(visitChild(node, context), node.getProjectList());
}

@Override
public PhysicalPlan visitRemove(LogicalRemove node, C context) {
return new RemoveOperator(visitChild(node, context), node.getRemoveList());
}

@Override
public PhysicalPlan visitEval(LogicalEval node, C context) {
return new EvalOperator(visitChild(node, context), node.getExpressions());
}

@Override
public PhysicalPlan visitSort(LogicalSort node, C context) {
return new SortOperator(visitChild(node, context), node.getCount(), node.getSortList());
}

@Override
public PhysicalPlan visitRename(LogicalRename node, C context) {
return new RenameOperator(visitChild(node, context), node.getRenameMap());
}

@Override
public PhysicalPlan visitAggregation(LogicalAggregation node, C context) {
return new AggregationOperator(
visitChild(node, context), node.getAggregatorList(), node.getGroupByList());
}

@Override
public PhysicalPlan visitFilter(LogicalFilter node, C context) {
return new FilterOperator(visitChild(node, context), node.getCondition());
}

@Override
public PhysicalPlan visitValues(LogicalValues node, C context) {
return new ValuesOperator(node.getValues());
}

@Override
public PhysicalPlan visitRelation(LogicalRelation node, C context) {
throw new UnsupportedOperationException("Storage engine is responsible for "
+ "implementing and optimizing logical plan with relation involved");
}

protected PhysicalPlan visitChild(LogicalPlan node, C context) {
// Logical operators visited here must have a single child
return node.getChild().get(0).accept(this, context);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,15 @@

package com.amazon.opendistroforelasticsearch.sql.planner;

import static com.google.common.base.Strings.isNullOrEmpty;

import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlan;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalPlanNodeVisitor;
import com.amazon.opendistroforelasticsearch.sql.planner.logical.LogicalRelation;
import com.amazon.opendistroforelasticsearch.sql.planner.physical.PhysicalPlan;
import com.amazon.opendistroforelasticsearch.sql.storage.StorageEngine;
import com.amazon.opendistroforelasticsearch.sql.storage.Table;
import java.util.List;
import lombok.RequiredArgsConstructor;

/**
Expand All @@ -36,14 +39,19 @@ public class Planner {
private final StorageEngine storageEngine;

/**
* Generate optimal physical plan for logical plan.
* Generate optimal physical plan for logical plan. If no table involved,
* translate logical plan to physical by default implementor.
* TODO: for now just delegate entire logical plan to storage engine.
*
* @param plan logical plan
* @return optimal physical plan
*/
public PhysicalPlan plan(LogicalPlan plan) {
String tableName = findTableName(plan);
if (isNullOrEmpty(tableName)) {
return plan.accept(new DefaultImplementor<>(), null);
}

Table table = storageEngine.getTable(tableName);
return table.implement(plan);
}
Expand All @@ -53,9 +61,11 @@ private String findTableName(LogicalPlan plan) {

@Override
protected String visitNode(LogicalPlan node, Object context) {
// So far all logical node has single child except LogicalRelation
// whose visitRelation() is already overridden.
return node.getChild().get(0).accept(this, context);
List<LogicalPlan> children = node.getChild();
if (children.isEmpty()) {
return "";
}
return children.get(0).accept(this, context);
}

@Override
Expand Down
Loading

0 comments on commit 087da88

Please sign in to comment.