diff --git a/src/common/cmake/Common.cmake b/src/common/cmake/Common.cmake index 63c03441a94b..3dea1096123b 100644 --- a/src/common/cmake/Common.cmake +++ b/src/common/cmake/Common.cmake @@ -3,7 +3,7 @@ include(ExternalProject) include(CMakeParseArguments) -set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11") +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11 -g -Werror -Wall -Wno-error=unused-function -Wno-error=strict-aliasing") set(FLATBUFFERS_VERSION "1.7.1") diff --git a/src/common/common.cc b/src/common/common.cc index b8b1709b3670..c1f95a050a83 100644 --- a/src/common/common.cc +++ b/src/common/common.cc @@ -12,9 +12,6 @@ #include "io.h" #include -/* This is used to define the array of object IDs. */ -const UT_icd object_id_icd = {sizeof(ObjectID), NULL, NULL, NULL}; - const UniqueID NIL_ID = UniqueID::nil(); const unsigned char NIL_DIGEST[DIGEST_SIZE] = {0}; diff --git a/src/common/common_protocol.cc b/src/common/common_protocol.cc index 2787ce70f202..e746673fa10b 100644 --- a/src/common/common_protocol.cc +++ b/src/common/common_protocol.cc @@ -19,7 +19,7 @@ to_flatbuf(flatbuffers::FlatBufferBuilder &fbb, ObjectID object_ids[], int64_t num_objects) { std::vector> results; - for (size_t i = 0; i < num_objects; i++) { + for (int64_t i = 0; i < num_objects; i++) { results.push_back(to_flatbuf(fbb, object_ids[i])); } return fbb.CreateVector(results); diff --git a/src/common/io.cc b/src/common/io.cc index 4ec28850cff4..4cf01e315a10 100644 --- a/src/common/io.cc +++ b/src/common/io.cc @@ -369,7 +369,7 @@ int64_t read_vector(int fd, int64_t *type, std::vector &buffer) { if (closed) { goto disconnected; } - if (length > buffer.size()) { + if (static_cast(length) > buffer.size()) { buffer.resize(length); } closed = read_bytes(fd, buffer.data(), length); diff --git a/src/common/lib/python/common_extension.h b/src/common/lib/python/common_extension.h index c91b631cd330..71830fc95d13 100644 --- a/src/common/lib/python/common_extension.h +++ b/src/common/lib/python/common_extension.h @@ -8,7 +8,7 @@ #include "common.h" typedef uint8_t TaskSpec; -struct TaskBuilder; +class TaskBuilder; extern PyObject *CommonError; diff --git a/src/common/state/error_table.cc b/src/common/state/error_table.cc index 7453cd3c61b8..516bd98b85a3 100644 --- a/src/common/state/error_table.cc +++ b/src/common/state/error_table.cc @@ -1,6 +1,14 @@ #include "error_table.h" #include "redis.h" +const char *error_types[] = {"object_hash_mismatch", "put_reconstruction", + "worker_died"}; +const char *error_messages[] = { + "A nondeterministic task was reexecuted.", + "An object created by ray.put was evicted and could not be reconstructed. " + "The driver may need to be restarted.", + "A worker died or was killed while executing a task."}; + void push_error(DBHandle *db_handle, DBClientID driver_id, int error_index, diff --git a/src/common/state/error_table.h b/src/common/state/error_table.h index f2f7c5651f07..0b297283fcf4 100644 --- a/src/common/state/error_table.h +++ b/src/common/state/error_table.h @@ -27,13 +27,8 @@ typedef enum { } error_index; /** Information about the error to be displayed to the user. */ -static const char *error_types[] = {"object_hash_mismatch", - "put_reconstruction", "worker_died"}; -static const char *error_messages[] = { - "A nondeterministic task was reexecuted.", - "An object created by ray.put was evicted and could not be reconstructed. " - "The driver may need to be restarted.", - "A worker died or was killed while executing a task."}; +extern const char *error_types[]; +extern const char *error_messages[]; /** * Push an error to the given Python driver. diff --git a/src/common/state/redis.cc b/src/common/state/redis.cc index 58aca1ec53bb..bb7af45abcea 100644 --- a/src/common/state/redis.cc +++ b/src/common/state/redis.cc @@ -128,7 +128,7 @@ void get_redis_shards(redisContext *context, /* Try to read the Redis shard locations from the primary shard. If we find * that all of them are present, exit. */ reply = (redisReply *) redisCommand(context, "LRANGE RedisShards 0 -1"); - if (reply->elements == num_redis_shards) { + if (static_cast(reply->elements) == num_redis_shards) { break; } @@ -146,7 +146,7 @@ void get_redis_shards(redisContext *context, /* Parse the Redis shard addresses. */ char db_shard_address[16]; int db_shard_port; - for (int i = 0; i < reply->elements; ++i) { + for (size_t i = 0; i < reply->elements; ++i) { /* Parse the shard addresses and ports. */ CHECK(reply->element[i]->type == REDIS_REPLY_STRING); CHECK(parse_ip_addr_port(reply->element[i]->str, db_shard_address, @@ -297,7 +297,7 @@ DBHandle *db_connect(const std::string &db_primary_address, get_redis_shards(db->sync_context, db_shards_addresses, db_shards_ports); CHECKM(db_shards_addresses.size() > 0, "No Redis shards found"); /* Connect to the shards. */ - for (int i = 0; i < db_shards_addresses.size(); ++i) { + for (size_t i = 0; i < db_shards_addresses.size(); ++i) { db_connect_shard(db_shards_addresses[i], db_shards_ports[i], client, client_type, node_ip_address, num_args, args, db, &context, &subscribe_context, &sync_context); @@ -317,7 +317,7 @@ void DBHandle_free(DBHandle *db) { /* Clean up the Redis shards. */ CHECK(db->contexts.size() == db->subscribe_contexts.size()); - for (int i = 0; i < db->contexts.size(); ++i) { + for (size_t i = 0; i < db->contexts.size(); ++i) { redisAsyncFree(db->contexts[i]); redisAsyncFree(db->subscribe_contexts[i]); } @@ -360,7 +360,7 @@ void db_attach(DBHandle *db, event_loop *loop, bool reattach) { } /* Attach other redis shards to the event loop. */ CHECK(db->contexts.size() == db->subscribe_contexts.size()); - for (int i = 0; i < db->contexts.size(); ++i) { + for (size_t i = 0; i < db->contexts.size(); ++i) { int err = redisAeAttach(loop, db->contexts[i]); /* If the database is reattached in the tests, redis normally gives * an error which we can safely ignore. */ @@ -678,7 +678,7 @@ void redis_object_table_lookup_callback(redisAsyncContext *c, /* Extract the manager IDs from the response into a vector. */ std::vector manager_ids; - for (int j = 0; j < reply->elements; ++j) { + for (size_t j = 0; j < reply->elements; ++j) { CHECK(reply->element[j]->type == REDIS_REPLY_STRING); DBClientID manager_id; memcpy(manager_id.id, reply->element[j]->str, sizeof(manager_id.id)); @@ -777,7 +777,7 @@ void redis_object_table_subscribe_to_notifications( * src/common/redismodule/ray_redis_module.cc. */ const char *object_channel_prefix = "OC:"; const char *object_channel_bcast = "BCAST"; - for (int i = 0; i < db->subscribe_contexts.size(); ++i) { + for (size_t i = 0; i < db->subscribe_contexts.size(); ++i) { int status = REDIS_OK; /* Subscribe to notifications from the object table. This uses the client ID * as the channel name so this channel is specific to this client. @@ -1075,8 +1075,6 @@ void redis_task_table_subscribe_callback(redisAsyncContext *c, strcmp(message_type->str, "pmessage") == 0) { /* Handle a task table event. Parse the payload and call the callback. */ auto message = flatbuffers::GetRoot(payload->str); - /* Extract the task ID. */ - TaskID task_id = from_flatbuf(message->task_id()); /* Extract the scheduling state. */ int64_t state = message->state(); /* Extract the local scheduler ID. */ @@ -1188,7 +1186,7 @@ void redis_db_client_table_scan(DBHandle *db, } /* Get all the database client information. */ CHECK(reply->type == REDIS_REPLY_ARRAY); - for (int i = 0; i < reply->elements; ++i) { + for (size_t i = 0; i < reply->elements; ++i) { redisReply *client_reply = (redisReply *) redisCommand( db->sync_context, "HGETALL %b", reply->element[i]->str, reply->element[i]->len); @@ -1198,7 +1196,7 @@ void redis_db_client_table_scan(DBHandle *db, memset(&db_client, 0, sizeof(db_client)); int num_fields = 0; /* Parse the fields into a DBClient. */ - for (int j = 0; j < client_reply->elements; j = j + 2) { + for (size_t j = 0; j < client_reply->elements; j = j + 2) { const char *key = client_reply->element[j]->str; const char *value = client_reply->element[j + 1]->str; if (strcmp(key, "ray_client_id") == 0) { diff --git a/src/common/task.cc b/src/common/task.cc index c8d8f7d13c39..a78c16ac3715 100644 --- a/src/common/task.cc +++ b/src/common/task.cc @@ -72,10 +72,10 @@ class TaskBuilder { } void SetRequiredResource(int64_t resource_index, double value) { - if (resource_index >= resource_vector_.size()) { + if (static_cast(resource_index) >= resource_vector_.size()) { /* Make sure the resource vector is constructed entry by entry, * in order. */ - CHECK(resource_index == resource_vector_.size()); + CHECK(static_cast(resource_index) == resource_vector_.size()); resource_vector_.resize(resource_index + 1); } resource_vector_[resource_index] = value; diff --git a/src/common/task.h b/src/common/task.h index db1b36899b84..ce2590a1e08d 100644 --- a/src/common/task.h +++ b/src/common/task.h @@ -10,7 +10,7 @@ typedef uint8_t TaskSpec; -struct TaskBuilder; +class TaskBuilder; #define NIL_TASK_ID NIL_ID #define NIL_ACTOR_ID NIL_ID diff --git a/src/common/test/task_table_tests.cc b/src/common/test/task_table_tests.cc index 2fcb1d7aaa6a..2cae0e078850 100644 --- a/src/common/test/task_table_tests.cc +++ b/src/common/test/task_table_tests.cc @@ -161,7 +161,7 @@ TEST subscribe_timeout_test(void) { (void *) subscribe_timeout_context); /* Disconnect the database to see if the subscribe times out. */ close(db->subscribe_context->c.fd); - for (int i = 0; i < db->subscribe_contexts.size(); ++i) { + for (size_t i = 0; i < db->subscribe_contexts.size(); ++i) { close(db->subscribe_contexts[i]->c.fd); } aeProcessEvents(g_loop, AE_TIME_EVENTS); @@ -176,7 +176,6 @@ TEST subscribe_timeout_test(void) { /* === Test publish timeout === */ const char *publish_timeout_context = "publish_timeout"; -const int publish_test_number = 272; int publish_failed = 0; void publish_done_callback(TaskID task_id, void *user_context) { @@ -205,7 +204,7 @@ TEST publish_timeout_test(void) { (void *) publish_timeout_context); /* Disconnect the database to see if the publish times out. */ close(db->context->c.fd); - for (int i = 0; i < db->contexts.size(); ++i) { + for (size_t i = 0; i < db->contexts.size(); ++i) { close(db->contexts[i]->c.fd); } aeProcessEvents(g_loop, AE_TIME_EVENTS); @@ -227,7 +226,7 @@ int64_t reconnect_db_callback(event_loop *loop, redisAsyncFree(db->subscribe_context); db->subscribe_context = redisAsyncConnect("127.0.0.1", 6379); db->subscribe_context->data = (void *) db; - for (int i = 0; i < db->subscribe_contexts.size(); ++i) { + for (size_t i = 0; i < db->subscribe_contexts.size(); ++i) { redisAsyncFree(db->subscribe_contexts[i]); db->subscribe_contexts[i] = redisAsyncConnect("127.0.0.1", 6380 + i); db->subscribe_contexts[i]->data = (void *) db; @@ -247,7 +246,6 @@ int64_t terminate_event_loop_callback(event_loop *loop, /* === Test subscribe retry === */ const char *subscribe_retry_context = "subscribe_retry"; -const int subscribe_retry_test_number = 273; int subscribe_retry_succeeded = 0; void subscribe_retry_done_callback(ObjectID object_id, void *user_context) { @@ -277,7 +275,7 @@ TEST subscribe_retry_test(void) { (void *) subscribe_retry_context); /* Disconnect the database to see if the subscribe times out. */ close(db->subscribe_context->c.fd); - for (int i = 0; i < db->subscribe_contexts.size(); ++i) { + for (size_t i = 0; i < db->subscribe_contexts.size(); ++i) { close(db->subscribe_contexts[i]->c.fd); } /* Install handler for reconnecting the database. */ @@ -329,7 +327,7 @@ TEST publish_retry_test(void) { (void *) publish_retry_context); /* Disconnect the database to see if the publish times out. */ close(db->subscribe_context->c.fd); - for (int i = 0; i < db->subscribe_contexts.size(); ++i) { + for (size_t i = 0; i < db->subscribe_contexts.size(); ++i) { close(db->subscribe_contexts[i]->c.fd); } /* Install handler for reconnecting the database. */ diff --git a/src/common/test/test_common.h b/src/common/test/test_common.h index 94c6ef3bb1fc..9d482e91d459 100644 --- a/src/common/test/test_common.h +++ b/src/common/test/test_common.h @@ -60,7 +60,7 @@ static inline void flushall_redis(void) { /* Readd the shard locations. */ freeReplyObject(redisCommand(context, "SET NumRedisShards %d", db_shards_addresses.size())); - for (int i = 0; i < db_shards_addresses.size(); ++i) { + for (size_t i = 0; i < db_shards_addresses.size(); ++i) { freeReplyObject(redisCommand(context, "RPUSH RedisShards %s:%d", db_shards_addresses[i].c_str(), db_shards_ports[i])); @@ -68,7 +68,7 @@ static inline void flushall_redis(void) { redisFree(context); /* Flush the remaining shards. */ - for (int i = 0; i < db_shards_addresses.size(); ++i) { + for (size_t i = 0; i < db_shards_addresses.size(); ++i) { context = redisConnect(db_shards_addresses[i].c_str(), db_shards_ports[i]); freeReplyObject(redisCommand(context, "FLUSHALL")); redisFree(context); diff --git a/src/global_scheduler/CMakeLists.txt b/src/global_scheduler/CMakeLists.txt index 6e09b0c1cb55..bd6365bb8740 100644 --- a/src/global_scheduler/CMakeLists.txt +++ b/src/global_scheduler/CMakeLists.txt @@ -4,5 +4,7 @@ project(global_scheduler) include(${CMAKE_CURRENT_LIST_DIR}/../common/cmake/Common.cmake) +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Werror -Wall") + add_executable(global_scheduler global_scheduler.cc global_scheduler_algorithm.cc) target_link_libraries(global_scheduler common ${HIREDIS_LIB}) diff --git a/src/global_scheduler/global_scheduler.cc b/src/global_scheduler/global_scheduler.cc index 3650d85d3e95..2785caca956f 100644 --- a/src/global_scheduler/global_scheduler.cc +++ b/src/global_scheduler/global_scheduler.cc @@ -290,7 +290,7 @@ void object_table_subscribe_callback( ObjectID_to_string(object_id, id_string, ID_STRING_SIZE)); ARROW_UNUSED(id_string); LOG_DEBUG("\tManagers<%d>:", manager_vector.size()); - for (int i = 0; i < manager_vector.size(); i++) { + for (size_t i = 0; i < manager_vector.size(); i++) { LOG_DEBUG("\t\t%s", manager_vector[i]); } @@ -304,7 +304,7 @@ void object_table_subscribe_callback( LOG_DEBUG("New object added to object_info_table with id = %s", ObjectID_to_string(object_id, id_string, ID_STRING_SIZE)); LOG_DEBUG("\tmanager locations:"); - for (int i = 0; i < manager_vector.size(); i++) { + for (size_t i = 0; i < manager_vector.size(); i++) { LOG_DEBUG("\t\t%s", manager_vector[i]); } } @@ -314,7 +314,7 @@ void object_table_subscribe_callback( /* In all cases, replace the object location vector on each callback. */ obj_info_entry.object_locations.clear(); - for (int i = 0; i < manager_vector.size(); i++) { + for (size_t i = 0; i < manager_vector.size(); i++) { obj_info_entry.object_locations.push_back(std::string(manager_vector[i])); } } diff --git a/src/global_scheduler/global_scheduler_algorithm.cc b/src/global_scheduler/global_scheduler_algorithm.cc index 30c692f28773..79959d714da2 100644 --- a/src/global_scheduler/global_scheduler_algorithm.cc +++ b/src/global_scheduler/global_scheduler_algorithm.cc @@ -141,8 +141,7 @@ double calculate_cost_pending(const GlobalSchedulerState *state, TaskSpec *task_spec) { /* Calculate how much data is already present on this machine. TODO(rkn): Note * that this information is not being used yet. Fix this. */ - int64_t data_size = - locally_available_data_size(state, scheduler->id, task_spec); + locally_available_data_size(state, scheduler->id, task_spec); /* TODO(rkn): This logic does not load balance properly when the different * machines have different sizes. Fix this. */ return scheduler->num_recent_tasks_sent + scheduler->info.task_queue_length; @@ -157,11 +156,8 @@ bool handle_task_waiting(GlobalSchedulerState *state, "task wait handler encounted a task with NULL spec"); bool task_feasible = false; - /* The total size of the task's data. */ - int64_t task_object_size = 0; /* Go through all the nodes, calculate the score for each, pick max score. */ - LocalScheduler *scheduler = NULL; double best_local_scheduler_score = INT32_MIN; CHECKM(best_local_scheduler_score < 0, "We might have a floating point underflow"); diff --git a/src/local_scheduler/CMakeLists.txt b/src/local_scheduler/CMakeLists.txt index dbfe56d6f33d..c38a36ff2d41 100644 --- a/src/local_scheduler/CMakeLists.txt +++ b/src/local_scheduler/CMakeLists.txt @@ -19,7 +19,7 @@ endif(APPLE) include_directories("${PYTHON_INCLUDE_DIRS}") -# set(CMAKE_C_FLAGS "${CMAKE_CXX_FLAGS} --std=c99 -Werror") +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Werror -Wall") if(UNIX AND NOT APPLE) link_libraries(rt) diff --git a/src/local_scheduler/local_scheduler_algorithm.cc b/src/local_scheduler/local_scheduler_algorithm.cc index 9cfbc08c5b92..66343f134356 100644 --- a/src/local_scheduler/local_scheduler_algorithm.cc +++ b/src/local_scheduler/local_scheduler_algorithm.cc @@ -153,7 +153,7 @@ void SchedulingAlgorithmState_free(SchedulingAlgorithmState *algorithm_state) { remove_actor(algorithm_state, actor_id); } /* Free the list of cached actor task specs and the task specs themselves. */ - for (int i = 0; i < algorithm_state->cached_submitted_actor_tasks.size(); + for (size_t i = 0; i < algorithm_state->cached_submitted_actor_tasks.size(); ++i) { TaskQueueEntry task = algorithm_state->cached_submitted_actor_tasks[i]; TaskQueueEntry_free(&task); @@ -682,7 +682,7 @@ int reconstruct_object_timeout_handler(event_loop *loop, int64_t max_num_to_reconstruct = 10000; int64_t num_reconstructed = 0; - for (int64_t i = 0; i < object_ids_to_reconstruct.size(); i++) { + for (size_t i = 0; i < object_ids_to_reconstruct.size(); i++) { ObjectID object_id = object_ids_to_reconstruct[i]; /* Only call reconstruct if we are still missing the object. */ if (state->algorithm_state->remote_objects.find(object_id) != @@ -1112,7 +1112,6 @@ void handle_actor_creation_notification( for (int i = 0; i < num_cached_actor_tasks; ++i) { TaskQueueEntry task = algorithm_state->cached_submitted_actor_tasks[i]; - TaskSpec *spec = task.spec; /* Note that handle_actor_task_submitted may append the spec to the end of * the cached_submitted_actor_tasks array. */ handle_actor_task_submitted(state, algorithm_state, task.spec, diff --git a/src/local_scheduler/local_scheduler_client.cc b/src/local_scheduler/local_scheduler_client.cc index 10886e7d2c53..ea34a5bfdcbb 100644 --- a/src/local_scheduler/local_scheduler_client.cc +++ b/src/local_scheduler/local_scheduler_client.cc @@ -45,7 +45,7 @@ LocalSchedulerConnection *LocalSchedulerConnection_init( /* Parse the reply object. */ auto reply_message = flatbuffers::GetRoot(reply); - for (int i = 0; i < reply_message->gpu_ids()->size(); ++i) { + for (size_t i = 0; i < reply_message->gpu_ids()->size(); ++i) { result->gpu_ids.push_back(reply_message->gpu_ids()->Get(i)); } /* If the worker is not an actor, there should not be any GPU IDs here. */ @@ -122,7 +122,7 @@ TaskSpec *local_scheduler_get_task(LocalSchedulerConnection *conn, * actor methods. */ if (ActorID_equal(conn->actor_id, NIL_ACTOR_ID)) { conn->gpu_ids.clear(); - for (int i = 0; i < reply_message->gpu_ids()->size(); ++i) { + for (size_t i = 0; i < reply_message->gpu_ids()->size(); ++i) { conn->gpu_ids.push_back(reply_message->gpu_ids()->Get(i)); } } diff --git a/src/local_scheduler/test/local_scheduler_tests.cc b/src/local_scheduler/test/local_scheduler_tests.cc index 0bedbd3e53bc..bfa553eba467 100644 --- a/src/local_scheduler/test/local_scheduler_tests.cc +++ b/src/local_scheduler/test/local_scheduler_tests.cc @@ -610,7 +610,7 @@ TEST start_kill_workers_test(void) { /* We start off with num_workers children processes, but no workers * registered yet. */ ASSERT_EQ(local_scheduler->local_scheduler_state->child_pids.size(), - num_workers); + static_cast(num_workers)); ASSERT_EQ(local_scheduler->local_scheduler_state->workers.size(), 0); /* Make sure that each worker connects to the local_scheduler scheduler. This @@ -625,9 +625,9 @@ TEST start_kill_workers_test(void) { * workers accounted for, but we haven't yet matched up process IDs with our * children processes. */ ASSERT_EQ(local_scheduler->local_scheduler_state->child_pids.size(), - num_workers); + static_cast(num_workers)); ASSERT_EQ(local_scheduler->local_scheduler_state->workers.size(), - num_workers); + static_cast(num_workers)); /* Each worker should register its process ID. */ for (auto const &worker : local_scheduler->local_scheduler_state->workers) { @@ -636,7 +636,7 @@ TEST start_kill_workers_test(void) { } ASSERT_EQ(local_scheduler->local_scheduler_state->child_pids.size(), 0); ASSERT_EQ(local_scheduler->local_scheduler_state->workers.size(), - num_workers); + static_cast(num_workers)); /* After killing a worker, its state is cleaned up. */ LocalSchedulerClient *worker = @@ -644,7 +644,7 @@ TEST start_kill_workers_test(void) { kill_worker(local_scheduler->local_scheduler_state, worker, false, false); ASSERT_EQ(local_scheduler->local_scheduler_state->child_pids.size(), 0); ASSERT_EQ(local_scheduler->local_scheduler_state->workers.size(), - num_workers - 1); + static_cast(num_workers - 1)); /* Start a worker after the local scheduler has been initialized. */ start_worker(local_scheduler->local_scheduler_state, NIL_ACTOR_ID, false); @@ -653,21 +653,21 @@ TEST start_kill_workers_test(void) { /* The new worker should register its process ID. */ ASSERT_EQ(local_scheduler->local_scheduler_state->child_pids.size(), 1); ASSERT_EQ(local_scheduler->local_scheduler_state->workers.size(), - num_workers - 1); + static_cast(num_workers - 1)); /* Make sure the new worker connects to the local_scheduler scheduler. */ new_client_connection(local_scheduler->loop, local_scheduler->local_scheduler_fd, (void *) local_scheduler->local_scheduler_state, 0); ASSERT_EQ(local_scheduler->local_scheduler_state->child_pids.size(), 1); ASSERT_EQ(local_scheduler->local_scheduler_state->workers.size(), - num_workers); + static_cast(num_workers)); /* Make sure that the new worker registers its process ID. */ worker = local_scheduler->local_scheduler_state->workers.back(); process_message(local_scheduler->local_scheduler_state->loop, worker->sock, worker, 0); ASSERT_EQ(local_scheduler->local_scheduler_state->child_pids.size(), 0); ASSERT_EQ(local_scheduler->local_scheduler_state->workers.size(), - num_workers); + static_cast(num_workers)); /* Clean up. */ close(new_worker_fd); diff --git a/src/plasma/CMakeLists.txt b/src/plasma/CMakeLists.txt index 1f09b683f39e..d3748ada43e6 100644 --- a/src/plasma/CMakeLists.txt +++ b/src/plasma/CMakeLists.txt @@ -12,7 +12,7 @@ find_package(Plasma) include_directories(SYSTEM ${PLASMA_INCLUDE_DIR}) set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --std=c99 -D_XOPEN_SOURCE=500 -D_POSIX_C_SOURCE=200809L -O3") -set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --std=c++11 -D_XOPEN_SOURCE=500 -D_POSIX_C_SOURCE=200809L -O3") +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --std=c++11 -D_XOPEN_SOURCE=500 -D_POSIX_C_SOURCE=200809L -O3 -Werror -Wall") if(UNIX AND NOT APPLE) link_libraries(rt) diff --git a/src/plasma/plasma_manager.cc b/src/plasma/plasma_manager.cc index d771a1fc8ada..4ba80fb2db18 100644 --- a/src/plasma/plasma_manager.cc +++ b/src/plasma/plasma_manager.cc @@ -317,7 +317,6 @@ void add_wait_request_for_object(PlasmaManagerState *manager_state, auto &object_wait_requests = object_wait_requests_from_type(manager_state, type); - auto object_wait_requests_it = object_wait_requests.find(object_id); /* Add this wait request to the vector of wait requests involving this object * ID. Creates a vector of wait requests if none exist involving the object * ID. */ @@ -336,7 +335,7 @@ void remove_wait_request_for_object(PlasmaManagerState *manager_state, * vector. */ if (object_wait_requests_it != object_wait_requests.end()) { std::vector &wait_requests = object_wait_requests_it->second; - for (int i = 0; i < wait_requests.size(); ++i) { + for (size_t i = 0; i < wait_requests.size(); ++i) { if (wait_requests[i] == wait_req) { /* Remove the wait request from the array. */ wait_requests.erase(wait_requests.begin() + i); @@ -413,7 +412,7 @@ void update_object_wait_requests(PlasmaManagerState *manager_state, index += 1; } } - DCHECK(index == wait_requests.size()); + DCHECK(static_cast(index) == wait_requests.size()); /* Remove the array of wait requests for this object, since no one should be * waiting for this object anymore. */ object_wait_requests.erase(object_wait_requests_it); @@ -862,7 +861,8 @@ void request_transfer_from(PlasmaManagerState *manager_state, FetchRequest *fetch_req) { CHECK(fetch_req->manager_vector.size() > 0); CHECK(fetch_req->next_manager >= 0 && - fetch_req->next_manager < fetch_req->manager_vector.size()); + static_cast(fetch_req->next_manager) < + fetch_req->manager_vector.size()); char addr[16]; int port; parse_ip_addr_port(fetch_req->manager_vector[fetch_req->next_manager].c_str(), diff --git a/src/plasma/test/client_tests.cc b/src/plasma/test/client_tests.cc index 5362483592e2..39e25828d07b 100644 --- a/src/plasma/test/client_tests.cc +++ b/src/plasma/test/client_tests.cc @@ -111,13 +111,13 @@ TEST plasma_fetch_tests(void) { } void init_data_123(uint8_t *data, uint64_t size, uint8_t base) { - for (int i = 0; i < size; i++) { + for (size_t i = 0; i < size; i++) { data[i] = base + i; } } bool is_equal_data_123(uint8_t *data1, uint8_t *data2, uint64_t size) { - for (int i = 0; i < size; i++) { + for (size_t i = 0; i < size; i++) { if (data1[i] != data2[i]) { return false; };