[
https://issues.apache.org/jira/browse/BEAM-4388?focusedWorklogId=113277&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-113277
]
ASF GitHub Bot logged work on BEAM-4388:
----------------------------------------
Author: ASF GitHub Bot
Created on: 19/Jun/18 17:33
Start Date: 19/Jun/18 17:33
Worklog Time Spent: 10m
Work Description: kennknowles closed pull request #5481: [BEAM-4388]
Support optimized logical plan
URL: https://github.com/apache/beam/pull/5481
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamQueryPlanner.java
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamQueryPlanner.java
index 22491710141..3d11ebf43a4 100644
---
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamQueryPlanner.java
+++
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamQueryPlanner.java
@@ -26,6 +26,7 @@
import org.apache.calcite.jdbc.CalciteSchema;
import org.apache.calcite.plan.Contexts;
import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptPlanner.CannotPlanException;
import org.apache.calcite.plan.RelOptUtil;
import org.apache.calcite.plan.RelTraitDef;
import org.apache.calcite.plan.RelTraitSet;
@@ -113,8 +114,8 @@ public SqlNode parse(String sqlStatement) throws
SqlParseException {
}
/** It parses and validate the input query, then convert into a {@link
BeamRelNode} tree. */
- BeamRelNode convertToBeamRel(String sqlStatement)
- throws ValidationException, RelConversionException, SqlParseException {
+ public BeamRelNode convertToBeamRel(String sqlStatement)
+ throws ValidationException, RelConversionException, SqlParseException,
CannotPlanException {
BeamRelNode beamRelNode;
Planner planner = getPlanner();
try {
@@ -122,6 +123,7 @@ BeamRelNode convertToBeamRel(String sqlStatement)
SqlNode validated = planner.validate(parsed);
LOG.info("SQL:\n" + validated);
+ // root of original logical plan
RelRoot root = planner.rel(validated);
LOG.info("SQLPlan>\n" + RelOptUtil.toString(root.rel));
@@ -131,8 +133,10 @@ BeamRelNode convertToBeamRel(String sqlStatement)
.replace(BeamLogicalConvention.INSTANCE)
.replace(root.collation)
.simplify();
+
+ // beam physical plan
beamRelNode = (BeamRelNode) planner.transform(0, desiredTraits,
root.rel);
- LOG.info("BeamSQL>\n" + RelOptUtil.toString(beamRelNode));
+ LOG.info("BEAMPlan>\n" + RelOptUtil.toString(beamRelNode));
} finally {
planner.close();
}
diff --git
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
index 92c8efaad33..6c13da8b1fe 100644
---
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
+++
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
@@ -34,11 +34,29 @@
import org.apache.beam.sdk.extensions.sql.impl.rule.BeamValuesRule;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.rules.AggregateJoinTransposeRule;
+import org.apache.calcite.rel.rules.AggregateProjectMergeRule;
+import org.apache.calcite.rel.rules.AggregateRemoveRule;
+import org.apache.calcite.rel.rules.AggregateUnionAggregateRule;
import org.apache.calcite.rel.rules.CalcMergeRule;
+import org.apache.calcite.rel.rules.FilterAggregateTransposeRule;
import org.apache.calcite.rel.rules.FilterCalcMergeRule;
+import org.apache.calcite.rel.rules.FilterJoinRule;
+import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
+import org.apache.calcite.rel.rules.FilterSetOpTransposeRule;
import org.apache.calcite.rel.rules.FilterToCalcRule;
+import org.apache.calcite.rel.rules.JoinPushExpressionsRule;
import org.apache.calcite.rel.rules.ProjectCalcMergeRule;
+import org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
+import org.apache.calcite.rel.rules.ProjectMergeRule;
+import org.apache.calcite.rel.rules.ProjectSetOpTransposeRule;
+import org.apache.calcite.rel.rules.ProjectSortTransposeRule;
import org.apache.calcite.rel.rules.ProjectToCalcRule;
+import org.apache.calcite.rel.rules.PruneEmptyRules;
+import org.apache.calcite.rel.rules.SortProjectTransposeRule;
+import org.apache.calcite.rel.rules.SortRemoveRule;
+import org.apache.calcite.rel.rules.UnionEliminatorRule;
+import org.apache.calcite.rel.rules.UnionToDistinctRule;
import org.apache.calcite.tools.RuleSet;
import org.apache.calcite.tools.RuleSets;
@@ -48,7 +66,6 @@
*/
@Internal
public class BeamRuleSets {
-
private static final List<RelOptRule> LOGICAL_OPTIMIZATIONS =
ImmutableList.of(
// Rules so we only have to implement Calc
@@ -58,7 +75,61 @@
ProjectToCalcRule.INSTANCE,
// https://issues.apache.org/jira/browse/BEAM-4522
// CalcRemoveRule.INSTANCE,
- CalcMergeRule.INSTANCE);
+ CalcMergeRule.INSTANCE,
+
+ // push a filter into a join
+ FilterJoinRule.FILTER_ON_JOIN,
+ // push filter into the children of a join
+ FilterJoinRule.JOIN,
+ // push filter through an aggregation
+ FilterAggregateTransposeRule.INSTANCE,
+ // push filter through set operation
+ FilterSetOpTransposeRule.INSTANCE,
+ // push project through set operation
+ ProjectSetOpTransposeRule.INSTANCE,
+
+ // aggregation and projection rules
+ AggregateProjectMergeRule.INSTANCE,
+ // push a projection past a filter or vice versa
+ ProjectFilterTransposeRule.INSTANCE,
+ FilterProjectTransposeRule.INSTANCE,
+ // push a projection to the children of a join
+ // merge projections
+ ProjectMergeRule.INSTANCE,
+ // ProjectRemoveRule.INSTANCE,
+ // reorder sort and projection
+ SortProjectTransposeRule.INSTANCE,
+ ProjectSortTransposeRule.INSTANCE,
+
+ // join rules
+ JoinPushExpressionsRule.INSTANCE,
+
+ // remove union with only a single child
+ UnionEliminatorRule.INSTANCE,
+ // convert non-all union into all-union + distinct
+ UnionToDistinctRule.INSTANCE,
+
+ // remove aggregation if it does not aggregate and input is already
distinct
+ AggregateRemoveRule.INSTANCE,
+ // push aggregate through join
+ AggregateJoinTransposeRule.EXTENDED,
+ // aggregate union rule
+ AggregateUnionAggregateRule.INSTANCE,
+
+ // reduce aggregate functions like AVG, STDDEV_POP etc.
+ // AggregateReduceFunctionsRule.INSTANCE,
+
+ // remove unnecessary sort rule
+ SortRemoveRule.INSTANCE,
+
+ // prune empty results rules
+ PruneEmptyRules.AGGREGATE_INSTANCE,
+ PruneEmptyRules.FILTER_INSTANCE,
+ PruneEmptyRules.JOIN_LEFT_INSTANCE,
+ PruneEmptyRules.JOIN_RIGHT_INSTANCE,
+ PruneEmptyRules.PROJECT_INSTANCE,
+ PruneEmptyRules.SORT_INSTANCE,
+ PruneEmptyRules.UNION_INSTANCE);
private static final List<RelOptRule> BEAM_CONVERTERS =
ImmutableList.of(
@@ -80,9 +151,9 @@
return new RuleSet[] {
RuleSets.ofList(
ImmutableList.<RelOptRule>builder()
- .addAll(LOGICAL_OPTIMIZATIONS)
.addAll(BEAM_CONVERTERS)
.addAll(BEAM_TO_ENUMERABLE)
+ .addAll(LOGICAL_OPTIMIZATIONS)
.build())
};
}
diff --git
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
index 1cdde3315da..0ec39ab1e71 100644
---
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
+++
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
@@ -25,7 +25,6 @@
/** A {@link RelNode} that can also give a {@link PTransform} that implements
the expression. */
public interface BeamRelNode extends RelNode {
-
/**
* A {@link BeamRelNode} is a recursive structure, the {@code
BeamQueryPlanner} visits it with a
* DFS(Depth-First-Search) algorithm.
diff --git
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinRule.java
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinRule.java
index 88020991d99..1e3a492582a 100644
---
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinRule.java
+++
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinRule.java
@@ -37,6 +37,7 @@ private BeamJoinRule() {
@Override
public RelNode convert(RelNode rel) {
Join join = (Join) rel;
+
return new BeamJoinRel(
join.getCluster(),
join.getTraitSet().replace(BeamLogicalConvention.INSTANCE),
diff --git
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java
new file mode 100644
index 00000000000..cf5b9b37f23
--- /dev/null
+++
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License 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 org.apache.beam.sdk.extensions.sql;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTableProvider;
+import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+import org.junit.Before;
+import org.junit.Ignore;
+
+/** UnitTest for Explain Plan. */
+public class BeamSqlExplainTest {
+ private InMemoryMetaStore metaStore;
+ private BeamSqlCli cli;
+
+ @Before
+ public void setUp() throws SqlParseException, RelConversionException,
ValidationException {
+ metaStore = new InMemoryMetaStore();
+ metaStore.registerProvider(new TextTableProvider());
+ cli = new BeamSqlCli().metaStore(metaStore);
+
+ cli.execute(
+ "create table person (\n"
+ + "id int COMMENT 'id', \n"
+ + "name varchar COMMENT 'name', \n"
+ + "age int COMMENT 'age') \n"
+ + "TYPE 'text' \n"
+ + "COMMENT '' ");
+
+ cli.execute(
+ "create table A (\n"
+ + "c1 int COMMENT 'c1',\n"
+ + "c2 int COMMENT 'c2')\n"
+ + "TYPE 'text'\n"
+ + "COMMENT '' ");
+
+ cli.execute(
+ "create table B (\n"
+ + "c1 int COMMENT 'c1',\n"
+ + "c2 int COMMENT 'c2')\n"
+ + "TYPE 'text'\n"
+ + "COMMENT '' ");
+ }
+
+ // TODO: (BEAM-4561) 5/30/2017 The test here is too fragile.
+ @Ignore
+ public void testExplainCommaJoin() {
+ String plan = cli.explainQuery("SELECT A.c1, B.c2 FROM A, B WHERE A.c1 =
B.c2 AND A.c1 > 0");
+
+ assertEquals(
+ "BeamCalcRel(expr#0..3=[{inputs}], c1=[$t0], c2=[$t3])\n"
+ + " BeamJoinRel(condition=[=($0, $3)], joinType=[inner])\n"
+ + " BeamCalcRel(expr#0..1=[{inputs}], expr#2=[0],
expr#3=[>($t0, $t2)],"
+ + " proj#0..1=[{exprs}], $condition=[$t3])\n"
+ + " BeamIOSourceRel(table=[[beam, A]])\n"
+ + " BeamIOSourceRel(table=[[beam, B]])\n",
+ plan);
+ }
+}
diff --git
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTestBase.java
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTestBase.java
index 3ff562acbcf..cfce911c22c 100644
---
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTestBase.java
+++
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTestBase.java
@@ -17,8 +17,7 @@
*/
package org.apache.beam.sdk.extensions.sql.impl.interpreter;
-import java.util.ArrayList;
-import java.util.List;
+import com.google.common.collect.ImmutableList;
import
org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRuleSets;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
@@ -71,9 +70,8 @@ public static void prepare() {
.build();
SchemaPlus schema = Frameworks.createRootSchema(true);
- final List<RelTraitDef> traitDefs = new ArrayList<>();
- traitDefs.add(ConventionTraitDef.INSTANCE);
- traitDefs.add(RelCollationTraitDef.INSTANCE);
+ final ImmutableList<RelTraitDef> traitDefs =
+ ImmutableList.of(ConventionTraitDef.INSTANCE,
RelCollationTraitDef.INSTANCE);
FrameworkConfig config =
Frameworks.newConfigBuilder()
.parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build())
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 113277)
Time Spent: 6h 10m (was: 6h)
> Support optimized logical plan
> ------------------------------
>
> Key: BEAM-4388
> URL: https://issues.apache.org/jira/browse/BEAM-4388
> Project: Beam
> Issue Type: New Feature
> Components: dsl-sql
> Reporter: Kai Jiang
> Assignee: Kai Jiang
> Priority: Major
> Time Spent: 6h 10m
> Remaining Estimate: 0h
>
> Before converting into Beam Pipeline physical plan, logical plan should be
> optimized and it will be super helpful for efficiently executing Beam
> PTransforms pipeline.
> Calcite has two ways for optimizing logical plan (HepPlanner and
> VolcanoPlanner). We can support VolcanoPlanner first and apply calcite
> builtin optimize rules (like
> FilterJoinRule.FILTER_ON_JOIN) to sql query optimize plans.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)