remove UniqueIDHasher (#1957)

* remove UniqueIDHasher

* Format the change

* remove unused line

* Fix format

* fix lint error

* fix linting whitespace
This commit is contained in:
eric-jj 2018-04-30 21:31:23 +08:00 committed by Philipp Moritz
parent 47bc4c3009
commit 34bc6ce6ea
36 changed files with 116 additions and 138 deletions

4
.gitignore vendored
View file

@ -13,6 +13,7 @@
/src/thirdparty/catapult/
/src/thirdparty/flatbuffers/
/src/thirdparty/parquet-cpp
/thirdparty/pkg/
# Files generated by flatc should be ignored
/src/common/format/*.py
@ -143,3 +144,6 @@ build
# Pytest Cache
**/.pytest_cache
# Vscode
.vscode/

View file

@ -195,8 +195,8 @@ static PyObject *PyObjectID_richcompare(PyObjectID *self,
static PyObject *PyObjectID_redis_shard_hash(PyObjectID *self) {
/* NOTE: The hash function used here must match the one in get_redis_context
* in src/common/state/redis.cc. Changes to the hash function should only be
* made through UniqueIDHasher in src/common/common.h */
UniqueIDHasher hash;
* made through std::hash in src/common/common.h */
std::hash<ray::UniqueID> hash;
return PyLong_FromSize_t(hash(self->object_id));
}

View file

@ -80,13 +80,13 @@ redisAsyncContext *get_redis_context(DBHandle *db, UniqueID id) {
/* NOTE: The hash function used here must match the one in
* PyObjectID_redis_shard_hash in src/common/lib/python/common_extension.cc.
* Changes to the hash function should only be made through
* UniqueIDHasher in src/common/common.h */
UniqueIDHasher index;
* std::hash in src/common/common.h */
std::hash<ray::UniqueID> index;
return db->contexts[index(id) % db->contexts.size()];
}
redisAsyncContext *get_redis_subscribe_context(DBHandle *db, UniqueID id) {
UniqueIDHasher index;
std::hash<ray::UniqueID> index;
return db->subscribe_contexts[index(id) % db->subscribe_contexts.size()];
}

View file

@ -48,7 +48,7 @@ struct DBHandle {
int64_t db_index;
/** Cache for the IP addresses of db clients. This is an unordered map mapping
* client IDs to addresses. */
std::unordered_map<DBClientID, DBClient, UniqueIDHasher> db_client_cache;
std::unordered_map<DBClientID, DBClient> db_client_cache;
/** Redis context for synchronous connections. This should only be used very
* rarely, it is not asynchronous. */
redisContext *sync_context;

View file

@ -234,11 +234,9 @@ void add_local_scheduler(GlobalSchedulerState *state,
handle_new_local_scheduler(state, state->policy_state, db_client_id);
}
std::unordered_map<DBClientID, LocalScheduler, UniqueIDHasher>::iterator
remove_local_scheduler(
std::unordered_map<DBClientID, LocalScheduler>::iterator remove_local_scheduler(
GlobalSchedulerState *state,
std::unordered_map<DBClientID, LocalScheduler, UniqueIDHasher>::iterator
it) {
std::unordered_map<DBClientID, LocalScheduler>::iterator it) {
RAY_CHECK(it != state->local_schedulers.end());
DBClientID local_scheduler_id = it->first;
it = state->local_schedulers.erase(it);

View file

@ -55,18 +55,15 @@ typedef struct {
ray::gcs::AsyncGcsClient gcs_client;
/** A hash table mapping local scheduler ID to the local schedulers that are
* connected to Redis. */
std::unordered_map<DBClientID, LocalScheduler, UniqueIDHasher>
local_schedulers;
std::unordered_map<DBClientID, LocalScheduler> local_schedulers;
/** The state managed by the scheduling policy. */
GlobalSchedulerPolicyState *policy_state;
/** The plasma_manager ip:port -> local_scheduler_db_client_id association. */
std::unordered_map<std::string, DBClientID> plasma_local_scheduler_map;
/** The local_scheduler_db_client_id -> plasma_manager ip:port association. */
std::unordered_map<DBClientID, std::string, UniqueIDHasher>
local_scheduler_plasma_map;
std::unordered_map<DBClientID, std::string> local_scheduler_plasma_map;
/** Objects cached by this global scheduler instance. */
std::unordered_map<ObjectID, SchedulerObjectInfo, UniqueIDHasher>
scheduler_object_info_table;
std::unordered_map<ObjectID, SchedulerObjectInfo> scheduler_object_info_table;
/** An array of tasks that haven't been scheduled yet. */
std::vector<Task *> pending_tasks;
} GlobalSchedulerState;

View file

@ -1052,8 +1052,8 @@ void handle_set_actor_frontier(LocalSchedulerState *state,
ActorFrontier const &frontier) {
/* Parse the ActorFrontier flatbuffer. */
ActorID actor_id = from_flatbuf(*frontier.actor_id());
std::unordered_map<ActorID, int64_t, UniqueIDHasher> task_counters;
std::unordered_map<ActorID, ObjectID, UniqueIDHasher> frontier_dependencies;
std::unordered_map<ActorID, int64_t> task_counters;
std::unordered_map<ActorID, ObjectID> frontier_dependencies;
for (size_t i = 0; i < frontier.handle_ids()->size(); ++i) {
ActorID handle_id = from_flatbuf(*frontier.handle_ids()->Get(i));
task_counters[handle_id] = frontier.task_counters()->Get(i);

View file

@ -53,13 +53,12 @@ typedef struct {
* handle. This is used to guarantee execution of tasks on actors in the
* order that the tasks were submitted, per handle. Tasks from different
* handles to the same actor may be interleaved. */
std::unordered_map<ActorHandleID, int64_t, UniqueIDHasher> task_counters;
std::unordered_map<ActorHandleID, int64_t> task_counters;
/** These are the execution dependencies that make up the frontier of the
* actor's runnable tasks. For each actor handle, we store the object ID
* that represents the execution dependency for the next runnable task
* submitted by that handle. */
std::unordered_map<ActorHandleID, ObjectID, UniqueIDHasher>
frontier_dependencies;
std::unordered_map<ActorHandleID, ObjectID> frontier_dependencies;
/** The return value of the most recently executed task. The next task to
* execute should take this as an execution dependency at dispatch time. Set
* to nil if there are no execution dependencies (e.g., this is the first
@ -85,12 +84,12 @@ struct SchedulingAlgorithmState {
/** This is a hash table from actor ID to information about that actor. In
* particular, a queue of tasks that are waiting to execute on that actor.
* This is only used for actors that exist locally. */
std::unordered_map<ActorID, LocalActorInfo, UniqueIDHasher> local_actor_infos;
std::unordered_map<ActorID, LocalActorInfo> local_actor_infos;
/** This is a set of the IDs of the actors that have tasks waiting to run.
* The purpose is to make it easier to dispatch tasks without looping over
* all of the actors. Note that this is an optimization and is not strictly
* necessary. */
std::unordered_set<ActorID, UniqueIDHasher> actors_with_pending_tasks;
std::unordered_set<ActorID> actors_with_pending_tasks;
/** A vector of actor tasks that have been submitted but this local scheduler
* doesn't know which local scheduler is responsible for them, so cannot
* assign them to the correct local scheduler yet. Whenever a notification
@ -112,13 +111,13 @@ struct SchedulingAlgorithmState {
std::vector<LocalSchedulerClient *> blocked_workers;
/** A hash map of the objects that are available in the local Plasma store.
* The key is the object ID. This information could be a little stale. */
std::unordered_map<ObjectID, ObjectEntry, UniqueIDHasher> local_objects;
std::unordered_map<ObjectID, ObjectEntry> local_objects;
/** A hash map of the objects that are not available locally. These are
* currently being fetched by this local scheduler. The key is the object
* ID. Every local_scheduler_fetch_timeout_milliseconds, a Plasma fetch
* request will be sent the object IDs in this table. Each entry also holds
* an array of queued tasks that are dependent on it. */
std::unordered_map<ObjectID, ObjectEntry, UniqueIDHasher> remote_objects;
std::unordered_map<ObjectID, ObjectEntry> remote_objects;
};
SchedulingAlgorithmState *SchedulingAlgorithmState_init(void) {
@ -809,7 +808,7 @@ int rerun_actor_creation_tasks_timeout_handler(event_loop *loop,
// Create a set of the dummy object IDs for the actor creation tasks to
// reconstruct.
std::unordered_set<ObjectID, UniqueIDHasher> actor_dummy_objects;
std::unordered_set<ObjectID> actor_dummy_objects;
for (auto const &execution_spec :
state->algorithm_state->cached_submitted_actor_tasks) {
ObjectID actor_creation_dummy_object_id =
@ -1805,9 +1804,9 @@ void print_worker_info(const char *message,
<< " blocked";
}
std::unordered_map<ActorHandleID, int64_t, UniqueIDHasher>
get_actor_task_counters(SchedulingAlgorithmState *algorithm_state,
ActorID actor_id) {
std::unordered_map<ActorHandleID, int64_t> get_actor_task_counters(
SchedulingAlgorithmState *algorithm_state,
ActorID actor_id) {
RAY_CHECK(algorithm_state->local_actor_infos.count(actor_id) != 0);
return algorithm_state->local_actor_infos[actor_id].task_counters;
}
@ -1815,8 +1814,7 @@ get_actor_task_counters(SchedulingAlgorithmState *algorithm_state,
void set_actor_task_counters(
SchedulingAlgorithmState *algorithm_state,
ActorID actor_id,
const std::unordered_map<ActorHandleID, int64_t, UniqueIDHasher>
&task_counters) {
const std::unordered_map<ActorHandleID, int64_t> &task_counters) {
RAY_CHECK(algorithm_state->local_actor_infos.count(actor_id) != 0);
/* Overwrite the current task counters for the actor. This is necessary
* during reconstruction when resuming from a checkpoint so that we can
@ -1860,7 +1858,7 @@ void set_actor_task_counters(
}
}
std::unordered_map<ActorHandleID, ObjectID, UniqueIDHasher> get_actor_frontier(
std::unordered_map<ActorHandleID, ObjectID> get_actor_frontier(
SchedulingAlgorithmState *algorithm_state,
ActorID actor_id) {
RAY_CHECK(algorithm_state->local_actor_infos.count(actor_id) != 0);
@ -1871,8 +1869,7 @@ void set_actor_frontier(
LocalSchedulerState *state,
SchedulingAlgorithmState *algorithm_state,
ActorID actor_id,
const std::unordered_map<ActorHandleID, ObjectID, UniqueIDHasher>
&frontier_dependencies) {
const std::unordered_map<ActorHandleID, ObjectID> &frontier_dependencies) {
RAY_CHECK(algorithm_state->local_actor_infos.count(actor_id) != 0);
auto entry = algorithm_state->local_actor_infos[actor_id];
entry.frontier_dependencies = frontier_dependencies;

View file

@ -362,9 +362,9 @@ void print_worker_info(const char *message,
* @return A map from handle ID to the number of tasks submitted by that handle
* that have executed so far.
*/
std::unordered_map<ActorHandleID, int64_t, UniqueIDHasher>
get_actor_task_counters(SchedulingAlgorithmState *algorithm_state,
ActorID actor_id);
std::unordered_map<ActorHandleID, int64_t> get_actor_task_counters(
SchedulingAlgorithmState *algorithm_state,
ActorID actor_id);
/**
* Set the number of tasks, per actor handle, that have been executed on an
@ -381,8 +381,7 @@ get_actor_task_counters(SchedulingAlgorithmState *algorithm_state,
void set_actor_task_counters(
SchedulingAlgorithmState *algorithm_state,
ActorID actor_id,
const std::unordered_map<ActorHandleID, int64_t, UniqueIDHasher>
&task_counters);
const std::unordered_map<ActorHandleID, int64_t> &task_counters);
/**
* Get the actor's frontier of task dependencies.
@ -395,7 +394,7 @@ void set_actor_task_counters(
* @return A map from handle ID to execution dependency for the earliest
* runnable task submitted through that handle.
*/
std::unordered_map<ActorHandleID, ObjectID, UniqueIDHasher> get_actor_frontier(
std::unordered_map<ActorHandleID, ObjectID> get_actor_frontier(
SchedulingAlgorithmState *algorithm_state,
ActorID actor_id);
@ -414,8 +413,7 @@ void set_actor_frontier(
LocalSchedulerState *state,
SchedulingAlgorithmState *algorithm_state,
ActorID actor_id,
const std::unordered_map<ActorHandleID, ObjectID, UniqueIDHasher>
&frontier_dependencies);
const std::unordered_map<ActorHandleID, ObjectID> &frontier_dependencies);
/** The following methods are for testing purposes only. */
#ifdef LOCAL_SCHEDULER_TEST

View file

@ -48,16 +48,16 @@ struct LocalSchedulerState {
std::list<LocalSchedulerClient *> workers;
/** A set of driver IDs corresponding to drivers that have been removed. This
* is used to make sure we don't execute any tasks belong to dead drivers. */
std::unordered_set<WorkerID, UniqueIDHasher> removed_drivers;
std::unordered_set<WorkerID> removed_drivers;
/** A set of actors IDs corresponding to local actors that have been removed.
* This ensures we can reject any tasks destined for dead actors. */
std::unordered_set<ActorID, UniqueIDHasher> removed_actors;
std::unordered_set<ActorID> removed_actors;
/** List of the process IDs for child processes (workers) started by the
* local scheduler that have not sent a REGISTER_PID message yet. */
std::vector<pid_t> child_pids;
/** A hash table mapping actor IDs to the db_client_id of the local scheduler
* that is responsible for the actor. */
std::unordered_map<ActorID, ActorMapEntry, UniqueIDHasher> actor_mapping;
std::unordered_map<ActorID, ActorMapEntry> actor_mapping;
/** The handle to the database. */
DBHandle *db;
/** The handle to the GCS (modern version of the above). */

View file

@ -221,20 +221,20 @@ struct PlasmaManagerState {
int port;
/** Unordered map of outstanding fetch requests. The key is the object ID. The
* value is the data needed to perform the fetch. */
std::unordered_map<ObjectID, FetchRequest *, UniqueIDHasher> fetch_requests;
std::unordered_map<ObjectID, FetchRequest *> fetch_requests;
/** Unordered map of outstanding wait requests. The key is the object ID. The
* value is the vector of wait requests that are waiting for the object to
* arrive locally. */
std::unordered_map<ObjectID, std::vector<WaitRequest *>, UniqueIDHasher>
std::unordered_map<ObjectID, std::vector<WaitRequest *>>
object_wait_requests_local;
/** Unordered map of outstanding wait requests. The key is the object ID. The
* value is the vector of wait requests that are waiting for the object to
* be available somewhere in the system. */
std::unordered_map<ObjectID, std::vector<WaitRequest *>, UniqueIDHasher>
std::unordered_map<ObjectID, std::vector<WaitRequest *>>
object_wait_requests_remote;
/** Initialize an empty unordered set for the cache of local available object.
*/
std::unordered_set<ObjectID, UniqueIDHasher> local_available_objects;
std::unordered_set<ObjectID> local_available_objects;
/** The time (in milliseconds since the Unix epoch) when the most recent
* heartbeat was sent. */
int64_t previous_heartbeat_time;
@ -247,7 +247,7 @@ struct PlasmaManagerState {
* object is removed. If object transfers between managers is parallelized,
* then all objects being received from a remote manager will need to be
* removed if the connection to the remote manager fails. */
std::unordered_set<ObjectID, UniqueIDHasher> receives_in_progress;
std::unordered_set<ObjectID> receives_in_progress;
};
PlasmaManagerState *g_manager_state = NULL;
@ -265,8 +265,7 @@ struct ClientConnection {
/* A set of object IDs which are queued in the transfer_queue and waiting to
* be sent. This is used to avoid sending the same object ID to the same
* manager multiple times. */
std::unordered_map<ObjectID, PlasmaRequestBuffer *, UniqueIDHasher>
pending_object_transfers;
std::unordered_map<ObjectID, PlasmaRequestBuffer *> pending_object_transfers;
/** Buffer used to receive transfers (data fetches) we want to ignore */
PlasmaRequestBuffer *ignore_buffer;
/** File descriptor for the socket connected to the other
@ -317,7 +316,7 @@ bool ClientConnection_request_finished(ClientConnection *client_conn) {
return client_conn->cursor == -1;
}
std::unordered_map<ObjectID, std::vector<WaitRequest *>, UniqueIDHasher> &
std::unordered_map<ObjectID, std::vector<WaitRequest *>> &
object_wait_requests_from_type(PlasmaManagerState *manager_state, int type) {
/* We use different types of hash tables for different requests. */
RAY_CHECK(type == plasma::PLASMA_QUERY_LOCAL ||

View file

@ -370,11 +370,10 @@ Status ReadGetRequest(uint8_t *data,
return Status::OK();
}
Status SendGetReply(
int sock,
ObjectID object_ids[],
std::unordered_map<ObjectID, PlasmaObject, UniqueIDHasher> &plasma_objects,
int64_t num_objects) {
Status SendGetReply(int sock,
ObjectID object_ids[],
std::unordered_map<ObjectID, PlasmaObject> &plasma_objects,
int64_t num_objects) {
flatbuffers::FlatBufferBuilder fbb;
std::vector<PlasmaObjectSpec> objects;

View file

@ -517,7 +517,7 @@ class ClientTable : private Log<UniqueID, ClientTableData> {
/// The callback to call when a client is removed.
ClientTableCallback client_removed_callback_;
/// A cache for information about all clients.
std::unordered_map<ClientID, ClientTableDataT, UniqueIDHasher> client_cache_;
std::unordered_map<ClientID, ClientTableDataT> client_cache_;
};
} // namespace gcs

View file

@ -81,6 +81,12 @@ bool UniqueID::operator==(const UniqueID &rhs) const {
return std::memcmp(data(), rhs.data(), kUniqueIDSize) == 0;
}
size_t UniqueID::hash() const {
size_t result;
std::memcpy(&result, id_, sizeof(size_t));
return result;
}
std::ostream &operator<<(std::ostream &os, const UniqueID &id) {
os << id.hex();
return os;

View file

@ -19,6 +19,7 @@ class RAY_EXPORT UniqueID {
static UniqueID from_random();
static UniqueID from_binary(const std::string &binary);
static const UniqueID nil();
size_t hash() const;
bool is_nil() const;
bool operator==(const UniqueID &rhs) const;
const uint8_t *data() const;
@ -35,15 +36,6 @@ class RAY_EXPORT UniqueID {
static_assert(std::is_standard_layout<UniqueID>::value,
"UniqueID must be standard");
struct UniqueIDHasher {
// ID hashing function.
size_t operator()(const UniqueID &id) const {
size_t result;
std::memcpy(&result, id.data(), sizeof(size_t));
return result;
}
};
std::ostream &operator<<(std::ostream &os, const UniqueID &id);
typedef UniqueID TaskID;
@ -98,4 +90,15 @@ int64_t ComputeObjectIndex(const ObjectID &object_id);
} // namespace ray
namespace std {
template <>
struct hash<::ray::UniqueID> {
size_t operator()(const ::ray::UniqueID &id) const { return id.hash(); }
};
template <>
struct hash<const ::ray::UniqueID> {
size_t operator()(const ::ray::UniqueID &id) const { return id.hash(); }
};
}
#endif // RAY_ID_H_

View file

@ -91,11 +91,10 @@ class ConnectionPool {
private:
/// A container type that maps ClientID to a connection type.
using SenderMapType =
std::unordered_map<ray::ClientID, std::vector<std::shared_ptr<SenderConnection>>,
ray::UniqueIDHasher>;
std::unordered_map<ray::ClientID, std::vector<std::shared_ptr<SenderConnection>>>;
using ReceiverMapType =
std::unordered_map<ray::ClientID, std::vector<std::shared_ptr<TcpClientConnection>>,
ray::UniqueIDHasher>;
std::unordered_map<ray::ClientID,
std::vector<std::shared_ptr<TcpClientConnection>>>;
/// Adds a receiver for ClientID to the given map.
void Add(ReceiverMapType &conn_map, const ClientID &client_id,

View file

@ -179,11 +179,9 @@ class ObjectBufferPool {
/// Determines the maximum chunk size to be transferred by a single thread.
const uint64_t chunk_size_;
/// The state of a buffer that's currently being used.
std::unordered_map<ray::ObjectID, GetBufferState, ray::UniqueIDHasher>
get_buffer_state_;
std::unordered_map<ray::ObjectID, GetBufferState> get_buffer_state_;
/// The state of a buffer that's currently being used.
std::unordered_map<ray::ObjectID, CreateBufferState, ray::UniqueIDHasher>
create_buffer_state_;
std::unordered_map<ray::ObjectID, CreateBufferState> create_buffer_state_;
/// Plasma client pool.
plasma::PlasmaClient store_client_;

View file

@ -77,7 +77,7 @@ void ObjectDirectory::GetLocationsComplete(
return;
}
// Build the set of current locations based on the entries in the log.
std::unordered_set<ClientID, UniqueIDHasher> locations;
std::unordered_set<ClientID> locations;
for (auto entry : location_entries) {
ClientID client_id = ClientID::from_binary(entry.manager);
if (!entry.is_eviction) {

View file

@ -123,7 +123,7 @@ class ObjectDirectory : public ObjectDirectoryInterface {
const std::vector<ObjectTableDataT> &location_entries);
/// Maintain map of in-flight GetLocation requests.
std::unordered_map<ObjectID, ODCallbacks, UniqueIDHasher> existing_requests_;
std::unordered_map<ObjectID, ODCallbacks> existing_requests_;
/// Reference to the gcs client.
std::shared_ptr<gcs::AsyncGcsClient> gcs_client_;
};

View file

@ -179,12 +179,11 @@ class ObjectManager {
ConnectionPool connection_pool_;
/// Timeout for failed pull requests.
std::unordered_map<ObjectID, std::shared_ptr<boost::asio::deadline_timer>,
UniqueIDHasher>
std::unordered_map<ObjectID, std::shared_ptr<boost::asio::deadline_timer>>
pull_requests_;
/// Cache of locally available objects.
std::unordered_map<ObjectID, ObjectInfoT, UniqueIDHasher> local_objects_;
std::unordered_map<ObjectID, ObjectInfoT> local_objects_;
/// Handle starting, running, and stopping asio io_service.
void StartIOService();

View file

@ -23,7 +23,7 @@ const ObjectID ActorRegistration::GetExecutionDependency() const {
return execution_dependency_;
}
const std::unordered_map<ActorHandleID, ActorRegistration::FrontierLeaf, UniqueIDHasher>
const std::unordered_map<ActorHandleID, ActorRegistration::FrontierLeaf>
&ActorRegistration::GetFrontier() const {
return frontier_;
}

View file

@ -63,8 +63,7 @@ class ActorRegistration {
///
/// \return The actor frontier, a map from handle ID to execution state for
/// that handle.
const std::unordered_map<ActorHandleID, FrontierLeaf, UniqueIDHasher> &GetFrontier()
const;
const std::unordered_map<ActorHandleID, FrontierLeaf> &GetFrontier() const;
/// Extend the frontier of the actor by a single task. This should be called
/// whenever the actor executes a task.
@ -86,7 +85,7 @@ class ActorRegistration {
/// The execution frontier of the actor, which represents which tasks have
/// executed so far and which tasks may execute next, based on execution
/// dependencies. This is indexed by handle.
std::unordered_map<ActorHandleID, FrontierLeaf, UniqueIDHasher> frontier_;
std::unordered_map<ActorHandleID, FrontierLeaf> frontier_;
};
} // namespace raylet

View file

@ -27,9 +27,8 @@ const TaskID LineageEntry::GetEntryId() const {
return task_.GetTaskSpecification().TaskId();
}
const std::unordered_set<UniqueID, UniqueIDHasher> LineageEntry::GetParentTaskIds()
const {
std::unordered_set<UniqueID, UniqueIDHasher> parent_ids;
const std::unordered_set<UniqueID> LineageEntry::GetParentTaskIds() const {
std::unordered_set<UniqueID> parent_ids;
// A task's parents are the tasks that created its arguments.
auto dependencies = task_.GetDependencies();
for (auto &dependency : dependencies) {
@ -104,8 +103,7 @@ boost::optional<LineageEntry> Lineage::PopEntry(const UniqueID &task_id) {
}
}
const std::unordered_map<const UniqueID, LineageEntry, UniqueIDHasher>
&Lineage::GetEntries() const {
const std::unordered_map<const UniqueID, LineageEntry> &Lineage::GetEntries() const {
return entries_;
}

View file

@ -76,7 +76,7 @@ class LineageEntry {
/// that created its arguments.
///
/// \return The IDs of the parent entries.
const std::unordered_set<TaskID, UniqueIDHasher> GetParentTaskIds() const;
const std::unordered_set<TaskID> GetParentTaskIds() const;
/// Get the task data.
///
@ -85,7 +85,6 @@ class LineageEntry {
Task &TaskDataMutable();
private:
/// The current state of this entry according to its status in the GCS.
GcsStatus status_;
/// The task data to be written to the GCS. This is nullptr if the entry is
@ -139,8 +138,7 @@ class Lineage {
/// Get all entries in the lineage.
///
/// \return A const reference to the lineage entries.
const std::unordered_map<const TaskID, LineageEntry, UniqueIDHasher> &GetEntries()
const;
const std::unordered_map<const TaskID, LineageEntry> &GetEntries() const;
/// Serialize this lineage to a ForwardTaskRequest flatbuffer.
///
@ -153,7 +151,7 @@ class Lineage {
private:
/// The lineage entries.
std::unordered_map<const TaskID, LineageEntry, UniqueIDHasher> entries_;
std::unordered_map<const TaskID, LineageEntry> entries_;
};
/// \class LineageCache
@ -226,13 +224,13 @@ class LineageCache {
// which tasks are flushable, to avoid iterating over tasks that are in
// UNCOMMITTED_READY, but that have dependencies that have not been committed
// yet.
std::unordered_set<TaskID, UniqueIDHasher> uncommitted_ready_tasks_;
std::unordered_set<TaskID> uncommitted_ready_tasks_;
/// All tasks and objects that we are responsible for writing back to the
/// GCS, and the tasks and objects in their lineage.
Lineage lineage_;
/// The tasks that we've subscribed to notifications for from the pubsub
/// storage system. We will receive a notification for these tasks on commit.
std::unordered_set<TaskID, UniqueIDHasher> subscribed_tasks_;
std::unordered_set<TaskID> subscribed_tasks_;
};
} // namespace raylet

View file

@ -68,21 +68,17 @@ class MockGcs : public gcs::TableInterface<TaskID, protocol::Task>,
callbacks_.clear();
}
const std::unordered_map<TaskID, std::shared_ptr<protocol::TaskT>, UniqueIDHasher>
&TaskTable() const {
const std::unordered_map<TaskID, std::shared_ptr<protocol::TaskT>> &TaskTable() const {
return task_table_;
}
const std::unordered_set<TaskID, UniqueIDHasher> &SubscribedTasks() const {
return subscribed_tasks_;
}
const std::unordered_set<TaskID> &SubscribedTasks() const { return subscribed_tasks_; }
private:
std::unordered_map<TaskID, std::shared_ptr<protocol::TaskT>, UniqueIDHasher>
task_table_;
std::unordered_map<TaskID, std::shared_ptr<protocol::TaskT>> task_table_;
std::vector<std::pair<gcs::raylet::TaskTable::WriteCallback, TaskID>> callbacks_;
gcs::raylet::TaskTable::WriteCallback notification_callback_;
std::unordered_set<TaskID, UniqueIDHasher> subscribed_tasks_;
std::unordered_set<TaskID> subscribed_tasks_;
};
class LineageCacheTest : public ::testing::Test {

View file

@ -31,7 +31,7 @@ ray::Status ObjectTable::Add(const ObjectID &object_id, const ClientID &client_i
const DoneCallback &done_callback) {
if (client_lookup.count(object_id) == 0) {
RAY_LOG(DEBUG) << "Add ObjectID set " << object_id;
client_lookup[object_id] = std::unordered_set<ClientID, UniqueIDHasher>();
client_lookup[object_id] = std::unordered_set<ClientID>();
} else if (client_lookup[object_id].count(client_id) != 0) {
return ray::Status::KeyError("ClientID already exists.");
}

View file

@ -31,9 +31,7 @@ class ObjectTable {
private:
std::vector<ClientID> empty_set_;
std::unordered_map<ObjectID, std::unordered_set<ClientID, UniqueIDHasher>,
UniqueIDHasher>
client_lookup;
std::unordered_map<ObjectID, std::unordered_set<ClientID>> client_lookup;
};
class ClientInformation {
@ -53,7 +51,7 @@ class ClientInformation {
class ClientTable {
public:
typedef std::unordered_map<ClientID, ClientInformation, UniqueIDHasher> info_type;
typedef std::unordered_map<ClientID, ClientInformation> info_type;
using ClientIDsCallback = std::function<void(std::vector<ray::ClientID>)>;
using SingleInfoCallback = std::function<void(ClientInformation info)>;

View file

@ -44,9 +44,9 @@ class Monitor {
boost::asio::deadline_timer heartbeat_timer_;
/// For each Raylet that we receive a heartbeat from, the number of ticks
/// that may pass before the Raylet will be declared dead.
std::unordered_map<ClientID, int64_t, UniqueIDHasher> heartbeats_;
std::unordered_map<ClientID, int64_t> heartbeats_;
/// The Raylets that have been marked as dead in the client table.
std::unordered_set<ClientID, UniqueIDHasher> dead_clients_;
std::unordered_set<ClientID> dead_clients_;
};
} // namespace raylet

View file

@ -32,8 +32,7 @@ RAY_CHECK_ENUM(protocol::MessageType_SetActorFrontier, MessageType_SetActorFront
/// A helper function to determine whether a given actor task has already been executed
/// according to the given actor registry. Returns true if the task is a duplicate.
bool CheckDuplicateActorTask(
const std::unordered_map<ActorID, ray::raylet::ActorRegistration, UniqueIDHasher>
&actor_registry,
const std::unordered_map<ActorID, ray::raylet::ActorRegistration> &actor_registry,
const ray::raylet::TaskSpecification &spec) {
auto actor_entry = actor_registry.find(spec.ActorId());
RAY_CHECK(actor_entry != actor_registry.end());
@ -262,7 +261,7 @@ void NodeManager::HandleActorCreation(const ActorID &actor_id,
// Dequeue any methods that were submitted before the actor's location was
// known.
const auto &methods = local_queues_.GetUncreatedActorMethods();
std::unordered_set<TaskID, UniqueIDHasher> created_actor_method_ids;
std::unordered_set<TaskID> created_actor_method_ids;
for (const auto &method : methods) {
if (method.GetTaskSpecification().ActorId() == actor_id) {
created_actor_method_ids.insert(method.GetTaskSpecification().TaskId());
@ -482,7 +481,7 @@ void NodeManager::ScheduleTasks() {
}
// Extract decision for this local scheduler.
std::unordered_set<TaskID, UniqueIDHasher> local_task_ids;
std::unordered_set<TaskID> local_task_ids;
// Iterate over (taskid, clientid) pairs, extract tasks assigned to the local node.
for (const auto &task_schedule : policy_decision) {
TaskID task_id = task_schedule.first;

View file

@ -100,7 +100,7 @@ class NodeManager {
/// The resources local to this node.
const SchedulingResources local_resources_;
// TODO(atumanov): Add resource information from other nodes.
std::unordered_map<ClientID, SchedulingResources, UniqueIDHasher> cluster_resource_map_;
std::unordered_map<ClientID, SchedulingResources> cluster_resource_map_;
/// A pool of workers.
WorkerPool worker_pool_;
/// A set of queues to maintain tasks.
@ -114,9 +114,8 @@ class NodeManager {
/// The lineage cache for the GCS object and task tables.
LineageCache lineage_cache_;
std::vector<ClientID> remote_clients_;
std::unordered_map<ClientID, TcpServerConnection, UniqueIDHasher>
remote_server_connections_;
std::unordered_map<ActorID, ActorRegistration, UniqueIDHasher> actor_registry_;
std::unordered_map<ClientID, TcpServerConnection> remote_server_connections_;
std::unordered_map<ActorID, ActorRegistration> actor_registry_;
};
} // namespace raylet

View file

@ -9,12 +9,11 @@ namespace raylet {
SchedulingPolicy::SchedulingPolicy(const SchedulingQueue &scheduling_queue)
: scheduling_queue_(scheduling_queue), gen_(rd_()) {}
std::unordered_map<TaskID, ClientID, UniqueIDHasher> SchedulingPolicy::Schedule(
const std::unordered_map<ClientID, SchedulingResources, UniqueIDHasher>
&cluster_resources,
std::unordered_map<TaskID, ClientID> SchedulingPolicy::Schedule(
const std::unordered_map<ClientID, SchedulingResources> &cluster_resources,
const ClientID &local_client_id, const std::vector<ClientID> &others) {
// The policy decision to be returned.
std::unordered_map<TaskID, ClientID, UniqueIDHasher> decision;
std::unordered_map<TaskID, ClientID> decision;
// TODO(atumanov): protect DEBUG code blocks with ifdef DEBUG
RAY_LOG(DEBUG) << "[Schedule] cluster resource map: ";
for (const auto &client_resource_pair : cluster_resources) {

View file

@ -28,9 +28,8 @@ class SchedulingPolicy {
/// \param cluster_resources: a set of cluster resources representing
/// configured and current resource capacity on each node.
/// \return Scheduling decision, mapping tasks to node managers for placement.
std::unordered_map<TaskID, ClientID, UniqueIDHasher> Schedule(
const std::unordered_map<ClientID, SchedulingResources, UniqueIDHasher>
&cluster_resources,
std::unordered_map<TaskID, ClientID> Schedule(
const std::unordered_map<ClientID, SchedulingResources> &cluster_resources,
const ClientID &local_client_id, const std::vector<ClientID> &others);
/// \brief SchedulingPolicy destructor.

View file

@ -36,8 +36,7 @@ const std::list<Task> &SchedulingQueue::GetReadyMethods() const {
// Helper function to remove tasks in the given set of task_ids from a
// queue, and append them to the given vector removed_tasks.
void removeTasksFromQueue(std::list<Task> &queue,
std::unordered_set<TaskID, UniqueIDHasher> &task_ids,
void removeTasksFromQueue(std::list<Task> &queue, std::unordered_set<TaskID> &task_ids,
std::vector<Task> &removed_tasks) {
for (auto it = queue.begin(); it != queue.end();) {
auto task_id = task_ids.find(it->GetTaskSpecification().TaskId());
@ -58,8 +57,7 @@ void queueTasks(std::list<Task> &queue, const std::vector<Task> &tasks) {
}
}
std::vector<Task> SchedulingQueue::RemoveTasks(
std::unordered_set<TaskID, UniqueIDHasher> task_ids) {
std::vector<Task> SchedulingQueue::RemoveTasks(std::unordered_set<TaskID> task_ids) {
// List of removed tasks to be returned.
std::vector<Task> removed_tasks;

View file

@ -77,7 +77,7 @@ class SchedulingQueue {
/// \param tasks The set of task IDs to remove from the queue. The
/// corresponding tasks must be contained in the queue.
/// \return A vector of the tasks that were removed.
std::vector<Task> RemoveTasks(std::unordered_set<TaskID, UniqueIDHasher> tasks);
std::vector<Task> RemoveTasks(std::unordered_set<TaskID> tasks);
/// Queue tasks that are destined for actors that have not yet been created.
///

View file

@ -73,14 +73,12 @@ class TaskDependencyManager {
ObjectManager &object_manager_;
/// A mapping from task ID of each subscribed task to its list of
/// dependencies.
std::unordered_map<ray::TaskID, std::vector<ray::ObjectID>, UniqueIDHasher>
task_dependencies_;
std::unordered_map<ray::TaskID, std::vector<ray::ObjectID>> task_dependencies_;
// A mapping from object ID of each object that is not locally available to
// the list of subscribed tasks that are dependent on it.
std::unordered_map<ray::ObjectID, std::vector<ray::TaskID>, UniqueIDHasher>
remote_object_dependencies_;
std::unordered_map<ray::ObjectID, std::vector<ray::TaskID>> remote_object_dependencies_;
// The set of locally available objects.
std::unordered_set<ray::ObjectID, UniqueIDHasher> local_objects_;
std::unordered_set<ray::ObjectID> local_objects_;
// The callback to call when a subscribed task becomes ready.
std::function<void(const TaskID &)> task_ready_callback_;
};

View file

@ -103,7 +103,7 @@ class WorkerPool {
/// The pool of idle workers.
std::list<std::shared_ptr<Worker>> pool_;
/// The pool of idle actor workers.
std::unordered_map<ActorID, std::shared_ptr<Worker>, UniqueIDHasher> actor_pool_;
std::unordered_map<ActorID, std::shared_ptr<Worker>> actor_pool_;
/// All workers that have registered and are still connected, including both
/// idle and executing.
// TODO(swang): Make this a map to make GetRegisteredWorker faster.