-
Notifications
You must be signed in to change notification settings - Fork 28.7k
[SPARK-8357] [SQL] Memory leakage on unsafe aggregation path with empty input #6810
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
@@ -270,7 +270,9 @@ case class GeneratedAggregate( | |||
|
|||
val joinedRow = new JoinedRow3 | |||
|
|||
if (groupingExpressions.isEmpty) { | |||
if (!iter.hasNext) { | |||
Iterator[InternalRow]() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we add a comment here to explain why we're doing this? Just needs to mention the memory leak issue.
I guess there's not a super easy way to add a regression test for this since it's kind of hard to just instantiate the operator and run it with some empty input. |
Jenkins, this is ok to test. |
I'm actually working on some tooling that will make it very easy to write regressions tests for these kinds of issues (basically, a way to instantiate individual operators and run them with canned inputs, which is much easier than trying to rig a high-level query that triggers the code path under test). This should not block on that, though, but I'll ping this thread later today with an example of how we could write such a test. |
Test build #34889 has finished for PR 6810 at commit
|
Now that we've merged #6885, it should be possible to write a regression test for this. Take a look and let me know if you're interested in doing that. If not, I can merge this as-is. |
@JoshRosen Sure, I'll review your patch first. Thanks. |
@JoshRosen Done. |
Test build #35440 has finished for PR 6810 at commit
|
Test build #35450 has finished for PR 6810 at commit
|
val groupExpr = df.col("b").expr | ||
val aggrExpr = Alias(Count(Cast(groupExpr, LongType)), "Count")() | ||
|
||
SparkEnv.get.conf.set("spark.unsafe.exceptionOnMemoryLeak", "true") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This option is automatically enabled when running tests in Maven or SBT, so we don't need to put this here. Additionally, mutating the SparkConf after it's already been used to create a SparkContext is not always safe, so I think that we should remove this line.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, I know that. Maybe I'm wrong but when running the test in IDE(intelliJ), the policy seemed not applied. I'll remove this line, anyway.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah, I can see how this would be an issue for IntelliJ. If we do want to modify the conf, though, we should do it elsewhere (when creating the SparkContext that's used to create the TestSQLContext).
Test build #35504 has finished for PR 6810 at commit
|
Test build #35507 has finished for PR 6810 at commit
|
I appreciate you trying to add a more general save / restore / backup mechanism for SqlConf state, but I'd rather not introduce that change as part of a bugfix. I think that we might want to be able to introduce / test / backport that test infra change independent of the change here. Do you mind just mimicking the simple try-finally save/restore that's used by other SQL tests and rolling back the wider test infra changes? Happy to include them, just not here. |
Also, if we do introduce a more general way of handling this sort of state cleanup as part of the test runner, then I suspect we'll want to both remove a lot of the old try-finally code and will have to fix any bugs due to tests that implicitly relied on the old behavior. Since that has the potential to grow to cover a lot more changes than this bugfix, I'd like to tackle it separately. |
|
||
for ((codegen, unsafe) <- Seq((false, false), (true, false), (true, true)); | ||
partial <- Seq(false, true)) { | ||
TestSQLContext.conf.setConfString("spark.sql.codegen", String.valueOf(codegen)) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could you use the new SparkConf API? This line will be TestSQLContext.setConf(SQLConf.CODEGEN_ENABLED, codegen)
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done. Thanks.
Test build #35532 has finished for PR 6810 at commit
|
class InterpretedProjection(expressions: Seq[Expression], mutableRow: Boolean = false) | ||
extends Projection { | ||
def this(expressions: Seq[Expression], | ||
inputSchema: Seq[Attribute], mutableRow: Boolean = false) = |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could you explain this mutableRow
change? If we do need to keep it, the style here isn't quite right; see https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide for how to properly wrap long parameter lists.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I see that your comment mentioned that this is avoiding a potential ClassCastException. Did you actually run into that exception somewhere?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry for late reply. It happened when I changed empty input in AggregateSuite test to something like ("Hello", 4, 2.0). In GeneratedAggreate, whenever the grouping expression is empty or unsafeEnabled is not applied for some reason spark tries to cast GenericRow to MutableRow.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sound like there should be another test.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
NAVER - http://www.naver.com/
[email protected] 님께 보내신 메일 <Re: [spark] [SPARK-8357] [SQL] Memory leakage on unsafe aggregation path with empty input (#6810)> 이 다음과 같은 이유로 전송 실패했습니다.
받는 사람이 회원님의 메일을 수신차단 하였습니다.
c4b1f32
to
19907a5
Compare
Test build #35849 has finished for PR 6810 at commit
|
Test build #35850 has finished for PR 6810 at commit
|
val colC = df.col("c").expr | ||
val aggrExpr = Alias(Count(Cast(colC, LongType)), "Count")() | ||
|
||
for ((codegen, unsafe) <- Seq((false, false), (true, false), (true, true)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
AFAIK we only need to test the (true, true)
case, since this bug only occurred when using the unsafe aggregation path?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just wanted to see all is well. I'll remove the condition.
Test build #36072 has finished for PR 6810 at commit
|
As I understand it, there are two separate bugs here:
If this is the case, can we fix these separately? Combining them into the same PR / test code makes them harder to understand. |
Thanks, @JoshRosen. Actually, it's two bugs which is
I've booked second bug to SPARK-8826(#7225). |
Now that you've moved the second bug to #7225, do you mind cleaning this PR up to reflect only the first bug? |
I'm working on a patch to enable Unsafe mode by default and ran into this problem as well, so I've opened #7560 to bring this patch up to date and merge it. I'll still credit you with the primary patch authorship. |
Currently, unsafe-based hash is released on 'next' call but if input is empty, it would not be called ever.