Revert "Updating zero capacity resource semantics (#4555)"

This reverts commit 0f42f87ebc.
This commit is contained in:
Devin Petersohn 2019-04-18 11:16:15 -07:00
parent 8f37d49b8b
commit 618147f57f
17 changed files with 193 additions and 248 deletions

View file

@ -7,19 +7,13 @@ import java.util.Map;
* The options class for RayCall or ActorCreation. * The options class for RayCall or ActorCreation.
*/ */
public abstract class BaseTaskOptions { public abstract class BaseTaskOptions {
public final Map<String, Double> resources; public Map<String, Double> resources;
public BaseTaskOptions() { public BaseTaskOptions() {
resources = new HashMap<>(); resources = new HashMap<>();
} }
public BaseTaskOptions(Map<String, Double> resources) { public BaseTaskOptions(Map<String, Double> resources) {
for (Map.Entry<String, Double> entry : resources.entrySet()) {
if (entry.getValue().compareTo(0.0) <= 0) {
throw new IllegalArgumentException(String.format("Resource capacity should be " +
"positive, but got resource %s = %f.", entry.getKey(), entry.getValue()));
}
}
this.resources = resources; this.resources = resources;
} }

View file

@ -14,7 +14,7 @@ ray {
run-mode = CLUSTER run-mode = CLUSTER
// Available resources on this node. // Available resources on this node.
resources: "CPU:4" resources: "CPU:4,GPU:0"
// The address of the redis server to connect, in format `ip:port`. // The address of the redis server to connect, in format `ip:port`.
// If not provided, Ray processes will be started locally, including // If not provided, Ray processes will be started locally, including

View file

@ -32,6 +32,7 @@ import org.ray.runtime.raylet.RayletClient;
import org.ray.runtime.task.ArgumentsBuilder; import org.ray.runtime.task.ArgumentsBuilder;
import org.ray.runtime.task.TaskLanguage; import org.ray.runtime.task.TaskLanguage;
import org.ray.runtime.task.TaskSpec; import org.ray.runtime.task.TaskSpec;
import org.ray.runtime.util.ResourceUtil;
import org.ray.runtime.util.UniqueIdUtil; import org.ray.runtime.util.UniqueIdUtil;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -356,6 +357,11 @@ public abstract class AbstractRayRuntime implements RayRuntime {
resources = new HashMap<>(taskOptions.resources); resources = new HashMap<>(taskOptions.resources);
} }
if (!resources.containsKey(ResourceUtil.CPU_LITERAL)
&& !resources.containsKey(ResourceUtil.CPU_LITERAL.toLowerCase())) {
resources.put(ResourceUtil.CPU_LITERAL, 0.0);
}
int maxActorReconstruction = 0; int maxActorReconstruction = 0;
if (taskOptions instanceof ActorCreationOptions) { if (taskOptions instanceof ActorCreationOptions) {
maxActorReconstruction = ((ActorCreationOptions) taskOptions).maxReconstructions; maxActorReconstruction = ((ActorCreationOptions) taskOptions).maxReconstructions;

View file

@ -104,6 +104,10 @@ public class RayConfig {
+ "setting it to the number of CPU cores: {}", numCpu); + "setting it to the number of CPU cores: {}", numCpu);
resources.put("CPU", numCpu * 1.0); resources.put("CPU", numCpu * 1.0);
} }
if (!resources.containsKey("GPU")) {
LOGGER.warn("No GPU resource is set in configuration, setting it to 0");
resources.put("GPU", 0.0);
}
} }
// Driver id. // Driver id.
String driverId = config.getString("ray.driver.id"); String driverId = config.getString("ray.driver.id");

View file

@ -1,5 +1,5 @@
ray { ray {
run-mode = SINGLE_PROCESS run-mode = SINGLE_PROCESS
resources = "CPU:4" resources = "CPU:4,GPU:0"
redis.address = "" redis.address = ""
} }

View file

@ -46,30 +46,22 @@ public class ResourcesManagementTest extends BaseTest {
@Test @Test
public void testMethods() { public void testMethods() {
TestUtils.skipTestUnderSingleProcess(); TestUtils.skipTestUnderSingleProcess();
CallOptions callOptions1 = new CallOptions(ImmutableMap.of("CPU", 4.0)); CallOptions callOptions1 = new CallOptions(ImmutableMap.of("CPU", 4.0, "GPU", 0.0));
// This is a case that can satisfy required resources. // This is a case that can satisfy required resources.
// The static resources for test are "CPU:4,RES-A:4". // The static resources for test are "CPU:4,RES-A:4".
RayObject<Integer> result1 = Ray.call(ResourcesManagementTest::echo, 100, callOptions1); RayObject<Integer> result1 = Ray.call(ResourcesManagementTest::echo, 100, callOptions1);
Assert.assertEquals(100, (int) result1.get()); Assert.assertEquals(100, (int) result1.get());
CallOptions callOptions2 = new CallOptions(ImmutableMap.of("CPU", 4.0)); CallOptions callOptions2 = new CallOptions(ImmutableMap.of("CPU", 4.0, "GPU", 2.0));
// This is a case that can't satisfy required resources. // This is a case that can't satisfy required resources.
// The static resources for test are "CPU:4,RES-A:4". // The static resources for test are "CPU:4,RES-A:4".
final RayObject<Integer> result2 = Ray.call(ResourcesManagementTest::echo, 200, callOptions2); final RayObject<Integer> result2 = Ray.call(ResourcesManagementTest::echo, 200, callOptions2);
WaitResult<Integer> waitResult = Ray.wait(ImmutableList.of(result2), 1, 1000); WaitResult<Integer> waitResult = Ray.wait(ImmutableList.of(result2), 1, 1000);
Assert.assertEquals(1, waitResult.getReady().size()); Assert.assertEquals(0, waitResult.getReady().size());
Assert.assertEquals(0, waitResult.getUnready().size()); Assert.assertEquals(1, waitResult.getUnready().size());
try {
CallOptions callOptions3 = new CallOptions(ImmutableMap.of("CPU", 0.0));
Assert.fail();
} catch (RuntimeException e) {
// We should receive a RuntimeException indicates that we should not
// pass a zero capacity resource.
}
} }
@Test @Test
@ -77,7 +69,7 @@ public class ResourcesManagementTest extends BaseTest {
TestUtils.skipTestUnderSingleProcess(); TestUtils.skipTestUnderSingleProcess();
ActorCreationOptions actorCreationOptions1 = ActorCreationOptions actorCreationOptions1 =
new ActorCreationOptions(ImmutableMap.of("CPU", 2.0)); new ActorCreationOptions(ImmutableMap.of("CPU", 2.0, "GPU", 0.0));
// This is a case that can satisfy required resources. // This is a case that can satisfy required resources.
// The static resources for test are "CPU:4,RES-A:4". // The static resources for test are "CPU:4,RES-A:4".
@ -88,7 +80,7 @@ public class ResourcesManagementTest extends BaseTest {
// This is a case that can't satisfy required resources. // This is a case that can't satisfy required resources.
// The static resources for test are "CPU:4,RES-A:4". // The static resources for test are "CPU:4,RES-A:4".
ActorCreationOptions actorCreationOptions2 = ActorCreationOptions actorCreationOptions2 =
new ActorCreationOptions(ImmutableMap.of("CPU", 8.0)); new ActorCreationOptions(ImmutableMap.of("CPU", 8.0, "GPU", 0.0));
RayActor<ResourcesManagementTest.Echo> echo2 = RayActor<ResourcesManagementTest.Echo> echo2 =
Ray.createActor(Echo::new, actorCreationOptions2); Ray.createActor(Echo::new, actorCreationOptions2);

View file

@ -44,6 +44,8 @@ cdef class Task:
# Parse the resource map. # Parse the resource map.
if resource_map is not None: if resource_map is not None:
required_resources = resource_map_from_dict(resource_map) required_resources = resource_map_from_dict(resource_map)
if required_resources.count(b"CPU") == 0:
required_resources[b"CPU"] = 1.0
if placement_resource_map is not None: if placement_resource_map is not None:
required_placement_resources = ( required_placement_resources = (
resource_map_from_dict(placement_resource_map)) resource_map_from_dict(placement_resource_map))

View file

@ -1029,25 +1029,14 @@ def check_and_update_resources(num_cpus, num_gpus, resources):
if gpu_ids is not None: if gpu_ids is not None:
resources["GPU"] = min(resources["GPU"], len(gpu_ids)) resources["GPU"] = min(resources["GPU"], len(gpu_ids))
resources = {
resource_label: resource_quantity
for resource_label, resource_quantity in resources.items()
if resource_quantity != 0
}
# Check types. # Check types.
for _, resource_quantity in resources.items(): for _, resource_quantity in resources.items():
assert (isinstance(resource_quantity, int) assert (isinstance(resource_quantity, int)
or isinstance(resource_quantity, float)) or isinstance(resource_quantity, float))
if (isinstance(resource_quantity, float) if (isinstance(resource_quantity, float)
and not resource_quantity.is_integer()): and not resource_quantity.is_integer()):
raise ValueError( raise ValueError("Resource quantities must all be whole numbers.")
"Resource quantities must all be whole numbers. Received {}.".
format(resources))
if resource_quantity < 0:
raise ValueError(
"Resource quantities must be nonnegative. Received {}.".format(
resources))
if resource_quantity > ray_constants.MAX_RESOURCE_QUANTITY: if resource_quantity > ray_constants.MAX_RESOURCE_QUANTITY:
raise ValueError("Resource quantities must be at most {}.".format( raise ValueError("Resource quantities must be at most {}.".format(
ray_constants.MAX_RESOURCE_QUANTITY)) ray_constants.MAX_RESOURCE_QUANTITY))
@ -1124,9 +1113,8 @@ def start_raylet(redis_address,
# Limit the number of workers that can be started in parallel by the # Limit the number of workers that can be started in parallel by the
# raylet. However, make sure it is at least 1. # raylet. However, make sure it is at least 1.
num_cpus_static = static_resources.get("CPU", 0)
maximum_startup_concurrency = max( maximum_startup_concurrency = max(
1, min(multiprocessing.cpu_count(), num_cpus_static)) 1, min(multiprocessing.cpu_count(), static_resources["CPU"]))
# Format the resource argument in a form like 'CPU,1.0,GPU,0,Custom,3'. # Format the resource argument in a form like 'CPU,1.0,GPU,0,Custom,3'.
resource_argument = ",".join( resource_argument = ",".join(

View file

@ -1948,15 +1948,15 @@ def test_multiple_raylets(ray_start_cluster):
store_names = [] store_names = []
store_names += [ store_names += [
client["ObjectStoreSocketName"] for client in client_table client["ObjectStoreSocketName"] for client in client_table
if client["Resources"].get("GPU", 0) == 0 if client["Resources"]["GPU"] == 0
] ]
store_names += [ store_names += [
client["ObjectStoreSocketName"] for client in client_table client["ObjectStoreSocketName"] for client in client_table
if client["Resources"].get("GPU", 0) == 5 if client["Resources"]["GPU"] == 5
] ]
store_names += [ store_names += [
client["ObjectStoreSocketName"] for client in client_table client["ObjectStoreSocketName"] for client in client_table
if client["Resources"].get("GPU", 0) == 1 if client["Resources"]["GPU"] == 1
] ]
assert len(store_names) == 3 assert len(store_names) == 3
@ -2126,32 +2126,6 @@ def test_many_custom_resources(shutdown_only):
ray.get(results) ray.get(results)
def test_zero_capacity_deletion_semantics(shutdown_only):
ray.init(num_cpus=2, num_gpus=1, resources={"test_resource": 1})
def test():
resources = ray.global_state.available_resources()
retry_count = 0
while resources and retry_count < 5:
time.sleep(0.1)
resources = ray.global_state.available_resources()
retry_count += 1
if retry_count >= 5:
raise RuntimeError("Resources were available even after retries.")
return resources
function = ray.remote(
num_cpus=2, num_gpus=1, resources={"test_resource": 1})(test)
cluster_resources = ray.get(function.remote())
# All cluster resources should be utilized and
# cluster_resources must be empty
assert cluster_resources == {}
@pytest.fixture @pytest.fixture
def save_gpu_ids_shutdown_only(): def save_gpu_ids_shutdown_only():
# Record the curent value of this environment variable so that we can # Record the curent value of this environment variable so that we can

View file

@ -51,8 +51,8 @@ def test_uses_resources(ray_start_regular):
while not resource_used: while not resource_used:
available_resources = ray.global_state.available_resources() available_resources = ray.global_state.available_resources()
resource_used = available_resources.get( resource_used = available_resources[
"CPU", 0) == cluster_resources.get("CPU", 0) - 1 "CPU"] == cluster_resources["CPU"] - 1
assert resource_used assert resource_used

View file

@ -11,7 +11,7 @@ import time
import traceback import traceback
import ray import ray
from ray.tune.error import AbortTrialExecution from ray.tune.error import TuneError, AbortTrialExecution
from ray.tune.logger import NoopLogger from ray.tune.logger import NoopLogger
from ray.tune.trial import Trial, Resources, Checkpoint from ray.tune.trial import Trial, Resources, Checkpoint
from ray.tune.trial_executor import TrialExecutor from ray.tune.trial_executor import TrialExecutor
@ -363,22 +363,17 @@ class RayTrialExecutor(TrialExecutor):
resources = ray.services.check_and_update_resources( resources = ray.services.check_and_update_resources(
None, None, None) None, None, None)
if not resources: if not resources:
logger.warning( logger.warning("Cluster resources not detected. Retrying...")
"Cluster resources not detected or are 0. Retrying...")
time.sleep(0.5) time.sleep(0.5)
if not resources: if not resources or "CPU" not in resources:
# NOTE: This hides the possibility that Ray may be waiting for raise TuneError("Cluster resources cannot be detected. "
# clients to connect. "You can resume this experiment by passing in "
resources.setdefault("CPU", 0) "`resume=True` to `run`.")
resources.setdefault("GPU", 0)
logger.warning("Cluster resources cannot be detected or are 0. "
"You can resume this experiment by passing in "
"`resume=True` to `run`.")
resources = resources.copy() resources = resources.copy()
num_cpus = resources.pop("CPU", 0) num_cpus = resources.pop("CPU")
num_gpus = resources.pop("GPU", 0) num_gpus = resources.pop("GPU")
custom_resources = resources custom_resources = resources
self._avail_resources = Resources( self._avail_resources = Resources(

View file

@ -653,13 +653,6 @@ class Worker(object):
raise ValueError( raise ValueError(
"Resource quantities must all be whole numbers.") "Resource quantities must all be whole numbers.")
# Remove any resources with zero quantity requirements
resources = {
resource_label: resource_quantity
for resource_label, resource_quantity in resources.items()
if resource_quantity > 0
}
if placement_resources is None: if placement_resources is None:
placement_resources = {} placement_resources = {}
@ -1877,7 +1870,7 @@ def connect(node,
nil_actor_counter, # actor_counter. nil_actor_counter, # actor_counter.
[], # new_actor_handles. [], # new_actor_handles.
[], # execution_dependencies. [], # execution_dependencies.
{}, # resource_map. {"CPU": 0}, # resource_map.
{}, # placement_resource_map. {}, # placement_resource_map.
) )

View file

@ -1469,16 +1469,15 @@ void NodeManager::HandleTaskBlocked(const std::shared_ptr<LocalClientConnection>
// Get the CPU resources required by the running task. // Get the CPU resources required by the running task.
const auto required_resources = task.GetTaskSpecification().GetRequiredResources(); const auto required_resources = task.GetTaskSpecification().GetRequiredResources();
double required_cpus = required_resources.GetNumCpus(); double required_cpus = required_resources.GetNumCpus();
std::unordered_map<std::string, double> cpu_resources; const std::unordered_map<std::string, double> cpu_resources = {
if (required_cpus > 0) { {kCPU_ResourceLabel, required_cpus}};
cpu_resources[kCPU_ResourceLabel] = required_cpus;
}
// Release the CPU resources. // Release the CPU resources.
auto const cpu_resource_ids = worker->ReleaseTaskCpuResources(); auto const cpu_resource_ids = worker->ReleaseTaskCpuResources();
local_available_resources_.Release(cpu_resource_ids); local_available_resources_.Release(cpu_resource_ids);
cluster_resource_map_[gcs_client_->client_table().GetLocalClientId()].Release( RAY_CHECK(
ResourceSet(cpu_resources)); cluster_resource_map_[gcs_client_->client_table().GetLocalClientId()].Release(
ResourceSet(cpu_resources)));
worker->MarkBlocked(); worker->MarkBlocked();
// Try dispatching tasks since we may have released some resources. // Try dispatching tasks since we may have released some resources.
@ -1522,11 +1521,9 @@ void NodeManager::HandleTaskUnblocked(
// Get the CPU resources required by the running task. // Get the CPU resources required by the running task.
const auto required_resources = task.GetTaskSpecification().GetRequiredResources(); const auto required_resources = task.GetTaskSpecification().GetRequiredResources();
double required_cpus = required_resources.GetNumCpus(); double required_cpus = required_resources.GetNumCpus();
std::unordered_map<std::string, double> cpu_resources_map; const ResourceSet cpu_resources(
if (required_cpus > 0) { std::unordered_map<std::string, double>({{kCPU_ResourceLabel, required_cpus}}));
cpu_resources_map[kCPU_ResourceLabel] = required_cpus;
}
const ResourceSet cpu_resources(cpu_resources_map);
// Check if we can reacquire the CPU resources. // Check if we can reacquire the CPU resources.
bool oversubscribed = !local_available_resources_.Contains(cpu_resources); bool oversubscribed = !local_available_resources_.Contains(cpu_resources);
@ -1536,8 +1533,9 @@ void NodeManager::HandleTaskUnblocked(
// reacquire here may be different from the ones that the task started with. // reacquire here may be different from the ones that the task started with.
auto const resource_ids = local_available_resources_.Acquire(cpu_resources); auto const resource_ids = local_available_resources_.Acquire(cpu_resources);
worker->AcquireTaskCpuResources(resource_ids); worker->AcquireTaskCpuResources(resource_ids);
cluster_resource_map_[gcs_client_->client_table().GetLocalClientId()].Acquire( RAY_CHECK(
cpu_resources); cluster_resource_map_[gcs_client_->client_table().GetLocalClientId()].Acquire(
cpu_resources));
} else { } else {
// In this case, we simply don't reacquire the CPU resources for the worker. // In this case, we simply don't reacquire the CPU resources for the worker.
// The worker can keep running and when the task finishes, it will simply // The worker can keep running and when the task finishes, it will simply
@ -1629,7 +1627,7 @@ bool NodeManager::AssignTask(const Task &task) {
auto acquired_resources = auto acquired_resources =
local_available_resources_.Acquire(spec.GetRequiredResources()); local_available_resources_.Acquire(spec.GetRequiredResources());
const auto &my_client_id = gcs_client_->client_table().GetLocalClientId(); const auto &my_client_id = gcs_client_->client_table().GetLocalClientId();
cluster_resource_map_[my_client_id].Acquire(spec.GetRequiredResources()); RAY_CHECK(cluster_resource_map_[my_client_id].Acquire(spec.GetRequiredResources()));
if (spec.IsActorCreationTask()) { if (spec.IsActorCreationTask()) {
// Check that we are not placing an actor creation task on a node with 0 CPUs. // Check that we are not placing an actor creation task on a node with 0 CPUs.
@ -1743,8 +1741,8 @@ void NodeManager::FinishAssignedTask(Worker &worker) {
// Release task's resources. The worker's lifetime resources are still held. // Release task's resources. The worker's lifetime resources are still held.
auto const &task_resources = worker.GetTaskResourceIds(); auto const &task_resources = worker.GetTaskResourceIds();
local_available_resources_.Release(task_resources); local_available_resources_.Release(task_resources);
cluster_resource_map_[gcs_client_->client_table().GetLocalClientId()].Release( RAY_CHECK(cluster_resource_map_[gcs_client_->client_table().GetLocalClientId()].Release(
task_resources.ToResourceSet()); task_resources.ToResourceSet()));
worker.ResetTaskResourceIds(); worker.ResetTaskResourceIds();
// If this was an actor or actor creation task, handle the actor's new state. // If this was an actor or actor creation task, handle the actor's new state.
@ -2036,9 +2034,6 @@ void NodeManager::ForwardTaskOrResubmit(const Task &task,
RAY_LOG(INFO) << "Failed to forward task " << task_id << " to node manager " RAY_LOG(INFO) << "Failed to forward task " << task_id << " to node manager "
<< node_manager_id; << node_manager_id;
// TODO(romilb): We should probably revert the load subtraction from
// SchedulingPolicy::Schedule()
// Mark the failed task as pending to let other raylets know that we still // Mark the failed task as pending to let other raylets know that we still
// have the task. TaskDependencyManager::TaskPending() is assumed to be // have the task. TaskDependencyManager::TaskPending() is assumed to be
// idempotent. // idempotent.

View file

@ -49,13 +49,11 @@ std::unordered_map<TaskID, ClientID> SchedulingPolicy::Schedule(
const auto &node_resources = client_resource_pair.second; const auto &node_resources = client_resource_pair.second;
ResourceSet available_node_resources = ResourceSet available_node_resources =
ResourceSet(node_resources.GetAvailableResources()); ResourceSet(node_resources.GetAvailableResources());
// TODO(romilb): Why do we need to subtract load from available resources? available_node_resources.SubtractResourcesStrict(node_resources.GetLoadResources());
// Even if we don't the code path below for choosing a dst_client_id would be
// similar.
available_node_resources.SubtractResources(node_resources.GetLoadResources());
RAY_LOG(DEBUG) << "client_id " << node_client_id RAY_LOG(DEBUG) << "client_id " << node_client_id
<< " avail: " << node_resources.GetAvailableResources().ToString() << " avail: " << node_resources.GetAvailableResources().ToString()
<< " load: " << node_resources.GetLoadResources().ToString(); << " load: " << node_resources.GetLoadResources().ToString()
<< " avail-load: " << available_node_resources.ToString();
if (resource_demand.IsSubset(available_node_resources)) { if (resource_demand.IsSubset(available_node_resources)) {
// This node is a feasible candidate. // This node is a feasible candidate.

View file

@ -9,39 +9,16 @@ namespace ray {
namespace raylet { namespace raylet {
bool EqualsZeroEpsilon(double quantity) {
if ((quantity <= EPSILON) && (quantity >= -1 * EPSILON)) {
return true;
}
return false;
}
bool EqualsOneEpsilon(double quantity) {
if ((quantity <= 1 + EPSILON) && (quantity >= 1 - EPSILON)) {
return true;
}
return false;
}
ResourceSet::ResourceSet() {} ResourceSet::ResourceSet() {}
ResourceSet::ResourceSet(const std::unordered_map<std::string, double> &resource_map) ResourceSet::ResourceSet(const std::unordered_map<std::string, double> &resource_map)
: resource_capacity_(resource_map) { : resource_capacity_(resource_map) {}
for (auto const &resource_pair : resource_capacity_) {
RAY_CHECK(resource_pair.second > 0 + EPSILON)
<< "Resource " << resource_pair.first << " capacity is " << resource_pair.second
<< ". Should have been greater than zero.";
}
}
ResourceSet::ResourceSet(const std::vector<std::string> &resource_labels, ResourceSet::ResourceSet(const std::vector<std::string> &resource_labels,
const std::vector<double> resource_capacity) { const std::vector<double> resource_capacity) {
RAY_CHECK(resource_labels.size() == resource_capacity.size()); RAY_CHECK(resource_labels.size() == resource_capacity.size());
for (uint i = 0; i < resource_labels.size(); i++) { for (uint i = 0; i < resource_labels.size(); i++) {
RAY_CHECK(resource_capacity[i] > 0 + EPSILON) RAY_CHECK(AddResource(resource_labels[i], resource_capacity[i]));
<< "Resource " << resource_labels[i] << " capacity is " << resource_capacity[i]
<< ". Should have been greater than zero.";
resource_capacity_[resource_labels[i]] = resource_capacity[i];
} }
} }
@ -53,7 +30,13 @@ bool ResourceSet::operator==(const ResourceSet &rhs) const {
bool ResourceSet::IsEmpty() const { bool ResourceSet::IsEmpty() const {
// Check whether the capacity of each resource type is zero. Exit early if not. // Check whether the capacity of each resource type is zero. Exit early if not.
return resource_capacity_.empty(); if (resource_capacity_.empty()) return true;
for (const auto &resource_pair : resource_capacity_) {
if (resource_pair.second > 0) {
return false;
}
}
return true;
} }
bool ResourceSet::IsSubset(const ResourceSet &other) const { bool ResourceSet::IsSubset(const ResourceSet &other) const {
@ -61,7 +44,11 @@ bool ResourceSet::IsSubset(const ResourceSet &other) const {
for (const auto &resource_pair : resource_capacity_) { for (const auto &resource_pair : resource_capacity_) {
const auto &resource_name = resource_pair.first; const auto &resource_name = resource_pair.first;
const double lhs_quantity = resource_pair.second; const double lhs_quantity = resource_pair.second;
double rhs_quantity = other.GetResource(resource_name); double rhs_quantity = 0;
if (!other.GetResource(resource_name, &rhs_quantity)) {
// Resource not found in rhs, therefore lhs is not a subset of rhs.
return false;
}
if (lhs_quantity > rhs_quantity) { if (lhs_quantity > rhs_quantity) {
// Resource found in rhs, but lhs capacity exceeds rhs capacity. // Resource found in rhs, but lhs capacity exceeds rhs capacity.
return false; return false;
@ -80,42 +67,41 @@ bool ResourceSet::IsEqual(const ResourceSet &rhs) const {
return (this->IsSubset(rhs) && rhs.IsSubset(*this)); return (this->IsSubset(rhs) && rhs.IsSubset(*this));
} }
bool ResourceSet::AddResource(const std::string &resource_name, double capacity) {
resource_capacity_[resource_name] = capacity;
return true;
}
bool ResourceSet::RemoveResource(const std::string &resource_name) { bool ResourceSet::RemoveResource(const std::string &resource_name) {
throw std::runtime_error("Method not implemented"); throw std::runtime_error("Method not implemented");
} }
void ResourceSet::SubtractResources(const ResourceSet &other) {
// Subtract the resources and delete any if new capacity is zero.
for (const auto &resource_pair : other.GetResourceMap()) {
const std::string &resource_label = resource_pair.first;
const double &resource_capacity = resource_pair.second;
if (resource_capacity_.count(resource_label) == 1) {
resource_capacity_[resource_label] -= resource_capacity;
if (resource_capacity_[resource_label] < 0 + EPSILON) {
resource_capacity_.erase(resource_label);
}
}
}
}
void ResourceSet::SubtractResourcesStrict(const ResourceSet &other) { bool ResourceSet::SubtractResourcesStrict(const ResourceSet &other) {
// Subtract the resources, make sure none goes below zero and delete any if new capacity // Subtract the resources and track whether a resource goes below zero.
// is zero. bool oversubscribed = false;
for (const auto &resource_pair : other.GetResourceMap()) { for (const auto &resource_pair : other.GetResourceMap()) {
const std::string &resource_label = resource_pair.first; const std::string &resource_label = resource_pair.first;
const double &resource_capacity = resource_pair.second; const double &resource_capacity = resource_pair.second;
RAY_CHECK(resource_capacity_.count(resource_label) == 1) RAY_CHECK(resource_capacity_.count(resource_label) == 1)
<< "Attempt to acquire unknown resource: " << resource_label; << "Attempt to acquire unknown resource: " << resource_label;
resource_capacity_[resource_label] -= resource_capacity; resource_capacity_[resource_label] -= resource_capacity;
// TODO(romilb): Double precision subtraction may sometimes be less than zero by a if (resource_capacity_[resource_label] < 0) {
// small epsilon - need to fix. oversubscribed = true;
RAY_CHECK(resource_capacity_[resource_label] >= 0 - EPSILON)
<< "Capacity of resource " << resource_label << " after subtraction is negative ("
<< resource_capacity_[resource_label] << ")."
<< " Debug: resource_capacity_:" << ToString() << ", other: " << other.ToString();
if (EqualsZeroEpsilon(resource_capacity_[resource_label])) {
resource_capacity_.erase(resource_label);
} }
} }
return !oversubscribed;
}
// Perform a left join.
bool ResourceSet::AddResourcesStrict(const ResourceSet &other) {
// Return failure if attempting to perform vector addition with unknown labels.
for (const auto &resource_pair : other.GetResourceMap()) {
const std::string &resource_label = resource_pair.first;
const double &resource_capacity = resource_pair.second;
RAY_CHECK(resource_capacity_.count(resource_label) != 0);
resource_capacity_[resource_label] += resource_capacity;
}
return true;
} }
// Perform an outer join. // Perform an outer join.
@ -123,22 +109,33 @@ void ResourceSet::AddResources(const ResourceSet &other) {
for (const auto &resource_pair : other.GetResourceMap()) { for (const auto &resource_pair : other.GetResourceMap()) {
const std::string &resource_label = resource_pair.first; const std::string &resource_label = resource_pair.first;
const double &resource_capacity = resource_pair.second; const double &resource_capacity = resource_pair.second;
resource_capacity_[resource_label] += resource_capacity; if (resource_capacity_.count(resource_label) == 0) {
// Add the new label if not found.
RAY_CHECK(AddResource(resource_label, resource_capacity));
} else {
// Increment the resource by its capacity.
resource_capacity_[resource_label] += resource_capacity;
}
} }
} }
double ResourceSet::GetResource(const std::string &resource_name) const { bool ResourceSet::GetResource(const std::string &resource_name, double *value) const {
if (EqualsZeroEpsilon(resource_capacity_.count(resource_name))) { if (!value) {
return 0; return false;
} }
double capacity = resource_capacity_.at(resource_name); if (resource_capacity_.count(resource_name) == 0) {
RAY_CHECK(capacity > 0 + EPSILON) << "Resource " << resource_name << " capacity is " *value = std::nan("");
<< capacity return false;
<< ". Should have been greater than zero."; }
return capacity; *value = resource_capacity_.at(resource_name);
return true;
} }
double ResourceSet::GetNumCpus() const { return GetResource(kCPU_ResourceLabel); } double ResourceSet::GetNumCpus() const {
double num_cpus;
RAY_CHECK(GetResource(kCPU_ResourceLabel, &num_cpus));
return num_cpus;
}
const std::string ResourceSet::ToString() const { const std::string ResourceSet::ToString() const {
std::string return_string = ""; std::string return_string = "";
@ -225,12 +222,6 @@ ResourceIds ResourceIds::Acquire(double resource_quantity) {
if (fractional_pair.second >= resource_quantity) { if (fractional_pair.second >= resource_quantity) {
auto return_pair = std::make_pair(fractional_pair.first, resource_quantity); auto return_pair = std::make_pair(fractional_pair.first, resource_quantity);
fractional_pair.second -= resource_quantity; fractional_pair.second -= resource_quantity;
// Remove the fractional pair if the new capacity is 0
if (EqualsZeroEpsilon(fractional_pair.second)) {
std::swap(fractional_pair, fractional_ids_[fractional_ids_.size() - 1]);
fractional_ids_.pop_back();
}
return ResourceIds({return_pair}); return ResourceIds({return_pair});
} }
} }
@ -266,18 +257,15 @@ void ResourceIds::Release(const ResourceIds &resource_ids) {
if (fractional_pair_it == fractional_ids_.end()) { if (fractional_pair_it == fractional_ids_.end()) {
fractional_ids_.push_back(fractional_pair_to_return); fractional_ids_.push_back(fractional_pair_to_return);
} else { } else {
RAY_CHECK(fractional_pair_it->second < 1);
fractional_pair_it->second += fractional_pair_to_return.second; fractional_pair_it->second += fractional_pair_to_return.second;
// TODO(romilb): Double precision addition may sometimes exceed 1 by a small epsilon
// - need to fix this.
RAY_CHECK(fractional_pair_it->second <= 1 + EPSILON)
<< "Fractional Resource Id " << fractional_pair_it->first << " capacity is "
<< fractional_pair_it->second << ". Should have been less than one.";
// If this makes the ID whole, then return it to the list of whole IDs. // If this makes the ID whole, then return it to the list of whole IDs.
// TODO(romilb): Double precision addition may sometimes exceed 1 by a small epsilon if (fractional_pair_it->second >= 1) {
// - need to fix this.
if (EqualsOneEpsilon(fractional_pair_it->second)) {
whole_ids_.push_back(resource_id); whole_ids_.push_back(resource_id);
fractional_ids_.erase(fractional_pair_it); fractional_pair_it->second -= 1;
if (fractional_pair_it->second < 1e-6) {
fractional_ids_.erase(fractional_pair_it);
}
} }
} }
} }
@ -295,10 +283,6 @@ const std::vector<std::pair<int64_t, double>> &ResourceIds::FractionalIds() cons
return fractional_ids_; return fractional_ids_;
} }
bool ResourceIds::TotalQuantityIsZero() const {
return whole_ids_.empty() && fractional_ids_.empty();
}
double ResourceIds::TotalQuantity() const { double ResourceIds::TotalQuantity() const {
double total_quantity = whole_ids_.size(); double total_quantity = whole_ids_.size();
for (auto const &fractional_pair : fractional_ids_) { for (auto const &fractional_pair : fractional_ids_) {
@ -346,9 +330,9 @@ bool ResourceIdSet::Contains(const ResourceSet &resource_set) const {
for (auto const &resource_pair : resource_set.GetResourceMap()) { for (auto const &resource_pair : resource_set.GetResourceMap()) {
auto const &resource_name = resource_pair.first; auto const &resource_name = resource_pair.first;
double resource_quantity = resource_pair.second; double resource_quantity = resource_pair.second;
RAY_CHECK(resource_quantity > 0 + EPSILON) << "Resource " << resource_name if (resource_quantity == 0) {
<< " capacity is " << resource_quantity continue;
<< ". Should have been greater than zero."; }
auto it = available_resources_.find(resource_name); auto it = available_resources_.find(resource_name);
if (it == available_resources_.end()) { if (it == available_resources_.end()) {
@ -368,16 +352,14 @@ ResourceIdSet ResourceIdSet::Acquire(const ResourceSet &resource_set) {
for (auto const &resource_pair : resource_set.GetResourceMap()) { for (auto const &resource_pair : resource_set.GetResourceMap()) {
auto const &resource_name = resource_pair.first; auto const &resource_name = resource_pair.first;
double resource_quantity = resource_pair.second; double resource_quantity = resource_pair.second;
RAY_CHECK(resource_quantity > 0 + EPSILON) << "Resource " << resource_name
<< " capacity is " << resource_quantity if (resource_quantity == 0) {
<< ". Should have been greater than zero."; continue;
}
auto it = available_resources_.find(resource_name); auto it = available_resources_.find(resource_name);
RAY_CHECK(it != available_resources_.end()); RAY_CHECK(it != available_resources_.end());
acquired_resources[resource_name] = it->second.Acquire(resource_quantity); acquired_resources[resource_name] = it->second.Acquire(resource_quantity);
if (it->second.TotalQuantityIsZero()) {
available_resources_.erase(it);
}
} }
return ResourceIdSet(acquired_resources); return ResourceIdSet(acquired_resources);
} }
@ -386,10 +368,15 @@ void ResourceIdSet::Release(const ResourceIdSet &resource_id_set) {
for (auto const &resource_pair : resource_id_set.AvailableResources()) { for (auto const &resource_pair : resource_id_set.AvailableResources()) {
auto const &resource_name = resource_pair.first; auto const &resource_name = resource_pair.first;
auto const &resource_ids = resource_pair.second; auto const &resource_ids = resource_pair.second;
RAY_CHECK(!resource_ids.TotalQuantityIsZero());
if (resource_ids.TotalQuantity() == 0) {
continue;
}
auto it = available_resources_.find(resource_name); auto it = available_resources_.find(resource_name);
if (it == available_resources_.end()) { if (it == available_resources_.end()) {
// This should not happen when Release is called on resources that were obtained
// through a corresponding call to Acquire.
available_resources_[resource_name] = resource_ids; available_resources_[resource_name] = resource_ids;
} else { } else {
it->second.Release(resource_ids); it->second.Release(resource_ids);
@ -487,6 +474,18 @@ SchedulingResources::SchedulingResources(const ResourceSet &total)
SchedulingResources::~SchedulingResources() {} SchedulingResources::~SchedulingResources() {}
ResourceAvailabilityStatus SchedulingResources::CheckResourcesSatisfied(
ResourceSet &resources) const {
if (!resources.IsSubset(resources_total_)) {
return ResourceAvailabilityStatus::kInfeasible;
}
// Resource demand specified is feasible. Check if it's available.
if (!resources.IsSubset(resources_available_)) {
return ResourceAvailabilityStatus::kResourcesUnavailable;
}
return ResourceAvailabilityStatus::kFeasible;
}
const ResourceSet &SchedulingResources::GetAvailableResources() const { const ResourceSet &SchedulingResources::GetAvailableResources() const {
return resources_available_; return resources_available_;
} }
@ -508,13 +507,13 @@ const ResourceSet &SchedulingResources::GetLoadResources() const {
} }
// Return specified resources back to SchedulingResources. // Return specified resources back to SchedulingResources.
void SchedulingResources::Release(const ResourceSet &resources) { bool SchedulingResources::Release(const ResourceSet &resources) {
resources_available_.AddResources(resources); return resources_available_.AddResourcesStrict(resources);
} }
// Take specified resources from SchedulingResources. // Take specified resources from SchedulingResources.
void SchedulingResources::Acquire(const ResourceSet &resources) { bool SchedulingResources::Acquire(const ResourceSet &resources) {
resources_available_.SubtractResourcesStrict(resources); return resources_available_.SubtractResourcesStrict(resources);
} }
std::string SchedulingResources::DebugString() const { std::string SchedulingResources::DebugString() const {

View file

@ -8,25 +8,19 @@
#include "ray/raylet/format/node_manager_generated.h" #include "ray/raylet/format/node_manager_generated.h"
#define EPSILON 0.00001
namespace ray { namespace ray {
namespace raylet { namespace raylet {
const std::string kCPU_ResourceLabel = "CPU"; const std::string kCPU_ResourceLabel = "CPU";
/// \brief Test if the quantity is within epsilon bounds of 0. /// Resource availability status reports whether the resource requirement is
/// /// (1) infeasible, (2) feasible but currently unavailable, or (3) available.
/// \param quantity: Quantity to check enum class ResourceAvailabilityStatus : int {
/// \return True if -epsilon <= Quantity <= epsilon, False otherwise. kInfeasible, ///< Cannot ever satisfy resource requirements.
bool EqualsZeroEpsilon(double quantity); kResourcesUnavailable, ///< Feasible, but not currently available.
kFeasible ///< Feasible and currently available.
/// \brief Test if the quantity is within epsilon bounds of 1. };
///
/// \param quantity: Quantity to check
/// \return True if 1 - epsilon <= Quantity <= 1 + epsilon, False otherwise.
bool EqualsOneEpsilon(double quantity);
/// \class ResourceSet /// \class ResourceSet
/// \brief Encapsulates and operates on a set of resources, including CPUs, /// \brief Encapsulates and operates on a set of resources, including CPUs,
@ -73,12 +67,26 @@ class ResourceSet {
/// False otherwise. /// False otherwise.
bool IsSuperset(const ResourceSet &other) const; bool IsSuperset(const ResourceSet &other) const;
/// \brief Add a new resource to the resource set.
///
/// \param resource_name: name/label of the resource to add.
/// \param capacity: numeric capacity value for the resource to add.
/// \return True, if the resource was successfully added. False otherwise.
bool AddResource(const std::string &resource_name, double capacity);
/// \brief Remove the specified resource from the resource set. /// \brief Remove the specified resource from the resource set.
/// ///
/// \param resource_name: name/label of the resource to remove. /// \param resource_name: name/label of the resource to remove.
/// \return True, if the resource was successfully removed. False otherwise. /// \return True, if the resource was successfully removed. False otherwise.
bool RemoveResource(const std::string &resource_name); bool RemoveResource(const std::string &resource_name);
/// \brief Add a set of resources to the current set of resources only if the resource
/// labels match.
///
/// \param other: The other resource set to add.
/// \return True if the resource set was added successfully. False otherwise.
bool AddResourcesStrict(const ResourceSet &other);
/// \brief Aggregate resources from the other set into this set, adding any missing /// \brief Aggregate resources from the other set into this set, adding any missing
/// resource labels to this set. /// resource labels to this set.
/// ///
@ -86,28 +94,21 @@ class ResourceSet {
/// \return Void. /// \return Void.
void AddResources(const ResourceSet &other); void AddResources(const ResourceSet &other);
/// \brief Subtract a set of resources from the current set of resources. /// \brief Subtract a set of resources from the current set of resources, only if
/// Deletes any resource if the capacity after subtraction is zero or negative. /// resource labels match.
/// ///
/// \param other: The resource set to subtract from the current resource set. /// \param other: The resource set to subtract from the current resource set.
/// \return Void. /// \return True if the resource set was subtracted successfully.
void SubtractResources(const ResourceSet &other); /// False otherwise.
bool SubtractResourcesStrict(const ResourceSet &other);
/// \brief Subtract a set of resources from the current set of resources and
/// check that the post-subtraction result nonnegative. Assumes other
/// is a subset of the ResourceSet. Deletes any resource if the capacity after
/// subtraction is zero.
///
/// \param other: The resource set to subtract from the current resource set.
/// \return Void.
void SubtractResourcesStrict(const ResourceSet &other);
/// Return the capacity value associated with the specified resource. /// Return the capacity value associated with the specified resource.
/// ///
/// \param resource_name: Resource name for which capacity is requested. /// \param resource_name: Resource name for which capacity is requested.
/// \return The capacity value associated with the specified resource, zero if resource /// \param[out] value: Resource capacity value.
/// does not exist. /// \return True if the resource capacity value was successfully retrieved.
double GetResource(const std::string &resource_name) const; /// False otherwise.
bool GetResource(const std::string &resource_name, double *value) const;
/// Return the number of CPUs. /// Return the number of CPUs.
/// ///
@ -125,7 +126,7 @@ class ResourceSet {
const std::string ToString() const; const std::string ToString() const;
private: private:
/// Resource capacity map. The capacities (double) are always positive. /// Resource capacity map.
std::unordered_map<std::string, double> resource_capacity_; std::unordered_map<std::string, double> resource_capacity_;
}; };
@ -204,11 +205,6 @@ class ResourceIds {
/// \return The fractional IDs. /// \return The fractional IDs.
const std::vector<std::pair<int64_t, double>> &FractionalIds() const; const std::vector<std::pair<int64_t, double>> &FractionalIds() const;
/// \brief Check if ResourceIds has any resources.
///
/// \return True if there are no whole or fractional resources. False otherwise.
bool TotalQuantityIsZero() const;
/// \brief Return the total quantity of resources, ignoring the specific IDs. /// \brief Return the total quantity of resources, ignoring the specific IDs.
/// ///
/// \return The total quantity of the resource. /// \return The total quantity of the resource.
@ -333,6 +329,13 @@ class SchedulingResources {
/// \brief SchedulingResources destructor. /// \brief SchedulingResources destructor.
~SchedulingResources(); ~SchedulingResources();
/// \brief Check if the specified resource request can be satisfied.
///
/// \param set: The set of resources representing the resource request.
/// \return Availability status that specifies if the requested resource set
/// is feasible, infeasible, or feasible but unavailable.
ResourceAvailabilityStatus CheckResourcesSatisfied(ResourceSet &set) const;
/// \brief Request the set and capacity of resources currently available. /// \brief Request the set and capacity of resources currently available.
/// ///
/// \return Immutable set of resources with currently available capacity. /// \return Immutable set of resources with currently available capacity.
@ -360,14 +363,16 @@ class SchedulingResources {
/// \brief Release the amount of resources specified. /// \brief Release the amount of resources specified.
/// ///
/// \param resources: the amount of resources to be released. /// \param resources: the amount of resources to be released.
/// \return Void. /// \return True if resources were successfully released. False otherwise.
void Release(const ResourceSet &resources); bool Release(const ResourceSet &resources);
/// \brief Acquire the amount of resources specified. /// \brief Acquire the amount of resources specified.
/// ///
/// \param resources: the amount of resources to be acquired. /// \param resources: the amount of resources to be acquired.
/// \return Void. /// \return True if resources were acquired without oversubscription. If this
void Acquire(const ResourceSet &resources); /// returns false, then the resources were still acquired, but we are now at
/// negative resources.
bool Acquire(const ResourceSet &resources);
/// Returns debug string for class. /// Returns debug string for class.
/// ///

View file

@ -77,7 +77,7 @@ static inline TaskSpecification ExampleTaskSpec(
std::vector<std::string> function_descriptor(3); std::vector<std::string> function_descriptor(3);
return TaskSpecification(DriverID::nil(), TaskID::nil(), 0, ActorID::nil(), return TaskSpecification(DriverID::nil(), TaskID::nil(), 0, ActorID::nil(),
ObjectID::nil(), 0, actor_id, ActorHandleID::nil(), 0, {}, {}, ObjectID::nil(), 0, actor_id, ActorHandleID::nil(), 0, {}, {},
0, {}, {}, language, function_descriptor); 0, {{}}, {{}}, language, function_descriptor);
} }
TEST_F(WorkerPoolTest, HandleWorkerRegistration) { TEST_F(WorkerPoolTest, HandleWorkerRegistration) {