mirror of
https://github.com/vale981/ray
synced 2025-03-06 02:21:39 -05:00
print python backtrace in error message (#209)
This commit is contained in:
parent
8297522580
commit
49decce6f6
3 changed files with 26 additions and 8 deletions
|
@ -8,6 +8,7 @@ import funcsigs
|
|||
import numpy as np
|
||||
import colorama
|
||||
import copy
|
||||
import traceback
|
||||
|
||||
import ray
|
||||
import ray.config as config
|
||||
|
@ -120,7 +121,7 @@ def print_failed_task(task_status):
|
|||
Error: Task failed
|
||||
Function Name: {}
|
||||
Task ID: {}
|
||||
Error Message: {}
|
||||
Error Message: \n{}
|
||||
""".format(task_status["function_name"], task_status["operationid"], task_status["error_message"])
|
||||
|
||||
# This is a helper method. It should not be called by users.
|
||||
|
@ -266,6 +267,22 @@ def restart_workers_local(num_workers, worker_path, worker=global_worker):
|
|||
return False
|
||||
services.start_workers(worker.scheduler_address, worker.objstore_address, num_workers, worker_path)
|
||||
|
||||
def format_error_message(exception_message):
|
||||
"""
|
||||
This method takes an backtrace from an exception and makes it nicer by
|
||||
removing a few uninformative lines and adding some space to indent the
|
||||
remaining lines nicely.
|
||||
|
||||
:param exception_message: a string generated by traceback.format_exc()
|
||||
:rtype: a string
|
||||
"""
|
||||
lines = exception_message.split("\n")
|
||||
# Remove lines 1, 2, 3, and 4, which are always the same, they just contain
|
||||
# information about the main loop.
|
||||
lines = lines[0:1] + lines[5:]
|
||||
lines = [10 * " " + line for line in lines]
|
||||
return "\n".join(lines)
|
||||
|
||||
def main_loop(worker=global_worker):
|
||||
if not ray.lib.connected(worker.handle):
|
||||
raise Exception("Worker is attempting to enter main_loop but has not been connected yet.")
|
||||
|
@ -278,12 +295,13 @@ def main_loop(worker=global_worker):
|
|||
if len(return_objrefs) == 1:
|
||||
outputs = (outputs,)
|
||||
except Exception as e:
|
||||
exception_message = format_error_message(traceback.format_exc())
|
||||
# Here we are storing RayFailedObjects in the object store to indicate
|
||||
# failure (this is only interpreted by the worker).
|
||||
failure_objects = [RayFailedObject(str(e)) for _ in range(len(return_objrefs))]
|
||||
failure_objects = [RayFailedObject(exception_message) for _ in range(len(return_objrefs))]
|
||||
store_outputs_in_objstore(return_objrefs, failure_objects, worker)
|
||||
ray.lib.notify_task_completed(worker.handle, False, str(e)) # notify the scheduler that the task threw an exception
|
||||
logging.info("Worker through exception with message: {}, while running function {}.".format(str(e), func_name))
|
||||
ray.lib.notify_task_completed(worker.handle, False, exception_message) # notify the scheduler that the task threw an exception
|
||||
logging.info("Worker threw exception with message: \n\n{}\n, while running function {}.".format(exception_message, func_name))
|
||||
else:
|
||||
store_outputs_in_objstore(return_objrefs, outputs, worker) # store output in local object store
|
||||
ray.lib.notify_task_completed(worker.handle, True, "") # notify the scheduler that the task completed successfully
|
||||
|
|
|
@ -164,7 +164,7 @@ Status SchedulerService::ReadyForNewTask(ServerContext* context, const ReadyForN
|
|||
(*workers)[workerid].current_task = NO_OPERATION; // clear operation ID
|
||||
}
|
||||
if (!request->previous_task_info().task_succeeded()) {
|
||||
RAY_LOG(RAY_INFO, "Error: Task " << info.operationid() << " executing function " << info.function_name() << " on worker " << workerid << " failed with error message: " << info.error_message());
|
||||
RAY_LOG(RAY_INFO, "Error: Task " << info.operationid() << " executing function " << info.function_name() << " on worker " << workerid << " failed with error message:\n" << info.error_message());
|
||||
failed_tasks_.get()->push_back(info);
|
||||
} else {
|
||||
successful_tasks_.get()->push_back(info.operationid());
|
||||
|
|
|
@ -289,7 +289,7 @@ class TaskStatusTest(unittest.TestCase):
|
|||
for task in result["failed_tasks"]:
|
||||
self.assertTrue(task.has_key("worker_address"))
|
||||
self.assertTrue(task.has_key("operationid"))
|
||||
self.assertEqual(task.get("error_message"), "Test function 1 intentionally failed.")
|
||||
self.assertTrue("Test function 1 intentionally failed." in task.get("error_message"))
|
||||
self.assertTrue(task["operationid"] not in task_ids)
|
||||
task_ids.add(task["operationid"])
|
||||
|
||||
|
@ -297,7 +297,7 @@ class TaskStatusTest(unittest.TestCase):
|
|||
try:
|
||||
ray.get(x)
|
||||
except Exception as e:
|
||||
self.assertEqual(str(e), "The task that created this object reference failed with error message: Test function 2 intentionally failed.")
|
||||
self.assertTrue("Test function 2 intentionally failed."in str(e))
|
||||
else:
|
||||
self.assertTrue(False) # ray.get should throw an exception
|
||||
|
||||
|
@ -306,7 +306,7 @@ class TaskStatusTest(unittest.TestCase):
|
|||
try:
|
||||
ray.get(ref)
|
||||
except Exception as e:
|
||||
self.assertEqual(str(e), "The task that created this object reference failed with error message: Test function 3 intentionally failed.")
|
||||
self.assertTrue("Test function 3 intentionally failed."in str(e))
|
||||
else:
|
||||
self.assertTrue(False) # ray.get should throw an exception
|
||||
|
||||
|
|
Loading…
Add table
Reference in a new issue