diff --git a/CMakeLists.txt b/CMakeLists.txt index 4779a03fcd00..47f7b2ebe677 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -4,6 +4,7 @@ project(ray) set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${CMAKE_SOURCE_DIR}/cmake/Modules") + # This ensures that things like gnu++11 get passed correctly set(CMAKE_CXX_STANDARD 11) @@ -101,7 +102,6 @@ if ("${CMAKE_RAY_LANG_PYTHON}" STREQUAL "YES") set(ray_file_list "src/ray/thirdparty/redis/src/redis-server" "src/ray/gcs/redis_module/libray_redis_module.so" - "src/ray/raylet/libraylet_library_python.so" "src/ray/raylet/raylet_monitor" "src/ray/raylet/raylet") @@ -127,9 +127,8 @@ if ("${CMAKE_RAY_LANG_PYTHON}" STREQUAL "YES") add_custom_target(copy_ray_files DEPENDS ${build_ray_file_list} copy_redis ray_redis_module) add_dependencies(copy_ray copy_ray_files) - # Make sure that the Python extensions are built before copying the files. - get_raylet_library("python" RAYLET_LIBRARY_PYTHON) - add_dependencies(copy_ray ${RAYLET_LIBRARY_PYTHON}) + # Build Cython extensions + add_subdirectory(${CMAKE_CURRENT_LIST_DIR}/python) foreach(file ${ray_file_list}) add_custom_command(TARGET copy_ray POST_BUILD @@ -143,6 +142,10 @@ if ("${CMAKE_RAY_LANG_PYTHON}" STREQUAL "YES") COMMAND ${CMAKE_COMMAND} -E copy ${ARROW_HOME}/bin/plasma_store_server ${CMAKE_SOURCE_DIR}/python/ray/core/src/plasma/) + # copy Cython files + add_custom_command(TARGET copy_ray POST_BUILD + COMMAND ${CMAKE_COMMAND} -E copy ${CMAKE_BINARY_DIR}/python/_raylet.so ${CMAKE_SOURCE_DIR}/python/ray/) + endif() if ("${CMAKE_RAY_LANG_JAVA}" STREQUAL "YES") diff --git a/cmake/Modules/UseCython.cmake b/cmake/Modules/UseCython.cmake new file mode 100755 index 000000000000..9a1a2f3c242c --- /dev/null +++ b/cmake/Modules/UseCython.cmake @@ -0,0 +1,317 @@ +# Define a function to create Cython modules. +# +# For more information on the Cython project, see http://cython.org/. +# "Cython is a language that makes writing C extensions for the Python language +# as easy as Python itself." +# +# This file defines a CMake function to build a Cython Python module. +# To use it, first include this file. +# +# include( UseCython ) +# +# Then call cython_add_module to create a module. +# +# cython_add_module( ... ) +# +# To create a standalone executable, the function +# +# cython_add_standalone_executable( [MAIN_MODULE src1] ... ) +# +# To avoid dependence on Python, set the PYTHON_LIBRARY cache variable to point +# to a static library. If a MAIN_MODULE source is specified, +# the "if __name__ == '__main__':" from that module is used as the C main() method +# for the executable. If MAIN_MODULE, the source with the same basename as +# is assumed to be the MAIN_MODULE. +# +# Where is the name of the resulting Python module and +# ... are source files to be compiled into the module, e.g. *.pyx, +# *.py, *.c, *.cxx, etc. A CMake target is created with name . This can +# be used for target_link_libraries(), etc. +# +# The sample paths set with the CMake include_directories() command will be used +# for include directories to search for *.pxd when running the Cython complire. +# +# Cache variables that effect the behavior include: +# +# CYTHON_ANNOTATE +# CYTHON_NO_DOCSTRINGS +# CYTHON_FLAGS +# +# Source file properties that effect the build process are +# +# CYTHON_IS_CXX +# +# If this is set of a *.pyx file with CMake set_source_files_properties() +# command, the file will be compiled as a C++ file. +# +# See also FindCython.cmake + +#============================================================================= +# Copyright 2011 Kitware, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +#============================================================================= + +# Configuration options. +set( CYTHON_ANNOTATE OFF + CACHE BOOL "Create an annotated .html file when compiling *.pyx." ) +set( CYTHON_NO_DOCSTRINGS OFF + CACHE BOOL "Strip docstrings from the compiled module." ) +set( CYTHON_FLAGS "" CACHE STRING + "Extra flags to the cython compiler." ) +mark_as_advanced( CYTHON_ANNOTATE CYTHON_NO_DOCSTRINGS CYTHON_FLAGS ) + +find_package( PythonLibs REQUIRED ) + +set( CYTHON_CXX_EXTENSION "cxx" ) +set( CYTHON_C_EXTENSION "c" ) + +# Create a *.c or *.cxx file from a *.pyx file. +# Input the generated file basename. The generate file will put into the variable +# placed in the "generated_file" argument. Finally all the *.py and *.pyx files. +function( compile_pyx _name generated_file ) + # Default to assuming all files are C. + set( cxx_arg "" ) + set( extension ${CYTHON_C_EXTENSION} ) + set( pyx_lang "C" ) + set( comment "Compiling Cython C source for ${_name}..." ) + + set( cython_include_directories "" ) + set( pxd_dependencies "" ) + set( pxi_dependencies "" ) + set( c_header_dependencies "" ) + set( pyx_locations "" ) + + foreach( pyx_file ${ARGN} ) + get_filename_component( pyx_file_basename "${pyx_file}" NAME_WE ) + + # Determine if it is a C or C++ file. + get_source_file_property( property_is_cxx ${pyx_file} CYTHON_IS_CXX ) + if( ${property_is_cxx} ) + set( cxx_arg "--cplus" ) + set( extension ${CYTHON_CXX_EXTENSION} ) + set( pyx_lang "CXX" ) + set( comment "Compiling Cython CXX source for ${_name}..." ) + endif() + + # Get the include directories. + get_source_file_property( pyx_location ${pyx_file} LOCATION ) + get_filename_component( pyx_path ${pyx_location} PATH ) + # TODO(suquark): Temporarily commented out for compiling errors. + # get_directory_property( cmake_include_directories DIRECTORY ${pyx_path} INCLUDE_DIRECTORIES ) + # list( APPEND cython_include_directories ${cmake_include_directories} ) + list( APPEND pyx_locations "${pyx_location}" ) + + # Determine dependencies. + # Add the pxd file will the same name as the given pyx file. + unset( corresponding_pxd_file CACHE ) + find_file( corresponding_pxd_file ${pyx_file_basename}.pxd + PATHS "${pyx_path}" ${cmake_include_directories} + NO_DEFAULT_PATH ) + if( corresponding_pxd_file ) + list( APPEND pxd_dependencies "${corresponding_pxd_file}" ) + endif() + + # Look for included pxi files + file(STRINGS "${pyx_file}" include_statements REGEX "include +['\"]([^'\"]+).*") + foreach(statement ${include_statements}) + string(REGEX REPLACE "include +['\"]([^'\"]+).*" "\\1" pxi_file "${statement}") + unset(pxi_location CACHE) + find_file(pxi_location ${pxi_file} + PATHS "${pyx_path}" ${cmake_include_directories} NO_DEFAULT_PATH) + if (pxi_location) + list(APPEND pxi_dependencies ${pxi_location}) + get_filename_component( found_pyi_file_basename "${pxi_file}" NAME_WE ) + get_filename_component( found_pyi_path ${pxi_location} PATH ) + unset( found_pyi_pxd_file CACHE ) + find_file( found_pyi_pxd_file ${found_pyi_file_basename}.pxd + PATHS "${found_pyi_path}" ${cmake_include_directories} NO_DEFAULT_PATH ) + if (found_pyi_pxd_file) + list( APPEND pxd_dependencies "${found_pyi_pxd_file}" ) + endif() + endif() + endforeach() # for each include statement found + + # pxd files to check for additional dependencies. + set( pxds_to_check "${pyx_file}" "${pxd_dependencies}" ) + set( pxds_checked "" ) + set( number_pxds_to_check 1 ) + while( ${number_pxds_to_check} GREATER 0 ) + foreach( pxd ${pxds_to_check} ) + list( APPEND pxds_checked "${pxd}" ) + list( REMOVE_ITEM pxds_to_check "${pxd}" ) + + # check for C header dependencies + file( STRINGS "${pxd}" extern_from_statements + REGEX "cdef[ ]+extern[ ]+from.*$" ) + foreach( statement ${extern_from_statements} ) + # Had trouble getting the quote in the regex + string( REGEX REPLACE "cdef[ ]+extern[ ]+from[ ]+[\"]([^\"]+)[\"].*" "\\1" header "${statement}" ) + unset( header_location CACHE ) + find_file( header_location ${header} PATHS ${cmake_include_directories} ) + if( header_location ) + list( FIND c_header_dependencies "${header_location}" header_idx ) + if( ${header_idx} LESS 0 ) + list( APPEND c_header_dependencies "${header_location}" ) + endif() + endif() + endforeach() + + # check for pxd dependencies + + # Look for cimport statements. + set( module_dependencies "" ) + file( STRINGS "${pxd}" cimport_statements REGEX cimport ) + foreach( statement ${cimport_statements} ) + if( ${statement} MATCHES from ) + string( REGEX REPLACE "from[ ]+([^ ]+).*" "\\1" module "${statement}" ) + else() + string( REGEX REPLACE "cimport[ ]+([^ ]+).*" "\\1" module "${statement}" ) + endif() + list( APPEND module_dependencies ${module} ) + endforeach() + list( REMOVE_DUPLICATES module_dependencies ) + # Add the module to the files to check, if appropriate. + foreach( module ${module_dependencies} ) + unset( pxd_location CACHE ) + find_file( pxd_location ${module}.pxd + PATHS "${pyx_path}" ${cmake_include_directories} NO_DEFAULT_PATH ) + if( pxd_location ) + list( FIND pxds_checked ${pxd_location} pxd_idx ) + if( ${pxd_idx} LESS 0 ) + list( FIND pxds_to_check ${pxd_location} pxd_idx ) + if( ${pxd_idx} LESS 0 ) + list( APPEND pxds_to_check ${pxd_location} ) + list( APPEND pxd_dependencies ${pxd_location} ) + endif() # if it is not already going to be checked + endif() # if it has not already been checked + endif() # if pxd file can be found + endforeach() # for each module dependency discovered + endforeach() # for each pxd file to check + list( LENGTH pxds_to_check number_pxds_to_check ) + endwhile() + + + + endforeach() # pyx_file + + # Set additional flags. + if( CYTHON_ANNOTATE ) + set( annotate_arg "--annotate" ) + endif() + + if( CYTHON_NO_DOCSTRINGS ) + set( no_docstrings_arg "--no-docstrings" ) + endif() + + if( "${CMAKE_BUILD_TYPE}" STREQUAL "Debug" OR + "${CMAKE_BUILD_TYPE}" STREQUAL "RelWithDebInfo" ) + set( cython_debug_arg "--gdb" ) + endif() + + if( "${PYTHONLIBS_VERSION_STRING}" MATCHES "^2." ) + set( version_arg "-2" ) + elseif( "${PYTHONLIBS_VERSION_STRING}" MATCHES "^3." ) + set( version_arg "-3" ) + else() + set( version_arg ) + endif() + + # Include directory arguments. + list( REMOVE_DUPLICATES cython_include_directories ) + set( include_directory_arg "" ) + foreach( _include_dir ${cython_include_directories} ) + set( include_directory_arg ${include_directory_arg} "-I" "${_include_dir}" ) + endforeach() + + # Determining generated file name. + set( _generated_file "${CMAKE_CURRENT_BINARY_DIR}/${_name}.${extension}" ) + set_source_files_properties( ${_generated_file} PROPERTIES GENERATED TRUE ) + set( ${generated_file} ${_generated_file} PARENT_SCOPE ) + + list( REMOVE_DUPLICATES pxd_dependencies ) + list( REMOVE_DUPLICATES c_header_dependencies ) + + # Add the command to run the compiler. + add_custom_command( OUTPUT ${_generated_file} + COMMAND ${CYTHON_EXECUTABLE} + ARGS ${cxx_arg} ${include_directory_arg} ${version_arg} + ${annotate_arg} ${no_docstrings_arg} ${cython_debug_arg} ${CYTHON_FLAGS} + --output-file ${_generated_file} ${pyx_locations} + DEPENDS ${pyx_locations} ${pxd_dependencies} ${pxi_dependencies} + IMPLICIT_DEPENDS ${pyx_lang} ${c_header_dependencies} + COMMENT ${comment} + ) + + # Remove their visibility to the user. + set( corresponding_pxd_file "" CACHE INTERNAL "" ) + set( header_location "" CACHE INTERNAL "" ) + set( pxd_location "" CACHE INTERNAL "" ) +endfunction() + +# cython_add_module( src1 src2 ... srcN ) +# Build the Cython Python module. +function( cython_add_module _name ) + set( pyx_module_sources "" ) + set( other_module_sources "" ) + foreach( _file ${ARGN} ) + if( ${_file} MATCHES ".*\\.py[x]?$" ) + list( APPEND pyx_module_sources ${_file} ) + else() + list( APPEND other_module_sources ${_file} ) + endif() + endforeach() + compile_pyx( ${_name} generated_file ${pyx_module_sources} ) + include_directories( ${PYTHON_INCLUDE_DIRS} ) + python_add_module( ${_name} ${generated_file} ${other_module_sources} ) + if( APPLE ) + set_target_properties( ${_name} PROPERTIES LINK_FLAGS "-undefined dynamic_lookup" ) + endif() +endfunction() + +include( CMakeParseArguments ) +# cython_add_standalone_executable( _name [MAIN_MODULE src3.py] src1 src2 ... srcN ) +# Creates a standalone executable the given sources. +function( cython_add_standalone_executable _name ) + set( pyx_module_sources "" ) + set( other_module_sources "" ) + set( main_module "" ) + cmake_parse_arguments( cython_arguments "" "MAIN_MODULE" "" ${ARGN} ) + include_directories( ${PYTHON_INCLUDE_DIRS} ) + foreach( _file ${cython_arguments_UNPARSED_ARGUMENTS} ) + if( ${_file} MATCHES ".*\\.py[x]?$" ) + get_filename_component( _file_we ${_file} NAME_WE ) + if( "${_file_we}" STREQUAL "${_name}" ) + set( main_module "${_file}" ) + elseif( NOT "${_file}" STREQUAL "${cython_arguments_MAIN_MODULE}" ) + set( PYTHON_MODULE_${_file_we}_static_BUILD_SHARED OFF ) + compile_pyx( "${_file_we}_static" generated_file "${_file}" ) + list( APPEND pyx_module_sources "${generated_file}" ) + endif() + else() + list( APPEND other_module_sources ${_file} ) + endif() + endforeach() + + if( cython_arguments_MAIN_MODULE ) + set( main_module ${cython_arguments_MAIN_MODULE} ) + endif() + if( NOT main_module ) + message( FATAL_ERROR "main module not found." ) + endif() + get_filename_component( main_module_we "${main_module}" NAME_WE ) + set( CYTHON_FLAGS ${CYTHON_FLAGS} --embed ) + compile_pyx( "${main_module_we}_static" generated_file ${main_module} ) + add_executable( ${_name} ${generated_file} ${pyx_module_sources} ${other_module_sources} ) + target_link_libraries( ${_name} ${pyx_module_libs} ) +endfunction() diff --git a/doc/source/conf.py b/doc/source/conf.py index aefe7b015edd..69e17376899c 100644 --- a/doc/source/conf.py +++ b/doc/source/conf.py @@ -35,8 +35,6 @@ "tensorflow.python", "tensorflow.python.client", "tensorflow.python.util", - "ray.raylet", - "ray.core", "ray.core.generated", "ray.core.generated.ClientTableData", "ray.core.generated.GcsTableEntry", @@ -50,6 +48,7 @@ "ray.core.generated.TablePrefix", "ray.core.generated.TablePubsub", "ray.core.generated.Language", + "ray._raylet" ] for mod_name in MOCK_MODULES: sys.modules[mod_name] = mock.Mock() diff --git a/python/CMakeLists.txt b/python/CMakeLists.txt new file mode 100644 index 000000000000..e2f6094cae16 --- /dev/null +++ b/python/CMakeLists.txt @@ -0,0 +1,44 @@ +# NOTE: These must be checked before including Cython packages to ensure we are using the right python version. +# Segfaults could happen if we are using the wrong version. +set(PYTHON_INCLUDE_DIR ${PYTHON_INCLUDE_DIRS}) + +# Find Cython executable +get_filename_component(_python_path ${PYTHON_EXECUTABLE} PATH) +find_program(CYTHON_EXECUTABLE + NAMES cython cython.bat cython3 + HINTS ${_python_path}) + +include(FindPackageHandleStandardArgs) +FIND_PACKAGE_HANDLE_STANDARD_ARGS(Cython REQUIRED_VARS CYTHON_EXECUTABLE) + +include(UseCython) + +include_directories("${NUMPY_INCLUDE_DIR}") +include_directories("${CMAKE_CURRENT_LIST_DIR}/../src") +include_directories("${CMAKE_CURRENT_LIST_DIR}") + +# Include some generated Arrow headers. +include_directories("${ARROW_SOURCE_DIR}/../arrow_ep-build/src") + +# If the pyx file is a C++ file, we should specify that here. +set_source_files_properties( + ${CMAKE_CURRENT_LIST_DIR}/ray/_raylet.pyx + PROPERTIES CYTHON_IS_CXX TRUE) + +set(RAY_SRC_DIRECTORY "${CMAKE_CURRENT_LIST_DIR}/../src/ray") + +cython_add_module( + _raylet + ${RAY_SRC_DIRECTORY}/raylet/raylet_client.cc + ${CMAKE_CURRENT_LIST_DIR}/ray/_raylet.pyx) + +add_dependencies(_raylet ray_static) + +if(APPLE) + target_link_libraries(_raylet "-undefined dynamic_lookup" ray_static) +else() + target_link_libraries(_raylet ray_static) +endif() + +# Make sure that the Python extensions are built before copying the files. +add_dependencies(copy_ray _raylet) diff --git a/python/ray/__init__.py b/python/ray/__init__.py index 2e42a560ee3f..4d99dead0f6e 100644 --- a/python/ray/__init__.py +++ b/python/ray/__init__.py @@ -49,7 +49,12 @@ modin_path = os.path.join(os.path.abspath(os.path.dirname(__file__)), "modin") sys.path.append(modin_path) -from ray.raylet import ObjectID, _config # noqa: E402 +from ray._raylet import (UniqueID, ObjectID, DriverID, ClientID, ActorID, + ActorHandleID, FunctionID, ActorClassID, TaskID, + Config as _Config) # noqa: E402 + +_config = _Config() + from ray.profiling import profile # noqa: E402 from ray.worker import (error_info, init, connect, disconnect, get, put, wait, remote, get_gpu_ids, get_resource_ids, get_webui_url, @@ -72,8 +77,12 @@ "remote", "profile", "actor", "method", "get_gpu_ids", "get_resource_ids", "get_webui_url", "register_custom_serializer", "shutdown", "is_initialized", "SCRIPT_MODE", "WORKER_MODE", "LOCAL_MODE", - "PYTHON_MODE", "global_state", "ObjectID", "_config", "__version__", - "internal" + "PYTHON_MODE", "global_state", "_config", "__version__", "internal" +] + +__all__ += [ + "UniqueID", "ObjectID", "DriverID", "ClientID", "ActorID", "ActorHandleID", + "FunctionID", "ActorClassID", "TaskID" ] import ctypes # noqa: E402 diff --git a/python/ray/_raylet.pyx b/python/ray/_raylet.pyx new file mode 100644 index 000000000000..d0cbd825a2f8 --- /dev/null +++ b/python/ray/_raylet.pyx @@ -0,0 +1,320 @@ +# cython: profile=False +# distutils: language = c++ +# cython: embedsignature = True +# cython: language_level = 3 + +from libc.stdint cimport int32_t, int64_t +from libcpp cimport bool as c_bool +from libcpp.memory cimport unique_ptr +from libcpp.string cimport string as c_string +from libcpp.utility cimport pair +from libcpp.unordered_map cimport unordered_map +from libcpp.vector cimport vector as c_vector + +from ray.includes.common cimport ( + CUniqueID, CTaskID, CObjectID, CFunctionID, CActorClassID, CActorID, + CActorHandleID, CWorkerID, CDriverID, CConfigID, CClientID, + CLanguage, CRayStatus, LANGUAGE_CPP, LANGUAGE_JAVA, LANGUAGE_PYTHON) +from ray.includes.libraylet cimport ( + CRayletClient, GCSProfileTableDataT, GCSProfileEventT, + ResourceMappingType, WaitResultPair) +from ray.includes.task cimport CTaskSpecification +from ray.includes.ray_config cimport RayConfig +from ray.utils import decode + +from cython.operator import dereference, postincrement +cimport cpython + +include "includes/unique_ids.pxi" +include "includes/ray_config.pxi" +include "includes/task.pxi" + + +if cpython.PY_MAJOR_VERSION >= 3: + import pickle +else: + import cPickle as pickle +import numpy + + +cdef int check_status(const CRayStatus& status) nogil except -1: + if status.ok(): + return 0 + + with gil: + message = status.message().decode() + raise Exception(message) + + +cdef c_vector[CObjectID] ObjectIDsToVector(object_ids): + """A helper function that converts a Python list of object IDs to a vector. + + Args: + object_ids (list): The Python list of object IDs. + + Returns: + The output vector. + """ + cdef: + ObjectID object_id + c_vector[CObjectID] result + for object_id in object_ids: + result.push_back(object_id.data) + return result + + +cdef VectorToObjectIDs(c_vector[CObjectID] object_ids): + result = [] + for i in range(object_ids.size()): + result.append(ObjectID.from_native(object_ids[i])) + return result + + +def compute_put_id(TaskID task_id, int64_t put_index): + if put_index < 1 or put_index > kMaxTaskPuts: + raise ValueError("The range of 'put_index' should be [1, %d]" % kMaxTaskPuts) + return ObjectID.from_native(ComputePutId(task_id.data, put_index)) + + +def compute_task_id(ObjectID object_id): + return TaskID.from_native(ComputeTaskId(object_id.data)) + + +cdef c_bool is_simple_value(value, int *num_elements_contained): + num_elements_contained[0] += 1 + + if num_elements_contained[0] >= RayConfig.instance().num_elements_limit(): + return False + + if (cpython.PyInt_Check(value) or cpython.PyLong_Check(value) or value is False or + value is True or cpython.PyFloat_Check(value) or value is None): + return True + + if cpython.PyBytes_CheckExact(value): + num_elements_contained[0] += cpython.PyBytes_Size(value) + return num_elements_contained[0] < RayConfig.instance().num_elements_limit() + + if cpython.PyUnicode_CheckExact(value): + num_elements_contained[0] += cpython.PyUnicode_GET_SIZE(value) + return num_elements_contained[0] < RayConfig.instance().num_elements_limit() + + if cpython.PyList_CheckExact(value) and cpython.PyList_Size(value) < RayConfig.instance().size_limit(): + for item in value: + if not is_simple_value(item, num_elements_contained): + return False + return num_elements_contained[0] < RayConfig.instance().num_elements_limit() + + if cpython.PyDict_CheckExact(value) and cpython.PyDict_Size(value) < RayConfig.instance().size_limit(): + # TODO(suquark): Using "items" in Python2 is not very efficient. + for k, v in value.items(): + if not (is_simple_value(k, num_elements_contained) and is_simple_value(v, num_elements_contained)): + return False + return num_elements_contained[0] < RayConfig.instance().num_elements_limit() + + if cpython.PyTuple_CheckExact(value) and cpython.PyTuple_Size(value) < RayConfig.instance().size_limit(): + for item in value: + if not is_simple_value(item, num_elements_contained): + return False + return num_elements_contained[0] < RayConfig.instance().num_elements_limit() + + if isinstance(value, numpy.ndarray): + if value.dtype == "O": + return False + num_elements_contained[0] += value.nbytes + return num_elements_contained[0] < RayConfig.instance().num_elements_limit() + + return False + + +def check_simple_value(value): + """Check if value is simple enough to be send by value. + + This method checks if a Python object is sufficiently simple that it can be + serialized and passed by value as an argument to a task (without being put in + the object store). The details of which objects are sufficiently simple are + defined by this method and are not particularly important. But for + performance reasons, it is better to place "small" objects in the task itself + and "large" objects in the object store. + + Args: + value: Python object that should be checked. + + Returns: + True if the value should be send by value, False otherwise. + """ + + cdef int num_elements_contained = 0 + return is_simple_value(value, &num_elements_contained) + + +cdef class Language: + cdef CLanguage lang + def __cinit__(self, int32_t lang): + self.lang = lang + + @staticmethod + cdef from_native(const CLanguage& lang): + return Language(lang) + + def __eq__(self, other): + return isinstance(other, Language) and (self.lang) == (other.lang) + + def __repr__(self): + if self.lang == LANGUAGE_PYTHON: + return "PYTHON" + elif self.lang == LANGUAGE_CPP: + return "CPP" + elif self.lang == LANGUAGE_JAVA: + return "JAVA" + else: + raise Exception("Unexpected error") + + +# Programming language enum values. +cdef Language LANG_PYTHON = Language.from_native(LANGUAGE_PYTHON) +cdef Language LANG_CPP = Language.from_native(LANGUAGE_CPP) +cdef Language LANG_JAVA = Language.from_native(LANGUAGE_JAVA) + + +cdef unordered_map[c_string, double] resource_map_from_python_dict(resource_map): + cdef: + unordered_map[c_string, double] out + c_string resource_name + if not isinstance(resource_map, dict): + raise TypeError("resource_map must be a dictionary") + for key, value in resource_map.items(): + out[key.encode("ascii")] = float(value) + return out + + +cdef class RayletClient: + cdef unique_ptr[CRayletClient] client + def __cinit__(self, raylet_socket, + ClientID client_id, + c_bool is_worker, + DriverID driver_id): + # We know that we are using Python, so just skip the language parameter. + # TODO(suquark): Should we allow unicode chars in "raylet_socket"? + self.client.reset(new CRayletClient(raylet_socket.encode("ascii"), client_id.data, + is_worker, driver_id.data, LANGUAGE_PYTHON)) + + def disconnect(self): + check_status(self.client.get().Disconnect()) + + def submit_task(self, Task task_spec): + check_status(self.client.get().SubmitTask(task_spec.execution_dependencies.get()[0], task_spec.task_spec.get()[0])) + + def get_task(self): + cdef: + unique_ptr[CTaskSpecification] task_spec + + with nogil: + check_status(self.client.get().GetTask(&task_spec)) + return Task.make(task_spec) + + def task_done(self): + check_status(self.client.get().TaskDone()) + + def fetch_or_reconstruct(self, object_ids, + c_bool fetch_only, TaskID current_task_id=TaskID.nil()): + cdef c_vector[CObjectID] fetch_ids = ObjectIDsToVector(object_ids) + check_status(self.client.get().FetchOrReconstruct(fetch_ids, fetch_only, current_task_id.data)) + + def notify_unblocked(self, TaskID current_task_id): + check_status(self.client.get().NotifyUnblocked(current_task_id.data)) + + def wait(self, object_ids, int num_returns, int64_t timeout_milliseconds, + c_bool wait_local, TaskID current_task_id): + cdef: + WaitResultPair result + c_vector[CObjectID] wait_ids + wait_ids = ObjectIDsToVector(object_ids) + check_status(self.client.get().Wait(wait_ids, num_returns, timeout_milliseconds, + wait_local, current_task_id.data, &result)) + return VectorToObjectIDs(result.first), VectorToObjectIDs(result.second) + + def resource_ids(self): + cdef: + ResourceMappingType resource_mapping = self.client.get().GetResourceIDs() + unordered_map[c_string, c_vector[pair[int64_t, double]]].iterator iterator = resource_mapping.begin() + c_vector[pair[int64_t, double]] c_value + resources_dict = {} + while iterator != resource_mapping.end(): + key = decode(dereference(iterator).first) + c_value = dereference(iterator).second + ids_and_fractions = [] + for i in range(c_value.size()): + ids_and_fractions.append((c_value[i].first, c_value[i].second)) + resources_dict[key] = ids_and_fractions + postincrement(iterator) + return resources_dict + + def push_error(self, DriverID job_id, error_type, error_message, + double timestamp): + check_status(self.client.get().PushError(job_id.data, + error_type.encode("ascii"), + error_message.encode("ascii"), + timestamp)) + + def push_profile_events(self, component_type, UniqueID component_id, + node_ip_address, profile_data): + cdef: + GCSProfileTableDataT profile_info + GCSProfileEventT *profile_event + c_string event_type + + if len(profile_data) == 0: + return # Short circuit if there are no profile events. + + profile_info.component_type = component_type.encode("ascii") + profile_info.component_id = component_id.binary() + profile_info.node_ip_address = node_ip_address.encode("ascii") + + for py_profile_event in profile_data: + profile_event = new GCSProfileEventT() + if not isinstance(py_profile_event, dict): + raise TypeError("Incorrect type for a profile event. Expected dict instead of '%s'" % str(type(py_profile_event))) + # TODO(rkn): If the dictionary is formatted incorrectly, that could lead + # to errors. E.g., if any of the strings are empty, that will cause + # segfaults in the node manager. + for key_string, event_data in py_profile_event.items(): + if key_string == "event_type": + profile_event.event_type = event_data.encode("ascii") + if profile_event.event_type.length() == 0: + raise ValueError("'event_type' should not be a null string.") + elif key_string == "start_time": + profile_event.start_time = float(event_data) + elif key_string == "end_time": + profile_event.end_time = float(event_data) + elif key_string == "extra_data": + profile_event.extra_data = event_data.encode("ascii") + if profile_event.extra_data.length() == 0: + raise ValueError("'extra_data' should not be a null string.") + else: + raise ValueError("Unknown profile event key '%s'" % key_string) + # Note that profile_info.profile_events is a vector of unique pointers, so + # profile_event will be deallocated when profile_info goes out of scope. + # "emplace_back" of vector has not been supported by Cython + profile_info.profile_events.push_back(unique_ptr[GCSProfileEventT](profile_event)) + + check_status(self.client.get().PushProfileEvents(profile_info)) + + def free_objects(self, object_ids, c_bool local_only): + cdef c_vector[CObjectID] free_ids = ObjectIDsToVector(object_ids) + check_status(self.client.get().FreeObjects(free_ids, local_only)) + + @property + def language(self): + return Language.from_native(self.client.get().GetLanguage()) + + @property + def client_id(self): + return ClientID.from_native(self.client.get().GetClientID()) + + @property + def driver_id(self): + return DriverID.from_native(self.client.get().GetDriverID()) + + @property + def is_worker(self): + return self.client.get().IsWorker() diff --git a/python/ray/actor.py b/python/ray/actor.py index e742201b32d6..6bb279573253 100644 --- a/python/ray/actor.py +++ b/python/ray/actor.py @@ -12,12 +12,12 @@ import ray.cloudpickle as pickle from ray.function_manager import FunctionDescriptor -import ray.raylet import ray.ray_constants as ray_constants import ray.signature as signature import ray.worker from ray.utils import _random_string -from ray import ObjectID +from ray import (ObjectID, ActorID, ActorHandleID, ActorClassID, TaskID, + DriverID) DEFAULT_ACTOR_METHOD_NUM_RETURN_VALS = 1 @@ -38,11 +38,12 @@ def compute_actor_handle_id(actor_handle_id, num_forks): Returns: An ID for the new actor handle. """ + assert isinstance(actor_handle_id, ActorHandleID) handle_id_hash = hashlib.sha1() - handle_id_hash.update(actor_handle_id.id()) + handle_id_hash.update(actor_handle_id.binary()) handle_id_hash.update(str(num_forks).encode("ascii")) handle_id = handle_id_hash.digest() - return ObjectID(handle_id) + return ActorHandleID(handle_id) def compute_actor_handle_id_non_forked(actor_handle_id, current_task_id): @@ -65,11 +66,13 @@ def compute_actor_handle_id_non_forked(actor_handle_id, current_task_id): Returns: An ID for the new actor handle. """ + assert isinstance(actor_handle_id, ActorHandleID) + assert isinstance(current_task_id, TaskID) handle_id_hash = hashlib.sha1() - handle_id_hash.update(actor_handle_id.id()) - handle_id_hash.update(current_task_id.id()) + handle_id_hash.update(actor_handle_id.binary()) + handle_id_hash.update(current_task_id.binary()) handle_id = handle_id_hash.digest() - return ObjectID(handle_id) + return ActorHandleID(handle_id) def set_actor_checkpoint(worker, actor_id, checkpoint_index, checkpoint, @@ -83,7 +86,8 @@ def set_actor_checkpoint(worker, actor_id, checkpoint_index, checkpoint, checkpoint: The state object to save. frontier: The task frontier at the time of the checkpoint. """ - actor_key = b"Actor:" + actor_id.id() + assert isinstance(actor_id, ActorID) + actor_key = b"Actor:" + actor_id.binary() worker.redis_client.hmset( actor_key, { "checkpoint_index": checkpoint_index, @@ -155,7 +159,8 @@ def get_actor_checkpoint(worker, actor_id): exists, all objects are set to None. The checkpoint index is the . executed on the actor before the checkpoint was made. """ - actor_key = b"Actor:" + actor_id.id() + assert isinstance(actor_id, ActorID) + actor_key = b"Actor:" + actor_id.binary() checkpoint_index, checkpoint, frontier = worker.redis_client.hmget( actor_key, ["checkpoint_index", "checkpoint", "frontier"]) if checkpoint_index is not None: @@ -370,7 +375,7 @@ def _remote(self, raise Exception("Actors cannot be created before ray.init() " "has been called.") - actor_id = ObjectID(_random_string()) + actor_id = ActorID(_random_string()) # The actor cursor is a dummy object representing the most recent # actor method invocation. For each subsequent method invocation, # the current cursor should be added as a dependency, and then @@ -423,6 +428,7 @@ def _remote(self, num_return_vals=1, resources=resources, placement_resources=actor_placement_resources) + assert isinstance(actor_cursor, ObjectID) actor_handle = ActorHandle( actor_id, self._modified_class.__module__, self._class_name, @@ -502,14 +508,17 @@ def __init__(self, actor_method_cpus, actor_driver_id, actor_handle_id=None): + assert isinstance(actor_id, ActorID) + assert isinstance(actor_driver_id, DriverID) self._ray_actor_id = actor_id self._ray_module_name = module_name # False if this actor handle was created by forking or pickling. True # if it was created by the _serialization_helper function. self._ray_original_handle = actor_handle_id is None if self._ray_original_handle: - self._ray_actor_handle_id = ObjectID.nil_id() + self._ray_actor_handle_id = ActorHandleID.nil() else: + assert isinstance(actor_handle_id, ActorHandleID) self._ray_actor_handle_id = actor_handle_id self._ray_actor_cursor = actor_cursor self._ray_actor_counter = 0 @@ -646,7 +655,7 @@ def __del__(self): # not just the first one. worker = ray.worker.get_global_worker() if (worker.mode == ray.worker.SCRIPT_MODE - and self._ray_actor_driver_id.id() != worker.worker_id): + and self._ray_actor_driver_id.binary() != worker.worker_id): # If the worker is a driver and driver id has changed because # Ray was shut down re-initialized, the actor is already cleaned up # and we don't need to send `__ray_terminate__` again. @@ -684,22 +693,22 @@ def _serialization_helper(self, ray_forking): else: actor_handle_id = self._ray_actor_handle_id + # Note: _ray_actor_cursor and _ray_actor_creation_dummy_object_id + # could be None. state = { - "actor_id": self._ray_actor_id.id(), - "actor_handle_id": actor_handle_id.id(), + "actor_id": self._ray_actor_id, + "actor_handle_id": actor_handle_id, "module_name": self._ray_module_name, "class_name": self._ray_class_name, - "actor_cursor": self._ray_actor_cursor.id() - if self._ray_actor_cursor is not None else None, + "actor_cursor": self._ray_actor_cursor, "actor_method_names": self._ray_actor_method_names, "method_signatures": self._ray_method_signatures, "method_num_return_vals": self._ray_method_num_return_vals, # Actors in local mode don't have dummy objects. "actor_creation_dummy_object_id": self. - _ray_actor_creation_dummy_object_id.id() - if self._ray_actor_creation_dummy_object_id is not None else None, + _ray_actor_creation_dummy_object_id, "actor_method_cpus": self._ray_actor_method_cpus, - "actor_driver_id": self._ray_actor_driver_id.id(), + "actor_driver_id": self._ray_actor_driver_id, "ray_forking": ray_forking } @@ -711,7 +720,7 @@ def _serialization_helper(self, ray_forking): # to release, since it could be unpickled and submit another # dependent task at any time. Therefore, we notify the backend of a # random handle ID that will never actually be used. - new_actor_handle_id = ObjectID(_random_string()) + new_actor_handle_id = ActorHandleID(_random_string()) # Notify the backend to expect this new actor handle. The backend will # not release the cursor for any new handles until the first task for # each of the new handles is submitted. @@ -733,7 +742,7 @@ def _deserialization_helper(self, state, ray_forking): worker.check_connected() if state["ray_forking"]: - actor_handle_id = ObjectID(state["actor_handle_id"]) + actor_handle_id = state["actor_handle_id"] else: # Right now, if the actor handle has been pickled, we create a # temporary actor handle id for invocations. @@ -747,25 +756,21 @@ def _deserialization_helper(self, state, ray_forking): # same actor is likely a performance bug. We should consider # logging a warning in these cases. actor_handle_id = compute_actor_handle_id_non_forked( - ObjectID(state["actor_handle_id"]), worker.current_task_id) - - # This is the driver ID of the driver that owns the actor, not - # necessarily the driver that owns this actor handle. - actor_driver_id = ObjectID(state["actor_driver_id"]) + state["actor_handle_id"], worker.current_task_id) self.__init__( - ObjectID(state["actor_id"]), + state["actor_id"], state["module_name"], state["class_name"], - ObjectID(state["actor_cursor"]) - if state["actor_cursor"] is not None else None, + state["actor_cursor"], state["actor_method_names"], state["method_signatures"], state["method_num_return_vals"], - ObjectID(state["actor_creation_dummy_object_id"]) - if state["actor_creation_dummy_object_id"] is not None else None, + state["actor_creation_dummy_object_id"], state["actor_method_cpus"], - actor_driver_id, + # This is the driver ID of the driver that owns the actor, not + # necessarily the driver that owns this actor handle. + state["actor_driver_id"], actor_handle_id=actor_handle_id) def __getstate__(self): @@ -887,7 +892,7 @@ def __ray_checkpoint_restore__(self): Class.__module__ = cls.__module__ Class.__name__ = cls.__name__ - class_id = _random_string() + class_id = ActorClassID(_random_string()) return ActorClass(Class, class_id, checkpoint_interval, max_reconstructions, num_cpus, num_gpus, resources, diff --git a/python/ray/experimental/async_plasma.py b/python/ray/experimental/async_plasma.py index 2c0f806f2467..58bb859408bf 100644 --- a/python/ray/experimental/async_plasma.py +++ b/python/ray/experimental/async_plasma.py @@ -215,7 +215,7 @@ def as_future(self, object_id, check_ready=True): if not isinstance(object_id, ray.ObjectID): raise TypeError("Input should be an ObjectID.") - plain_object_id = plasma.ObjectID(object_id.id()) + plain_object_id = plasma.ObjectID(object_id.binary()) fut = PlasmaObjectFuture(loop=self._loop, object_id=plain_object_id) if check_ready: diff --git a/python/ray/experimental/features.py b/python/ray/experimental/features.py index ad3ebca77560..90f893f271fb 100644 --- a/python/ray/experimental/features.py +++ b/python/ray/experimental/features.py @@ -91,7 +91,7 @@ def _task_table_shard(shard_index): for key in task_table_keys: task_id_binary = key[len(TASK_PREFIX):] results[binary_to_hex(task_id_binary)] = ray.global_state._task_table( - ray.ObjectID(task_id_binary)) + ray.TaskID(task_id_binary)) return results diff --git a/python/ray/experimental/state.py b/python/ray/experimental/state.py index 20a3bc6137fa..ffc12173a24a 100644 --- a/python/ray/experimental/state.py +++ b/python/ray/experimental/state.py @@ -25,7 +25,7 @@ def parse_client_table(redis_client): Returns: A list of information about the nodes in the cluster. """ - NIL_CLIENT_ID = ray.ObjectID.nil_id().id() + NIL_CLIENT_ID = ray.ObjectID.nil().binary() message = redis_client.execute_command("RAY.TABLE_LOOKUP", ray.gcs_utils.TablePrefix.CLIENT, "", NIL_CLIENT_ID) @@ -216,8 +216,7 @@ def _object_table(self, object_id): """Fetch and parse the object table information for a single object ID. Args: - object_id_binary: A string of bytes with the object ID to get - information about. + object_id: An object ID to get information about. Returns: A dictionary with information about the object ID in question. @@ -229,7 +228,7 @@ def _object_table(self, object_id): # Return information about a single object ID. message = self._execute_command(object_id, "RAY.TABLE_LOOKUP", ray.gcs_utils.TablePrefix.OBJECT, "", - object_id.id()) + object_id.binary()) gcs_entry = ray.gcs_utils.GcsTableEntry.GetRootAsGcsTableEntry( message, 0) @@ -284,15 +283,15 @@ def _task_table(self, task_id): """Fetch and parse the task table information for a single task ID. Args: - task_id_binary: A string of bytes with the task ID to get - information about. + task_id: A task ID to get information about. Returns: A dictionary with information about the task ID in question. """ + assert isinstance(task_id, ray.TaskID) message = self._execute_command(task_id, "RAY.TABLE_LOOKUP", ray.gcs_utils.TablePrefix.RAYLET_TASK, - "", task_id.id()) + "", task_id.binary()) gcs_entries = ray.gcs_utils.GcsTableEntry.GetRootAsGcsTableEntry( message, 0) @@ -303,23 +302,23 @@ def _task_table(self, task_id): execution_spec = task_table_message.TaskExecutionSpec() task_spec = task_table_message.TaskSpecification() - task_spec = ray.raylet.task_from_string(task_spec) - function_descriptor_list = task_spec.function_descriptor_list() + task = ray._raylet.Task.from_string(task_spec) + function_descriptor_list = task.function_descriptor_list() function_descriptor = FunctionDescriptor.from_bytes_list( function_descriptor_list) task_spec_info = { - "DriverID": task_spec.driver_id().hex(), - "TaskID": task_spec.task_id().hex(), - "ParentTaskID": task_spec.parent_task_id().hex(), - "ParentCounter": task_spec.parent_counter(), - "ActorID": (task_spec.actor_id().hex()), - "ActorCreationID": task_spec.actor_creation_id().hex(), + "DriverID": task.driver_id().hex(), + "TaskID": task.task_id().hex(), + "ParentTaskID": task.parent_task_id().hex(), + "ParentCounter": task.parent_counter(), + "ActorID": (task.actor_id().hex()), + "ActorCreationID": task.actor_creation_id().hex(), "ActorCreationDummyObjectID": ( - task_spec.actor_creation_dummy_object_id().hex()), - "ActorCounter": task_spec.actor_counter(), - "Args": task_spec.arguments(), - "ReturnObjectIDs": task_spec.returns(), - "RequiredResources": task_spec.required_resources(), + task.actor_creation_dummy_object_id().hex()), + "ActorCounter": task.actor_counter(), + "Args": task.arguments(), + "ReturnObjectIDs": task.returns(), + "RequiredResources": task.required_resources(), "FunctionID": function_descriptor.function_id.hex(), "FunctionHash": binary_to_hex(function_descriptor.function_hash), "ModuleName": function_descriptor.module_name, @@ -351,7 +350,7 @@ def task_table(self, task_id=None): """ self._check_connected() if task_id is not None: - task_id = ray.ObjectID(hex_to_binary(task_id)) + task_id = ray.TaskID(hex_to_binary(task_id)) return self._task_table(task_id) else: task_table_keys = self._keys( @@ -364,7 +363,7 @@ def task_table(self, task_id=None): results = {} for task_id_binary in task_ids_binary: results[binary_to_hex(task_id_binary)] = self._task_table( - ray.ObjectID(task_id_binary)) + ray.TaskID(task_id_binary)) return results def function_table(self, function_id=None): @@ -439,7 +438,7 @@ def _profile_table(self, batch_id): # events and should also support returning a window of events. message = self._execute_command(batch_id, "RAY.TABLE_LOOKUP", ray.gcs_utils.TablePrefix.PROFILE, "", - batch_id.id()) + batch_id.binary()) if message is None: return [] @@ -877,9 +876,10 @@ def _error_messages(self, job_id): Returns: A list of the error messages for this job. """ + assert isinstance(job_id, ray.DriverID) message = self.redis_client.execute_command( "RAY.TABLE_LOOKUP", ray.gcs_utils.TablePrefix.ERROR_INFO, "", - job_id.id()) + job_id.binary()) # If there are no errors, return early. if message is None: @@ -891,7 +891,7 @@ def _error_messages(self, job_id): for i in range(gcs_entries.EntriesLength()): error_data = ray.gcs_utils.ErrorTableData.GetRootAsErrorTableData( gcs_entries.Entries(i), 0) - assert job_id.id() == error_data.JobId() + assert job_id.binary() == error_data.JobId() error_message = { "type": decode(error_data.Type()), "message": decode(error_data.ErrorMessage()), @@ -912,6 +912,7 @@ def error_messages(self, job_id=None): that job. """ if job_id is not None: + assert isinstance(job_id, ray.DriverID) return self._error_messages(job_id) error_table_keys = self.redis_client.keys( @@ -922,6 +923,6 @@ def error_messages(self, job_id=None): ] return { - binary_to_hex(job_id): self._error_messages(ray.ObjectID(job_id)) + binary_to_hex(job_id): self._error_messages(ray.DriverID(job_id)) for job_id in job_ids } diff --git a/python/ray/function_manager.py b/python/ray/function_manager.py index 45c22004bba8..aa081a4d9572 100644 --- a/python/ray/function_manager.py +++ b/python/ray/function_manager.py @@ -223,7 +223,7 @@ def _get_function_id(self): ray.ObjectID to represent the function descriptor. """ if self.is_for_driver_task: - return ray.ObjectID.nil_id() + return ray.FunctionID.nil() function_id_hash = hashlib.sha1() # Include the function module and name in the hash. function_id_hash.update(self.module_name.encode("ascii")) @@ -232,7 +232,7 @@ def _get_function_id(self): function_id_hash.update(self._function_source_hash) # Compute the function ID. function_id = function_id_hash.digest() - return ray.ObjectID(function_id) + return ray.FunctionID(function_id) def get_function_descriptor_list(self): """Return a list of bytes representing the function descriptor. @@ -355,13 +355,13 @@ def _do_export(self, remote_function): check_oversized_pickle(pickled_function, remote_function._function_name, "remote function", self._worker) - key = (b"RemoteFunction:" + self._worker.task_driver_id.id() + b":" + - remote_function._function_descriptor.function_id.id()) + key = (b"RemoteFunction:" + self._worker.task_driver_id.binary() + b":" + + remote_function._function_descriptor.function_id.binary()) self._worker.redis_client.hmset( key, { - "driver_id": self._worker.task_driver_id.id(), + "driver_id": self._worker.task_driver_id.binary(), "function_id": remote_function._function_descriptor. - function_id.id(), + function_id.binary(), "name": remote_function._function_name, "module": function.__module__, "function": pickled_function, @@ -377,8 +377,8 @@ def fetch_and_register_remote_function(self, key): "driver_id", "function_id", "name", "function", "num_return_vals", "module", "resources", "max_calls" ]) - function_id = ray.ObjectID(function_id_str) - driver_id = ray.ObjectID(driver_id_str) + function_id = ray.FunctionID(function_id_str) + driver_id = ray.DriverID(driver_id_str) function_name = decode(function_name) max_calls = int(max_calls) module = decode(module) @@ -406,7 +406,7 @@ def f(): traceback_str, driver_id=driver_id, data={ - "function_id": function_id.id(), + "function_id": function_id.binary(), "function_name": function_name }) else: @@ -423,7 +423,8 @@ def f(): max_calls=max_calls)) # Add the function to the function table. self._worker.redis_client.rpush( - b"FunctionTable:" + function_id.id(), self._worker.worker_id) + b"FunctionTable:" + function_id.binary(), + self._worker.worker_id) def get_execution_info(self, driver_id, function_descriptor): """Get the FunctionExecutionInfo of a remote function. @@ -524,14 +525,14 @@ def export_actor_class(self, Class, actor_method_names, "You might have started a background thread in a non-actor task, " "please make sure the thread finishes before the task finishes.") driver_id = self._worker.task_driver_id - key = (b"ActorClass:" + driver_id.id() + b":" + - function_descriptor.function_id.id()) + key = (b"ActorClass:" + driver_id.binary() + b":" + + function_descriptor.function_id.binary()) actor_class_info = { "class_name": Class.__name__, "module": Class.__module__, "class": pickle.dumps(Class), "checkpoint_interval": checkpoint_interval, - "driver_id": driver_id.id(), + "driver_id": driver_id.binary(), "actor_method_names": json.dumps(list(actor_method_names)) } @@ -556,8 +557,8 @@ def export_actor_class(self, Class, actor_method_names, # because of https://github.com/ray-project/ray/issues/1146. def load_actor(self, driver_id, function_descriptor): - key = (b"ActorClass:" + driver_id.id() + b":" + - function_descriptor.function_id.id()) + key = (b"ActorClass:" + driver_id.binary() + b":" + + function_descriptor.function_id.binary()) # Wait for the actor class key to have been imported by the # import thread. TODO(rkn): It shouldn't be possible to end # up in an infinite loop here, but we should push an error to @@ -588,7 +589,7 @@ def fetch_and_register_actor(self, actor_class_key): class_name = decode(class_name) module = decode(module) - driver_id = ray.ObjectID(driver_id_str) + driver_id = ray.DriverID(driver_id_str) checkpoint_interval = int(checkpoint_interval) actor_method_names = json.loads(decode(actor_method_names)) @@ -645,7 +646,7 @@ def temporary_actor_method(*xs): ray_constants.REGISTER_ACTOR_PUSH_ERROR, traceback_str, driver_id, - data={"actor_id": actor_id.id()}) + data={"actor_id": actor_id.binary()}) # TODO(rkn): In the future, it might make sense to have the worker # exit here. However, currently that would lead to hanging if # someone calls ray.get on a method invoked on the actor. diff --git a/python/ray/gcs_utils.py b/python/ray/gcs_utils.py index 511cbeea09f3..c3ba3d06b237 100644 --- a/python/ray/gcs_utils.py +++ b/python/ray/gcs_utils.py @@ -58,7 +58,7 @@ def construct_error_message(driver_id, error_type, message, timestamp): The serialized object. """ builder = flatbuffers.Builder(0) - driver_offset = builder.CreateString(driver_id.id()) + driver_offset = builder.CreateString(driver_id.binary()) error_type_offset = builder.CreateString(error_type) message_offset = builder.CreateString(message) diff --git a/python/ray/import_thread.py b/python/ray/import_thread.py index e1c8742b9c44..c848517eda1f 100644 --- a/python/ray/import_thread.py +++ b/python/ray/import_thread.py @@ -112,7 +112,7 @@ def fetch_and_execute_function_to_run(self, key): if (utils.decode(run_on_other_drivers) == "False" and self.worker.mode == ray.SCRIPT_MODE - and driver_id != self.worker.task_driver_id.id()): + and driver_id != self.worker.task_driver_id.binary()): return try: @@ -131,5 +131,5 @@ def fetch_and_execute_function_to_run(self, key): self.worker, ray_constants.FUNCTION_TO_RUN_PUSH_ERROR, traceback_str, - driver_id=ray.ObjectID(driver_id), + driver_id=ray.DriverID(driver_id), data={"name": name}) diff --git a/src/ray/api.h b/python/ray/includes/__init__.pxd similarity index 100% rename from src/ray/api.h rename to python/ray/includes/__init__.pxd diff --git a/python/ray/includes/common.pxd b/python/ray/includes/common.pxd new file mode 100644 index 000000000000..5601ab2807b6 --- /dev/null +++ b/python/ray/includes/common.pxd @@ -0,0 +1,120 @@ +from libcpp.string cimport string as c_string +from libcpp cimport bool as c_bool + +from libc.stdint cimport int64_t +from libcpp.unordered_map cimport unordered_map +from libcpp.vector cimport vector as c_vector + +from ray.includes.unique_ids cimport ( + CUniqueID, TaskID as CTaskID, ObjectID as CObjectID, + FunctionID as CFunctionID, ActorClassID as CActorClassID, ActorID as CActorID, + ActorHandleID as CActorHandleID, WorkerID as CWorkerID, + DriverID as CDriverID, ConfigID as CConfigID, ClientID as CClientID) + + +cdef extern from "ray/status.h" namespace "ray" nogil: + cdef cppclass StatusCode: + pass + + cdef cppclass CRayStatus "ray::Status": + RayStatus() + RayStatus(StatusCode code, const c_string &msg) + RayStatus(const CRayStatus &s); + + @staticmethod + CRayStatus OK() + @staticmethod + CRayStatus OutOfMemory() + @staticmethod + CRayStatus KeyError() + @staticmethod + CRayStatus Invalid() + @staticmethod + CRayStatus IOError() + @staticmethod + CRayStatus TypeError() + @staticmethod + CRayStatus UnknownError() + @staticmethod + CRayStatus NotImplemented() + @staticmethod + CRayStatus RedisError() + + c_bool ok() + c_bool IsOutOfMemory() + c_bool IsKeyError() + c_bool IsInvalid() + c_bool IsIOError() + c_bool IsTypeError() + c_bool IsUnknownError() + c_bool IsNotImplemented() + c_bool IsRedisError() + + c_string ToString() + c_string CodeAsString() + StatusCode code() + c_string message() + + # We can later add more of the common status factory methods as needed + cdef CRayStatus RayStatus_OK "Status::OK"() + cdef CRayStatus RayStatus_Invalid "Status::Invalid"() + + +cdef extern from "ray/status.h" namespace "ray::StatusCode" nogil: + cdef StatusCode StatusCode_OK "OK" + cdef StatusCode StatusCode_OutOfMemory "OutOfMemory" + cdef StatusCode StatusCode_KeyError "KeyError" + cdef StatusCode StatusCode_TypeError "TypeError" + cdef StatusCode StatusCode_Invalid "Invalid" + cdef StatusCode StatusCode_IOError "IOError" + cdef StatusCode StatusCode_UnknownError "UnknownError" + cdef StatusCode StatusCode_NotImplemented "NotImplemented" + cdef StatusCode StatusCode_RedisError "RedisError" + + +cdef extern from "ray/id.h" namespace "ray" nogil: + const CTaskID FinishTaskId(const CTaskID &task_id) + const CObjectID ComputeReturnId(const CTaskID &task_id, + int64_t return_index) + const CObjectID ComputePutId(const CTaskID &task_id, int64_t put_index) + const CTaskID ComputeTaskId(const CObjectID &object_id) + const CTaskID GenerateTaskId(const CDriverID &driver_id, + const CTaskID &parent_task_id, + int parent_task_counter) + int64_t ComputeObjectIndex(const CObjectID &object_id) + + +cdef extern from "ray/gcs/format/gcs_generated.h" nogil: + cdef cppclass GCSArg "Arg": + pass + + cdef cppclass CLanguage "Language": + pass + + +# This is a workaround for C++ enum class since Cython has no corresponding representation. +cdef extern from "ray/gcs/format/gcs_generated.h" namespace "Language" nogil: + cdef CLanguage LANGUAGE_PYTHON "Language::PYTHON" + cdef CLanguage LANGUAGE_CPP "Language::CPP" + cdef CLanguage LANGUAGE_JAVA "Language::JAVA" + + +cdef extern from "ray/raylet/scheduling_resources.h" namespace "ray::raylet" nogil: + cdef cppclass ResourceSet "ResourceSet": + ResourceSet() + ResourceSet(const unordered_map[c_string, double] &resource_map) + ResourceSet(const c_vector[c_string] &resource_labels, const c_vector[double] resource_capacity) + c_bool operator==(const ResourceSet &rhs) const + c_bool IsEqual(const ResourceSet &other) const + c_bool IsSubset(const ResourceSet &other) const + c_bool IsSuperset(const ResourceSet &other) const + c_bool AddResource(const c_string &resource_name, double capacity) + c_bool RemoveResource(const c_string &resource_name) + c_bool AddResourcesStrict(const ResourceSet &other) + void AddResources(const ResourceSet &other) + c_bool SubtractResourcesStrict(const ResourceSet &other) + c_bool GetResource(const c_string &resource_name, double *value) const + double GetNumCpus() const + c_bool IsEmpty() const + const unordered_map[c_string, double] &GetResourceMap() const + const c_string ToString() const diff --git a/python/ray/includes/libraylet.pxd b/python/ray/includes/libraylet.pxd new file mode 100644 index 000000000000..63c15f659489 --- /dev/null +++ b/python/ray/includes/libraylet.pxd @@ -0,0 +1,64 @@ +from libc.stdint cimport int64_t +from libcpp cimport bool as c_bool +from libcpp.memory cimport unique_ptr +from libcpp.string cimport string as c_string +from libcpp.utility cimport pair +from libcpp.unordered_map cimport unordered_map +from libcpp.vector cimport vector as c_vector + + +from ray.includes.common cimport ( + CUniqueID, CTaskID, CObjectID, CFunctionID, CActorClassID, CActorID, + CActorHandleID, CWorkerID, CDriverID, CConfigID, CClientID, + CLanguage, CRayStatus) +from ray.includes.task cimport CTaskSpecification + + +cdef extern from "ray/gcs/format/gcs_generated.h" nogil: + cdef cppclass GCSProfileEventT "ProfileEventT": + c_string event_type + double start_time + double end_time + c_string extra_data + GCSProfileEventT() + + cdef cppclass GCSProfileTableDataT "ProfileTableDataT": + c_string component_type + c_string component_id + c_string node_ip_address + c_vector[unique_ptr[GCSProfileEventT]] profile_events + GCSProfileTableDataT() + + +ctypedef unordered_map[c_string, c_vector[pair[int64_t, double]]] ResourceMappingType +ctypedef pair[c_vector[CObjectID], c_vector[CObjectID]] WaitResultPair + + +cdef extern from "ray/raylet/raylet_client.h" nogil: + cdef cppclass CRayletClient "RayletClient": + CRayletClient(const c_string &raylet_socket, + const CClientID &client_id, + c_bool is_worker, const CDriverID &driver_id, + const CLanguage &language) + CRayStatus Disconnect() + CRayStatus SubmitTask(const c_vector[CObjectID] &execution_dependencies, + const CTaskSpecification &task_spec) + CRayStatus GetTask(unique_ptr[CTaskSpecification] *task_spec) + CRayStatus TaskDone() + CRayStatus FetchOrReconstruct(c_vector[CObjectID] &object_ids, + c_bool fetch_only, + const CTaskID ¤t_task_id) + CRayStatus NotifyUnblocked(const CTaskID ¤t_task_id) + CRayStatus Wait(const c_vector[CObjectID] &object_ids, int num_returns, + int64_t timeout_milliseconds, c_bool wait_local, + const CTaskID ¤t_task_id, WaitResultPair *result) + CRayStatus PushError(const CDriverID &job_id, const c_string &type, + const c_string &error_message, double timestamp) + CRayStatus PushProfileEvents(const GCSProfileTableDataT &profile_events) + CRayStatus FreeObjects(const c_vector[CObjectID] &object_ids, + c_bool local_only) + CLanguage GetLanguage() const + CClientID GetClientID() const + CDriverID GetDriverID() const + c_bool IsWorker() const + const ResourceMappingType &GetResourceIDs() const diff --git a/python/ray/includes/ray_config.pxd b/python/ray/includes/ray_config.pxd new file mode 100644 index 000000000000..906addc99e5c --- /dev/null +++ b/python/ray/includes/ray_config.pxd @@ -0,0 +1,83 @@ +from libc.stdint cimport int64_t, uint64_t +from libcpp.string cimport string as c_string +from libcpp.unordered_map cimport unordered_map + + +cdef extern from "ray/ray_config.h" nogil: + cdef cppclass RayConfig "RayConfig": + @staticmethod + RayConfig &instance() + + int64_t ray_protocol_version() const + + int64_t handler_warning_timeout_ms() const + + int64_t heartbeat_timeout_milliseconds() const + + int64_t debug_dump_period_milliseconds() const + + int64_t num_heartbeats_timeout() const + + uint64_t num_heartbeats_warning() const + + int64_t initial_reconstruction_timeout_milliseconds() const + + int64_t get_timeout_milliseconds() const + + uint64_t max_lineage_size() const + + int64_t worker_get_request_size() const + + int64_t worker_fetch_request_size() const + + int64_t actor_max_dummy_objects() const + + int64_t num_connect_attempts() const + + int64_t connect_timeout_milliseconds() const + + int64_t local_scheduler_fetch_timeout_milliseconds() const + + int64_t local_scheduler_reconstruction_timeout_milliseconds() const + + int64_t max_num_to_reconstruct() const + + int64_t local_scheduler_fetch_request_size() const + + int64_t kill_worker_timeout_milliseconds() const + + int64_t max_time_for_handler_milliseconds() const + + int64_t size_limit() const + + int64_t num_elements_limit() const + + int64_t max_time_for_loop() const + + int64_t redis_db_connect_retries() + + int64_t redis_db_connect_wait_milliseconds() const + + int64_t plasma_default_release_delay() const + + int64_t L3_cache_size_bytes() const + + int64_t max_tasks_to_spillback() const + + int64_t actor_creation_num_spillbacks_warning() const + + int node_manager_forward_task_retry_timeout_milliseconds() const + + int object_manager_pull_timeout_ms() const + + int object_manager_push_timeout_ms() const + + int object_manager_repeated_push_delay_ms() const + + uint64_t object_manager_default_chunk_size() const + + int num_workers_per_process() const + + int64_t max_task_lease_timeout_ms() const + + void initialize(const unordered_map[c_string, int] &config_map) diff --git a/python/ray/includes/ray_config.pxi b/python/ray/includes/ray_config.pxi new file mode 100644 index 000000000000..72b3e1e3d051 --- /dev/null +++ b/python/ray/includes/ray_config.pxi @@ -0,0 +1,146 @@ +from ray.includes.ray_config cimport RayConfig + +cdef class Config: + @staticmethod + def ray_protocol_version(): + return RayConfig.instance().ray_protocol_version() + + @staticmethod + def handler_warning_timeout_ms(): + return RayConfig.instance().handler_warning_timeout_ms() + + @staticmethod + def heartbeat_timeout_milliseconds(): + return RayConfig.instance().heartbeat_timeout_milliseconds() + + @staticmethod + def debug_dump_period_milliseconds(): + return RayConfig.instance().debug_dump_period_milliseconds() + + @staticmethod + def num_heartbeats_timeout(): + return RayConfig.instance().num_heartbeats_timeout() + + @staticmethod + def num_heartbeats_warning(): + return RayConfig.instance().num_heartbeats_warning() + + @staticmethod + def initial_reconstruction_timeout_milliseconds(): + return RayConfig.instance().initial_reconstruction_timeout_milliseconds() + + @staticmethod + def get_timeout_milliseconds(): + return RayConfig.instance().get_timeout_milliseconds() + + @staticmethod + def max_lineage_size(): + return RayConfig.instance().max_lineage_size() + + @staticmethod + def worker_get_request_size(): + return RayConfig.instance().worker_get_request_size() + + @staticmethod + def worker_fetch_request_size(): + return RayConfig.instance().worker_fetch_request_size() + + @staticmethod + def actor_max_dummy_objects(): + return RayConfig.instance().actor_max_dummy_objects() + + @staticmethod + def num_connect_attempts(): + return RayConfig.instance().num_connect_attempts() + + @staticmethod + def connect_timeout_milliseconds(): + return RayConfig.instance().connect_timeout_milliseconds() + + @staticmethod + def local_scheduler_fetch_timeout_milliseconds(): + return RayConfig.instance().local_scheduler_fetch_timeout_milliseconds() + + @staticmethod + def local_scheduler_reconstruction_timeout_milliseconds(): + return RayConfig.instance().local_scheduler_reconstruction_timeout_milliseconds() + + @staticmethod + def max_num_to_reconstruct(): + return RayConfig.instance().max_num_to_reconstruct() + + @staticmethod + def local_scheduler_fetch_request_size(): + return RayConfig.instance().local_scheduler_fetch_request_size() + + @staticmethod + def kill_worker_timeout_milliseconds(): + return RayConfig.instance().kill_worker_timeout_milliseconds() + + @staticmethod + def max_time_for_handler_milliseconds(): + return RayConfig.instance().max_time_for_handler_milliseconds() + + @staticmethod + def size_limit(): + return RayConfig.instance().size_limit() + + @staticmethod + def num_elements_limit(): + return RayConfig.instance().num_elements_limit() + + @staticmethod + def max_time_for_loop(): + return RayConfig.instance().max_time_for_loop() + + @staticmethod + def redis_db_connect_retries(): + return RayConfig.instance().redis_db_connect_retries() + + @staticmethod + def redis_db_connect_wait_milliseconds(): + return RayConfig.instance().redis_db_connect_wait_milliseconds() + + @staticmethod + def plasma_default_release_delay(): + return RayConfig.instance().plasma_default_release_delay() + + @staticmethod + def L3_cache_size_bytes(): + return RayConfig.instance().L3_cache_size_bytes() + + @staticmethod + def max_tasks_to_spillback(): + return RayConfig.instance().max_tasks_to_spillback() + + @staticmethod + def actor_creation_num_spillbacks_warning(): + return RayConfig.instance().actor_creation_num_spillbacks_warning() + + @staticmethod + def node_manager_forward_task_retry_timeout_milliseconds(): + return RayConfig.instance().node_manager_forward_task_retry_timeout_milliseconds() + + @staticmethod + def object_manager_pull_timeout_ms(): + return RayConfig.instance().object_manager_pull_timeout_ms() + + @staticmethod + def object_manager_push_timeout_ms(): + return RayConfig.instance().object_manager_push_timeout_ms() + + @staticmethod + def object_manager_repeated_push_delay_ms(): + return RayConfig.instance().object_manager_repeated_push_delay_ms() + + @staticmethod + def object_manager_default_chunk_size(): + return RayConfig.instance().object_manager_default_chunk_size() + + @staticmethod + def num_workers_per_process(): + return RayConfig.instance().num_workers_per_process() + + @staticmethod + def max_task_lease_timeout_ms(): + return RayConfig.instance().max_task_lease_timeout_ms() diff --git a/python/ray/includes/task.pxd b/python/ray/includes/task.pxd new file mode 100644 index 000000000000..80b02c7d4546 --- /dev/null +++ b/python/ray/includes/task.pxd @@ -0,0 +1,102 @@ +from libc.stdint cimport int64_t, uint8_t +from libcpp cimport bool as c_bool +from libcpp.memory cimport unique_ptr, shared_ptr +from libcpp.string cimport string as c_string +from libcpp.unordered_map cimport unordered_map +from libcpp.vector cimport vector as c_vector + +from ray.includes.common cimport ( + CUniqueID, CTaskID, CObjectID, CFunctionID, CActorClassID, CActorID, + CActorHandleID, CWorkerID, CDriverID, CConfigID, CClientID, + CLanguage, ResourceSet) + + +cdef extern from "ray/raylet/task_execution_spec.h" namespace "ray::raylet" nogil: + cdef cppclass CTaskExecutionSpecification "ray::raylet::TaskExecutionSpecification": + CTaskExecutionSpecification(const c_vector[CObjectID] &&dependencies) + CTaskExecutionSpecification(const c_vector[CObjectID] &&dependencies, int num_forwards) + c_vector[CObjectID] ExecutionDependencies() const + void SetExecutionDependencies(const c_vector[CObjectID] &dependencies) + int NumForwards() const + void IncrementNumForwards() + int64_t LastTimestamp() const + void SetLastTimestamp(int64_t new_timestamp) + + +cdef extern from "ray/raylet/task_spec.h" namespace "ray::raylet" nogil: + cdef cppclass CTaskArgument "ray::raylet::TaskArgument": + pass + + cdef cppclass CTaskArgumentByReference "ray::raylet::TaskArgumentByReference": + CTaskArgumentByReference(const c_vector[CObjectID] &references); + + cdef cppclass CTaskArgumentByValue "ray::raylet::TaskArgumentByValue": + CTaskArgumentByValue(const uint8_t *value, size_t length); + + cdef cppclass CTaskSpecification "ray::raylet::TaskSpecification": + CTaskSpecification(const CDriverID &driver_id, const CTaskID &parent_task_id, + int64_t parent_counter, + const c_vector[shared_ptr[CTaskArgument]] &task_arguments, + int64_t num_returns, + const unordered_map[c_string, double] &required_resources, + const CLanguage &language, + const c_vector[c_string] &function_descriptor) + CTaskSpecification( + const CDriverID &driver_id, const CTaskID &parent_task_id, int64_t parent_counter, + const CActorID &actor_creation_id, const CObjectID &actor_creation_dummy_object_id, + int64_t max_actor_reconstructions, const CActorID &actor_id, + const CActorHandleID &actor_handle_id, int64_t actor_counter, + const c_vector[CActorHandleID] &new_actor_handles, + const c_vector[shared_ptr[CTaskArgument]] &task_arguments, + int64_t num_returns, + const unordered_map[c_string, double] &required_resources, + const unordered_map[c_string, double] &required_placement_resources, + const CLanguage &language, const c_vector[c_string] &function_descriptor) + CTaskSpecification(const c_string &string) + c_string SerializeAsString() const + + CTaskID TaskId() const + CDriverID DriverId() const + CTaskID ParentTaskId() const + int64_t ParentCounter() const + c_vector[c_string] FunctionDescriptor() const + c_string FunctionDescriptorString() const + int64_t NumArgs() const + int64_t NumReturns() const + c_bool ArgByRef(int64_t arg_index) const + int ArgIdCount(int64_t arg_index) const + CObjectID ArgId(int64_t arg_index, int64_t id_index) const + CObjectID ReturnId(int64_t return_index) const + const uint8_t *ArgVal(int64_t arg_index) const + size_t ArgValLength(int64_t arg_index) const + double GetRequiredResource(const c_string &resource_name) const + const ResourceSet GetRequiredResources() const + const ResourceSet GetRequiredPlacementResources() const + c_bool IsDriverTask() const + CLanguage GetLanguage() const + + c_bool IsActorCreationTask() const + c_bool IsActorTask() const + CActorID ActorCreationId() const + CObjectID ActorCreationDummyObjectId() const + int64_t MaxActorReconstructions() const + CActorID ActorId() const + CActorHandleID ActorHandleId() const + int64_t ActorCounter() const + CObjectID ActorDummyObject() const + c_vector[CActorHandleID] NewActorHandles() const + + +cdef extern from "ray/raylet/task.h" namespace "ray::raylet" nogil: + cdef cppclass CTask "ray::raylet::Task": + CTask(const CTaskExecutionSpecification &execution_spec, + const CTaskSpecification &task_spec) + const CTaskExecutionSpecification &GetTaskExecutionSpec() const + const CTaskSpecification &GetTaskSpecification() const + void SetExecutionDependencies(const c_vector[CObjectID] &dependencies) + void IncrementNumForwards() + const c_vector[CObjectID] &GetDependencies() const + void CopyTaskExecutionSpec(const CTask &task) + + cdef c_string SerializeTaskAsString(const c_vector[CObjectID] *dependencies, + const CTaskSpecification *task_spec) diff --git a/python/ray/includes/task.pxi b/python/ray/includes/task.pxi new file mode 100644 index 000000000000..aa1e8f66cb2f --- /dev/null +++ b/python/ray/includes/task.pxi @@ -0,0 +1,185 @@ +from libc.stdint cimport uint8_t +from libcpp.memory cimport shared_ptr, make_shared, static_pointer_cast +from ray.includes.task cimport CTaskSpecification, CTaskArgument, CTaskArgumentByValue, CTaskArgumentByReference, SerializeTaskAsString + +from ray.utils import _random_string + +cdef class Task: + cdef: + unique_ptr[CTaskSpecification] task_spec + unique_ptr[c_vector[CObjectID]] execution_dependencies + + def __init__(self, DriverID driver_id, function_descriptor, arguments, + int num_returns, TaskID parent_task_id, int parent_counter, + ActorID actor_creation_id, + ObjectID actor_creation_dummy_object_id, + int32_t max_actor_reconstructions, ActorID actor_id, + ActorHandleID actor_handle_id, int actor_counter, + new_actor_handles, execution_arguments, resource_map, + placement_resource_map): + cdef: + unordered_map[c_string, double] required_resources + unordered_map[c_string, double] required_placement_resources + c_vector[shared_ptr[CTaskArgument]] task_args + c_vector[CActorHandleID] task_new_actor_handles + c_vector[c_string] c_function_descriptor + c_string pickled_str + c_vector[CObjectID] references + + for item in function_descriptor: + if not isinstance(item, bytes): + raise TypeError("'function_descriptor' takes a list of byte strings.") + c_function_descriptor.push_back(item) + + # Parse the resource map. + if resource_map is not None: + required_resources = resource_map_from_python_dict(resource_map) + if required_resources.count(b"CPU") == 0: + required_resources[b"CPU"] = 1.0 + if placement_resource_map is not None: + required_placement_resources = resource_map_from_python_dict(placement_resource_map) + + # Parse the arguments from the list. + for arg in arguments: + if isinstance(arg, ObjectID): + references = c_vector[CObjectID]() + references.push_back((arg).data) + task_args.push_back(static_pointer_cast[CTaskArgument, CTaskArgumentByReference](make_shared[CTaskArgumentByReference](references))) + else: + pickled_str = pickle.dumps(arg, protocol=pickle.HIGHEST_PROTOCOL) + task_args.push_back(static_pointer_cast[CTaskArgument, CTaskArgumentByValue](make_shared[CTaskArgumentByValue](pickled_str.c_str(), pickled_str.size()))) + + for new_actor_handle in new_actor_handles: + task_new_actor_handles.push_back((new_actor_handle).data) + + self.task_spec.reset(new CTaskSpecification( + CUniqueID(driver_id.data), parent_task_id.data, parent_counter, actor_creation_id.data, + actor_creation_dummy_object_id.data, max_actor_reconstructions, CUniqueID(actor_id.data), + CUniqueID(actor_handle_id.data), actor_counter, task_new_actor_handles, task_args, num_returns, + required_resources, required_placement_resources, LANGUAGE_PYTHON, + c_function_descriptor)) + + # Set the task's execution dependencies. + self.execution_dependencies.reset(new c_vector[CObjectID]()) + if execution_arguments is not None: + for execution_arg in execution_arguments: + self.execution_dependencies.get().push_back((execution_arg).data) + + @staticmethod + cdef make(unique_ptr[CTaskSpecification]& task_spec): + cdef Task self = Task.__new__(Task) + self.task_spec.reset(task_spec.release()) + # The created task does not include any execution dependencies. + self.execution_dependencies.reset(new c_vector[CObjectID]()) + return self + + @staticmethod + def from_string(const c_string& task_spec_str): + """Convert a string to a Ray task specification Python object. + + Args: + task_spec_str: String representation of the task specification. + + Returns: + Python task specification object. + """ + cdef Task self = Task.__new__(Task) + # TODO(pcm): Use flatbuffers validation here. + self.task_spec.reset(new CTaskSpecification(task_spec_str)) + # The created task does not include any execution dependencies. + self.execution_dependencies.reset(new c_vector[CObjectID]()) + return self + + def to_string(self): + """Convert a Ray task specification Python object to a string. + + Returns: + String representing the task specification. + """ + return self.task_spec.get().SerializeAsString() + + def _serialized_raylet_task(self): + return SerializeTaskAsString(self.execution_dependencies.get(), self.task_spec.get()) + + def driver_id(self): + """Return the driver ID for this task.""" + return DriverID.from_native(self.task_spec.get().DriverId()) + + def task_id(self): + """Return the task ID for this task.""" + return TaskID.from_native(self.task_spec.get().TaskId()) + + def parent_task_id(self): + """Return the task ID of the parent task.""" + return TaskID.from_native(self.task_spec.get().ParentTaskId()) + + def parent_counter(self): + """Return the parent counter of this task.""" + return self.task_spec.get().ParentCounter() + + def function_descriptor_list(self): + """Return the function descriptor for this task.""" + cdef c_vector[c_string] function_descriptor = self.task_spec.get().FunctionDescriptor() + results = [] + for i in range(function_descriptor.size()): + results.append(function_descriptor[i]) + return results + + def arguments(self): + """Return the arguments for the task.""" + cdef: + CTaskSpecification *task_spec = self.task_spec.get() + int64_t num_args = task_spec.NumArgs() + int count + arg_list = [] + for i in range(num_args): + count = task_spec.ArgIdCount(i) + if count > 0: + assert count == 1 + arg_list.append(ObjectID.from_native(task_spec.ArgId(i, 0))) + else: + serialized_str = task_spec.ArgVal(i)[:task_spec.ArgValLength(i)] + obj = pickle.loads(serialized_str) + arg_list.append(obj) + return arg_list + + def returns(self): + """Return the object IDs for the return values of the task.""" + cdef CTaskSpecification *task_spec = self.task_spec.get() + return_id_list = [] + for i in range(task_spec.NumReturns()): + return_id_list.append(ObjectID.from_native(task_spec.ReturnId(i))) + return return_id_list + + def required_resources(self): + """Return the resource dictionary of the task.""" + cdef: + unordered_map[c_string, double] resource_map = self.task_spec.get().GetRequiredResources().GetResourceMap() + c_string resource_name + double resource_value + unordered_map[c_string, double].iterator iterator = resource_map.begin() + + required_resources = {} + while iterator != resource_map.end(): + resource_name = dereference(iterator).first + py_resource_name = str(resource_name) # bytes for Py2, unicode for Py3 + resource_value = dereference(iterator).second + required_resources[py_resource_name] = resource_value + postincrement(iterator) + return required_resources + + def actor_creation_id(self): + """Return the actor creation ID for the task.""" + return ActorID.from_native(self.task_spec.get().ActorCreationId()) + + def actor_creation_dummy_object_id(self): + """Return the actor creation dummy object ID for the task.""" + return ObjectID.from_native(self.task_spec.get().ActorCreationDummyObjectId()) + + def actor_id(self): + """Return the actor ID for this task.""" + return ActorID.from_native(self.task_spec.get().ActorId()) + + def actor_counter(self): + """Return the actor counter for this task.""" + return self.task_spec.get().ActorCounter() diff --git a/python/ray/includes/unique_ids.pxd b/python/ray/includes/unique_ids.pxd new file mode 100644 index 000000000000..5e0df0bf21f7 --- /dev/null +++ b/python/ray/includes/unique_ids.pxd @@ -0,0 +1,34 @@ +from libcpp cimport bool as c_bool +from libcpp.string cimport string as c_string +from libc.stdint cimport uint8_t + +cdef extern from "ray/id.h" namespace "ray" nogil: + cdef cppclass CUniqueID "ray::UniqueID": + CUniqueID() + CUniqueID(const CUniqueID &from_id) + @staticmethod + CUniqueID from_random() + @staticmethod + CUniqueID from_binary(const c_string & binary) + @staticmethod + const CUniqueID nil() + size_t hash() const + c_bool is_nil() const + c_bool operator==(const CUniqueID& rhs) const + c_bool operator!=(const CUniqueID& rhs) const + const uint8_t *data() const + uint8_t *mutable_data(); + size_t size() const; + c_string binary() const; + c_string hex() const; + + ctypedef CUniqueID TaskID + ctypedef CUniqueID ObjectID + ctypedef CUniqueID FunctionID + ctypedef CUniqueID ActorID + ctypedef CUniqueID ActorClassID + ctypedef CUniqueID ActorHandleID + ctypedef CUniqueID WorkerID + ctypedef CUniqueID DriverID + ctypedef CUniqueID ConfigID + ctypedef CUniqueID ClientID diff --git a/python/ray/includes/unique_ids.pxi b/python/ray/includes/unique_ids.pxi new file mode 100644 index 000000000000..a29476742163 --- /dev/null +++ b/python/ray/includes/unique_ids.pxi @@ -0,0 +1,280 @@ +"""This is a module for unique IDs in Ray. +We define different types for different IDs for type safety. + +See https://github.com/ray-project/ray/issues/3721. +""" + +from ray.includes.common cimport ( + CUniqueID, CTaskID, CObjectID, CFunctionID, CActorClassID, CActorID, + CActorHandleID, CWorkerID, CDriverID, CConfigID, CClientID, + ComputePutId, ComputeTaskId) + +from ray.utils import decode + + +def check_id(b): + if not isinstance(b, bytes): + raise TypeError("Unsupported type: " + str(type(b))) + if len(b) != kUniqueIDSize: + raise ValueError("ID string needs to have length " + str(kUniqueIDSize)) + + +cdef extern from "ray/constants.h" nogil: + cdef int64_t kUniqueIDSize + cdef int64_t kMaxTaskPuts + + +cdef class UniqueID: + cdef CUniqueID data + + def __init__(self, id): + if not id: + self.data = CUniqueID() + else: + check_id(id) + self.data = CUniqueID.from_binary(id) + + @staticmethod + cdef from_native(const CUniqueID& cpp_id): + cdef UniqueID self = UniqueID.__new__(UniqueID) + self.data = cpp_id + return self + + @classmethod + def from_binary(cls, id_bytes): + if not isinstance(id_bytes, bytes): + raise TypeError("Expect bytes, got " + str(type(id_bytes))) + return cls(id_bytes) + + @staticmethod + def nil(): + return UniqueID.from_native(CUniqueID.nil()) + + def __hash__(self): + return self.data.hash() + + def is_nil(self): + return self.data.is_nil() + + def __eq__(self, other): + return self.binary() == other.binary() + + def __ne__(self, other): + return self.binary() != other.binary() + + def size(self): + return self.data.size() + + def __len__(self): + return self.size() + + def binary(self): + return self.data.binary() + + def __bytes__(self): + return self.binary() + + def hex(self): + return decode(self.data.hex()) + + def __hex__(self): + return self.hex() + + def __repr__(self): + return "UniqueID(" + self.hex() + ")" + + def __str__(self): + return self.__repr__() + + def __reduce__(self): + return type(self), (self.binary(),) + + def redis_shard_hash(self): + # NOTE: The hash function used here must match the one in GetRedisContext in + # src/ray/gcs/tables.h. Changes to the hash function should only be made + # through std::hash in src/common/common.h + return self.data.hash() + + +cdef class ObjectID(UniqueID): + + def __init__(self, id): + if not id: + self.data = CUniqueID() + else: + check_id(id) + self.data = CUniqueID.from_binary(id) + + @staticmethod + cdef from_native(const CObjectID& cpp_id): + cdef ObjectID self = ObjectID.__new__(ObjectID) + self.data = cpp_id + return self + + @staticmethod + def nil(): + return ObjectID.from_native(CObjectID.nil()) + + def __repr__(self): + return "ObjectID(" + self.hex() + ")" + + +cdef class TaskID(UniqueID): + + def __init__(self, id): + if not id: + self.data = CUniqueID() + else: + check_id(id) + self.data = CUniqueID.from_binary(id) + + @staticmethod + cdef from_native(const CTaskID& cpp_id): + cdef TaskID self = TaskID.__new__(TaskID) + self.data = cpp_id + return self + + @staticmethod + def nil(): + return TaskID.from_native(CTaskID.nil()) + + def __repr__(self): + return "TaskID(" + self.hex() + ")" + + +cdef class ClientID(UniqueID): + + def __init__(self, id): + if not id: + self.data = CUniqueID() + else: + check_id(id) + self.data = CUniqueID.from_binary(id) + + @staticmethod + cdef from_native(const CClientID& cpp_id): + cdef ClientID self = ClientID.__new__(ClientID) + self.data = cpp_id + return self + + @staticmethod + def nil(): + return ClientID.from_native(CClientID.nil()) + + def __repr__(self): + return "ClientID(" + self.hex() + ")" + + +cdef class DriverID(UniqueID): + + def __init__(self, id): + if not id: + self.data = CUniqueID() + else: + check_id(id) + self.data = CUniqueID.from_binary(id) + + @staticmethod + cdef from_native(const CDriverID& cpp_id): + cdef DriverID self = DriverID.__new__(DriverID) + self.data = cpp_id + return self + + @staticmethod + def nil(): + return DriverID.from_native(CDriverID.nil()) + + def __repr__(self): + return "DriverID(" + self.hex() + ")" + + +cdef class ActorID(UniqueID): + + def __init__(self, id): + if not id: + self.data = CUniqueID() + else: + check_id(id) + self.data = CUniqueID.from_binary(id) + + @staticmethod + cdef from_native(const CActorID& cpp_id): + cdef ActorID self = ActorID.__new__(ActorID) + self.data = cpp_id + return self + + @staticmethod + def nil(): + return ActorID.from_native(CActorID.nil()) + + def __repr__(self): + return "ActorID(" + self.hex() + ")" + + +cdef class ActorHandleID(UniqueID): + + def __init__(self, id): + if not id: + self.data = CUniqueID() + else: + check_id(id) + self.data = CUniqueID.from_binary(id) + + @staticmethod + cdef from_native(const CActorHandleID& cpp_id): + cdef ActorHandleID self = ActorHandleID.__new__(ActorHandleID) + self.data = cpp_id + return self + + @staticmethod + def nil(): + return ActorHandleID.from_native(CActorHandleID.nil()) + + def __repr__(self): + return "ActorHandleID(" + self.hex() + ")" + + +cdef class FunctionID(UniqueID): + + def __init__(self, id): + if not id: + self.data = CUniqueID() + else: + check_id(id) + self.data = CUniqueID.from_binary(id) + + @staticmethod + cdef from_native(const CFunctionID& cpp_id): + cdef FunctionID self = FunctionID.__new__(FunctionID) + self.data = cpp_id + return self + + @staticmethod + def nil(): + return FunctionID.from_native(CFunctionID.nil()) + + def __repr__(self): + return "FunctionID(" + self.hex() + ")" + + +cdef class ActorClassID(UniqueID): + + def __init__(self, id): + if not id: + self.data = CUniqueID() + else: + check_id(id) + self.data = CUniqueID.from_binary(id) + + @staticmethod + cdef from_native(const CActorClassID& cpp_id): + cdef ActorClassID self = ActorClassID.__new__(ActorClassID) + self.data = cpp_id + return self + + @staticmethod + def nil(): + return ActorClassID.from_native(CActorClassID.nil()) + + def __repr__(self): + return "ActorClassID(" + self.hex() + ")" diff --git a/python/ray/internal/internal_api.py b/python/ray/internal/internal_api.py index c4acbbbf60cf..65fce457b22f 100644 --- a/python/ray/internal/internal_api.py +++ b/python/ray/internal/internal_api.py @@ -2,7 +2,6 @@ from __future__ import division from __future__ import print_function -import ray.raylet import ray.worker from ray import profiling diff --git a/python/ray/monitor.py b/python/ray/monitor.py index b3d908f2b74a..782370298a44 100644 --- a/python/ray/monitor.py +++ b/python/ray/monitor.py @@ -160,9 +160,9 @@ def _xray_clean_up_entries_for_driver(self, driver_id): object_table_objects = self.state.object_table() driver_object_id_bins = set() for object_id, _ in object_table_objects.items(): - task_id_bin = ray.raylet.compute_task_id(object_id).id() + task_id_bin = ray._raylet.compute_task_id(object_id).binary() if task_id_bin in driver_task_id_bins: - driver_object_id_bins.add(object_id.id()) + driver_object_id_bins.add(object_id.binary()) def to_shard_index(id_bin): return binary_to_object_id(id_bin).redis_shard_hash() % len( diff --git a/python/ray/profiling.py b/python/ray/profiling.py index 692c5752cf05..ba0daed1914d 100644 --- a/python/ray/profiling.py +++ b/python/ray/profiling.py @@ -97,6 +97,10 @@ def _periodically_flush_profile_events(self): time.sleep(1) self.flush_profile_data() except AttributeError: + # TODO(suquark): It is a bad idea to ignore "AttributeError". + # It has caused some very unexpected behaviors when implementing + # new features (related to AttributeError). + # This is to suppress errors that occur at shutdown. pass @@ -120,7 +124,7 @@ def flush_profile_data(self): component_type = "driver" self.worker.raylet_client.push_profile_events( - component_type, ray.ObjectID(self.worker.worker_id), + component_type, ray.UniqueID(self.worker.worker_id), self.worker.node_ip_address, events) def add_event(self, event): diff --git a/python/ray/raylet/__init__.py b/python/ray/raylet/__init__.py deleted file mode 100644 index 67a1976b8d50..000000000000 --- a/python/ray/raylet/__init__.py +++ /dev/null @@ -1,13 +0,0 @@ -from __future__ import absolute_import -from __future__ import division -from __future__ import print_function - -from ray.core.src.ray.raylet.libraylet_library_python import ( - Task, RayletClient, ObjectID, check_simple_value, compute_task_id, - task_from_string, task_to_string, _config, RayCommonError) - -__all__ = [ - "Task", "RayletClient", "ObjectID", "check_simple_value", - "compute_task_id", "task_from_string", "task_to_string", - "start_local_scheduler", "_config", "RayCommonError" -] diff --git a/python/ray/rllib/utils/actors.py b/python/ray/rllib/utils/actors.py index d7affb5f7e78..44cbc98e2cdb 100644 --- a/python/ray/rllib/utils/actors.py +++ b/python/ray/rllib/utils/actors.py @@ -39,14 +39,14 @@ def completed_prefetch(self): Assumes obj_id only is one id.""" for worker, obj_id in self.completed(): - plasma_id = ray.pyarrow.plasma.ObjectID(obj_id.id()) + plasma_id = ray.pyarrow.plasma.ObjectID(obj_id.binary()) (ray.worker.global_worker.raylet_client.fetch_or_reconstruct( [obj_id], True)) self._fetching.append((worker, obj_id)) remaining = [] for worker, obj_id in self._fetching: - plasma_id = ray.pyarrow.plasma.ObjectID(obj_id.id()) + plasma_id = ray.pyarrow.plasma.ObjectID(obj_id.binary()) if ray.worker.global_worker.plasma_client.contains(plasma_id): yield (worker, obj_id) else: diff --git a/python/ray/tune/util.py b/python/ray/tune/util.py index 5d2db272679f..fcde28732003 100644 --- a/python/ray/tune/util.py +++ b/python/ray/tune/util.py @@ -23,13 +23,13 @@ def pin_in_object_store(obj): obj_id = ray.put(_to_pinnable(obj)) _pinned_objects.append(ray.get(obj_id)) return "{}{}".format(PINNED_OBJECT_PREFIX, - base64.b64encode(obj_id.id()).decode("utf-8")) + base64.b64encode(obj_id.binary()).decode("utf-8")) def get_pinned_object(pinned_id): """Retrieve a pinned object from the object store.""" - from ray.raylet import ObjectID + from ray import ObjectID return _from_pinnable( ray.get( diff --git a/python/ray/utils.py b/python/ray/utils.py index 3b660befd0d8..87e2ec121ee4 100644 --- a/python/ray/utils.py +++ b/python/ray/utils.py @@ -15,7 +15,6 @@ import uuid import ray.gcs_utils -import ray.raylet import ray.ray_constants as ray_constants @@ -67,7 +66,7 @@ def push_error_to_driver(worker, will be serialized with json and stored in Redis. """ if driver_id is None: - driver_id = ray.ObjectID.nil_id() + driver_id = ray.DriverID.nil() data = {} if data is None else data worker.raylet_client.push_error(driver_id, error_type, message, time.time()) @@ -96,7 +95,7 @@ def push_error_to_driver_through_redis(redis_client, will be serialized with json and stored in Redis. """ if driver_id is None: - driver_id = ray.ObjectID.nil_id() + driver_id = ray.DriverID.nil() data = {} if data is None else data # Do everything in Python and through the Python Redis client instead # of through the raylet. @@ -105,7 +104,7 @@ def push_error_to_driver_through_redis(redis_client, redis_client.execute_command("RAY.TABLE_APPEND", ray.gcs_utils.TablePrefix.ERROR_INFO, ray.gcs_utils.TablePubsub.ERROR_INFO, - driver_id.id(), error_data) + driver_id.binary(), error_data) def is_cython(obj): diff --git a/python/ray/worker.py b/python/ray/worker.py index ec5233618559..9aa0f8979e10 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -32,10 +32,9 @@ import ray.services as services import ray.signature import ray.tempfile_services as tempfile_services -import ray.raylet import ray.ray_constants as ray_constants from ray import import_thread -from ray import ObjectID +from ray import ObjectID, DriverID, ActorID, ActorHandleID, ClientID, TaskID from ray import profiling from ray.function_manager import (FunctionActorManager, FunctionDescriptor) import ray.parameter @@ -161,7 +160,8 @@ def __init__(self): self.serialization_context_map = {} self.function_actor_manager = FunctionActorManager(self) # Identity of the driver that this worker is processing. - self.task_driver_id = ObjectID.nil_id() + # It is a DriverID. + self.task_driver_id = DriverID.nil() self._task_context = threading.local() @property @@ -182,13 +182,13 @@ def task_context(self): # If this is running on the main thread, initialize it to # NIL. The actual value will set when the worker receives # a task from raylet backend. - self._task_context.current_task_id = ObjectID.nil_id() + self._task_context.current_task_id = TaskID.nil() else: # If this is running on a separate thread, then the mapping # to the current task ID may not be correct. Generate a # random task ID so that the backend can differentiate # between different threads. - self._task_context.current_task_id = ObjectID(random_string()) + self._task_context.current_task_id = TaskID(random_string()) if getattr(self, '_multithreading_warned', False) is not True: logger.warning( "Calling ray.get or ray.wait in a separate thread " @@ -286,7 +286,7 @@ def store_and_register(self, object_id, value, depth=100): try: self.plasma_client.put( value, - object_id=pyarrow.plasma.ObjectID(object_id.id()), + object_id=pyarrow.plasma.ObjectID(object_id.binary()), memcopy_threads=self.memcopy_threads, serialization_context=self.get_serialization_context( self.task_driver_id)) @@ -450,7 +450,7 @@ def get_object(self, object_ids): # smaller fetches so as to not block the manager for a prolonged period # of time in a single call. plain_object_ids = [ - plasma.ObjectID(object_id.id()) for object_id in object_ids + plasma.ObjectID(object_id.binary()) for object_id in object_ids ] for i in range(0, len(object_ids), ray._config.worker_fetch_request_size()): @@ -567,16 +567,16 @@ def submit_task(self, with profiling.profile("submit_task", worker=self): if actor_id is None: assert actor_handle_id is None - actor_id = ObjectID.nil_id() - actor_handle_id = ObjectID.nil_id() + actor_id = ActorID.nil() + actor_handle_id = ActorHandleID.nil() else: assert actor_handle_id is not None if actor_creation_id is None: - actor_creation_id = ObjectID.nil_id() + actor_creation_id = ActorID.nil() if actor_creation_dummy_object_id is None: - actor_creation_dummy_object_id = ObjectID.nil_id() + actor_creation_dummy_object_id = ObjectID.nil() # Put large or complex arguments that are passed by value in the # object store first. @@ -584,7 +584,7 @@ def submit_task(self, for arg in args: if isinstance(arg, ObjectID): args_for_local_scheduler.append(arg) - elif ray.raylet.check_simple_value(arg): + elif ray._raylet.check_simple_value(arg): args_for_local_scheduler.append(arg) else: args_for_local_scheduler.append(put(arg)) @@ -625,7 +625,8 @@ def submit_task(self, # Submit the task to local scheduler. function_descriptor_list = ( function_descriptor.get_function_descriptor_list()) - task = ray.raylet.Task( + assert isinstance(driver_id, DriverID) + task = ray._raylet.Task( driver_id, function_descriptor_list, args_for_local_scheduler, @@ -693,7 +694,7 @@ def run_function_on_all_workers(self, function, # Run the function on all workers. self.redis_client.hmset( key, { - "driver_id": self.task_driver_id.id(), + "driver_id": self.task_driver_id.binary(), "function_id": function_to_run_id, "function": pickled_function, "run_on_other_drivers": str(run_on_other_drivers) @@ -880,7 +881,7 @@ def _handle_process_task_failure(self, function_descriptor, str(failure_object), driver_id=self.task_driver_id, data={ - "function_id": function_id.id(), + "function_id": function_id.binary(), "function_name": function_name, "module_name": function_descriptor.module_name, "class_name": function_descriptor.class_name @@ -939,14 +940,14 @@ def _wait_for_and_process_task(self, task): with _changeproctitle(title, next_title): self._process_task(task, execution_info) # Reset the state fields so the next task can run. - self.task_context.current_task_id = ObjectID.nil_id() + self.task_context.current_task_id = TaskID.nil() self.task_context.task_index = 0 self.task_context.put_index = 1 if self.actor_id.is_nil(): # Don't need to reset task_driver_id if the worker is an # actor. Because the following tasks should all have the # same driver id. - self.task_driver_id = ObjectID.nil_id() + self.task_driver_id = DriverID.nil() # Increase the task execution counter. self.function_actor_manager.increase_task_counter( @@ -1100,17 +1101,16 @@ def error_applies_to_driver(error_key, worker=global_worker): + ray_constants.ID_SIZE), error_key # If the driver ID in the error message is a sequence of all zeros, then # the message is intended for all drivers. - driver_id = ObjectID(error_key[len(ERROR_KEY_PREFIX):( + driver_id = DriverID(error_key[len(ERROR_KEY_PREFIX):( len(ERROR_KEY_PREFIX) + ray_constants.ID_SIZE)]) - return (driver_id == worker.task_driver_id - or driver_id == ObjectID.nil_id()) + return (driver_id == worker.task_driver_id or driver_id == DriverID.nil()) def error_info(worker=global_worker): """Return information about failed tasks.""" worker.check_connected() return (global_state.error_messages(job_id=worker.task_driver_id) + - global_state.error_messages(job_id=ObjectID.nil_id())) + global_state.error_messages(job_id=DriverID.nil())) def _initialize_serialization(driver_id, worker=global_worker): @@ -1127,7 +1127,7 @@ def _initialize_serialization(driver_id, worker=global_worker): # Define a custom serializer and deserializer for handling Object IDs. def object_id_custom_serializer(obj): - return obj.id() + return obj.binary() def object_id_custom_deserializer(serialized_obj): return ObjectID(serialized_obj) @@ -1656,8 +1656,8 @@ def listen_error_messages_raylet(worker, task_error_queue): gcs_entry.Entries(0), 0) job_id = error_data.JobId() if job_id not in [ - worker.task_driver_id.id(), - ObjectID.nil_id().id() + worker.task_driver_id.binary(), + DriverID.nil().binary() ]: continue @@ -1768,23 +1768,23 @@ def connect(info, else: # This is the code path of driver mode. if driver_id is None: - driver_id = ObjectID(random_string()) + driver_id = DriverID(random_string()) - if not isinstance(driver_id, ObjectID): - raise Exception("The type of given driver id must be ObjectID.") + if not isinstance(driver_id, DriverID): + raise Exception("The type of given driver id must be DriverID.") - worker.worker_id = driver_id.id() + worker.worker_id = driver_id.binary() # When tasks are executed on remote workers in the context of multiple # drivers, the task driver ID is used to keep track of which driver is # responsible for the task so that error messages will be propagated to # the correct driver. if mode != WORKER_MODE: - worker.task_driver_id = ObjectID(worker.worker_id) + worker.task_driver_id = DriverID(worker.worker_id) # All workers start out as non-actors. A worker can be turned into an actor # after it is created. - worker.actor_id = ObjectID.nil_id() + worker.actor_id = ActorID.nil() worker.connected = True worker.set_mode(mode) @@ -1910,18 +1910,18 @@ def connect(info, nil_actor_counter = 0 function_descriptor = FunctionDescriptor.for_driver_task() - driver_task = ray.raylet.Task( + driver_task = ray._raylet.Task( worker.task_driver_id, function_descriptor.get_function_descriptor_list(), [], # arguments. 0, # num_returns. - ObjectID(random_string()), # parent_task_id. + TaskID(random_string()), # parent_task_id. 0, # parent_counter. - ObjectID.nil_id(), # actor_creation_id. - ObjectID.nil_id(), # actor_creation_dummy_object_id. + ActorID.nil(), # actor_creation_id. + ObjectID.nil(), # actor_creation_dummy_object_id. 0, # max_actor_reconstructions. - ObjectID.nil_id(), # actor_id. - ObjectID.nil_id(), # actor_handle_id. + ActorID.nil(), # actor_id. + ActorHandleID.nil(), # actor_handle_id. nil_actor_counter, # actor_counter. [], # new_actor_handles. [], # execution_dependencies. @@ -1933,18 +1933,18 @@ def connect(info, global_state._execute_command(driver_task.task_id(), "RAY.TABLE_ADD", ray.gcs_utils.TablePrefix.RAYLET_TASK, ray.gcs_utils.TablePubsub.RAYLET_TASK, - driver_task.task_id().id(), + driver_task.task_id().binary(), driver_task._serialized_raylet_task()) # Set the driver's current task ID to the task ID assigned to the # driver task. worker.task_context.current_task_id = driver_task.task_id() - worker.raylet_client = ray.raylet.RayletClient( + worker.raylet_client = ray._raylet.RayletClient( raylet_socket, - worker.worker_id, + ClientID(worker.worker_id), is_worker, - worker.current_task_id, + DriverID(worker.current_task_id.binary()), ) # Start the import thread @@ -2144,6 +2144,7 @@ def register_custom_serializer(cls, if driver_id is None: driver_id = worker.task_driver_id + assert isinstance(driver_id, DriverID) def register_class_for_serialization(worker_info): # TODO(rkn): We need to be more thoughtful about what to do if custom @@ -2228,7 +2229,7 @@ def put(value, worker=global_worker): if worker.mode == LOCAL_MODE: # In LOCAL_MODE, ray.put is the identity operation. return value - object_id = worker.raylet_client.compute_put_id( + object_id = ray._raylet.compute_put_id( worker.current_task_id, worker.task_context.put_index, ) diff --git a/python/setup.py b/python/setup.py index 66daea19bc19..56bd9d0caf7d 100644 --- a/python/setup.py +++ b/python/setup.py @@ -22,8 +22,7 @@ ray_files = [ "ray/core/src/ray/thirdparty/redis/src/redis-server", "ray/core/src/ray/gcs/redis_module/libray_redis_module.so", - "ray/core/src/plasma/plasma_store_server", - "ray/core/src/ray/raylet/libraylet_library_python.so", + "ray/core/src/plasma/plasma_store_server", "ray/_raylet.so", "ray/core/src/ray/raylet/raylet_monitor", "ray/core/src/ray/raylet/raylet", "ray/WebUI.ipynb" ] diff --git a/src/ray/gcs/tables.h b/src/ray/gcs/tables.h index e879d3dd0bc4..1eab649601ad 100644 --- a/src/ray/gcs/tables.h +++ b/src/ray/gcs/tables.h @@ -391,7 +391,7 @@ class FunctionTable : public Table { }; }; -using ClassTable = Table; +using ClassTable = Table; /// Actor table starts with an ALIVE entry, which represents the first time the actor /// is created. This may be followed by 0 or more pairs of RECONSTRUCTING, ALIVE entries, @@ -513,7 +513,7 @@ class ProfileTable : private Log { std::string DebugString() const; }; -using CustomSerializerTable = Table; +using CustomSerializerTable = Table; using ConfigTable = Table; diff --git a/src/ray/id.h b/src/ray/id.h index 0ab0c56408e7..ed60f5b823f2 100644 --- a/src/ray/id.h +++ b/src/ray/id.h @@ -42,7 +42,7 @@ typedef UniqueID TaskID; typedef UniqueID JobID; typedef UniqueID ObjectID; typedef UniqueID FunctionID; -typedef UniqueID ClassID; +typedef UniqueID ActorClassID; typedef UniqueID ActorID; typedef UniqueID ActorHandleID; typedef UniqueID WorkerID; diff --git a/src/ray/ray_config.h b/src/ray/ray_config.h index a5c9ec1217fb..4a029810f172 100644 --- a/src/ray/ray_config.h +++ b/src/ray/ray_config.h @@ -20,6 +20,7 @@ class RayConfig { \ public: \ inline type name() { return name##_; } + #include "ray_config_def.h" /// ------------------------------------------------------------------------- #undef RAY_CONFIG diff --git a/src/ray/ray_config_def.h b/src/ray/ray_config_def.h index 74a912d29f88..894109f22af6 100644 --- a/src/ray/ray_config_def.h +++ b/src/ray/ray_config_def.h @@ -4,6 +4,11 @@ // Macro definition format: RAY_CONFIG(type, name, default_value). // NOTE: This file should NOT be included in any file other than ray_config.h. +// IF YOU MODIFY THIS FILE and add a configuration parameter, you must change +// at least two additional things: +// 1. You must update the file "ray/python/ray/includes/ray_config.pxd". +// 2. You must update the file "ray/python/ray/includes/ray_config.pxi". + /// In theory, this is used to detect Ray version mismatches. RAY_CONFIG(int64_t, ray_protocol_version, 0x0000000000000000); diff --git a/src/ray/raylet/CMakeLists.txt b/src/ray/raylet/CMakeLists.txt index 5834b0300621..c81a9ed3c8cb 100644 --- a/src/ray/raylet/CMakeLists.txt +++ b/src/ray/raylet/CMakeLists.txt @@ -86,12 +86,6 @@ macro(set_raylet_library LANG) install(TARGETS ${RAYLET_LIBRARY_LANG} DESTINATION ${CMAKE_SOURCE_DIR}/raylet) endmacro() -if ("${CMAKE_RAY_LANG_PYTHON}" STREQUAL "YES") - set_raylet_library("python") - include_directories("${PYTHON_INCLUDE_DIRS}") - include_directories("${NUMPY_INCLUDE_DIR}") -endif() - if ("${CMAKE_RAY_LANG_JAVA}" STREQUAL "YES") add_compile_options("-I$ENV{JAVA_HOME}/include/") if(WIN32) diff --git a/src/ray/raylet/lib/python/common_extension.cc b/src/ray/raylet/lib/python/common_extension.cc deleted file mode 100644 index ccfcf5cb3b12..000000000000 --- a/src/ray/raylet/lib/python/common_extension.cc +++ /dev/null @@ -1,843 +0,0 @@ -#include -#include "bytesobject.h" -#include "node.h" - -// Don't use the deprecated Numpy functions. -#define NPY_NO_DEPRECATED_API NPY_1_7_API_VERSION - -#include - -#include "common_extension.h" -#include "ray/common/common_protocol.h" -#include "ray/id.h" -#include "ray/ray_config.h" -#include "ray/raylet/task.h" -#include "ray/raylet/task_execution_spec.h" -#include "ray/raylet/task_spec.h" -#include "ray/util/logging.h" - -#include - -#if PY_MAJOR_VERSION >= 3 -#define PyInt_Check PyLong_Check -#endif - -// TODO(rkn): Remove this later. -using ray::ObjectID; -using ray::ActorID; -using ray::ActorHandleID; -using ray::UniqueID; -using ray::FunctionID; -using ray::TaskID; - -/* Initialize pickle module. */ - -PyObject *pickle_module = NULL; -PyObject *pickle_loads = NULL; -PyObject *pickle_dumps = NULL; -PyObject *pickle_protocol = NULL; - -int init_numpy_module(void) { - import_array1(-1); - return 0; -} - -void init_pickle_module(void) { -#if PY_MAJOR_VERSION >= 3 - pickle_module = PyImport_ImportModule("pickle"); -#else - pickle_module = PyImport_ImportModuleNoBlock("cPickle"); -#endif - RAY_CHECK(pickle_module != NULL); - RAY_CHECK(PyObject_HasAttrString(pickle_module, "loads")); - RAY_CHECK(PyObject_HasAttrString(pickle_module, "dumps")); - RAY_CHECK(PyObject_HasAttrString(pickle_module, "HIGHEST_PROTOCOL")); - pickle_loads = PyUnicode_FromString("loads"); - pickle_dumps = PyUnicode_FromString("dumps"); - pickle_protocol = PyObject_GetAttrString(pickle_module, "HIGHEST_PROTOCOL"); - RAY_CHECK(pickle_protocol != NULL); -} - -TaskBuilder *g_task_builder = NULL; - -/* Define the PyObjectID class. */ - -int PyStringToUniqueID(PyObject *object, ObjectID *object_id) { - if (PyBytes_Check(object)) { - std::memcpy(object_id->mutable_data(), PyBytes_AsString(object), sizeof(*object_id)); - return 1; - } else { - PyErr_SetString(PyExc_TypeError, "must be a 20 character string"); - return 0; - } -} - -int PyObjectToUniqueID(PyObject *object, ObjectID *objectid) { - if (PyObject_IsInstance(object, (PyObject *)&PyObjectIDType)) { - *objectid = ((PyObjectID *)object)->object_id; - return 1; - } else { - PyErr_SetString(PyExc_TypeError, "must be an ObjectID"); - return 0; - } -} - -int PyListStringToStringVector(PyObject *object, - std::vector *function_descriptor) { - if (function_descriptor == nullptr) { - PyErr_SetString(PyExc_TypeError, "function descriptor must be non-empty pointer"); - return 0; - } - function_descriptor->clear(); - std::vector string_vector; - if (PyList_Check(object)) { - Py_ssize_t size = PyList_Size(object); - for (Py_ssize_t i = 0; i < size; ++i) { - PyObject *item = PyList_GetItem(object, i); - if (PyBytes_Check(item) == 0) { - PyErr_SetString(PyExc_TypeError, - "PyListStringToStringVector takes a list of byte strings."); - return 0; - } - function_descriptor->emplace_back(PyBytes_AsString(item), PyBytes_Size(item)); - } - return 1; - } else { - PyErr_SetString(PyExc_TypeError, "must be a list of strings"); - return 0; - } -} - -static int PyObjectID_init(PyObjectID *self, PyObject *args, PyObject *kwds) { - const char *data; - int size; - if (!PyArg_ParseTuple(args, "s#", &data, &size)) { - return -1; - } - if (size != sizeof(ObjectID)) { - PyErr_SetString(PyExc_ValueError, - "ObjectID: object id string needs to have length 20"); - return -1; - } - std::memcpy(self->object_id.mutable_data(), data, sizeof(self->object_id)); - return 0; -} - -static PyObject *PyObjectID_nil_id(PyObject *cls) { - return PyObjectID_make(ray::UniqueID()); -} - -/* Create a PyObjectID from C. */ -PyObject *PyObjectID_make(ObjectID object_id) { - PyObjectID *result = PyObject_New(PyObjectID, &PyObjectIDType); - result = (PyObjectID *)PyObject_Init((PyObject *)result, &PyObjectIDType); - result->object_id = object_id; - return (PyObject *)result; -} - -TaskSpec *TaskSpec_copy(TaskSpec *spec, int64_t task_spec_size) { - TaskSpec *copy = (TaskSpec *)malloc(task_spec_size); - memcpy(copy, spec, task_spec_size); - return copy; -} - -/** - * Convert a string to a Ray task specification Python object. - * - * This is called from Python like - * - * task = raylet.task_from_string("...") - * - * @param task_string String representation of the task specification. - * @return Python task specification object. - */ -PyObject *PyTask_from_string(PyObject *self, PyObject *args) { - const char *data; - int size; - if (!PyArg_ParseTuple(args, "s#", &data, &size)) { - return NULL; - } - PyTask *result = PyObject_New(PyTask, &PyTaskType); - result = (PyTask *)PyObject_Init((PyObject *)result, &PyTaskType); - result->task_spec = new ray::raylet::TaskSpecification(std::string(data, size)); - /* The created task does not include any execution dependencies. */ - result->execution_dependencies = new std::vector(); - /* TODO(pcm): Use flatbuffers validation here. */ - return (PyObject *)result; -} - -/** - * Convert a Ray task specification Python object to a string. - * - * This is called from Python like - * - * s = raylet.task_to_string(task) - * - * @param task Ray task specification Python object. - * @return String representing the task specification. - */ -PyObject *PyTask_to_string(PyObject *self, PyObject *args) { - PyObject *arg; - if (!PyArg_ParseTuple(args, "O", &arg)) { - return NULL; - } - PyTask *task = (PyTask *)arg; - flatbuffers::FlatBufferBuilder fbb; - auto task_spec_string = task->task_spec->ToFlatbuffer(fbb); - fbb.Finish(task_spec_string); - return PyBytes_FromStringAndSize((char *)fbb.GetBufferPointer(), fbb.GetSize()); -} - -static PyObject *PyObjectID_id(PyObject *self) { - PyObjectID *s = (PyObjectID *)self; - return PyBytes_FromStringAndSize((const char *)s->object_id.data(), - sizeof(s->object_id)); -} - -static PyObject *PyObjectID_hex(PyObject *self) { - PyObjectID *s = (PyObjectID *)self; - std::string hex_id = s->object_id.hex(); -#if PY_MAJOR_VERSION >= 3 - PyObject *result = PyUnicode_FromStringAndSize(hex_id.data(), hex_id.size()); -#else - PyObject *result = PyBytes_FromStringAndSize(hex_id.data(), hex_id.size()); -#endif - return result; -} - -static PyObject *PyObjectID_is_nil(PyObject *self) { - ObjectID object_id; - PyObjectToUniqueID(self, &object_id); - if (object_id.is_nil()) { - Py_RETURN_TRUE; - } else { - Py_RETURN_FALSE; - } -} - -static PyObject *PyObjectID_richcompare(PyObjectID *self, PyObject *other, int op) { - PyObject *result = NULL; - if (Py_TYPE(self)->tp_richcompare != Py_TYPE(other)->tp_richcompare) { - result = Py_NotImplemented; - } else { - PyObjectID *other_id = (PyObjectID *)other; - switch (op) { - case Py_LT: - result = Py_NotImplemented; - break; - case Py_LE: - result = Py_NotImplemented; - break; - case Py_EQ: - result = self->object_id == other_id->object_id ? Py_True : Py_False; - break; - case Py_NE: - result = !(self->object_id == other_id->object_id) ? Py_True : Py_False; - break; - case Py_GT: - result = Py_NotImplemented; - break; - case Py_GE: - result = Py_NotImplemented; - break; - } - } - Py_XINCREF(result); - return result; -} - -static PyObject *PyObjectID_redis_shard_hash(PyObjectID *self) { - // NOTE: The hash function used here must match the one in GetRedisContext in - // src/ray/gcs/tables.h. Changes to the hash function should only be made - // through std::hash in src/common/common.h - std::hash hash; - return PyLong_FromSize_t(hash(self->object_id)); -} - -static long PyObjectID_hash(PyObjectID *self) { - // TODO(pcm): Replace this with a faster hash function. This currently - // creates a tuple of length 20 and hashes it, which is slow - PyObject *tuple = PyTuple_New(kUniqueIDSize); - for (int i = 0; i < kUniqueIDSize; ++i) { - PyTuple_SetItem(tuple, i, PyLong_FromLong(self->object_id.data()[i])); - } - long hash = PyObject_Hash(tuple); - Py_XDECREF(tuple); - return hash; -} - -static PyObject *PyObjectID_repr(PyObjectID *self) { - std::string repr = "ObjectID(" + self->object_id.hex() + ")"; - PyObject *result = PyUnicode_FromString(repr.c_str()); - return result; -} - -static PyObject *PyObjectID_getstate(PyObjectID *self) { - PyObject *field; - field = PyBytes_FromStringAndSize((char *)self->object_id.data(), sizeof(ObjectID)); - return Py_BuildValue("(N)", field); -} - -static PyObject *PyObjectID___reduce__(PyObjectID *self, PyObject *arg) { - return Py_BuildValue("(ON)", Py_TYPE(self), PyObjectID_getstate(self)); -} - -static PyMethodDef PyObjectID_methods[] = { - {"id", (PyCFunction)PyObjectID_id, METH_NOARGS, - "Return the hash associated with this ObjectID"}, - {"redis_shard_hash", (PyCFunction)PyObjectID_redis_shard_hash, METH_NOARGS, - "Return the redis shard that this ObjectID is associated with"}, - {"hex", (PyCFunction)PyObjectID_hex, METH_NOARGS, - "Return the object ID as a string in hex."}, - {"is_nil", (PyCFunction)PyObjectID_is_nil, METH_NOARGS, - "Return whether the ObjectID is nil"}, - {"__reduce__", (PyCFunction)PyObjectID___reduce__, METH_VARARGS, - "Provide a way to pickle this ObjectID."}, - {"nil_id", (PyCFunction)PyObjectID_nil_id, METH_NOARGS | METH_CLASS, - "Create an instance of ray.ObjectID from random string"}, - {NULL} /* Sentinel */ -}; - -static PyMemberDef PyObjectID_members[] = { - {NULL} /* Sentinel */ -}; - -// This python class is introduced by python/ray/raylet/__init__.py. -// Therefore, tp_name should match the path. ray.ObjectID is also OK. -PyTypeObject PyObjectIDType = { - PyVarObject_HEAD_INIT(NULL, 0) /* ob_size */ - "ray.raylet.ObjectID", /* tp_name */ - sizeof(PyObjectID), /* tp_basicsize */ - 0, /* tp_itemsize */ - 0, /* tp_dealloc */ - 0, /* tp_print */ - 0, /* tp_getattr */ - 0, /* tp_setattr */ - 0, /* tp_compare */ - (reprfunc)PyObjectID_repr, /* tp_repr */ - 0, /* tp_as_number */ - 0, /* tp_as_sequence */ - 0, /* tp_as_mapping */ - (hashfunc)PyObjectID_hash, /* tp_hash */ - 0, /* tp_call */ - 0, /* tp_str */ - 0, /* tp_getattro */ - 0, /* tp_setattro */ - 0, /* tp_as_buffer */ - Py_TPFLAGS_DEFAULT, /* tp_flags */ - "ObjectID object", /* tp_doc */ - 0, /* tp_traverse */ - 0, /* tp_clear */ - (richcmpfunc)PyObjectID_richcompare, /* tp_richcompare */ - 0, /* tp_weaklistoffset */ - 0, /* tp_iter */ - 0, /* tp_iternext */ - PyObjectID_methods, /* tp_methods */ - PyObjectID_members, /* tp_members */ - 0, /* tp_getset */ - 0, /* tp_base */ - 0, /* tp_dict */ - 0, /* tp_descr_get */ - 0, /* tp_descr_set */ - 0, /* tp_dictoffset */ - (initproc)PyObjectID_init, /* tp_init */ - 0, /* tp_alloc */ - PyType_GenericNew, /* tp_new */ -}; - -// Define the PyTask class. - -int resource_map_from_python_dict(PyObject *resource_map, - std::unordered_map &out) { - RAY_CHECK(out.size() == 0); - - PyObject *key, *value; - Py_ssize_t position = 0; - if (!PyDict_Check(resource_map)) { - PyErr_SetString(PyExc_TypeError, "resource_map must be a dictionary"); - return -1; - } - - while (PyDict_Next(resource_map, &position, &key, &value)) { -#if PY_MAJOR_VERSION >= 3 - if (!PyUnicode_Check(key)) { - PyErr_SetString(PyExc_TypeError, "the keys in resource_map must be strings"); - return -1; - } -#else - if (!PyBytes_Check(key)) { - PyErr_SetString(PyExc_TypeError, "the keys in resource_map must be strings"); - return -1; - } -#endif - - // Check that the resource quantities are numbers. - if (!(PyFloat_Check(value) || PyInt_Check(value) || PyLong_Check(value))) { - PyErr_SetString(PyExc_TypeError, "the values in resource_map must be floats"); - return -1; - } - // Handle the case where the key is a bytes object and the case where it - // is a unicode object. - std::string resource_name; - if (PyUnicode_Check(key)) { - PyObject *ascii_key = PyUnicode_AsASCIIString(key); - resource_name = std::string(PyBytes_AsString(ascii_key), PyBytes_Size(ascii_key)); - Py_DECREF(ascii_key); - } else { - resource_name = std::string(PyBytes_AsString(key), PyBytes_Size(key)); - } - out[resource_name] = PyFloat_AsDouble(value); - } - return 0; -} - -static int PyTask_init(PyTask *self, PyObject *args, PyObject *kwds) { - // ID of the driver that this task originates from. - UniqueID driver_id; - // ID of the actor this task should run on. - UniqueID actor_id; - // ID of the actor handle used to submit this task. - UniqueID actor_handle_id; - // How many tasks have been launched on the actor so far? - int actor_counter = 0; - // Arguments of the task (can be PyObjectIDs or Python values). - PyObject *arguments; - // Number of return values of this task. - int num_returns; - // Task language type enum number. - int language = static_cast(Language::PYTHON); - // The ID of the task that called this task. - TaskID parent_task_id; - // The number of tasks that the parent task has called prior to this one. - int parent_counter; - // The actor creation ID. - ActorID actor_creation_id; - // The dummy object for the actor creation task (if this is an actor method). - ObjectID actor_creation_dummy_object_id; - // Max number of times to reconstruct this actor (only used for actor creation - // task). - int32_t max_actor_reconstructions; - PyObject *new_actor_handles; - // Arguments of the task that are execution-dependent. These must be - // PyObjectIDs). - PyObject *execution_arguments = nullptr; - // Dictionary of resource requirements for this task. - PyObject *resource_map = nullptr; - // Dictionary of required placement resources for this task. - PyObject *placement_resource_map = nullptr; - // Function descriptor. - std::vector function_descriptor; - if (!PyArg_ParseTuple( - args, "O&O&OiO&i|O&O&iO&O&iOOOOi", &PyObjectToUniqueID, &driver_id, - &PyListStringToStringVector, &function_descriptor, &arguments, &num_returns, - &PyObjectToUniqueID, &parent_task_id, &parent_counter, &PyObjectToUniqueID, - &actor_creation_id, &PyObjectToUniqueID, &actor_creation_dummy_object_id, - &max_actor_reconstructions, &PyObjectToUniqueID, &actor_id, &PyObjectToUniqueID, - &actor_handle_id, &actor_counter, &new_actor_handles, &execution_arguments, - &resource_map, &placement_resource_map, &language)) { - return -1; - } - - // Parse the resource map. - std::unordered_map required_resources; - std::unordered_map required_placement_resources; - - if (resource_map != nullptr) { - if (resource_map_from_python_dict(resource_map, required_resources) != 0) { - return -1; - } - } - - if (required_resources.count("CPU") == 0) { - required_resources["CPU"] = 1.0; - } - - if (placement_resource_map != nullptr) { - if (resource_map_from_python_dict(placement_resource_map, - required_placement_resources) != 0) { - return -1; - } - } - - Py_ssize_t num_args = PyList_Size(arguments); - - // Create the task spec. - - // Parse the arguments from the list. - std::vector> task_args; - for (Py_ssize_t i = 0; i < num_args; ++i) { - PyObject *arg = PyList_GetItem(arguments, i); - if (PyObject_IsInstance(arg, reinterpret_cast(&PyObjectIDType))) { - std::vector references = {reinterpret_cast(arg)->object_id}; - task_args.push_back( - std::make_shared(references)); - } else { - PyObject *data = PyObject_CallMethodObjArgs(pickle_module, pickle_dumps, arg, - pickle_protocol, NULL); - task_args.push_back(std::make_shared( - reinterpret_cast(PyBytes_AsString(data)), PyBytes_Size(data))); - Py_DECREF(data); - } - } - - std::vector task_new_actor_handles; - Py_ssize_t num_new_actor_handles = PyList_Size(new_actor_handles); - for (Py_ssize_t i = 0; i < num_new_actor_handles; ++i) { - PyObject *new_actor_handle = PyList_GetItem(new_actor_handles, i); - if (!PyObject_IsInstance(new_actor_handle, (PyObject *)&PyObjectIDType)) { - PyErr_SetString(PyExc_TypeError, "New actor handles must be a ray.ObjectID."); - return -1; - } - task_new_actor_handles.push_back(((PyObjectID *)new_actor_handle)->object_id); - } - - self->task_spec = new ray::raylet::TaskSpecification( - driver_id, parent_task_id, parent_counter, actor_creation_id, - actor_creation_dummy_object_id, max_actor_reconstructions, actor_id, - actor_handle_id, actor_counter, task_new_actor_handles, task_args, num_returns, - required_resources, required_placement_resources, Language::PYTHON, - function_descriptor); - - /* Set the task's execution dependencies. */ - self->execution_dependencies = new std::vector(); - if (execution_arguments != NULL) { - Py_ssize_t num_execution_args = PyList_Size(execution_arguments); - for (Py_ssize_t i = 0; i < num_execution_args; ++i) { - PyObject *execution_arg = PyList_GetItem(execution_arguments, i); - if (!PyObject_IsInstance(execution_arg, (PyObject *)&PyObjectIDType)) { - PyErr_SetString(PyExc_TypeError, "Execution arguments must be an ObjectID."); - return -1; - } - self->execution_dependencies->push_back(((PyObjectID *)execution_arg)->object_id); - } - } - - return 0; -} - -static void PyTask_dealloc(PyTask *self) { - delete self->task_spec; - delete self->execution_dependencies; - Py_TYPE(self)->tp_free(reinterpret_cast(self)); -} - -// Helper function to change a c++ string vector to a Python string list. -static PyObject *VectorStringToPyBytesList( - const std::vector &function_descriptor) { - size_t size = function_descriptor.size(); - PyObject *return_list = PyList_New(static_cast(size)); - for (size_t i = 0; i < size; ++i) { - auto py_bytes = PyBytes_FromStringAndSize(function_descriptor[i].data(), - function_descriptor[i].size()); - PyList_SetItem(return_list, i, py_bytes); - } - return return_list; -} - -static PyObject *PyTask_function_descriptor_vector(PyTask *self) { - std::vector function_descriptor; - function_descriptor = self->task_spec->FunctionDescriptor(); - return VectorStringToPyBytesList(function_descriptor); -} - -static PyObject *PyTask_actor_id(PyTask *self) { - ActorID actor_id = self->task_spec->ActorId(); - return PyObjectID_make(actor_id); -} - -static PyObject *PyTask_actor_counter(PyTask *self) { - int64_t actor_counter = self->task_spec->ActorCounter(); - return PyLong_FromLongLong(actor_counter); -} - -static PyObject *PyTask_driver_id(PyTask *self) { - UniqueID driver_id = self->task_spec->DriverId(); - return PyObjectID_make(driver_id); -} - -static PyObject *PyTask_task_id(PyTask *self) { - TaskID task_id = self->task_spec->TaskId(); - return PyObjectID_make(task_id); -} - -static PyObject *PyTask_parent_task_id(PyTask *self) { - TaskID task_id = self->task_spec->ParentTaskId(); - return PyObjectID_make(task_id); -} - -static PyObject *PyTask_parent_counter(PyTask *self) { - int64_t parent_counter = self->task_spec->ParentCounter(); - return PyLong_FromLongLong(parent_counter); -} - -static PyObject *PyTask_arguments(PyTask *self) { - ray::raylet::TaskSpecification *task_spec = self->task_spec; - - int64_t num_args = self->task_spec->NumArgs(); - - PyObject *arg_list = PyList_New((Py_ssize_t)num_args); - for (int i = 0; i < num_args; ++i) { - int count = task_spec->ArgIdCount(i); - - if (count > 0) { - assert(count == 1); - - const ObjectID object_id = task_spec->ArgId(i, 0); - - PyList_SetItem(arg_list, i, PyObjectID_make(object_id)); - } else { - RAY_CHECK(pickle_module != NULL); - RAY_CHECK(pickle_loads != NULL); - - const uint8_t *arg_val = task_spec->ArgVal(i); - int64_t arg_length = task_spec->ArgValLength(i); - - PyObject *str = PyBytes_FromStringAndSize(reinterpret_cast(arg_val), - static_cast(arg_length)); - PyObject *val = PyObject_CallMethodObjArgs(pickle_module, pickle_loads, str, NULL); - Py_XDECREF(str); - PyList_SetItem(arg_list, i, val); - } - } - return arg_list; -} - -static PyObject *PyTask_actor_creation_id(PyTask *self) { - ActorID actor_creation_id = self->task_spec->ActorCreationId(); - return PyObjectID_make(actor_creation_id); -} - -static PyObject *PyTask_actor_creation_dummy_object_id(PyTask *self) { - ObjectID actor_creation_dummy_object_id = self->task_spec->ActorCreationDummyObjectId(); - return PyObjectID_make(actor_creation_dummy_object_id); -} - -static PyObject *PyTask_required_resources(PyTask *self) { - PyObject *required_resources = PyDict_New(); - - std::unordered_map resource_map = - self->task_spec->GetRequiredResources().GetResourceMap(); - - for (auto const &resource_pair : resource_map) { - std::string resource_name = resource_pair.first; -#if PY_MAJOR_VERSION >= 3 - PyObject *key = - PyUnicode_FromStringAndSize(resource_name.data(), resource_name.size()); -#else - PyObject *key = PyBytes_FromStringAndSize(resource_name.data(), resource_name.size()); -#endif - PyObject *value = PyFloat_FromDouble(resource_pair.second); - PyDict_SetItem(required_resources, key, value); - Py_DECREF(key); - Py_DECREF(value); - } - return required_resources; -} - -static PyObject *PyTask_returns(PyTask *self) { - ray::raylet::TaskSpecification *task_spec = self->task_spec; - - int64_t num_returns = task_spec->NumReturns(); - - PyObject *return_id_list = PyList_New((Py_ssize_t)num_returns); - for (int i = 0; i < num_returns; ++i) { - ObjectID object_id = task_spec->ReturnId(i); - PyList_SetItem(return_id_list, i, PyObjectID_make(object_id)); - } - return return_id_list; -} - -static PyObject *PyTask_to_serialized_flatbuf(PyTask *self) { - const std::vector execution_dependencies(*self->execution_dependencies); - auto const execution_spec = - ray::raylet::TaskExecutionSpecification(std::move(execution_dependencies)); - auto const task = ray::raylet::Task(execution_spec, *self->task_spec); - - flatbuffers::FlatBufferBuilder fbb; - auto task_flatbuffer = task.ToFlatbuffer(fbb); - fbb.Finish(task_flatbuffer); - - return PyBytes_FromStringAndSize(reinterpret_cast(fbb.GetBufferPointer()), - fbb.GetSize()); -} - -static PyMethodDef PyTask_methods[] = { - {"function_descriptor_list", (PyCFunction)PyTask_function_descriptor_vector, - METH_NOARGS, "Return the function descriptor for this task."}, - {"parent_task_id", (PyCFunction)PyTask_parent_task_id, METH_NOARGS, - "Return the task ID of the parent task."}, - {"parent_counter", (PyCFunction)PyTask_parent_counter, METH_NOARGS, - "Return the parent counter of this task."}, - {"actor_id", (PyCFunction)PyTask_actor_id, METH_NOARGS, - "Return the actor ID for this task."}, - {"actor_counter", (PyCFunction)PyTask_actor_counter, METH_NOARGS, - "Return the actor counter for this task."}, - {"driver_id", (PyCFunction)PyTask_driver_id, METH_NOARGS, - "Return the driver ID for this task."}, - {"task_id", (PyCFunction)PyTask_task_id, METH_NOARGS, - "Return the task ID for this task."}, - {"arguments", (PyCFunction)PyTask_arguments, METH_NOARGS, - "Return the arguments for the task."}, - {"actor_creation_id", (PyCFunction)PyTask_actor_creation_id, METH_NOARGS, - "Return the actor creation ID for the task."}, - {"actor_creation_dummy_object_id", (PyCFunction)PyTask_actor_creation_dummy_object_id, - METH_NOARGS, "Return the actor creation dummy object ID for the task."}, - {"required_resources", (PyCFunction)PyTask_required_resources, METH_NOARGS, - "Return the resource vector of the task."}, - {"returns", (PyCFunction)PyTask_returns, METH_NOARGS, - "Return the object IDs for the return values of the task."}, - {"_serialized_raylet_task", (PyCFunction)PyTask_to_serialized_flatbuf, METH_NOARGS, - "This is a hack used to create a serialized flatbuffer object for the " - "driver task. We're doing this because creating the flatbuffer object in " - "Python didn't seem to work."}, - {NULL} /* Sentinel */ -}; - -PyTypeObject PyTaskType = { - PyVarObject_HEAD_INIT(NULL, 0) /* ob_size */ - "task.Task", /* tp_name */ - sizeof(PyTask), /* tp_basicsize */ - 0, /* tp_itemsize */ - (destructor)PyTask_dealloc, /* tp_dealloc */ - 0, /* tp_print */ - 0, /* tp_getattr */ - 0, /* tp_setattr */ - 0, /* tp_compare */ - 0, /* tp_repr */ - 0, /* tp_as_number */ - 0, /* tp_as_sequence */ - 0, /* tp_as_mapping */ - 0, /* tp_hash */ - 0, /* tp_call */ - 0, /* tp_str */ - 0, /* tp_getattro */ - 0, /* tp_setattro */ - 0, /* tp_as_buffer */ - Py_TPFLAGS_DEFAULT, /* tp_flags */ - "Task object", /* tp_doc */ - 0, /* tp_traverse */ - 0, /* tp_clear */ - 0, /* tp_richcompare */ - 0, /* tp_weaklistoffset */ - 0, /* tp_iter */ - 0, /* tp_iternext */ - PyTask_methods, /* tp_methods */ - 0, /* tp_members */ - 0, /* tp_getset */ - 0, /* tp_base */ - 0, /* tp_dict */ - 0, /* tp_descr_get */ - 0, /* tp_descr_set */ - 0, /* tp_dictoffset */ - (initproc)PyTask_init, /* tp_init */ - 0, /* tp_alloc */ - PyType_GenericNew, /* tp_new */ -}; - -// Create a PyTask from a C struct. The resulting PyTask takes ownership of the -// TaskSpec and will deallocate the TaskSpec in the PyTask destructor. -PyObject *PyTask_make(ray::raylet::TaskSpecification *task_spec) { - PyTask *result = PyObject_New(PyTask, &PyTaskType); - result = (PyTask *)PyObject_Init((PyObject *)result, &PyTaskType); - result->task_spec = task_spec; - // The created task does not include any execution dependencies. - result->execution_dependencies = new std::vector(); - return (PyObject *)result; -} - -/* Define the methods for the module. */ - -/** - * This method checks if a Python object is sufficiently simple that it can be - * serialized and passed by value as an argument to a task (without being put in - * the object store). The details of which objects are sufficiently simple are - * defined by this method and are not particularly important. But for - * performance reasons, it is better to place "small" objects in the task itself - * and "large" objects in the object store. - * - * @param value The Python object in question. - * @param num_elements_contained If this method returns 1, then the number of - * objects recursively contained within this object will be added to the - * value at this address. This is used to make sure that we do not - * serialize objects that are too large. - * @return False if the object cannot be serialized in the task and true if it - * can. - */ -bool is_simple_value(PyObject *value, int *num_elements_contained) { - *num_elements_contained += 1; - if (*num_elements_contained >= RayConfig::instance().num_elements_limit()) { - return false; - } - if (PyInt_Check(value) || PyLong_Check(value) || value == Py_False || - value == Py_True || PyFloat_Check(value) || value == Py_None) { - return true; - } - if (PyBytes_CheckExact(value)) { - *num_elements_contained += PyBytes_Size(value); - return (*num_elements_contained < RayConfig::instance().num_elements_limit()); - } - if (PyUnicode_CheckExact(value)) { - *num_elements_contained += PyUnicode_GET_SIZE(value); - return (*num_elements_contained < RayConfig::instance().num_elements_limit()); - } - if (PyList_CheckExact(value) && - PyList_Size(value) < RayConfig::instance().size_limit()) { - for (Py_ssize_t i = 0; i < PyList_Size(value); ++i) { - if (!is_simple_value(PyList_GetItem(value, i), num_elements_contained)) { - return false; - } - } - return (*num_elements_contained < RayConfig::instance().num_elements_limit()); - } - if (PyDict_CheckExact(value) && - PyDict_Size(value) < RayConfig::instance().size_limit()) { - PyObject *key, *val; - Py_ssize_t pos = 0; - while (PyDict_Next(value, &pos, &key, &val)) { - if (!is_simple_value(key, num_elements_contained) || - !is_simple_value(val, num_elements_contained)) { - return false; - } - } - return (*num_elements_contained < RayConfig::instance().num_elements_limit()); - } - if (PyTuple_CheckExact(value) && - PyTuple_Size(value) < RayConfig::instance().size_limit()) { - for (Py_ssize_t i = 0; i < PyTuple_Size(value); ++i) { - if (!is_simple_value(PyTuple_GetItem(value, i), num_elements_contained)) { - return false; - } - } - return (*num_elements_contained < RayConfig::instance().num_elements_limit()); - } - if (PyArray_CheckExact(value)) { - PyArrayObject *array = reinterpret_cast(value); - if (PyArray_TYPE(array) == NPY_OBJECT) { - return false; - } - *num_elements_contained += PyArray_NBYTES(array); - return (*num_elements_contained < RayConfig::instance().num_elements_limit()); - } - return false; -} - -PyObject *check_simple_value(PyObject *self, PyObject *args) { - PyObject *value; - if (!PyArg_ParseTuple(args, "O", &value)) { - return NULL; - } - int num_elements_contained = 0; - if (is_simple_value(value, &num_elements_contained)) { - Py_RETURN_TRUE; - } - Py_RETURN_FALSE; -} - -PyObject *compute_task_id(PyObject *self, PyObject *args) { - ObjectID object_id; - if (!PyArg_ParseTuple(args, "O&", &PyObjectToUniqueID, &object_id)) { - return NULL; - } - TaskID task_id = ray::ComputeTaskId(object_id); - return PyObjectID_make(task_id); -} diff --git a/src/ray/raylet/lib/python/common_extension.h b/src/ray/raylet/lib/python/common_extension.h deleted file mode 100644 index a0c26e53babf..000000000000 --- a/src/ray/raylet/lib/python/common_extension.h +++ /dev/null @@ -1,60 +0,0 @@ -#ifndef COMMON_EXTENSION_H -#define COMMON_EXTENSION_H - -#include - -#include -#include "marshal.h" -#include "structmember.h" - -#include "ray/raylet/task_spec.h" - -typedef char TaskSpec; -class TaskBuilder; - -extern PyObject *ray_common_error; - -// clang-format off -typedef struct { - PyObject_HEAD - ray::ObjectID object_id; -} PyObjectID; - -typedef struct { - PyObject_HEAD - // The task spec to use in the raylet case. - ray::raylet::TaskSpecification *task_spec; - std::vector *execution_dependencies; -} PyTask; -// clang-format on - -extern PyTypeObject PyObjectIDType; - -extern PyTypeObject PyTaskType; - -/* Python module for pickling. */ -extern PyObject *pickle_module; -extern PyObject *pickle_dumps; -extern PyObject *pickle_loads; - -int init_numpy_module(void); - -void init_pickle_module(void); - -extern TaskBuilder *g_task_builder; - -int PyStringToUniqueID(PyObject *object, ray::ObjectID *object_id); - -int PyObjectToUniqueID(PyObject *object, ray::ObjectID *object_id); - -PyObject *PyObjectID_make(ray::ObjectID object_id); - -PyObject *check_simple_value(PyObject *self, PyObject *args); -PyObject *compute_task_id(PyObject *self, PyObject *args); - -PyObject *PyTask_to_string(PyObject *, PyObject *args); -PyObject *PyTask_from_string(PyObject *, PyObject *args); - -PyObject *PyTask_make(ray::raylet::TaskSpecification *task_spec); - -#endif /* COMMON_EXTENSION_H */ diff --git a/src/ray/raylet/lib/python/config_extension.cc b/src/ray/raylet/lib/python/config_extension.cc deleted file mode 100644 index 3431641d22a5..000000000000 --- a/src/ray/raylet/lib/python/config_extension.cc +++ /dev/null @@ -1,192 +0,0 @@ -#include -#include "bytesobject.h" - -#include "config_extension.h" -#include "ray/ray_config.h" - -PyObject *PyRayConfig_make() { - PyRayConfig *result = PyObject_New(PyRayConfig, &PyRayConfigType); - result = (PyRayConfig *)PyObject_Init((PyObject *)result, &PyRayConfigType); - return (PyObject *)result; -} - -PyObject *PyRayConfig_ray_protocol_version(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().ray_protocol_version()); -} - -PyObject *PyRayConfig_heartbeat_timeout_milliseconds(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().heartbeat_timeout_milliseconds()); -} - -PyObject *PyRayConfig_num_heartbeats_timeout(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().num_heartbeats_timeout()); -} - -PyObject *PyRayConfig_get_timeout_milliseconds(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().get_timeout_milliseconds()); -} - -PyObject *PyRayConfig_worker_get_request_size(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().worker_get_request_size()); -} - -PyObject *PyRayConfig_worker_fetch_request_size(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().worker_fetch_request_size()); -} - -PyObject *PyRayConfig_actor_max_dummy_objects(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().actor_max_dummy_objects()); -} - -PyObject *PyRayConfig_num_connect_attempts(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().num_connect_attempts()); -} - -PyObject *PyRayConfig_connect_timeout_milliseconds(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().connect_timeout_milliseconds()); -} - -PyObject *PyRayConfig_local_scheduler_fetch_timeout_milliseconds(PyObject *self) { - return PyLong_FromLongLong( - RayConfig::instance().local_scheduler_fetch_timeout_milliseconds()); -} - -PyObject *PyRayConfig_local_scheduler_reconstruction_timeout_milliseconds( - PyObject *self) { - return PyLong_FromLongLong( - RayConfig::instance().local_scheduler_reconstruction_timeout_milliseconds()); -} - -PyObject *PyRayConfig_max_num_to_reconstruct(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().max_num_to_reconstruct()); -} - -PyObject *PyRayConfig_local_scheduler_fetch_request_size(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().local_scheduler_fetch_request_size()); -} - -PyObject *PyRayConfig_kill_worker_timeout_milliseconds(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().kill_worker_timeout_milliseconds()); -} - -PyObject *PyRayConfig_size_limit(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().size_limit()); -} - -PyObject *PyRayConfig_num_elements_limit(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().num_elements_limit()); -} - -PyObject *PyRayConfig_max_time_for_loop(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().max_time_for_loop()); -} - -PyObject *PyRayConfig_redis_db_connect_retries(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().redis_db_connect_retries()); -} - -PyObject *PyRayConfig_redis_db_connect_wait_milliseconds(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().redis_db_connect_wait_milliseconds()); -} - -PyObject *PyRayConfig_plasma_default_release_delay(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().plasma_default_release_delay()); -} - -PyObject *PyRayConfig_L3_cache_size_bytes(PyObject *self) { - return PyLong_FromLongLong(RayConfig::instance().L3_cache_size_bytes()); -} - -static PyMethodDef PyRayConfig_methods[] = { - {"ray_protocol_version", (PyCFunction)PyRayConfig_ray_protocol_version, METH_NOARGS, - "Return ray_protocol_version"}, - {"heartbeat_timeout_milliseconds", - (PyCFunction)PyRayConfig_heartbeat_timeout_milliseconds, METH_NOARGS, - "Return heartbeat_timeout_milliseconds"}, - {"num_heartbeats_timeout", (PyCFunction)PyRayConfig_num_heartbeats_timeout, - METH_NOARGS, "Return num_heartbeats_timeout"}, - {"get_timeout_milliseconds", (PyCFunction)PyRayConfig_get_timeout_milliseconds, - METH_NOARGS, "Return get_timeout_milliseconds"}, - {"worker_get_request_size", (PyCFunction)PyRayConfig_worker_get_request_size, - METH_NOARGS, "Return worker_get_request_size"}, - {"worker_fetch_request_size", (PyCFunction)PyRayConfig_worker_fetch_request_size, - METH_NOARGS, "Return worker_fetch_request_size"}, - {"actor_max_dummy_objects", (PyCFunction)PyRayConfig_actor_max_dummy_objects, - METH_NOARGS, "Return actor_max_dummy_objects"}, - {"num_connect_attempts", (PyCFunction)PyRayConfig_num_connect_attempts, METH_NOARGS, - "Return num_connect_attempts"}, - {"connect_timeout_milliseconds", - (PyCFunction)PyRayConfig_connect_timeout_milliseconds, METH_NOARGS, - "Return connect_timeout_milliseconds"}, - {"local_scheduler_fetch_timeout_milliseconds", - (PyCFunction)PyRayConfig_local_scheduler_fetch_timeout_milliseconds, METH_NOARGS, - "Return local_scheduler_fetch_timeout_milliseconds"}, - {"local_scheduler_reconstruction_timeout_milliseconds", - (PyCFunction)PyRayConfig_local_scheduler_reconstruction_timeout_milliseconds, - METH_NOARGS, "Return local_scheduler_reconstruction_timeout_milliseconds"}, - {"max_num_to_reconstruct", (PyCFunction)PyRayConfig_max_num_to_reconstruct, - METH_NOARGS, "Return max_num_to_reconstruct"}, - {"local_scheduler_fetch_request_size", - (PyCFunction)PyRayConfig_local_scheduler_fetch_request_size, METH_NOARGS, - "Return local_scheduler_fetch_request_size"}, - {"kill_worker_timeout_milliseconds", - (PyCFunction)PyRayConfig_kill_worker_timeout_milliseconds, METH_NOARGS, - "Return kill_worker_timeout_milliseconds"}, - {"size_limit", (PyCFunction)PyRayConfig_size_limit, METH_NOARGS, "Return size_limit"}, - {"num_elements_limit", (PyCFunction)PyRayConfig_num_elements_limit, METH_NOARGS, - "Return num_elements_limit"}, - {"max_time_for_loop", (PyCFunction)PyRayConfig_max_time_for_loop, METH_NOARGS, - "Return max_time_for_loop"}, - {"redis_db_connect_retries", (PyCFunction)PyRayConfig_redis_db_connect_retries, - METH_NOARGS, "Return redis_db_connect_retries"}, - {"redis_db_connect_wait_milliseconds", - (PyCFunction)PyRayConfig_redis_db_connect_wait_milliseconds, METH_NOARGS, - "Return redis_db_connect_wait_milliseconds"}, - {"plasma_default_release_delay", - (PyCFunction)PyRayConfig_plasma_default_release_delay, METH_NOARGS, - "Return plasma_default_release_delay"}, - {"L3_cache_size_bytes", (PyCFunction)PyRayConfig_L3_cache_size_bytes, METH_NOARGS, - "Return L3_cache_size_bytes"}, - {NULL} /* Sentinel */ -}; - -PyTypeObject PyRayConfigType = { - PyVarObject_HEAD_INIT(NULL, 0) /* ob_size */ - "common.RayConfig", /* tp_name */ - sizeof(PyRayConfig), /* tp_basicsize */ - 0, /* tp_itemsize */ - 0, /* tp_dealloc */ - 0, /* tp_print */ - 0, /* tp_getattr */ - 0, /* tp_setattr */ - 0, /* tp_compare */ - 0, /* tp_repr */ - 0, /* tp_as_number */ - 0, /* tp_as_sequence */ - 0, /* tp_as_mapping */ - 0, /* tp_hash */ - 0, /* tp_call */ - 0, /* tp_str */ - 0, /* tp_getattro */ - 0, /* tp_setattro */ - 0, /* tp_as_buffer */ - Py_TPFLAGS_DEFAULT, /* tp_flags */ - "RayConfig object", /* tp_doc */ - 0, /* tp_traverse */ - 0, /* tp_clear */ - 0, /* tp_richcompare */ - 0, /* tp_weaklistoffset */ - 0, /* tp_iter */ - 0, /* tp_iternext */ - PyRayConfig_methods, /* tp_methods */ - 0, /* tp_members */ - 0, /* tp_getset */ - 0, /* tp_base */ - 0, /* tp_dict */ - 0, /* tp_descr_get */ - 0, /* tp_descr_set */ - 0, /* tp_dictoffset */ - 0, /* tp_init */ - 0, /* tp_alloc */ - PyType_GenericNew, /* tp_new */ -}; diff --git a/src/ray/raylet/lib/python/config_extension.h b/src/ray/raylet/lib/python/config_extension.h deleted file mode 100644 index 182158e9bd8e..000000000000 --- a/src/ray/raylet/lib/python/config_extension.h +++ /dev/null @@ -1,39 +0,0 @@ -#ifndef CONFIG_EXTENSION_H -#define CONFIG_EXTENSION_H - -#include - -// clang-format off -typedef struct { - PyObject_HEAD -} PyRayConfig; -// clang-format on - -extern PyTypeObject PyRayConfigType; - -/* Create a PyRayConfig from C++. */ -PyObject *PyRayConfig_make(); - -PyObject *PyRayConfig_ray_protocol_version(PyObject *self); -PyObject *PyRayConfig_heartbeat_timeout_milliseconds(PyObject *self); -PyObject *PyRayConfig_num_heartbeats_timeout(PyObject *self); -PyObject *PyRayConfig_get_timeout_milliseconds(PyObject *self); -PyObject *PyRayConfig_worker_get_request_size(PyObject *self); -PyObject *PyRayConfig_worker_fetch_request_size(PyObject *self); -PyObject *PyRayConfig_actor_max_dummy_objects(PyObject *self); -PyObject *PyRayConfig_num_connect_attempts(PyObject *self); -PyObject *PyRayConfig_connect_timeout_milliseconds(PyObject *self); -PyObject *PyRayConfig_local_scheduler_fetch_timeout_milliseconds(PyObject *self); -PyObject *PyRayConfig_local_scheduler_reconstruction_timeout_milliseconds(PyObject *self); -PyObject *PyRayConfig_max_num_to_reconstruct(PyObject *self); -PyObject *PyRayConfig_local_scheduler_fetch_request_size(PyObject *self); -PyObject *PyRayConfig_kill_worker_timeout_milliseconds(PyObject *self); -PyObject *PyRayConfig_size_limit(PyObject *self); -PyObject *PyRayConfig_num_elements_limit(PyObject *self); -PyObject *PyRayConfig_max_time_for_loop(PyObject *self); -PyObject *PyRayConfig_redis_db_connect_retries(PyObject *self); -PyObject *PyRayConfig_redis_db_connect_wait_milliseconds(PyObject *self); -PyObject *PyRayConfig_plasma_default_release_delay(PyObject *self); -PyObject *PyRayConfig_L3_cache_size_bytes(PyObject *self); - -#endif /* CONFIG_EXTENSION_H */ diff --git a/src/ray/raylet/lib/python/raylet_extension.cc b/src/ray/raylet/lib/python/raylet_extension.cc deleted file mode 100644 index 755ddf1ddd1c..000000000000 --- a/src/ray/raylet/lib/python/raylet_extension.cc +++ /dev/null @@ -1,505 +0,0 @@ -#include -#include - -#include "common_extension.h" -#include "config_extension.h" -#include "ray/raylet/raylet_client.h" - -PyObject *LocalSchedulerError; - -// clang-format off -typedef struct { - PyObject_HEAD - RayletClient *raylet_client; -} PyRayletClient; -// clang-format on - -static int PyRayletClient_init(PyRayletClient *self, PyObject *args, PyObject *kwds) { - char *socket_name; - UniqueID client_id; - PyObject *is_worker; - JobID driver_id; - if (!PyArg_ParseTuple(args, "sO&OO&", &socket_name, PyStringToUniqueID, &client_id, - &is_worker, &PyObjectToUniqueID, &driver_id)) { - self->raylet_client = NULL; - return -1; - } - /* Connect to the local scheduler. */ - self->raylet_client = new RayletClient(socket_name, client_id, - static_cast(PyObject_IsTrue(is_worker)), - driver_id, Language::PYTHON); - return 0; -} - -PyObject *ray_common_error = nullptr; - -static void PyRayletClient_dealloc(PyRayletClient *self) { - if (self->raylet_client != NULL) { - delete self->raylet_client; - } - Py_TYPE(self)->tp_free((PyObject *)self); -} - -static PyObject *PyRayletClient_Disconnect(PyRayletClient *self) { - auto status = self->raylet_client->Disconnect(); - RAY_CHECK_OK_PREPEND(status, "[RayletClient] Failed to disconnect."); - Py_RETURN_NONE; -} - -static PyObject *PyRayletClient_SubmitTask(PyRayletClient *self, PyObject *args) { - PyObject *py_task; - if (!PyArg_ParseTuple(args, "O", &py_task)) { - return NULL; - } - PyTask *task = reinterpret_cast(py_task); - auto status = - self->raylet_client->SubmitTask(*task->execution_dependencies, *task->task_spec); - RAY_CHECK_OK_PREPEND(status, "[RayletClient] Failed to submit a task to raylet."); - Py_RETURN_NONE; -} - -// clang-format off -static PyObject *PyRayletClient_GetTask(PyRayletClient *self) { - std::unique_ptr task_spec; - /* Drop the global interpreter lock while we get a task because - * raylet_GetTask may block for a long time. */ - Py_BEGIN_ALLOW_THREADS - auto status = self->raylet_client->GetTask(&task_spec); - RAY_CHECK_OK_PREPEND(status, "[RayletClient] Failed to get a task from raylet."); - Py_END_ALLOW_THREADS - return PyTask_make(task_spec.release()); -} -// clang-format on - -/// A helper function that converts a Python list of object ids to a vector. -/// -/// \param py_list The Python list of object ids. -/// \param output The output vector. -/// \return True if an error occurred when parsing the Python object ids, false otherwise. -bool py_object_id_list_to_vector(PyObject *py_list, std::vector &output) { - Py_ssize_t n = PyList_Size(py_list); - for (int64_t i = 0; i < n; ++i) { - ObjectID object_id; - PyObject *py_object_id = PyList_GetItem(py_list, i); - if (!PyObjectToUniqueID(py_object_id, &object_id)) { - return true; - } - output.push_back(object_id); - } - return false; -} - -static PyObject *PyRayletClient_FetchOrReconstruct(PyRayletClient *self, PyObject *args) { - PyObject *py_object_ids; - PyObject *py_fetch_only; - TaskID current_task_id; - if (!PyArg_ParseTuple(args, "OO|O&", &py_object_ids, &py_fetch_only, - &PyObjectToUniqueID, ¤t_task_id)) { - return NULL; - } - bool fetch_only = PyObject_IsTrue(py_fetch_only); - - // Convert object ids. - std::vector object_ids; - if (py_object_id_list_to_vector(py_object_ids, object_ids)) { - return NULL; - } - - auto status = - self->raylet_client->FetchOrReconstruct(object_ids, fetch_only, current_task_id); - if (status.ok()) { - Py_RETURN_NONE; - } else { - std::ostringstream stream; - stream << "[RayletClient] FetchOrReconstruct failed: " - << "raylet client may be closed, check raylet status. error message: " - << status.ToString(); - PyErr_SetString(ray_common_error, stream.str().c_str()); - return NULL; - } -} - -static PyObject *PyRayletClient_NotifyUnblocked(PyRayletClient *self, PyObject *args) { - TaskID current_task_id; - if (!PyArg_ParseTuple(args, "O&", &PyObjectToUniqueID, ¤t_task_id)) { - return NULL; - } - auto status = self->raylet_client->NotifyUnblocked(current_task_id); - RAY_CHECK_OK_PREPEND(status, "[RayletClient] Failed to notify unblocked."); - Py_RETURN_NONE; -} - -static PyObject *PyRayletClient_compute_put_id(PyObject *self, PyObject *args) { - int put_index; - TaskID task_id; - if (!PyArg_ParseTuple(args, "O&i", &PyObjectToUniqueID, &task_id, &put_index)) { - return NULL; - } - const ObjectID put_id = ComputePutId(task_id, put_index); - return PyObjectID_make(put_id); -} - -static PyObject *PyRayletClient_resource_ids(PyRayletClient *self) { - // Construct a Python dictionary of resource IDs and resource fractions. - PyObject *resource_ids = PyDict_New(); - for (auto const &resource_info : self->raylet_client->GetResourceIDs()) { - auto const &resource_name = resource_info.first; - auto const &ids_and_fractions = resource_info.second; - -#if PY_MAJOR_VERSION >= 3 - PyObject *key = - PyUnicode_FromStringAndSize(resource_name.data(), resource_name.size()); -#else - PyObject *key = PyBytes_FromStringAndSize(resource_name.data(), resource_name.size()); -#endif - PyObject *value = PyList_New(ids_and_fractions.size()); - for (size_t i = 0; i < ids_and_fractions.size(); ++i) { - auto const &id_and_fraction = ids_and_fractions[i]; - PyObject *id_fraction_pair = - Py_BuildValue("(Ld)", id_and_fraction.first, id_and_fraction.second); - PyList_SetItem(value, i, id_fraction_pair); - } - PyDict_SetItem(resource_ids, key, value); - Py_DECREF(key); - Py_DECREF(value); - } - - return resource_ids; -} - -static PyObject *PyRayletClient_Wait(PyRayletClient *self, PyObject *args) { - PyObject *py_object_ids; - int num_returns; - int64_t timeout_ms; - PyObject *py_wait_local; - TaskID current_task_id; - - if (!PyArg_ParseTuple(args, "OilOO&", &py_object_ids, &num_returns, &timeout_ms, - &py_wait_local, &PyObjectToUniqueID, ¤t_task_id)) { - return NULL; - } - - bool wait_local = PyObject_IsTrue(py_wait_local); - - // Convert object ids. - std::vector object_ids; - if (py_object_id_list_to_vector(py_object_ids, object_ids)) { - return NULL; - } - - // Invoke wait. - WaitResultPair result; - auto status = self->raylet_client->Wait(object_ids, num_returns, timeout_ms, wait_local, - current_task_id, &result); - RAY_CHECK_OK_PREPEND(status, "[RayletClient] Failed to wait for objects."); - - // Convert result to py object. - PyObject *py_found = PyList_New(static_cast(result.first.size())); - for (uint i = 0; i < result.first.size(); ++i) { - PyList_SetItem(py_found, i, PyObjectID_make(result.first[i])); - } - PyObject *py_remaining = PyList_New(static_cast(result.second.size())); - for (uint i = 0; i < result.second.size(); ++i) { - PyList_SetItem(py_remaining, i, PyObjectID_make(result.second[i])); - } - return Py_BuildValue("(NN)", py_found, py_remaining); -} - -static PyObject *PyRayletClient_PushError(PyRayletClient *self, PyObject *args) { - JobID job_id; - const char *type; - int type_length; - const char *error_message; - int error_message_length; - double timestamp; - if (!PyArg_ParseTuple(args, "O&s#s#d", &PyObjectToUniqueID, &job_id, &type, - &type_length, &error_message, &error_message_length, - ×tamp)) { - return NULL; - } - - auto status = self->raylet_client->PushError( - job_id, std::string(type, type_length), - std::string(error_message, error_message_length), timestamp); - RAY_CHECK_OK_PREPEND(status, "[RayletClient] Failed to push errors to raylet."); - Py_RETURN_NONE; -} - -int PyBytes_or_PyUnicode_to_string(PyObject *py_string, std::string &out) { - // Handle the case where the key is a bytes object and the case where it - // is a unicode object. - if (PyUnicode_Check(py_string)) { - PyObject *ascii_string = PyUnicode_AsASCIIString(py_string); - out = std::string(PyBytes_AsString(ascii_string), PyBytes_Size(ascii_string)); - Py_DECREF(ascii_string); - } else if (PyBytes_Check(py_string)) { - out = std::string(PyBytes_AsString(py_string), PyBytes_Size(py_string)); - } else { - return -1; - } - - return 0; -} - -static PyObject *PyRayletClient_PushProfileEvents(PyRayletClient *self, PyObject *args) { - const char *component_type; - int component_type_length; - UniqueID component_id; - PyObject *profile_data; - const char *node_ip_address; - int node_ip_address_length; - - if (!PyArg_ParseTuple(args, "s#O&s#O", &component_type, &component_type_length, - &PyObjectToUniqueID, &component_id, &node_ip_address, - &node_ip_address_length, &profile_data)) { - return NULL; - } - - ProfileTableDataT profile_info; - profile_info.component_type = std::string(component_type, component_type_length); - profile_info.component_id = component_id.binary(); - profile_info.node_ip_address = std::string(node_ip_address, node_ip_address_length); - - if (PyList_Size(profile_data) == 0) { - // Short circuit if there are no profile events. - Py_RETURN_NONE; - } - - for (int64_t i = 0; i < PyList_Size(profile_data); ++i) { - ProfileEventT profile_event; - PyObject *py_profile_event = PyList_GetItem(profile_data, i); - - if (!PyDict_CheckExact(py_profile_event)) { - return NULL; - } - - PyObject *key, *val; - Py_ssize_t pos = 0; - while (PyDict_Next(py_profile_event, &pos, &key, &val)) { - std::string key_string; - if (PyBytes_or_PyUnicode_to_string(key, key_string) == -1) { - return NULL; - } - - // TODO(rkn): If the dictionary is formatted incorrectly, that could lead - // to errors. E.g., if any of the strings are empty, that will cause - // segfaults in the node manager. - - if (key_string == std::string("event_type")) { - if (PyBytes_or_PyUnicode_to_string(val, profile_event.event_type) == -1) { - return NULL; - } - if (profile_event.event_type.size() == 0) { - return NULL; - } - } else if (key_string == std::string("start_time")) { - profile_event.start_time = PyFloat_AsDouble(val); - } else if (key_string == std::string("end_time")) { - profile_event.end_time = PyFloat_AsDouble(val); - } else if (key_string == std::string("extra_data")) { - if (PyBytes_or_PyUnicode_to_string(val, profile_event.extra_data) == -1) { - return NULL; - } - if (profile_event.extra_data.size() == 0) { - return NULL; - } - } else { - return NULL; - } - } - - // Note that profile_info.profile_events is a vector of unique pointers, so - // profile_event will be deallocated when profile_info goes out of scope. - profile_info.profile_events.emplace_back(new ProfileEventT(profile_event)); - } - - auto status = self->raylet_client->PushProfileEvents(profile_info); - RAY_CHECK_OK_PREPEND(status, "[RayletClient] Failed to push profile events to raylet."); - Py_RETURN_NONE; -} - -static PyObject *PyRayletClient_FreeObjects(PyRayletClient *self, PyObject *args) { - PyObject *py_object_ids; - PyObject *py_local_only; - - if (!PyArg_ParseTuple(args, "OO", &py_object_ids, &py_local_only)) { - return NULL; - } - - bool local_only = static_cast(PyObject_IsTrue(py_local_only)); - - // Convert object ids. - std::vector object_ids; - if (py_object_id_list_to_vector(py_object_ids, object_ids)) { - return NULL; - } - - // Invoke raylet_FreeObjects. - auto status = self->raylet_client->FreeObjects(object_ids, local_only); - RAY_CHECK_OK_PREPEND(status, "[RayletClient] Failed to free objects."); - Py_RETURN_NONE; -} - -static PyMethodDef PyRayletClient_methods[] = { - {"disconnect", (PyCFunction)PyRayletClient_Disconnect, METH_NOARGS, - "Notify the local scheduler that this client is exiting gracefully."}, - {"submit_task", (PyCFunction)PyRayletClient_SubmitTask, METH_VARARGS, - "Submit a task to the local scheduler."}, - {"get_task", (PyCFunction)PyRayletClient_GetTask, METH_NOARGS, - "Get a task from the local scheduler."}, - {"fetch_or_reconstruct", (PyCFunction)PyRayletClient_FetchOrReconstruct, METH_VARARGS, - "Ask the local scheduler to reconstruct an object."}, - {"notify_unblocked", (PyCFunction)PyRayletClient_NotifyUnblocked, METH_VARARGS, - "Notify the local scheduler that we are unblocked."}, - {"compute_put_id", (PyCFunction)PyRayletClient_compute_put_id, METH_VARARGS, - "Return the object ID for a put call within a task."}, - {"resource_ids", (PyCFunction)PyRayletClient_resource_ids, METH_NOARGS, - "Get the IDs of the resources that are reserved for this client."}, - {"wait", (PyCFunction)PyRayletClient_Wait, METH_VARARGS, - "Wait for a list of objects to be created."}, - {"push_error", (PyCFunction)PyRayletClient_PushError, METH_VARARGS, - "Push an error message to the relevant driver."}, - {"push_profile_events", (PyCFunction)PyRayletClient_PushProfileEvents, METH_VARARGS, - "Store some profiling events in the GCS."}, - {"free_objects", (PyCFunction)PyRayletClient_FreeObjects, METH_VARARGS, - "Free a list of objects from object stores."}, - {NULL} /* Sentinel */ -}; - -static PyTypeObject PyRayletClientType = { - PyVarObject_HEAD_INIT(NULL, 0) /* ob_size */ - "raylet.RayletClient", /* tp_name */ - sizeof(PyRayletClient), /* tp_basicsize */ - 0, /* tp_itemsize */ - (destructor)PyRayletClient_dealloc, /* tp_dealloc */ - 0, /* tp_print */ - 0, /* tp_getattr */ - 0, /* tp_setattr */ - 0, /* tp_compare */ - 0, /* tp_repr */ - 0, /* tp_as_number */ - 0, /* tp_as_sequence */ - 0, /* tp_as_mapping */ - 0, /* tp_hash */ - 0, /* tp_call */ - 0, /* tp_str */ - 0, /* tp_getattro */ - 0, /* tp_setattro */ - 0, /* tp_as_buffer */ - Py_TPFLAGS_DEFAULT, /* tp_flags */ - "RayletClient object", /* tp_doc */ - 0, /* tp_traverse */ - 0, /* tp_clear */ - 0, /* tp_richcompare */ - 0, /* tp_weaklistoffset */ - 0, /* tp_iter */ - 0, /* tp_iternext */ - PyRayletClient_methods, /* tp_methods */ - 0, /* tp_members */ - 0, /* tp_getset */ - 0, /* tp_base */ - 0, /* tp_dict */ - 0, /* tp_descr_get */ - 0, /* tp_descr_set */ - 0, /* tp_dictoffset */ - (initproc)PyRayletClient_init, /* tp_init */ - 0, /* tp_alloc */ - PyType_GenericNew, /* tp_new */ -}; - -static PyMethodDef raylet_methods[] = { - {"check_simple_value", check_simple_value, METH_VARARGS, - "Should the object be passed by value?"}, - {"compute_task_id", compute_task_id, METH_VARARGS, - "Return the task ID of an object ID."}, - {"task_from_string", PyTask_from_string, METH_VARARGS, - "Creates a Python PyTask object from a string representation of " - "TaskSpec."}, - {"task_to_string", PyTask_to_string, METH_VARARGS, - "Translates a PyTask python object to a byte string."}, - {NULL} /* Sentinel */ -}; - -#if PY_MAJOR_VERSION >= 3 -static struct PyModuleDef moduledef = { - PyModuleDef_HEAD_INIT, - "libraylet", /* m_name */ - "A module for the raylet.", /* m_doc */ - 0, /* m_size */ - raylet_methods, /* m_methods */ - NULL, /* m_reload */ - NULL, /* m_traverse */ - NULL, /* m_clear */ - NULL, /* m_free */ -}; -#endif - -#if PY_MAJOR_VERSION >= 3 -#define INITERROR return NULL -#else -#define INITERROR return -#endif - -#ifndef PyMODINIT_FUNC /* declarations for DLL import/export */ -#define PyMODINIT_FUNC void -#endif - -#if PY_MAJOR_VERSION >= 3 -#define MOD_INIT(name) PyMODINIT_FUNC PyInit_##name(void) -#else -#define MOD_INIT(name) PyMODINIT_FUNC init##name(void) -#endif - -MOD_INIT(libraylet_library_python) { - if (PyType_Ready(&PyTaskType) < 0) { - INITERROR; - } - - if (PyType_Ready(&PyObjectIDType) < 0) { - INITERROR; - } - - if (PyType_Ready(&PyRayletClientType) < 0) { - INITERROR; - } - - if (PyType_Ready(&PyRayConfigType) < 0) { - INITERROR; - } - -#if PY_MAJOR_VERSION >= 3 - PyObject *m = PyModule_Create(&moduledef); -#else - PyObject *m = Py_InitModule3("libraylet_library_python", raylet_methods, - "A module for the raylet."); -#endif - - init_numpy_module(); - init_pickle_module(); - - Py_INCREF(&PyTaskType); - PyModule_AddObject(m, "Task", (PyObject *)&PyTaskType); - - Py_INCREF(&PyObjectIDType); - PyModule_AddObject(m, "ObjectID", (PyObject *)&PyObjectIDType); - - Py_INCREF(&PyRayletClientType); - PyModule_AddObject(m, "RayletClient", (PyObject *)&PyRayletClientType); - - char common_error[] = "common.error"; - ray_common_error = PyErr_NewException(common_error, NULL, NULL); - Py_INCREF(ray_common_error); - PyModule_AddObject(m, "RayCommonError", ray_common_error); - - Py_INCREF(&PyRayConfigType); - PyModule_AddObject(m, "RayConfig", (PyObject *)&PyRayConfigType); - - /* Create the global config object. */ - PyObject *config = PyRayConfig_make(); - /* TODO(rkn): Do we need Py_INCREF(config)? */ - PyModule_AddObject(m, "_config", config); - -#if PY_MAJOR_VERSION >= 3 - return m; -#endif -} diff --git a/src/ray/raylet/raylet_client.h b/src/ray/raylet/raylet_client.h index 9d9391971787..8aeaa5fa73c9 100644 --- a/src/ray/raylet/raylet_client.h +++ b/src/ray/raylet/raylet_client.h @@ -14,6 +14,7 @@ using ray::JobID; using ray::ObjectID; using ray::TaskID; using ray::UniqueID; +using ray::ClientID; using MessageType = ray::protocol::MessageType; using ResourceMappingType = @@ -147,7 +148,7 @@ class RayletClient { Language GetLanguage() const { return language_; } - JobID GetClientID() const { return client_id_; } + ClientID GetClientID() const { return client_id_; } JobID GetDriverID() const { return driver_id_; } @@ -156,7 +157,7 @@ class RayletClient { const ResourceMappingType &GetResourceIDs() const { return resource_ids_; } private: - const UniqueID client_id_; + const ClientID client_id_; const bool is_worker_; const JobID driver_id_; const Language language_; diff --git a/src/ray/raylet/task.cc b/src/ray/raylet/task.cc index 880956ea395c..5d6a02186ced 100644 --- a/src/ray/raylet/task.cc +++ b/src/ray/raylet/task.cc @@ -46,6 +46,16 @@ void Task::CopyTaskExecutionSpec(const Task &task) { ComputeDependencies(); } +std::string SerializeTaskAsString(const std::vector *dependencies, + const TaskSpecification *task_spec) { + flatbuffers::FlatBufferBuilder fbb; + std::vector execution_dependencies(*dependencies); + TaskExecutionSpecification execution_spec(std::move(execution_dependencies)); + Task task(execution_spec, *task_spec); + fbb.Finish(task.ToFlatbuffer(fbb)); + return std::string(fbb.GetBufferPointer(), fbb.GetBufferPointer() + fbb.GetSize()); +} + } // namespace raylet } // namespace ray diff --git a/src/ray/raylet/task.h b/src/ray/raylet/task.h index 3663ff2139cd..b942e2bf2c03 100644 --- a/src/ray/raylet/task.h +++ b/src/ray/raylet/task.h @@ -100,6 +100,9 @@ class Task { std::vector dependencies_; }; +std::string SerializeTaskAsString(const std::vector *dependencies, + const TaskSpecification *task_spec); + } // namespace raylet } // namespace ray diff --git a/src/ray/raylet/task_spec.h b/src/ray/raylet/task_spec.h index 3c33d56aab2d..11e93050b9d1 100644 --- a/src/ray/raylet/task_spec.h +++ b/src/ray/raylet/task_spec.h @@ -155,6 +155,13 @@ class TaskSpecification { flatbuffers::Offset ToFlatbuffer( flatbuffers::FlatBufferBuilder &fbb) const; + std::string SerializeAsString() const { + flatbuffers::FlatBufferBuilder fbb; + auto string = ToFlatbuffer(fbb); + fbb.Finish(string); + return std::string(fbb.GetBufferPointer(), fbb.GetBufferPointer() + fbb.GetSize()); + } + // TODO(swang): Finalize and document these methods. TaskID TaskId() const; UniqueID DriverId() const; diff --git a/test/actor_test.py b/test/actor_test.py index 3617f78e8185..8a4d2ea3f992 100644 --- a/test/actor_test.py +++ b/test/actor_test.py @@ -331,7 +331,7 @@ def __init__(self): random.seed(1234) f2 = Foo.remote() - assert f1._ray_actor_id.id() != f2._ray_actor_id.id() + assert f1._ray_actor_id != f2._ray_actor_id def test_actor_class_name(ray_start_regular): diff --git a/test/failure_test.py b/test/failure_test.py index 03e3dc8a7b94..0a5a6255e336 100644 --- a/test/failure_test.py +++ b/test/failure_test.py @@ -675,8 +675,6 @@ def sleep_to_kill_raylet(): thread = threading.Thread(target=sleep_to_kill_raylet) thread.start() - with pytest.raises( - ray.raylet.RayCommonError, - match=r".*raylet client may be closed.*"): + with pytest.raises(Exception, match=r".*Connection closed unexpectedly.*"): ray.get(nonexistent_id) thread.join() diff --git a/test/object_manager_test.py b/test/object_manager_test.py index 271334e81c15..bf37a2644fe7 100644 --- a/test/object_manager_test.py +++ b/test/object_manager_test.py @@ -226,7 +226,7 @@ def f(size): x_ids = [f.remote(10**i) for i in [1, 2, 3, 4, 5, 6, 7]] assert not any( ray.worker.global_worker.plasma_client.contains( - ray.pyarrow.plasma.ObjectID(x_id.id())) for x_id in x_ids) + ray.pyarrow.plasma.ObjectID(x_id.binary())) for x_id in x_ids) start_time = time.time() @@ -240,7 +240,7 @@ def f(size): ray.put(x) assert not any( ray.worker.global_worker.plasma_client.contains( - ray.pyarrow.plasma.ObjectID(x_id.id())) for x_id in x_ids) + ray.pyarrow.plasma.ObjectID(x_id.binary())) for x_id in x_ids) end_time = time.time() @@ -264,7 +264,7 @@ def f(size): ray.put(x) assert not any( ray.worker.global_worker.plasma_client.contains( - ray.pyarrow.plasma.ObjectID(x_id.id())) for x_id in x_ids) + ray.pyarrow.plasma.ObjectID(x_id.binary())) for x_id in x_ids) time.sleep(repeated_push_delay) ray.get(x_ids) diff --git a/test/runtest.py b/test/runtest.py index 1e7af676eef4..4c7310bd5346 100644 --- a/test/runtest.py +++ b/test/runtest.py @@ -2311,7 +2311,7 @@ def test_global_state_api(shutdown_only): assert len(task_table) == 1 assert driver_task_id == list(task_table.keys())[0] task_spec = task_table[driver_task_id]["TaskSpec"] - nil_id_hex = ray.ObjectID.nil_id().hex() + nil_id_hex = ray.ObjectID.nil().hex() assert task_spec["TaskID"] == driver_task_id assert task_spec["ActorID"] == nil_id_hex @@ -2442,17 +2442,17 @@ def f(): def test_specific_driver_id(): - dummy_driver_id = ray.ObjectID(b"00112233445566778899") + dummy_driver_id = ray.DriverID(b"00112233445566778899") ray.init(driver_id=dummy_driver_id) @ray.remote def f(): - return ray.worker.global_worker.task_driver_id.id() + return ray.worker.global_worker.task_driver_id.binary() - assert_equal(dummy_driver_id.id(), ray.worker.global_worker.worker_id) + assert_equal(dummy_driver_id.binary(), ray.worker.global_worker.worker_id) task_driver_id = ray.get(f.remote()) - assert_equal(dummy_driver_id.id(), task_driver_id) + assert_equal(dummy_driver_id.binary(), task_driver_id) ray.shutdown() @@ -2460,8 +2460,8 @@ def f(): def test_object_id_properties(): id_bytes = b"00112233445566778899" object_id = ray.ObjectID(id_bytes) - assert object_id.id() == id_bytes - object_id = ray.ObjectID.nil_id() + assert object_id.binary() == id_bytes + object_id = ray.ObjectID.nil() assert object_id.is_nil() with pytest.raises(ValueError, match=r".*needs to have length 20.*"): ray.ObjectID(id_bytes + b"1234") @@ -2469,7 +2469,7 @@ def test_object_id_properties(): ray.ObjectID(b"0123456789") object_id = ray.ObjectID(_random_string()) assert not object_id.is_nil() - assert object_id.id() != id_bytes + assert object_id.binary() != id_bytes id_dumps = pickle.dumps(object_id) id_from_dumps = pickle.loads(id_dumps) assert id_from_dumps == object_id @@ -2506,7 +2506,7 @@ def f(): ray.wait([x_id]) ray.wait([f.remote()]) assert not ray.worker.global_worker.plasma_client.contains( - ray.pyarrow.plasma.ObjectID(x_id.id())) + ray.pyarrow.plasma.ObjectID(x_id.binary())) ready_ids, _ = ray.wait([x_id]) assert len(ready_ids) == 1 @@ -2534,7 +2534,7 @@ def unique_1(): def test_duplicate_error_messages(shutdown_only): ray.init(num_cpus=0) - driver_id = ray.ObjectID.nil_id() + driver_id = ray.DriverID.nil() error_data = ray.gcs_utils.construct_error_message(driver_id, "test", "message", 0) @@ -2544,13 +2544,13 @@ def test_duplicate_error_messages(shutdown_only): r = ray.worker.global_worker.redis_client r.execute_command("RAY.TABLE_APPEND", ray.gcs_utils.TablePrefix.ERROR_INFO, - ray.gcs_utils.TablePubsub.ERROR_INFO, driver_id.id(), + ray.gcs_utils.TablePubsub.ERROR_INFO, driver_id.binary(), error_data) # Before https://github.com/ray-project/ray/pull/3316 this would # give an error r.execute_command("RAY.TABLE_APPEND", ray.gcs_utils.TablePrefix.ERROR_INFO, - ray.gcs_utils.TablePubsub.ERROR_INFO, driver_id.id(), + ray.gcs_utils.TablePubsub.ERROR_INFO, driver_id.binary(), error_data)