-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-4033][Examples]Input of the SparkPi too big causes the emption exception #2874
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
if the Input of the SparkPi is too big, it will throw new UnsupportedOperationException("empty collection"), because the n may be a negative int number.
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.
How about something like math.min(100000L * slices, Int.MaxValue).toInt? Or even just cap slices with val slices = math.min(..., 21474), with a comment. I know it's a demo but maybe it just makes sense to cap the value of slices to something much lower 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.
@srowen when the value n is larger than the Int.MaxValue, it's better to set just a small value for slices.But user inputting a big slices may just wish that the SparkPi can run for more time.So I just limit the size of Seqs and do not modify the parallelism of the app.
|
Could we fix this by reading in the argument as a long instead of an integer? |
|
Yes, that would also fix the overflow, or at least, push it much farther away. Does it ever make sense to have tens of billions of elements in the RDD for this computation though? Allowing quardillions doesn't probably work anyway. I'd just cap the number of slices to something sane. |
|
@JoshRosen the n inside "0 until n " must be an integer so we can't make input as a long |
|
@srowen I had considered that just judging the number of the input to be a small int will be a easy way. |
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.
Why Int.MaxValue - 1? I believe (1 to Int.MaxValue) will just work fine.
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, this can just be Int.MaxValue
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.
maybe also add a quick comment at the end to explain why we're doing this:
val n = math.min(100000L * slices, Int.MaxValue).toInt // avoid overflow
|
@SaintBacchus sure, maybe this is used as a load test in some situations. What if n map tasks computed 100000 iterations each (or some smaller quantum)? rather than 100000n map tasks computing 1? You might still check for negative n to be tidy but int overflow is no longer an issue then. |
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 think you need to supply a new slices number if slices is too large.
something like math.min(slices, n / 100000).
So, a better solution may be :
val realSlices = math.min(slices, Int.MaxValue/100000)
val n = 100000 * realSlices
|
@SaintBacchus why did you close this? seems like it still needs a fix and you had an improvement going here. |
|
Hi @srowen , I considered it was a very small improvement and counld be found easily if the spark developer check this code so I closed it. |
|
Test build #24349 has finished for PR 2874 at commit
|
|
@SaintBacchus Any update on this PR? As @JoshRosen mentioned I think the correct fix here is to use |
|
@andrewor14 I had explained why it can not use |
|
How about just capping the argument to the largest |
|
I see. I think it's simplest to just exit and tell the user to provide a smaller input if the provided input would've cause an overflow. For instance: https://github.com/andrewor14/spark/compare/spark-pi-validate-args. I don't feel strongly for or against the alternative that @srowen suggested, but I personally find the fail-fast solution easiest to understand. |
|
Actually, on second thought I think the existing solution in this PR is probably sufficient. The issue with the validation solution is that it adds a lot of extra code to this supposedly small and isolated example. For this reason I'm also OK with not printing a warning when overflow happens, since it adds extra logic. LGTM once you remove the |
|
Sorry for forgetting explain why there must be def positions(length: Long, numSlices: Int): Iterator[(Int, Int)] = {
(0 until numSlices).iterator.map(i => {
val start = ((i * length) / numSlices).toInt
val end = (((i + 1) * length) / numSlices).toInt
(start, end)
})
}in line 122 at sc.makeRDD(1 to (Int.MaxValue)).count // result = 0
sc.makeRDD(1 to (Int.MaxValue - 1)).count // result = 2147483646 = Int.MaxValue - 1
sc.makeRDD(1 until (Int.MaxValue)).count // result = 2147483646 = Int.MaxValue - 1 |
Add a commit to tell the user why we had a judgement here. And use `until` instead of `to` for the purpose of avoid the limit size of the input seq and make the code more readable.
|
Test build #25376 has finished for PR 2874 at commit
|
|
Hi @SaintBacchus, I get the idea that there is a limit in the seq size. But I don't think the buggy code is from implementation you listed. The actually buggy code is the first case in the pattern matching. case r: Range.Inclusive => {
val sign = if (r.step < 0) {
-1
} else {
1
}
slice(new Range(
r.start, r.end + sign, r.step).asInstanceOf[Seq[T]], numSlices) // r.end + sign is overflow
}We should consider fix that bug. @srowen and @andrewor14, do you think we need another pr? |
|
@advancedxy I do think it's a separate issue. The case of integer overflow could be handled better in both places. |
|
I see, @advancedxy would you mind filing a JIRA for this? If it's convenient for you to do so, feel free to also create a PR for it. As for this PR, the latest changes LGTM. The difference between |
|
@andrewor14, of course, I will file a jira and send a pr when I get some spare time later today. |
|
@advancedxy I had showed the wrong bad code:sparkles: Thanks for point it out |
…D.slice method There is an int overflow in the ParallelCollectionRDD.slice method. That's originally reported by SaintBacchus. ``` sc.makeRDD(1 to (Int.MaxValue)).count // result = 0 sc.makeRDD(1 to (Int.MaxValue - 1)).count // result = 2147483646 = Int.MaxValue - 1 sc.makeRDD(1 until (Int.MaxValue)).count // result = 2147483646 = Int.MaxValue - 1 ``` see #2874 for more details. This pr try to fix the overflow. However, There's another issue I don't address. ``` val largeRange = Int.MinValue to Int.MaxValue largeRange.length // throws java.lang.IllegalArgumentException: -2147483648 to 2147483647 by 1: seqs cannot contain more than Int.MaxValue elements. ``` So, the range we feed to sc.makeRDD cannot contain more than Int.MaxValue elements. This is the limitation of Scala. However I think we may want to support that kind of range. But the fix is beyond this pr. srowen andrewor14 would you mind take a look at this pr? Author: Ye Xianjin <[email protected]> Closes #4002 from advancedxy/SPARk-5201 and squashes the following commits: 96265a1 [Ye Xianjin] Update slice method comment and some responding docs. e143d7a [Ye Xianjin] Update inclusive range check for splitting inclusive range. b3f5577 [Ye Xianjin] We can include the last element in the last slice in general for inclusive range, hence eliminate the need to check Int.MaxValue or Int.MinValue. 7d39b9e [Ye Xianjin] Convert the two cases pattern matching to one case. 651c959 [Ye Xianjin] rename sign to needsInclusiveRange. add some comments 196f8a8 [Ye Xianjin] Add test cases for ranges end with Int.MaxValue or Int.MinValue e66e60a [Ye Xianjin] Deal with inclusive and exclusive ranges in one case. If the range is inclusive and the end of the range is (Int.MaxValue or Int.MinValue), we should use inclusive range instead of exclusive
… exception
If input of the SparkPi args is larger than the 25000, the integer 'n' inside the code will be overflow, and may be a negative number.
And it causes the (0 until n) Seq as an empty seq, then doing the action 'reduce' will throw the UnsupportedOperationException("empty collection").
The max size of the input of sc.parallelize is Int.MaxValue - 1, not the Int.MaxValue.
Author: huangzhaowei <[email protected]>
Closes #2874 from SaintBacchus/SparkPi and squashes the following commits:
62d7cd7 [huangzhaowei] Add a commit to explain the modify
4cdc388 [huangzhaowei] Update SparkPi.scala
9a2fb7b [huangzhaowei] Input of the SparkPi is too big
…D.slice method There is an int overflow in the ParallelCollectionRDD.slice method. That's originally reported by SaintBacchus. ``` sc.makeRDD(1 to (Int.MaxValue)).count // result = 0 sc.makeRDD(1 to (Int.MaxValue - 1)).count // result = 2147483646 = Int.MaxValue - 1 sc.makeRDD(1 until (Int.MaxValue)).count // result = 2147483646 = Int.MaxValue - 1 ``` see #2874 for more details. This pr try to fix the overflow. However, There's another issue I don't address. ``` val largeRange = Int.MinValue to Int.MaxValue largeRange.length // throws java.lang.IllegalArgumentException: -2147483648 to 2147483647 by 1: seqs cannot contain more than Int.MaxValue elements. ``` So, the range we feed to sc.makeRDD cannot contain more than Int.MaxValue elements. This is the limitation of Scala. However I think we may want to support that kind of range. But the fix is beyond this pr. srowen andrewor14 would you mind take a look at this pr? Author: Ye Xianjin <[email protected]> Closes #4002 from advancedxy/SPARk-5201 and squashes the following commits: 96265a1 [Ye Xianjin] Update slice method comment and some responding docs. e143d7a [Ye Xianjin] Update inclusive range check for splitting inclusive range. b3f5577 [Ye Xianjin] We can include the last element in the last slice in general for inclusive range, hence eliminate the need to check Int.MaxValue or Int.MinValue. 7d39b9e [Ye Xianjin] Convert the two cases pattern matching to one case. 651c959 [Ye Xianjin] rename sign to needsInclusiveRange. add some comments 196f8a8 [Ye Xianjin] Add test cases for ranges end with Int.MaxValue or Int.MinValue e66e60a [Ye Xianjin] Deal with inclusive and exclusive ranges in one case. If the range is inclusive and the end of the range is (Int.MaxValue or Int.MinValue), we should use inclusive range instead of exclusive
If input of the SparkPi args is larger than the 25000, the integer 'n' inside the code will be overflow, and may be a negative number.
And it causes the (0 until n) Seq as an empty seq, then doing the action 'reduce' will throw the UnsupportedOperationException("empty collection").
The max size of the input of sc.parallelize is Int.MaxValue - 1, not the Int.MaxValue.