diff --git a/src/ray/common/task/task_spec.cc b/src/ray/common/task/task_spec.cc index 2bf888ad7164e..8ae5429cd9354 100644 --- a/src/ray/common/task/task_spec.cc +++ b/src/ray/common/task/task_spec.cc @@ -389,6 +389,23 @@ std::string TaskSpecification::DebugString() const { << "}"; } + // Print runtime env. + if (HasRuntimeEnv()) { + const auto &runtime_env_info = RuntimeEnvInfo(); + stream << ", serialized_runtime_env=" << SerializedRuntimeEnv(); + const auto &uris = runtime_env_info.uris(); + if (uris.size() > 0) { + stream << ", runtime_env_uris="; + for (const auto &uri : uris) { + stream << uri << ":"; + } + // Erase the last ":" + stream.seekp(-1, std::ios_base::end); + } + stream << ", runtime_env_eager_install=" + << runtime_env_info.runtime_env_eager_install(); + } + return stream.str(); } diff --git a/src/ray/core_worker/core_worker.cc b/src/ray/core_worker/core_worker.cc index 8ce560a58fe91..397b9393bc1a3 100644 --- a/src/ray/core_worker/core_worker.cc +++ b/src/ray/core_worker/core_worker.cc @@ -1864,7 +1864,7 @@ std::vector CoreWorker::SubmitTask( task_options.serialized_runtime_env); builder.SetNormalTaskSpec(max_retries, retry_exceptions); TaskSpecification task_spec = builder.Build(); - RAY_LOG(DEBUG) << "Submit task " << task_spec.DebugString(); + RAY_LOG(DEBUG) << "Submitting normal task " << task_spec.DebugString(); std::vector returned_refs; if (options_.is_local_mode) { returned_refs = ExecuteTaskLocalMode(task_spec); @@ -1949,6 +1949,7 @@ Status CoreWorker::CreateActor(const RayFunction &function, << "Actor " << actor_id << " already exists"; *return_actor_id = actor_id; TaskSpecification task_spec = builder.Build(); + RAY_LOG(DEBUG) << "Submitting actor creation task " << task_spec.DebugString(); if (options_.is_local_mode) { // TODO(suquark): Should we consider namespace in local mode? Currently // it looks like two actors with two different namespaces become the @@ -2126,6 +2127,7 @@ std::vector CoreWorker::SubmitActorTask( // Submit task. TaskSpecification task_spec = builder.Build(); + RAY_LOG(DEBUG) << "Submitting actor task " << task_spec.DebugString(); std::vector returned_refs; if (options_.is_local_mode) { returned_refs = ExecuteTaskLocalMode(task_spec, actor_id);