Skip to content
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

[Core] [runtime env] Add RuntimeEnvHash and JobID to SchedulingKey #16766

Merged

Conversation

architkulkarni
Copy link
Contributor

@architkulkarni architkulkarni commented Jun 30, 2021

Why are these changes needed?

A worker can only be assigned to a task if its runtime env hash matches the one in the task spec. (https://github.com/ray-project/ray/blob/master/src/ray/raylet/worker_pool.cc#L929-L942)

If a remote task called that requires a runtime env to be installed, the worker lease request will remain unfulfilled for the duration of the installation, which can take ~50 seconds. This blocks other worker lease requests, because we ratelimit to a single lease request per SchedulingKey. (https://sourcegraph.com/github.com/ray-project/ray/-/blob/src/ray/core_worker/transport/direct_task_transport.cc?L441)

The issue is if we subsequently call another remote task with a different runtime env (or the same runtime env), it won't be scheduled until the first worker lease request is fulfilled:

# Start a remote task with a new env, will take ~50s to install and start the worker
f.options(runtime_env={"pip": ["pip-install-test==0.5"]}).remote()
start = time.time()
ray.get(f.remote()) # Empty env, should return nearly instantaneously...
print(time.time() - start) # ...but this returns ~50

The fix is to add RuntimeEnvHash to the SchedulingKey, which this PR does.

Related issue number

Closes #16226.
Closes #16537 by adding a test which fails before this PR, but passes with the fix in this PR. The test is a single-driver version of the reproduction in #16226, which required two drivers.

Checks

  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

@architkulkarni architkulkarni changed the title [WIP] [Core] [runtime env] Add python integration test for task scheduler blocking [Core] [runtime env] Add python integration test for task scheduler blocking Jun 30, 2021
Copy link
Contributor

@edoakes edoakes left a comment

Choose a reason for hiding this comment

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

Looks good! Should we also have a test for the case where they use the same env and a new worker is started? (I remember there being a similar issue there)

python/ray/tests/test_runtime_env_complicated.py Outdated Show resolved Hide resolved
@architkulkarni architkulkarni added the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Jun 30, 2021
@architkulkarni architkulkarni changed the title [Core] [runtime env] Add python integration test for task scheduler blocking [Core] [runtime env] Add RuntimeEnvHash and JobID to SchedulingKey Jul 2, 2021
@@ -74,7 +74,8 @@ Status CoreWorkerDirectTaskSubmitter::SubmitTask(TaskSpecification task_spec) {
const SchedulingKey scheduling_key(
task_spec.GetSchedulingClass(), task_spec.GetDependencyIds(),
task_spec.IsActorCreationTask() ? task_spec.ActorCreationId()
: ActorID::Nil());
: ActorID::Nil(),
task_spec.GetRuntimeEnvHash(), task_spec.JobId());
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Can we confirm that it makes sense to add JobID here?

Copy link
Contributor

Choose a reason for hiding this comment

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

Hmmm, probably best not to have job id since it doesn't really do anything. Workers belong to jobs, so job id will always be the same.

@architkulkarni
Copy link
Contributor Author

Sorry, there's still some pretty blatant compiler errors, not sure how it compiled locally... let me fix them

serialized_runtime_env(serialized_runtime_env) {}

bool WorkerCacheKey::operator==(const WorkerCacheKey &k) const {
return Hash() == k.Hash();
Copy link
Contributor

Choose a reason for hiding this comment

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

This has the potential to return True when the env vars aren't actually the same doesn't it? Is that intentional? If so, can we add a comment explaining why?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good point, I'm fine with comparing all the fields for equality. I think @edoakes you had some input on this earlier, do you remember? Was it for speed or something?

Copy link
Contributor

Choose a reason for hiding this comment

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

@architkulkarni can't we remove the override_environment_variables very soon? In that case they'd just be included in the hashed env. We should probably include everything.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Oh yeah, the env vars are already always included in the hash. Alex's concern was just about comparing hashes in operator== as possed to comparing all the fields.

I'll soon make a PR to remove override_environment_variables, and in that PR I'll switch out the implementation of operator== to compare the full serialized_runtime_env sinstead of the hashes.

@architkulkarni
Copy link
Contributor Author

Looks good! Should we also have a test for the case where they use the same env and a new worker is started? (I remember there being a similar issue there)

You're right, I'll add a test for that

@architkulkarni architkulkarni added tests-ok The tagger certifies test failures are unrelated and assumes personal liability. and removed @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. labels Jul 2, 2021
@wuisawesome
Copy link
Contributor

Tests look fine. Merging.

@wuisawesome wuisawesome merged commit f02e41a into ray-project:master Jul 3, 2021
jiaodong pushed a commit that referenced this pull request Jul 11, 2021
…16766)

* add python integration test

* improve readability

* remove unneccessary ray start --head

* add shutdown_only

* move RuntimeEnvHash from worker_pool to task_spec

* lint

* Add runtimeEnvHash and JobID to SchedulingKey

* remove JobID from key and hopefully fix compile

* add test for same env

* lint
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
tests-ok The tagger certifies test failures are unrelated and assumes personal liability.
Projects
None yet
3 participants