brentondwalker opened a new pull request, #54326:
URL: https://github.com/apache/spark/pull/54326
<!--
Thanks for sending a pull request! Here are some tips for you:
1. If this is your first time, please read our contributor guidelines:
https://spark.apache.org/contributing.html
2. Ensure you have added or run the appropriate tests for your PR:
https://spark.apache.org/developer-tools.html
3. If the PR is unfinished, add '[WIP]' in your PR title, e.g.,
'[WIP][SPARK-XXXX] Your PR title ...'.
4. Be sure to keep the PR description updated to reflect all changes.
5. Please write your PR title to summarize what this PR proposes.
6. If possible, provide a concise example to reproduce the issue for a
faster review.
7. If you want to add a new configuration, please read the guideline first
for naming configurations in
'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
8. If you want to add or modify an error type or message, please read the
guideline first in
'common/utils/src/main/resources/error/README.md'.
-->
### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section
is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster
reviews in your PR. See the examples below.
1. If you refactor some codes with changing classes, showing the class
hierarchy will help reviewers.
2. If you fix some SQL features, you can provide some references of other
DBMSes.
3. If there is design documentation, please add the link.
4. If there is a discussion in the mailing list, please add the link.
-->
- core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala:
add checkBarrierTasks() function which returns true if any pending taskSets
have isBarrier set.
-
core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala:
in receive(), if a task has finished, check if barrier taskSets are pending.
If so, make a global resource offer (all executors).
-
core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala:
add unit test to check that the patch is working. The test is
performance-based. It checks the time it takes to run the test code, which is
drastically reduced by the patch.
### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
1. If you propose a new API, clarify the use case for a new API.
2. If you fix a bug, you can clarify why it is a bug.
-->
If a barrier mode job (requiring _k_ cores) is pending, it is not serviced
immediately when _k_ cores become available, unless all _k_ available cores are
on the same executor.
The events that can cause a pending barrier job to be serviced are:
* revive timer (1 sec interval)
* job arrival
* rare state changes like executor add/remove
This leads to an average 500 ms delay in starting pending barrier stages.
Barrier jobs already have scaling issues because cores have to sit idle until
enough are available to meet the requirements. The extra 500 ms could have a
large effect on performance.
*More detail:*
When a task completes and the core becomes available,
CoarseGrainedSchedulerBackend only makes a resource offer of that one executor.
That is OK for normal tasks, but if a barrier taskSet is pending, the cores
needed to service it may be on other executors.
A global offer of all executors is only triggered by the events listed above.
*Why?*
Not clear if this is intentional or not.
If there are N cores, then the rate of task completions is proportional to
N. The number of executors to be iterated over for a global resource offer is
also proportional to N. So the load of doing global resource offers for every
task completion is O(N^2), which is not so nice. The current situation is an
understandable optimization.
*Suggested fix:*
Lots of ways this could be addressed. I implemented a minor patch that,
when a task finishes, checks if any barrier tasks are pending. if barrier
tasks are pending it makes a global offer.
This would keep the behavior the same for non-barrier jobs, but allow
barrier jobs to be serviced immediately when enough cores are available.
### Does this PR introduce _any_ user-facing change?
<!--
Note that it means *any* user-facing change including all aspects such as
new features, bug fixes, or other behavior changes. Documentation-only updates
are not considered user-facing changes.
If yes, please clarify the previous behavior and the change this PR proposes
- provide the console output, description and/or an example to show the
behavior difference if possible.
If possible, please also clarify if this is a user-facing change compared to
the released Spark versions or within the unreleased branches such as master.
If no, write 'No'.
-->
No
### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some
test cases that check the changes thoroughly including negative and positive
cases if possible.
If it was tested in a way different from regular unit tests, please clarify
how you tested step by step, ideally copy and paste-able, so that other
reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why
it was difficult to add.
If benchmark tests were added, please run the benchmarks in GitHub Actions
for the consistent environment, and the instructions could accord to:
https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
-->
Ran all unit tests.
Wrote some small benchmarking code to verify before/after performance
difference. That code is in the JIRA ticket.
Added unit test based on that benchmarking code that verifies the change.
### Was this patch authored or co-authored using generative AI tooling?
<!--
If generative AI tooling has been used in the process of authoring this
patch, please include the
phrase: 'Generated-by: ' followed by the name of the tool and its version.
If no, write 'No'.
Please refer to the [ASF Generative Tooling
Guidance](https://www.apache.org/legal/generative-tooling.html) for details.
-->
No
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]