Skip to content

Conversation

@mgaido91
Copy link
Contributor

What changes were proposed in this pull request?

As pointed out in the JIRA, there is a bug which causes an exception to be thrown if isin is called with an empty list on a cached DataFrame. The PR fixes it.

How was this patch tested?

Added UT.

Copy link
Member

@srowen srowen left a comment

Choose a reason for hiding this comment

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

This should just be simplified to list.exists(...)

case In(a: AttributeReference, list: Seq[Expression]) if list.forall(_.isInstanceOf[Literal]) =>
list.map(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] &&
l.asInstanceOf[Literal] <= statsFor(a).upperBound).reduce(_ || _)
l.asInstanceOf[Literal] <= statsFor(a).upperBound)
Copy link
Member

Choose a reason for hiding this comment

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

This still looks more complex and less efficient than

list.exists(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] && l.asInstanceOf[Literal] <= statsFor(a).upperBound)

or better

val stats = statsFor(a)
list.exists { l =>
  val literal = l.asInstanceOf[Literal]
  stats.lowerBound <= literal && literal <= stats.upperBound
}

The point being that you should be able to short-circuit evaluation here.
Or have I missed something basic like that these aren't Booleans?

Copy link
Contributor Author

@mgaido91 mgaido91 Oct 13, 2017

Choose a reason for hiding this comment

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

Exactly, they aren't Booleans. They are Expressions. And this is a PartialFunction which must return an Expression. Thus I can't think of a different (more efficient) way of doing this.

Copy link
Member

Choose a reason for hiding this comment

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

I see. How does .contains(true) work then? or did that not work?
I suppose all I mean is that we should write something that works on an empty list (returns false?) and also short-circuits (stops when anything is true). Is that possible?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It was a mistake, sorry. It returned always false.
I see what you mean, but in this piece of code we are only building the Expression and we are not evaluating it. Thus it is not possible to short-circuit, because the Expression must be built entirely.

Copy link
Member

Choose a reason for hiding this comment

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

OK I understand. Rather than build an extra term into the expression every time, why not special-case an empty list? return Literal(false) if empty, otherwise return the same thing as before.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think this is a bit more elegant and concise as code style, but your idea can be more efficient (though I am not sure how much overhead is introduced by a False evaluation). Then I am updating this PR according to your suggestion, thanks.

@mgaido91
Copy link
Contributor Author

@srowen I also updated the UT to check all the possible cases.

Copy link
Member

@srowen srowen left a comment

Choose a reason for hiding this comment

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

This seems OK to me. @dongjoon-hyun do you think this is OK? I think you wrote the original lines.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

Thank you for pining me, @srowen .
And, thank you for fixing this, @mgaido91 .
+1, LGTM.

@HyukjinKwon
Copy link
Member

ok to test

@SparkQA
Copy link

SparkQA commented Oct 16, 2017

Test build #82780 has finished for PR 19494 at commit f5bc105.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

retest this please

@SparkQA
Copy link

SparkQA commented Oct 16, 2017

Test build #82789 has finished for PR 19494 at commit f5bc105.

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

Copy link
Member

@HyukjinKwon HyukjinKwon left a comment

Choose a reason for hiding this comment

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

LGTM too

asfgit pushed a commit that referenced this pull request Oct 17, 2017
…taFrame

## What changes were proposed in this pull request?

As pointed out in the JIRA, there is a bug which causes an exception to be thrown if `isin` is called with an empty list on a cached DataFrame. The PR fixes it.

## How was this patch tested?

Added UT.

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #19494 from mgaido91/SPARK-22249.

(cherry picked from commit 8148f19)
Signed-off-by: Sean Owen <sowen@cloudera.com>
@srowen
Copy link
Member

srowen commented Oct 17, 2017

Merged to master/2.2. It didn't cherry pick cleanly to earlier branches but if it's needed there and someone opens a backport PR I can merge that too

@asfgit asfgit closed this in 8148f19 Oct 17, 2017
case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0

case In(_: AttributeReference, list: Seq[Expression]) if list.isEmpty => Literal.FalseLiteral
case In(a: AttributeReference, list: Seq[Expression]) if list.forall(_.isInstanceOf[Literal]) =>
Copy link
Member

@gatorsmile gatorsmile Oct 17, 2017

Choose a reason for hiding this comment

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

Please change it to

if list.forall(_.isInstanceOf[Literal]) && list.nonEmpty()

or using exists.

case IsNull(a: Attribute) => statsFor(a).nullCount > 0
case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0

case In(_: AttributeReference, list: Seq[Expression]) if list.isEmpty => Literal.FalseLiteral
Copy link
Member

@gatorsmile gatorsmile Oct 17, 2017

Choose a reason for hiding this comment

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

This should be moved to optimizer rules. It will cover both cached and non-cached cases.

Copy link
Member

Choose a reason for hiding this comment

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

BTW, this is wrong. If the left value is null, it should return null instead of false.

Copy link
Member

Choose a reason for hiding this comment

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

Ah, really. OK, yeah we need to change this. It can be reverted too.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

since this is only for filters, does it make any difference null or false?

Copy link
Member

Choose a reason for hiding this comment

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

Under this special context of filtering partitions, this In with empty list will result in a false literal in the end, no matter the attribute is null or not. We don't possibly have some expressions like IsNull(In(a, Nil)) as the filter predicate for now.

}
}

test("SPARK-22249: IN should work also with cached DataFrame") {
Copy link
Member

Choose a reason for hiding this comment

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

This test case is just an end-to-end test. This test will still pass if the optimizer has a change. We also need a unit test case.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

do you mean this test case is not enough and I should add a test case to check the proper behavior of the optimizer after the change?

@gatorsmile
Copy link
Member

Sorry for my late review. @mgaido91 Could you submit a follow-up PR to address my comments. Thanks!

@gatorsmile
Copy link
Member

I will not revert this PR but please submit the fix ASAP.

@mgaido91
Copy link
Contributor Author

I will, thanks for your suggestions @gatorsmile.

@mgaido91
Copy link
Contributor Author

I created the followup PR: #19522. Thanks.

asfgit pushed a commit that referenced this pull request Oct 18, 2017
…n the optimizer

## What changes were proposed in this pull request?

This PR addresses the comments by gatorsmile on [the previous PR](#19494).

## How was this patch tested?

Previous UT and added UT.

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #19522 from mgaido91/SPARK-22249_FOLLOWUP.

(cherry picked from commit 1f25d86)
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
ghost pushed a commit to dbtsai/spark that referenced this pull request Oct 18, 2017
…n the optimizer

## What changes were proposed in this pull request?

This PR addresses the comments by gatorsmile on [the previous PR](apache#19494).

## How was this patch tested?

Previous UT and added UT.

Author: Marco Gaido <marcogaido91@gmail.com>

Closes apache#19522 from mgaido91/SPARK-22249_FOLLOWUP.
@mgaido91 mgaido91 deleted the SPARK-22249 branch November 4, 2017 08:49
MatthewRBruce pushed a commit to Shopify/spark that referenced this pull request Jul 31, 2018
…taFrame

## What changes were proposed in this pull request?

As pointed out in the JIRA, there is a bug which causes an exception to be thrown if `isin` is called with an empty list on a cached DataFrame. The PR fixes it.

## How was this patch tested?

Added UT.

Author: Marco Gaido <marcogaido91@gmail.com>

Closes apache#19494 from mgaido91/SPARK-22249.

(cherry picked from commit 8148f19)
Signed-off-by: Sean Owen <sowen@cloudera.com>
MatthewRBruce pushed a commit to Shopify/spark that referenced this pull request Jul 31, 2018
…n the optimizer

## What changes were proposed in this pull request?

This PR addresses the comments by gatorsmile on [the previous PR](apache#19494).

## How was this patch tested?

Previous UT and added UT.

Author: Marco Gaido <marcogaido91@gmail.com>

Closes apache#19522 from mgaido91/SPARK-22249_FOLLOWUP.

(cherry picked from commit 1f25d86)
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
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.

7 participants