Skip to content

Conversation

@aayushmaanjain
Copy link
Contributor

@aayushmaanjain aayushmaanjain commented Apr 2, 2019

What changes were proposed in this pull request?

With this change, unnecessary file scans are avoided in case of Limit 0 queries.

I added a case (rule) to PropagateEmptyRelation to replace GlobalLimit 0 and LocalLimit 0 nodes with an empty LocalRelation. This prunes the subtree under the Limit 0 node and further allows other rules of PropagateEmptyRelation to optimize the Logical Plan - while remaining semantically consistent with the Limit 0 query.

For instance:
Query:
SELECT * FROM table1 INNER JOIN (SELECT * FROM table2 LIMIT 0) AS table2 ON table1.id = table2.id

Optimized Plan without fix:

Join Inner, (id#79 = id#87)
:- Filter isnotnull(id#79)
:  +- Relation[id#79,num1#80] parquet
+- Filter isnotnull(id#87)
   +- GlobalLimit 0
      +- LocalLimit 0
         +- Relation[id#87,num2#88] parquet

Optimized Plan with fix:
LocalRelation <empty>, [id#75, num1#76, id#77, num2#78]

How was this patch tested?

Added unit tests to verify Limit 0 optimization for:

  • Simple query containing Limit 0
  • Inner Join, Left Outer Join, Right Outer Join, Full Outer Join queries containing Limit 0 as one of their children
  • Nested Inner Joins between 3 tables with one of them having a Limit 0 clause.
  • Intersect query wherein one of the subqueries was a Limit 0 query.

@srowen
Copy link
Member

srowen commented Apr 2, 2019

Seems reasonable, but @gengliangwang do you have an opinion?

@SparkQA
Copy link

SparkQA commented Apr 2, 2019

Test build #4680 has finished for PR 24271 at commit d5fed71.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

// changes up the Logical Plan.
//
// Replace Global Limit 0 nodes with empty Local Relation
case p @ GlobalLimit(IntegerLiteral(limit), _) if limit == 0 =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it's better to do a new rule, to make it more modular.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, we can also create a new Batch for the new rule, since it requires only one-time transformation.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@rxin @gengliangwang Hi!
We are relying on the other case rules in PropagateEmptyRelation to propagate the effects of substituting Limit 0 with empty LocalRelation through the Logical Plan - such as in case of joins, project, etc.
So, should we handle this case also part of this rule, and then propagation of empty relation happens only as part of this rule? Or do you recommend doing it as a separate rule?

Copy link
Member

@gengliangwang gengliangwang Apr 3, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The optimization in this PR is not directly related to "propagate". We can create a separate rule and batch , e.g.

    Batch("OptimizeLimitZero", Once, OptimizeLimitZero) :+
    Batch("LocalRelation", fixedPoint,
      ConvertToLocalRelation,
      PropagateEmptyRelation
   )

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@rxin @gengliangwang @srowen I have made the recommended changes. Please have a look. Thanks!

Copy link
Contributor

@attilapiros attilapiros left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just minor stuff and a question. Otherwise LGTM.

val testRelation1 = LocalRelation.fromExternalRows(Seq('a.int), data = Seq(Row(1)))
val testRelation2 = LocalRelation.fromExternalRows(Seq('b.int), data = Seq(Row(1)))


Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

super Nit: too much empty lines

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

comparePlans(optimized, correctAnswer)
}

test("Limit 0: Joins") {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You know you can create easily separated test for each join types, like this:

Seq(
    (Inner, LocalRelation('a.int, 'b.int)),
    (LeftOuter, Project(Seq('a, Literal(null).cast(IntegerType).as('b)), testRelation1).analyze),
    (RightOuter, LocalRelation('a.int, 'b.int)),
    (FullOuter, Project(Seq('a, Literal(null).cast(IntegerType).as('b)), testRelation1).analyze)
  ).foreach { case (jt, answer) =>
    test(s"Limit 0: for join type $jt") {
        val query = testRelation1
          .join(testRelation2.limit(0), jt, condition = Some('a.attr == 'b.attr))

        val optimized = Optimize.execute(query.analyze)
        val correctAnswer = answer

        comparePlans(optimized, correctAnswer)
      }
    }

So this way the suite execution will contain sth like:

[info] - Limit 0: for join type Inner (30 milliseconds)
[info] - Limit 0: for join type LeftOuter (21 milliseconds)
[info] - Limit 0: for join type RightOuter (13 milliseconds)
[info] - Limit 0: for join type FullOuter (12 milliseconds)

It has only one little advantage if more than one would fail execution would not stop at the first assert.
But your current solution is also very fine.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@attilapiros thanks for the input. I made the required changes.

// For example, a query such as Filter(LocalRelation) would go through all the heavy
// optimizer rules that are triggered when there is a filter
// (e.g. InferFiltersFromConstraints). If we run this batch earlier, the query becomes just
// LocalRelation and does not trigger many rules.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The comment should be before the batch LocalRelation early.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@gengliangwang, fixed this.

class OptimizeLimitZeroSuite extends PlanTest {
object Optimize extends RuleExecutor[LogicalPlan] {
val batches =
Batch("OptimizeLimitZero", Once,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we can have two batches here, just the same as Optimizer.

Copy link
Contributor Author

@aayushmaanjain aayushmaanjain Apr 4, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@gengliangwang Is there a specific reason for having it that way? I modelled the test suite based on PropagateEmptyRelation, wherein also they have all rules as part of the same batch..

Copy link
Member

@gengliangwang gengliangwang left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM except my last comment in the test suite.

.join(testRelation2.limit(0), joinType = jt, condition = Some('a.attr == 'b.attr))

val optimized = Optimize.execute(query.analyze)
val correctAnswer = answer
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

redundant var ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed


/**
* Replaces GlobalLimit 0 and LocalLimit 0 nodes (subtree) with empty Local Relation, as they don't
* return any rows.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fix the alignment here ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

// changes up the Logical Plan.
//
// Replace Global Limit 0 nodes with empty Local Relation
case gl @ GlobalLimit(IntegerLiteral(limit), _) if limit == 0 =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

case gl @ GlobalLimit(IntegerLiteral(0), _) => ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@dilipbiswal good catch! Made this change.

// then the following rule will handle that case as well.
//
// Replace Local Limit 0 nodes with empty Local Relation
case ll @ LocalLimit(IntegerLiteral(limit), _) if limit == 0 =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same as above..

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@dilipbiswal
Copy link
Contributor

retest this please

@dilipbiswal
Copy link
Contributor

@srowen Hi Sean.. for some reason, its not responding to "retest" ?

@srowen
Copy link
Member

srowen commented Apr 4, 2019

Jenkins add to whitelist

@SparkQA
Copy link

SparkQA commented Apr 5, 2019

Test build #104310 has finished for PR 24271 at commit 1936ec6.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@dilipbiswal
Copy link
Contributor

LGTM

@SparkQA
Copy link

SparkQA commented Apr 5, 2019

Test build #4690 has finished for PR 24271 at commit 1936ec6.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Member

@gatorsmile gatorsmile left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

Thanks! Merged to master.

@gatorsmile gatorsmile changed the title [SPAR-27342][SQL] Optimize Limit 0 queries [SPARK-27342][SQL] Optimize Limit 0 queries Apr 5, 2019
@gatorsmile gatorsmile closed this in 04e53d2 Apr 5, 2019
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

8 participants