Skip to content

Introduce concept of resources required for placing a task.#2837

Merged
atumanov merged 12 commits intoray-project:masterfrom
robertnishihara:placementresources
Oct 4, 2018
Merged

Introduce concept of resources required for placing a task.#2837
atumanov merged 12 commits intoray-project:masterfrom
robertnishihara:placementresources

Conversation

@robertnishihara
Copy link
Copy Markdown
Collaborator

This PR introduces the concept of "resources that are required for scheduling/placing a task", which are distinct from (and a superset of) "resources acquired during task execution". Typically, these two things are the same, however, in the case of actor creation tasks, we require additional resources to schedule the task on a node because it must take into account the future resource requirements of the actor methods. This fixes #2803.

@robertnishihara
Copy link
Copy Markdown
Collaborator Author

robertnishihara commented Sep 7, 2018

@atumanov @ericl please take a look.

@robertnishihara robertnishihara changed the title Introduce concept of resources required for placement. Introduce concept of resources required for placing a task. Sep 7, 2018
@AmplabJenkins
Copy link
Copy Markdown

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Ray-PRB/8045/
Test PASSed.

@AmplabJenkins
Copy link
Copy Markdown

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Ray-PRB/8046/
Test FAILed.

Comment thread src/ray/raylet/node_manager.cc Outdated
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we make it just a little less verbose than Java? How about GetRequiredPlacementResources

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed.

Copy link
Copy Markdown
Contributor

@atumanov atumanov left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

partial set of comments for now.

Comment thread src/ray/raylet/node_manager.cc Outdated
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we need to wrap this line? Hard to tell, but looks like < 90chars to me if combined. Ignore if not the case.

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed.

Comment thread src/ray/raylet/scheduling_resources.cc Outdated
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

are you trying to generate a json-compatible string here? I wonder if we should just use the json library or add a ToJson() method that explicitly does that.

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The goal was not to make a JSON compatible string, but rather to make the printing nicer. This changes it from {GPU,1.000000}, {CPU,1.000000}, (note the extra comma and space at the end) to {GPU,1.000000}, {CPU,1.000000}.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(can we reduce the number of decimals?)

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also there's no implode/join built-in for this?

@AmplabJenkins
Copy link
Copy Markdown

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Ray-PRB/8075/
Test FAILed.

Comment thread src/ray/raylet/scheduling_policy.cc Outdated
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

const ref : const auto &spec to avoid a copy.

Comment thread src/ray/raylet/scheduling_policy.cc Outdated
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

const ref : const auto &spec (same as above)

@AmplabJenkins
Copy link
Copy Markdown

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Ray-PRB/8076/
Test FAILed.

@robertnishihara
Copy link
Copy Markdown
Collaborator Author

jenkins, retest this please

@AmplabJenkins
Copy link
Copy Markdown

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Ray-PRB/8078/
Test FAILed.

Comment thread src/ray/raylet/task_spec.cc Outdated
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it always 1 more? What about the num_cpus=0 or num_cpus=N case?

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh good catch, in that case it should be 0. I'll fix this.

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That was a bug!

Comment thread src/ray/raylet/node_manager.cc Outdated
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

IMO this error message would be better if it reported the current state (i.e., resources currently used out of total resources in cluster)

Comment thread src/ray/raylet/scheduling_policy.cc Outdated
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same here (more info about "no nodes have the necessary resources")

Comment thread src/ray/raylet/scheduling_resources.cc Outdated
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(can we reduce the number of decimals?)

Comment thread src/ray/raylet/scheduling_resources.cc Outdated
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also there's no implode/join built-in for this?

@robertnishihara
Copy link
Copy Markdown
Collaborator Author

@raulchen @jovan-wong What is the correct procedure for updating java/runtime/src/main/java/org/ray/runtime/generated/TaskInfo.java after updating the flatbuffer definition?

I basically had to copy it over from ray/build/generated/java and then modify a handful of things by hand (e.g., change the imports at the top and the code at the bottom to minimize the diff).

What's the reason for committing the autogenerated files?

@AmplabJenkins
Copy link
Copy Markdown

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Ray-PRB/8512/
Test FAILed.

@AmplabJenkins
Copy link
Copy Markdown

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Ray-PRB/8513/
Test PASSed.

@AmplabJenkins
Copy link
Copy Markdown

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Ray-PRB/8515/
Test FAILed.

@AmplabJenkins
Copy link
Copy Markdown

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Ray-PRB/8516/
Test PASSed.

@jovany-wang
Copy link
Copy Markdown
Contributor

@robertnishihara Thanks for reminding me about this.

I took a look at TaskInfo.java.
The reason for this is that there are some manual code here. And it can be moved out of TaskInfo i think. It's time to auto-gen all of these code right now.

But there is a problem that is not well resolved:
We should add the package declaration in TaskInfo.java, like package org.ray.runtime.generated; . This requires us to declare the namespace in fbs file, like namespace org.ray.runtime.generated, right?
But I think this namespace declaration in fbs will not suitable in python or cpp.

How do you guys think of it?

@robertnishihara
Copy link
Copy Markdown
Collaborator Author

cc @romilbhardwaj

Copy link
Copy Markdown
Contributor

@atumanov atumanov left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just a few minor comments. Made a full pass on C++.

void NodeManager::SubmitTask(const Task &task, const Lineage &uncommitted_lineage,
bool forwarded) {
const TaskID task_id = task.GetTaskSpecification().TaskId();
const TaskID &task_id = task.GetTaskSpecification().TaskId();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍

Comment thread src/ray/raylet/scheduling_policy.cc Outdated
int client_key_index = distribution(gen_);
const ClientID &dst_client_id = client_keys[client_key_index];
decision[t.GetTaskSpecification().TaskId()] = dst_client_id;
decision[spec.TaskId()] = dst_client_id;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we can just use task_id here

Comment thread src/ray/raylet/scheduling_policy.cc Outdated
RAY_LOG(INFO) << "This task requires "
<< t.GetTaskSpecification().GetRequiredResources().ToString()
<< ", but no nodes have the necessary resources.";
RAY_LOG(INFO) << "The task with ID " << spec.TaskId() << " requires "
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same, spec.TaskId() --> task_id

Comment thread src/ray/raylet/scheduling_resources.cc Outdated

bool ResourceSet::operator==(const ResourceSet &rhs) const {
return (this->IsSubset(rhs) && rhs.IsSubset(*this));
return (IsSubset(rhs) && rhs.IsSubset(*this));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

actually, in the case where we overload binary operators, it's useful to keep this-> for code readability. You get "this" is a subset of the right hand side. Removing it makes it a little less clear what rhs is supposed to be a subset of (because this-> is implicit).

Comment thread src/ray/raylet/scheduling_resources.cc Outdated
/// Test whether this ResourceSet is precisely equal to the other ResourceSet.
bool ResourceSet::IsEqual(const ResourceSet &rhs) const {
return (this->IsSubset(rhs) && rhs.IsSubset(*this));
return (IsSubset(rhs) && rhs.IsSubset(*this));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same, I would prefer to keep this-> for readability when overloading binary operators of the form A.BinaryOperator(B).

return false;
}
if (this->resource_capacity_.count(resource_name) == 0) {
if (resource_capacity_.count(resource_name) == 0) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

and here, it makes sense to drop this-> as you did.

Comment thread src/ray/raylet/scheduling_resources.cc Outdated
it++;

// Add the remaining elements to the string (along with a comma).
while (it != resource_capacity_.end()) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why not:
for ( ; it != resource_capacity_.end() ; ++it )
But I don't have a strong opinion on this.

@AmplabJenkins
Copy link
Copy Markdown

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Ray-PRB/8541/
Test FAILed.

@AmplabJenkins
Copy link
Copy Markdown

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Ray-PRB/8545/
Test FAILed.

@AmplabJenkins
Copy link
Copy Markdown

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Ray-PRB/8546/
Test PASSed.

Copy link
Copy Markdown
Contributor

@atumanov atumanov left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@atumanov atumanov merged commit faa31ae into ray-project:master Oct 4, 2018
@robertnishihara robertnishihara deleted the placementresources branch November 17, 2018 02:19
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Hanging in test/runtest.py::ResourcesTest::testZeroCPUsActor on Travis.

6 participants