Option to disable profiling and task timeline (#10414)

This commit is contained in:
Stephanie Wang 2020-08-29 11:35:22 -07:00 committed by GitHub
parent 910d5d2550
commit 9a31166050
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
4 changed files with 24 additions and 3 deletions

View file

@ -697,6 +697,14 @@ cdef void terminate_asyncio_thread() nogil:
core_worker.destroy_event_loop_if_exists()
# An empty profile event context to be used when the timeline is disabled.
cdef class EmptyProfileEvent:
def __enter__(self):
pass
def __exit__(self, *args):
pass
cdef class CoreWorker:
def __cinit__(self, worker_type, store_socket, raylet_socket,
@ -1172,9 +1180,12 @@ cdef class CoreWorker:
return resources_dict
def profile_event(self, c_string event_type, object extra_data=None):
return ProfileEvent.make(
CCoreWorkerProcess.GetCoreWorker().CreateProfileEvent(event_type),
extra_data)
if RayConfig.instance().enable_timeline():
return ProfileEvent.make(
CCoreWorkerProcess.GetCoreWorker().CreateProfileEvent(
event_type), extra_data)
else:
return EmptyProfileEvent()
def remove_actor_handle_reference(self, ActorID actor_id):
cdef:

View file

@ -64,3 +64,5 @@ cdef extern from "ray/common/ray_config.h" nogil:
uint32_t max_tasks_in_flight_per_worker() const
uint64_t metrics_report_interval_ms() const
c_bool enable_timeline() const

View file

@ -111,3 +111,7 @@ cdef class Config:
@staticmethod
def metrics_report_interval_ms():
return RayConfig.instance().metrics_report_interval_ms()
@staticmethod
def enable_timeline():
return RayConfig.instance().enable_timeline()

View file

@ -294,3 +294,7 @@ RAY_CONFIG(uint64_t, metrics_report_interval_ms, 10000)
/// The maximum number of I/O worker that raylet starts.
RAY_CONFIG(int, max_io_workers, 1)
/// Enable the task timeline. If this is enabled, certain events such as task
/// execution are profiled and sent to the GCS.
RAY_CONFIG(bool, enable_timeline, true)