mirror of
https://github.com/vale981/ray
synced 2025-03-05 10:01:43 -05:00
This reverts commit 3e492a79ec
.
This commit is contained in:
parent
a092433bc8
commit
5d987f5988
19 changed files with 54 additions and 54 deletions
|
@ -112,16 +112,20 @@ def test_memory_table(disable_aiohttp_cache, ray_start_with_dashboard):
|
|||
def check_mem_table():
|
||||
resp = requests.get(f"{webui_url}/memory/memory_table")
|
||||
resp_data = resp.json()
|
||||
assert resp_data["result"]
|
||||
if not resp_data["result"]:
|
||||
return False
|
||||
latest_memory_table = resp_data["data"]["memoryTable"]
|
||||
summary = latest_memory_table["summary"]
|
||||
# 1 ref per handle and per object the actor has a ref to
|
||||
assert summary["totalActorHandles"] == len(actors) * 2
|
||||
# 1 ref for my_obj
|
||||
assert summary["totalLocalRefCount"] == 1
|
||||
try:
|
||||
# 1 ref per handle and per object the actor has a ref to
|
||||
assert summary["totalActorHandles"] == len(actors) * 2
|
||||
# 1 ref for my_obj
|
||||
assert summary["totalLocalRefCount"] == 1
|
||||
return True
|
||||
except AssertionError:
|
||||
return False
|
||||
|
||||
wait_until_succeeded_without_exception(
|
||||
check_mem_table, (AssertionError, ), timeout_ms=1000)
|
||||
wait_for_condition(check_mem_table, 10)
|
||||
|
||||
|
||||
def test_get_all_node_details(disable_aiohttp_cache, ray_start_with_dashboard):
|
||||
|
|
|
@ -7,9 +7,8 @@ from ray.new_dashboard.memory_utils import (
|
|||
NODE_ADDRESS = "127.0.0.1"
|
||||
IS_DRIVER = True
|
||||
PID = 1
|
||||
|
||||
OBJECT_ID = "ZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZmZg=="
|
||||
ACTOR_ID = "fffffffffffffffffffffffffffffffff66d17ba010000c801000000"
|
||||
OBJECT_ID = "7wpsIhgZiBz/////AQAAyAEAAAA="
|
||||
ACTOR_ID = "fffffffffffffffff66d17ba010000c801000000"
|
||||
DECODED_ID = decode_object_ref_if_needed(OBJECT_ID)
|
||||
OBJECT_SIZE = 100
|
||||
|
||||
|
|
|
@ -7,7 +7,7 @@ import java.util.Random;
|
|||
|
||||
public class ActorId extends BaseId implements Serializable {
|
||||
|
||||
private static final int UNIQUE_BYTES_LENGTH = 12;
|
||||
private static final int UNIQUE_BYTES_LENGTH = 4;
|
||||
|
||||
public static final int LENGTH = JobId.LENGTH + UNIQUE_BYTES_LENGTH;
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ import java.util.Random;
|
|||
*/
|
||||
public class ObjectId extends BaseId implements Serializable {
|
||||
|
||||
public static final int LENGTH = 28;
|
||||
public static final int LENGTH = 20;
|
||||
|
||||
/**
|
||||
* Create an ObjectId from a ByteBuffer.
|
||||
|
|
|
@ -11,7 +11,7 @@ import java.util.Random;
|
|||
*/
|
||||
public class UniqueId extends BaseId implements Serializable {
|
||||
|
||||
public static final int LENGTH = 28;
|
||||
public static final int LENGTH = 20;
|
||||
public static final UniqueId NIL = genNil();
|
||||
|
||||
/**
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package io.ray.runtime;
|
||||
|
||||
import io.ray.api.id.UniqueId;
|
||||
import io.ray.runtime.util.IdUtil;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import javax.xml.bind.DatatypeConverter;
|
||||
|
@ -12,12 +13,12 @@ public class UniqueIdTest {
|
|||
@Test
|
||||
public void testConstructUniqueId() {
|
||||
// Test `fromHexString()`
|
||||
UniqueId id1 = UniqueId.fromHexString("00000000123456789ABCDEF123456789ABCDEF0123456789ABCDEF00");
|
||||
Assert.assertEquals("00000000123456789abcdef123456789abcdef0123456789abcdef00", id1.toString());
|
||||
UniqueId id1 = UniqueId.fromHexString("00000000123456789ABCDEF123456789ABCDEF00");
|
||||
Assert.assertEquals("00000000123456789abcdef123456789abcdef00", id1.toString());
|
||||
Assert.assertFalse(id1.isNil());
|
||||
|
||||
try {
|
||||
UniqueId id2 = UniqueId.fromHexString("000000123456789ABCDEF123456789ABCDEF0123456789ABCDEF00");
|
||||
UniqueId id2 = UniqueId.fromHexString("000000123456789ABCDEF123456789ABCDEF00");
|
||||
// This shouldn't be happened.
|
||||
Assert.assertTrue(false);
|
||||
} catch (IllegalArgumentException e) {
|
||||
|
@ -33,16 +34,23 @@ public class UniqueIdTest {
|
|||
}
|
||||
|
||||
// Test `fromByteBuffer()`
|
||||
byte[] bytes = DatatypeConverter.parseHexBinary("0123456789ABCDEF0123456789ABCDEF012345670123456789ABCDEF");
|
||||
ByteBuffer byteBuffer = ByteBuffer.wrap(bytes, 0, 28);
|
||||
byte[] bytes = DatatypeConverter.parseHexBinary("0123456789ABCDEF0123456789ABCDEF01234567");
|
||||
ByteBuffer byteBuffer = ByteBuffer.wrap(bytes, 0, 20);
|
||||
UniqueId id4 = UniqueId.fromByteBuffer(byteBuffer);
|
||||
Assert.assertTrue(Arrays.equals(bytes, id4.getBytes()));
|
||||
Assert.assertEquals("0123456789abcdef0123456789abcdef012345670123456789abcdef", id4.toString());
|
||||
Assert.assertEquals("0123456789abcdef0123456789abcdef01234567", id4.toString());
|
||||
|
||||
|
||||
// Test `genNil()`
|
||||
UniqueId id6 = UniqueId.NIL;
|
||||
Assert.assertEquals("FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF".toLowerCase(), id6.toString());
|
||||
Assert.assertEquals("FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF".toLowerCase(), id6.toString());
|
||||
Assert.assertTrue(id6.isNil());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testMurmurHash() {
|
||||
UniqueId id = UniqueId.fromHexString("3131313131313131313132323232323232323232");
|
||||
long remainder = Long.remainderUnsigned(IdUtil.murmurHashCode(id), 1000000000);
|
||||
Assert.assertEquals(remainder, 787616861);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -142,8 +142,7 @@ class WorkerCrashedError(RayError):
|
|||
"""Indicates that the worker died unexpectedly while executing a task."""
|
||||
|
||||
def __str__(self):
|
||||
return ("The worker died unexpectedly while executing this task. "
|
||||
"Check python-core-worker-*.log files for more information.")
|
||||
return "The worker died unexpectedly while executing this task."
|
||||
|
||||
|
||||
class RayActorError(RayError):
|
||||
|
@ -154,8 +153,7 @@ class RayActorError(RayError):
|
|||
"""
|
||||
|
||||
def __str__(self):
|
||||
return ("The actor died unexpectedly before finishing this task. "
|
||||
"Check python-core-worker-*.log files for more information.")
|
||||
return "The actor died unexpectedly before finishing this task."
|
||||
|
||||
|
||||
class RaySystemError(RayError):
|
||||
|
|
|
@ -12,7 +12,6 @@ import hashlib
|
|||
import cython
|
||||
import inspect
|
||||
import uuid
|
||||
import ray.ray_constants as ray_constants
|
||||
|
||||
|
||||
ctypedef object (*FunctionDescriptor_from_cpp)(const CFunctionDescriptor &)
|
||||
|
@ -189,8 +188,7 @@ cdef class PythonFunctionDescriptor(FunctionDescriptor):
|
|||
function_name = function.__name__
|
||||
class_name = ""
|
||||
|
||||
pickled_function_hash = hashlib.shake_128(pickled_function).hexdigest(
|
||||
ray_constants.ID_SIZE)
|
||||
pickled_function_hash = hashlib.sha1(pickled_function).hexdigest()
|
||||
|
||||
return cls(module_name, function_name, class_name,
|
||||
pickled_function_hash)
|
||||
|
@ -210,10 +208,7 @@ cdef class PythonFunctionDescriptor(FunctionDescriptor):
|
|||
module_name = target_class.__module__
|
||||
class_name = target_class.__name__
|
||||
# Use a random uuid as function hash to solve actor name conflict.
|
||||
return cls(
|
||||
module_name, "__init__", class_name,
|
||||
hashlib.shake_128(
|
||||
uuid.uuid4().bytes).hexdigest(ray_constants.ID_SIZE))
|
||||
return cls(module_name, "__init__", class_name, str(uuid.uuid4()))
|
||||
|
||||
@property
|
||||
def module_name(self):
|
||||
|
@ -273,14 +268,14 @@ cdef class PythonFunctionDescriptor(FunctionDescriptor):
|
|||
Returns:
|
||||
ray.ObjectRef to represent the function descriptor.
|
||||
"""
|
||||
function_id_hash = hashlib.shake_128()
|
||||
function_id_hash = hashlib.sha1()
|
||||
# Include the function module and name in the hash.
|
||||
function_id_hash.update(self.typed_descriptor.ModuleName())
|
||||
function_id_hash.update(self.typed_descriptor.FunctionName())
|
||||
function_id_hash.update(self.typed_descriptor.ClassName())
|
||||
function_id_hash.update(self.typed_descriptor.FunctionHash())
|
||||
# Compute the function ID.
|
||||
function_id = function_id_hash.digest(ray_constants.ID_SIZE)
|
||||
function_id = function_id_hash.digest()
|
||||
return ray.FunctionID(function_id)
|
||||
|
||||
def is_actor_method(self):
|
||||
|
|
|
@ -31,7 +31,7 @@ def check_id(b, size=kUniqueIDSize):
|
|||
raise TypeError("Unsupported type: " + str(type(b)))
|
||||
if len(b) != size:
|
||||
raise ValueError("ID string needs to have length " +
|
||||
str(size) + ", got " + str(len(b)))
|
||||
str(size))
|
||||
|
||||
|
||||
cdef extern from "ray/common/constants.h" nogil:
|
||||
|
|
|
@ -22,7 +22,7 @@ from ray.ray_logging import setup_component_logger
|
|||
logger = logging.getLogger(__name__)
|
||||
|
||||
# The groups are worker id, job id, and pid.
|
||||
JOB_LOG_PATTERN = re.compile(".*worker-([0-9a-f]+)-(\d+)-(\d+)")
|
||||
JOB_LOG_PATTERN = re.compile(".*worker-([0-9a-f]{40})-(\d+)-(\d+)")
|
||||
|
||||
|
||||
class LogFileInfo:
|
||||
|
|
|
@ -19,7 +19,7 @@ def env_bool(key, default):
|
|||
return default
|
||||
|
||||
|
||||
ID_SIZE = 28
|
||||
ID_SIZE = 20
|
||||
|
||||
# The default maximum number of bytes to allocate to the object store unless
|
||||
# overridden by the user.
|
||||
|
|
|
@ -74,8 +74,7 @@ def _try_to_compute_deterministic_class_id(cls, depth=5):
|
|||
new_class_id = pickle.dumps(pickle.loads(class_id))
|
||||
if new_class_id == class_id:
|
||||
# We appear to have reached a fix point, so use this as the ID.
|
||||
return hashlib.shake_128(new_class_id).digest(
|
||||
ray_constants.ID_SIZE)
|
||||
return hashlib.sha1(new_class_id).digest()
|
||||
class_id = new_class_id
|
||||
|
||||
# We have not reached a fixed point, so we may end up with a different
|
||||
|
@ -83,7 +82,7 @@ def _try_to_compute_deterministic_class_id(cls, depth=5):
|
|||
# same class definition being exported many many times.
|
||||
logger.warning(
|
||||
f"WARNING: Could not produce a deterministic class ID for class {cls}")
|
||||
return hashlib.shake_128(new_class_id).digest(ray_constants.ID_SIZE)
|
||||
return hashlib.sha1(new_class_id).digest()
|
||||
|
||||
|
||||
def object_ref_deserializer(reduced_obj_ref, owner_address):
|
||||
|
|
|
@ -284,14 +284,14 @@ def test_workers(shutdown_only):
|
|||
|
||||
|
||||
def test_object_ref_properties():
|
||||
id_bytes = b"0011223344556677889900001111"
|
||||
id_bytes = b"00112233445566778899"
|
||||
object_ref = ray.ObjectRef(id_bytes)
|
||||
assert object_ref.binary() == id_bytes
|
||||
object_ref = ray.ObjectRef.nil()
|
||||
assert object_ref.is_nil()
|
||||
with pytest.raises(ValueError, match=r".*needs to have length.*"):
|
||||
with pytest.raises(ValueError, match=r".*needs to have length 20.*"):
|
||||
ray.ObjectRef(id_bytes + b"1234")
|
||||
with pytest.raises(ValueError, match=r".*needs to have length.*"):
|
||||
with pytest.raises(ValueError, match=r".*needs to have length 20.*"):
|
||||
ray.ObjectRef(b"0123456789")
|
||||
object_ref = ray.ObjectRef.from_random()
|
||||
assert not object_ref.is_nil()
|
||||
|
|
|
@ -741,10 +741,10 @@ ray.get(main_wait.release.remote())
|
|||
driver1_out_split = driver1_out.split("\n")
|
||||
driver2_out_split = driver2_out.split("\n")
|
||||
|
||||
assert driver1_out_split[0][-1] == "1", driver1_out_split
|
||||
assert driver1_out_split[1][-1] == "2", driver1_out_split
|
||||
assert driver2_out_split[0][-1] == "3", driver2_out_split
|
||||
assert driver2_out_split[1][-1] == "4", driver2_out_split
|
||||
assert driver1_out_split[0][-1] == "1"
|
||||
assert driver1_out_split[1][-1] == "2"
|
||||
assert driver2_out_split[0][-1] == "3"
|
||||
assert driver2_out_split[1][-1] == "4"
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
|
|
|
@ -50,9 +50,9 @@ def get_ray_temp_dir():
|
|||
|
||||
|
||||
def _random_string():
|
||||
id_hash = hashlib.shake_128()
|
||||
id_hash = hashlib.sha1()
|
||||
id_hash.update(uuid.uuid4().bytes)
|
||||
id_bytes = id_hash.digest(ray_constants.ID_SIZE)
|
||||
id_bytes = id_hash.digest()
|
||||
assert len(id_bytes) == ray_constants.ID_SIZE
|
||||
return id_bytes
|
||||
|
||||
|
|
|
@ -345,8 +345,7 @@ class Worker:
|
|||
# actually run the function locally.
|
||||
pickled_function = pickle.dumps(function)
|
||||
|
||||
function_to_run_id = hashlib.shake_128(pickled_function).digest(
|
||||
ray_constants.ID_SIZE)
|
||||
function_to_run_id = hashlib.sha1(pickled_function).digest()
|
||||
key = b"FunctionsToRun:" + function_to_run_id
|
||||
# First run the function on the driver.
|
||||
# We always run the task locally.
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
#include <stdint.h>
|
||||
|
||||
/// Length of Ray full-length IDs in bytes.
|
||||
constexpr size_t kUniqueIDSize = 28;
|
||||
constexpr size_t kUniqueIDSize = 20;
|
||||
|
||||
/// An ObjectID's bytes are split into the task ID itself and the index of the
|
||||
/// object's creation. This is the maximum width of the object index in bits.
|
||||
|
|
|
@ -124,7 +124,7 @@ class JobID : public BaseID<JobID> {
|
|||
|
||||
class ActorID : public BaseID<ActorID> {
|
||||
private:
|
||||
static constexpr size_t kUniqueBytesLength = 12;
|
||||
static constexpr size_t kUniqueBytesLength = 4;
|
||||
|
||||
public:
|
||||
/// Length of `ActorID` in bytes.
|
||||
|
|
|
@ -91,8 +91,6 @@ bool ActorManager::AddActorHandle(std::unique_ptr<ActorHandle> actor_handle,
|
|||
std::placeholders::_1, std::placeholders::_2);
|
||||
RAY_CHECK_OK(gcs_client_->Actors().AsyncSubscribe(
|
||||
actor_id, actor_notification_callback, nullptr));
|
||||
} else {
|
||||
RAY_LOG(ERROR) << "Actor handle already exists " << actor_id.Hex();
|
||||
}
|
||||
|
||||
return inserted;
|
||||
|
|
Loading…
Add table
Reference in a new issue