-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-27659][PYTHON] Allow PySpark to prefetch during toLocalIterator #25515
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
[SPARK-27659][PYTHON] Allow PySpark to prefetch during toLocalIterator #25515
Conversation
…looking at the next elem not the one we are about to block on, and fix the Python tests.
|
cc @BryanCutler who created 5e79ae3 |
|
Test build #109437 has finished for PR 25515 at commit
|
|
Test build #109448 has finished for PR 25515 at commit
|
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.
Thanks for doing this @holdenk ! It can definitely improve performance when calculating partitions takes some time. I know this issue was just for Python, but Scala toLocalIterators could also benefit from prefetch, I believe. WDYT?
| if (prefetchPartitions) { | ||
| prefetchIter.headOption | ||
| } | ||
| val partitionArray = ThreadUtils.awaitResult(partitionFuture, Duration.Inf) |
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.
It might be best to avoid awaitResult if possible. Could you make a buffered iterator yourself?
maybe something like
var next = collectPartitionIter.next()
val prefetchIter = collectPartitionIter.map { part =>
val tmp = next
next = part
tmp
} ++ Iterator(next)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.
So the awaitFuture (or something similar) is required for us to use futures. If we just used a buffered iterator without allowing the job to schedule separately we'd just block for both partitions right away instead of evaluating the other future in the background while we block on the first. (Implicitly this awaitResult is already effectively done inside of the previous DAGScheduler's runJob.
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 yes, you are totally right. That would block while getting the prefetched partition. This looks pretty good to me then.
One question though, when should the first job be triggered? I think the old behavior used to start the first job as soon as toLocalIterator() was called. From what I can tell, this will wait until the first iteration and then trigger the first 2 jobs. Either way is probably fine, but you might get slightly better performance by starting the first job immediately.
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.
In either case it waits for reading a request of data from the Python side before starting a job, because the map on the partition indices is lazily evaluated.
python/pyspark/tests/test_rdd.py
Outdated
| timesPrefetchNext = next(timesIterPrefetch) | ||
| print("With prefetch times are: " + str(timesPrefetchHead) + "," + str(timesPrefetchNext)) | ||
| self.assertTrue(timesNext - timesHead >= timedelta(seconds=2)) | ||
| self.assertTrue(timesPrefetchNext - timesPrefetchHead < timedelta(seconds=1)) |
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 is a pretty clever test! Anything with timings make me a bit worried about flakiness, but I don't have any other idea how to test this.. Is it possible to see if the jobs were scheduled?
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 we could if we used a fresh SparkContext but with the reused context I'm not sure how I'd know if the job was run or not.
|
I think Scala support is worth exploring too, I'm happy to file a follow up issue. |
HyukjinKwon
left a comment
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 quick nit and to double check, is the benchmark was performed with #25515 (comment)? Seems like the feature was mistakenly disabled.
examples/src/main/python/prefetch.py
Outdated
| @@ -0,0 +1,86 @@ | |||
| # | |||
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 examples in this directory target to show how the feature or API is used rather than showing the perf results .. - I think it can be just shown in the PR description.
Virtually the example seems it has to be just .toLocalIterator(prefetchPartitions=True) which I don't think worth as a separate example file.
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.
reasonable, I'll remove it from the examples, was mostly a simple way to share the microbenchmark.
python/pyspark/tests/test_rdd.py
Outdated
| rdd = self.sc.parallelize(range(2), 2) | ||
| times1 = rdd.map(lambda x: datetime.now()) | ||
| times2 = rdd.map(lambda x: datetime.now()) | ||
| timesIterPrefetch = times1.toLocalIterator(prefetchPartitions=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.
Shall we stick to underscore naming rule?
|
So the benchmark was done on RDDs not on dataframes (you can see the benchmark code in this PR). |
| // Client requested more data, attempt to collect the next partition | ||
| val partitionArray = collectPartitionIter.next() | ||
| val partitionFuture = prefetchIter.next() | ||
| // Cause the next job to be submitted if prefecthPartitions is enabled. |
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.
typo: prefecthPartitions -> prefetchPartitions
python/pyspark/tests/test_rdd.py
Outdated
| time.sleep(2) | ||
| timesNext = next(timesIter) | ||
| timesPrefetchNext = next(timesIterPrefetch) | ||
| print("With prefetch times are: " + str(timesPrefetchHead) + "," + str(timesPrefetchNext)) |
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.
Shall we remove print?
|
Test build #110432 has finished for PR 25515 at commit
|
|
Filed the follow up issue in https://issues.apache.org/jira/browse/SPARK-29083 |
|
If there are no more comments by Monday I'll merge this :) |
|
Merged to master |
|
Late review, but LGTM. Thanks @holdenk ! |
What changes were proposed in this pull request?
This PR allows Python toLocalIterator to prefetch the next partition while the first partition is being collected. The PR also adds a demo micro bench mark in the examples directory, we may wish to keep this or not.
Why are the changes needed?
In https://issues.apache.org/jira/browse/SPARK-23961 / 5e79ae3 we changed PySpark to only pull one partition at a time. This is memory efficient, but if partitions take time to compute this can mean we're spending more time blocking.
Does this PR introduce any user-facing change?
A new param is added to toLocalIterator
How was this patch tested?
New unit test inside of
test_rdd.pychecks the time that the elements are evaluated at. Another test that the results remain the same are added totest_dataframe.py.I also ran a micro benchmark in the examples directory
prefetch.pywhich shows an improvement of ~40% in this specific use case.