Revert "Increase the number of unique bits for actors to avoid handle collisions (#12894)" (#12988)

This reverts commit 3e492a79ec.
This commit is contained in:
Eric Liang 2020-12-18 23:51:44 -08:00 committed by GitHub
parent a092433bc8
commit 5d987f5988
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
19 changed files with 54 additions and 54 deletions

View file

@ -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):

View file

@ -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

View file

@ -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;

View file

@ -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.

View file

@ -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();
/**

View file

@ -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);
}
}

View file

@ -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):

View file

@ -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):

View file

@ -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:

View file

@ -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:

View file

@ -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.

View file

@ -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):

View file

@ -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()

View file

@ -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__":

View file

@ -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

View file

@ -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.

View file

@ -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.

View file

@ -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.

View file

@ -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;