Skip to content

Commit 23aaa05

Browse files
author
davik
committed
Address PR comments to change dependencies to ref count interface when applicable
This commit addresses the PR comments to change dependencies from reference_counter to reference_counter_interface when possible to speed up build times. As a result Some tests are modified to reflect the new change in dependecies as they depend on the actual ReferenceCounter class. Signed-off-by: davik <davik@anyscale.com>
1 parent 8430535 commit 23aaa05

File tree

11 files changed

+21
-16
lines changed

11 files changed

+21
-16
lines changed

src/ray/core_worker/BUILD.bazel

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -181,7 +181,7 @@ ray_cc_library(
181181
":actor_handle",
182182
":common",
183183
":core_worker_context",
184-
":reference_counter",
184+
":reference_counter_interface",
185185
"//src/ray/common:id",
186186
"//src/ray/common:protobuf_utils",
187187
"//src/ray/common:task_common",
@@ -273,7 +273,7 @@ ray_cc_library(
273273
hdrs = ["store_provider/memory_store/memory_store.h"],
274274
deps = [
275275
":core_worker_context",
276-
":reference_counter",
276+
":reference_counter_interface",
277277
"//src/ray/common:asio",
278278
"//src/ray/common:id",
279279
"//src/ray/common:ray_config",
@@ -382,7 +382,7 @@ ray_cc_library(
382382
hdrs = ["object_recovery_manager.h"],
383383
deps = [
384384
":memory_store",
385-
":reference_counter",
385+
":reference_counter_interface",
386386
":task_manager",
387387
"//src/ray/common:id",
388388
"//src/ray/raylet_rpc_client:raylet_client_pool",
@@ -409,7 +409,7 @@ ray_cc_library(
409409
deps = [
410410
":common",
411411
":core_worker_context",
412-
":reference_counter",
412+
":reference_counter_interface",
413413
"//src/ray/common:buffer",
414414
"//src/ray/common:id",
415415
"//src/ray/common:ray_config",

src/ray/core_worker/actor_manager.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@
2424
#include "absl/container/flat_hash_map.h"
2525
#include "ray/core_worker/actor_creator.h"
2626
#include "ray/core_worker/actor_handle.h"
27-
#include "ray/core_worker/reference_counter.h"
27+
#include "ray/core_worker/reference_counter_interface.h"
2828
#include "ray/core_worker/task_submission/actor_task_submitter.h"
2929
#include "ray/gcs_rpc_client/gcs_client.h"
3030
namespace ray {

src/ray/core_worker/object_recovery_manager.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@
2222
#include "absl/base/thread_annotations.h"
2323
#include "absl/synchronization/mutex.h"
2424
#include "ray/common/id.h"
25-
#include "ray/core_worker/reference_counter.h"
25+
#include "ray/core_worker/reference_counter_interface.h"
2626
#include "ray/core_worker/store_provider/memory_store/memory_store.h"
2727
#include "ray/core_worker/task_manager.h"
2828
#include "ray/raylet_rpc_client/raylet_client_interface.h"

src/ray/core_worker/reference_counter_interface.h

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@ class ReferenceCounterInterface {
3434
public:
3535
using ReferenceTableProto =
3636
::google::protobuf::RepeatedPtrField<rpc::ObjectReferenceCount>;
37+
// Returns the amount of lineage in bytes released.
3738
using LineageReleasedCallback =
3839
std::function<int64_t(const ObjectID &, std::vector<ObjectID> *)>;
3940

src/ray/core_worker/store_provider/memory_store/memory_store.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@
2626
#include "ray/common/id.h"
2727
#include "ray/common/status.h"
2828
#include "ray/core_worker/context.h"
29-
#include "ray/core_worker/reference_counter.h"
29+
#include "ray/core_worker/reference_counter_interface.h"
3030
#include "ray/raylet_ipc_client/raylet_ipc_client_interface.h"
3131
#include "ray/rpc/utils.h"
3232

src/ray/core_worker/store_provider/plasma_store_provider.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@
2626
#include "ray/common/status.h"
2727
#include "ray/common/status_or.h"
2828
#include "ray/core_worker/context.h"
29-
#include "ray/core_worker/reference_counter.h"
29+
#include "ray/core_worker/reference_counter_interface.h"
3030
#include "ray/object_manager/plasma/client.h"
3131
#include "ray/raylet_ipc_client/raylet_ipc_client_interface.h"
3232
#include "src/ray/protobuf/common.pb.h"

src/ray/core_worker/tests/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -51,6 +51,7 @@ ray_cc_test(
5151
"//src/ray/common:asio",
5252
"//src/ray/common:ray_object",
5353
"//src/ray/core_worker:memory_store",
54+
"//src/ray/core_worker:reference_counter",
5455
"//src/ray/core_worker_rpc_client:fake_core_worker_client",
5556
"//src/ray/pubsub:fake_subscriber",
5657
"//src/ray/pubsub:publisher",
@@ -73,6 +74,7 @@ ray_cc_test(
7374
"//src/ray/common:test_utils",
7475
"//src/ray/core_worker:memory_store",
7576
"//src/ray/core_worker:object_recovery_manager",
77+
"//src/ray/core_worker:reference_counter",
7678
"//src/ray/object_manager:object_manager_common",
7779
"//src/ray/pubsub:fake_subscriber",
7880
"//src/ray/raylet_rpc_client:fake_raylet_client",

src/ray/core_worker/tests/core_worker_test.cc

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -286,7 +286,7 @@ class CoreWorkerTest : public ::testing::Test {
286286
boost::thread io_thread_;
287287

288288
rpc::Address rpc_address_;
289-
std::shared_ptr<ReferenceCounter> reference_counter_;
289+
std::shared_ptr<ReferenceCounterInterface> reference_counter_;
290290
std::shared_ptr<CoreWorkerMemoryStore> memory_store_;
291291
ActorTaskSubmitter *actor_task_submitter_;
292292
pubsub::Publisher *object_info_publisher_;
@@ -537,9 +537,10 @@ TEST_F(CoreWorkerTest, HandleGetObjectStatusObjectOutOfScope) {
537537

538538
namespace {
539539

540-
ObjectID CreateInlineObjectInMemoryStoreAndRefCounter(CoreWorkerMemoryStore &memory_store,
541-
ReferenceCounter &reference_counter,
542-
rpc::Address &rpc_address) {
540+
ObjectID CreateInlineObjectInMemoryStoreAndRefCounter(
541+
CoreWorkerMemoryStore &memory_store,
542+
ReferenceCounterInterface &reference_counter,
543+
rpc::Address &rpc_address) {
543544
auto inlined_dependency_id = ObjectID::FromRandom();
544545
std::string data = "hello";
545546
auto data_ptr = const_cast<uint8_t *>(reinterpret_cast<const uint8_t *>(data.data()));

src/ray/core_worker/tests/object_recovery_manager_test.cc

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
#include "mock/ray/core_worker/task_manager_interface.h"
2626
#include "mock/ray/pubsub/publisher.h"
2727
#include "ray/common/test_utils.h"
28+
#include "ray/core_worker/reference_counter.h"
2829
#include "ray/core_worker/store_provider/memory_store/memory_store.h"
2930
#include "ray/pubsub/fake_subscriber.h"
3031
#include "ray/raylet_rpc_client/fake_raylet_client.h"
@@ -181,7 +182,7 @@ class ObjectRecoveryManagerTestBase : public ::testing::Test {
181182
std::shared_ptr<rpc::RayletClientPool> raylet_client_pool_;
182183
std::shared_ptr<MockRayletClient> raylet_client_;
183184
std::shared_ptr<MockTaskManager> task_manager_;
184-
std::shared_ptr<ReferenceCounter> ref_counter_;
185+
std::shared_ptr<ReferenceCounterInterface> ref_counter_;
185186
ObjectRecoveryManager manager_;
186187
};
187188

src/ray/core_worker/tests/reference_counter_test.cc

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,7 @@ static const ReferenceCounterInterface::ReferenceTableProto empty_refs;
4141

4242
class ReferenceCountTest : public ::testing::Test {
4343
protected:
44-
std::unique_ptr<ReferenceCounter> rc;
44+
std::unique_ptr<ReferenceCounterInterface> rc;
4545
virtual void SetUp() {
4646
rpc::Address addr;
4747
publisher_ = std::make_shared<pubsub::MockPublisher>();
@@ -67,7 +67,7 @@ class ReferenceCountTest : public ::testing::Test {
6767

6868
class ReferenceCountLineageEnabledTest : public ::testing::Test {
6969
protected:
70-
std::unique_ptr<ReferenceCounter> rc;
70+
std::unique_ptr<ReferenceCounterInterface> rc;
7171
virtual void SetUp() {
7272
rpc::Address addr;
7373
publisher_ = std::make_shared<pubsub::MockPublisher>();

0 commit comments

Comments
 (0)