mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
[core] Supress infeasible warning (#14068)
This commit is contained in:
parent
ea7d4c6607
commit
e1fbb8489e
4 changed files with 22 additions and 15 deletions
|
@ -162,6 +162,12 @@ Let's create a placement group. Recall that each bundle is a collection of resou
|
|||
# Wait until placement group is created.
|
||||
ray.get(pg.ready())
|
||||
|
||||
.. note::
|
||||
|
||||
When using placement groups, users should ensure that their placement groups are ready (by calling ``ray.get(pg.ready())``)
|
||||
and have the proper resources. Ray assumes that the placement group will be properly created and does *not*
|
||||
print a warning about infeasible tasks.
|
||||
|
||||
Now let's define an actor that uses GPU. We'll also define a task that use ``extra_resources``.
|
||||
|
||||
.. code-block:: python
|
||||
|
@ -229,7 +235,7 @@ because they are scheduled on a placement group with the STRICT_PACK strategy.
|
|||
# scheduled with the parent's placement group.
|
||||
ray.get(child.options(placement_group=None).remote())
|
||||
|
||||
Note that you can anytime remove the placement group to clean up resources.
|
||||
You can remove a placement group at any time to free its allocated resources.
|
||||
|
||||
.. code-block:: python
|
||||
|
||||
|
|
|
@ -15,6 +15,7 @@ import ray
|
|||
from ray.experimental.internal_kv import _internal_kv_get
|
||||
from ray.autoscaler._private.util import DEBUG_AUTOSCALING_ERROR
|
||||
import ray.utils
|
||||
from ray.util.placement_group import placement_group
|
||||
import ray.ray_constants as ray_constants
|
||||
from ray.exceptions import RayTaskError
|
||||
from ray.cluster_utils import Cluster
|
||||
|
@ -707,6 +708,15 @@ def test_warning_for_infeasible_tasks(ray_start_regular, error_pubsub):
|
|||
assert len(errors) == 1
|
||||
assert errors[0].type == ray_constants.INFEASIBLE_TASK_ERROR
|
||||
|
||||
# Placement group cannot be made, but no warnings should occur.
|
||||
pg = placement_group([{"GPU": 1}], strategy="STRICT_PACK")
|
||||
pg.ready()
|
||||
f.options(placement_group=pg).remote()
|
||||
|
||||
errors = get_error_message(
|
||||
p, 1, ray_constants.INFEASIBLE_TASK_ERROR, timeout=5)
|
||||
assert len(errors) == 0, errors
|
||||
|
||||
|
||||
def test_warning_for_infeasible_zero_cpu_actor(shutdown_only):
|
||||
# Check that we cannot place an actor on a 0 CPU machine and that we get an
|
||||
|
|
|
@ -109,9 +109,7 @@ class PlacementGroup:
|
|||
return len(self.bundle_cache)
|
||||
|
||||
def _get_none_zero_resource(self, bundle: List[Dict]):
|
||||
# This number shouldn't be changed.
|
||||
# When it is specified, node manager won't warn about infeasible
|
||||
# tasks.
|
||||
# Any number between 0-1 should be fine.
|
||||
INFEASIBLE_TASK_SUPPRESS_MAGIC_NUMBER = 0.0101
|
||||
for key, value in bundle.items():
|
||||
if value > 0:
|
||||
|
|
|
@ -2156,18 +2156,11 @@ void NodeManager::RecordMetrics() {
|
|||
}
|
||||
|
||||
void NodeManager::PublishInfeasibleTaskError(const Task &task) const {
|
||||
// This block is used to suppress infeasible task warning.
|
||||
bool suppress_warning = false;
|
||||
const auto &required_resources = task.GetTaskSpecification().GetRequiredResources();
|
||||
const auto &resources_map = required_resources.GetResourceMap();
|
||||
const auto &it = resources_map.begin();
|
||||
// It is a hack to suppress infeasible task warning.
|
||||
// If the first resource of a task requires this magic number, infeasible warning is
|
||||
// suppressed. It is currently only used by placement group ready API. We don't want
|
||||
// to have this in ray_config_def.h because the use case is very narrow, and we don't
|
||||
// want to expose this anywhere.
|
||||
double INFEASIBLE_TASK_SUPPRESS_MAGIC_NUMBER = 0.0101;
|
||||
if (it != resources_map.end() && it->second == INFEASIBLE_TASK_SUPPRESS_MAGIC_NUMBER) {
|
||||
|
||||
if (!task.GetTaskSpecification().PlacementGroupBundleId().first.IsNil()) {
|
||||
// If the task is part of a placement group, do nothing. If necessary, the infeasible
|
||||
// warning should come from the placement group scheduling, not the task scheduling.
|
||||
suppress_warning = true;
|
||||
}
|
||||
|
||||
|
|
Loading…
Add table
Reference in a new issue