From 4b4d203518903cb43bc432bcf4423d67c887a737 Mon Sep 17 00:00:00 2001 From: Philipp Moritz Date: Thu, 30 Mar 2017 21:13:04 -0700 Subject: [PATCH 1/7] Initial check in of plasma from https://github.com/ray-project/ray, commit f1b48f2fd48b40c6ef60308c4239c90ae3c9d981. Contributors of this code: Philipp Moritz, Robert Nishihara, Richard Shin, Ion Stoica, Alexey Tumanov, Stephanie Wang @ RISElab, UC Berkeley (2017) --- cpp/src/plasma/CMakeLists.txt | 116 + cpp/src/plasma/cmake/Common.cmake | 95 + cpp/src/plasma/common/CMakeLists.txt | 31 + cpp/src/plasma/common/common.cc | 57 + cpp/src/plasma/common/common.h | 206 ++ cpp/src/plasma/common/common_extension.cc | 292 ++ cpp/src/plasma/common/common_extension.h | 56 + cpp/src/plasma/common/event_loop.cc | 61 + cpp/src/plasma/common/event_loop.h | 103 + cpp/src/plasma/common/io.cc | 412 +++ cpp/src/plasma/common/io.h | 233 ++ cpp/src/plasma/doc/plasma-doxy-config | 2473 ++++++++++++++++ cpp/src/plasma/eviction_policy.cc | 215 ++ cpp/src/plasma/eviction_policy.h | 149 + cpp/src/plasma/fling.c | 76 + cpp/src/plasma/fling.h | 43 + cpp/src/plasma/format/plasma.fbs | 292 ++ cpp/src/plasma/format/plasma_generated.h | 2999 ++++++++++++++++++++ cpp/src/plasma/malloc.c | 169 ++ cpp/src/plasma/malloc.h | 9 + cpp/src/plasma/plasma.cc | 41 + cpp/src/plasma/plasma.h | 142 + cpp/src/plasma/plasma_client.cc | 767 +++++ cpp/src/plasma/plasma_client.h | 335 +++ cpp/src/plasma/plasma_extension.cc | 464 +++ cpp/src/plasma/plasma_extension.h | 15 + cpp/src/plasma/plasma_protocol.cc | 637 +++++ cpp/src/plasma/plasma_protocol.h | 285 ++ cpp/src/plasma/plasma_store.cc | 942 ++++++ cpp/src/plasma/plasma_store.h | 103 + cpp/src/plasma/setup-env.sh | 5 + cpp/src/plasma/setup.py | 34 + cpp/src/plasma/test/client_tests.cc | 311 ++ cpp/src/plasma/test/run_tests.sh | 35 + cpp/src/plasma/test/run_valgrind.sh | 10 + cpp/src/plasma/test/serialization_tests.cc | 442 +++ 36 files changed, 12655 insertions(+) create mode 100644 cpp/src/plasma/CMakeLists.txt create mode 100644 cpp/src/plasma/cmake/Common.cmake create mode 100644 cpp/src/plasma/common/CMakeLists.txt create mode 100644 cpp/src/plasma/common/common.cc create mode 100644 cpp/src/plasma/common/common.h create mode 100644 cpp/src/plasma/common/common_extension.cc create mode 100644 cpp/src/plasma/common/common_extension.h create mode 100644 cpp/src/plasma/common/event_loop.cc create mode 100644 cpp/src/plasma/common/event_loop.h create mode 100644 cpp/src/plasma/common/io.cc create mode 100644 cpp/src/plasma/common/io.h create mode 100644 cpp/src/plasma/doc/plasma-doxy-config create mode 100644 cpp/src/plasma/eviction_policy.cc create mode 100644 cpp/src/plasma/eviction_policy.h create mode 100644 cpp/src/plasma/fling.c create mode 100644 cpp/src/plasma/fling.h create mode 100644 cpp/src/plasma/format/plasma.fbs create mode 100644 cpp/src/plasma/format/plasma_generated.h create mode 100644 cpp/src/plasma/malloc.c create mode 100644 cpp/src/plasma/malloc.h create mode 100644 cpp/src/plasma/plasma.cc create mode 100644 cpp/src/plasma/plasma.h create mode 100644 cpp/src/plasma/plasma_client.cc create mode 100644 cpp/src/plasma/plasma_client.h create mode 100644 cpp/src/plasma/plasma_extension.cc create mode 100644 cpp/src/plasma/plasma_extension.h create mode 100644 cpp/src/plasma/plasma_protocol.cc create mode 100644 cpp/src/plasma/plasma_protocol.h create mode 100644 cpp/src/plasma/plasma_store.cc create mode 100644 cpp/src/plasma/plasma_store.h create mode 100644 cpp/src/plasma/setup-env.sh create mode 100644 cpp/src/plasma/setup.py create mode 100644 cpp/src/plasma/test/client_tests.cc create mode 100644 cpp/src/plasma/test/run_tests.sh create mode 100644 cpp/src/plasma/test/run_valgrind.sh create mode 100644 cpp/src/plasma/test/serialization_tests.cc diff --git a/cpp/src/plasma/CMakeLists.txt b/cpp/src/plasma/CMakeLists.txt new file mode 100644 index 00000000000..33a47183ccb --- /dev/null +++ b/cpp/src/plasma/CMakeLists.txt @@ -0,0 +1,116 @@ +cmake_minimum_required(VERSION 2.8) + +project(plasma) + +add_subdirectory(${CMAKE_CURRENT_LIST_DIR}/common/) + +# Recursively include common +include(${CMAKE_CURRENT_LIST_DIR}/cmake/Common.cmake) + +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11") + +if(APPLE) + SET(CMAKE_SHARED_LIBRARY_SUFFIX ".so") +endif(APPLE) + +include_directories("${PYTHON_INCLUDE_DIRS}" thirdparty) + +set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --std=c99 -D_XOPEN_SOURCE=500 -D_POSIX_C_SOURCE=200809L") + +# Compile flatbuffers + +set(PLASMA_FBS_SRC "${CMAKE_CURRENT_LIST_DIR}/format/plasma.fbs") +set(OUTPUT_DIR ${CMAKE_CURRENT_LIST_DIR}/format/) + +add_custom_target(gen_plasma_fbs ALL) + +add_custom_command( + TARGET gen_plasma_fbs + # The --gen-object-api flag generates a C++ class MessageT for each + # flatbuffers message Message, which can be used to store deserialized + # messages in data structures. This is currently used for ObjectInfo for + # example. + COMMAND ${FLATBUFFERS_COMPILER} -c -o ${OUTPUT_DIR} ${PLASMA_FBS_SRC} --gen-object-api + DEPENDS ${PLASMA_FBS_SRC} + COMMENT "Running flatc compiler on ${PLASMA_FBS_SRC}" + VERBATIM +) + +add_dependencies(gen_plasma_fbs flatbuffers_ep) + +if(UNIX AND NOT APPLE) + link_libraries(rt) +endif() + +include_directories("${CMAKE_CURRENT_LIST_DIR}/") +include_directories("${CMAKE_CURRENT_LIST_DIR}/common") +include_directories("${CMAKE_CURRENT_LIST_DIR}/common/thirdparty") + +add_library(plasma SHARED + plasma.cc + plasma_extension.cc + common/common_extension.cc + plasma_protocol.cc + plasma_client.cc + thirdparty/xxhash.c + fling.c) + +add_dependencies(plasma gen_plasma_fbs) + +get_filename_component(PYTHON_SHARED_LIBRARY ${PYTHON_LIBRARIES} NAME) +if(APPLE) + add_custom_command(TARGET plasma + POST_BUILD COMMAND ${CMAKE_INSTALL_NAME_TOOL} -change ${PYTHON_SHARED_LIBRARY} ${PYTHON_LIBRARIES} libplasma.so) +endif(APPLE) + +if(APPLE) + target_link_libraries(plasma -Wl,-force_load,${FLATBUFFERS_STATIC_LIB} common ${PYTHON_LIBRARIES} ${FLATBUFFERS_STATIC_LIB} -lpthread) +else(APPLE) + target_link_libraries(plasma -Wl,--whole-archive ${FLATBUFFERS_STATIC_LIB} -Wl,--no-whole-archive common ${PYTHON_LIBRARIES} ${FLATBUFFERS_STATIC_LIB} -lpthread) +endif(APPLE) + +include_directories("${FLATBUFFERS_INCLUDE_DIR}") + +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fPIC") + +set_source_files_properties(thirdparty/dlmalloc.c PROPERTIES COMPILE_FLAGS -Wno-all) + +add_executable(plasma_store + plasma_store.cc + plasma.cc + plasma_protocol.cc + eviction_policy.cc + fling.c + malloc.c) + +add_dependencies(plasma_store hiredis gen_plasma_fbs) + +target_link_libraries(plasma_store common ${FLATBUFFERS_STATIC_LIB}) + +add_library(plasma_lib STATIC + plasma_client.cc + plasma.cc + plasma_protocol.cc + fling.c + thirdparty/xxhash.c) + +target_link_libraries(plasma_lib common ${FLATBUFFERS_STATIC_LIB} -lpthread) +add_dependencies(plasma_lib gen_plasma_fbs) + +add_dependencies(plasma protocol_fbs) + +add_library(plasma_client SHARED plasma_client.cc) +target_link_libraries(plasma_client ${FLATBUFFERS_STATIC_LIB}) + +target_link_libraries(plasma_client common plasma_lib ${FLATBUFFERS_STATIC_LIB}) + +function(define_test test_name library) + add_executable(${test_name} test/${test_name}.cc ${ARGN}) + add_dependencies(${test_name} hiredis flatbuffers_ep) + target_link_libraries(${test_name} common ${FLATBUFFERS_STATIC_LIB} ${library}) + target_compile_options(${test_name} PUBLIC "-DPLASMA_TEST -DLOCAL_SCHEDULER_TEST -DCOMMON_TEST -DRAY_COMMON_LOG_LEVEL=4 -DRAY_TIMEOUT=50") +endfunction() + +# define_test(client_tests plasma_lib) +# define_test(manager_tests plasma_lib plasma_manager.cc) +# define_test(serialization_tests plasma_lib) diff --git a/cpp/src/plasma/cmake/Common.cmake b/cpp/src/plasma/cmake/Common.cmake new file mode 100644 index 00000000000..fa13b26a229 --- /dev/null +++ b/cpp/src/plasma/cmake/Common.cmake @@ -0,0 +1,95 @@ +# Code for compiling flatbuffers + +include(ExternalProject) +include(CMakeParseArguments) + +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11") + +set(FLATBUFFERS_VERSION "1.6.0") + +set(FLATBUFFERS_PREFIX "${CMAKE_BINARY_DIR}/flatbuffers_ep-prefix/src/flatbuffers_ep-install") +if (NOT TARGET flatbuffers_ep) + ExternalProject_Add(flatbuffers_ep + URL "https://github.com/google/flatbuffers/archive/v${FLATBUFFERS_VERSION}.tar.gz" + CMAKE_ARGS + "-DCMAKE_CXX_FLAGS=-fPIC" + "-DCMAKE_INSTALL_PREFIX:PATH=${FLATBUFFERS_PREFIX}" + "-DFLATBUFFERS_BUILD_TESTS=OFF") +endif() + +set(FLATBUFFERS_INCLUDE_DIR "${FLATBUFFERS_PREFIX}/include") +set(FLATBUFFERS_STATIC_LIB "${FLATBUFFERS_PREFIX}/lib/libflatbuffers.a") +set(FLATBUFFERS_COMPILER "${FLATBUFFERS_PREFIX}/bin/flatc") + +message(STATUS "Flatbuffers include dir: ${FLATBUFFERS_INCLUDE_DIR}") +message(STATUS "Flatbuffers static library: ${FLATBUFFERS_STATIC_LIB}") +message(STATUS "Flatbuffers compiler: ${FLATBUFFERS_COMPILER}") +include_directories(SYSTEM ${FLATBUFFERS_INCLUDE_DIR}) + +# Custom CFLAGS + +set(CMAKE_C_FLAGS "-g -Wall -Wextra -Werror=implicit-function-declaration -Wno-sign-compare -Wno-unused-parameter -Wno-type-limits -Wno-missing-field-initializers --std=c99 -D_XOPEN_SOURCE=500 -D_POSIX_C_SOURCE=200809L -fPIC -std=c99") + +# Code for finding Python + +message(STATUS "Trying custom approach for finding Python.") +# Start off by figuring out which Python executable to use. +find_program(CUSTOM_PYTHON_EXECUTABLE python) +message(STATUS "Found Python program: ${CUSTOM_PYTHON_EXECUTABLE}") +execute_process(COMMAND ${CUSTOM_PYTHON_EXECUTABLE} -c "import sys; print('python' + sys.version[0:3])" + OUTPUT_VARIABLE PYTHON_LIBRARY_NAME OUTPUT_STRIP_TRAILING_WHITESPACE) +message(STATUS "PYTHON_LIBRARY_NAME: " ${PYTHON_LIBRARY_NAME}) +# Now find the Python include directories. +execute_process(COMMAND ${CUSTOM_PYTHON_EXECUTABLE} -c "from distutils.sysconfig import *; print(get_python_inc())" + OUTPUT_VARIABLE PYTHON_INCLUDE_DIRS OUTPUT_STRIP_TRAILING_WHITESPACE) +message(STATUS "PYTHON_INCLUDE_DIRS: " ${PYTHON_INCLUDE_DIRS}) +# Now find the Python libraries. We'll start by looking near the Python +# executable. If that fails, then we'll look near the Python include +# directories. +execute_process(COMMAND ${CUSTOM_PYTHON_EXECUTABLE} -c "import sys; print(sys.exec_prefix)" + OUTPUT_VARIABLE PYTHON_PREFIX OUTPUT_STRIP_TRAILING_WHITESPACE) +message(STATUS "PYTHON_PREFIX: " ${PYTHON_PREFIX}) +# The name ending in "m" is for miniconda. +FIND_LIBRARY(PYTHON_LIBRARIES + NAMES "${PYTHON_LIBRARY_NAME}" "${PYTHON_LIBRARY_NAME}m" + HINTS "${PYTHON_PREFIX}" + PATH_SUFFIXES "lib" "libs" + NO_DEFAULT_PATH) +message(STATUS "PYTHON_LIBRARIES: " ${PYTHON_LIBRARIES}) +# If that failed, perhaps because the user is in a virtualenv, search around +# the Python include directories. +if(NOT PYTHON_LIBRARIES) + message(STATUS "Failed to find PYTHON_LIBRARIES near the Python executable, so now looking near the Python include directories.") + # The name ending in "m" is for miniconda. + FIND_LIBRARY(PYTHON_LIBRARIES + NAMES "${PYTHON_LIBRARY_NAME}" "${PYTHON_LIBRARY_NAME}m" + HINTS "${PYTHON_INCLUDE_DIRS}/../.." + PATH_SUFFIXES "lib" "libs" + NO_DEFAULT_PATH) + message(STATUS "PYTHON_LIBRARIES: " ${PYTHON_LIBRARIES}) +endif() +# If we found the Python libraries and the include directories, then continue +# on. If not, then try find_package as a last resort, but it probably won't +# work. +if(PYTHON_LIBRARIES AND PYTHON_INCLUDE_DIRS) + message(STATUS "The custom approach for finding Python succeeded.") + SET(PYTHONLIBS_FOUND TRUE) +else() + message(WARNING "The custom approach for finding Python failed. Defaulting to find_package.") + find_package(PythonInterp REQUIRED) + find_package(PythonLibs ${PYTHON_VERSION_STRING} EXACT REQUIRED) + set(CUSTOM_PYTHON_EXECUTABLE ${PYTHON_EXECUTABLE}) +endif() + +message(STATUS "Using CUSTOM_PYTHON_EXECUTABLE: " ${CUSTOM_PYTHON_EXECUTABLE}) +message(STATUS "Using PYTHON_LIBRARIES: " ${PYTHON_LIBRARIES}) +message(STATUS "Using PYTHON_INCLUDE_DIRS: " ${PYTHON_INCLUDE_DIRS}) + +# Common libraries + +set(COMMON_LIB "${CMAKE_BINARY_DIR}/src/common/libcommon.a" + CACHE STRING "Path to libcommon.a") + +include_directories("${CMAKE_CURRENT_LIST_DIR}/..") +include_directories("${CMAKE_CURRENT_LIST_DIR}/../thirdparty/") +include_directories("${CMAKE_CURRENT_LIST_DIR}/../lib/python") diff --git a/cpp/src/plasma/common/CMakeLists.txt b/cpp/src/plasma/common/CMakeLists.txt new file mode 100644 index 00000000000..dfa671c4700 --- /dev/null +++ b/cpp/src/plasma/common/CMakeLists.txt @@ -0,0 +1,31 @@ +cmake_minimum_required(VERSION 2.8) + +project(common) + +include(${CMAKE_CURRENT_LIST_DIR}/../cmake/Common.cmake) + +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fPIC") + +include_directories(thirdparty) + +add_library(common STATIC + event_loop.cc + common.cc + io.cc + thirdparty/ae/ae.c + thirdparty/sha256.c) + +function(define_test test_name library) + add_executable(${test_name} test/${test_name}.cc ${ARGN}) + add_dependencies(${test_name} hiredis flatbuffers_ep) + target_link_libraries(${test_name} common ${FLATBUFFERS_STATIC_LIB} ${library}) + target_compile_options(${test_name} PUBLIC "-DPLASMA_TEST -DLOCAL_SCHEDULER_TEST -DCOMMON_TEST -DRAY_COMMON_LOG_LEVEL=4 -DRAY_TIMEOUT=50") +endfunction() + +# define_test(common_tests "") +# define_test(db_tests "") +# define_test(io_tests "") +# define_test(task_tests "") +# define_test(redis_tests "") +# define_test(task_table_tests "") +# define_test(object_table_tests "") diff --git a/cpp/src/plasma/common/common.cc b/cpp/src/plasma/common/common.cc new file mode 100644 index 00000000000..35844733c01 --- /dev/null +++ b/cpp/src/plasma/common/common.cc @@ -0,0 +1,57 @@ +#include "common.h" + +#include +#include +#include +#include +#include + +#include "io.h" + +/* 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 = {{255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255}}; + +const unsigned char NIL_DIGEST[DIGEST_SIZE] = {0}; + +UniqueID globally_unique_id(void) { + /* Use /dev/urandom for "real" randomness. */ + int fd; + int const flags = 0 /* for Windows compatibility */; + if ((fd = open("/dev/urandom", O_RDONLY, flags)) == -1) { + LOG_ERROR("Could not generate random number"); + } + UniqueID result; + CHECK(read_bytes(fd, &result.id[0], UNIQUE_ID_SIZE) >= 0); + close(fd); + return result; +} + +bool ObjectID_equal(ObjectID first_id, ObjectID second_id) { + return UNIQUE_ID_EQ(first_id, second_id); +} + +bool ObjectID_is_nil(ObjectID id) { + return ObjectID_equal(id, NIL_OBJECT_ID); +} + +bool DBClientID_equal(DBClientID first_id, DBClientID second_id) { + return UNIQUE_ID_EQ(first_id, second_id); +} + +char *ObjectID_to_string(ObjectID obj_id, char *id_string, int id_length) { + CHECK(id_length >= ID_STRING_SIZE); + static const char hex[] = "0123456789abcdef"; + char *buf = id_string; + + for (int i = 0; i < UNIQUE_ID_SIZE; i++) { + unsigned int val = obj_id.id[i]; + *buf++ = hex[val >> 4]; + *buf++ = hex[val & 0xf]; + } + *buf = '\0'; + + return id_string; +} diff --git a/cpp/src/plasma/common/common.h b/cpp/src/plasma/common/common.h new file mode 100644 index 00000000000..398ab7fadb0 --- /dev/null +++ b/cpp/src/plasma/common/common.h @@ -0,0 +1,206 @@ +#ifndef COMMON_H +#define COMMON_H + +#include +#include +#include +#include +#include +#include +#ifndef _WIN32 +#include +#endif + +#include "utarray.h" +#ifdef __cplusplus +extern "C" { +#endif +#include "sha256.h" +#ifdef __cplusplus +} +#endif + +/** The duration between heartbeats. These are sent by the plasma manager and + * local scheduler. */ +#define HEARTBEAT_TIMEOUT_MILLISECONDS 100 +/** If a component has not sent a heartbeat in the last NUM_HEARTBEATS_TIMEOUT + * heartbeat intervals, the global scheduler or monitor process will report it + * as dead to the db_client table. */ +#define NUM_HEARTBEATS_TIMEOUT 100 + +/** 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 + +/* Arrow defines the same macro, only define it if it has not already been + * defined. */ +#ifndef UNUSED +#define UNUSED(x) ((void) (x)) +#endif + +/** + * 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 diff --git a/cpp/src/plasma/common/common_extension.cc b/cpp/src/plasma/common/common_extension.cc new file mode 100644 index 00000000000..1a198710a05 --- /dev/null +++ b/cpp/src/plasma/common/common_extension.cc @@ -0,0 +1,292 @@ +#include +#include "bytesobject.h" +#include "node.h" + +#include "common.h" +#include "common_extension.h" +#include "utarray.h" +#include "utstring.h" + +PyObject *CommonError; + +/* Initialize pickle module. */ + +PyObject *pickle_module = NULL; +PyObject *pickle_loads = NULL; +PyObject *pickle_dumps = NULL; +PyObject *pickle_protocol = NULL; + +void init_pickle_module(void) { +#if PY_MAJOR_VERSION >= 3 + pickle_module = PyImport_ImportModule("pickle"); +#else + pickle_module = PyImport_ImportModuleNoBlock("cPickle"); +#endif + CHECK(pickle_module != NULL); + CHECK(PyObject_HasAttrString(pickle_module, "loads")); + CHECK(PyObject_HasAttrString(pickle_module, "dumps")); + CHECK(PyObject_HasAttrString(pickle_module, "HIGHEST_PROTOCOL")); + pickle_loads = PyUnicode_FromString("loads"); + pickle_dumps = PyUnicode_FromString("dumps"); + pickle_protocol = PyObject_GetAttrString(pickle_module, "HIGHEST_PROTOCOL"); + CHECK(pickle_protocol != NULL); +} + +/* Define the PyObjectID class. */ + +int PyStringToUniqueID(PyObject *object, ObjectID *object_id) { + if (PyBytes_Check(object)) { + memcpy(&object_id->id[0], PyBytes_AsString(object), UNIQUE_ID_SIZE); + return 1; + } else { + PyErr_SetString(PyExc_TypeError, "must be a 20 character string"); + return 0; + } +} + +int PyObjectToUniqueID(PyObject *object, ObjectID *objectid) { + if (PyObject_IsInstance(object, (PyObject *) &PyObjectIDType)) { + *objectid = ((PyObjectID *) object)->object_id; + return 1; + } else { + PyErr_SetString(PyExc_TypeError, "must be an ObjectID"); + return 0; + } +} + +static int PyObjectID_init(PyObjectID *self, PyObject *args, PyObject *kwds) { + const char *data; + int size; + if (!PyArg_ParseTuple(args, "s#", &data, &size)) { + return -1; + } + if (size != sizeof(ObjectID)) { + PyErr_SetString(CommonError, + "ObjectID: object id string needs to have length 20"); + return -1; + } + memcpy(&self->object_id.id[0], data, sizeof(self->object_id.id)); + return 0; +} + +/* Create a PyObjectID from C. */ +PyObject *PyObjectID_make(ObjectID object_id) { + PyObjectID *result = PyObject_New(PyObjectID, &PyObjectIDType); + result = (PyObjectID *) PyObject_Init((PyObject *) result, &PyObjectIDType); + result->object_id = object_id; + return (PyObject *) result; +} + +static PyObject *PyObjectID_id(PyObject *self) { + PyObjectID *s = (PyObjectID *) self; + return PyBytes_FromStringAndSize((char *) &s->object_id.id[0], + sizeof(s->object_id.id)); +} + +static PyObject *PyObjectID_hex(PyObject *self) { + PyObjectID *s = (PyObjectID *) self; + char hex_id[ID_STRING_SIZE]; + ObjectID_to_string(s->object_id, hex_id, ID_STRING_SIZE); + PyObject *result = PyUnicode_FromString(hex_id); + return result; +} + +static PyObject *PyObjectID_richcompare(PyObjectID *self, + PyObject *other, + int op) { + PyObject *result = NULL; + if (Py_TYPE(self)->tp_richcompare != Py_TYPE(other)->tp_richcompare) { + result = Py_NotImplemented; + } else { + PyObjectID *other_id = (PyObjectID *) other; + switch (op) { + case Py_LT: + result = Py_NotImplemented; + break; + case Py_LE: + result = Py_NotImplemented; + break; + case Py_EQ: + result = ObjectID_equal(self->object_id, other_id->object_id) ? Py_True + : Py_False; + break; + case Py_NE: + result = !ObjectID_equal(self->object_id, other_id->object_id) ? Py_True + : Py_False; + break; + case Py_GT: + result = Py_NotImplemented; + break; + case Py_GE: + result = Py_NotImplemented; + break; + } + } + Py_XINCREF(result); + return result; +} + +static long PyObjectID_hash(PyObjectID *self) { + PyObject *tuple = PyTuple_New(UNIQUE_ID_SIZE); + for (int i = 0; i < UNIQUE_ID_SIZE; ++i) { + PyTuple_SetItem(tuple, i, PyLong_FromLong(self->object_id.id[i])); + } + long hash = PyObject_Hash(tuple); + Py_XDECREF(tuple); + return hash; +} + +static PyObject *PyObjectID_repr(PyObjectID *self) { + char hex_id[ID_STRING_SIZE]; + ObjectID_to_string(self->object_id, hex_id, ID_STRING_SIZE); + UT_string *repr; + utstring_new(repr); + utstring_printf(repr, "ObjectID(%s)", hex_id); + PyObject *result = PyUnicode_FromString(utstring_body(repr)); + utstring_free(repr); + return result; +} + +static PyObject *PyObjectID___reduce__(PyObjectID *self) { + PyErr_SetString(CommonError, "ObjectID objects cannot be serialized."); + return NULL; +} + +static PyMethodDef PyObjectID_methods[] = { + {"id", (PyCFunction) PyObjectID_id, METH_NOARGS, + "Return the hash associated with this ObjectID"}, + {"hex", (PyCFunction) PyObjectID_hex, METH_NOARGS, + "Return the object ID as a string in hex."}, + {"__reduce__", (PyCFunction) PyObjectID___reduce__, METH_NOARGS, + "Say how to pickle this ObjectID. This raises an exception to prevent" + "object IDs from being serialized."}, + {NULL} /* Sentinel */ +}; + +static PyMemberDef PyObjectID_members[] = { + {NULL} /* Sentinel */ +}; + +PyTypeObject PyObjectIDType = { + PyVarObject_HEAD_INIT(NULL, 0) /* ob_size */ + "common.ObjectID", /* tp_name */ + sizeof(PyObjectID), /* tp_basicsize */ + 0, /* tp_itemsize */ + 0, /* tp_dealloc */ + 0, /* tp_print */ + 0, /* tp_getattr */ + 0, /* tp_setattr */ + 0, /* tp_compare */ + (reprfunc) PyObjectID_repr, /* tp_repr */ + 0, /* tp_as_number */ + 0, /* tp_as_sequence */ + 0, /* tp_as_mapping */ + (hashfunc) PyObjectID_hash, /* tp_hash */ + 0, /* tp_call */ + 0, /* tp_str */ + 0, /* tp_getattro */ + 0, /* tp_setattro */ + 0, /* tp_as_buffer */ + Py_TPFLAGS_DEFAULT, /* tp_flags */ + "ObjectID object", /* tp_doc */ + 0, /* tp_traverse */ + 0, /* tp_clear */ + (richcmpfunc) PyObjectID_richcompare, /* tp_richcompare */ + 0, /* tp_weaklistoffset */ + 0, /* tp_iter */ + 0, /* tp_iternext */ + PyObjectID_methods, /* tp_methods */ + PyObjectID_members, /* tp_members */ + 0, /* tp_getset */ + 0, /* tp_base */ + 0, /* tp_dict */ + 0, /* tp_descr_get */ + 0, /* tp_descr_set */ + 0, /* tp_dictoffset */ + (initproc) PyObjectID_init, /* tp_init */ + 0, /* tp_alloc */ + PyType_GenericNew, /* tp_new */ +}; + +/* Define the methods for the module. */ + +#define SIZE_LIMIT 100 +#define NUM_ELEMENTS_LIMIT 1000 + +#if PY_MAJOR_VERSION >= 3 +#define PyInt_Check PyLong_Check +#endif + +/** + * This method checks if a Python object is sufficiently simple that it can be + * serialized and passed by value as an argument to a task (without being put in + * the object store). The details of which objects are sufficiently simple are + * defined by this method and are not particularly important. But for + * performance reasons, it is better to place "small" objects in the task itself + * and "large" objects in the object store. + * + * @param value The Python object in question. + * @param num_elements_contained If this method returns 1, then the number of + * objects recursively contained within this object will be added to the + * value at this address. This is used to make sure that we do not + * serialize objects that are too large. + * @return 0 if the object cannot be serialized in the task and 1 if it can. + */ +int is_simple_value(PyObject *value, int *num_elements_contained) { + *num_elements_contained += 1; + if (*num_elements_contained >= NUM_ELEMENTS_LIMIT) { + return 0; + } + if (PyInt_Check(value) || PyLong_Check(value) || value == Py_False || + value == Py_True || PyFloat_Check(value) || value == Py_None) { + return 1; + } + if (PyBytes_CheckExact(value)) { + *num_elements_contained += PyBytes_Size(value); + return (*num_elements_contained < NUM_ELEMENTS_LIMIT); + } + if (PyUnicode_CheckExact(value)) { + *num_elements_contained += PyUnicode_GET_SIZE(value); + return (*num_elements_contained < NUM_ELEMENTS_LIMIT); + } + if (PyList_CheckExact(value) && PyList_Size(value) < SIZE_LIMIT) { + for (Py_ssize_t i = 0; i < PyList_Size(value); ++i) { + if (!is_simple_value(PyList_GetItem(value, i), num_elements_contained)) { + return 0; + } + } + return (*num_elements_contained < NUM_ELEMENTS_LIMIT); + } + if (PyDict_CheckExact(value) && PyDict_Size(value) < SIZE_LIMIT) { + PyObject *key, *val; + Py_ssize_t pos = 0; + while (PyDict_Next(value, &pos, &key, &val)) { + if (!is_simple_value(key, num_elements_contained) || + !is_simple_value(val, num_elements_contained)) { + return 0; + } + } + return (*num_elements_contained < NUM_ELEMENTS_LIMIT); + } + if (PyTuple_CheckExact(value) && PyTuple_Size(value) < SIZE_LIMIT) { + for (Py_ssize_t i = 0; i < PyTuple_Size(value); ++i) { + if (!is_simple_value(PyTuple_GetItem(value, i), num_elements_contained)) { + return 0; + } + } + return (*num_elements_contained < NUM_ELEMENTS_LIMIT); + } + return 0; +} + +PyObject *check_simple_value(PyObject *self, PyObject *args) { + PyObject *value; + if (!PyArg_ParseTuple(args, "O", &value)) { + return NULL; + } + int num_elements_contained = 0; + if (is_simple_value(value, &num_elements_contained)) { + Py_RETURN_TRUE; + } + Py_RETURN_FALSE; +} diff --git a/cpp/src/plasma/common/common_extension.h b/cpp/src/plasma/common/common_extension.h new file mode 100644 index 00000000000..c91b631cd33 --- /dev/null +++ b/cpp/src/plasma/common/common_extension.h @@ -0,0 +1,56 @@ +#ifndef COMMON_EXTENSION_H +#define COMMON_EXTENSION_H + +#include +#include "marshal.h" +#include "structmember.h" + +#include "common.h" + +typedef uint8_t TaskSpec; +struct TaskBuilder; + +extern PyObject *CommonError; + +// clang-format off +typedef struct { + PyObject_HEAD + ObjectID object_id; +} PyObjectID; + +typedef struct { + PyObject_HEAD + int64_t size; + TaskSpec *spec; +} PyTask; +// clang-format on + +extern PyTypeObject PyObjectIDType; + +extern PyTypeObject PyTaskType; + +/* Python module for pickling. */ +extern PyObject *pickle_module; +extern PyObject *pickle_dumps; +extern PyObject *pickle_loads; + +void init_pickle_module(void); + +extern TaskBuilder *g_task_builder; + +int PyStringToUniqueID(PyObject *object, ObjectID *object_id); + +int PyObjectToUniqueID(PyObject *object, ObjectID *object_id); + +PyObject *PyObjectID_make(ObjectID object_id); + +PyObject *check_simple_value(PyObject *self, PyObject *args); + +PyObject *PyTask_to_string(PyObject *, PyObject *args); +PyObject *PyTask_from_string(PyObject *, PyObject *args); + +PyObject *compute_put_id(PyObject *self, PyObject *args); + +PyObject *PyTask_make(TaskSpec *task_spec, int64_t task_size); + +#endif /* COMMON_EXTENSION_H */ diff --git a/cpp/src/plasma/common/event_loop.cc b/cpp/src/plasma/common/event_loop.cc new file mode 100644 index 00000000000..fa5b2030fde --- /dev/null +++ b/cpp/src/plasma/common/event_loop.cc @@ -0,0 +1,61 @@ +#include "event_loop.h" + +#include "common.h" +#include + +#define INITIAL_EVENT_LOOP_SIZE 1024 + +event_loop *event_loop_create(void) { + return aeCreateEventLoop(INITIAL_EVENT_LOOP_SIZE); +} + +void event_loop_destroy(event_loop *loop) { + /* Clean up timer events. This is to make valgrind happy. */ + aeTimeEvent *te = loop->timeEventHead; + while (te) { + aeTimeEvent *next = te->next; + free(te); + te = next; + } + aeDeleteEventLoop(loop); +} + +void event_loop_add_file(event_loop *loop, + int fd, + int events, + event_loop_file_handler handler, + void *context) { + /* Try to add the file descriptor. */ + int err = aeCreateFileEvent(loop, fd, events, handler, context); + /* If it cannot be added, increase the size of the event loop. */ + if (err == AE_ERR && errno == ERANGE) { + err = aeResizeSetSize(loop, 3 * aeGetSetSize(loop) / 2); + CHECK(err == AE_OK); + err = aeCreateFileEvent(loop, fd, events, handler, context); + } + /* In any case, test if there were errors. */ + CHECK(err == AE_OK); +} + +void event_loop_remove_file(event_loop *loop, int fd) { + aeDeleteFileEvent(loop, fd, EVENT_LOOP_READ | EVENT_LOOP_WRITE); +} + +int64_t event_loop_add_timer(event_loop *loop, + int64_t timeout, + event_loop_timer_handler handler, + void *context) { + return aeCreateTimeEvent(loop, timeout, handler, context, NULL); +} + +int event_loop_remove_timer(event_loop *loop, int64_t id) { + return aeDeleteTimeEvent(loop, id); +} + +void event_loop_run(event_loop *loop) { + aeMain(loop); +} + +void event_loop_stop(event_loop *loop) { + aeStop(loop); +} diff --git a/cpp/src/plasma/common/event_loop.h b/cpp/src/plasma/common/event_loop.h new file mode 100644 index 00000000000..90e10a9725c --- /dev/null +++ b/cpp/src/plasma/common/event_loop.h @@ -0,0 +1,103 @@ +#ifndef EVENT_LOOP_H +#define EVENT_LOOP_H + +#include + +extern "C" { +#ifdef _WIN32 +/* Quirks mean that Windows version needs to be included differently */ +#include +#include +#else +#include "ae/ae.h" +#endif +} + +/* Unique timer ID that will be generated when the timer is added to the + * event loop. Will not be reused later on in another call + * to event_loop_add_timer. */ +typedef long long timer_id; + +typedef aeEventLoop event_loop; + +/* File descriptor is readable. */ +#define EVENT_LOOP_READ AE_READABLE + +/* File descriptor is writable. */ +#define EVENT_LOOP_WRITE AE_WRITABLE + +/* Constant specifying that the timer is done and it will be removed. */ +#define EVENT_LOOP_TIMER_DONE AE_NOMORE + +/* Signature of the handler that will be called when there is a new event + * on the file descriptor that this handler has been registered for. The + * context is the one that was passed into add_file by the user. The + * events parameter indicates which event is available on the file, + * it can be EVENT_LOOP_READ or EVENT_LOOP_WRITE. */ +typedef void (*event_loop_file_handler)(event_loop *loop, + int fd, + void *context, + int events); + +/* This handler will be called when a timer times out. The id of the timer + * as well as the context that was specified when registering this handler + * are passed as arguments. The return is the number of milliseconds the + * timer shall be reset to or EVENT_LOOP_TIMER_DONE if the timer shall + * not be triggered again. */ +typedef int (*event_loop_timer_handler)(event_loop *loop, + timer_id timer_id, + void *context); + +/* Create and return a new event loop. */ +event_loop *event_loop_create(void); + +/* Deallocate space associated with the event loop that was created + * with the "create" function. */ +void event_loop_destroy(event_loop *loop); + +/* Register a handler that will be called any time a new event happens on + * a file descriptor. Can specify a context that will be passed as an + * argument to the handler. Currently there can only be one handler per file. + * The events parameter specifies which events we listen to: EVENT_LOOP_READ + * or EVENT_LOOP_WRITE. */ +void event_loop_add_file(event_loop *loop, + int fd, + int events, + event_loop_file_handler handler, + void *context); + +/* Remove a registered file event handler from the event loop. */ +void event_loop_remove_file(event_loop *loop, int fd); + +/** Register a handler that will be called after a time slice of + * "timeout" milliseconds. + * + * @param loop The event loop. + * @param timeout The timeout in milliseconds. + * @param handler The handler for the timeout. + * @param context User context that can be passed in and will be passed in + * as an argument for the timer handler. + * @return The ID of the timer. + */ +int64_t event_loop_add_timer(event_loop *loop, + int64_t timeout, + event_loop_timer_handler handler, + void *context); + +/** + * Remove a registered time event handler from the event loop. Can be called + * multiple times on the same timer. + * + * @param loop The event loop. + * @param timer_id The ID of the timer to be removed. + * @return Returns 0 if the removal was successful. + */ +int event_loop_remove_timer(event_loop *loop, int64_t timer_id); + +/* Run the event loop. */ +void event_loop_run(event_loop *loop); + +/* Stop the event loop. */ +void event_loop_stop(event_loop *loop); + +#endif diff --git a/cpp/src/plasma/common/io.cc b/cpp/src/plasma/common/io.cc new file mode 100644 index 00000000000..25944ee2691 --- /dev/null +++ b/cpp/src/plasma/common/io.cc @@ -0,0 +1,412 @@ +#include "io.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common.h" +#include "event_loop.h" + +#ifndef _WIN32 +/* This function is actually not declared in standard POSIX, so declare it. */ +extern int usleep(useconds_t usec); +#endif + +int bind_inet_sock(const int port, bool shall_listen) { + struct sockaddr_in name; + int socket_fd = socket(PF_INET, SOCK_STREAM, 0); + if (socket_fd < 0) { + LOG_ERROR("socket() failed for port %d.", port); + return -1; + } + name.sin_family = AF_INET; + name.sin_port = htons(port); + name.sin_addr.s_addr = htonl(INADDR_ANY); + int on = 1; + /* TODO(pcm): http://stackoverflow.com/q/1150635 */ + if (ioctl(socket_fd, FIONBIO, (char *) &on) < 0) { + LOG_ERROR("ioctl failed"); + close(socket_fd); + return -1; + } + int *const pon = (int *const) & on; + if (setsockopt(socket_fd, SOL_SOCKET, SO_REUSEADDR, pon, sizeof(on)) < 0) { + LOG_ERROR("setsockopt failed for port %d", port); + close(socket_fd); + return -1; + } + if (bind(socket_fd, (struct sockaddr *) &name, sizeof(name)) < 0) { + LOG_ERROR("Bind failed for port %d", port); + close(socket_fd); + return -1; + } + if (shall_listen && listen(socket_fd, 5) == -1) { + LOG_ERROR("Could not listen to socket %d", port); + close(socket_fd); + return -1; + } + return socket_fd; +} + +int bind_ipc_sock(const char *socket_pathname, bool shall_listen) { + struct sockaddr_un socket_address; + int socket_fd = socket(AF_UNIX, SOCK_STREAM, 0); + if (socket_fd < 0) { + LOG_ERROR("socket() failed for pathname %s.", socket_pathname); + return -1; + } + /* Tell the system to allow the port to be reused. */ + int on = 1; + if (setsockopt(socket_fd, SOL_SOCKET, SO_REUSEADDR, (char *) &on, + sizeof(on)) < 0) { + LOG_ERROR("setsockopt failed for pathname %s", socket_pathname); + close(socket_fd); + return -1; + } + + unlink(socket_pathname); + memset(&socket_address, 0, sizeof(socket_address)); + socket_address.sun_family = AF_UNIX; + if (strlen(socket_pathname) + 1 > sizeof(socket_address.sun_path)) { + LOG_ERROR("Socket pathname is too long."); + close(socket_fd); + return -1; + } + strncpy(socket_address.sun_path, socket_pathname, + strlen(socket_pathname) + 1); + + if (bind(socket_fd, (struct sockaddr *) &socket_address, + sizeof(socket_address)) != 0) { + LOG_ERROR("Bind failed for pathname %s.", socket_pathname); + close(socket_fd); + return -1; + } + if (shall_listen && listen(socket_fd, 5) == -1) { + LOG_ERROR("Could not listen to socket %s", socket_pathname); + close(socket_fd); + return -1; + } + return socket_fd; +} + +int connect_ipc_sock_retry(const char *socket_pathname, + int num_retries, + int64_t timeout) { + /* Pick the default values if the user did not specify. */ + if (num_retries < 0) { + num_retries = NUM_CONNECT_ATTEMPTS; + } + if (timeout < 0) { + timeout = CONNECT_TIMEOUT_MS; + } + + CHECK(socket_pathname); + int fd = -1; + for (int num_attempts = 0; num_attempts < num_retries; ++num_attempts) { + fd = connect_ipc_sock(socket_pathname); + if (fd >= 0) { + break; + } + if (num_attempts == 0) { + LOG_ERROR("Connection to socket failed for pathname %s.", + socket_pathname); + } + /* Sleep for timeout milliseconds. */ + usleep(timeout * 1000); + } + /* If we could not connect to the socket, exit. */ + if (fd == -1) { + LOG_FATAL("Could not connect to socket %s", socket_pathname); + } + return fd; +} + +int connect_ipc_sock(const char *socket_pathname) { + struct sockaddr_un socket_address; + int socket_fd; + + socket_fd = socket(AF_UNIX, SOCK_STREAM, 0); + if (socket_fd < 0) { + LOG_ERROR("socket() failed for pathname %s.", socket_pathname); + return -1; + } + + memset(&socket_address, 0, sizeof(socket_address)); + socket_address.sun_family = AF_UNIX; + if (strlen(socket_pathname) + 1 > sizeof(socket_address.sun_path)) { + LOG_ERROR("Socket pathname is too long."); + return -1; + } + strncpy(socket_address.sun_path, socket_pathname, + strlen(socket_pathname) + 1); + + if (connect(socket_fd, (struct sockaddr *) &socket_address, + sizeof(socket_address)) != 0) { + close(socket_fd); + return -1; + } + + return socket_fd; +} + +int connect_inet_sock_retry(const char *ip_addr, + int port, + int num_retries, + int64_t timeout) { + /* Pick the default values if the user did not specify. */ + if (num_retries < 0) { + num_retries = NUM_CONNECT_ATTEMPTS; + } + if (timeout < 0) { + timeout = CONNECT_TIMEOUT_MS; + } + + CHECK(ip_addr); + int fd = -1; + for (int num_attempts = 0; num_attempts < num_retries; ++num_attempts) { + fd = connect_inet_sock(ip_addr, port); + if (fd >= 0) { + break; + } + if (num_attempts == 0) { + LOG_ERROR("Connection to socket failed for address %s:%d.", ip_addr, + port); + } + /* Sleep for timeout milliseconds. */ + usleep(timeout * 1000); + } + /* If we could not connect to the socket, exit. */ + if (fd == -1) { + LOG_FATAL("Could not connect to address %s:%d", ip_addr, port); + } + return fd; +} + +int connect_inet_sock(const char *ip_addr, int port) { + int fd = socket(PF_INET, SOCK_STREAM, 0); + if (fd < 0) { + LOG_ERROR("socket() failed for address %s:%d.", ip_addr, port); + return -1; + } + + struct hostent *manager = gethostbyname(ip_addr); /* TODO(pcm): cache this */ + if (!manager) { + LOG_ERROR("Failed to get hostname from address %s:%d.", ip_addr, port); + close(fd); + return -1; + } + + struct sockaddr_in addr; + addr.sin_family = AF_INET; + memcpy(&addr.sin_addr.s_addr, manager->h_addr_list[0], manager->h_length); + addr.sin_port = htons(port); + + if (connect(fd, (struct sockaddr *) &addr, sizeof(addr)) != 0) { + close(fd); + return -1; + } + return fd; +} + +int accept_client(int socket_fd) { + int client_fd = accept(socket_fd, NULL, NULL); + if (client_fd < 0) { + LOG_ERROR("Error reading from socket."); + return -1; + } + return client_fd; +} + +int write_bytes(int fd, uint8_t *cursor, size_t length) { + ssize_t nbytes = 0; + size_t bytesleft = length; + size_t offset = 0; + while (bytesleft > 0) { + /* While we haven't written the whole message, write to the file + * descriptor, advance the cursor, and decrease the amount left to write. */ + nbytes = write(fd, cursor + offset, bytesleft); + if (nbytes < 0) { + if (errno == EAGAIN || errno == EWOULDBLOCK || errno == EINTR) { + continue; + } + return -1; /* Errno will be set. */ + } else if (0 == nbytes) { + /* Encountered early EOF. */ + return -1; + } + CHECK(nbytes > 0); + bytesleft -= nbytes; + offset += nbytes; + } + + return 0; +} + +int write_message(int fd, int64_t type, int64_t length, uint8_t *bytes) { + int64_t version = RAY_PROTOCOL_VERSION; + int closed; + closed = write_bytes(fd, (uint8_t *) &version, sizeof(version)); + if (closed) { + return closed; + } + closed = write_bytes(fd, (uint8_t *) &type, sizeof(type)); + if (closed) { + return closed; + } + closed = write_bytes(fd, (uint8_t *) &length, sizeof(length)); + if (closed) { + return closed; + } + closed = write_bytes(fd, bytes, length * sizeof(char)); + if (closed) { + return closed; + } + return 0; +} + +int read_bytes(int fd, uint8_t *cursor, size_t length) { + ssize_t nbytes = 0; + /* Termination condition: EOF or read 'length' bytes total. */ + size_t bytesleft = length; + size_t offset = 0; + while (bytesleft > 0) { + nbytes = read(fd, cursor + offset, bytesleft); + if (nbytes < 0) { + if (errno == EAGAIN || errno == EWOULDBLOCK || errno == EINTR) { + continue; + } + return -1; /* Errno will be set. */ + } else if (0 == nbytes) { + /* Encountered early EOF. */ + return -1; + } + CHECK(nbytes > 0); + bytesleft -= nbytes; + offset += nbytes; + } + + return 0; +} + +void read_message(int fd, int64_t *type, int64_t *length, uint8_t **bytes) { + int64_t version; + int closed = read_bytes(fd, (uint8_t *) &version, sizeof(version)); + if (closed) { + goto disconnected; + } + CHECK(version == RAY_PROTOCOL_VERSION); + closed = read_bytes(fd, (uint8_t *) type, sizeof(*type)); + if (closed) { + goto disconnected; + } + closed = read_bytes(fd, (uint8_t *) length, sizeof(*length)); + if (closed) { + goto disconnected; + } + *bytes = (uint8_t *) malloc(*length * sizeof(uint8_t)); + closed = read_bytes(fd, *bytes, *length); + if (closed) { + free(*bytes); + goto disconnected; + } + return; + +disconnected: + /* Handle the case in which the socket is closed. */ + *type = DISCONNECT_CLIENT; + *length = 0; + *bytes = NULL; + return; +} + +uint8_t *read_message_async(event_loop *loop, int sock) { + int64_t size; + int error = read_bytes(sock, (uint8_t *) &size, sizeof(int64_t)); + if (error < 0) { + /* The other side has closed the socket. */ + LOG_DEBUG("Socket has been closed, or some other error has occurred."); + if (loop != NULL) { + event_loop_remove_file(loop, sock); + } + close(sock); + return NULL; + } + uint8_t *message = (uint8_t *) malloc(size); + error = read_bytes(sock, message, size); + if (error < 0) { + /* The other side has closed the socket. */ + LOG_DEBUG("Socket has been closed, or some other error has occurred."); + if (loop != NULL) { + event_loop_remove_file(loop, sock); + } + close(sock); + return NULL; + } + return message; +} + +int64_t read_buffer(int fd, int64_t *type, UT_array *buffer) { + int64_t version; + int closed = read_bytes(fd, (uint8_t *) &version, sizeof(version)); + if (closed) { + goto disconnected; + } + CHECK(version == RAY_PROTOCOL_VERSION); + int64_t length; + closed = read_bytes(fd, (uint8_t *) type, sizeof(*type)); + if (closed) { + goto disconnected; + } + closed = read_bytes(fd, (uint8_t *) &length, sizeof(length)); + if (closed) { + goto disconnected; + } + if (length > utarray_len(buffer)) { + utarray_resize(buffer, length); + } + closed = read_bytes(fd, (uint8_t *) utarray_front(buffer), length); + if (closed) { + goto disconnected; + } + return length; +disconnected: + /* Handle the case in which the socket is closed. */ + *type = DISCONNECT_CLIENT; + return 0; +} + +void write_log_message(int fd, const char *message) { + /* Account for the \0 at the end of the string. */ + write_message(fd, LOG_MESSAGE, strlen(message) + 1, (uint8_t *) message); +} + +char *read_log_message(int fd) { + uint8_t *bytes; + int64_t type; + int64_t length; + read_message(fd, &type, &length, &bytes); + CHECK(type == LOG_MESSAGE); + return (char *) bytes; +} + +void write_formatted_log_message(int socket_fd, const char *format, ...) { + UT_string *cmd; + va_list ap; + + utstring_new(cmd); + va_start(ap, format); + utstring_printf_va(cmd, format, ap); + va_end(ap); + + write_log_message(socket_fd, utstring_body(cmd)); + utstring_free(cmd); +} diff --git a/cpp/src/plasma/common/io.h b/cpp/src/plasma/common/io.h new file mode 100644 index 00000000000..cf9f16aa0f5 --- /dev/null +++ b/cpp/src/plasma/common/io.h @@ -0,0 +1,233 @@ +#ifndef IO_H +#define IO_H + +#include +#include + +#include "utarray.h" + +#define RAY_PROTOCOL_VERSION 0x0000000000000000 + +/* Number of times we try binding to a socket. */ +#define NUM_BIND_ATTEMPTS 5 +#define BIND_TIMEOUT_MS 100 + +/* Number of times we try connecting to a socket. */ +#define NUM_CONNECT_ATTEMPTS 50 +#define CONNECT_TIMEOUT_MS 100 + +struct aeEventLoop; +typedef aeEventLoop event_loop; + +enum common_message_type { + /** Disconnect a client. */ + DISCONNECT_CLIENT, + /** Log a message from a client. */ + LOG_MESSAGE, + /** Submit a task to the local scheduler. */ + SUBMIT_TASK, +}; + +/* Helper functions for socket communication. */ + +/** + * Binds to an Internet socket at the given port. Removes any existing file at + * the pathname. Returns a non-blocking file descriptor for the socket, or -1 + * if an error occurred. + * + * @note Since the returned file descriptor is non-blocking, it is not + * recommended to use the Linux read and write calls directly, since these + * might read or write a partial message. Instead, use the provided + * write_message and read_message methods. + * + * @param port The port to bind to. + * @param shall_listen Are we also starting to listen on the socket? + * @return A non-blocking file descriptor for the socket, or -1 if an error + * occurs. + */ +int bind_inet_sock(const int port, bool shall_listen); + +/** + * Binds to a Unix domain streaming socket at the given + * pathname. Removes any existing file at the pathname. + * + * @param socket_pathname The pathname for the socket. + * @param shall_listen Are we also starting to listen on the socket? + * @return A blocking file descriptor for the socket, or -1 if an error + * occurs. + */ +int bind_ipc_sock(const char *socket_pathname, bool shall_listen); + +/** + * Connect to a Unix domain streaming socket at the given + * pathname. + * + * @param socket_pathname The pathname for the socket. + * @return A file descriptor for the socket, or -1 if an error occurred. + */ +int connect_ipc_sock(const char *socket_pathname); + +/** + * Connect to a Unix domain streaming socket at the given + * pathname, or fail after some number of retries. + * + * @param socket_pathname The pathname for the socket. + * @param num_retries The number of times to retry the connection + * before exiting. If -1 is provided, then this defaults to + * NUM_CONNECT_ATTEMPTS. + * @param timeout The number of milliseconds to wait in between + * retries. If -1 is provided, then this defaults to CONNECT_TIMEOUT_MS. + * @return A file descriptor for the socket, or -1 if an error occurred. + */ +int connect_ipc_sock_retry(const char *socket_pathname, + int num_retries, + int64_t timeout); + +/** + * Connect to an Internet socket at the given address and port. + * + * @param ip_addr The IP address to connect to. + * @param port The port number to connect to. + * + * @param socket_pathname The pathname for the socket. + * @return A file descriptor for the socket, or -1 if an error occurred. + */ +int connect_inet_sock(const char *ip_addr, int port); + +/** + * Connect to an Internet socket at the given address and port, or fail after + * some number of retries. + * + * @param ip_addr The IP address to connect to. + * @param port The port number to connect to. + * @param num_retries The number of times to retry the connection + * before exiting. If -1 is provided, then this defaults to + * NUM_CONNECT_ATTEMPTS. + * @param timeout The number of milliseconds to wait in between + * retries. If -1 is provided, then this defaults to CONNECT_TIMEOUT_MS. + * @return A file descriptor for the socket, or -1 if an error occurred. + */ +int connect_inet_sock_retry(const char *ip_addr, + int port, + int num_retries, + int64_t timeout); + +/** + * Accept a new client connection on the given socket + * descriptor. Returns a descriptor for the new socket. + */ +int accept_client(int socket_fd); + +/* Reading and writing data. */ + +/** + * Write a sequence of bytes on a file descriptor. The bytes should then be read + * by read_message. + * + * @param fd The file descriptor to write to. It can be non-blocking. + * @param version The protocol version. + * @param type The type of the message to send. + * @param length The size in bytes of the bytes parameter. + * @param bytes The address of the message to send. + * @return int Whether there was an error while writing. 0 corresponds to + * success and -1 corresponds to an error (errno will be set). + */ +int write_message(int fd, int64_t type, int64_t length, uint8_t *bytes); + +/** + * Read a sequence of bytes written by write_message from a file descriptor. + * This allocates space for the message. + * + * @note The caller must free the memory. + * + * @param fd The file descriptor to read from. It can be non-blocking. + * @param type The type of the message that is read will be written at this + * address. If there was an error while reading, this will be + * DISCONNECT_CLIENT. + * @param length The size in bytes of the message that is read will be written + * at this address. This size does not include the bytes used to encode + * the type and length. If there was an error while reading, this will + * be 0. + * @param bytes The address at which to write the pointer to the bytes that are + * read and allocated by this function. If there was an error while + * reading, this will be NULL. + * @return Void. + */ +void read_message(int fd, int64_t *type, int64_t *length, uint8_t **bytes); + +/** + * Read a message from a file descriptor and remove the file descriptor from the + * event loop if there is an error. This will actually do two reads. The first + * read reads sizeof(int64_t) bytes to determine the number of bytes to read in + * the next read. + * + * @param loop: The event loop. + * @param sock: The file descriptor to read from. + * @return A byte buffer contining the message or NULL if there was an + * error. The buffer needs to be freed by the user. + */ +uint8_t *read_message_async(event_loop *loop, int sock); + +/** + * Read a sequence of bytes written by write_message from a file descriptor. + * This does not allocate space for the message if the provided buffer is + * large enough and can therefore often avoid allocations. + * + * @note The caller must create and free the buffer. + * + * @param fd The file descriptor to read from. It can be non-blocking. + * @param type The type of the message that is read will be written at this + * address. If there was an error while reading, this will be + * DISCONNECT_CLIENT. + * @param buffer The array the message will be written to. If it is not + * large enough to hold the message, it will be enlarged by read_buffer. + * @return Number of bytes of the message that were read. This size does not + * include the bytes used to encode the type and length. If there was + * an error while reading, this will be 0. + */ +int64_t read_buffer(int fd, int64_t *type, UT_array *buffer); + +/** + * Write a null-terminated string to a file descriptor. + */ +void write_log_message(int fd, const char *message); + +void write_formatted_log_message(int fd, const char *format, ...); + +/** + * Reads a null-terminated string from the file descriptor that has been + * written by write_log_message. Allocates and returns a pointer to the string. + * NOTE: Caller must free the memory! + */ +char *read_log_message(int fd); + +/** + * Read a sequence of bytes from a file descriptor into a buffer. This will + * block until one of the following happens: (1) there is an error (2) end of + * file, or (3) all length bytes have been written. + * + * @note The buffer pointed to by cursor must already have length number of + * bytes allocated before calling this method. + * + * @param fd The file descriptor to read from. It can be non-blocking. + * @param cursor The cursor pointing to the beginning of the buffer. + * @param length The size of the byte sequence to read. + * @return int Whether there was an error while reading. 0 corresponds to + * success and -1 corresponds to an error (errno will be set). + */ +int read_bytes(int fd, uint8_t *cursor, size_t length); + +/** + * Write a sequence of bytes into a file descriptor. This will block until one + * of the following happens: (1) there is an error (2) end of file, or (3) all + * length bytes have been written. + * + * @param fd The file descriptor to write to. It can be non-blocking. + * @param cursor The cursor pointing to the beginning of the bytes to send. + * @param length The size of the bytes sequence to write. + * @return int Whether there was an error while writing. 0 corresponds to + * success and -1 corresponds to an error (errno will be set). + */ +int write_bytes(int fd, uint8_t *cursor, size_t length); + +#endif /* IO_H */ diff --git a/cpp/src/plasma/doc/plasma-doxy-config b/cpp/src/plasma/doc/plasma-doxy-config new file mode 100644 index 00000000000..9c291f83888 --- /dev/null +++ b/cpp/src/plasma/doc/plasma-doxy-config @@ -0,0 +1,2473 @@ +# Doxyfile 1.8.13 + +# This file describes the settings to be used by the documentation system +# doxygen (www.doxygen.org) for a project. +# +# All text after a double hash (##) is considered a comment and is placed in +# front of the TAG it is preceding. +# +# All text after a single hash (#) is considered a comment and will be ignored. +# The format is: +# TAG = value [value, ...] +# For lists, items can also be appended using: +# TAG += value [value, ...] +# Values that contain spaces should be placed between quotes (\" \"). + +#--------------------------------------------------------------------------- +# Project related configuration options +#--------------------------------------------------------------------------- + +# This tag specifies the encoding used for all characters in the config file +# that follow. The default is UTF-8 which is also the encoding used for all text +# before the first occurrence of this tag. Doxygen uses libiconv (or the iconv +# built into libc) for the transcoding. See http://www.gnu.org/software/libiconv +# for the list of possible encodings. +# The default value is: UTF-8. + +DOXYFILE_ENCODING = UTF-8 + +# The PROJECT_NAME tag is a single word (or a sequence of words surrounded by +# double-quotes, unless you are using Doxywizard) that should identify the +# project for which the documentation is generated. This name is used in the +# title of most generated pages and in a few other places. +# The default value is: My Project. + +PROJECT_NAME = "Plasma" + +# The PROJECT_NUMBER tag can be used to enter a project or revision number. This +# could be handy for archiving the generated documentation or if some version +# control system is used. + +PROJECT_NUMBER = + +# Using the PROJECT_BRIEF tag one can provide an optional one line description +# for a project that appears at the top of each page and should give viewer a +# quick idea about the purpose of the project. Keep the description short. + +PROJECT_BRIEF = + +# With the PROJECT_LOGO tag one can specify a logo or an icon that is included +# in the documentation. The maximum height of the logo should not exceed 55 +# pixels and the maximum width should not exceed 200 pixels. Doxygen will copy +# the logo to the output directory. + +PROJECT_LOGO = + +# The OUTPUT_DIRECTORY tag is used to specify the (relative or absolute) path +# into which the generated documentation will be written. If a relative path is +# entered, it will be relative to the location where doxygen was started. If +# left blank the current directory will be used. + +OUTPUT_DIRECTORY = + +# If the CREATE_SUBDIRS tag is set to YES then doxygen will create 4096 sub- +# directories (in 2 levels) under the output directory of each output format and +# will distribute the generated files over these directories. Enabling this +# option can be useful when feeding doxygen a huge amount of source files, where +# putting all generated files in the same directory would otherwise causes +# performance problems for the file system. +# The default value is: NO. + +CREATE_SUBDIRS = NO + +# If the ALLOW_UNICODE_NAMES tag is set to YES, doxygen will allow non-ASCII +# characters to appear in the names of generated files. If set to NO, non-ASCII +# characters will be escaped, for example _xE3_x81_x84 will be used for Unicode +# U+3044. +# The default value is: NO. + +ALLOW_UNICODE_NAMES = NO + +# The OUTPUT_LANGUAGE tag is used to specify the language in which all +# documentation generated by doxygen is written. Doxygen will use this +# information to generate all constant output in the proper language. +# Possible values are: Afrikaans, Arabic, Armenian, Brazilian, Catalan, Chinese, +# Chinese-Traditional, Croatian, Czech, Danish, Dutch, English (United States), +# Esperanto, Farsi (Persian), Finnish, French, German, Greek, Hungarian, +# Indonesian, Italian, Japanese, Japanese-en (Japanese with English messages), +# Korean, Korean-en (Korean with English messages), Latvian, Lithuanian, +# Macedonian, Norwegian, Persian (Farsi), Polish, Portuguese, Romanian, Russian, +# Serbian, Serbian-Cyrillic, Slovak, Slovene, Spanish, Swedish, Turkish, +# Ukrainian and Vietnamese. +# The default value is: English. + +OUTPUT_LANGUAGE = English + +# If the BRIEF_MEMBER_DESC tag is set to YES, doxygen will include brief member +# descriptions after the members that are listed in the file and class +# documentation (similar to Javadoc). Set to NO to disable this. +# The default value is: YES. + +BRIEF_MEMBER_DESC = YES + +# If the REPEAT_BRIEF tag is set to YES, doxygen will prepend the brief +# description of a member or function before the detailed description +# +# Note: If both HIDE_UNDOC_MEMBERS and BRIEF_MEMBER_DESC are set to NO, the +# brief descriptions will be completely suppressed. +# The default value is: YES. + +REPEAT_BRIEF = YES + +# This tag implements a quasi-intelligent brief description abbreviator that is +# used to form the text in various listings. Each string in this list, if found +# as the leading text of the brief description, will be stripped from the text +# and the result, after processing the whole list, is used as the annotated +# text. Otherwise, the brief description is used as-is. If left blank, the +# following values are used ($name is automatically replaced with the name of +# the entity):The $name class, The $name widget, The $name file, is, provides, +# specifies, contains, represents, a, an and the. + +ABBREVIATE_BRIEF = "The $name class" \ + "The $name widget" \ + "The $name file" \ + is \ + provides \ + specifies \ + contains \ + represents \ + a \ + an \ + the + +# If the ALWAYS_DETAILED_SEC and REPEAT_BRIEF tags are both set to YES then +# doxygen will generate a detailed section even if there is only a brief +# description. +# The default value is: NO. + +ALWAYS_DETAILED_SEC = NO + +# If the INLINE_INHERITED_MEMB tag is set to YES, doxygen will show all +# inherited members of a class in the documentation of that class as if those +# members were ordinary class members. Constructors, destructors and assignment +# operators of the base classes will not be shown. +# The default value is: NO. + +INLINE_INHERITED_MEMB = NO + +# If the FULL_PATH_NAMES tag is set to YES, doxygen will prepend the full path +# before files name in the file list and in the header files. If set to NO the +# shortest path that makes the file name unique will be used +# The default value is: YES. + +FULL_PATH_NAMES = YES + +# The STRIP_FROM_PATH tag can be used to strip a user-defined part of the path. +# Stripping is only done if one of the specified strings matches the left-hand +# part of the path. The tag can be used to show relative paths in the file list. +# If left blank the directory from which doxygen is run is used as the path to +# strip. +# +# Note that you can specify absolute paths here, but also relative paths, which +# will be relative from the directory where doxygen is started. +# This tag requires that the tag FULL_PATH_NAMES is set to YES. + +STRIP_FROM_PATH = + +# The STRIP_FROM_INC_PATH tag can be used to strip a user-defined part of the +# path mentioned in the documentation of a class, which tells the reader which +# header file to include in order to use a class. If left blank only the name of +# the header file containing the class definition is used. Otherwise one should +# specify the list of include paths that are normally passed to the compiler +# using the -I flag. + +STRIP_FROM_INC_PATH = + +# If the SHORT_NAMES tag is set to YES, doxygen will generate much shorter (but +# less readable) file names. This can be useful is your file systems doesn't +# support long names like on DOS, Mac, or CD-ROM. +# The default value is: NO. + +SHORT_NAMES = NO + +# If the JAVADOC_AUTOBRIEF tag is set to YES then doxygen will interpret the +# first line (until the first dot) of a Javadoc-style comment as the brief +# description. If set to NO, the Javadoc-style will behave just like regular Qt- +# style comments (thus requiring an explicit @brief command for a brief +# description.) +# The default value is: NO. + +JAVADOC_AUTOBRIEF = NO + +# If the QT_AUTOBRIEF tag is set to YES then doxygen will interpret the first +# line (until the first dot) of a Qt-style comment as the brief description. If +# set to NO, the Qt-style will behave just like regular Qt-style comments (thus +# requiring an explicit \brief command for a brief description.) +# The default value is: NO. + +QT_AUTOBRIEF = NO + +# The MULTILINE_CPP_IS_BRIEF tag can be set to YES to make doxygen treat a +# multi-line C++ special comment block (i.e. a block of //! or /// comments) as +# a brief description. This used to be the default behavior. The new default is +# to treat a multi-line C++ comment block as a detailed description. Set this +# tag to YES if you prefer the old behavior instead. +# +# Note that setting this tag to YES also means that rational rose comments are +# not recognized any more. +# The default value is: NO. + +MULTILINE_CPP_IS_BRIEF = NO + +# If the INHERIT_DOCS tag is set to YES then an undocumented member inherits the +# documentation from any documented member that it re-implements. +# The default value is: YES. + +INHERIT_DOCS = YES + +# If the SEPARATE_MEMBER_PAGES tag is set to YES then doxygen will produce a new +# page for each member. If set to NO, the documentation of a member will be part +# of the file/class/namespace that contains it. +# The default value is: NO. + +SEPARATE_MEMBER_PAGES = NO + +# The TAB_SIZE tag can be used to set the number of spaces in a tab. Doxygen +# uses this value to replace tabs by spaces in code fragments. +# Minimum value: 1, maximum value: 16, default value: 4. + +TAB_SIZE = 2 + +# This tag can be used to specify a number of aliases that act as commands in +# the documentation. An alias has the form: +# name=value +# For example adding +# "sideeffect=@par Side Effects:\n" +# will allow you to put the command \sideeffect (or @sideeffect) in the +# documentation, which will result in a user-defined paragraph with heading +# "Side Effects:". You can put \n's in the value part of an alias to insert +# newlines. + +ALIASES = + +# This tag can be used to specify a number of word-keyword mappings (TCL only). +# A mapping has the form "name=value". For example adding "class=itcl::class" +# will allow you to use the command class in the itcl::class meaning. + +TCL_SUBST = + +# Set the OPTIMIZE_OUTPUT_FOR_C tag to YES if your project consists of C sources +# only. Doxygen will then generate output that is more tailored for C. For +# instance, some of the names that are used will be different. The list of all +# members will be omitted, etc. +# The default value is: NO. + +OPTIMIZE_OUTPUT_FOR_C = NO + +# Set the OPTIMIZE_OUTPUT_JAVA tag to YES if your project consists of Java or +# Python sources only. Doxygen will then generate output that is more tailored +# for that language. For instance, namespaces will be presented as packages, +# qualified scopes will look different, etc. +# The default value is: NO. + +OPTIMIZE_OUTPUT_JAVA = NO + +# Set the OPTIMIZE_FOR_FORTRAN tag to YES if your project consists of Fortran +# sources. Doxygen will then generate output that is tailored for Fortran. +# The default value is: NO. + +OPTIMIZE_FOR_FORTRAN = NO + +# Set the OPTIMIZE_OUTPUT_VHDL tag to YES if your project consists of VHDL +# sources. Doxygen will then generate output that is tailored for VHDL. +# The default value is: NO. + +OPTIMIZE_OUTPUT_VHDL = NO + +# Doxygen selects the parser to use depending on the extension of the files it +# parses. With this tag you can assign which parser to use for a given +# extension. Doxygen has a built-in mapping, but you can override or extend it +# using this tag. The format is ext=language, where ext is a file extension, and +# language is one of the parsers supported by doxygen: IDL, Java, Javascript, +# C#, C, C++, D, PHP, Objective-C, Python, Fortran (fixed format Fortran: +# FortranFixed, free formatted Fortran: FortranFree, unknown formatted Fortran: +# Fortran. In the later case the parser tries to guess whether the code is fixed +# or free formatted code, this is the default for Fortran type files), VHDL. For +# instance to make doxygen treat .inc files as Fortran files (default is PHP), +# and .f files as C (default is Fortran), use: inc=Fortran f=C. +# +# Note: For files without extension you can use no_extension as a placeholder. +# +# Note that for custom extensions you also need to set FILE_PATTERNS otherwise +# the files are not read by doxygen. + +EXTENSION_MAPPING = + +# If the MARKDOWN_SUPPORT tag is enabled then doxygen pre-processes all comments +# according to the Markdown format, which allows for more readable +# documentation. See http://daringfireball.net/projects/markdown/ for details. +# The output of markdown processing is further processed by doxygen, so you can +# mix doxygen, HTML, and XML commands with Markdown formatting. Disable only in +# case of backward compatibilities issues. +# The default value is: YES. + +MARKDOWN_SUPPORT = YES + +# When the TOC_INCLUDE_HEADINGS tag is set to a non-zero value, all headings up +# to that level are automatically included in the table of contents, even if +# they do not have an id attribute. +# Note: This feature currently applies only to Markdown headings. +# Minimum value: 0, maximum value: 99, default value: 0. +# This tag requires that the tag MARKDOWN_SUPPORT is set to YES. + +TOC_INCLUDE_HEADINGS = 0 + +# When enabled doxygen tries to link words that correspond to documented +# classes, or namespaces to their corresponding documentation. Such a link can +# be prevented in individual cases by putting a % sign in front of the word or +# globally by setting AUTOLINK_SUPPORT to NO. +# The default value is: YES. + +AUTOLINK_SUPPORT = YES + +# If you use STL classes (i.e. std::string, std::vector, etc.) but do not want +# to include (a tag file for) the STL sources as input, then you should set this +# tag to YES in order to let doxygen match functions declarations and +# definitions whose arguments contain STL classes (e.g. func(std::string); +# versus func(std::string) {}). This also make the inheritance and collaboration +# diagrams that involve STL classes more complete and accurate. +# The default value is: NO. + +BUILTIN_STL_SUPPORT = NO + +# If you use Microsoft's C++/CLI language, you should set this option to YES to +# enable parsing support. +# The default value is: NO. + +CPP_CLI_SUPPORT = NO + +# Set the SIP_SUPPORT tag to YES if your project consists of sip (see: +# http://www.riverbankcomputing.co.uk/software/sip/intro) sources only. Doxygen +# will parse them like normal C++ but will assume all classes use public instead +# of private inheritance when no explicit protection keyword is present. +# The default value is: NO. + +SIP_SUPPORT = NO + +# For Microsoft's IDL there are propget and propput attributes to indicate +# getter and setter methods for a property. Setting this option to YES will make +# doxygen to replace the get and set methods by a property in the documentation. +# This will only work if the methods are indeed getting or setting a simple +# type. If this is not the case, or you want to show the methods anyway, you +# should set this option to NO. +# The default value is: YES. + +IDL_PROPERTY_SUPPORT = YES + +# If member grouping is used in the documentation and the DISTRIBUTE_GROUP_DOC +# tag is set to YES then doxygen will reuse the documentation of the first +# member in the group (if any) for the other members of the group. By default +# all members of a group must be documented explicitly. +# The default value is: NO. + +DISTRIBUTE_GROUP_DOC = NO + +# If one adds a struct or class to a group and this option is enabled, then also +# any nested class or struct is added to the same group. By default this option +# is disabled and one has to add nested compounds explicitly via \ingroup. +# The default value is: NO. + +GROUP_NESTED_COMPOUNDS = NO + +# Set the SUBGROUPING tag to YES to allow class member groups of the same type +# (for instance a group of public functions) to be put as a subgroup of that +# type (e.g. under the Public Functions section). Set it to NO to prevent +# subgrouping. Alternatively, this can be done per class using the +# \nosubgrouping command. +# The default value is: YES. + +SUBGROUPING = YES + +# When the INLINE_GROUPED_CLASSES tag is set to YES, classes, structs and unions +# are shown inside the group in which they are included (e.g. using \ingroup) +# instead of on a separate page (for HTML and Man pages) or section (for LaTeX +# and RTF). +# +# Note that this feature does not work in combination with +# SEPARATE_MEMBER_PAGES. +# The default value is: NO. + +INLINE_GROUPED_CLASSES = NO + +# When the INLINE_SIMPLE_STRUCTS tag is set to YES, structs, classes, and unions +# with only public data fields or simple typedef fields will be shown inline in +# the documentation of the scope in which they are defined (i.e. file, +# namespace, or group documentation), provided this scope is documented. If set +# to NO, structs, classes, and unions are shown on a separate page (for HTML and +# Man pages) or section (for LaTeX and RTF). +# The default value is: NO. + +INLINE_SIMPLE_STRUCTS = NO + +# When TYPEDEF_HIDES_STRUCT tag is enabled, a typedef of a struct, union, or +# enum is documented as struct, union, or enum with the name of the typedef. So +# typedef struct TypeS {} TypeT, will appear in the documentation as a struct +# with name TypeT. When disabled the typedef will appear as a member of a file, +# namespace, or class. And the struct will be named TypeS. This can typically be +# useful for C code in case the coding convention dictates that all compound +# types are typedef'ed and only the typedef is referenced, never the tag name. +# The default value is: NO. + +TYPEDEF_HIDES_STRUCT = NO + +# The size of the symbol lookup cache can be set using LOOKUP_CACHE_SIZE. This +# cache is used to resolve symbols given their name and scope. Since this can be +# an expensive process and often the same symbol appears multiple times in the +# code, doxygen keeps a cache of pre-resolved symbols. If the cache is too small +# doxygen will become slower. If the cache is too large, memory is wasted. The +# cache size is given by this formula: 2^(16+LOOKUP_CACHE_SIZE). The valid range +# is 0..9, the default is 0, corresponding to a cache size of 2^16=65536 +# symbols. At the end of a run doxygen will report the cache usage and suggest +# the optimal cache size from a speed point of view. +# Minimum value: 0, maximum value: 9, default value: 0. + +LOOKUP_CACHE_SIZE = 0 + +#--------------------------------------------------------------------------- +# Build related configuration options +#--------------------------------------------------------------------------- + +# If the EXTRACT_ALL tag is set to YES, doxygen will assume all entities in +# documentation are documented, even if no documentation was available. Private +# class members and static file members will be hidden unless the +# EXTRACT_PRIVATE respectively EXTRACT_STATIC tags are set to YES. +# Note: This will also disable the warnings about undocumented members that are +# normally produced when WARNINGS is set to YES. +# The default value is: NO. + +EXTRACT_ALL = YES + +# If the EXTRACT_PRIVATE tag is set to YES, all private members of a class will +# be included in the documentation. +# The default value is: NO. + +EXTRACT_PRIVATE = NO + +# If the EXTRACT_PACKAGE tag is set to YES, all members with package or internal +# scope will be included in the documentation. +# The default value is: NO. + +EXTRACT_PACKAGE = NO + +# If the EXTRACT_STATIC tag is set to YES, all static members of a file will be +# included in the documentation. +# The default value is: NO. + +EXTRACT_STATIC = NO + +# If the EXTRACT_LOCAL_CLASSES tag is set to YES, classes (and structs) defined +# locally in source files will be included in the documentation. If set to NO, +# only classes defined in header files are included. Does not have any effect +# for Java sources. +# The default value is: YES. + +EXTRACT_LOCAL_CLASSES = YES + +# This flag is only useful for Objective-C code. If set to YES, local methods, +# which are defined in the implementation section but not in the interface are +# included in the documentation. If set to NO, only methods in the interface are +# included. +# The default value is: NO. + +EXTRACT_LOCAL_METHODS = NO + +# If this flag is set to YES, the members of anonymous namespaces will be +# extracted and appear in the documentation as a namespace called +# 'anonymous_namespace{file}', where file will be replaced with the base name of +# the file that contains the anonymous namespace. By default anonymous namespace +# are hidden. +# The default value is: NO. + +EXTRACT_ANON_NSPACES = NO + +# If the HIDE_UNDOC_MEMBERS tag is set to YES, doxygen will hide all +# undocumented members inside documented classes or files. If set to NO these +# members will be included in the various overviews, but no documentation +# section is generated. This option has no effect if EXTRACT_ALL is enabled. +# The default value is: NO. + +HIDE_UNDOC_MEMBERS = NO + +# If the HIDE_UNDOC_CLASSES tag is set to YES, doxygen will hide all +# undocumented classes that are normally visible in the class hierarchy. If set +# to NO, these classes will be included in the various overviews. This option +# has no effect if EXTRACT_ALL is enabled. +# The default value is: NO. + +HIDE_UNDOC_CLASSES = NO + +# If the HIDE_FRIEND_COMPOUNDS tag is set to YES, doxygen will hide all friend +# (class|struct|union) declarations. If set to NO, these declarations will be +# included in the documentation. +# The default value is: NO. + +HIDE_FRIEND_COMPOUNDS = NO + +# If the HIDE_IN_BODY_DOCS tag is set to YES, doxygen will hide any +# documentation blocks found inside the body of a function. If set to NO, these +# blocks will be appended to the function's detailed documentation block. +# The default value is: NO. + +HIDE_IN_BODY_DOCS = NO + +# The INTERNAL_DOCS tag determines if documentation that is typed after a +# \internal command is included. If the tag is set to NO then the documentation +# will be excluded. Set it to YES to include the internal documentation. +# The default value is: NO. + +INTERNAL_DOCS = NO + +# If the CASE_SENSE_NAMES tag is set to NO then doxygen will only generate file +# names in lower-case letters. If set to YES, upper-case letters are also +# allowed. This is useful if you have classes or files whose names only differ +# in case and if your file system supports case sensitive file names. Windows +# and Mac users are advised to set this option to NO. +# The default value is: system dependent. + +CASE_SENSE_NAMES = NO + +# If the HIDE_SCOPE_NAMES tag is set to NO then doxygen will show members with +# their full class and namespace scopes in the documentation. If set to YES, the +# scope will be hidden. +# The default value is: NO. + +HIDE_SCOPE_NAMES = NO + +# If the HIDE_COMPOUND_REFERENCE tag is set to NO (default) then doxygen will +# append additional text to a page's title, such as Class Reference. If set to +# YES the compound reference will be hidden. +# The default value is: NO. + +HIDE_COMPOUND_REFERENCE= NO + +# If the SHOW_INCLUDE_FILES tag is set to YES then doxygen will put a list of +# the files that are included by a file in the documentation of that file. +# The default value is: YES. + +SHOW_INCLUDE_FILES = YES + +# If the SHOW_GROUPED_MEMB_INC tag is set to YES then Doxygen will add for each +# grouped member an include statement to the documentation, telling the reader +# which file to include in order to use the member. +# The default value is: NO. + +SHOW_GROUPED_MEMB_INC = NO + +# If the FORCE_LOCAL_INCLUDES tag is set to YES then doxygen will list include +# files with double quotes in the documentation rather than with sharp brackets. +# The default value is: NO. + +FORCE_LOCAL_INCLUDES = NO + +# If the INLINE_INFO tag is set to YES then a tag [inline] is inserted in the +# documentation for inline members. +# The default value is: YES. + +INLINE_INFO = YES + +# If the SORT_MEMBER_DOCS tag is set to YES then doxygen will sort the +# (detailed) documentation of file and class members alphabetically by member +# name. If set to NO, the members will appear in declaration order. +# The default value is: YES. + +SORT_MEMBER_DOCS = YES + +# If the SORT_BRIEF_DOCS tag is set to YES then doxygen will sort the brief +# descriptions of file, namespace and class members alphabetically by member +# name. If set to NO, the members will appear in declaration order. Note that +# this will also influence the order of the classes in the class list. +# The default value is: NO. + +SORT_BRIEF_DOCS = NO + +# If the SORT_MEMBERS_CTORS_1ST tag is set to YES then doxygen will sort the +# (brief and detailed) documentation of class members so that constructors and +# destructors are listed first. If set to NO the constructors will appear in the +# respective orders defined by SORT_BRIEF_DOCS and SORT_MEMBER_DOCS. +# Note: If SORT_BRIEF_DOCS is set to NO this option is ignored for sorting brief +# member documentation. +# Note: If SORT_MEMBER_DOCS is set to NO this option is ignored for sorting +# detailed member documentation. +# The default value is: NO. + +SORT_MEMBERS_CTORS_1ST = NO + +# If the SORT_GROUP_NAMES tag is set to YES then doxygen will sort the hierarchy +# of group names into alphabetical order. If set to NO the group names will +# appear in their defined order. +# The default value is: NO. + +SORT_GROUP_NAMES = NO + +# If the SORT_BY_SCOPE_NAME tag is set to YES, the class list will be sorted by +# fully-qualified names, including namespaces. If set to NO, the class list will +# be sorted only by class name, not including the namespace part. +# Note: This option is not very useful if HIDE_SCOPE_NAMES is set to YES. +# Note: This option applies only to the class list, not to the alphabetical +# list. +# The default value is: NO. + +SORT_BY_SCOPE_NAME = NO + +# If the STRICT_PROTO_MATCHING option is enabled and doxygen fails to do proper +# type resolution of all parameters of a function it will reject a match between +# the prototype and the implementation of a member function even if there is +# only one candidate or it is obvious which candidate to choose by doing a +# simple string match. By disabling STRICT_PROTO_MATCHING doxygen will still +# accept a match between prototype and implementation in such cases. +# The default value is: NO. + +STRICT_PROTO_MATCHING = NO + +# The GENERATE_TODOLIST tag can be used to enable (YES) or disable (NO) the todo +# list. This list is created by putting \todo commands in the documentation. +# The default value is: YES. + +GENERATE_TODOLIST = YES + +# The GENERATE_TESTLIST tag can be used to enable (YES) or disable (NO) the test +# list. This list is created by putting \test commands in the documentation. +# The default value is: YES. + +GENERATE_TESTLIST = YES + +# The GENERATE_BUGLIST tag can be used to enable (YES) or disable (NO) the bug +# list. This list is created by putting \bug commands in the documentation. +# The default value is: YES. + +GENERATE_BUGLIST = YES + +# The GENERATE_DEPRECATEDLIST tag can be used to enable (YES) or disable (NO) +# the deprecated list. This list is created by putting \deprecated commands in +# the documentation. +# The default value is: YES. + +GENERATE_DEPRECATEDLIST= YES + +# The ENABLED_SECTIONS tag can be used to enable conditional documentation +# sections, marked by \if ... \endif and \cond +# ... \endcond blocks. + +ENABLED_SECTIONS = + +# The MAX_INITIALIZER_LINES tag determines the maximum number of lines that the +# initial value of a variable or macro / define can have for it to appear in the +# documentation. If the initializer consists of more lines than specified here +# it will be hidden. Use a value of 0 to hide initializers completely. The +# appearance of the value of individual variables and macros / defines can be +# controlled using \showinitializer or \hideinitializer command in the +# documentation regardless of this setting. +# Minimum value: 0, maximum value: 10000, default value: 30. + +MAX_INITIALIZER_LINES = 30 + +# Set the SHOW_USED_FILES tag to NO to disable the list of files generated at +# the bottom of the documentation of classes and structs. If set to YES, the +# list will mention the files that were used to generate the documentation. +# The default value is: YES. + +SHOW_USED_FILES = YES + +# Set the SHOW_FILES tag to NO to disable the generation of the Files page. This +# will remove the Files entry from the Quick Index and from the Folder Tree View +# (if specified). +# The default value is: YES. + +SHOW_FILES = YES + +# Set the SHOW_NAMESPACES tag to NO to disable the generation of the Namespaces +# page. This will remove the Namespaces entry from the Quick Index and from the +# Folder Tree View (if specified). +# The default value is: YES. + +SHOW_NAMESPACES = YES + +# The FILE_VERSION_FILTER tag can be used to specify a program or script that +# doxygen should invoke to get the current version for each file (typically from +# the version control system). Doxygen will invoke the program by executing (via +# popen()) the command command input-file, where command is the value of the +# FILE_VERSION_FILTER tag, and input-file is the name of an input file provided +# by doxygen. Whatever the program writes to standard output is used as the file +# version. For an example see the documentation. + +FILE_VERSION_FILTER = + +# The LAYOUT_FILE tag can be used to specify a layout file which will be parsed +# by doxygen. The layout file controls the global structure of the generated +# output files in an output format independent way. To create the layout file +# that represents doxygen's defaults, run doxygen with the -l option. You can +# optionally specify a file name after the option, if omitted DoxygenLayout.xml +# will be used as the name of the layout file. +# +# Note that if you run doxygen from a directory containing a file called +# DoxygenLayout.xml, doxygen will parse it automatically even if the LAYOUT_FILE +# tag is left empty. + +LAYOUT_FILE = + +# The CITE_BIB_FILES tag can be used to specify one or more bib files containing +# the reference definitions. This must be a list of .bib files. The .bib +# extension is automatically appended if omitted. This requires the bibtex tool +# to be installed. See also http://en.wikipedia.org/wiki/BibTeX for more info. +# For LaTeX the style of the bibliography can be controlled using +# LATEX_BIB_STYLE. To use this feature you need bibtex and perl available in the +# search path. See also \cite for info how to create references. + +CITE_BIB_FILES = + +#--------------------------------------------------------------------------- +# Configuration options related to warning and progress messages +#--------------------------------------------------------------------------- + +# The QUIET tag can be used to turn on/off the messages that are generated to +# standard output by doxygen. If QUIET is set to YES this implies that the +# messages are off. +# The default value is: NO. + +QUIET = NO + +# The WARNINGS tag can be used to turn on/off the warning messages that are +# generated to standard error (stderr) by doxygen. If WARNINGS is set to YES +# this implies that the warnings are on. +# +# Tip: Turn warnings on while writing the documentation. +# The default value is: YES. + +WARNINGS = YES + +# If the WARN_IF_UNDOCUMENTED tag is set to YES then doxygen will generate +# warnings for undocumented members. If EXTRACT_ALL is set to YES then this flag +# will automatically be disabled. +# The default value is: YES. + +WARN_IF_UNDOCUMENTED = YES + +# If the WARN_IF_DOC_ERROR tag is set to YES, doxygen will generate warnings for +# potential errors in the documentation, such as not documenting some parameters +# in a documented function, or documenting parameters that don't exist or using +# markup commands wrongly. +# The default value is: YES. + +WARN_IF_DOC_ERROR = YES + +# This WARN_NO_PARAMDOC option can be enabled to get warnings for functions that +# are documented, but have no documentation for their parameters or return +# value. If set to NO, doxygen will only warn about wrong or incomplete +# parameter documentation, but not about the absence of documentation. +# The default value is: NO. + +WARN_NO_PARAMDOC = NO + +# If the WARN_AS_ERROR tag is set to YES then doxygen will immediately stop when +# a warning is encountered. +# The default value is: NO. + +WARN_AS_ERROR = NO + +# The WARN_FORMAT tag determines the format of the warning messages that doxygen +# can produce. The string should contain the $file, $line, and $text tags, which +# will be replaced by the file and line number from which the warning originated +# and the warning text. Optionally the format may contain $version, which will +# be replaced by the version of the file (if it could be obtained via +# FILE_VERSION_FILTER) +# The default value is: $file:$line: $text. + +WARN_FORMAT = "$file:$line: $text" + +# The WARN_LOGFILE tag can be used to specify a file to which warning and error +# messages should be written. If left blank the output is written to standard +# error (stderr). + +WARN_LOGFILE = + +#--------------------------------------------------------------------------- +# Configuration options related to the input files +#--------------------------------------------------------------------------- + +# The INPUT tag is used to specify the files and/or directories that contain +# documented source files. You may enter file names like myfile.cpp or +# directories like /usr/src/myproject. Separate the files or directories with +# spaces. See also FILE_PATTERNS and EXTENSION_MAPPING +# Note: If this tag is empty the current directory is searched. + +INPUT = ../src + +# This tag can be used to specify the character encoding of the source files +# that doxygen parses. Internally doxygen uses the UTF-8 encoding. Doxygen uses +# libiconv (or the iconv built into libc) for the transcoding. See the libiconv +# documentation (see: http://www.gnu.org/software/libiconv) for the list of +# possible encodings. +# The default value is: UTF-8. + +INPUT_ENCODING = UTF-8 + +# If the value of the INPUT tag contains directories, you can use the +# FILE_PATTERNS tag to specify one or more wildcard patterns (like *.cpp and +# *.h) to filter out the source-files in the directories. +# +# Note that for custom extensions or not directly supported extensions you also +# need to set EXTENSION_MAPPING for the extension otherwise the files are not +# read by doxygen. +# +# If left blank the following patterns are tested:*.c, *.cc, *.cxx, *.cpp, +# *.c++, *.java, *.ii, *.ixx, *.ipp, *.i++, *.inl, *.idl, *.ddl, *.odl, *.h, +# *.hh, *.hxx, *.hpp, *.h++, *.cs, *.d, *.php, *.php4, *.php5, *.phtml, *.inc, +# *.m, *.markdown, *.md, *.mm, *.dox, *.py, *.pyw, *.f90, *.f95, *.f03, *.f08, +# *.f, *.for, *.tcl, *.vhd, *.vhdl, *.ucf and *.qsf. + +FILE_PATTERNS = *.c \ + *.cc \ + *.cxx \ + *.cpp \ + *.c++ \ + *.java \ + *.ii \ + *.ixx \ + *.ipp \ + *.i++ \ + *.inl \ + *.idl \ + *.ddl \ + *.odl \ + *.h \ + *.hh \ + *.hxx \ + *.hpp \ + *.h++ \ + *.cs \ + *.d \ + *.php \ + *.php4 \ + *.php5 \ + *.phtml \ + *.inc \ + *.m \ + *.markdown \ + *.md \ + *.mm \ + *.dox \ + *.py \ + *.pyw \ + *.f90 \ + *.f95 \ + *.f03 \ + *.f08 \ + *.f \ + *.for \ + *.tcl \ + *.vhd \ + *.vhdl \ + *.ucf \ + *.qsf + +# The RECURSIVE tag can be used to specify whether or not subdirectories should +# be searched for input files as well. +# The default value is: NO. + +RECURSIVE = NO + +# The EXCLUDE tag can be used to specify files and/or directories that should be +# excluded from the INPUT source files. This way you can easily exclude a +# subdirectory from a directory tree whose root is specified with the INPUT tag. +# +# Note that relative paths are relative to the directory from which doxygen is +# run. + +EXCLUDE = ../src/utarray.h ../src/uthash.h + +# The EXCLUDE_SYMLINKS tag can be used to select whether or not files or +# directories that are symbolic links (a Unix file system feature) are excluded +# from the input. +# The default value is: NO. + +EXCLUDE_SYMLINKS = NO + +# If the value of the INPUT tag contains directories, you can use the +# EXCLUDE_PATTERNS tag to specify one or more wildcard patterns to exclude +# certain files from those directories. +# +# Note that the wildcards are matched against the file with absolute path, so to +# exclude all test directories for example use the pattern */test/* + +EXCLUDE_PATTERNS = + +# The EXCLUDE_SYMBOLS tag can be used to specify one or more symbol names +# (namespaces, classes, functions, etc.) that should be excluded from the +# output. The symbol name can be a fully qualified name, a word, or if the +# wildcard * is used, a substring. Examples: ANamespace, AClass, +# AClass::ANamespace, ANamespace::*Test +# +# Note that the wildcards are matched against the file with absolute path, so to +# exclude all test directories use the pattern */test/* + +EXCLUDE_SYMBOLS = + +# The EXAMPLE_PATH tag can be used to specify one or more files or directories +# that contain example code fragments that are included (see the \include +# command). + +EXAMPLE_PATH = + +# If the value of the EXAMPLE_PATH tag contains directories, you can use the +# EXAMPLE_PATTERNS tag to specify one or more wildcard pattern (like *.cpp and +# *.h) to filter out the source-files in the directories. If left blank all +# files are included. + +EXAMPLE_PATTERNS = * + +# If the EXAMPLE_RECURSIVE tag is set to YES then subdirectories will be +# searched for input files to be used with the \include or \dontinclude commands +# irrespective of the value of the RECURSIVE tag. +# The default value is: NO. + +EXAMPLE_RECURSIVE = NO + +# The IMAGE_PATH tag can be used to specify one or more files or directories +# that contain images that are to be included in the documentation (see the +# \image command). + +IMAGE_PATH = + +# The INPUT_FILTER tag can be used to specify a program that doxygen should +# invoke to filter for each input file. Doxygen will invoke the filter program +# by executing (via popen()) the command: +# +# +# +# where is the value of the INPUT_FILTER tag, and is the +# name of an input file. Doxygen will then use the output that the filter +# program writes to standard output. If FILTER_PATTERNS is specified, this tag +# will be ignored. +# +# Note that the filter must not add or remove lines; it is applied before the +# code is scanned, but not when the output code is generated. If lines are added +# or removed, the anchors will not be placed correctly. +# +# Note that for custom extensions or not directly supported extensions you also +# need to set EXTENSION_MAPPING for the extension otherwise the files are not +# properly processed by doxygen. + +INPUT_FILTER = + +# The FILTER_PATTERNS tag can be used to specify filters on a per file pattern +# basis. Doxygen will compare the file name with each pattern and apply the +# filter if there is a match. The filters are a list of the form: pattern=filter +# (like *.cpp=my_cpp_filter). See INPUT_FILTER for further information on how +# filters are used. If the FILTER_PATTERNS tag is empty or if none of the +# patterns match the file name, INPUT_FILTER is applied. +# +# Note that for custom extensions or not directly supported extensions you also +# need to set EXTENSION_MAPPING for the extension otherwise the files are not +# properly processed by doxygen. + +FILTER_PATTERNS = + +# If the FILTER_SOURCE_FILES tag is set to YES, the input filter (if set using +# INPUT_FILTER) will also be used to filter the input files that are used for +# producing the source files to browse (i.e. when SOURCE_BROWSER is set to YES). +# The default value is: NO. + +FILTER_SOURCE_FILES = NO + +# The FILTER_SOURCE_PATTERNS tag can be used to specify source filters per file +# pattern. A pattern will override the setting for FILTER_PATTERN (if any) and +# it is also possible to disable source filtering for a specific pattern using +# *.ext= (so without naming a filter). +# This tag requires that the tag FILTER_SOURCE_FILES is set to YES. + +FILTER_SOURCE_PATTERNS = + +# If the USE_MDFILE_AS_MAINPAGE tag refers to the name of a markdown file that +# is part of the input, its contents will be placed on the main page +# (index.html). This can be useful if you have a project on for instance GitHub +# and want to reuse the introduction page also for the doxygen output. + +USE_MDFILE_AS_MAINPAGE = + +#--------------------------------------------------------------------------- +# Configuration options related to source browsing +#--------------------------------------------------------------------------- + +# If the SOURCE_BROWSER tag is set to YES then a list of source files will be +# generated. Documented entities will be cross-referenced with these sources. +# +# Note: To get rid of all source code in the generated output, make sure that +# also VERBATIM_HEADERS is set to NO. +# The default value is: NO. + +SOURCE_BROWSER = NO + +# Setting the INLINE_SOURCES tag to YES will include the body of functions, +# classes and enums directly into the documentation. +# The default value is: NO. + +INLINE_SOURCES = NO + +# Setting the STRIP_CODE_COMMENTS tag to YES will instruct doxygen to hide any +# special comment blocks from generated source code fragments. Normal C, C++ and +# Fortran comments will always remain visible. +# The default value is: YES. + +STRIP_CODE_COMMENTS = YES + +# If the REFERENCED_BY_RELATION tag is set to YES then for each documented +# function all documented functions referencing it will be listed. +# The default value is: NO. + +REFERENCED_BY_RELATION = NO + +# If the REFERENCES_RELATION tag is set to YES then for each documented function +# all documented entities called/used by that function will be listed. +# The default value is: NO. + +REFERENCES_RELATION = NO + +# If the REFERENCES_LINK_SOURCE tag is set to YES and SOURCE_BROWSER tag is set +# to YES then the hyperlinks from functions in REFERENCES_RELATION and +# REFERENCED_BY_RELATION lists will link to the source code. Otherwise they will +# link to the documentation. +# The default value is: YES. + +REFERENCES_LINK_SOURCE = YES + +# If SOURCE_TOOLTIPS is enabled (the default) then hovering a hyperlink in the +# source code will show a tooltip with additional information such as prototype, +# brief description and links to the definition and documentation. Since this +# will make the HTML file larger and loading of large files a bit slower, you +# can opt to disable this feature. +# The default value is: YES. +# This tag requires that the tag SOURCE_BROWSER is set to YES. + +SOURCE_TOOLTIPS = YES + +# If the USE_HTAGS tag is set to YES then the references to source code will +# point to the HTML generated by the htags(1) tool instead of doxygen built-in +# source browser. The htags tool is part of GNU's global source tagging system +# (see http://www.gnu.org/software/global/global.html). You will need version +# 4.8.6 or higher. +# +# To use it do the following: +# - Install the latest version of global +# - Enable SOURCE_BROWSER and USE_HTAGS in the config file +# - Make sure the INPUT points to the root of the source tree +# - Run doxygen as normal +# +# Doxygen will invoke htags (and that will in turn invoke gtags), so these +# tools must be available from the command line (i.e. in the search path). +# +# The result: instead of the source browser generated by doxygen, the links to +# source code will now point to the output of htags. +# The default value is: NO. +# This tag requires that the tag SOURCE_BROWSER is set to YES. + +USE_HTAGS = NO + +# If the VERBATIM_HEADERS tag is set the YES then doxygen will generate a +# verbatim copy of the header file for each class for which an include is +# specified. Set to NO to disable this. +# See also: Section \class. +# The default value is: YES. + +VERBATIM_HEADERS = YES + +#--------------------------------------------------------------------------- +# Configuration options related to the alphabetical class index +#--------------------------------------------------------------------------- + +# If the ALPHABETICAL_INDEX tag is set to YES, an alphabetical index of all +# compounds will be generated. Enable this if the project contains a lot of +# classes, structs, unions or interfaces. +# The default value is: YES. + +ALPHABETICAL_INDEX = YES + +# The COLS_IN_ALPHA_INDEX tag can be used to specify the number of columns in +# which the alphabetical index list will be split. +# Minimum value: 1, maximum value: 20, default value: 5. +# This tag requires that the tag ALPHABETICAL_INDEX is set to YES. + +COLS_IN_ALPHA_INDEX = 5 + +# In case all classes in a project start with a common prefix, all classes will +# be put under the same header in the alphabetical index. The IGNORE_PREFIX tag +# can be used to specify a prefix (or a list of prefixes) that should be ignored +# while generating the index headers. +# This tag requires that the tag ALPHABETICAL_INDEX is set to YES. + +IGNORE_PREFIX = + +#--------------------------------------------------------------------------- +# Configuration options related to the HTML output +#--------------------------------------------------------------------------- + +# If the GENERATE_HTML tag is set to YES, doxygen will generate HTML output +# The default value is: YES. + +GENERATE_HTML = YES + +# The HTML_OUTPUT tag is used to specify where the HTML docs will be put. If a +# relative path is entered the value of OUTPUT_DIRECTORY will be put in front of +# it. +# The default directory is: html. +# This tag requires that the tag GENERATE_HTML is set to YES. + +HTML_OUTPUT = html + +# The HTML_FILE_EXTENSION tag can be used to specify the file extension for each +# generated HTML page (for example: .htm, .php, .asp). +# The default value is: .html. +# This tag requires that the tag GENERATE_HTML is set to YES. + +HTML_FILE_EXTENSION = .html + +# The HTML_HEADER tag can be used to specify a user-defined HTML header file for +# each generated HTML page. If the tag is left blank doxygen will generate a +# standard header. +# +# To get valid HTML the header file that includes any scripts and style sheets +# that doxygen needs, which is dependent on the configuration options used (e.g. +# the setting GENERATE_TREEVIEW). It is highly recommended to start with a +# default header using +# doxygen -w html new_header.html new_footer.html new_stylesheet.css +# YourConfigFile +# and then modify the file new_header.html. See also section "Doxygen usage" +# for information on how to generate the default header that doxygen normally +# uses. +# Note: The header is subject to change so you typically have to regenerate the +# default header when upgrading to a newer version of doxygen. For a description +# of the possible markers and block names see the documentation. +# This tag requires that the tag GENERATE_HTML is set to YES. + +HTML_HEADER = + +# The HTML_FOOTER tag can be used to specify a user-defined HTML footer for each +# generated HTML page. If the tag is left blank doxygen will generate a standard +# footer. See HTML_HEADER for more information on how to generate a default +# footer and what special commands can be used inside the footer. See also +# section "Doxygen usage" for information on how to generate the default footer +# that doxygen normally uses. +# This tag requires that the tag GENERATE_HTML is set to YES. + +HTML_FOOTER = + +# The HTML_STYLESHEET tag can be used to specify a user-defined cascading style +# sheet that is used by each HTML page. It can be used to fine-tune the look of +# the HTML output. If left blank doxygen will generate a default style sheet. +# See also section "Doxygen usage" for information on how to generate the style +# sheet that doxygen normally uses. +# Note: It is recommended to use HTML_EXTRA_STYLESHEET instead of this tag, as +# it is more robust and this tag (HTML_STYLESHEET) will in the future become +# obsolete. +# This tag requires that the tag GENERATE_HTML is set to YES. + +HTML_STYLESHEET = + +# The HTML_EXTRA_STYLESHEET tag can be used to specify additional user-defined +# cascading style sheets that are included after the standard style sheets +# created by doxygen. Using this option one can overrule certain style aspects. +# This is preferred over using HTML_STYLESHEET since it does not replace the +# standard style sheet and is therefore more robust against future updates. +# Doxygen will copy the style sheet files to the output directory. +# Note: The order of the extra style sheet files is of importance (e.g. the last +# style sheet in the list overrules the setting of the previous ones in the +# list). For an example see the documentation. +# This tag requires that the tag GENERATE_HTML is set to YES. + +HTML_EXTRA_STYLESHEET = + +# The HTML_EXTRA_FILES tag can be used to specify one or more extra images or +# other source files which should be copied to the HTML output directory. Note +# that these files will be copied to the base HTML output directory. Use the +# $relpath^ marker in the HTML_HEADER and/or HTML_FOOTER files to load these +# files. In the HTML_STYLESHEET file, use the file name only. Also note that the +# files will be copied as-is; there are no commands or markers available. +# This tag requires that the tag GENERATE_HTML is set to YES. + +HTML_EXTRA_FILES = + +# The HTML_COLORSTYLE_HUE tag controls the color of the HTML output. Doxygen +# will adjust the colors in the style sheet and background images according to +# this color. Hue is specified as an angle on a colorwheel, see +# http://en.wikipedia.org/wiki/Hue for more information. For instance the value +# 0 represents red, 60 is yellow, 120 is green, 180 is cyan, 240 is blue, 300 +# purple, and 360 is red again. +# Minimum value: 0, maximum value: 359, default value: 220. +# This tag requires that the tag GENERATE_HTML is set to YES. + +HTML_COLORSTYLE_HUE = 220 + +# The HTML_COLORSTYLE_SAT tag controls the purity (or saturation) of the colors +# in the HTML output. For a value of 0 the output will use grayscales only. A +# value of 255 will produce the most vivid colors. +# Minimum value: 0, maximum value: 255, default value: 100. +# This tag requires that the tag GENERATE_HTML is set to YES. + +HTML_COLORSTYLE_SAT = 100 + +# The HTML_COLORSTYLE_GAMMA tag controls the gamma correction applied to the +# luminance component of the colors in the HTML output. Values below 100 +# gradually make the output lighter, whereas values above 100 make the output +# darker. The value divided by 100 is the actual gamma applied, so 80 represents +# a gamma of 0.8, The value 220 represents a gamma of 2.2, and 100 does not +# change the gamma. +# Minimum value: 40, maximum value: 240, default value: 80. +# This tag requires that the tag GENERATE_HTML is set to YES. + +HTML_COLORSTYLE_GAMMA = 80 + +# If the HTML_TIMESTAMP tag is set to YES then the footer of each generated HTML +# page will contain the date and time when the page was generated. Setting this +# to YES can help to show when doxygen was last run and thus if the +# documentation is up to date. +# The default value is: NO. +# This tag requires that the tag GENERATE_HTML is set to YES. + +HTML_TIMESTAMP = NO + +# If the HTML_DYNAMIC_SECTIONS tag is set to YES then the generated HTML +# documentation will contain sections that can be hidden and shown after the +# page has loaded. +# The default value is: NO. +# This tag requires that the tag GENERATE_HTML is set to YES. + +HTML_DYNAMIC_SECTIONS = NO + +# With HTML_INDEX_NUM_ENTRIES one can control the preferred number of entries +# shown in the various tree structured indices initially; the user can expand +# and collapse entries dynamically later on. Doxygen will expand the tree to +# such a level that at most the specified number of entries are visible (unless +# a fully collapsed tree already exceeds this amount). So setting the number of +# entries 1 will produce a full collapsed tree by default. 0 is a special value +# representing an infinite number of entries and will result in a full expanded +# tree by default. +# Minimum value: 0, maximum value: 9999, default value: 100. +# This tag requires that the tag GENERATE_HTML is set to YES. + +HTML_INDEX_NUM_ENTRIES = 100 + +# If the GENERATE_DOCSET tag is set to YES, additional index files will be +# generated that can be used as input for Apple's Xcode 3 integrated development +# environment (see: http://developer.apple.com/tools/xcode/), introduced with +# OSX 10.5 (Leopard). To create a documentation set, doxygen will generate a +# Makefile in the HTML output directory. Running make will produce the docset in +# that directory and running make install will install the docset in +# ~/Library/Developer/Shared/Documentation/DocSets so that Xcode will find it at +# startup. See http://developer.apple.com/tools/creatingdocsetswithdoxygen.html +# for more information. +# The default value is: NO. +# This tag requires that the tag GENERATE_HTML is set to YES. + +GENERATE_DOCSET = NO + +# This tag determines the name of the docset feed. A documentation feed provides +# an umbrella under which multiple documentation sets from a single provider +# (such as a company or product suite) can be grouped. +# The default value is: Doxygen generated docs. +# This tag requires that the tag GENERATE_DOCSET is set to YES. + +DOCSET_FEEDNAME = "Doxygen generated docs" + +# This tag specifies a string that should uniquely identify the documentation +# set bundle. This should be a reverse domain-name style string, e.g. +# com.mycompany.MyDocSet. Doxygen will append .docset to the name. +# The default value is: org.doxygen.Project. +# This tag requires that the tag GENERATE_DOCSET is set to YES. + +DOCSET_BUNDLE_ID = org.doxygen.Project + +# The DOCSET_PUBLISHER_ID tag specifies a string that should uniquely identify +# the documentation publisher. This should be a reverse domain-name style +# string, e.g. com.mycompany.MyDocSet.documentation. +# The default value is: org.doxygen.Publisher. +# This tag requires that the tag GENERATE_DOCSET is set to YES. + +DOCSET_PUBLISHER_ID = org.doxygen.Publisher + +# The DOCSET_PUBLISHER_NAME tag identifies the documentation publisher. +# The default value is: Publisher. +# This tag requires that the tag GENERATE_DOCSET is set to YES. + +DOCSET_PUBLISHER_NAME = Publisher + +# If the GENERATE_HTMLHELP tag is set to YES then doxygen generates three +# additional HTML index files: index.hhp, index.hhc, and index.hhk. The +# index.hhp is a project file that can be read by Microsoft's HTML Help Workshop +# (see: http://www.microsoft.com/en-us/download/details.aspx?id=21138) on +# Windows. +# +# The HTML Help Workshop contains a compiler that can convert all HTML output +# generated by doxygen into a single compiled HTML file (.chm). Compiled HTML +# files are now used as the Windows 98 help format, and will replace the old +# Windows help format (.hlp) on all Windows platforms in the future. Compressed +# HTML files also contain an index, a table of contents, and you can search for +# words in the documentation. The HTML workshop also contains a viewer for +# compressed HTML files. +# The default value is: NO. +# This tag requires that the tag GENERATE_HTML is set to YES. + +GENERATE_HTMLHELP = NO + +# The CHM_FILE tag can be used to specify the file name of the resulting .chm +# file. You can add a path in front of the file if the result should not be +# written to the html output directory. +# This tag requires that the tag GENERATE_HTMLHELP is set to YES. + +CHM_FILE = + +# The HHC_LOCATION tag can be used to specify the location (absolute path +# including file name) of the HTML help compiler (hhc.exe). If non-empty, +# doxygen will try to run the HTML help compiler on the generated index.hhp. +# The file has to be specified with full path. +# This tag requires that the tag GENERATE_HTMLHELP is set to YES. + +HHC_LOCATION = + +# The GENERATE_CHI flag controls if a separate .chi index file is generated +# (YES) or that it should be included in the master .chm file (NO). +# The default value is: NO. +# This tag requires that the tag GENERATE_HTMLHELP is set to YES. + +GENERATE_CHI = NO + +# The CHM_INDEX_ENCODING is used to encode HtmlHelp index (hhk), content (hhc) +# and project file content. +# This tag requires that the tag GENERATE_HTMLHELP is set to YES. + +CHM_INDEX_ENCODING = + +# The BINARY_TOC flag controls whether a binary table of contents is generated +# (YES) or a normal table of contents (NO) in the .chm file. Furthermore it +# enables the Previous and Next buttons. +# The default value is: NO. +# This tag requires that the tag GENERATE_HTMLHELP is set to YES. + +BINARY_TOC = NO + +# The TOC_EXPAND flag can be set to YES to add extra items for group members to +# the table of contents of the HTML help documentation and to the tree view. +# The default value is: NO. +# This tag requires that the tag GENERATE_HTMLHELP is set to YES. + +TOC_EXPAND = NO + +# If the GENERATE_QHP tag is set to YES and both QHP_NAMESPACE and +# QHP_VIRTUAL_FOLDER are set, an additional index file will be generated that +# can be used as input for Qt's qhelpgenerator to generate a Qt Compressed Help +# (.qch) of the generated HTML documentation. +# The default value is: NO. +# This tag requires that the tag GENERATE_HTML is set to YES. + +GENERATE_QHP = NO + +# If the QHG_LOCATION tag is specified, the QCH_FILE tag can be used to specify +# the file name of the resulting .qch file. The path specified is relative to +# the HTML output folder. +# This tag requires that the tag GENERATE_QHP is set to YES. + +QCH_FILE = + +# The QHP_NAMESPACE tag specifies the namespace to use when generating Qt Help +# Project output. For more information please see Qt Help Project / Namespace +# (see: http://qt-project.org/doc/qt-4.8/qthelpproject.html#namespace). +# The default value is: org.doxygen.Project. +# This tag requires that the tag GENERATE_QHP is set to YES. + +QHP_NAMESPACE = org.doxygen.Project + +# The QHP_VIRTUAL_FOLDER tag specifies the namespace to use when generating Qt +# Help Project output. For more information please see Qt Help Project / Virtual +# Folders (see: http://qt-project.org/doc/qt-4.8/qthelpproject.html#virtual- +# folders). +# The default value is: doc. +# This tag requires that the tag GENERATE_QHP is set to YES. + +QHP_VIRTUAL_FOLDER = doc + +# If the QHP_CUST_FILTER_NAME tag is set, it specifies the name of a custom +# filter to add. For more information please see Qt Help Project / Custom +# Filters (see: http://qt-project.org/doc/qt-4.8/qthelpproject.html#custom- +# filters). +# This tag requires that the tag GENERATE_QHP is set to YES. + +QHP_CUST_FILTER_NAME = + +# The QHP_CUST_FILTER_ATTRS tag specifies the list of the attributes of the +# custom filter to add. For more information please see Qt Help Project / Custom +# Filters (see: http://qt-project.org/doc/qt-4.8/qthelpproject.html#custom- +# filters). +# This tag requires that the tag GENERATE_QHP is set to YES. + +QHP_CUST_FILTER_ATTRS = + +# The QHP_SECT_FILTER_ATTRS tag specifies the list of the attributes this +# project's filter section matches. Qt Help Project / Filter Attributes (see: +# http://qt-project.org/doc/qt-4.8/qthelpproject.html#filter-attributes). +# This tag requires that the tag GENERATE_QHP is set to YES. + +QHP_SECT_FILTER_ATTRS = + +# The QHG_LOCATION tag can be used to specify the location of Qt's +# qhelpgenerator. If non-empty doxygen will try to run qhelpgenerator on the +# generated .qhp file. +# This tag requires that the tag GENERATE_QHP is set to YES. + +QHG_LOCATION = + +# If the GENERATE_ECLIPSEHELP tag is set to YES, additional index files will be +# generated, together with the HTML files, they form an Eclipse help plugin. To +# install this plugin and make it available under the help contents menu in +# Eclipse, the contents of the directory containing the HTML and XML files needs +# to be copied into the plugins directory of eclipse. The name of the directory +# within the plugins directory should be the same as the ECLIPSE_DOC_ID value. +# After copying Eclipse needs to be restarted before the help appears. +# The default value is: NO. +# This tag requires that the tag GENERATE_HTML is set to YES. + +GENERATE_ECLIPSEHELP = NO + +# A unique identifier for the Eclipse help plugin. When installing the plugin +# the directory name containing the HTML and XML files should also have this +# name. Each documentation set should have its own identifier. +# The default value is: org.doxygen.Project. +# This tag requires that the tag GENERATE_ECLIPSEHELP is set to YES. + +ECLIPSE_DOC_ID = org.doxygen.Project + +# If you want full control over the layout of the generated HTML pages it might +# be necessary to disable the index and replace it with your own. The +# DISABLE_INDEX tag can be used to turn on/off the condensed index (tabs) at top +# of each HTML page. A value of NO enables the index and the value YES disables +# it. Since the tabs in the index contain the same information as the navigation +# tree, you can set this option to YES if you also set GENERATE_TREEVIEW to YES. +# The default value is: NO. +# This tag requires that the tag GENERATE_HTML is set to YES. + +DISABLE_INDEX = NO + +# The GENERATE_TREEVIEW tag is used to specify whether a tree-like index +# structure should be generated to display hierarchical information. If the tag +# value is set to YES, a side panel will be generated containing a tree-like +# index structure (just like the one that is generated for HTML Help). For this +# to work a browser that supports JavaScript, DHTML, CSS and frames is required +# (i.e. any modern browser). Windows users are probably better off using the +# HTML help feature. Via custom style sheets (see HTML_EXTRA_STYLESHEET) one can +# further fine-tune the look of the index. As an example, the default style +# sheet generated by doxygen has an example that shows how to put an image at +# the root of the tree instead of the PROJECT_NAME. Since the tree basically has +# the same information as the tab index, you could consider setting +# DISABLE_INDEX to YES when enabling this option. +# The default value is: NO. +# This tag requires that the tag GENERATE_HTML is set to YES. + +GENERATE_TREEVIEW = NO + +# The ENUM_VALUES_PER_LINE tag can be used to set the number of enum values that +# doxygen will group on one line in the generated HTML documentation. +# +# Note that a value of 0 will completely suppress the enum values from appearing +# in the overview section. +# Minimum value: 0, maximum value: 20, default value: 4. +# This tag requires that the tag GENERATE_HTML is set to YES. + +ENUM_VALUES_PER_LINE = 4 + +# If the treeview is enabled (see GENERATE_TREEVIEW) then this tag can be used +# to set the initial width (in pixels) of the frame in which the tree is shown. +# Minimum value: 0, maximum value: 1500, default value: 250. +# This tag requires that the tag GENERATE_HTML is set to YES. + +TREEVIEW_WIDTH = 250 + +# If the EXT_LINKS_IN_WINDOW option is set to YES, doxygen will open links to +# external symbols imported via tag files in a separate window. +# The default value is: NO. +# This tag requires that the tag GENERATE_HTML is set to YES. + +EXT_LINKS_IN_WINDOW = NO + +# Use this tag to change the font size of LaTeX formulas included as images in +# the HTML documentation. When you change the font size after a successful +# doxygen run you need to manually remove any form_*.png images from the HTML +# output directory to force them to be regenerated. +# Minimum value: 8, maximum value: 50, default value: 10. +# This tag requires that the tag GENERATE_HTML is set to YES. + +FORMULA_FONTSIZE = 10 + +# Use the FORMULA_TRANPARENT tag to determine whether or not the images +# generated for formulas are transparent PNGs. Transparent PNGs are not +# supported properly for IE 6.0, but are supported on all modern browsers. +# +# Note that when changing this option you need to delete any form_*.png files in +# the HTML output directory before the changes have effect. +# The default value is: YES. +# This tag requires that the tag GENERATE_HTML is set to YES. + +FORMULA_TRANSPARENT = YES + +# Enable the USE_MATHJAX option to render LaTeX formulas using MathJax (see +# http://www.mathjax.org) which uses client side Javascript for the rendering +# instead of using pre-rendered bitmaps. Use this if you do not have LaTeX +# installed or if you want to formulas look prettier in the HTML output. When +# enabled you may also need to install MathJax separately and configure the path +# to it using the MATHJAX_RELPATH option. +# The default value is: NO. +# This tag requires that the tag GENERATE_HTML is set to YES. + +USE_MATHJAX = NO + +# When MathJax is enabled you can set the default output format to be used for +# the MathJax output. See the MathJax site (see: +# http://docs.mathjax.org/en/latest/output.html) for more details. +# Possible values are: HTML-CSS (which is slower, but has the best +# compatibility), NativeMML (i.e. MathML) and SVG. +# The default value is: HTML-CSS. +# This tag requires that the tag USE_MATHJAX is set to YES. + +MATHJAX_FORMAT = HTML-CSS + +# When MathJax is enabled you need to specify the location relative to the HTML +# output directory using the MATHJAX_RELPATH option. The destination directory +# should contain the MathJax.js script. For instance, if the mathjax directory +# is located at the same level as the HTML output directory, then +# MATHJAX_RELPATH should be ../mathjax. The default value points to the MathJax +# Content Delivery Network so you can quickly see the result without installing +# MathJax. However, it is strongly recommended to install a local copy of +# MathJax from http://www.mathjax.org before deployment. +# The default value is: http://cdn.mathjax.org/mathjax/latest. +# This tag requires that the tag USE_MATHJAX is set to YES. + +MATHJAX_RELPATH = http://cdn.mathjax.org/mathjax/latest + +# The MATHJAX_EXTENSIONS tag can be used to specify one or more MathJax +# extension names that should be enabled during MathJax rendering. For example +# MATHJAX_EXTENSIONS = TeX/AMSmath TeX/AMSsymbols +# This tag requires that the tag USE_MATHJAX is set to YES. + +MATHJAX_EXTENSIONS = + +# The MATHJAX_CODEFILE tag can be used to specify a file with javascript pieces +# of code that will be used on startup of the MathJax code. See the MathJax site +# (see: http://docs.mathjax.org/en/latest/output.html) for more details. For an +# example see the documentation. +# This tag requires that the tag USE_MATHJAX is set to YES. + +MATHJAX_CODEFILE = + +# When the SEARCHENGINE tag is enabled doxygen will generate a search box for +# the HTML output. The underlying search engine uses javascript and DHTML and +# should work on any modern browser. Note that when using HTML help +# (GENERATE_HTMLHELP), Qt help (GENERATE_QHP), or docsets (GENERATE_DOCSET) +# there is already a search function so this one should typically be disabled. +# For large projects the javascript based search engine can be slow, then +# enabling SERVER_BASED_SEARCH may provide a better solution. It is possible to +# search using the keyboard; to jump to the search box use + S +# (what the is depends on the OS and browser, but it is typically +# , /