ray/src/common/common.h
Philipp Moritz a30eed452e Change type naming convention. (#315)
* Rename object_id -> ObjectID.

* Rename ray_logger -> RayLogger.

* rename task_id -> TaskID, actor_id -> ActorID, function_id -> FunctionID

* Rename plasma_store_info -> PlasmaStoreInfo.

* Rename plasma_store_state -> PlasmaStoreState.

* Rename plasma_object -> PlasmaObject.

* Rename object_request -> ObjectRequests.

* Rename eviction_state -> EvictionState.

* Bug fix.

* rename db_handle -> DBHandle

* Rename local_scheduler_state -> LocalSchedulerState.

* rename db_client_id -> DBClientID

* rename task -> Task

* make redis.c C++ compatible

* Rename scheduling_algorithm_state -> SchedulingAlgorithmState.

* Rename plasma_connection -> PlasmaConnection.

* Rename client_connection -> ClientConnection.

* Fixes from rebase.

* Rename local_scheduler_client -> LocalSchedulerClient.

* Rename object_buffer -> ObjectBuffer.

* Rename client -> Client.

* Rename notification_queue -> NotificationQueue.

* Rename object_get_requests -> ObjectGetRequests.

* Rename get_request -> GetRequest.

* Rename object_info -> ObjectInfo.

* Rename scheduler_object_info -> SchedulerObjectInfo.

* Rename local_scheduler -> LocalScheduler and some fixes.

* Rename local_scheduler_info -> LocalSchedulerInfo.

* Rename global_scheduler_state -> GlobalSchedulerState.

* Rename global_scheduler_policy_state -> GlobalSchedulerPolicyState.

* Rename object_size_entry -> ObjectSizeEntry.

* Rename aux_address_entry -> AuxAddressEntry.

* Rename various ID helper methods.

* Rename Task helper methods.

* Rename db_client_cache_entry -> DBClientCacheEntry.

* Rename local_actor_info -> LocalActorInfo.

* Rename actor_info -> ActorInfo.

* Rename retry_info -> RetryInfo.

* Rename actor_notification_table_subscribe_data -> ActorNotificationTableSubscribeData.

* Rename local_scheduler_table_send_info_data -> LocalSchedulerTableSendInfoData.

* Rename table_callback_data -> TableCallbackData.

* Rename object_info_subscribe_data -> ObjectInfoSubscribeData.

* Rename local_scheduler_table_subscribe_data -> LocalSchedulerTableSubscribeData.

* Rename more redis call data structures.

* Rename photon_conn PhotonConnection.

* Rename photon_mock -> PhotonMock.

* Fix formatting errors.
2017-02-26 00:32:43 -08:00

188 lines
5.8 KiB
C

#ifndef COMMON_H
#define COMMON_H
#include <stdbool.h>
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <errno.h>
#include <inttypes.h>
#ifndef _WIN32
#include <execinfo.h>
#endif
#include "utarray.h"
#include "sha256.h"
/** Definitions for Ray logging levels. */
#define RAY_COMMON_DEBUG 0
#define RAY_COMMON_INFO 1
#define RAY_COMMON_WARNING 2
#define RAY_COMMON_ERROR 3
#define RAY_COMMON_FATAL 4
/**
* RAY_COMMON_LOG_LEVEL should be defined to one of the above logging level
* integer values. Any logging statement in the code with a logging level
* greater than or equal to RAY_COMMON_LOG_LEVEL will be outputted to stderr.
* The default logging level is INFO. */
#ifndef RAY_COMMON_LOG_LEVEL
#define RAY_COMMON_LOG_LEVEL RAY_COMMON_INFO
#endif
#define UNUSED(x) ((void) (x))
/**
* Macros to enable each level of Ray logging statements depending on the
* current logging level. */
#if (RAY_COMMON_LOG_LEVEL > RAY_COMMON_DEBUG)
#define LOG_DEBUG(M, ...)
#else
#define LOG_DEBUG(M, ...) \
fprintf(stderr, "[DEBUG] (%s:%d) " M "\n", __FILE__, __LINE__, ##__VA_ARGS__)
#endif
#if (RAY_COMMON_LOG_LEVEL > RAY_COMMON_INFO)
#define LOG_INFO(M, ...)
#else
#define LOG_INFO(M, ...) \
fprintf(stderr, "[INFO] (%s:%d) " M "\n", __FILE__, __LINE__, ##__VA_ARGS__)
#endif
#if (RAY_COMMON_LOG_LEVEL > RAY_COMMON_WARNING)
#define LOG_WARN(M, ...)
#else
#define LOG_WARN(M, ...) \
fprintf(stderr, "[WARN] (%s:%d) " M "\n", __FILE__, __LINE__, ##__VA_ARGS__)
#endif
#if (RAY_COMMON_LOG_LEVEL > RAY_COMMON_ERROR)
#define LOG_ERROR(M, ...)
#else
#define LOG_ERROR(M, ...) \
fprintf(stderr, "[ERROR] (%s:%d: errno: %s) " M "\n", __FILE__, __LINE__, \
errno == 0 ? "None" : strerror(errno), ##__VA_ARGS__)
#endif
#if (RAY_COMMON_LOG_LEVEL > RAY_COMMON_FATAL)
#define LOG_FATAL(M, ...)
#elif defined(_EXECINFO_H) || !defined(_WIN32)
#define LOG_FATAL(M, ...) \
do { \
fprintf(stderr, "[FATAL] (%s:%d: errno: %s) " M "\n", __FILE__, __LINE__, \
errno == 0 ? "None" : strerror(errno), ##__VA_ARGS__); \
void *buffer[255]; \
const int calls = backtrace(buffer, sizeof(buffer) / sizeof(void *)); \
backtrace_symbols_fd(buffer, calls, 1); \
abort(); \
} while (0)
#else
#define LOG_FATAL(M, ...) \
do { \
fprintf(stderr, "[FATAL] (%s:%d: errno: %s) " M "\n", __FILE__, __LINE__, \
errno == 0 ? "None" : strerror(errno), ##__VA_ARGS__); \
exit(-1); \
} while (0)
#endif
/** Assertion definitions, with optional logging. */
#define CHECKM(COND, M, ...) \
if (!(COND)) { \
LOG_FATAL("Check failure: %s \n" M, #COND, ##__VA_ARGS__); \
}
#define CHECK(COND) CHECKM(COND, "")
/* This should be defined if we want to check calls to DCHECK. */
#define RAY_DCHECK
/* Arrow also defines the DCHECK macro, so undo that definition. */
#ifdef DCHECK
#undef DCHECK
#endif
#ifdef RAY_DCHECK
#define DCHECK(COND) CHECK(COND)
#else
#define DCHECK(COND)
#endif
/* These are exit codes for common errors that can occur in Ray components. */
#define EXIT_COULD_NOT_BIND_PORT -2
/** This macro indicates that this pointer owns the data it is pointing to
* and is responsible for freeing it. */
#define OWNER
/** Definitions for unique ID types. */
#define UNIQUE_ID_SIZE 20
#define UNIQUE_ID_EQ(id1, id2) (memcmp((id1).id, (id2).id, UNIQUE_ID_SIZE) == 0)
#define IS_NIL_ID(id) UNIQUE_ID_EQ(id, NIL_ID)
typedef struct { unsigned char id[UNIQUE_ID_SIZE]; } UniqueID;
extern const UT_icd object_id_icd;
extern const UniqueID NIL_ID;
/* Generate a globally unique ID. */
UniqueID globally_unique_id(void);
#define NIL_OBJECT_ID NIL_ID
typedef UniqueID ObjectID;
#define ID_STRING_SIZE (2 * UNIQUE_ID_SIZE + 1)
/**
* Convert an object ID to a hexdecimal string. This function assumes that
* buffer points to an already allocated char array of size ID_STRING_SIZE. And
* it writes a null-terminated hex-formatted string to id_string.
*
* @param obj_id The object ID to convert to a string.
* @param id_string A buffer to write the string to. It is assumed that this is
* managed by the caller and is sufficiently long to store the object ID
* string.
* @param id_length The length of the id_string buffer.
*/
char *ObjectID_to_string(ObjectID obj_id, char *id_string, int id_length);
/**
* Compare two object IDs.
*
* @param first_id The first object ID to compare.
* @param second_id The first object ID to compare.
* @return True if the object IDs are the same and false otherwise.
*/
bool ObjectID_equal(ObjectID first_id, ObjectID second_id);
/**
* Compare a object ID to the nil ID.
*
* @param id The object ID to compare to nil.
* @return True if the object ID is equal to nil.
*/
bool ObjectID_is_nil(ObjectID id);
typedef UniqueID DBClientID;
/**
* Compare two db client IDs.
*
* @param first_id The first db client ID to compare.
* @param second_id The first db client ID to compare.
* @return True if the db client IDs are the same and false otherwise.
*/
bool DBClientID_equal(DBClientID first_id, DBClientID second_id);
#define MAX(x, y) ((x) >= (y) ? (x) : (y))
#define MIN(x, y) ((x) <= (y) ? (x) : (y))
/** Definitions for computing hash digests. */
#define DIGEST_SIZE SHA256_BLOCK_SIZE
extern const unsigned char NIL_DIGEST[DIGEST_SIZE];
#endif