[TEST]use cc_test to run core_worker_test, enforce/reuse RedisServiceManagerForTest (#8443)

This commit is contained in:
Tao Wang 2020-05-17 18:43:00 +08:00 committed by GitHub
parent fb23bd6fc0
commit acffdb2349
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
15 changed files with 85 additions and 109 deletions

View file

@ -85,7 +85,6 @@ matrix:
before_script:
- . ./ci/travis/ci.sh build
script:
- ./ci/suppress_output bash src/ray/test/run_core_worker_tests.sh
- ./ci/suppress_output bash streaming/src/test/run_streaming_queue_test.sh
- ./java/test.sh
@ -280,8 +279,7 @@ matrix:
- . ./ci/travis/ci.sh test_cpp
script:
# raylet integration tests
- ./ci/suppress_output bash src/ray/test/run_core_worker_tests.sh
# raylet integration tests (core_worker_tests included in bazel tests below)
- ./ci/suppress_output bash src/ray/test/run_object_manager_tests.sh
# cc bazel tests (w/o RLlib)

View file

@ -530,13 +530,21 @@ cc_binary(
],
)
cc_library(
name = "core_worker_test_lib",
cc_test(
name = "core_worker_test",
srcs = ["src/ray/core_worker/test/core_worker_test.cc"],
hdrs = glob([
"src/ray/core_worker/test/*.h",
]),
args = ["$(location @plasma//:plasma_store_server) $(location raylet) $(location raylet_monitor) $(location mock_worker) $(location gcs_server) $(location redis-cli) $(location redis-server) $(location libray_redis_module.so)"],
copts = COPTS,
data = [
"//:gcs_server",
"//:libray_redis_module.so",
"//:mock_worker",
"//:raylet",
"//:raylet_monitor",
"//:redis-cli",
"//:redis-server",
"@plasma//:plasma_store_server",
],
deps = [
":core_worker_lib",
":gcs",
@ -546,14 +554,6 @@ cc_library(
],
)
cc_binary(
name = "core_worker_test",
copts = COPTS,
deps = [
":core_worker_test_lib",
],
)
cc_test(
name = "direct_actor_transport_test",
srcs = ["src/ray/core_worker/test/direct_actor_transport_test.cc"],

View file

@ -280,7 +280,7 @@ invoke other test scripts via ``pytest``, ``bazel``-based test or other bash
scripts. Some of the examples include:
* Raylet integration tests commands:
* ``src/ray/test/run_core_worker_tests.sh``
* ``bazel test //:core_worker_test``
* ``src/ray/test/run_object_manager_tests.sh``
* Bazel test command:

View file

@ -23,11 +23,24 @@
namespace ray {
void RedisServiceManagerForTest::SetUpTestCase() {
auto seed = std::chrono::high_resolution_clock::now().time_since_epoch().count();
std::mt19937 gen(seed);
std::uniform_int_distribution<int> random_gen{2000, 7000};
// Use random port to avoid port conflicts between UTs.
REDIS_SERVER_PORT = random_gen(gen);
std::vector<int> actual_redis_server_ports;
if (REDIS_SERVER_PORTS.empty()) {
actual_redis_server_ports.push_back(StartUpRedisServer(0));
} else {
for (const auto &port : REDIS_SERVER_PORTS) {
actual_redis_server_ports.push_back(StartUpRedisServer(port));
}
}
REDIS_SERVER_PORTS = actual_redis_server_ports;
}
// start a redis server with specified port, use random one when 0 given
int RedisServiceManagerForTest::StartUpRedisServer(int port) {
int actual_port = port;
if (port == 0) {
// Use random port (in range [2000, 7000) to avoid port conflicts between UTs.
actual_port = rand() % 5000 + 2000;
}
std::string load_module_command;
if (!REDIS_MODULE_LIBRARY_PATH.empty()) {
@ -37,15 +50,22 @@ void RedisServiceManagerForTest::SetUpTestCase() {
std::string start_redis_command = REDIS_SERVER_EXEC_PATH + " --loglevel warning " +
load_module_command + " --port " +
std::to_string(REDIS_SERVER_PORT) + " &";
std::to_string(actual_port) + " &";
RAY_LOG(INFO) << "Start redis command is: " << start_redis_command;
RAY_CHECK(system(start_redis_command.c_str()) == 0);
usleep(200 * 1000);
return actual_port;
}
void RedisServiceManagerForTest::TearDownTestCase() {
for (const auto &port : REDIS_SERVER_PORTS) {
ShutDownRedisServer(port);
}
}
void RedisServiceManagerForTest::ShutDownRedisServer(int port) {
std::string stop_redis_command =
REDIS_CLIENT_EXEC_PATH + " -p " + std::to_string(REDIS_SERVER_PORT) + " shutdown";
REDIS_CLIENT_EXEC_PATH + " -p " + std::to_string(port) + " shutdown";
RAY_LOG(INFO) << "Stop redis command is: " << stop_redis_command;
if (system(stop_redis_command.c_str()) != 0) {
RAY_LOG(WARNING) << "Failed to stop redis. The redis process may no longer exist.";
@ -54,8 +74,14 @@ void RedisServiceManagerForTest::TearDownTestCase() {
}
void RedisServiceManagerForTest::FlushAll() {
for (const auto &port : REDIS_SERVER_PORTS) {
FlushRedisServer(port);
}
}
void RedisServiceManagerForTest::FlushRedisServer(int port) {
std::string flush_all_redis_command =
REDIS_CLIENT_EXEC_PATH + " -p " + std::to_string(REDIS_SERVER_PORT) + " flushall";
REDIS_CLIENT_EXEC_PATH + " -p " + std::to_string(port) + " flushall";
RAY_LOG(INFO) << "Cleaning up redis with command: " << flush_all_redis_command;
if (system(flush_all_redis_command.c_str()) != 0) {
RAY_LOG(WARNING) << "Failed to flush redis. The redis process may no longer exist.";
@ -109,7 +135,7 @@ std::string REDIS_SERVER_EXEC_PATH;
std::string REDIS_CLIENT_EXEC_PATH;
/// Path to redis module library.
std::string REDIS_MODULE_LIBRARY_PATH;
/// Port of redis server.
int REDIS_SERVER_PORT;
/// Ports of redis server.
std::vector<int> REDIS_SERVER_PORTS;
} // namespace ray

View file

@ -53,16 +53,19 @@ extern std::string REDIS_SERVER_EXEC_PATH;
extern std::string REDIS_CLIENT_EXEC_PATH;
/// Path to redis module library.
extern std::string REDIS_MODULE_LIBRARY_PATH;
/// Port of redis server.
extern int REDIS_SERVER_PORT;
/// Ports of redis server.
extern std::vector<int> REDIS_SERVER_PORTS;
/// Test helper class, it will start redis server before the test runs,
/// and stop redis server after the test is completed.
class RedisServiceManagerForTest : public ::testing::Test {
public:
static void SetUpTestCase();
static int StartUpRedisServer(int port);
static void TearDownTestCase();
static void ShutDownRedisServer(int port);
static void FlushAll();
static void FlushRedisServer(int port);
};
} // namespace ray

View file

@ -89,7 +89,8 @@ std::string MetadataToString(std::shared_ptr<RayObject> obj) {
return std::string(reinterpret_cast<const char *>(metadata->Data()), metadata->Size());
}
class CoreWorkerTest : public ::testing::Test {
// inherit from RedisServiceManagerForTest for setting up redis server(s)
class CoreWorkerTest : public RedisServiceManagerForTest {
public:
CoreWorkerTest(int num_nodes)
: num_nodes_(num_nodes), gcs_options_("127.0.0.1", 6379, "") {
@ -1029,12 +1030,23 @@ TEST_F(TwoNodeTest, TestActorTaskCrossNodesFailure) {
int main(int argc, char **argv) {
::testing::InitGoogleTest(&argc, argv);
RAY_CHECK(argc == 7);
RAY_CHECK(argc == 9);
store_executable = std::string(argv[1]);
raylet_executable = std::string(argv[2]);
node_manager_port = std::stoi(std::string(argv[3]));
raylet_monitor_executable = std::string(argv[4]);
mock_worker_executable = std::string(argv[5]);
gcs_server_executable = std::string(argv[6]);
auto seed = std::chrono::high_resolution_clock::now().time_since_epoch().count();
std::mt19937 gen(seed);
std::uniform_int_distribution<int> random_gen{2000, 2009};
// Use random port to avoid port conflicts between UTs.
node_manager_port = random_gen(gen);
raylet_monitor_executable = std::string(argv[3]);
mock_worker_executable = std::string(argv[4]);
gcs_server_executable = std::string(argv[5]);
ray::REDIS_CLIENT_EXEC_PATH = std::string(argv[6]);
ray::REDIS_SERVER_EXEC_PATH = std::string(argv[7]);
ray::REDIS_MODULE_LIBRARY_PATH = std::string(argv[8]);
ray::REDIS_SERVER_PORTS.push_back(6379);
ray::REDIS_SERVER_PORTS.push_back(6380);
return RUN_ALL_TESTS();
}

View file

@ -30,7 +30,7 @@ class ServiceBasedGcsClientTest : public RedisServiceManagerForTest {
config.grpc_server_thread_num = 1;
config.redis_address = "127.0.0.1";
config.is_test = true;
config.redis_port = REDIS_SERVER_PORT;
config.redis_port = REDIS_SERVER_PORTS.front();
gcs_server_.reset(new gcs::GcsServer(config));
io_service_.reset(new boost::asio::io_service());

View file

@ -29,7 +29,7 @@ class GcsServerTest : public RedisServiceManagerForTest {
config.grpc_server_thread_num = 1;
config.redis_address = "127.0.0.1";
config.is_test = true;
config.redis_port = REDIS_SERVER_PORT;
config.redis_port = REDIS_SERVER_PORTS.front();
gcs_server_.reset(new gcs::GcsServer(config));
thread_io_service_.reset(new std::thread([this] {

View file

@ -27,7 +27,7 @@ class RedisGcsTableStorageTest : public gcs::GcsTableStorageTestBase {
static void TearDownTestCase() { RedisServiceManagerForTest::TearDownTestCase(); }
void SetUp() override {
gcs::RedisClientOptions options("127.0.0.1", REDIS_SERVER_PORT, "", true);
gcs::RedisClientOptions options("127.0.0.1", REDIS_SERVER_PORTS.front(), "", true);
redis_client_ = std::make_shared<gcs::RedisClient>(options);
RAY_CHECK_OK(redis_client_->Connect(io_service_pool_->GetAll()));

View file

@ -29,8 +29,8 @@ class GcsPubSubTest : public RedisServiceManagerForTest {
io_service_.run();
}));
gcs::RedisClientOptions redis_client_options("127.0.0.1", REDIS_SERVER_PORT, "",
true);
gcs::RedisClientOptions redis_client_options("127.0.0.1", REDIS_SERVER_PORTS.front(),
"", true);
client_ = std::make_shared<gcs::RedisClient>(redis_client_options);
RAY_CHECK_OK(client_->Connect(io_service_));
pub_sub_ = std::make_shared<gcs::GcsPubSub>(client_);

View file

@ -31,7 +31,7 @@ class RedisStoreClientTest : public StoreClientTestBase {
static void TearDownTestCase() { RedisServiceManagerForTest::TearDownTestCase(); }
void InitStoreClient() override {
RedisClientOptions options("127.0.0.1", REDIS_SERVER_PORT, "", true);
RedisClientOptions options("127.0.0.1", REDIS_SERVER_PORTS.front(), "", true);
redis_client_ = std::make_shared<RedisClient>(options);
RAY_CHECK_OK(redis_client_->Connect(io_service_pool_->GetAll()));

View file

@ -40,7 +40,8 @@ class AccessorTestBase : public RedisServiceManagerForTest {
virtual void SetUp() {
GenTestData();
GcsClientOptions options = GcsClientOptions("127.0.0.1", REDIS_SERVER_PORT, "", true);
GcsClientOptions options =
GcsClientOptions("127.0.0.1", REDIS_SERVER_PORTS.front(), "", true);
gcs_client_.reset(new RedisGcsClient(options));
RAY_CHECK_OK(gcs_client_->Connect(io_service_));

View file

@ -49,7 +49,7 @@ void GetCallback(redisAsyncContext *c, void *r, void *privdata) {
class RedisAsioTest : public RedisServiceManagerForTest {};
TEST_F(RedisAsioTest, TestRedisCommands) {
redisAsyncContext *ac = redisAsyncConnect("127.0.0.1", REDIS_SERVER_PORT);
redisAsyncContext *ac = redisAsyncConnect("127.0.0.1", REDIS_SERVER_PORTS.front());
ASSERT_TRUE(ac->err == 0);
ray::gcs::RedisAsyncContext redis_async_context(ac);

View file

@ -31,7 +31,7 @@ namespace gcs {
/* Flush redis. */
static inline void flushall_redis(void) {
redisContext *context = redisConnect("127.0.0.1", REDIS_SERVER_PORT);
redisContext *context = redisConnect("127.0.0.1", REDIS_SERVER_PORTS.front());
freeReplyObject(redisCommand(context, "FLUSHALL"));
redisFree(context);
}
@ -85,7 +85,7 @@ class TestGcsWithAsio : public TestGcs {
}
void SetUp() override {
GcsClientOptions options("127.0.0.1", REDIS_SERVER_PORT, "", true);
GcsClientOptions options("127.0.0.1", REDIS_SERVER_PORTS.front(), "", true);
client_ = std::make_shared<gcs::RedisGcsClient>(options, command_type_);
RAY_CHECK_OK(client_->Connect(io_service_));
}

View file

@ -1,64 +0,0 @@
#!/usr/bin/env bash
# This needs to be run in the root directory.
# Try to find an unused port for raylet to use.
PORTS="2000 2001 2002 2003 2004 2005 2006 2007 2008 2009"
RAYLET_PORT=0
for port in $PORTS; do
nc -z localhost $port
if [[ $? != 0 ]]; then
RAYLET_PORT=$port
break
fi
done
if [[ $RAYLET_PORT == 0 ]]; then
echo "WARNING: Could not find unused port for raylet to use. Exiting without running tests."
exit
fi
# Cause the script to exit if a single command fails.
set -e
set -x
bazel build -c dbg $RAY_BAZEL_CONFIG "//:core_worker_test" "//:mock_worker" "//:raylet" "//:raylet_monitor" "//:gcs_server" "//:libray_redis_module.so" "@plasma//:plasma_store_server"
# Get the directory in which this script is executing.
SCRIPT_DIR="`dirname \"$0\"`"
RAY_ROOT="$SCRIPT_DIR/../../.."
# Makes $RAY_ROOT an absolute path.
RAY_ROOT="`( cd \"$RAY_ROOT\" && pwd )`"
if [ -z "$RAY_ROOT" ] ; then
exit 1
fi
# Ensure we're in the right directory.
if [ ! -d "$RAY_ROOT/python" ]; then
echo "Unable to find root Ray directory. Has this script moved?"
exit 1
fi
REDIS_MODULE="./bazel-bin/libray_redis_module.so"
BAZEL_BIN_PREFIX="$(bazel info -c dbg $RAY_BAZEL_CONFIG bazel-bin)"
LOAD_MODULE_ARGS="--loadmodule ${REDIS_MODULE}"
STORE_EXEC="$BAZEL_BIN_PREFIX/external/plasma/plasma_store_server"
RAYLET_EXEC="$BAZEL_BIN_PREFIX/raylet"
RAYLET_MONITOR_EXEC="$BAZEL_BIN_PREFIX/raylet_monitor"
MOCK_WORKER_EXEC="$BAZEL_BIN_PREFIX/mock_worker"
GCS_SERVER_EXEC="$BAZEL_BIN_PREFIX/gcs_server"
# Allow cleanup commands to fail.
bazel run "//:redis-cli" -- -p 6379 shutdown || true
sleep 1s
bazel run "//:redis-cli" -- -p 6380 shutdown || true
sleep 1s
bazel run "//:redis-server" -- --loglevel warning ${LOAD_MODULE_ARGS} --port 6379 &
sleep 2s
bazel run "//:redis-server" -- --loglevel warning ${LOAD_MODULE_ARGS} --port 6380 &
sleep 2s
# Run tests.
bazel run -c dbg $RAY_BAZEL_CONFIG "//:core_worker_test" $STORE_EXEC $RAYLET_EXEC $RAYLET_PORT $RAYLET_MONITOR_EXEC $MOCK_WORKER_EXEC $GCS_SERVER_EXEC
sleep 1s
bazel run "//:redis-cli" -- -p 6379 shutdown
bazel run "//:redis-cli" -- -p 6380 shutdown
sleep 1s