mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
[XRay] Pass in node IP address to Raylet (#1808)
This commit is contained in:
parent
0bda11e009
commit
71829a2af9
6 changed files with 30 additions and 19 deletions
|
@ -5,12 +5,13 @@
|
|||
|
||||
#ifndef RAYLET_TEST
|
||||
int main(int argc, char *argv[]) {
|
||||
RAY_CHECK(argc == 5);
|
||||
RAY_CHECK(argc == 6);
|
||||
|
||||
const std::string raylet_socket_name = std::string(argv[1]);
|
||||
const std::string store_socket_name = std::string(argv[2]);
|
||||
const std::string redis_address = std::string(argv[3]);
|
||||
int redis_port = std::stoi(argv[4]);
|
||||
const std::string node_ip_address = std::string(argv[3]);
|
||||
const std::string redis_address = std::string(argv[4]);
|
||||
int redis_port = std::stoi(argv[5]);
|
||||
|
||||
// Configuration for the node manager.
|
||||
ray::raylet::NodeManagerConfig node_manager_config;
|
||||
|
@ -42,8 +43,9 @@ int main(int argc, char *argv[]) {
|
|||
std::unique_ptr<boost::asio::io_service> object_manager_service;
|
||||
object_manager_service.reset(new boost::asio::io_service());
|
||||
ray::raylet::Raylet server(main_service, std::move(object_manager_service),
|
||||
raylet_socket_name, redis_address, redis_port,
|
||||
node_manager_config, object_manager_config, gcs_client);
|
||||
raylet_socket_name, node_ip_address, redis_address,
|
||||
redis_port, node_manager_config, object_manager_config,
|
||||
gcs_client);
|
||||
main_service.run();
|
||||
}
|
||||
#endif
|
||||
|
|
|
@ -61,16 +61,18 @@ class TestObjectManagerBase : public ::testing::Test {
|
|||
ObjectManagerConfig om_config_1;
|
||||
om_config_1.store_socket_name = store_sock_1;
|
||||
server1.reset(new ray::raylet::Raylet(
|
||||
main_service, std::move(object_manager_service_1), "raylet_1", "127.0.0.1", 6379,
|
||||
GetNodeManagerConfig("raylet_1", store_sock_1), om_config_1, gcs_client_1));
|
||||
main_service, std::move(object_manager_service_1), "raylet_1", "0.0.0.0",
|
||||
"127.0.0.1", 6379, GetNodeManagerConfig("raylet_1", store_sock_1), om_config_1,
|
||||
gcs_client_1));
|
||||
|
||||
// start second server
|
||||
gcs_client_2 = std::shared_ptr<gcs::AsyncGcsClient>(new gcs::AsyncGcsClient());
|
||||
ObjectManagerConfig om_config_2;
|
||||
om_config_2.store_socket_name = store_sock_2;
|
||||
server2.reset(new ray::raylet::Raylet(
|
||||
main_service, std::move(object_manager_service_2), "raylet_2", "127.0.0.1", 6379,
|
||||
GetNodeManagerConfig("raylet_2", store_sock_2), om_config_2, gcs_client_2));
|
||||
main_service, std::move(object_manager_service_2), "raylet_2", "0.0.0.0",
|
||||
"127.0.0.1", 6379, GetNodeManagerConfig("raylet_2", store_sock_2), om_config_2,
|
||||
gcs_client_2));
|
||||
|
||||
// connect to stores.
|
||||
ARROW_CHECK_OK(client1.Connect(store_sock_1, "", PLASMA_DEFAULT_RELEASE_DELAY));
|
||||
|
|
|
@ -13,8 +13,9 @@ namespace raylet {
|
|||
|
||||
Raylet::Raylet(boost::asio::io_service &main_service,
|
||||
std::unique_ptr<boost::asio::io_service> object_manager_service,
|
||||
const std::string &socket_name, const std::string &redis_address,
|
||||
int redis_port, const NodeManagerConfig &node_manager_config,
|
||||
const std::string &socket_name, const std::string &node_ip_address,
|
||||
const std::string &redis_address, int redis_port,
|
||||
const NodeManagerConfig &node_manager_config,
|
||||
const ObjectManagerConfig &object_manager_config,
|
||||
std::shared_ptr<gcs::AsyncGcsClient> gcs_client)
|
||||
: acceptor_(main_service, boost::asio::local::stream_protocol::endpoint(socket_name)),
|
||||
|
@ -34,7 +35,8 @@ Raylet::Raylet(boost::asio::io_service &main_service,
|
|||
DoAcceptObjectManager();
|
||||
DoAcceptNodeManager();
|
||||
|
||||
RAY_CHECK_OK(RegisterGcs(redis_address, redis_port, main_service, node_manager_config));
|
||||
RAY_CHECK_OK(RegisterGcs(node_ip_address, redis_address, redis_port, main_service,
|
||||
node_manager_config));
|
||||
|
||||
RAY_CHECK_OK(RegisterPeriodicTimer(main_service));
|
||||
}
|
||||
|
@ -50,15 +52,15 @@ ray::Status Raylet::RegisterPeriodicTimer(boost::asio::io_service &io_service) {
|
|||
return ray::Status::OK();
|
||||
}
|
||||
|
||||
ray::Status Raylet::RegisterGcs(const std::string &redis_address, int redis_port,
|
||||
ray::Status Raylet::RegisterGcs(const std::string &node_ip_address,
|
||||
const std::string &redis_address, int redis_port,
|
||||
boost::asio::io_service &io_service,
|
||||
const NodeManagerConfig &node_manager_config) {
|
||||
RAY_RETURN_NOT_OK(gcs_client_->Connect(redis_address, redis_port));
|
||||
RAY_RETURN_NOT_OK(gcs_client_->Attach(io_service));
|
||||
|
||||
ClientTableDataT client_info = gcs_client_->client_table().GetLocalClient();
|
||||
client_info.node_manager_address =
|
||||
node_manager_acceptor_.local_endpoint().address().to_string();
|
||||
client_info.node_manager_address = node_ip_address;
|
||||
client_info.object_manager_port = object_manager_acceptor_.local_endpoint().port();
|
||||
client_info.node_manager_port = node_manager_acceptor_.local_endpoint().port();
|
||||
// Add resource information.
|
||||
|
|
|
@ -26,6 +26,9 @@ class Raylet {
|
|||
/// \param main_service The event loop to run the server on.
|
||||
/// \param object_manager_service The asio io_service tied to the object manager.
|
||||
/// \param socket_name The Unix domain socket to listen on for local clients.
|
||||
/// \param node_ip_address The IP address of this node.
|
||||
/// \param redis_address The IP address of the redis instance we are connecting to.
|
||||
/// \param redis_port The port of the redis instance we are connecting to.
|
||||
/// \param node_manager_config Configuration to initialize the node manager.
|
||||
/// scheduler with.
|
||||
/// \param object_manager_config Configuration to initialize the object
|
||||
|
@ -33,7 +36,8 @@ class Raylet {
|
|||
/// \param gcs_client A client connection to the GCS.
|
||||
Raylet(boost::asio::io_service &main_service,
|
||||
std::unique_ptr<boost::asio::io_service> object_manager_service,
|
||||
const std::string &socket_name, const std::string &redis_address, int redis_port,
|
||||
const std::string &socket_name, const std::string &node_ip_address,
|
||||
const std::string &redis_address, int redis_port,
|
||||
const NodeManagerConfig &node_manager_config,
|
||||
const ObjectManagerConfig &object_manager_config,
|
||||
std::shared_ptr<gcs::AsyncGcsClient> gcs_client);
|
||||
|
@ -43,7 +47,8 @@ class Raylet {
|
|||
|
||||
private:
|
||||
/// Register GCS client.
|
||||
ray::Status RegisterGcs(const std::string &redis_address, int redis_port,
|
||||
ray::Status RegisterGcs(const std::string &node_ip_address,
|
||||
const std::string &redis_address, int redis_port,
|
||||
boost::asio::io_service &io_service, const NodeManagerConfig &);
|
||||
|
||||
ray::Status RegisterPeriodicTimer(boost::asio::io_service &io_service);
|
||||
|
|
|
@ -22,7 +22,7 @@ if [[ `stat $STORE_SOCKET_NAME` ]]; then
|
|||
fi
|
||||
|
||||
./src/plasma/plasma_store -m 1000000000 -s $STORE_SOCKET_NAME &
|
||||
./src/ray/raylet/raylet $RAYLET_SOCKET_NAME $STORE_SOCKET_NAME 127.0.0.1 6379 &
|
||||
./src/ray/raylet/raylet $RAYLET_SOCKET_NAME $STORE_SOCKET_NAME 127.0.0.1 127.0.0.1 6379 &
|
||||
|
||||
echo
|
||||
echo "WORKER COMMAND: python ../../../src/ray/python/worker.py $RAYLET_SOCKET_NAME $STORE_SOCKET_NAME"
|
||||
|
|
|
@ -28,7 +28,7 @@ for i in `seq 1 $NUM_RAYLETS`; do
|
|||
fi
|
||||
|
||||
./src/plasma/plasma_store -m 1000000000 -s $STORE_SOCKET_NAME &
|
||||
./src/ray/raylet/raylet $RAYLET_SOCKET_NAME $STORE_SOCKET_NAME 127.0.0.1 6379 &
|
||||
./src/ray/raylet/raylet $RAYLET_SOCKET_NAME $STORE_SOCKET_NAME 127.0.0.1 127.0.0.1 6379 &
|
||||
|
||||
echo
|
||||
echo "WORKER COMMAND: python ../../../src/ray/python/worker.py $RAYLET_SOCKET_NAME $STORE_SOCKET_NAME"
|
||||
|
|
Loading…
Add table
Reference in a new issue