[ 
https://issues.apache.org/jira/browse/BEAM-7545?focusedWorklogId=278432&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-278432
 ]

ASF GitHub Bot logged work on BEAM-7545:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 17/Jul/19 19:39
            Start Date: 17/Jul/19 19:39
    Worklog Time Spent: 10m 
      Work Description: akedin commented on pull request #9040: [BEAM-7545] 
Reordering Beam Joins
URL: https://github.com/apache/beam/pull/9040#discussion_r304607629
 
 

 ##########
 File path: 
sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java
 ##########
 @@ -0,0 +1,462 @@
+/*
+ * 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.impl.rule;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRuleSets;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
+import 
org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableMap;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.rules.JoinCommuteRule;
+import org.apache.calcite.rel.rules.SortProjectTransposeRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.Statistics;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.schema.impl.AbstractTable;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.Planner;
+import org.apache.calcite.tools.Programs;
+import org.apache.calcite.tools.RuleSet;
+import org.apache.calcite.tools.RuleSets;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * This test ensures that we are reordering joins and get a plan similar to 
Join(large,Join(small,
+ * medium)) instead of Join(small, Join(medium,large).
+ */
+public class JoinReorderingTest {
+  private final PipelineOptions defaultPipelineOptions = 
PipelineOptionsFactory.create();
+
+  @Test
+  public void testTableSizes() {
+    TestTableProvider tableProvider = new TestTableProvider();
+    createThreeTables(tableProvider);
+
+    Assert.assertEquals(
+        BigInteger.ONE,
+        tableProvider
+            .buildBeamSqlTable(tableProvider.getTable("small_table"))
+            .getRowCount(null)
+            .getRowCount());
+
+    Assert.assertEquals(
+        BigInteger.valueOf(3),
+        tableProvider
+            .buildBeamSqlTable(tableProvider.getTable("medium_table"))
+            .getRowCount(null)
+            .getRowCount());
+
+    Assert.assertEquals(
+        BigInteger.valueOf(100),
+        tableProvider
+            .buildBeamSqlTable(tableProvider.getTable("large_table"))
+            .getRowCount(null)
+            .getRowCount());
+  }
+
+  @Test
+  public void testBeamJoinAssociationRule() throws Exception {
+    RuleSet prepareRules =
+        RuleSets.ofList(
+            SortProjectTransposeRule.INSTANCE,
+            EnumerableRules.ENUMERABLE_JOIN_RULE,
+            EnumerableRules.ENUMERABLE_PROJECT_RULE,
+            EnumerableRules.ENUMERABLE_SORT_RULE,
+            EnumerableRules.ENUMERABLE_TABLE_SCAN_RULE);
+
+    String sqlQuery =
+        "select * from \"tt\".\"large_table\" as large_table "
+            + " JOIN \"tt\".\"medium_table\" as medium_table on 
large_table.\"medium_key\" = medium_table.\"large_key\" "
+            + " JOIN \"tt\".\"small_table\" as small_table on 
medium_table.\"small_key\" = small_table.\"medium_key\" ";
+
+    RelNode originalPlan = transform(sqlQuery, prepareRules);
+    RelNode optimizedPlan =
+        transform(
+            sqlQuery,
+            RuleSets.ofList(
+                ImmutableList.<RelOptRule>builder()
+                    .addAll(prepareRules)
+                    .add(BeamJoinAssociateRule.INSTANCE)
+                    .build()));
+
+    assertTopTableInJoins(originalPlan, "small_table");
+    assertTopTableInJoins(optimizedPlan, "large_table");
+  }
+
+  @Test
+  public void testBeamJoinPushThroughJoinRuleLeft() throws Exception {
+    RuleSet prepareRules =
+        RuleSets.ofList(
+            SortProjectTransposeRule.INSTANCE,
+            EnumerableRules.ENUMERABLE_JOIN_RULE,
+            EnumerableRules.ENUMERABLE_PROJECT_RULE,
+            EnumerableRules.ENUMERABLE_SORT_RULE,
+            EnumerableRules.ENUMERABLE_TABLE_SCAN_RULE);
+
+    String sqlQuery =
+        "select * from \"tt\".\"large_table\" as large_table "
+            + " JOIN \"tt\".\"medium_table\" as medium_table on 
large_table.\"medium_key\" = medium_table.\"large_key\" "
+            + " JOIN \"tt\".\"small_table\" as small_table on 
medium_table.\"small_key\" = small_table.\"medium_key\" ";
+
+    RelNode originalPlan = transform(sqlQuery, prepareRules);
+    RelNode optimizedPlan =
+        transform(
+            sqlQuery,
+            RuleSets.ofList(
+                ImmutableList.<RelOptRule>builder()
+                    .addAll(prepareRules)
+                    .add(BeamJoinPushThroughJoinRule.LEFT)
+                    .build()));
+
+    assertTopTableInJoins(originalPlan, "small_table");
+    assertTopTableInJoins(optimizedPlan, "large_table");
+  }
+
+  @Test
+  public void testBeamJoinPushThroughJoinRuleRight() throws Exception {
+    RuleSet prepareRules =
+        RuleSets.ofList(
+            SortProjectTransposeRule.INSTANCE,
+            EnumerableRules.ENUMERABLE_JOIN_RULE,
+            EnumerableRules.ENUMERABLE_PROJECT_RULE,
+            EnumerableRules.ENUMERABLE_SORT_RULE,
+            EnumerableRules.ENUMERABLE_TABLE_SCAN_RULE);
+
+    String sqlQuery =
+        "select * from \"tt\".\"medium_table\" as medium_table "
+            + " JOIN \"tt\".\"large_table\" as large_table on 
large_table.\"medium_key\" = medium_table.\"large_key\" "
+            + " JOIN \"tt\".\"small_table\" as small_table on 
medium_table.\"small_key\" = small_table.\"medium_key\" ";
+
+    RelNode originalPlan = transform(sqlQuery, prepareRules);
+    RelNode optimizedPlan =
+        transform(
+            sqlQuery,
+            RuleSets.ofList(
+                ImmutableList.<RelOptRule>builder()
+                    .addAll(prepareRules)
+                    .add(BeamJoinPushThroughJoinRule.RIGHT)
+                    .build()));
+
+    assertTopTableInJoins(originalPlan, "small_table");
+    assertTopTableInJoins(optimizedPlan, "large_table");
+  }
+
+  @Test
+  public void testSystemReorderingLargeMediumSmall() {
+    TestTableProvider tableProvider = new TestTableProvider();
+    createThreeTables(tableProvider);
+    BeamSqlEnv env = BeamSqlEnv.withTableProvider(tableProvider);
+
+    // This is Join(Join(large, medium), small) which should be converted to a 
join that large table
+    // is on the top.
+    BeamRelNode parsedQuery =
+        env.parseQuery(
+            "select * from large_table "
+                + " JOIN medium_table on large_table.medium_key = 
medium_table.large_key "
+                + " JOIN small_table on medium_table.small_key = 
small_table.medium_key ");
+    assertTopTableInJoins(parsedQuery, "large_table");
+  }
+
+  @Test
+  public void testSystemReorderingMediumLargeSmall() {
+    TestTableProvider tableProvider = new TestTableProvider();
+    createThreeTables(tableProvider);
+    BeamSqlEnv env = BeamSqlEnv.withTableProvider(tableProvider);
+
+    // This is Join(Join(medium, large), small) which should be converted to a 
join that large table
+    // is on the top.
+    BeamRelNode parsedQuery =
+        env.parseQuery(
+            "select * from medium_table "
+                + " JOIN large_table on large_table.medium_key = 
medium_table.large_key "
+                + " JOIN small_table on medium_table.small_key = 
small_table.medium_key ");
+    assertTopTableInJoins(parsedQuery, "large_table");
+  }
+
+  @Test
+  public void testSystemNotReorderingWithoutRules() {
+    TestTableProvider tableProvider = new TestTableProvider();
+    createThreeTables(tableProvider);
+    ImmutableList.Builder<RelOptRule> ruleSetBuilder = 
ImmutableList.<RelOptRule>builder();
+    for (RuleSet set : BeamRuleSets.getRuleSets()) {
+      for (RelOptRule rule : set) {
 
 Review comment:
   nit: I think it can be expressed in a declarative way, something like:
   ```java
   set
    .stream()
    .flatMap(set -> set.stream()) //whatever the right way to convert the 
ruleset to stream of rules
    .filter(rule -> ...)
    .collect(toList());
   ```
   This way it's stateless, has less moving parts, and less nesting
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 278432)
    Time Spent: 9h 10m  (was: 9h)

> Row Count Estimation for CSV TextTable
> --------------------------------------
>
>                 Key: BEAM-7545
>                 URL: https://issues.apache.org/jira/browse/BEAM-7545
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql
>            Reporter: Alireza Samadianzakaria
>            Assignee: Alireza Samadianzakaria
>            Priority: Major
>             Fix For: Not applicable
>
>          Time Spent: 9h 10m
>  Remaining Estimate: 0h
>
> Implementing Row Count Estimation for CSV Tables by reading the first few 
> lines of the file and estimating the number of records based on the length of 
> these lines and the total length of the file.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to