Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
30 changes: 15 additions & 15 deletions BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -281,16 +281,13 @@ cc_library(
name = "ray_util",
srcs = glob(
[
"src/ray/*.cc",
"src/ray/util/*.cc",
],
exclude = [
"src/ray/util/logging_test.cc",
"src/ray/util/signal_test.cc",
"src/ray/util/*_test.cc",
],
),
hdrs = glob([
"src/ray/*.h",
"src/ray/util/*.h",
]),
copts = COPTS,
Expand All @@ -306,22 +303,25 @@ cc_library(

cc_library(
name = "ray_common",
srcs = [
"src/ray/common/client_connection.cc",
"src/ray/common/common_protocol.cc",
],
hdrs = [
"src/ray/common/buffer.h",
"src/ray/common/client_connection.h",
"src/ray/common/common_protocol.h",
],
srcs = glob(
[
"src/ray/common/*.cc",
],
exclude = [
"src/ray/common/*_test.cc",
],
),
hdrs = glob(
[
"src/ray/common/*.h",
],
),
copts = COPTS,
includes = [
"src/ray/gcs/format",
],
deps = [
":gcs_fbs",
":node_manager_fbs",
":ray_util",
"@boost//:asio",
"@plasma//:plasma_client",
Expand Down Expand Up @@ -468,7 +468,7 @@ cc_binary(
srcs = [
"src/ray/raylet/lib/java/org_ray_runtime_raylet_RayletClientImpl.h",
"src/ray/raylet/lib/java/org_ray_runtime_raylet_RayletClientImpl.cc",
"src/ray/id.h",
"src/ray/common/id.h",
"src/ray/raylet/raylet_client.h",
"src/ray/util/logging.h",
"@bazel_tools//tools/jdk:jni_header",
Expand Down
4 changes: 2 additions & 2 deletions java/runtime/src/main/java/org/ray/runtime/util/IdUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@
/**
* Helper method for different Ids.
* Note: any changes to these methods must be synced with C++ helper functions
* in src/ray/id.h
* in src/ray/common/id.h
*/
public class IdUtil {
public static final int OBJECT_INDEX_POS = 16;
Expand Down Expand Up @@ -161,7 +161,7 @@ public static long murmurHashCode(BaseId id) {
}

/**
* This method is the same as `Hash()` method of `ID` class in ray/src/ray/id.h
* This method is the same as `Hash()` method of `ID` class in ray/src/ray/common/id.h
*/
private static long murmurHash64A(byte[] data, int length, int seed) {
final long m = 0xc6a4a7935bd1e995L;
Expand Down
6 changes: 3 additions & 3 deletions python/ray/includes/common.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@ from ray.includes.unique_ids cimport (
)


cdef extern from "ray/status.h" namespace "ray" nogil:
cdef extern from "ray/common/status.h" namespace "ray" nogil:
cdef cppclass StatusCode:
pass

Expand Down Expand Up @@ -68,7 +68,7 @@ cdef extern from "ray/status.h" namespace "ray" nogil:
cdef CRayStatus RayStatus_Invalid "Status::Invalid"()


cdef extern from "ray/status.h" namespace "ray::StatusCode" nogil:
cdef extern from "ray/common/status.h" namespace "ray::StatusCode" nogil:
cdef StatusCode StatusCode_OK "OK"
cdef StatusCode StatusCode_OutOfMemory "OutOfMemory"
cdef StatusCode StatusCode_KeyError "KeyError"
Expand All @@ -80,7 +80,7 @@ cdef extern from "ray/status.h" namespace "ray::StatusCode" nogil:
cdef StatusCode StatusCode_RedisError "RedisError"


cdef extern from "ray/id.h" namespace "ray" nogil:
cdef extern from "ray/common/id.h" namespace "ray" nogil:
const CTaskID GenerateTaskId(const CDriverID &driver_id,
const CTaskID &parent_task_id,
int parent_task_counter)
Expand Down
2 changes: 1 addition & 1 deletion python/ray/includes/ray_config.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@ from libcpp.string cimport string as c_string
from libcpp.unordered_map cimport unordered_map


cdef extern from "ray/ray_config.h" nogil:
cdef extern from "ray/common/ray_config.h" nogil:
cdef cppclass RayConfig "RayConfig":
@staticmethod
RayConfig &instance()
Expand Down
4 changes: 2 additions & 2 deletions python/ray/includes/unique_ids.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ from libcpp cimport bool as c_bool
from libcpp.string cimport string as c_string
from libc.stdint cimport uint8_t, int64_t

cdef extern from "ray/id.h" namespace "ray" nogil:
cdef extern from "ray/common/id.h" namespace "ray" nogil:
cdef cppclass CBaseID[T]:
@staticmethod
T from_random()
Expand Down Expand Up @@ -113,7 +113,7 @@ cdef extern from "ray/id.h" namespace "ray" nogil:

c_bool is_put()

int64_t ObjectIndex() const
int64_t ObjectIndex() const

CTaskID TaskId() const

Expand Down
8 changes: 4 additions & 4 deletions python/ray/includes/unique_ids.pxi
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ def check_id(b, size=kUniqueIDSize):
str(size))


cdef extern from "ray/constants.h" nogil:
cdef extern from "ray/common/constants.h" nogil:
cdef int64_t kUniqueIDSize
cdef int64_t kMaxTaskPuts

Expand Down Expand Up @@ -109,7 +109,7 @@ cdef class UniqueID(BaseID):
def nil(cls):
return cls(CUniqueID.Nil().Binary())


@classmethod
def from_random(cls):
return cls(os.urandom(CUniqueID.Size()))
Expand All @@ -122,7 +122,7 @@ cdef class UniqueID(BaseID):

def hex(self):
return decode(self.data.Hex())

def is_nil(self):
return self.data.IsNil()

Expand All @@ -148,7 +148,7 @@ cdef class ObjectID(BaseID):

def hex(self):
return decode(self.data.Hex())

def is_nil(self):
return self.data.IsNil()

Expand Down
3 changes: 1 addition & 2 deletions src/ray/common/client_connection.cc
Original file line number Diff line number Diff line change
Expand Up @@ -4,8 +4,7 @@
#include <boost/bind.hpp>
#include <sstream>

#include "ray/ray_config.h"
#include "ray/raylet/format/node_manager_generated.h"
#include "ray/common/ray_config.h"
#include "ray/util/util.h"

namespace ray {
Expand Down
4 changes: 2 additions & 2 deletions src/ray/common/client_connection.h
Original file line number Diff line number Diff line change
Expand Up @@ -8,8 +8,8 @@
#include <boost/asio/error.hpp>
#include <boost/enable_shared_from_this.hpp>

#include "ray/id.h"
#include "ray/status.h"
#include "ray/common/id.h"
#include "ray/common/status.h"

namespace ray {

Expand Down
2 changes: 1 addition & 1 deletion src/ray/common/common_protocol.h
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@

#include <unordered_map>

#include "ray/id.h"
#include "ray/common/id.h"
#include "ray/util/logging.h"

/// Convert an unique ID to a flatbuffer string.
Expand Down
File renamed without changes.
8 changes: 4 additions & 4 deletions src/ray/id.cc → src/ray/common/id.cc
Original file line number Diff line number Diff line change
@@ -1,16 +1,16 @@
#include "ray/id.h"
#include "ray/common/id.h"

#include <limits.h>

#include <chrono>
#include <mutex>
#include <random>

#include "ray/constants.h"
#include "ray/status.h"
#include "ray/common/constants.h"
#include "ray/common/status.h"

extern "C" {
#include "thirdparty/sha256.h"
#include "ray/thirdparty/sha256.h"
}

// Definitions for computing hash digests.
Expand Down
2 changes: 1 addition & 1 deletion src/ray/id.h → src/ray/common/id.h
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@
#include <string>

#include "plasma/common.h"
#include "ray/constants.h"
#include "ray/common/constants.h"
#include "ray/util/logging.h"
#include "ray/util/visibility.h"

Expand Down
File renamed without changes.
File renamed without changes.
File renamed without changes.
2 changes: 1 addition & 1 deletion src/ray/status.cc → src/ray/common/status.cc
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@

// Adapted from Apache Arrow, Apache Kudu, TensorFlow

#include "ray/status.h"
#include "ray/common/status.h"

#include <assert.h>

Expand Down
File renamed without changes.
2 changes: 1 addition & 1 deletion src/ray/core_worker/common.h
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@
#include <string>

#include "ray/common/buffer.h"
#include "ray/id.h"
#include "ray/common/id.h"

namespace ray {

Expand Down
2 changes: 1 addition & 1 deletion src/ray/core_worker/object_interface.cc
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
#include "object_interface.h"
#include "context.h"
#include "core_worker.h"
#include "ray/ray_config.h"
#include "ray/common/ray_config.h"

namespace ray {

Expand Down
4 changes: 2 additions & 2 deletions src/ray/core_worker/object_interface.h
Original file line number Diff line number Diff line change
Expand Up @@ -4,8 +4,8 @@
#include "common.h"
#include "plasma/client.h"
#include "ray/common/buffer.h"
#include "ray/id.h"
#include "ray/status.h"
#include "ray/common/id.h"
#include "ray/common/status.h"

namespace ray {

Expand Down
2 changes: 1 addition & 1 deletion src/ray/core_worker/task_execution.h
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@

#include "common.h"
#include "ray/common/buffer.h"
#include "ray/status.h"
#include "ray/common/status.h"

namespace ray {

Expand Down
4 changes: 2 additions & 2 deletions src/ray/core_worker/task_interface.h
Original file line number Diff line number Diff line change
Expand Up @@ -3,8 +3,8 @@

#include "common.h"
#include "ray/common/buffer.h"
#include "ray/id.h"
#include "ray/status.h"
#include "ray/common/id.h"
#include "ray/common/status.h"

namespace ray {

Expand Down
2 changes: 1 addition & 1 deletion src/ray/gcs/client.cc
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
#include "ray/gcs/client.h"

#include "ray/common/ray_config.h"
#include "ray/gcs/redis_context.h"
#include "ray/ray_config.h"

static void GetRedisShards(redisContext *context, std::vector<std::string> &addresses,
std::vector<int> &ports) {
Expand Down
4 changes: 2 additions & 2 deletions src/ray/gcs/client.h
Original file line number Diff line number Diff line change
Expand Up @@ -4,10 +4,10 @@
#include <map>
#include <string>

#include "ray/common/id.h"
#include "ray/common/status.h"
#include "ray/gcs/asio.h"
#include "ray/gcs/tables.h"
#include "ray/id.h"
#include "ray/status.h"
#include "ray/util/logging.h"

namespace ray {
Expand Down
2 changes: 1 addition & 1 deletion src/ray/gcs/client_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -5,9 +5,9 @@ extern "C" {
#include "ray/thirdparty/hiredis/hiredis.h"
}

#include "ray/common/ray_config.h"
#include "ray/gcs/client.h"
#include "ray/gcs/tables.h"
#include "ray/ray_config.h"

namespace ray {

Expand Down
2 changes: 1 addition & 1 deletion src/ray/gcs/redis_context.cc
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ extern "C" {
}

// TODO(pcm): Integrate into the C++ tree.
#include "ray/ray_config.h"
#include "ray/common/ray_config.h"

namespace {

Expand Down
4 changes: 2 additions & 2 deletions src/ray/gcs/redis_context.h
Original file line number Diff line number Diff line change
Expand Up @@ -5,8 +5,8 @@
#include <memory>
#include <unordered_map>

#include "ray/id.h"
#include "ray/status.h"
#include "ray/common/id.h"
#include "ray/common/status.h"
#include "ray/util/logging.h"

#include "ray/gcs/format/gcs_generated.h"
Expand Down
4 changes: 2 additions & 2 deletions src/ray/gcs/redis_module/ray_redis_module.cc
Original file line number Diff line number Diff line change
Expand Up @@ -2,9 +2,9 @@
#include <sstream>

#include "ray/common/common_protocol.h"
#include "ray/common/id.h"
#include "ray/common/status.h"
#include "ray/gcs/format/gcs_generated.h"
#include "ray/id.h"
#include "ray/status.h"
#include "ray/util/logging.h"
#include "redis_string.h"
#include "redismodule.h"
Expand Down
2 changes: 1 addition & 1 deletion src/ray/gcs/tables.cc
Original file line number Diff line number Diff line change
@@ -1,8 +1,8 @@
#include "ray/gcs/tables.h"

#include "ray/common/common_protocol.h"
#include "ray/common/ray_config.h"
#include "ray/gcs/client.h"
#include "ray/ray_config.h"
#include "ray/util/util.h"

namespace {
Expand Down
6 changes: 3 additions & 3 deletions src/ray/gcs/tables.h
Original file line number Diff line number Diff line change
Expand Up @@ -6,9 +6,9 @@
#include <unordered_map>
#include <unordered_set>

#include "ray/constants.h"
#include "ray/id.h"
#include "ray/status.h"
#include "ray/common/constants.h"
#include "ray/common/id.h"
#include "ray/common/status.h"
#include "ray/util/logging.h"

#include "ray/gcs/format/gcs_generated.h"
Expand Down
4 changes: 2 additions & 2 deletions src/ray/object_manager/connection_pool.h
Original file line number Diff line number Diff line change
Expand Up @@ -12,8 +12,8 @@
#include <boost/asio/error.hpp>
#include <boost/bind.hpp>

#include "ray/id.h"
#include "ray/status.h"
#include "ray/common/id.h"
#include "ray/common/status.h"

#include <mutex>
#include "ray/object_manager/format/object_manager_generated.h"
Expand Down
2 changes: 1 addition & 1 deletion src/ray/object_manager/object_buffer_pool.cc
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
#include "ray/object_manager/object_buffer_pool.h"

#include "ray/status.h"
#include "ray/common/status.h"
#include "ray/util/logging.h"

namespace ray {
Expand Down
Loading