diff --git a/README.md b/README.md index e710f65ed2144d4e6bb3fc21251d2f55e3a7c49d..cbfaa260c01885383eac5a0eca583d0b21599f1d 100644 --- a/README.md +++ b/README.md @@ -38,7 +38,7 @@ Note that before re-running `docker-compose up` you need to run `docker-compose ``` virtualenv -p python3 loom_client_env source loom_client_env/bin/activate -pip3 install cloudpickle protobuf +pip3 install -r python/requirements.txt cd ./python chmod +x generate.sh ./generate.sh diff --git a/pb/comm.proto b/pb/comm.proto index 8978af8b8cb9d099f8452212cdb0dbc915234af8..ea1f2fd1712867d776e62d4ddd2d53962705274e 100644 --- a/pb/comm.proto +++ b/pb/comm.proto @@ -12,6 +12,7 @@ message Task { repeated int32 input_ids = 3; optional int32 resource_request_index = 5 [default = -1]; optional bool result = 6; + optional string checkpoint_path = 7; optional string label = 12; optional bytes metadata = 13; @@ -62,6 +63,7 @@ message WorkerCommand { REMOVE = 3; DICTIONARY = 8; UPDATE = 9; + LOAD_CHECKPOINT = 10; } required Type type = 1; @@ -74,6 +76,9 @@ message WorkerCommand { repeated int32 task_inputs = 5; optional int32 n_cpus = 6; + // TASK + LOAD_CHECKPOINT + optional string checkpoint_path = 7; + // SEND optional string address = 10; @@ -88,17 +93,22 @@ message WorkerCommand { message WorkerResponse { enum Type { FINISHED = 1; - TRANSFERED = 2; - FAILED = 3; + FINISHED_AND_CHECKPOINTING = 2; + TRANSFERED = 3; + FAILED = 4; + CHECKPOINT_WRITTEN = 5; + CHECKPOINT_WRITE_FAILED = 6; + CHECKPOINT_LOADED = 7; + CHECKPOINT_LOAD_FAILED = 8; } required Type type = 1; required int32 id = 2; - // FINISHED + // FINISHED + CHECKPOINT_LOADED optional uint64 size = 3; optional uint64 length = 4; - // FAILED + // FAILED + CHECKPOINT_FAILED optional string error_msg = 100; } @@ -154,6 +164,7 @@ message ClientRequest { // PLAN optional Plan plan = 2; + optional bool load_checkpoints = 4; // FETCH + RELEASE optional int32 id = 3; diff --git a/python/loom/client/client.py b/python/loom/client/client.py index 67500bc3489ede1b98eb38a26ead6f96d05563a7..7d0e7dad034d92536f5c837be2bbc40259b1328b 100644 --- a/python/loom/client/client.py +++ b/python/loom/client/client.py @@ -12,27 +12,9 @@ import struct import cloudpickle import os -LOOM_PROTOCOL_VERSION = 2 - - -class LoomException(Exception): - """Base class for Loom exceptions""" - pass - - -class TaskFailed(LoomException): - """Exception when scheduler informs about failure of a task""" +from .errors import LoomError, LoomException, TaskFailed # noqa - def __init__(self, id, worker, error_msg): - self.id = id - self.worker = worker - self.error_msg = error_msg - message = "Task id={} failed: {}".format(id, error_msg) - LoomException.__init__(self, message) - - -class LoomError(LoomException): - """Generic error in Loom system""" +LOOM_PROTOCOL_VERSION = 2 class Client(object): @@ -259,7 +241,7 @@ class Client(object): print(t) assert 0 - def submit_one(self, task): + def submit_one(self, task, load=False): """Submits a task to the server and returns a future Args: @@ -274,9 +256,9 @@ class Client(object): >>> result = client.submit(task3) >>> print(result.gather()) """ - return self.submit((task,))[0] + return self.submit((task,), load=load)[0] - def submit(self, tasks): + def submit(self, tasks, load=False): """Submits tasks to the server and returns list of futures Args: @@ -297,6 +279,7 @@ class Client(object): futures = self.futures results = [] for task in tasks: + task.validate() if not isinstance(task, Task): raise Exception("{} is not a task".format(task)) plan.add(task) @@ -311,7 +294,7 @@ class Client(object): msg = ClientRequest() msg.type = ClientRequest.PLAN - + msg.load_checkpoints = load include_metadata = self.trace_path is not None msg.plan.id_base = id_base plan.set_message( diff --git a/python/loom/client/errors.py b/python/loom/client/errors.py new file mode 100644 index 0000000000000000000000000000000000000000..86286dd8744931f95648a4c3782d88ea6e7d826e --- /dev/null +++ b/python/loom/client/errors.py @@ -0,0 +1,19 @@ +class LoomException(Exception): + """Base class for Loom exceptions""" + pass + + +class TaskFailed(LoomException): + """Exception when scheduler informs about failure of a task""" + + def __init__(self, id, worker, error_msg): + self.id = id + self.worker = worker + self.error_msg = error_msg + message = "Task id={} failed: {}".format(id, error_msg) + LoomException.__init__(self, message) + + +class LoomError(LoomException): + """Generic error in Loom system""" + pass diff --git a/python/loom/client/plan.py b/python/loom/client/plan.py index 54bd259d768a0af2912a82d417a4fab75408a3af..8708527d04f7a4dc1adccd68c526dbd70ea99357 100644 --- a/python/loom/client/plan.py +++ b/python/loom/client/plan.py @@ -69,6 +69,8 @@ class Plan(object): msg_t.task_type = symbols[task.task_type] msg_t.input_ids.extend(self.get_id(t) for t in task.inputs) msg_t.result = task in results + if task.checkpoint_path: + msg_t.checkpoint_path = task.checkpoint_path if task.resource_request: msg_t.resource_request_index = \ requests.index(task.resource_request) diff --git a/python/loom/client/task.py b/python/loom/client/task.py index 0c68e3512be802c4ac20fd024f8fdf34b41e071e..3ff7fdde1495d5afbffe4f3d30b7ced2198ccebc 100644 --- a/python/loom/client/task.py +++ b/python/loom/client/task.py @@ -1,3 +1,6 @@ +import os.path +from .errors import LoomError + class Task(object): @@ -7,6 +10,12 @@ class Task(object): resource_request = None label = None metadata = None + checkpoint_path = None + + def validate(self): + if self.checkpoint_path is not None \ + and not os.path.isabs(self.checkpoint_path): + raise LoomError("Checkpoint has to be absolute path") def __repr__(self): if self.label: diff --git a/python/requirements.txt b/python/requirements.txt new file mode 100644 index 0000000000000000000000000000000000000000..a4ac96967438276ad4153e6fe74f99b528ad286b --- /dev/null +++ b/python/requirements.txt @@ -0,0 +1,3 @@ +bokeh==0.12.4 +protobuf==3.2.0 +cloudpickle==0.2.2 diff --git a/src/libloom/fsutils.cpp b/src/libloom/fsutils.cpp index fe273b4a127393ea189e4e8b8d2e0417f40b2e7a..9840ca2b98b87e4758853a0389bef0878fd5544c 100644 --- a/src/libloom/fsutils.cpp +++ b/src/libloom/fsutils.cpp @@ -61,3 +61,9 @@ int loom::base::make_path(const char *path, mode_t mode) } return 0; } + +bool loom::base::file_exists(const char *path) +{ + struct stat buffer; + return (stat(path, &buffer) == 0); +} diff --git a/src/libloom/fsutils.h b/src/libloom/fsutils.h index c878c374ce40289e3ced862eb8575f39414591d8..49bbd4fe9e01203dab25774c8cbfdbf4d3ad467f 100644 --- a/src/libloom/fsutils.h +++ b/src/libloom/fsutils.h @@ -12,6 +12,7 @@ namespace base { int make_path(const char *path, mode_t mode); size_t file_size(const char *path); +bool file_exists(const char *path); } } diff --git a/src/libloom/log.cpp b/src/libloom/log.cpp index 7cacf8d0a84a836ece70e8227dc7b306a4e7e0e8..148a31d91c9fd8f8cbfd3223e5ac2c7df7ecce1d 100644 --- a/src/libloom/log.cpp +++ b/src/libloom/log.cpp @@ -10,7 +10,7 @@ using namespace loom::base; namespace loom { namespace base { -std::shared_ptr<spdlog::logger> logger = spdlog::stdout_logger_mt("net", true); +std::shared_ptr<spdlog::logger> logger = spdlog::stdout_logger_mt("net"); }} void loom::base::report_uv_error(int error_code, int line_number, const char *filename) diff --git a/src/libloomw/CMakeLists.txt b/src/libloomw/CMakeLists.txt index 8c0f9910e80772ee3cb0ee38ae1847505cddda25..ec8babeedd83cad3d3a263227638f143740625d8 100644 --- a/src/libloomw/CMakeLists.txt +++ b/src/libloomw/CMakeLists.txt @@ -48,6 +48,8 @@ add_library(libloomw resalloc.cpp task.cpp task.h + checkpointwriter.h + checkpointwriter.cpp wtrace.cpp wtrace.h taskdesc.h diff --git a/src/libloomw/checkpointwriter.cpp b/src/libloomw/checkpointwriter.cpp new file mode 100644 index 0000000000000000000000000000000000000000..f369f07d3a2827cfff0234e5816790b60caddcc5 --- /dev/null +++ b/src/libloomw/checkpointwriter.cpp @@ -0,0 +1,57 @@ +#include "checkpointwriter.h" +#include "worker.h" +#include "libloom/log.h" +#include <stdio.h> + +loom::CheckPointWriter::CheckPointWriter(loom::Worker &worker, base::Id id, const loom::DataPtr &data, const std::string &path) + : worker(worker), id(id), data(data), path(path) +{ + work.data = this; +} + + +void loom::CheckPointWriter::start() +{ + UV_CHECK(uv_queue_work(worker.get_loop(), &work, _work_cb, _after_work_cb)); +} + +void loom::CheckPointWriter::_work_cb(uv_work_t *req) +{ + CheckPointWriter *writer = static_cast<CheckPointWriter*>(req->data); + + std::string &path = writer->path; + std::string tmp_path = path + ".loom.tmp"; + + std::ofstream fout(tmp_path.c_str()); + if (!fout.is_open()) { + writer->error = "Writing checkpoint '" + path + "' failed. Cannot create " + tmp_path + ":" + strerror(errno); + return; + } + + const char *ptr = writer->data->get_raw_data(); + + if (!ptr) { + writer->error = "Data '" + writer->data->get_info() + "' cannot be checkpointed"; + return; + } + + fout.write(writer->data->get_raw_data(), writer->data->get_size()); + fout.close(); + + if (rename(tmp_path.c_str(), path.c_str())) { + writer->error = "Writing checkpoint '" + path + "' failed. Cannot move " + tmp_path; + unlink(tmp_path.c_str()); + } +} + +void loom::CheckPointWriter::_after_work_cb(uv_work_t *req, int status) +{ + UV_CHECK(status); + CheckPointWriter *writer = static_cast<CheckPointWriter*>(req->data); + if (writer->error.empty()) { + writer->worker.checkpoint_written(writer->id); + } else { + writer->worker.checkpoint_write_failed(writer->id, writer->error); + } + delete writer; +} diff --git a/src/libloomw/checkpointwriter.h b/src/libloomw/checkpointwriter.h new file mode 100644 index 0000000000000000000000000000000000000000..d7f13e1abc936ca3139ed223bdc70c52dba05773 --- /dev/null +++ b/src/libloomw/checkpointwriter.h @@ -0,0 +1,30 @@ +#ifndef LIBLOOMW_CHECKPOINTWRITER_H +#define LIBLOOMW_CHECKPOINTWRITER_H + +#include "libloom/types.h" +#include "data.h" + +namespace loom { + +class CheckPointWriter { + +public: + CheckPointWriter(Worker &worker, base::Id id, const DataPtr &data, const std::string &path); + void start(); + +protected: + uv_work_t work; + Worker &worker; + base::Id id; + DataPtr data; + std::string path; + std::string error; + +private: + static void _work_cb(uv_work_t *req); + static void _after_work_cb(uv_work_t *req, int status); +}; + +} + +#endif diff --git a/src/libloomw/interconnect.cpp b/src/libloomw/interconnect.cpp index c2e67041bd54464ca440fe705bb49cdde99ba8d6..3e247513870e7a3f79ec4f144fa9e5ea1250ad6c 100644 --- a/src/libloomw/interconnect.cpp +++ b/src/libloomw/interconnect.cpp @@ -54,7 +54,7 @@ void InterConnection::finish_receive() { logger->debug("Interconnect: Data id={} received", unpacking_data_id); worker.data_transferred(unpacking_data_id); - worker.publish_data(unpacking_data_id, unpacker->finish()); + worker.publish_data(unpacking_data_id, unpacker->finish(), ""); auto &trace = worker.get_trace(); if (trace) { diff --git a/src/libloomw/task.h b/src/libloomw/task.h index 7bd84365c402f8705ac3f95025c9555b549ec06e..17b530e4c184298c14d91f39d333af42f4b51d31 100644 --- a/src/libloomw/task.h +++ b/src/libloomw/task.h @@ -16,8 +16,8 @@ class Worker; class Task { public: - Task(base::Id id, int task_type, const std::string &config, int n_cpus) - : id(id), task_type(task_type), config(config), n_cpus(n_cpus), n_unresolved(0) {} + Task(base::Id id, int task_type, const std::string &config, int n_cpus, const std::string &checkpoint_path) + : id(id), task_type(task_type), config(config), n_cpus(n_cpus), n_unresolved(0), checkpoint_path(checkpoint_path) {} Task(base::Id id, int task_type, std::string &&config, int n_cpus) : id(id), task_type(task_type), config(std::move(config)), n_cpus(n_cpus), n_unresolved(0) {} @@ -47,6 +47,10 @@ public: return n_cpus; } + const std::string& get_checkpoint_path() const { + return checkpoint_path; + } + const std::vector<base::Id>& get_inputs() const { return inputs; } @@ -61,6 +65,7 @@ protected: int n_cpus; size_t n_unresolved; std::unordered_set<base::Id> unresolved_set; + std::string checkpoint_path; }; } diff --git a/src/libloomw/taskinstance.cpp b/src/libloomw/taskinstance.cpp index 03e8d612a1240880c36ffda2dfed90c57cd74def..f43376ee8415a8143732a5f95f968c015f967b84 100644 --- a/src/libloomw/taskinstance.cpp +++ b/src/libloomw/taskinstance.cpp @@ -54,9 +54,9 @@ void TaskInstance::fail_libuv(const std::string &error_msg, int error_code) void TaskInstance::finish(const DataPtr &output) { assert(output); - worker.publish_data(get_id(), output); + worker.publish_data(get_id(), output, task->get_checkpoint_path()); assert(output); - worker.task_finished(*this, output); + worker.task_finished(*this, output, !task->get_checkpoint_path().empty()); } void TaskInstance::redirect(std::unique_ptr<TaskDescription> tdesc) diff --git a/src/libloomw/worker.cpp b/src/libloomw/worker.cpp index b80e78247dec9f6b780128eaebb38710d9049cf6..f1140ebe53c5dd3dc495d3b319f3e6a30266617f 100644 --- a/src/libloomw/worker.cpp +++ b/src/libloomw/worker.cpp @@ -21,8 +21,10 @@ #include "libloom/sendbuffer.h" #include "libloom/pbutils.h" #include "libloom/fsutils.h" +#include "data/externfile.h" #include "loom_define.h" +#include "checkpointwriter.h" #include <stdlib.h> #include <sstream> @@ -283,10 +285,59 @@ void Worker::start_task(std::unique_ptr<Task> task, ResourceAllocation &&ra) t->start(input_data); } -void Worker:: publish_data(Id id, const DataPtr &data) +void Worker::checkpoint_written(Id id) { + logger->debug("Checkpoint written id={}", id); + if (server_conn.is_connected()) { + loom::pb::comm::WorkerResponse msg; + msg.set_type(loom::pb::comm::WorkerResponse_Type_CHECKPOINT_WRITTEN); + msg.set_id(id); + send_message(server_conn, msg); + } +} + +void Worker::checkpoint_write_failed(Id id, const std::string &error_msg) { + logger->debug("Cannot write checkpoint id={}, error={}", id, error_msg); + if (server_conn.is_connected()) { + loom::pb::comm::WorkerResponse msg; + msg.set_type(loom::pb::comm::WorkerResponse_Type_CHECKPOINT_WRITE_FAILED); + msg.set_id(id); + msg.set_error_msg(error_msg); + send_message(server_conn, msg); + } +} + +void Worker::checkpoint_loaded(Id id, const DataPtr &data) { + logger->debug("Checkpoint loaded id={}", id); + if (server_conn.is_connected()) { + loom::pb::comm::WorkerResponse msg; + msg.set_type(loom::pb::comm::WorkerResponse_Type_CHECKPOINT_LOADED); + msg.set_id(id); + msg.set_size(data->get_size()); + msg.set_length(data->get_length()); + send_message(server_conn, msg); + } +} + +void Worker::checkpoint_load_failed(Id id, const std::string &error_msg) { + logger->debug("Cannot load checkpoint id={}, error={}", id, error_msg); + if (server_conn.is_connected()) { + loom::pb::comm::WorkerResponse msg; + msg.set_type(loom::pb::comm::WorkerResponse_Type_CHECKPOINT_LOAD_FAILED); + msg.set_id(id); + msg.set_error_msg(error_msg); + send_message(server_conn, msg); + } +} + +void Worker::publish_data(Id id, const DataPtr &data, const std::string &checkpoint_path) { logger->debug("Publishing data id={} size={} info={}", id, data->get_size(), data->get_info()); public_data[id] = data; + + if (!checkpoint_path.empty()) { + write_checkpoint(id, data, checkpoint_path); + } + check_waiting_tasks(id); } @@ -480,6 +531,20 @@ void Worker::remove_task(TaskInstance &task, bool free_resources) assert(0); } +void Worker::load_checkpoint(Id id, const std::string &path) { + if (!file_exists(path.c_str())) { + std::stringstream s; + s << "File '" << path << "' does not exists"; + std::string error = s.str(); + logger->error("Cannot load checkpoint {}: {}", id, error); + checkpoint_load_failed(id, error); + return; + } + DataPtr data = std::make_shared<ExternFile>(path); + checkpoint_loaded(id, data); + publish_data(id, data, ""); +} + void Worker::task_failed(TaskInstance &task, const std::string &error_msg) { logger->error("Task id={} failed: {}", task.get_id(), error_msg); @@ -519,12 +584,18 @@ void Worker::task_redirect(TaskInstance &task, t->start(new_task_desc->inputs); } -void Worker::task_finished(TaskInstance &task, const DataPtr &data) +void Worker::write_checkpoint(Id id, const DataPtr &data, const std::string &checkpoint_path) +{ + CheckPointWriter *writer = new CheckPointWriter(*this, id, data, checkpoint_path); + writer->start(); +} + +void Worker::task_finished(TaskInstance &task, const DataPtr &data, bool checkpointing) { using namespace loom::pb::comm; if (server_conn.is_connected()) { WorkerResponse msg; - msg.set_type(WorkerResponse_Type_FINISHED); + msg.set_type(checkpointing ? WorkerResponse_Type_FINISHED_AND_CHECKPOINTING : WorkerResponse_Type_FINISHED); msg.set_id(task.get_id()); msg.set_size(data->get_size()); msg.set_length(data->get_length()); @@ -571,7 +642,8 @@ void Worker::on_message(const char *data, size_t size) auto task = std::make_unique<Task>(msg.id(), msg.task_type(), msg.task_config(), - msg.n_cpus()); + msg.n_cpus(), + msg.checkpoint_path()); for (int i = 0; i < msg.task_inputs_size(); i++) { Id task_id = msg.task_inputs(i); task->add_input(task_id); @@ -603,6 +675,12 @@ void Worker::on_message(const char *data, size_t size) } break; } + case comm::WorkerCommand_Type_LOAD_CHECKPOINT: { + logger->debug("Loading checkpoint id={} path={}", msg.id(), msg.checkpoint_path()); + assert(msg.has_checkpoint_path()); + load_checkpoint(msg.id(), msg.checkpoint_path()); + break; + } case comm::WorkerCommand_Type_DICTIONARY: { auto count = msg.symbols_size(); logger->debug("New dictionary ({} symbols)", count); diff --git a/src/libloomw/worker.h b/src/libloomw/worker.h index 685c7ef1c6e8617cf10a290afb8d77c402b9e8a6..bf8e5422342103d3b8cacf5d0cd695e4b2b338e1 100644 --- a/src/libloomw/worker.h +++ b/src/libloomw/worker.h @@ -52,12 +52,13 @@ public: return true; } - void task_finished(TaskInstance &task_instance, const DataPtr &data); + void task_finished(TaskInstance &task_instance, const DataPtr &data, bool checkpointing); void task_failed(TaskInstance &task_instance, const std::string &error_msg); void data_transferred(base::Id task_id); void task_redirect(TaskInstance &task, std::unique_ptr<TaskDescription> new_task_desc); - void publish_data(base::Id id, const DataPtr &data); + void publish_data(base::Id id, const DataPtr &data, const std::string &checkpoint_path); + void write_checkpoint(base::Id id, const DataPtr &data, const std::string &checkpoint_path); void remove_data(base::Id id); bool has_data(base::Id id) const @@ -126,8 +127,16 @@ public: return trace; } + + void on_dictionary_updated(); + + void load_checkpoint(base::Id id, const std::string &path); + void checkpoint_written(base::Id id); + void checkpoint_write_failed(base::Id id, const std::string &error_msg); + void checkpoint_loaded(base::Id id, const DataPtr &Data); + void checkpoint_load_failed(base::Id id, const std::string &error_msg); private: void register_worker(); void create_trace(const std::string &trace_path, loom::base::Id worker_id); diff --git a/src/server/clientconn.cpp b/src/server/clientconn.cpp index 22c9fcbf992bee86ba4c058319ed8b3ace981416..a876e5064745c840722db6da4fc2252fffdb6d0f 100644 --- a/src/server/clientconn.cpp +++ b/src/server/clientconn.cpp @@ -56,8 +56,8 @@ void ClientConnection::on_message(const char *buffer, size_t size) case ClientRequest_Type_PLAN: { logger->debug("Plan received"); const Plan &plan = request.plan(); - loom::base::Id id_base = task_manager.add_plan(plan); - logger->info("Plan submitted tasks={}", plan.tasks_size()); + loom::base::Id id_base = task_manager.add_plan(plan, request.load_checkpoints()); + logger->info("Plan submitted tasks={}, load_checkpoints={}", plan.tasks_size(), request.load_checkpoints()); if (server.get_trace()) { server.create_file_in_trace_dir(std::to_string(id_base) + ".plan", buffer, size); diff --git a/src/server/compstate.cpp b/src/server/compstate.cpp index 39857df4ebc0cce73d20561c89c1a00737c635ab..cc2084457a9c42c8fd02734238200fcddb0ff509 100644 --- a/src/server/compstate.cpp +++ b/src/server/compstate.cpp @@ -5,6 +5,7 @@ #include "pb/comm.pb.h" #include "libloom/log.h" +#include "libloom/fsutils.h" constexpr static double TRANSFER_COST_COEF = 1.0 / (1024 * 1024); // 1MB = 1cost @@ -22,16 +23,34 @@ ComputationState::ComputationState(Server &server) : server(server) void ComputationState::add_node(std::unique_ptr<TaskNode> &&node) { auto id = node->get_id(); - for (TaskNode* input_node : node->get_inputs()) { - input_node->add_next(node.get()); + auto result = nodes.insert(std::make_pair(id, std::move(node))); + assert(result.second); // Check that ID is fresh +} + +void ComputationState::plan_node(TaskNode &node, bool load_checkpoints, std::vector<TaskNode *> &to_load) { + if (node.is_planned()) { + return; } + node.set_planned(); - if (node->is_ready()) { - pending_nodes.insert(node.get()); + if (load_checkpoints && !node.get_task_def().checkpoint_path.empty() && loom::base::file_exists(node.get_task_def().checkpoint_path.c_str())) { + node.set_checkpoint(); + to_load.push_back(&node); + return; } - auto result = nodes.insert(std::make_pair(id, std::move(node))); - assert(result.second); // Check that ID is fresh + int remaining_inputs = 0; + for (TaskNode *input_node : node.get_inputs()) { + plan_node(*input_node, load_checkpoints, to_load); + if (!input_node->is_computed()) { + remaining_inputs += 1; + input_node->add_next(&node); + } + } + node.set_remaining_inputs(remaining_inputs); + if (remaining_inputs == 0) { + pending_nodes.insert(&node); + } } void ComputationState::reserve_new_nodes(size_t size) @@ -241,7 +260,7 @@ void ComputationState::make_expansion(std::vector<std::string> &configs, } }*/ -loom::base::Id ComputationState::add_plan(const loom::pb::comm::Plan &plan) +loom::base::Id ComputationState::add_plan(const loom::pb::comm::Plan &plan, bool load_checkpoints, std::vector<TaskNode*> &to_load) { auto task_size = plan.tasks_size(); assert(plan.has_id_base()); @@ -267,8 +286,11 @@ loom::base::Id ComputationState::add_plan(const loom::pb::comm::Plan &plan) def.task_type = pt.task_type(); def.config = pt.config(); + def.checkpoint_path = pt.checkpoint_path(); + bool is_result = false; if (pt.has_result() && pt.result()) { - def.flags.set(static_cast<size_t>(TaskFlags::RESULT)); + is_result = true; + def.flags.set(static_cast<size_t>(TaskDefFlags::RESULT)); } auto inputs_size = pt.input_ids_size(); @@ -283,7 +305,12 @@ loom::base::Id ComputationState::add_plan(const loom::pb::comm::Plan &plan) n_cpus = resources[pt.resource_request_index()]; } def.n_cpus = n_cpus; - add_node(std::make_unique<TaskNode>(id, std::move(def))); + + auto new_node = std::make_unique<TaskNode>(id, std::move(def)); + if (is_result) { + plan_node(*new_node.get(), load_checkpoints, to_load); + } + add_node(std::move(new_node)); } return id_base; } diff --git a/src/server/compstate.h b/src/server/compstate.h index c930e18c42745e8afa0b702e30cd966a7ad52054..797042bfbdc2d355bd73b25e80344c18d318ebad 100644 --- a/src/server/compstate.h +++ b/src/server/compstate.h @@ -41,11 +41,9 @@ public: void remove_node(TaskNode &node); - bool is_ready(const TaskNode &node); - int get_n_data_objects() const; - loom::base::Id add_plan(const loom::pb::comm::Plan &plan); + loom::base::Id add_plan(const loom::pb::comm::Plan &plan, bool load_checkpoints, std::vector<TaskNode *> &to_load); void test_ready_nodes(std::vector<loom::base::Id> ids); loom::base::Id pop_result_client_id(loom::base::Id id); @@ -62,7 +60,8 @@ public: std::unique_ptr<TaskNode> pop_node(loom::base::Id id); void clear_all(); - void add_pending_node(TaskNode &node); + void add_pending_node(TaskNode &node); + void plan_node(TaskNode &node, bool load_checkpoints, std::vector<TaskNode*> &to_load); private: std::unordered_map<loom::base::Id, std::unique_ptr<TaskNode>> nodes; diff --git a/src/server/scheduler.cpp b/src/server/scheduler.cpp index c7c17b1f4b04e1b5b652cd3e13bbc9f0b3d35a41..589df76bdff9b19f04ca041894fdfea32c037bcb 100644 --- a/src/server/scheduler.cpp +++ b/src/server/scheduler.cpp @@ -128,7 +128,7 @@ static void compute_table(const TaskNode *node, if (node == input_node) { continue; } - if (input_node->has_state()) { + if (input_node->is_computed()) { Score score = score_from_next_size(input_node->get_size()); for (const auto &pair : input_node->get_workers()) { WorkerConnection *wc = pair.first; @@ -340,7 +340,7 @@ TaskDistribution schedule(const ComputationState &cstate) continue; } for (TaskNode *input_node : next_node->get_inputs()) { - if (input_node == best_node || input_node->has_state()) { + if (input_node == best_node || input_node->is_computed()) { continue; } auto it = context.units.find(input_node->get_id()); diff --git a/src/server/server.cpp b/src/server/server.cpp index 7ea1b9501338435fed2f03c36658292e1cae5483..9bf63f69b84371445f4a7bd1f369df8d4971e616 100644 --- a/src/server/server.cpp +++ b/src/server/server.cpp @@ -81,9 +81,29 @@ void Server::remove_freshconnection(FreshConnection &conn) fresh_connections.erase(i); } -void Server::on_task_finished(loom::base::Id id, size_t size, size_t length, WorkerConnection *wc) +void Server::on_checkpoint_write_finished(loom::base::Id id, WorkerConnection *wc) { - task_manager.on_task_finished(id, size, length, wc); + task_manager.on_checkpoint_write_finished(id, wc); +} + +void Server::on_checkpoint_write_failed(loom::base::Id id, WorkerConnection *wc, const std::string &error_msg) +{ + task_manager.on_checkpoint_write_failed(id, wc, error_msg); +} + +void Server::on_checkpoint_load_finished(loom::base::Id id, WorkerConnection *wc, size_t size, size_t length) +{ + task_manager.on_checkpoint_load_finished(id, wc, size, length); +} + +void Server::on_checkpoint_load_failed(loom::base::Id id, WorkerConnection *wc, const std::string &error_msg) +{ + task_manager.on_checkpoint_load_failed(id, wc, error_msg); +} + +void Server::on_task_finished(loom::base::Id id, size_t size, size_t length, WorkerConnection *wc, bool checkpointing) +{ + task_manager.on_task_finished(id, size, length, wc, checkpointing); } void Server::on_data_transferred(loom::base::Id id, WorkerConnection *wc) diff --git a/src/server/server.h b/src/server/server.h index 7abbda465e55ab6d491d080372030ad9a2fdbfc3..3382500548e6decaaec8143baae4eaa6d6b1ddef 100644 --- a/src/server/server.h +++ b/src/server/server.h @@ -56,7 +56,7 @@ public: void add_resend_task(loom::base::Id id); - void on_task_finished(loom::base::Id id, size_t size, size_t length, WorkerConnection *wc); + void on_task_finished(loom::base::Id id, size_t size, size_t length, WorkerConnection *wc, bool checkpointing); void on_data_transferred(loom::base::Id id, WorkerConnection *wc); loom::base::Dictionary& get_dictionary() { @@ -88,6 +88,10 @@ public: return trace; } + void on_checkpoint_write_failed(loom::base::Id id, WorkerConnection *wc, const std::string &error_msg); + void on_checkpoint_write_finished(loom::base::Id id, WorkerConnection *wc); + void on_checkpoint_load_failed(loom::base::Id id, WorkerConnection *wc, const std::string &error_msg); + void on_checkpoint_load_finished(loom::base::Id id, WorkerConnection *wc, size_t size, size_t length); private: diff --git a/src/server/taskmanager.cpp b/src/server/taskmanager.cpp index 4d2108809672f1b634d342fe64e9bb61058763b5..aca80595fcbe96c55ca6fcc880a9bf87b8bc9e66 100644 --- a/src/server/taskmanager.cpp +++ b/src/server/taskmanager.cpp @@ -9,6 +9,8 @@ #include <algorithm> #include <assert.h> #include <limits.h> +#include <stdlib.h> +#include <memory> using namespace loom; using namespace loom::base; @@ -18,9 +20,15 @@ TaskManager::TaskManager(Server &server) { } -loom::base::Id TaskManager::add_plan(const loom::pb::comm::Plan &plan) +loom::base::Id TaskManager::add_plan(const loom::pb::comm::Plan &plan, bool load_checkpoints) { - loom::base::Id id_base = cstate.add_plan(plan); + std::vector<TaskNode*> to_load; + loom::base::Id id_base = cstate.add_plan(plan, load_checkpoints, to_load); + for (TaskNode *node : to_load) { + WorkerConnection *wc = random_worker(); + node->set_as_loading(wc); + wc->load_checkpoint(node->get_id(), node->get_task_def().checkpoint_path); + } distribute_work(schedule(cstate)); return id_base; } @@ -76,13 +84,14 @@ void TaskManager::remove_node(TaskNode &node) assert(status == TaskStatus::OWNER); wc->remove_data(id); }); - cstate.remove_node(node); + node.set_not_needed(); + //cstate.remove_node(node); } -void TaskManager::on_task_finished(loom::base::Id id, size_t size, size_t length, WorkerConnection *wc) +void TaskManager::on_task_finished(loom::base::Id id, size_t size, size_t length, WorkerConnection *wc, bool checkpointing) { if (unlikely(wc->is_blocked())) { - wc->residual_task_finished(id, true); + wc->residual_task_finished(id, true, checkpointing); return; } TaskNode &node = cstate.get_node(id); @@ -93,35 +102,39 @@ void TaskManager::on_task_finished(loom::base::Id id, size_t size, size_t length trace->trace_task_end(*node, wc); }*/ - if (node.is_result()) { + // Task with checkpoints is "finished" for worker only if + // checkpoint is written + if (!checkpointing && node.is_result()) { logger->debug("Job id={} [RESULT] finished", id); - /*WorkerConnection *owner = node->get_random_owner(); - assert(owner); - owner->send_data(id, server.get_dummy_worker().get_address());*/ ClientConnection *cc = server.get_client_connection(); if (cc) { - cc->send_info_about_finished_result(node); + cc->send_info_about_finished_result(node); } } else { - assert(!node.get_nexts().empty()); + assert(checkpointing || !node.get_nexts().empty()); logger->debug("Job id={} finished (size={}, length={})", id, size, length); } + if (checkpointing) { + wc->change_checkpoint_writes(1); + } + for (TaskNode *input_node : node.get_inputs()) { if (input_node->next_finished(node) && !input_node->is_result()) { remove_node(*input_node); } } + if (!node.get_nexts().empty()) { for (TaskNode *nn : node.get_nexts()) { if (nn->input_is_ready(&node)) { cstate.add_pending_node(*nn); } } - } /*else { - remove_node(*node); - }*/ + } else if (!node.is_result()) { + remove_node(node); + } if (cstate.has_pending_nodes()) { server.need_task_distribution(); @@ -131,7 +144,7 @@ void TaskManager::on_task_finished(loom::base::Id id, size_t size, size_t length void TaskManager::on_data_transferred(Id id, WorkerConnection *wc) { if (unlikely(wc->is_blocked())) { - wc->residual_task_finished(id, true); + wc->residual_task_finished(id, true, false); return; } TaskNode &node = cstate.get_node(id); @@ -142,7 +155,7 @@ void TaskManager::on_data_transferred(Id id, WorkerConnection *wc) void TaskManager::on_task_failed(Id id, WorkerConnection *wc, const std::string &error_msg) { if (unlikely(wc->is_blocked())) { - wc->residual_task_finished(id, false); + wc->residual_task_finished(id, false, false); return; } logger->error("Task id={} failed on worker {}: {}", @@ -162,6 +175,92 @@ void TaskManager::on_task_failed(Id id, WorkerConnection *wc, const std::string } } +void TaskManager::on_checkpoint_write_finished(Id id, WorkerConnection *wc) +{ + if (unlikely(wc->is_blocked())) { + wc->residual_checkpoint_finished(id); + return; + } + logger->debug("Checkpoint id={} finished on worker {}", id, wc->get_address()); + wc->change_checkpoint_writes(-1); + TaskNode &node = cstate.get_node(id); + assert(node.has_defined_checkpoint()); + node.set_checkpoint(); + + if (node.is_result()) { + ClientConnection *cc = server.get_client_connection(); + if (cc) { + cc->send_info_about_finished_result(node); + } + } +} + +void TaskManager::on_checkpoint_load_finished(Id id, WorkerConnection *wc, size_t size, size_t length) +{ + if (unlikely(wc->is_blocked())) { + wc->residual_task_finished(id, false, false); + return; + } + wc->change_checkpoint_loads(-1); + + TaskNode &node = cstate.get_node(id); + node.set_as_loaded(wc, size, length); + + if (node.is_result()) { + logger->debug("Task id={} [RESULT] checkpoint loaded", id); + + ClientConnection *cc = server.get_client_connection(); + if (cc) { + cc->send_info_about_finished_result(node); + } + } else { + logger->debug("Task id={} checkpoint loaded", id); + } + + if (!node.get_nexts().empty()) { + for (TaskNode *nn : node.get_nexts()) { + if (nn->input_is_ready(&node)) { + cstate.add_pending_node(*nn); + } + } + } else if (!node.is_result()) { + remove_node(node); + } + if (cstate.has_pending_nodes()) { + server.need_task_distribution(); + } +} + +void TaskManager::on_checkpoint_write_failed(Id id, WorkerConnection *wc, const std::string &error_msg) +{ + wc->change_checkpoint_writes(-1); + if (unlikely(wc->is_blocked())) { + return; + } + logger->error("Checkpoint id={} failed on worker {}: {}", + id, wc->get_address(), error_msg); + auto cc = server.get_client_connection(); + if (cc) { + cc->send_task_failed(id, *wc, error_msg); + } + trash_all_tasks(); +} + +void TaskManager::on_checkpoint_load_failed(Id id, WorkerConnection *wc, const std::string &error_msg) +{ + if (unlikely(wc->is_blocked())) { + wc->residual_task_finished(id, false, false); + return; + } + wc->change_checkpoint_loads(-1); + logger->error("Checkpoint id={} load failed on worker {}: {}", + id, wc->get_address(), error_msg); + auto cc = server.get_client_connection(); + if (cc) { + cc->send_task_failed(id, *wc, error_msg); + } +} + void TaskManager::run_task_distribution() { uv_loop_t *loop = server.get_loop(); @@ -198,24 +297,25 @@ void TaskManager::run_task_distribution() void TaskManager::trash_all_tasks() { + for (auto &wc : server.get_connections()) { + wc->change_residual_tasks(wc->get_checkpoint_loads()); + wc->change_checkpoint_loads(-wc->get_checkpoint_loads()); + } cstate.foreach_node([](std::unique_ptr<TaskNode> &task) { - if (task->has_state()) { - task->foreach_worker([&task](WorkerConnection *wc, TaskStatus &status) { - if (status == TaskStatus::OWNER) { - wc->remove_data(task->get_id()); - } else if (status == TaskStatus::RUNNING) { - wc->change_residual_tasks(1); - wc->free_resources(*task); - logger->debug("Residual task id={} on worker={}", task->get_id(), wc->get_worker_id()); - } else { - assert(status == TaskStatus::TRANSFER); - wc->change_residual_tasks(1); - logger->debug("Residual transfer id={} on worker={}", task->get_id(), wc->get_worker_id()); - } - status = TaskStatus::NONE; - }); - - } + task->foreach_worker([&task](WorkerConnection *wc, TaskStatus status) { + if (status == TaskStatus::OWNER) { + wc->remove_data(task->get_id()); + } else if (status == TaskStatus::RUNNING) { + wc->change_residual_tasks(1); + wc->free_resources(*task); + logger->debug("Residual task id={} on worker={}", task->get_id(), wc->get_worker_id()); + } else { + assert(status == TaskStatus::TRANSFER); + wc->change_residual_tasks(1); + logger->debug("Residual transfer id={} on worker={}", task->get_id(), wc->get_worker_id()); + } + status = TaskStatus::NONE; + }); }); cstate.clear_all(); } @@ -230,3 +330,11 @@ void TaskManager::release_node(TaskNode *node) node->reset_result_flag(); } } + +WorkerConnection *TaskManager::random_worker() +{ + auto &connections = server.get_connections(); + assert(!connections.empty()); + int index = rand() % connections.size(); + return connections[index].get(); +} diff --git a/src/server/taskmanager.h b/src/server/taskmanager.h index 89e565138f9b4f4e942fd5a9637bc38767a1ca07..aabf619d3241a481e4b58eba0a62c4b821c3a273 100644 --- a/src/server/taskmanager.h +++ b/src/server/taskmanager.h @@ -23,11 +23,15 @@ public: cstate.add_node(std::move(node)); }*/ - loom::base::Id add_plan(const loom::pb::comm::Plan &plan); + loom::base::Id add_plan(const loom::pb::comm::Plan &plan, bool load_checkpoints); - void on_task_finished(loom::base::Id id, size_t size, size_t length, WorkerConnection *wc); + void on_task_finished(loom::base::Id id, size_t size, size_t length, WorkerConnection *wc, bool checkpointing); void on_data_transferred(loom::base::Id id, WorkerConnection *wc); void on_task_failed(loom::base::Id id, WorkerConnection *wc, const std::string &error_msg); + void on_checkpoint_write_finished(loom::base::Id id, WorkerConnection *wc); + void on_checkpoint_write_failed(loom::base::Id id, WorkerConnection *wc, const std::string &error_msg); + void on_checkpoint_load_finished(loom::base::Id id, WorkerConnection *wc, size_t size, size_t length); + void on_checkpoint_load_failed(loom::base::Id id, WorkerConnection *wc, const std::string &error_msg); int get_n_of_data_objects() const { return cstate.get_n_data_objects(); @@ -42,6 +46,7 @@ public: void trash_all_tasks(); void release_node(TaskNode *node); + WorkerConnection *random_worker(); private: Server &server; diff --git a/src/server/tasknode.cpp b/src/server/tasknode.cpp index 1d9fdbd2e0140b480cd690498ae922c6f2c83904..3214c88cc2e4cc8372a8bd5206b66297810778bb 100644 --- a/src/server/tasknode.cpp +++ b/src/server/tasknode.cpp @@ -6,7 +6,11 @@ #include <sstream> TaskNode::TaskNode(loom::base::Id id, TaskDef &&task) - : id(id), task(std::move(task)) + : id(id), + task(std::move(task)), + size(0), + length(0), + remaining_inputs(0) { } @@ -14,27 +18,12 @@ TaskNode::TaskNode(loom::base::Id id, TaskDef &&task) void TaskNode::reset_result_flag() { assert(is_result()); - task.flags.reset(static_cast<size_t>(TaskFlags::RESULT)); -} - -bool TaskNode::is_computed() const { - if (!state) { - return false; - } - for(auto &pair : state->workers) { - if (pair.second == TaskStatus::OWNER) { - return true; - } - } - return false; + task.flags.reset(static_cast<size_t>(TaskDefFlags::RESULT)); } WorkerConnection *TaskNode::get_random_owner() { - if (!state) { - return nullptr; - } - for(auto &pair : state->workers) { + for(auto &pair : workers) { if (pair.second == TaskStatus::OWNER) { return pair.first; } @@ -42,41 +31,9 @@ WorkerConnection *TaskNode::get_random_owner() return nullptr; } -void TaskNode::create_state(TaskNode *just_finishing_input) -{ - /* just_finishing_input has to be introduced - * to solve situation when a task takes the same input multiple times - * and an input is finished. When we create a state in situation without - * finishing a task, than just_finishing_input has to be nullptr - */ - assert(!state); - state = std::make_unique<RuntimeState>(); - state->size = 0; - state->length = 0; - int remaining_inputs = 0; - - if (just_finishing_input) { - for (TaskNode *input_node : task.inputs) { - if (just_finishing_input == input_node || !input_node->is_computed()) { - remaining_inputs += 1; - } - } - } else { - for (TaskNode *input_node : task.inputs) { - if (!input_node->is_computed()) { - remaining_inputs += 1; - } - } - } - state->remaining_inputs = remaining_inputs; -} - bool TaskNode::is_active() const { - if (!state) { - return false; - } - for (auto &pair : state->workers) { + for (auto &pair : workers) { if (pair.second == TaskStatus::RUNNING || pair.second == TaskStatus::TRANSFER) { return true; } @@ -86,7 +43,7 @@ bool TaskNode::is_active() const void TaskNode::reset_owners() { - for (auto &pair : state->workers) { + for (auto &pair : workers) { if (pair.second == TaskStatus::OWNER) { pair.second = TaskStatus::NONE; } @@ -115,22 +72,36 @@ void TaskNode::set_as_finished(WorkerConnection *wc, size_t size, size_t length) assert(get_worker_status(wc) == TaskStatus::RUNNING); wc->free_resources(*this); set_worker_status(wc, TaskStatus::OWNER); - state->size = size; - state->length = length; + this->size = size; + this->length = length; + flags.set(static_cast<size_t>(TaskNodeFlags::FINISHED)); +} + +void TaskNode::set_as_loaded(WorkerConnection *wc, size_t size, size_t length) { + assert(get_worker_status(wc) == TaskStatus::LOADING); + set_worker_status(wc, TaskStatus::OWNER); + this->size = size; + this->length = length; + flags.set(static_cast<size_t>(TaskNodeFlags::FINISHED)); +} + +void TaskNode::set_as_loading(WorkerConnection *wc) { + set_worker_status(wc, TaskStatus::LOADING); } void TaskNode::set_as_finished_no_check(WorkerConnection *wc, size_t size, size_t length) { set_worker_status(wc, TaskStatus::OWNER); - state->size = size; - state->length = length; + this->size = size; + this->length = length; + flags.set(static_cast<size_t>(TaskNodeFlags::FINISHED)); } std::string TaskNode::debug_str() const { std::stringstream s; s << "<Node id=" << id; - for(auto &pair : state->workers) { + for(auto &pair : workers) { s << ' ' << pair.first->get_address() << ':' << static_cast<int>(pair.second); } s << '>'; @@ -156,8 +127,7 @@ void TaskNode::set_as_running(WorkerConnection *wc) void TaskNode::set_as_transferred(WorkerConnection *wc) { - assert(state); - auto &s = state->workers[wc]; + auto &s = workers[wc]; assert(s == TaskStatus::TRANSFER); s = TaskStatus::OWNER; } diff --git a/src/server/tasknode.h b/src/server/tasknode.h index 4ca75f3912bd393b7fb5f0c430e06268e9550664..6d89839681e0141686a99fceaf8d32e2b285ecd2 100644 --- a/src/server/tasknode.h +++ b/src/server/tasknode.h @@ -14,10 +14,17 @@ class WorkerConnection; class TaskNode; -enum class TaskFlags : size_t { +enum class TaskDefFlags : size_t { RESULT }; +enum class TaskNodeFlags : size_t { + FINISHED, + CHECKPOINT, + PLANNED, + FLAGS_COUNT +}; + struct TaskDef { int n_cpus; // TODO: Replace by resource index @@ -25,12 +32,14 @@ struct TaskDef loom::base::Id task_type; std::string config; std::bitset<1> flags; + std::string checkpoint_path; }; enum class TaskStatus { NONE, RUNNING, TRANSFER, + LOADING, OWNER, }; @@ -42,37 +51,48 @@ class TaskNode { public: - struct RuntimeState { - WorkerMap<TaskStatus> workers; - size_t size; - size_t length; - size_t remaining_inputs; - }; - TaskNode(loom::base::Id id, TaskDef &&task); loom::base::Id get_id() const { return id; } - bool has_state() const { - return state != nullptr; + inline bool is_result() const { + return task.flags.test(static_cast<size_t>(TaskDefFlags::RESULT)); } - inline bool is_result() const { - return task.flags.test(static_cast<size_t>(TaskFlags::RESULT)); + bool is_computed() const { + return flags.test(static_cast<size_t>(TaskNodeFlags::FINISHED)); + } + + inline bool has_checkpoint() const { + return flags.test(static_cast<size_t>(TaskNodeFlags::CHECKPOINT)); + } + + bool has_defined_checkpoint() const { + return !task.checkpoint_path.empty(); + } + + void set_checkpoint() { + flags.set(static_cast<size_t>(TaskNodeFlags::CHECKPOINT)); + } + + bool is_planned() const { + return flags.test(static_cast<size_t>(TaskNodeFlags::PLANNED)); + } + + void set_planned() { + flags.set(static_cast<size_t>(TaskNodeFlags::PLANNED)); } void reset_result_flag(); inline size_t get_size() const { - //assert(state); - return state->size; + return size; } inline size_t get_length() const { - //assert(state); - return state->length; + return length; } int get_n_cpus() const { @@ -91,7 +111,6 @@ public: return nexts; } - bool is_computed() const; bool is_active() const; WorkerConnection* get_random_owner(); @@ -100,49 +119,36 @@ public: } TaskStatus get_worker_status(WorkerConnection *wc) { - if (!state) { - return TaskStatus::NONE; - } - auto i = state->workers.find(wc); - if (i == state->workers.end()) { + auto i = workers.find(wc); + if (i == workers.end()) { return TaskStatus::NONE; } return i->second; } void set_worker_status(WorkerConnection *wc, TaskStatus status) { - ensure_state(); - state->workers[wc] = status; + workers[wc] = status; } - inline void ensure_state() { - if (!state) { - create_state(); - } + void set_remaining_inputs(int value) { + remaining_inputs = value; + } + + int get_remaining_inputs() const { + return remaining_inputs; } inline bool input_is_ready(TaskNode *node) { - if (!state) { - create_state(node); - } - assert(state->remaining_inputs > 0); - return --state->remaining_inputs == 0; + assert(remaining_inputs > 0); + return --remaining_inputs == 0; } inline bool is_ready() const { - if (!state) { - return _slow_is_ready(); - } - return state->remaining_inputs == 0; + return remaining_inputs == 0; } - void create_state(TaskNode *just_finishing_input = nullptr); - template<typename F> inline void foreach_owner(const F &f) const { - if (!state) { - return; - } - for(auto &pair : state->workers) { + for(auto &pair : workers) { if (pair.second == TaskStatus::OWNER) { f(pair.first); } @@ -150,10 +156,7 @@ public: } template<typename F> inline void foreach_worker(const F &f) const { - if (!state) { - return; - } - for(auto &pair : state->workers) { + for(auto &pair : workers) { if (pair.second != TaskStatus::NONE) { f(pair.first, pair.second); } @@ -163,27 +166,40 @@ public: void reset_owners(); const WorkerMap<TaskStatus>& get_workers() const { - return state->workers; + return workers; } bool next_finished(TaskNode &); void set_as_finished(WorkerConnection *wc, size_t size, size_t length); + void set_as_loaded(WorkerConnection *wc, size_t size, size_t length); void set_as_running(WorkerConnection *wc); + void set_as_loading(WorkerConnection *wc); void set_as_transferred(WorkerConnection *wc); void set_as_none(WorkerConnection *wc); // For unit testing void set_as_finished_no_check(WorkerConnection *wc, size_t size, size_t length); + void set_not_needed() { + flags.reset(static_cast<size_t>(TaskNodeFlags::PLANNED)); + flags.reset(static_cast<size_t>(TaskNodeFlags::FINISHED)); + } std::string debug_str() const; private: + + // Declaration loom::base::Id id; TaskDef task; std::unordered_multiset<TaskNode*> nexts; - std::unique_ptr<RuntimeState> state; + // Runtime info + std::bitset<3> flags; + WorkerMap<TaskStatus> workers; + size_t size; + size_t length; + size_t remaining_inputs; bool _slow_is_ready() const; }; diff --git a/src/server/workerconn.cpp b/src/server/workerconn.cpp index cebeeee5eddfafa380fc477dd12a56d414ae3ad7..e47b2bbb5412d49d933b3743c2fbe86baeb4df88 100644 --- a/src/server/workerconn.cpp +++ b/src/server/workerconn.cpp @@ -23,7 +23,9 @@ WorkerConnection::WorkerConnection(Server &server, task_types(task_types), data_types(data_types), worker_id(worker_id), - n_residual_tasks(0) + n_residual_tasks(0), + checkpoint_writes(0), + checkpoint_loads(0) { logger->info("Worker {} connected (cpus={})", address, resource_cpus); if (this->socket) { @@ -48,21 +50,47 @@ void WorkerConnection::on_message(const char *buffer, size_t size) WorkerResponse msg; msg.ParseFromArray(buffer, size); - if (msg.type() == WorkerResponse_Type_FINISHED) { - server.on_task_finished(msg.id(), msg.size(), msg.length(), this); + auto type = msg.type(); + if (type == WorkerResponse_Type_FINISHED) { + server.on_task_finished(msg.id(), msg.size(), msg.length(), this, false); return; } - if (msg.type() == WorkerResponse_Type_TRANSFERED) { + if (type == WorkerResponse_Type_FINISHED_AND_CHECKPOINTING) { + server.on_task_finished(msg.id(), msg.size(), msg.length(), this, true); + return; + } + + if (type == WorkerResponse_Type_TRANSFERED) { server.on_data_transferred(msg.id(), this); return; } - if (msg.type() == WorkerResponse_Type_FAILED) { + if (type == WorkerResponse_Type_FAILED) { assert(msg.has_error_msg()); server.on_task_failed(msg.id(), this, msg.error_msg()); return; } + + if (type == WorkerResponse_Type_CHECKPOINT_WRITTEN) { + server.on_checkpoint_write_finished(msg.id(), this); + return; + } + + if (type == WorkerResponse_Type_CHECKPOINT_WRITE_FAILED) { + server.on_checkpoint_write_failed(msg.id(), this, msg.error_msg()); + return; + } + + if (type == WorkerResponse_Type_CHECKPOINT_LOADED) { + server.on_checkpoint_load_finished(msg.id(), this, msg.size(), msg.length()); + return; + } + + if (type == WorkerResponse_Type_CHECKPOINT_LOAD_FAILED) { + server.on_checkpoint_load_failed(msg.id(), this, msg.error_msg()); + return; + } } void WorkerConnection::send_task(const TaskNode &task) @@ -79,6 +107,7 @@ void WorkerConnection::send_task(const TaskNode &task) msg.set_task_type(def.task_type); msg.set_task_config(def.config); msg.set_n_cpus(def.n_cpus); + msg.set_checkpoint_path(def.checkpoint_path); for (TaskNode *input_node : task.get_inputs()) { msg.add_task_inputs(input_node->get_id()); @@ -98,6 +127,19 @@ void WorkerConnection::send_data(Id id, const std::string &address) send_message(*socket, msg); } +void WorkerConnection::load_checkpoint(Id id, const std::string &checkpoint_path) +{ + checkpoint_loads += 1; + using namespace loom::pb::comm; + logger->debug("Command for {}: LOAD_CHECKPOINT id={} path={}", this->address, id, checkpoint_path); + + WorkerCommand msg; + msg.set_type(WorkerCommand_Type_LOAD_CHECKPOINT); + msg.set_id(id); + msg.set_checkpoint_path(checkpoint_path); + send_message(*socket, msg); +} + void WorkerConnection::remove_data(Id id) { using namespace loom::pb::comm; @@ -113,10 +155,15 @@ void WorkerConnection::free_resources(TaskNode &node) add_free_cpus(node.get_n_cpus()); } -void WorkerConnection::residual_task_finished(Id id, bool success) +void WorkerConnection::residual_task_finished(Id id, bool success, bool checkpointing) { logger->debug("Residual tasks id={} finished on {}", id, address); change_residual_tasks(-1); + + if (checkpointing) { + checkpoint_writes += 1; + } + if (success) { remove_data(id); } @@ -125,6 +172,15 @@ void WorkerConnection::residual_task_finished(Id id, bool success) } } +void WorkerConnection::residual_checkpoint_finished(Id id) +{ + logger->debug("Residual checkpoint id={} finished on {}", id, address); + checkpoint_writes -= 1; + if (!is_blocked()) { + server.need_task_distribution(); + } +} + void WorkerConnection::create_trace(const std::string &trace_path) { using namespace loom::pb::comm; diff --git a/src/server/workerconn.h b/src/server/workerconn.h index da3790a4eb0ad19fc6f3f7119103f7ab25c4b9ee..1da0875644558ad73c2f8e9eb875d3c1fdc4c5d7 100644 --- a/src/server/workerconn.h +++ b/src/server/workerconn.h @@ -70,18 +70,32 @@ public: } bool is_blocked() const { - return n_residual_tasks > 0; + return n_residual_tasks > 0 && checkpoint_writes > 0; } void change_residual_tasks(int value) { n_residual_tasks += value; } + int get_checkpoint_loads() const { + return checkpoint_loads; + } + + void change_checkpoint_writes(int value) { + checkpoint_writes += value; + } + + void change_checkpoint_loads(int value) { + checkpoint_loads += value; + } + void free_resources(TaskNode &node); - void residual_task_finished(loom::base::Id id, bool success); + void residual_task_finished(loom::base::Id id, bool success, bool checkpointing); + void residual_checkpoint_finished(loom::base::Id id); void create_trace(const std::string &trace_path); + void load_checkpoint(loom::base::Id id, const std::string &checkpoint_path); private: Server &server; @@ -95,6 +109,9 @@ private: int worker_id; int n_residual_tasks; + int n_residual_checkpoints; + int checkpoint_writes; + int checkpoint_loads; int scheduler_index; int scheduler_free_cpus; diff --git a/src/spdlog/LICENSE b/src/spdlog/LICENSE index 6c53e204b20d636242b65b7d98b2478970048252..4b43e0640dc14257670ca25de1b7ba5dd62e7c8d 100644 --- a/src/spdlog/LICENSE +++ b/src/spdlog/LICENSE @@ -1,21 +1,22 @@ -The MIT License (MIT) - -Copyright (c) 2016 Gabi Melman. - -Permission is hereby granted, free of charge, to any person obtaining a copy -of this software and associated documentation files (the "Software"), to deal -in the Software without restriction, including without limitation the rights -to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -copies of the Software, and to permit persons to whom the Software is -furnished to do so, subject to the following conditions: - -The above copyright notice and this permission notice shall be included in -all copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -THE SOFTWARE. +The MIT License (MIT) + +Copyright (c) 2016 Gabi Melman. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + diff --git a/src/spdlog/README.md b/src/spdlog/README.md new file mode 100644 index 0000000000000000000000000000000000000000..733471390904913ea9bdc01068a4a91c355181cc --- /dev/null +++ b/src/spdlog/README.md @@ -0,0 +1,221 @@ +# spdlog + +Very fast, header only, C++ logging library. [](https://travis-ci.org/gabime/spdlog) [](https://ci.appveyor.com/project/gabime/spdlog) + + +## Install +#### Just copy the headers: + +* Copy the source [folder](https://github.com/gabime/spdlog/tree/master/include/spdlog) to your build tree and use a C++11 compiler. + +#### Or use your favorite package manager: + +* Ubuntu: `apt-get install libspdlog-dev` +* Homebrew: `brew install spdlog` +* FreeBSD: `cd /usr/ports/devel/spdlog/ && make install clean` +* Fedora: `yum install spdlog` +* Gentoo: `emerge dev-libs/spdlog` +* Arch Linux: `yaourt -S spdlog-git` +* vcpkg: `vcpkg install spdlog` + + +## Platforms + * Linux, FreeBSD, Solaris + * Windows (vc 2013+, cygwin) + * Mac OSX (clang 3.5+) + * Android + +## Features +* Very fast - performance is the primary goal (see [benchmarks](#benchmarks) below). +* Headers only, just copy and use. +* Feature rich [call style](#usage-example) using the excellent [fmt](https://github.com/fmtlib/fmt) library. +* Optional printf syntax support. +* Extremely fast asynchronous mode (optional) - using lockfree queues and other tricks to reach millions of calls/sec. +* [Custom](https://github.com/gabime/spdlog/wiki/3.-Custom-formatting) formatting. +* Conditional Logging +* Multi/Single threaded loggers. +* Various log targets: + * Rotating log files. + * Daily log files. + * Console logging (colors supported). + * syslog. + * Windows debugger (```OutputDebugString(..)```) + * Easily extendable with custom log targets (just implement a single function in the [sink](include/spdlog/sinks/sink.h) interface). +* Severity based filtering - threshold levels can be modified in runtime as well as in compile time. + + + +## Benchmarks + +Below are some [benchmarks](bench) comparing popular log libraries under Ubuntu 64 bit, Intel i7-4770 CPU @ 3.40GHz + +#### Synchronous mode +Time needed to log 1,000,000 lines in synchronous mode (in seconds, the best of 3 runs): + +|threads|boost log 1.54|glog |easylogging |spdlog| +|-------|:-------:|:-----:|----------:|------:| +|1| 4.169s |1.066s |0.975s |0.302s| +|10| 6.180s |3.032s |2.857s |0.968s| +|100| 5.981s |1.139s |4.512s |0.497s| + + +#### Asynchronous mode +Time needed to log 1,000,000 lines in asynchronous mode, i.e. the time it takes to put them in the async queue (in seconds, the best of 3 runs): + +|threads|g2log <sup>async logger</sup> |spdlog <sup>async mode</sup>| +|:-------|:-----:|-------------------------:| +|1| 1.850s |0.216s | +|10| 0.943s |0.173s| +|100| 0.959s |0.202s| + + + + +## Usage Example +```c++ + +#include "spdlog/spdlog.h" + +#include <iostream> +#include <memory> + +void async_example(); +void syslog_example(); +void user_defined_example(); +void err_handler_example(); + +namespace spd = spdlog; +int main(int, char*[]) +{ + try + { + // Console logger with color + auto console = spd::stdout_color_mt("console"); + console->info("Welcome to spdlog!"); + console->error("Some error message with arg{}..", 1); + + // Formatting examples + console->warn("Easy padding in numbers like {:08d}", 12); + console->critical("Support for int: {0:d}; hex: {0:x}; oct: {0:o}; bin: {0:b}", 42); + console->info("Support for floats {:03.2f}", 1.23456); + console->info("Positional args are {1} {0}..", "too", "supported"); + console->info("{:<30}", "left aligned"); + + // Use global registry to retrieve loggers + spd::get("console")->info("loggers can be retrieved from a global registry using the spdlog::get(logger_name) function"); + + // Create basic file logger (not rotated) + auto my_logger = spd::basic_logger_mt("basic_logger", "logs/basic.txt"); + my_logger->info("Some log message"); + + // Create a file rotating logger with 5mb size max and 3 rotated files + auto rotating_logger = spd::rotating_logger_mt("some_logger_name", "logs/mylogfile.txt", 1048576 * 5, 3); + for (int i = 0; i < 10; ++i) + rotating_logger->info("{} * {} equals {:>10}", i, i, i*i); + + // Create a daily logger - a new file is created every day on 2:30am + auto daily_logger = spd::daily_logger_mt("daily_logger", "logs/daily.txt", 2, 30); + // trigger flush if the log severity is error or higher + daily_logger->flush_on(spd::level::err); + daily_logger->info(123.44); + + // Customize msg format for all messages + spd::set_pattern("*** [%H:%M:%S %z] [thread %t] %v ***"); + rotating_logger->info("This is another message with custom format"); + + + // Runtime log levels + spd::set_level(spd::level::info); //Set global log level to info + console->debug("This message should not be displayed!"); + console->set_level(spd::level::debug); // Set specific logger's log level + console->debug("This message should be displayed.."); + + // Compile time log levels + // define SPDLOG_DEBUG_ON or SPDLOG_TRACE_ON + SPDLOG_TRACE(console, "Enabled only #ifdef SPDLOG_TRACE_ON..{} ,{}", 1, 3.23); + SPDLOG_DEBUG(console, "Enabled only #ifdef SPDLOG_DEBUG_ON.. {} ,{}", 1, 3.23); + + // Asynchronous logging is very fast.. + // Just call spdlog::set_async_mode(q_size) and all created loggers from now on will be asynchronous.. + async_example(); + + // syslog example. linux/osx only + syslog_example(); + + // android example. compile with NDK + android_example(); + + // Log user-defined types example + user_defined_example(); + + // Change default log error handler + err_handler_example(); + + // Apply a function on all registered loggers + spd::apply_all([&](std::shared_ptr<spd::logger> l) + { + l->info("End of example."); + }); + + // Release and close all loggers + spd::drop_all(); + } + // Exceptions will only be thrown upon failed logger or sink construction (not during logging) + catch (const spd::spdlog_ex& ex) + { + std::cout << "Log init failed: " << ex.what() << std::endl; + return 1; + } +} + +void async_example() +{ + size_t q_size = 4096; //queue size must be power of 2 + spd::set_async_mode(q_size); + auto async_file = spd::daily_logger_st("async_file_logger", "logs/async_log.txt"); + for (int i = 0; i < 100; ++i) + async_file->info("Async message #{}", i); +} + +//syslog example +void syslog_example() +{ +#ifdef SPDLOG_ENABLE_SYSLOG + std::string ident = "spdlog-example"; + auto syslog_logger = spd::syslog_logger("syslog", ident, LOG_PID); + syslog_logger->warn("This is warning that will end up in syslog.."); +#endif +} + +// user defined types logging by implementing operator<< +struct my_type +{ + int i; + template<typename OStream> + friend OStream& operator<<(OStream& os, const my_type &c) + { + return os << "[my_type i="<<c.i << "]"; + } +}; + +#include <spdlog/fmt/ostr.h> // must be included +void user_defined_example() +{ + spd::get("console")->info("user defined type: {}", my_type { 14 }); +} + +// +//custom error handler +// +void err_handler_example() +{ + spd::set_error_handler([](const std::string& msg) { + std::cerr << "my err handler: " << msg << std::endl; + }); + // (or logger->set_error_handler(..) to set for specific logger) +} + +``` + +## Documentation +Documentation can be found in the [wiki](https://github.com/gabime/spdlog/wiki/1.-QuickStart) pages. diff --git a/src/spdlog/async_logger.h b/src/spdlog/async_logger.h index 786eb02ed529ca4fa9a8909c29defb8a72d2feb9..78765972402dc11168300f491e3d5efc52e57ddf 100644 --- a/src/spdlog/async_logger.h +++ b/src/spdlog/async_logger.h @@ -15,62 +15,58 @@ // 3. will throw spdlog_ex upon log exceptions // Upon destruction, logs all remaining messages in the queue before destructing.. -#include <spdlog/common.h> -#include <spdlog/logger.h> +#include "common.h" +#include "logger.h" #include <chrono> #include <functional> -#include <string> #include <memory> +#include <string> -namespace spdlog -{ +namespace spdlog { -namespace details -{ +namespace details { class async_log_helper; } -class async_logger :public logger +class async_logger SPDLOG_FINAL : public logger { public: - template<class It> - async_logger(const std::string& name, - const It& begin, - const It& end, - size_t queue_size, - const async_overflow_policy overflow_policy = async_overflow_policy::block_retry, - const std::function<void()>& worker_warmup_cb = nullptr, - const std::chrono::milliseconds& flush_interval_ms = std::chrono::milliseconds::zero(), - const std::function<void()>& worker_teardown_cb = nullptr); - - async_logger(const std::string& logger_name, - sinks_init_list sinks, - size_t queue_size, - const async_overflow_policy overflow_policy = async_overflow_policy::block_retry, - const std::function<void()>& worker_warmup_cb = nullptr, - const std::chrono::milliseconds& flush_interval_ms = std::chrono::milliseconds::zero(), - const std::function<void()>& worker_teardown_cb = nullptr); + template <class It> + async_logger(const std::string &logger_name, const It &begin, const It &end, size_t queue_size, + const async_overflow_policy overflow_policy = async_overflow_policy::block_retry, + const std::function<void()> &worker_warmup_cb = nullptr, + const std::chrono::milliseconds &flush_interval_ms = std::chrono::milliseconds::zero(), + const std::function<void()> &worker_teardown_cb = nullptr); - async_logger(const std::string& logger_name, - sink_ptr single_sink, - size_t queue_size, - const async_overflow_policy overflow_policy = async_overflow_policy::block_retry, - const std::function<void()>& worker_warmup_cb = nullptr, - const std::chrono::milliseconds& flush_interval_ms = std::chrono::milliseconds::zero(), - const std::function<void()>& worker_teardown_cb = nullptr); + async_logger(const std::string &logger_name, sinks_init_list sinks, size_t queue_size, + const async_overflow_policy overflow_policy = async_overflow_policy::block_retry, + const std::function<void()> &worker_warmup_cb = nullptr, + const std::chrono::milliseconds &flush_interval_ms = std::chrono::milliseconds::zero(), + const std::function<void()> &worker_teardown_cb = nullptr); + async_logger(const std::string &logger_name, sink_ptr single_sink, size_t queue_size, + const async_overflow_policy overflow_policy = async_overflow_policy::block_retry, + const std::function<void()> &worker_warmup_cb = nullptr, + const std::chrono::milliseconds &flush_interval_ms = std::chrono::milliseconds::zero(), + const std::function<void()> &worker_teardown_cb = nullptr); + // Wait for the queue to be empty, and flush synchronously + // Warning: this can potentially last forever as we wait it to complete void flush() override; + + // Error handler + void set_error_handler(log_err_handler) override; + log_err_handler error_handler() override; + protected: - void _log_msg(details::log_msg& msg) override; + void _sink_it(details::log_msg &msg) override; void _set_formatter(spdlog::formatter_ptr msg_formatter) override; - void _set_pattern(const std::string& pattern) override; + void _set_pattern(const std::string &pattern, pattern_time_type pattern_time) override; private: std::unique_ptr<details::async_log_helper> _async_log_helper; }; -} - +} // namespace spdlog -#include <spdlog/details/async_logger_impl.h> +#include "details/async_logger_impl.h" diff --git a/src/spdlog/common.h b/src/spdlog/common.h index 1e4465553d6ccbfe92dbd96b2642fd47954cfaf0..dc5f37954eeb0549e76e9acc45c0e8c7699ae7af 100644 --- a/src/spdlog/common.h +++ b/src/spdlog/common.h @@ -5,20 +5,27 @@ #pragma once -#include <string> -#include <initializer_list> -#include <chrono> -#include <memory> +#define SPDLOG_VERSION "0.16.4-rc" + +#include "tweakme.h" + #include <atomic> +#include <chrono> #include <exception> +#include <functional> +#include <initializer_list> +#include <memory> +#include <string> +#include <unordered_map> + #if defined(_WIN32) && defined(SPDLOG_WCHAR_FILENAMES) #include <codecvt> #include <locale> #endif -#include <spdlog/details/null_mutex.h> +#include "details/null_mutex.h" -//visual studio upto 2013 does not support noexcept nor constexpr +// visual studio upto 2013 does not support noexcept nor constexpr #if defined(_MSC_VER) && (_MSC_VER < 1900) #define SPDLOG_NOEXCEPT throw() #define SPDLOG_CONSTEXPR @@ -27,84 +34,138 @@ #define SPDLOG_CONSTEXPR constexpr #endif +// final keyword support. On by default. See tweakme.h +#if defined(SPDLOG_NO_FINAL) +#define SPDLOG_FINAL +#else +#define SPDLOG_FINAL final +#endif -namespace spdlog -{ +#if defined(__GNUC__) || defined(__clang__) +#define SPDLOG_DEPRECATED __attribute__((deprecated)) +#elif defined(_MSC_VER) +#define SPDLOG_DEPRECATED __declspec(deprecated) +#else +#define SPDLOG_DEPRECATED +#endif + +#include "fmt/fmt.h" + +namespace spdlog { class formatter; -namespace sinks -{ +namespace sinks { class sink; } using log_clock = std::chrono::system_clock; -using sink_ptr = std::shared_ptr < sinks::sink >; -using sinks_init_list = std::initializer_list < sink_ptr >; +using sink_ptr = std::shared_ptr<sinks::sink>; +using sinks_init_list = std::initializer_list<sink_ptr>; using formatter_ptr = std::shared_ptr<spdlog::formatter>; #if defined(SPDLOG_NO_ATOMIC_LEVELS) using level_t = details::null_atomic_int; #else -using level_t = std::atomic_int; +using level_t = std::atomic<int>; #endif -//Log level enum -namespace level -{ -typedef enum +using log_err_handler = std::function<void(const std::string &err_msg)>; + +// Log level enum +namespace level { +enum level_enum { trace = 0, debug = 1, info = 2, - notice = 3, - warn = 4, - err = 5, - critical = 6, - alert = 7, - emerg = 8, - off = 9 -} level_enum; + warn = 3, + err = 4, + critical = 5, + off = 6 +}; -static const char* level_names[] { "trace", "debug", "info", "notice", "warning", "error", "critical", "alert", "emerg", "off"}; +#if !defined(SPDLOG_LEVEL_NAMES) +#define SPDLOG_LEVEL_NAMES \ + { \ + "trace", "debug", "info", "warning", "error", "critical", "off" \ + } +#endif +static const char *level_names[] SPDLOG_LEVEL_NAMES; -static const char* short_level_names[] { "T", "D", "I", "N", "W", "E", "C", "A", "M", "O"}; +static const char *short_level_names[]{"T", "D", "I", "W", "E", "C", "O"}; -inline const char* to_str(spdlog::level::level_enum l) +inline const char *to_str(spdlog::level::level_enum l) { return level_names[l]; } -inline const char* to_short_str(spdlog::level::level_enum l) +inline const char *to_short_str(spdlog::level::level_enum l) { return short_level_names[l]; } -} //level +inline spdlog::level::level_enum from_str(const std::string &name) +{ + static std::unordered_map<std::string, level_enum> name_to_level = // map string->level + {{level_names[0], level::trace}, // trace + {level_names[1], level::debug}, // debug + {level_names[2], level::info}, // info + {level_names[3], level::warn}, // warn + {level_names[4], level::err}, // err + {level_names[5], level::critical}, // critical + {level_names[6], level::off}}; // off + + auto lvl_it = name_to_level.find(name); + return lvl_it != name_to_level.end() ? lvl_it->second : level::off; +} +using level_hasher = std::hash<int>; +} // namespace level // // Async overflow policy - block by default. // enum class async_overflow_policy { - block_retry, // Block / yield / sleep until message can be enqueued + block_retry, // Block / yield / sleep until message can be enqueued discard_log_msg // Discard the message it enqueue fails }; +// +// Pattern time - specific time getting to use for pattern_formatter. +// local time by default +// +enum class pattern_time_type +{ + local, // log localtime + utc // log utc +}; // // Log exception // +namespace details { namespace os { +std::string errno_str(int err_num); +}} // namespace details::os class spdlog_ex : public std::exception { public: - spdlog_ex(const std::string& msg) :_msg(msg) {} - const char* what() const SPDLOG_NOEXCEPT override + explicit spdlog_ex(std::string msg) + : _msg(std::move(msg)) + { + } + + spdlog_ex(const std::string &msg, int last_errno) + { + _msg = msg + ": " + details::os::errno_str(last_errno); + } + + const char *what() const SPDLOG_NOEXCEPT override { return _msg.c_str(); } + private: std::string _msg; - }; // @@ -116,5 +177,4 @@ using filename_t = std::wstring; using filename_t = std::string; #endif - -} //spdlog +} // namespace spdlog diff --git a/src/spdlog/details/async_log_helper.h b/src/spdlog/details/async_log_helper.h index f6f002826acaa5fa83edcc08ccca1c8b0d846df2..78058975b1794d9ac5bea1cb02d7f4f7d195a22f 100644 --- a/src/spdlog/details/async_log_helper.h +++ b/src/spdlog/details/async_log_helper.h @@ -9,17 +9,15 @@ // If the internal queue of log messages reaches its max size, // then the client call will block until there is more room. // -// If the back thread throws during logging, a spdlog::spdlog_ex exception -// will be thrown in client's thread when tries to log the next message #pragma once -#include <spdlog/common.h> -#include <spdlog/sinks/sink.h> -#include <spdlog/details/mpmc_bounded_q.h> -#include <spdlog/details/log_msg.h> -#include <spdlog/details/os.h> -#include <spdlog/formatter.h> +#include "../common.h" +#include "../details/log_msg.h" +#include "../details/mpmc_bounded_q.h" +#include "../details/os.h" +#include "../formatter.h" +#include "../sinks/sink.h" #include <chrono> #include <exception> @@ -30,10 +28,7 @@ #include <utility> #include <vector> -namespace spdlog -{ -namespace details -{ +namespace spdlog { namespace details { class async_log_helper { @@ -45,6 +40,7 @@ class async_log_helper flush, terminate }; + struct async_msg { std::string logger_name; @@ -53,23 +49,30 @@ class async_log_helper size_t thread_id; std::string txt; async_msg_type msg_type; + size_t msg_id; async_msg() = default; ~async_msg() = default; + explicit async_msg(async_msg_type m_type) + : level(level::info) + , thread_id(0) + , msg_type(m_type) + , msg_id(0) + { + } -async_msg(async_msg&& other) SPDLOG_NOEXCEPT: - logger_name(std::move(other.logger_name)), - level(std::move(other.level)), - time(std::move(other.time)), - txt(std::move(other.txt)), - msg_type(std::move(other.msg_type)) - {} - - async_msg(async_msg_type m_type) :msg_type(m_type) - {}; + async_msg(async_msg &&other) SPDLOG_NOEXCEPT : logger_name(std::move(other.logger_name)), + level(std::move(other.level)), + time(std::move(other.time)), + thread_id(other.thread_id), + txt(std::move(other.txt)), + msg_type(std::move(other.msg_type)), + msg_id(other.msg_id) + { + } - async_msg& operator=(async_msg&& other) SPDLOG_NOEXCEPT + async_msg &operator=(async_msg &&other) SPDLOG_NOEXCEPT { logger_name = std::move(other.logger_name); level = other.level; @@ -77,62 +80,64 @@ async_msg(async_msg&& other) SPDLOG_NOEXCEPT: thread_id = other.thread_id; txt = std::move(other.txt); msg_type = other.msg_type; + msg_id = other.msg_id; return *this; } // never copy or assign. should only be moved.. - async_msg(const async_msg&) = delete; - async_msg& operator=(async_msg& other) = delete; + async_msg(const async_msg &) = delete; + async_msg &operator=(const async_msg &other) = delete; // construct from log_msg - async_msg(const details::log_msg& m) : - logger_name(m.logger_name), - level(m.level), - time(m.time), - thread_id(m.thread_id), - txt(m.raw.data(), m.raw.size()), - msg_type(async_msg_type::log) - {} - - + explicit async_msg(const details::log_msg &m) + : level(m.level) + , time(m.time) + , thread_id(m.thread_id) + , txt(m.raw.data(), m.raw.size()) + , msg_type(async_msg_type::log) + , msg_id(m.msg_id) + { +#ifndef SPDLOG_NO_NAME + logger_name = *m.logger_name; +#endif + } // copy into log_msg void fill_log_msg(log_msg &msg) { - msg.clear(); - msg.logger_name = logger_name; + msg.logger_name = &logger_name; msg.level = level; msg.time = time; msg.thread_id = thread_id; msg.raw << txt; + msg.msg_id = msg_id; } }; public: - using item_type = async_msg; using q_type = details::mpmc_bounded_queue<item_type>; using clock = std::chrono::steady_clock; + async_log_helper(formatter_ptr formatter, std::vector<sink_ptr> sinks, size_t queue_size, const log_err_handler err_handler, + const async_overflow_policy overflow_policy = async_overflow_policy::block_retry, std::function<void()> worker_warmup_cb = nullptr, + const std::chrono::milliseconds &flush_interval_ms = std::chrono::milliseconds::zero(), + std::function<void()> worker_teardown_cb = nullptr); - async_log_helper(formatter_ptr formatter, - const std::vector<sink_ptr>& sinks, - size_t queue_size, - const async_overflow_policy overflow_policy = async_overflow_policy::block_retry, - const std::function<void()>& worker_warmup_cb = nullptr, - const std::chrono::milliseconds& flush_interval_ms = std::chrono::milliseconds::zero(), - const std::function<void()>& worker_teardown_cb = nullptr); - - void log(const details::log_msg& msg); + void log(const details::log_msg &msg); // stop logging and join the back thread ~async_log_helper(); - void set_formatter(formatter_ptr); + async_log_helper(const async_log_helper &) = delete; + async_log_helper &operator=(const async_log_helper &) = delete; + + void set_formatter(formatter_ptr msg_formatter); - void flush(); + void flush(bool wait_for_q); + void set_error_handler(spdlog::log_err_handler err_handler); private: formatter_ptr _formatter; @@ -141,14 +146,12 @@ private: // queue of messages to log q_type _q; + log_err_handler _err_handler; + bool _flush_requested; bool _terminate_requested; - - // last exception thrown from the worker thread - std::shared_ptr<spdlog_ex> _last_workerthread_ex; - // overflow policy const async_overflow_policy _overflow_policy; @@ -164,49 +167,44 @@ private: // worker thread std::thread _worker_thread; - void push_msg(async_msg&& new_msg); - - // throw last worker thread exception or if worker thread is not active - void throw_if_bad_worker(); + void push_msg(async_msg &&new_msg); // worker thread main loop void worker_loop(); // pop next message from the queue and process it. will set the last_pop to the pop time // return false if termination of the queue is required - bool process_next_msg(log_clock::time_point& last_pop, log_clock::time_point& last_flush); + bool process_next_msg(log_clock::time_point &last_pop, log_clock::time_point &last_flush); - void handle_flush_interval(log_clock::time_point& now, log_clock::time_point& last_flush); + void handle_flush_interval(log_clock::time_point &now, log_clock::time_point &last_flush); - // sleep,yield or return immediatly using the time passed since last message as a hint - static void sleep_or_yield(const spdlog::log_clock::time_point& now, const log_clock::time_point& last_op_time); + // sleep,yield or return immediately using the time passed since last message as a hint + static void sleep_or_yield(const spdlog::log_clock::time_point &now, const log_clock::time_point &last_op_time); + // wait until the queue is empty + void wait_empty_q(); }; -} -} +}} // namespace spdlog::details /////////////////////////////////////////////////////////////////////////////// // async_sink class implementation /////////////////////////////////////////////////////////////////////////////// -inline spdlog::details::async_log_helper::async_log_helper( - formatter_ptr formatter, - const std::vector<sink_ptr>& sinks, - size_t queue_size, - const async_overflow_policy overflow_policy, - const std::function<void()>& worker_warmup_cb, - const std::chrono::milliseconds& flush_interval_ms, - const std::function<void()>& worker_teardown_cb): - _formatter(formatter), - _sinks(sinks), - _q(queue_size), - _flush_requested(false), - _terminate_requested(false), - _overflow_policy(overflow_policy), - _worker_warmup_cb(worker_warmup_cb), - _flush_interval_ms(flush_interval_ms), - _worker_teardown_cb(worker_teardown_cb), - _worker_thread(&async_log_helper::worker_loop, this) -{} +inline spdlog::details::async_log_helper::async_log_helper(formatter_ptr formatter, std::vector<sink_ptr> sinks, size_t queue_size, + log_err_handler err_handler, const async_overflow_policy overflow_policy, std::function<void()> worker_warmup_cb, + const std::chrono::milliseconds &flush_interval_ms, std::function<void()> worker_teardown_cb) + : _formatter(std::move(formatter)) + , _sinks(std::move(sinks)) + , _q(queue_size) + , _err_handler(std::move(err_handler)) + , _flush_requested(false) + , _terminate_requested(false) + , _overflow_policy(overflow_policy) + , _worker_warmup_cb(std::move(worker_warmup_cb)) + , _flush_interval_ms(flush_interval_ms) + , _worker_teardown_cb(std::move(worker_teardown_cb)) +{ + _worker_thread = std::thread(&async_log_helper::worker_loop, this); +} // Send to the worker thread termination message(level=off) // and wait for it to finish gracefully @@ -218,21 +216,18 @@ inline spdlog::details::async_log_helper::~async_log_helper() _worker_thread.join(); } catch (...) // don't crash in destructor - {} + { + } } - -//Try to push and block until succeeded (if the policy is not to discard when the queue is full) -inline void spdlog::details::async_log_helper::log(const details::log_msg& msg) +// Try to push and block until succeeded (if the policy is not to discard when the queue is full) +inline void spdlog::details::async_log_helper::log(const details::log_msg &msg) { push_msg(async_msg(msg)); - - } -inline void spdlog::details::async_log_helper::push_msg(details::async_log_helper::async_msg&& new_msg) +inline void spdlog::details::async_log_helper::push_msg(details::async_log_helper::async_msg &&new_msg) { - throw_if_bad_worker(); if (!_q.enqueue(std::move(new_msg)) && _overflow_policy != async_overflow_policy::discard_log_msg) { auto last_op_time = details::os::now(); @@ -241,44 +236,49 @@ inline void spdlog::details::async_log_helper::push_msg(details::async_log_helpe { now = details::os::now(); sleep_or_yield(now, last_op_time); - } - while (!_q.enqueue(std::move(new_msg))); + } while (!_q.enqueue(std::move(new_msg))); } - } -inline void spdlog::details::async_log_helper::flush() +// optionally wait for the queue be empty and request flush from the sinks +inline void spdlog::details::async_log_helper::flush(bool wait_for_q) { push_msg(async_msg(async_msg_type::flush)); + if (wait_for_q) + wait_empty_q(); // return when queue is empty } inline void spdlog::details::async_log_helper::worker_loop() { - try - { - if (_worker_warmup_cb) _worker_warmup_cb(); - auto last_pop = details::os::now(); - auto last_flush = last_pop; - while(process_next_msg(last_pop, last_flush)); - if (_worker_teardown_cb) _worker_teardown_cb(); - } - catch (const std::exception& ex) + if (_worker_warmup_cb) + _worker_warmup_cb(); + auto last_pop = details::os::now(); + auto last_flush = last_pop; + auto active = true; + while (active) { - _last_workerthread_ex = std::make_shared<spdlog_ex>(std::string("async_logger worker thread exception: ") + ex.what()); - } - catch (...) - { - _last_workerthread_ex = std::make_shared<spdlog_ex>("async_logger worker thread exception"); + try + { + active = process_next_msg(last_pop, last_flush); + } + catch (const std::exception &ex) + { + _err_handler(ex.what()); + } + catch (...) + { + _err_handler("Unknown exeption in async logger worker loop."); + } } + if (_worker_teardown_cb) + _worker_teardown_cb(); } // process next message in the queue // return true if this thread should still be active (while no terminate msg was received) -inline bool spdlog::details::async_log_helper::process_next_msg(log_clock::time_point& last_pop, log_clock::time_point& last_flush) +inline bool spdlog::details::async_log_helper::process_next_msg(log_clock::time_point &last_pop, log_clock::time_point &last_flush) { - async_msg incoming_async_msg; - log_msg incoming_log_msg; if (_q.dequeue(incoming_async_msg)) { @@ -295,30 +295,33 @@ inline bool spdlog::details::async_log_helper::process_next_msg(log_clock::time_ break; default: + log_msg incoming_log_msg; incoming_async_msg.fill_log_msg(incoming_log_msg); _formatter->format(incoming_log_msg); for (auto &s : _sinks) - s->log(incoming_log_msg); + { + if (s->should_log(incoming_log_msg.level)) + { + s->log(incoming_log_msg); + } + } } return true; } // Handle empty queue.. // This is the only place where the queue can terminate or flush to avoid losing messages already in the queue - else - { - auto now = details::os::now(); - handle_flush_interval(now, last_flush); - sleep_or_yield(now, last_pop); - return !_terminate_requested; - - } + auto now = details::os::now(); + handle_flush_interval(now, last_flush); + sleep_or_yield(now, last_pop); + return !_terminate_requested; } // flush all sinks if _flush_interval_ms has expired -inline void spdlog::details::async_log_helper::handle_flush_interval(log_clock::time_point& now, log_clock::time_point& last_flush) +inline void spdlog::details::async_log_helper::handle_flush_interval(log_clock::time_point &now, log_clock::time_point &last_flush) { - auto should_flush = _flush_requested || (_flush_interval_ms != std::chrono::milliseconds::zero() && now - last_flush >= _flush_interval_ms); + auto should_flush = + _flush_requested || (_flush_interval_ms != std::chrono::milliseconds::zero() && now - last_flush >= _flush_interval_ms); if (should_flush) { for (auto &s : _sinks) @@ -330,16 +333,15 @@ inline void spdlog::details::async_log_helper::handle_flush_interval(log_clock:: inline void spdlog::details::async_log_helper::set_formatter(formatter_ptr msg_formatter) { - _formatter = msg_formatter; + _formatter = std::move(msg_formatter); } - // spin, yield or sleep. use the time passed since last message as a hint -inline void spdlog::details::async_log_helper::sleep_or_yield(const spdlog::log_clock::time_point& now, const spdlog::log_clock::time_point& last_op_time) +inline void spdlog::details::async_log_helper::sleep_or_yield( + const spdlog::log_clock::time_point &now, const spdlog::log_clock::time_point &last_op_time) { - using namespace std::this_thread; - using std::chrono::milliseconds; using std::chrono::microseconds; + using std::chrono::milliseconds; auto time_since_op = now - last_op_time; @@ -349,30 +351,27 @@ inline void spdlog::details::async_log_helper::sleep_or_yield(const spdlog::log_ // yield upto 150 micros if (time_since_op <= microseconds(100)) - return yield(); - + return std::this_thread::yield(); // sleep for 20 ms upto 200 ms if (time_since_op <= milliseconds(200)) - return sleep_for(milliseconds(20)); + return details::os::sleep_for_millis(20); - // sleep for 200 ms - return sleep_for(milliseconds(200)); + // sleep for 500 ms + return details::os::sleep_for_millis(500); } -// throw if the worker thread threw an exception or not active -inline void spdlog::details::async_log_helper::throw_if_bad_worker() +// wait for the queue to be empty +inline void spdlog::details::async_log_helper::wait_empty_q() { - if (_last_workerthread_ex) + auto last_op = details::os::now(); + while (!_q.is_empty()) { - auto ex = std::move(_last_workerthread_ex); - throw *ex; + sleep_or_yield(details::os::now(), last_op); } } - - - - - - +inline void spdlog::details::async_log_helper::set_error_handler(spdlog::log_err_handler err_handler) +{ + _err_handler = std::move(err_handler); +} diff --git a/src/spdlog/details/async_logger_impl.h b/src/spdlog/details/async_logger_impl.h index 8fa698dda86816ffa1e30094acd8ad2a0c51188b..ca458740dbf9d6677834e0c974a43aea736ad2b0 100644 --- a/src/spdlog/details/async_logger_impl.h +++ b/src/spdlog/details/async_logger_impl.h @@ -8,54 +8,54 @@ // Async Logger implementation // Use an async_sink (queue per logger) to perform the logging in a worker thread -#include <spdlog/details/async_log_helper.h> -#include <spdlog/async_logger.h> +#include "../async_logger.h" +#include "../details/async_log_helper.h" -#include <string> -#include <functional> #include <chrono> +#include <functional> #include <memory> +#include <string> -template<class It> -inline spdlog::async_logger::async_logger(const std::string& logger_name, - const It& begin, - const It& end, - size_t queue_size, - const async_overflow_policy overflow_policy, - const std::function<void()>& worker_warmup_cb, - const std::chrono::milliseconds& flush_interval_ms, - const std::function<void()>& worker_teardown_cb) : - logger(logger_name, begin, end), - _async_log_helper(new details::async_log_helper(_formatter, _sinks, queue_size, overflow_policy, worker_warmup_cb, flush_interval_ms, worker_teardown_cb)) +template <class It> +inline spdlog::async_logger::async_logger(const std::string &logger_name, const It &begin, const It &end, size_t queue_size, + const async_overflow_policy overflow_policy, const std::function<void()> &worker_warmup_cb, + const std::chrono::milliseconds &flush_interval_ms, const std::function<void()> &worker_teardown_cb) + : logger(logger_name, begin, end) + , _async_log_helper(new details::async_log_helper( + _formatter, _sinks, queue_size, _err_handler, overflow_policy, worker_warmup_cb, flush_interval_ms, worker_teardown_cb)) { } -inline spdlog::async_logger::async_logger(const std::string& logger_name, - sinks_init_list sinks, - size_t queue_size, - const async_overflow_policy overflow_policy, - const std::function<void()>& worker_warmup_cb, - const std::chrono::milliseconds& flush_interval_ms, - const std::function<void()>& worker_teardown_cb) : - async_logger(logger_name, sinks.begin(), sinks.end(), queue_size, overflow_policy, worker_warmup_cb, flush_interval_ms, worker_teardown_cb) {} - -inline spdlog::async_logger::async_logger(const std::string& logger_name, - sink_ptr single_sink, - size_t queue_size, - const async_overflow_policy overflow_policy, - const std::function<void()>& worker_warmup_cb, - const std::chrono::milliseconds& flush_interval_ms, - const std::function<void()>& worker_teardown_cb) : - async_logger(logger_name, +inline spdlog::async_logger::async_logger(const std::string &logger_name, sinks_init_list sinks_list, size_t queue_size, + const async_overflow_policy overflow_policy, const std::function<void()> &worker_warmup_cb, + const std::chrono::milliseconds &flush_interval_ms, const std::function<void()> &worker_teardown_cb) + : async_logger(logger_name, sinks_list.begin(), sinks_list.end(), queue_size, overflow_policy, worker_warmup_cb, flush_interval_ms, + worker_teardown_cb) { - single_sink -}, queue_size, overflow_policy, worker_warmup_cb, flush_interval_ms, worker_teardown_cb) {} +} +inline spdlog::async_logger::async_logger(const std::string &logger_name, sink_ptr single_sink, size_t queue_size, + const async_overflow_policy overflow_policy, const std::function<void()> &worker_warmup_cb, + const std::chrono::milliseconds &flush_interval_ms, const std::function<void()> &worker_teardown_cb) + : async_logger( + logger_name, {std::move(single_sink)}, queue_size, overflow_policy, worker_warmup_cb, flush_interval_ms, worker_teardown_cb) +{ +} inline void spdlog::async_logger::flush() { + _async_log_helper->flush(true); +} - _async_log_helper->flush(); +// Error handler +inline void spdlog::async_logger::set_error_handler(spdlog::log_err_handler err_handler) +{ + _err_handler = err_handler; + _async_log_helper->set_error_handler(err_handler); +} +inline spdlog::log_err_handler spdlog::async_logger::error_handler() +{ + return _err_handler; } inline void spdlog::async_logger::_set_formatter(spdlog::formatter_ptr msg_formatter) @@ -64,14 +64,30 @@ inline void spdlog::async_logger::_set_formatter(spdlog::formatter_ptr msg_forma _async_log_helper->set_formatter(_formatter); } -inline void spdlog::async_logger::_set_pattern(const std::string& pattern) +inline void spdlog::async_logger::_set_pattern(const std::string &pattern, pattern_time_type pattern_time) { - _formatter = std::make_shared<pattern_formatter>(pattern); + _formatter = std::make_shared<pattern_formatter>(pattern, pattern_time); _async_log_helper->set_formatter(_formatter); } - -inline void spdlog::async_logger::_log_msg(details::log_msg& msg) +inline void spdlog::async_logger::_sink_it(details::log_msg &msg) { - _async_log_helper->log(msg); + try + { +#if defined(SPDLOG_ENABLE_MESSAGE_COUNTER) + _incr_msg_counter(msg); +#endif + _async_log_helper->log(msg); + if (_should_flush_on(msg)) + _async_log_helper->flush(false); // do async flush + } + catch (const std::exception &ex) + { + _err_handler(ex.what()); + } + catch (...) + { + _err_handler("Unknown exception in logger " + _name); + throw; + } } diff --git a/src/spdlog/details/file_helper.h b/src/spdlog/details/file_helper.h index a465c4d90a7464c486aaa4770698195968a47f6b..0d24a91ba4fc7a57a3ebb58e09792237a4e80d2d 100644 --- a/src/spdlog/details/file_helper.h +++ b/src/spdlog/details/file_helper.h @@ -7,21 +7,19 @@ // Helper class for file sink // When failing to open a file, retry several times(5) with small delay between the tries(10 ms) -// Can be set to auto flush on every line // Throw spdlog_ex exception on errors -#include <spdlog/details/os.h> -#include <spdlog/details/log_msg.h> +#include "../details/log_msg.h" +#include "../details/os.h" +#include <cerrno> #include <chrono> #include <cstdio> #include <string> #include <thread> +#include <tuple> -namespace spdlog -{ -namespace details -{ +namespace spdlog { namespace details { class file_helper { @@ -30,23 +28,18 @@ public: const int open_tries = 5; const int open_interval = 10; - explicit file_helper(bool force_flush) : - _fd(nullptr), - _force_flush(force_flush) - {} + explicit file_helper() = default; - file_helper(const file_helper&) = delete; - file_helper& operator=(const file_helper&) = delete; + file_helper(const file_helper &) = delete; + file_helper &operator=(const file_helper &) = delete; ~file_helper() { close(); } - - void open(const filename_t& fname, bool truncate = false) + void open(const filename_t &fname, bool truncate = false) { - close(); auto *mode = truncate ? SPDLOG_FILENAME_T("wb") : SPDLOG_FILENAME_T("ab"); _filename = fname; @@ -55,10 +48,10 @@ public: if (!os::fopen_s(&_fd, fname, mode)) return; - std::this_thread::sleep_for(std::chrono::milliseconds(open_interval)); + details::os::sleep_for_millis(open_interval); } - throw spdlog_ex("Failed opening file " + os::filename_to_str(_filename) + " for writing"); + throw spdlog_ex("Failed opening file " + os::filename_to_str(_filename) + " for writing", errno); } void reopen(bool truncate) @@ -66,7 +59,6 @@ public: if (_filename.empty()) throw spdlog_ex("Failed re opening file - was not opened before"); open(_filename, truncate); - } void flush() @@ -76,60 +68,72 @@ public: void close() { - if (_fd) + if (_fd != nullptr) { std::fclose(_fd); _fd = nullptr; } } - void write(const log_msg& msg) + void write(const log_msg &msg) { - size_t msg_size = msg.formatted.size(); auto data = msg.formatted.data(); if (std::fwrite(data, 1, msg_size, _fd) != msg_size) - throw spdlog_ex("Failed writing to file " + os::filename_to_str(_filename)); - - if (_force_flush) - std::fflush(_fd); + throw spdlog_ex("Failed writing to file " + os::filename_to_str(_filename), errno); } - long size() + size_t size() const { - if (!_fd) + if (_fd == nullptr) + { throw spdlog_ex("Cannot use size() on closed file " + os::filename_to_str(_filename)); - - auto pos = ftell(_fd); - if (fseek(_fd, 0, SEEK_END) != 0) - throw spdlog_ex("fseek failed on file " + os::filename_to_str(_filename)); - - auto file_size = ftell(_fd); - - if(fseek(_fd, pos, SEEK_SET) !=0) - throw spdlog_ex("fseek failed on file " + os::filename_to_str(_filename)); - - if (file_size == -1) - throw spdlog_ex("ftell failed on file " + os::filename_to_str(_filename)); - - return file_size; + } + return os::filesize(_fd); } - const filename_t& filename() const + const filename_t &filename() const { return _filename; } - static bool file_exists(const filename_t& name) + static bool file_exists(const filename_t &fname) { + return os::file_exists(fname); + } + + // + // return file path and its extension: + // + // "mylog.txt" => ("mylog", ".txt") + // "mylog" => ("mylog", "") + // "mylog." => ("mylog.", "") + // "/dir1/dir2/mylog.txt" => ("/dir1/dir2/mylog", ".txt") + // + // the starting dot in filenames is ignored (hidden files): + // + // ".mylog" => (".mylog". "") + // "my_folder/.mylog" => ("my_folder/.mylog", "") + // "my_folder/.mylog.txt" => ("my_folder/.mylog", ".txt") + static std::tuple<filename_t, filename_t> split_by_extenstion(const spdlog::filename_t &fname) + { + auto ext_index = fname.rfind('.'); + + // no valid extension found - return whole path and empty string as extension + if (ext_index == filename_t::npos || ext_index == 0 || ext_index == fname.size() - 1) + return std::make_tuple(fname, spdlog::filename_t()); + + // treat casese like "/etc/rc.d/somelogfile or "/abc/.hiddenfile" + auto folder_index = fname.rfind(details::os::folder_sep); + if (folder_index != fname.npos && folder_index >= ext_index - 1) + return std::make_tuple(fname, spdlog::filename_t()); - return os::file_exists(name); + // finally - return a valid base and extension tuple + return std::make_tuple(fname.substr(0, ext_index), fname.substr(ext_index)); } private: - FILE* _fd; + FILE *_fd{nullptr}; filename_t _filename; - bool _force_flush; }; -} -} +}} // namespace spdlog::details diff --git a/src/spdlog/details/format.cc b/src/spdlog/details/format.cc deleted file mode 100644 index cd246e4e7dddd51b36fb4ee749186c09cf8da665..0000000000000000000000000000000000000000 --- a/src/spdlog/details/format.cc +++ /dev/null @@ -1,1066 +0,0 @@ -/* -Formatting library for C++ - -Copyright (c) 2012 - 2015, Victor Zverovich -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, this -list of conditions and the following disclaimer. -2. Redistributions in binary form must reproduce the above copyright notice, -this list of conditions and the following disclaimer in the documentation -and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND -ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -*/ - -#include "format.h" - -#include <string.h> - -#include <cctype> -#include <cerrno> -#include <climits> -#include <cmath> -#include <cstdarg> -#include <cstddef> // for std::ptrdiff_t - -#if defined(_WIN32) && defined(__MINGW32__) -# include <cstring> -#endif - -#if FMT_USE_WINDOWS_H -# if defined(NOMINMAX) || defined(FMT_WIN_MINMAX) -# include <windows.h> -# else -# define NOMINMAX -# include <windows.h> -# undef NOMINMAX -# endif -#endif - -using fmt::internal::Arg; - -#if FMT_EXCEPTIONS -# define FMT_TRY try -# define FMT_CATCH(x) catch (x) -#else -# define FMT_TRY if (true) -# define FMT_CATCH(x) if (false) -#endif - -#ifdef FMT_HEADER_ONLY -# define FMT_FUNC inline -#else -# define FMT_FUNC -#endif - -#ifdef _MSC_VER -# pragma warning(push) -# pragma warning(disable: 4127) // conditional expression is constant -# pragma warning(disable: 4702) // unreachable code -// Disable deprecation warning for strerror. The latter is not called but -// MSVC fails to detect it. -# pragma warning(disable: 4996) -#endif - -// Dummy implementations of strerror_r and strerror_s called if corresponding -// system functions are not available. -static inline fmt::internal::Null<> strerror_r(int, char *, ...) -{ - return fmt::internal::Null<>(); -} -static inline fmt::internal::Null<> strerror_s(char *, std::size_t, ...) -{ - return fmt::internal::Null<>(); -} - -namespace fmt { - namespace { - -#ifndef _MSC_VER -# define FMT_SNPRINTF snprintf -#else // _MSC_VER - inline int fmt_snprintf(char *buffer, size_t size, const char *format, ...) - { - va_list args; - va_start(args, format); - int result = vsnprintf_s(buffer, size, _TRUNCATE, format, args); - va_end(args); - return result; - } -# define FMT_SNPRINTF fmt_snprintf -#endif // _MSC_VER - -#if defined(_WIN32) && defined(__MINGW32__) && !defined(__NO_ISOCEXT) -# define FMT_SWPRINTF snwprintf -#else -# define FMT_SWPRINTF swprintf -#endif // defined(_WIN32) && defined(__MINGW32__) && !defined(__NO_ISOCEXT) - - // Checks if a value fits in int - used to avoid warnings about comparing - // signed and unsigned integers. - template <bool IsSigned> - struct IntChecker - { - template <typename T> - static bool fits_in_int(T value) - { - unsigned max = INT_MAX; - return value <= max; - } - static bool fits_in_int(bool) - { - return true; - } - }; - - template <> - struct IntChecker<true> - { - template <typename T> - static bool fits_in_int(T value) - { - return value >= INT_MIN && value <= INT_MAX; - } - static bool fits_in_int(int) - { - return true; - } - }; - - const char RESET_COLOR[] = "\x1b[0m"; - - typedef void(*FormatFunc)(fmt::Writer &, int, fmt::StringRef); - - // Portable thread-safe version of strerror. - // Sets buffer to point to a string describing the error code. - // This can be either a pointer to a string stored in buffer, - // or a pointer to some static immutable string. - // Returns one of the following values: - // 0 - success - // ERANGE - buffer is not large enough to store the error message - // other - failure - // Buffer should be at least of size 1. - int safe_strerror( - int error_code, char *&buffer, std::size_t buffer_size) FMT_NOEXCEPT - { - FMT_ASSERT(buffer != 0 && buffer_size != 0, "invalid buffer"); - - class StrError - { - private: - int error_code_; - char *&buffer_; - std::size_t buffer_size_; - - // A noop assignment operator to avoid bogus warnings. - void operator=(const StrError &) - {} - - // Handle the result of XSI-compliant version of strerror_r. - int handle(int result) - { - // glibc versions before 2.13 return result in errno. - return result == -1 ? errno : result; - } - - // Handle the result of GNU-specific version of strerror_r. - int handle(char *message) - { - // If the buffer is full then the message is probably truncated. - if (message == buffer_ && strlen(buffer_) == buffer_size_ - 1) - return ERANGE; - buffer_ = message; - return 0; - } - - // Handle the case when strerror_r is not available. - int handle(fmt::internal::Null<>) - { - return fallback(strerror_s(buffer_, buffer_size_, error_code_)); - } - - // Fallback to strerror_s when strerror_r is not available. - int fallback(int result) - { - // If the buffer is full then the message is probably truncated. - return result == 0 && strlen(buffer_) == buffer_size_ - 1 ? - ERANGE : result; - } - - // Fallback to strerror if strerror_r and strerror_s are not available. - int fallback(fmt::internal::Null<>) - { - errno = 0; - buffer_ = strerror(error_code_); - return errno; - } - - public: - StrError(int err_code, char *&buf, std::size_t buf_size) - : error_code_(err_code), buffer_(buf), buffer_size_(buf_size) - {} - - int run() - { - strerror_r(0, 0, ""); // Suppress a warning about unused strerror_r. - return handle(strerror_r(error_code_, buffer_, buffer_size_)); - } - }; - return StrError(error_code, buffer, buffer_size).run(); - } - - void format_error_code(fmt::Writer &out, int error_code, - fmt::StringRef message) FMT_NOEXCEPT - { - // Report error code making sure that the output fits into - // INLINE_BUFFER_SIZE to avoid dynamic memory allocation and potential - // bad_alloc. - out.clear(); - static const char SEP[] = ": "; - static const char ERROR_STR[] = "error "; - // Subtract 2 to account for terminating null characters in SEP and ERROR_STR. - std::size_t error_code_size = sizeof(SEP) + sizeof(ERROR_STR) - 2; - typedef fmt::internal::IntTraits<int>::MainType MainType; - MainType abs_value = static_cast<MainType>(error_code); - if (internal::is_negative(error_code)) { - abs_value = 0 - abs_value; - ++error_code_size; - } - error_code_size += fmt::internal::count_digits(abs_value); - if (message.size() <= fmt::internal::INLINE_BUFFER_SIZE - error_code_size) - out << message << SEP; - out << ERROR_STR << error_code; - assert(out.size() <= fmt::internal::INLINE_BUFFER_SIZE); - } - - void report_error(FormatFunc func, - int error_code, fmt::StringRef message) FMT_NOEXCEPT - { - fmt::MemoryWriter full_message; - func(full_message, error_code, message); - // Use Writer::data instead of Writer::c_str to avoid potential memory - // allocation. - std::fwrite(full_message.data(), full_message.size(), 1, stderr); - std::fputc('\n', stderr); - } - - // IsZeroInt::visit(arg) returns true iff arg is a zero integer. - class IsZeroInt: public fmt::internal::ArgVisitor<IsZeroInt, bool> - { - public: - template <typename T> - bool visit_any_int(T value) - { - return value == 0; - } - }; - - // Checks if an argument is a valid printf width specifier and sets - // left alignment if it is negative. - class WidthHandler: public fmt::internal::ArgVisitor<WidthHandler, unsigned> - { - private: - fmt::FormatSpec &spec_; - - FMT_DISALLOW_COPY_AND_ASSIGN(WidthHandler); - - public: - explicit WidthHandler(fmt::FormatSpec &spec): spec_(spec) - {} - - void report_unhandled_arg() - { - FMT_THROW(fmt::FormatError("width is not integer")); - } - - template <typename T> - unsigned visit_any_int(T value) - { - typedef typename fmt::internal::IntTraits<T>::MainType UnsignedType; - UnsignedType width = static_cast<UnsignedType>(value); - if (fmt::internal::is_negative(value)) { - spec_.align_ = fmt::ALIGN_LEFT; - width = 0 - width; - } - if (width > INT_MAX) - FMT_THROW(fmt::FormatError("number is too big")); - return static_cast<unsigned>(width); - } - }; - - class PrecisionHandler: - public fmt::internal::ArgVisitor<PrecisionHandler, int> - { - public: - void report_unhandled_arg() - { - FMT_THROW(fmt::FormatError("precision is not integer")); - } - - template <typename T> - int visit_any_int(T value) - { - if (!IntChecker<std::numeric_limits<T>::is_signed>::fits_in_int(value)) - FMT_THROW(fmt::FormatError("number is too big")); - return static_cast<int>(value); - } - }; - - template <typename T, typename U> - struct is_same - { - enum - { - value = 0 - }; - }; - - template <typename T> - struct is_same<T, T> - { - enum - { - value = 1 - }; - }; - - // An argument visitor that converts an integer argument to T for printf, - // if T is an integral type. If T is void, the argument is converted to - // corresponding signed or unsigned type depending on the type specifier: - // 'd' and 'i' - signed, other - unsigned) - template <typename T = void> - class ArgConverter: public fmt::internal::ArgVisitor<ArgConverter<T>, void> - { - private: - fmt::internal::Arg &arg_; - wchar_t type_; - - FMT_DISALLOW_COPY_AND_ASSIGN(ArgConverter); - - public: - ArgConverter(fmt::internal::Arg &arg, wchar_t type) - : arg_(arg), type_(type) - {} - - void visit_bool(bool value) - { - if (type_ != 's') - visit_any_int(value); - } - - template <typename U> - void visit_any_int(U value) - { - bool is_signed = type_ == 'd' || type_ == 'i'; - using fmt::internal::Arg; - typedef typename fmt::internal::Conditional< - is_same<T, void>::value, U, T>::type TargetType; - if (sizeof(TargetType) <= sizeof(int)) { - // Extra casts are used to silence warnings. - if (is_signed) { - arg_.type = Arg::INT; - arg_.int_value = static_cast<int>(static_cast<TargetType>(value)); - } - else { - arg_.type = Arg::UINT; - typedef typename fmt::internal::MakeUnsigned<TargetType>::Type Unsigned; - arg_.uint_value = static_cast<unsigned>(static_cast<Unsigned>(value)); - } - } - else { - if (is_signed) { - arg_.type = Arg::LONG_LONG; - // glibc's printf doesn't sign extend arguments of smaller types: - // std::printf("%lld", -42); // prints "4294967254" - // but we don't have to do the same because it's a UB. - arg_.long_long_value = static_cast<fmt::LongLong>(value); - } - else { - arg_.type = Arg::ULONG_LONG; - arg_.ulong_long_value = - static_cast<typename fmt::internal::MakeUnsigned<U>::Type>(value); - } - } - } - }; - - // Converts an integer argument to char for printf. - class CharConverter: public fmt::internal::ArgVisitor<CharConverter, void> - { - private: - fmt::internal::Arg &arg_; - - FMT_DISALLOW_COPY_AND_ASSIGN(CharConverter); - - public: - explicit CharConverter(fmt::internal::Arg &arg): arg_(arg) - {} - - template <typename T> - void visit_any_int(T value) - { - arg_.type = Arg::CHAR; - arg_.int_value = static_cast<char>(value); - } - }; - - // Write the content of w to os. - void write(std::ostream &os, fmt::Writer &w) - { - const char *data = w.data(); - typedef internal::MakeUnsigned<std::streamsize>::Type UnsignedStreamSize; - UnsignedStreamSize size = w.size(); - UnsignedStreamSize max_size = - internal::to_unsigned((std::numeric_limits<std::streamsize>::max)()); - do { - UnsignedStreamSize n = size <= max_size ? size : max_size; - os.write(data, static_cast<std::streamsize>(n)); - data += n; - size -= n; - } while (size != 0); - } - } // namespace - - namespace internal { - - template <typename Char> - class PrintfArgFormatter: - public ArgFormatterBase<PrintfArgFormatter<Char>, Char> - { - - void write_null_pointer() - { - this->spec().type_ = 0; - this->write("(nil)"); - } - - typedef ArgFormatterBase<PrintfArgFormatter<Char>, Char> Base; - - public: - PrintfArgFormatter(BasicWriter<Char> &w, FormatSpec &s) - : ArgFormatterBase<PrintfArgFormatter<Char>, Char>(w, s) - {} - - void visit_bool(bool value) - { - FormatSpec &fmt_spec = this->spec(); - if (fmt_spec.type_ != 's') - return this->visit_any_int(value); - fmt_spec.type_ = 0; - this->write(value); - } - - void visit_char(int value) - { - const FormatSpec &fmt_spec = this->spec(); - BasicWriter<Char> &w = this->writer(); - if (fmt_spec.type_ && fmt_spec.type_ != 'c') - w.write_int(value, fmt_spec); - typedef typename BasicWriter<Char>::CharPtr CharPtr; - CharPtr out = CharPtr(); - if (fmt_spec.width_ > 1) { - Char fill = ' '; - out = w.grow_buffer(fmt_spec.width_); - if (fmt_spec.align_ != ALIGN_LEFT) { - std::fill_n(out, fmt_spec.width_ - 1, fill); - out += fmt_spec.width_ - 1; - } - else { - std::fill_n(out + 1, fmt_spec.width_ - 1, fill); - } - } - else { - out = w.grow_buffer(1); - } - *out = static_cast<Char>(value); - } - - void visit_cstring(const char *value) - { - if (value) - Base::visit_cstring(value); - else if (this->spec().type_ == 'p') - write_null_pointer(); - else - this->write("(null)"); - } - - void visit_pointer(const void *value) - { - if (value) - return Base::visit_pointer(value); - this->spec().type_ = 0; - write_null_pointer(); - } - - void visit_custom(Arg::CustomValue c) - { - BasicFormatter<Char> formatter(ArgList(), this->writer()); - const Char format_str[] = { '}', 0 }; - const Char *format = format_str; - c.format(&formatter, c.value, &format); - } - }; - } // namespace internal -} // namespace fmt - -FMT_FUNC void fmt::SystemError::init( - int err_code, CStringRef format_str, ArgList args) -{ - error_code_ = err_code; - MemoryWriter w; - internal::format_system_error(w, err_code, format(format_str, args)); - std::runtime_error &base = *this; - base = std::runtime_error(w.str()); -} - -template <typename T> -int fmt::internal::CharTraits<char>::format_float( - char *buffer, std::size_t size, const char *format, - unsigned width, int precision, T value) -{ - if (width == 0) { - return precision < 0 ? - FMT_SNPRINTF(buffer, size, format, value) : - FMT_SNPRINTF(buffer, size, format, precision, value); - } - return precision < 0 ? - FMT_SNPRINTF(buffer, size, format, width, value) : - FMT_SNPRINTF(buffer, size, format, width, precision, value); -} - -template <typename T> -int fmt::internal::CharTraits<wchar_t>::format_float( - wchar_t *buffer, std::size_t size, const wchar_t *format, - unsigned width, int precision, T value) -{ - if (width == 0) { - return precision < 0 ? - FMT_SWPRINTF(buffer, size, format, value) : - FMT_SWPRINTF(buffer, size, format, precision, value); - } - return precision < 0 ? - FMT_SWPRINTF(buffer, size, format, width, value) : - FMT_SWPRINTF(buffer, size, format, width, precision, value); -} - -template <typename T> -const char fmt::internal::BasicData<T>::DIGITS[] = -"0001020304050607080910111213141516171819" -"2021222324252627282930313233343536373839" -"4041424344454647484950515253545556575859" -"6061626364656667686970717273747576777879" -"8081828384858687888990919293949596979899"; - -#define FMT_POWERS_OF_10(factor) \ - factor * 10, \ - factor * 100, \ - factor * 1000, \ - factor * 10000, \ - factor * 100000, \ - factor * 1000000, \ - factor * 10000000, \ - factor * 100000000, \ - factor * 1000000000 - -template <typename T> -const uint32_t fmt::internal::BasicData<T>::POWERS_OF_10_32[] = { - 0, FMT_POWERS_OF_10(1) -}; - -template <typename T> -const uint64_t fmt::internal::BasicData<T>::POWERS_OF_10_64[] = { - 0, - FMT_POWERS_OF_10(1), - FMT_POWERS_OF_10(fmt::ULongLong(1000000000)), - // Multiply several constants instead of using a single long long constant - // to avoid warnings about C++98 not supporting long long. - fmt::ULongLong(1000000000) * fmt::ULongLong(1000000000) * 10 -}; - -FMT_FUNC void fmt::internal::report_unknown_type(char code, const char *type) -{ - (void)type; - if (std::isprint(static_cast<unsigned char>(code))) { - FMT_THROW(fmt::FormatError( - fmt::format("unknown format code '{}' for {}", code, type))); - } - FMT_THROW(fmt::FormatError( - fmt::format("unknown format code '\\x{:02x}' for {}", - static_cast<unsigned>(code), type))); -} - -#if FMT_USE_WINDOWS_H - -FMT_FUNC fmt::internal::UTF8ToUTF16::UTF8ToUTF16(fmt::StringRef s) -{ - static const char ERROR_MSG[] = "cannot convert string from UTF-8 to UTF-16"; - if (s.size() > INT_MAX) - FMT_THROW(WindowsError(ERROR_INVALID_PARAMETER, ERROR_MSG)); - int s_size = static_cast<int>(s.size()); - int length = MultiByteToWideChar( - CP_UTF8, MB_ERR_INVALID_CHARS, s.data(), s_size, 0, 0); - if (length == 0) - FMT_THROW(WindowsError(GetLastError(), ERROR_MSG)); - buffer_.resize(length + 1); - length = MultiByteToWideChar( - CP_UTF8, MB_ERR_INVALID_CHARS, s.data(), s_size, &buffer_[0], length); - if (length == 0) - FMT_THROW(WindowsError(GetLastError(), ERROR_MSG)); - buffer_[length] = 0; -} - -FMT_FUNC fmt::internal::UTF16ToUTF8::UTF16ToUTF8(fmt::WStringRef s) -{ - if (int error_code = convert(s)) { - FMT_THROW(WindowsError(error_code, - "cannot convert string from UTF-16 to UTF-8")); - } -} - -FMT_FUNC int fmt::internal::UTF16ToUTF8::convert(fmt::WStringRef s) -{ - if (s.size() > INT_MAX) - return ERROR_INVALID_PARAMETER; - int s_size = static_cast<int>(s.size()); - int length = WideCharToMultiByte(CP_UTF8, 0, s.data(), s_size, 0, 0, 0, 0); - if (length == 0) - return GetLastError(); - buffer_.resize(length + 1); - length = WideCharToMultiByte( - CP_UTF8, 0, s.data(), s_size, &buffer_[0], length, 0, 0); - if (length == 0) - return GetLastError(); - buffer_[length] = 0; - return 0; -} - -FMT_FUNC void fmt::WindowsError::init( - int err_code, CStringRef format_str, ArgList args) -{ - error_code_ = err_code; - MemoryWriter w; - internal::format_windows_error(w, err_code, format(format_str, args)); - std::runtime_error &base = *this; - base = std::runtime_error(w.str()); -} - -FMT_FUNC void fmt::internal::format_windows_error( - fmt::Writer &out, int error_code, - fmt::StringRef message) FMT_NOEXCEPT -{ - FMT_TRY{ - MemoryBuffer<wchar_t, INLINE_BUFFER_SIZE> buffer; - buffer.resize(INLINE_BUFFER_SIZE); - for (;;) { - wchar_t *system_message = &buffer[0]; - int result = FormatMessageW(FORMAT_MESSAGE_FROM_SYSTEM | FORMAT_MESSAGE_IGNORE_INSERTS, - 0, error_code, MAKELANGID(LANG_NEUTRAL, SUBLANG_DEFAULT), - system_message, static_cast<uint32_t>(buffer.size()), 0); - if (result != 0) { - UTF16ToUTF8 utf8_message; - if (utf8_message.convert(system_message) == ERROR_SUCCESS) { - out << message << ": " << utf8_message; - return; - } - break; - } - if (GetLastError() != ERROR_INSUFFICIENT_BUFFER) - break; // Can't get error message, report error code instead. - buffer.resize(buffer.size() * 2); - } - } FMT_CATCH(...) - {} - fmt::format_error_code(out, error_code, message); // 'fmt::' is for bcc32. -} - -#endif // FMT_USE_WINDOWS_H - -FMT_FUNC void fmt::internal::format_system_error( - fmt::Writer &out, int error_code, - fmt::StringRef message) FMT_NOEXCEPT -{ - FMT_TRY{ - MemoryBuffer<char, INLINE_BUFFER_SIZE> buffer; - buffer.resize(INLINE_BUFFER_SIZE); - for (;;) { - char *system_message = &buffer[0]; - int result = safe_strerror(error_code, system_message, buffer.size()); - if (result == 0) { - out << message << ": " << system_message; - return; - } - if (result != ERANGE) - break; // Can't get error message, report error code instead. - buffer.resize(buffer.size() * 2); - } - } FMT_CATCH(...) - {} - fmt::format_error_code(out, error_code, message); // 'fmt::' is for bcc32. -} - -template <typename Char> -void fmt::internal::ArgMap<Char>::init(const ArgList &args) -{ - if (!map_.empty()) - return; - typedef internal::NamedArg<Char> NamedArg; - const NamedArg *named_arg = 0; - bool use_values = - args.type(ArgList::MAX_PACKED_ARGS - 1) == internal::Arg::NONE; - if (use_values) { - for (unsigned i = 0;/*nothing*/; ++i) { - internal::Arg::Type arg_type = args.type(i); - switch (arg_type) { - case internal::Arg::NONE: - return; - case internal::Arg::NAMED_ARG: - named_arg = static_cast<const NamedArg*>(args.values_[i].pointer); - map_.push_back(Pair(named_arg->name, *named_arg)); - break; - default: - /*nothing*/; - } - } - return; - } - for (unsigned i = 0; i != ArgList::MAX_PACKED_ARGS; ++i) { - internal::Arg::Type arg_type = args.type(i); - if (arg_type == internal::Arg::NAMED_ARG) { - named_arg = static_cast<const NamedArg*>(args.args_[i].pointer); - map_.push_back(Pair(named_arg->name, *named_arg)); - } - } - for (unsigned i = ArgList::MAX_PACKED_ARGS;/*nothing*/; ++i) { - switch (args.args_[i].type) { - case internal::Arg::NONE: - return; - case internal::Arg::NAMED_ARG: - named_arg = static_cast<const NamedArg*>(args.args_[i].pointer); - map_.push_back(Pair(named_arg->name, *named_arg)); - break; - default: - /*nothing*/; - } - } -} - -template <typename Char> -void fmt::internal::FixedBuffer<Char>::grow(std::size_t) -{ - FMT_THROW(std::runtime_error("buffer overflow")); -} - -FMT_FUNC Arg fmt::internal::FormatterBase::do_get_arg( - unsigned arg_index, const char *&error) -{ - Arg arg = args_[arg_index]; - switch (arg.type) { - case Arg::NONE: - error = "argument index out of range"; - break; - case Arg::NAMED_ARG: - arg = *static_cast<const internal::Arg*>(arg.pointer); - break; - default: - /*nothing*/; - } - return arg; -} - -template <typename Char> -void fmt::internal::PrintfFormatter<Char>::parse_flags( - FormatSpec &spec, const Char *&s) -{ - for (;;) { - switch (*s++) { - case '-': - spec.align_ = ALIGN_LEFT; - break; - case '+': - spec.flags_ |= SIGN_FLAG | PLUS_FLAG; - break; - case '0': - spec.fill_ = '0'; - break; - case ' ': - spec.flags_ |= SIGN_FLAG; - break; - case '#': - spec.flags_ |= HASH_FLAG; - break; - default: - --s; - return; - } - } -} - -template <typename Char> -Arg fmt::internal::PrintfFormatter<Char>::get_arg( - const Char *s, unsigned arg_index) -{ - (void)s; - const char *error = 0; - Arg arg = arg_index == UINT_MAX ? - next_arg(error) : FormatterBase::get_arg(arg_index - 1, error); - if (error) - FMT_THROW(FormatError(!*s ? "invalid format string" : error)); - return arg; -} - -template <typename Char> -unsigned fmt::internal::PrintfFormatter<Char>::parse_header( - const Char *&s, FormatSpec &spec) -{ - unsigned arg_index = UINT_MAX; - Char c = *s; - if (c >= '0' && c <= '9') { - // Parse an argument index (if followed by '$') or a width possibly - // preceded with '0' flag(s). - unsigned value = parse_nonnegative_int(s); - if (*s == '$') { // value is an argument index - ++s; - arg_index = value; - } - else { - if (c == '0') - spec.fill_ = '0'; - if (value != 0) { - // Nonzero value means that we parsed width and don't need to - // parse it or flags again, so return now. - spec.width_ = value; - return arg_index; - } - } - } - parse_flags(spec, s); - // Parse width. - if (*s >= '0' && *s <= '9') { - spec.width_ = parse_nonnegative_int(s); - } - else if (*s == '*') { - ++s; - spec.width_ = WidthHandler(spec).visit(get_arg(s)); - } - return arg_index; -} - -template <typename Char> -void fmt::internal::PrintfFormatter<Char>::format( - BasicWriter<Char> &writer, BasicCStringRef<Char> format_str) -{ - const Char *start = format_str.c_str(); - const Char *s = start; - while (*s) { - Char c = *s++; - if (c != '%') continue; - if (*s == c) { - write(writer, start, s); - start = ++s; - continue; - } - write(writer, start, s - 1); - - FormatSpec spec; - spec.align_ = ALIGN_RIGHT; - - // Parse argument index, flags and width. - unsigned arg_index = parse_header(s, spec); - - // Parse precision. - if (*s == '.') { - ++s; - if ('0' <= *s && *s <= '9') { - spec.precision_ = static_cast<int>(parse_nonnegative_int(s)); - } - else if (*s == '*') { - ++s; - spec.precision_ = PrecisionHandler().visit(get_arg(s)); - } - } - - Arg arg = get_arg(s, arg_index); - if (spec.flag(HASH_FLAG) && IsZeroInt().visit(arg)) - spec.flags_ &= ~to_unsigned<int>(HASH_FLAG); - if (spec.fill_ == '0') { - if (arg.type <= Arg::LAST_NUMERIC_TYPE) - spec.align_ = ALIGN_NUMERIC; - else - spec.fill_ = ' '; // Ignore '0' flag for non-numeric types. - } - - // Parse length and convert the argument to the required type. - switch (*s++) { - case 'h': - if (*s == 'h') - ArgConverter<signed char>(arg, *++s).visit(arg); - else - ArgConverter<short>(arg, *s).visit(arg); - break; - case 'l': - if (*s == 'l') - ArgConverter<fmt::LongLong>(arg, *++s).visit(arg); - else - ArgConverter<long>(arg, *s).visit(arg); - break; - case 'j': - ArgConverter<intmax_t>(arg, *s).visit(arg); - break; - case 'z': - ArgConverter<std::size_t>(arg, *s).visit(arg); - break; - case 't': - ArgConverter<std::ptrdiff_t>(arg, *s).visit(arg); - break; - case 'L': - // printf produces garbage when 'L' is omitted for long double, no - // need to do the same. - break; - default: - --s; - ArgConverter<void>(arg, *s).visit(arg); - } - - // Parse type. - if (!*s) - FMT_THROW(FormatError("invalid format string")); - spec.type_ = static_cast<char>(*s++); - if (arg.type <= Arg::LAST_INTEGER_TYPE) { - // Normalize type. - switch (spec.type_) { - case 'i': case 'u': - spec.type_ = 'd'; - break; - case 'c': - // TODO: handle wchar_t - CharConverter(arg).visit(arg); - break; - } - } - - start = s; - - // Format argument. - internal::PrintfArgFormatter<Char>(writer, spec).visit(arg); - } - write(writer, start, s); -} - -FMT_FUNC void fmt::report_system_error( - int error_code, fmt::StringRef message) FMT_NOEXCEPT -{ - // 'fmt::' is for bcc32. - fmt::report_error(internal::format_system_error, error_code, message); -} - -#if FMT_USE_WINDOWS_H -FMT_FUNC void fmt::report_windows_error( - int error_code, fmt::StringRef message) FMT_NOEXCEPT -{ - // 'fmt::' is for bcc32. - fmt::report_error(internal::format_windows_error, error_code, message); -} -#endif - -FMT_FUNC void fmt::print(std::FILE *f, CStringRef format_str, ArgList args) -{ - MemoryWriter w; - w.write(format_str, args); - std::fwrite(w.data(), 1, w.size(), f); -} - -FMT_FUNC void fmt::print(CStringRef format_str, ArgList args) -{ - print(stdout, format_str, args); -} - -FMT_FUNC void fmt::print(std::ostream &os, CStringRef format_str, - ArgList args) -{ - MemoryWriter w; - w.write(format_str, args); - write(os, w); -} - -FMT_FUNC void fmt::print_colored(Color c, CStringRef format, ArgList args) -{ - char escape[] = "\x1b[30m"; - escape[3] = static_cast<char>('0' + c); - std::fputs(escape, stdout); - print(format, args); - std::fputs(RESET_COLOR, stdout); -} - -FMT_FUNC int fmt::fprintf(std::FILE *f, CStringRef format, ArgList args) -{ - MemoryWriter w; - printf(w, format, args); - std::size_t size = w.size(); - return std::fwrite(w.data(), 1, size, f) < size ? -1 : static_cast<int>(size); -} - -FMT_FUNC int fmt::fprintf(std::ostream &os, CStringRef format, ArgList args) -{ - MemoryWriter w; - printf(w, format, args); - write(os, w); - return static_cast<int>(w.size()); -} - -#ifndef FMT_HEADER_ONLY - -template struct fmt::internal::BasicData<void>; - -// Explicit instantiations for char. - -template void fmt::internal::FixedBuffer<char>::grow(std::size_t); - -template void fmt::internal::ArgMap<char>::init(const fmt::ArgList &args); - -template void fmt::internal::PrintfFormatter<char>::format( - BasicWriter<char> &writer, CStringRef format); - -template int fmt::internal::CharTraits<char>::format_float( - char *buffer, std::size_t size, const char *format, - unsigned width, int precision, double value); - -template int fmt::internal::CharTraits<char>::format_float( - char *buffer, std::size_t size, const char *format, - unsigned width, int precision, long double value); - -// Explicit instantiations for wchar_t. - -template void fmt::internal::FixedBuffer<wchar_t>::grow(std::size_t); - -template void fmt::internal::ArgMap<wchar_t>::init(const fmt::ArgList &args); - -template void fmt::internal::PrintfFormatter<wchar_t>::format( - BasicWriter<wchar_t> &writer, WCStringRef format); - -template int fmt::internal::CharTraits<wchar_t>::format_float( - wchar_t *buffer, std::size_t size, const wchar_t *format, - unsigned width, int precision, double value); - -template int fmt::internal::CharTraits<wchar_t>::format_float( - wchar_t *buffer, std::size_t size, const wchar_t *format, - unsigned width, int precision, long double value); - -#endif // FMT_HEADER_ONLY - -#ifdef _MSC_VER -# pragma warning(pop) -#endif \ No newline at end of file diff --git a/src/spdlog/details/format.h b/src/spdlog/details/format.h deleted file mode 100644 index 8046e6142d01a144ba5892718c4682afb0336951..0000000000000000000000000000000000000000 --- a/src/spdlog/details/format.h +++ /dev/null @@ -1,4500 +0,0 @@ -/* -Formatting library for C++ - -Copyright (c) 2012 - 2015, Victor Zverovich -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, this -list of conditions and the following disclaimer. -2. Redistributions in binary form must reproduce the above copyright notice, -this list of conditions and the following disclaimer in the documentation -and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND -ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -*/ - -#ifndef FMT_FORMAT_H_ -#define FMT_FORMAT_H_ - - -//Added to spdlog version for header only usage -#define FMT_HEADER_ONLY - -//Added to spdlog version in order to avoid including windows.h -#if !defined (FMT_USE_WINDOWS_H) -#define FMT_USE_WINDOWS_H 0 -#endif - -#include <cassert> -#include <cmath> -#include <cstdio> -#include <cstring> -#include <limits> -#include <memory> -#include <stdexcept> -#include <string> -#include <vector> -#include <utility> - -#ifndef FMT_USE_IOSTREAMS -# define FMT_USE_IOSTREAMS 1 -#endif - -#if FMT_USE_IOSTREAMS -# include <ostream> -#endif - -#ifdef _SECURE_SCL -# define FMT_SECURE_SCL _SECURE_SCL -#else -# define FMT_SECURE_SCL 0 -#endif - -#if FMT_SECURE_SCL -# include <iterator> -#endif - -#if defined(_MSC_VER) && _MSC_VER <= 1500 -typedef unsigned __int32 uint32_t; -typedef unsigned __int64 uint64_t; -typedef __int64 intmax_t; -#else -#include <stdint.h> -#endif - -#if !defined(FMT_HEADER_ONLY) && defined(_WIN32) -# ifdef FMT_EXPORT -# define FMT_API __declspec(dllexport) -# elif defined(FMT_SHARED) -# define FMT_API __declspec(dllimport) -# endif -#endif -#ifndef FMT_API -# define FMT_API -#endif - -#ifdef __GNUC__ -# define FMT_GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__) -# define FMT_GCC_EXTENSION __extension__ -# if FMT_GCC_VERSION >= 406 -# pragma GCC diagnostic push -// Disable the warning about "long long" which is sometimes reported even -// when using __extension__. -# pragma GCC diagnostic ignored "-Wlong-long" -// Disable the warning about declaration shadowing because it affects too -// many valid cases. -# pragma GCC diagnostic ignored "-Wshadow" -// Disable the warning about implicit conversions that may change the sign of -// an integer; silencing it otherwise would require many explicit casts. -# pragma GCC diagnostic ignored "-Wsign-conversion" -# endif -# if __cplusplus >= 201103L || defined __GXX_EXPERIMENTAL_CXX0X__ -# define FMT_HAS_GXX_CXX11 1 -# endif -#else -# define FMT_GCC_EXTENSION -#endif - -#if defined(__clang__) && !defined(__INTEL_COMPILER) -# pragma clang diagnostic push -# pragma clang diagnostic ignored "-Wdocumentation" -#endif - -#ifdef __GNUC_LIBSTD__ -# define FMT_GNUC_LIBSTD_VERSION (__GNUC_LIBSTD__ * 100 + __GNUC_LIBSTD_MINOR__) -#endif - -#ifdef __has_feature -# define FMT_HAS_FEATURE(x) __has_feature(x) -#else -# define FMT_HAS_FEATURE(x) 0 -#endif - -#ifdef __has_builtin -# define FMT_HAS_BUILTIN(x) __has_builtin(x) -#else -# define FMT_HAS_BUILTIN(x) 0 -#endif - -#ifdef __has_cpp_attribute -# define FMT_HAS_CPP_ATTRIBUTE(x) __has_cpp_attribute(x) -#else -# define FMT_HAS_CPP_ATTRIBUTE(x) 0 -#endif - -#ifndef FMT_USE_VARIADIC_TEMPLATES -// Variadic templates are available in GCC since version 4.4 -// (http://gcc.gnu.org/projects/cxx0x.html) and in Visual C++ -// since version 2013. -# define FMT_USE_VARIADIC_TEMPLATES \ - (FMT_HAS_FEATURE(cxx_variadic_templates) || \ - (FMT_GCC_VERSION >= 404 && FMT_HAS_GXX_CXX11) || _MSC_VER >= 1800) -#endif - -#ifndef FMT_USE_RVALUE_REFERENCES -// Don't use rvalue references when compiling with clang and an old libstdc++ -// as the latter doesn't provide std::move. -# if defined(FMT_GNUC_LIBSTD_VERSION) && FMT_GNUC_LIBSTD_VERSION <= 402 -# define FMT_USE_RVALUE_REFERENCES 0 -# else -# define FMT_USE_RVALUE_REFERENCES \ - (FMT_HAS_FEATURE(cxx_rvalue_references) || \ - (FMT_GCC_VERSION >= 403 && FMT_HAS_GXX_CXX11) || _MSC_VER >= 1600) -# endif -#endif - -#if FMT_USE_RVALUE_REFERENCES -# include <utility> // for std::move -#endif - -// Check if exceptions are disabled. -#if defined(__GNUC__) && !defined(__EXCEPTIONS) -# define FMT_EXCEPTIONS 0 -#endif -#if defined(_MSC_VER) && !_HAS_EXCEPTIONS -# define FMT_EXCEPTIONS 0 -#endif -#ifndef FMT_EXCEPTIONS -# define FMT_EXCEPTIONS 1 -#endif - -#ifndef FMT_THROW -# if FMT_EXCEPTIONS -# define FMT_THROW(x) throw x -# else -# define FMT_THROW(x) assert(false) -# endif -#endif - -// Define FMT_USE_NOEXCEPT to make C++ Format use noexcept (C++11 feature). -#ifndef FMT_USE_NOEXCEPT -# define FMT_USE_NOEXCEPT 0 -#endif - -#ifndef FMT_NOEXCEPT -# if FMT_EXCEPTIONS -# if FMT_USE_NOEXCEPT || FMT_HAS_FEATURE(cxx_noexcept) || \ - (FMT_GCC_VERSION >= 408 && FMT_HAS_GXX_CXX11) || \ - _MSC_VER >= 1900 -# define FMT_NOEXCEPT noexcept -# else -# define FMT_NOEXCEPT throw() -# endif -# else -# define FMT_NOEXCEPT -# endif -#endif - -// A macro to disallow the copy constructor and operator= functions -// This should be used in the private: declarations for a class -#ifndef FMT_USE_DELETED_FUNCTIONS -# define FMT_USE_DELETED_FUNCTIONS 0 -#endif - -#if FMT_USE_DELETED_FUNCTIONS || FMT_HAS_FEATURE(cxx_deleted_functions) || \ - (FMT_GCC_VERSION >= 404 && FMT_HAS_GXX_CXX11) || _MSC_VER >= 1800 -# define FMT_DELETED_OR_UNDEFINED = delete -# define FMT_DISALLOW_COPY_AND_ASSIGN(TypeName) \ - TypeName(const TypeName&) = delete; \ - TypeName& operator=(const TypeName&) = delete -#else -# define FMT_DELETED_OR_UNDEFINED -# define FMT_DISALLOW_COPY_AND_ASSIGN(TypeName) \ - TypeName(const TypeName&); \ - TypeName& operator=(const TypeName&) -#endif - -#ifndef FMT_USE_USER_DEFINED_LITERALS -// All compilers which support UDLs also support variadic templates. This -// makes the fmt::literals implementation easier. However, an explicit check -// for variadic templates is added here just in case. -# define FMT_USE_USER_DEFINED_LITERALS \ - FMT_USE_VARIADIC_TEMPLATES && FMT_USE_RVALUE_REFERENCES && \ - (FMT_HAS_FEATURE(cxx_user_literals) || \ - (FMT_GCC_VERSION >= 407 && FMT_HAS_GXX_CXX11) || _MSC_VER >= 1900) -#endif - -#ifndef FMT_ASSERT -# define FMT_ASSERT(condition, message) assert((condition) && message) -#endif - - -#if FMT_GCC_VERSION >= 400 || FMT_HAS_BUILTIN(__builtin_clz) -# define FMT_BUILTIN_CLZ(n) __builtin_clz(n) -#endif - -#if FMT_GCC_VERSION >= 400 || FMT_HAS_BUILTIN(__builtin_clzll) -# define FMT_BUILTIN_CLZLL(n) __builtin_clzll(n) -#endif - -// Some compilers masquerade as both MSVC and GCC-likes or -// otherwise support __builtin_clz and __builtin_clzll, so -// only define FMT_BUILTIN_CLZ using the MSVC intrinsics -// if the clz and clzll builtins are not available. -#if defined(_MSC_VER) && !defined(FMT_BUILTIN_CLZLL) -# include <intrin.h> // _BitScanReverse, _BitScanReverse64 - -namespace fmt -{ -namespace internal -{ -# pragma intrinsic(_BitScanReverse) -inline uint32_t clz(uint32_t x) -{ - unsigned long r = 0; - _BitScanReverse(&r, x); - - assert(x != 0); - // Static analysis complains about using uninitialized data - // "r", but the only way that can happen is if "x" is 0, - // which the callers guarantee to not happen. -# pragma warning(suppress: 6102) - return 31 - r; -} -# define FMT_BUILTIN_CLZ(n) fmt::internal::clz(n) - -# ifdef _WIN64 -# pragma intrinsic(_BitScanReverse64) -# endif - -inline uint32_t clzll(uint64_t x) -{ - unsigned long r = 0; -# ifdef _WIN64 - _BitScanReverse64(&r, x); -# else - // Scan the high 32 bits. - if (_BitScanReverse(&r, static_cast<uint32_t>(x >> 32))) - return 63 - (r + 32); - - // Scan the low 32 bits. - _BitScanReverse(&r, static_cast<uint32_t>(x)); -# endif - - assert(x != 0); - // Static analysis complains about using uninitialized data - // "r", but the only way that can happen is if "x" is 0, - // which the callers guarantee to not happen. -# pragma warning(suppress: 6102) - return 63 - r; -} -# define FMT_BUILTIN_CLZLL(n) fmt::internal::clzll(n) -} -} -#endif - -namespace fmt -{ -namespace internal -{ -struct DummyInt -{ - int data[2]; - operator int() const - { - return 0; - } -}; -typedef std::numeric_limits<fmt::internal::DummyInt> FPUtil; - -// Dummy implementations of system functions such as signbit and ecvt called -// if the latter are not available. -inline DummyInt signbit(...) -{ - return DummyInt(); -} -inline DummyInt _ecvt_s(...) -{ - return DummyInt(); -} -inline DummyInt isinf(...) -{ - return DummyInt(); -} -inline DummyInt _finite(...) -{ - return DummyInt(); -} -inline DummyInt isnan(...) -{ - return DummyInt(); -} -inline DummyInt _isnan(...) -{ - return DummyInt(); -} - -// A helper function to suppress bogus "conditional expression is constant" -// warnings. -template <typename T> -inline T check(T value) -{ - return value; -} -} -} // namespace fmt - -namespace std -{ -// Standard permits specialization of std::numeric_limits. This specialization -// is used to resolve ambiguity between isinf and std::isinf in glibc: -// https://gcc.gnu.org/bugzilla/show_bug.cgi?id=48891 -// and the same for isnan and signbit. -template <> -class numeric_limits<fmt::internal::DummyInt>: - public std::numeric_limits<int> -{ -public: - // Portable version of isinf. - template <typename T> - static bool isinfinity(T x) - { - using namespace fmt::internal; - // The resolution "priority" is: - // isinf macro > std::isinf > ::isinf > fmt::internal::isinf - if (check(sizeof(isinf(x)) == sizeof(bool) || - sizeof(isinf(x)) == sizeof(int))) - { - return isinf(x) != 0; - } - return !_finite(static_cast<double>(x)); - } - - // Portable version of isnan. - template <typename T> - static bool isnotanumber(T x) - { - using namespace fmt::internal; - if (check(sizeof(isnan(x)) == sizeof(bool) || - sizeof(isnan(x)) == sizeof(int))) - { - return isnan(x) != 0; - } - return _isnan(static_cast<double>(x)) != 0; - } - - // Portable version of signbit. - static bool isnegative(double x) - { - using namespace fmt::internal; - if (check(sizeof(signbit(x)) == sizeof(int))) - return signbit(x) != 0; - if (x < 0) return true; - if (!isnotanumber(x)) return false; - int dec = 0, sign = 0; - char buffer[2]; // The buffer size must be >= 2 or _ecvt_s will fail. - _ecvt_s(buffer, sizeof(buffer), x, 0, &dec, &sign); - return sign != 0; - } -}; -} // namespace std - -namespace fmt -{ - -// Fix the warning about long long on older versions of GCC -// that don't support the diagnostic pragma. -FMT_GCC_EXTENSION typedef long long LongLong; -FMT_GCC_EXTENSION typedef unsigned long long ULongLong; - -#if FMT_USE_RVALUE_REFERENCES -using std::move; -#endif - -template <typename Char> -class BasicWriter; - -typedef BasicWriter<char> Writer; -typedef BasicWriter<wchar_t> WWriter; - -namespace internal -{ -template <typename Char> -class BasicArgFormatter; -} - -template <typename CharType, - typename ArgFormatter = internal::BasicArgFormatter<CharType> > -class BasicFormatter; - -template <typename Char, typename T> -void format(BasicFormatter<Char> &f, const Char *&format_str, const T &value); - -/** -\rst -A string reference. It can be constructed from a C string or ``std::string``. - -You can use one of the following typedefs for common character types: - -+------------+-------------------------+ -| Type | Definition | -+============+=========================+ -| StringRef | BasicStringRef<char> | -+------------+-------------------------+ -| WStringRef | BasicStringRef<wchar_t> | -+------------+-------------------------+ - -This class is most useful as a parameter type to allow passing -different types of strings to a function, for example:: - -template <typename... Args> -std::string format(StringRef format_str, const Args & ... args); - -format("{}", 42); -format(std::string("{}"), 42); -\endrst -*/ -template <typename Char> -class BasicStringRef -{ -private: - const Char *data_; - std::size_t size_; - -public: - /** Constructs a string reference object from a C string and a size. */ - BasicStringRef(const Char *s, std::size_t size): data_(s), size_(size) - {} - - /** - \rst - Constructs a string reference object from a C string computing - the size with ``std::char_traits<Char>::length``. - \endrst - */ - BasicStringRef(const Char *s) - : data_(s), size_(std::char_traits<Char>::length(s)) - {} - - /** - \rst - Constructs a string reference from an ``std::string`` object. - \endrst - */ - BasicStringRef(const std::basic_string<Char> &s) - : data_(s.c_str()), size_(s.size()) - {} - - /** - \rst - Converts a string reference to an ``std::string`` object. - \endrst - */ - std::basic_string<Char> to_string() const - { - return std::basic_string<Char>(data_, size_); - } - - /** Returns a pointer to the string data. */ - const Char *data() const - { - return data_; - } - - /** Returns the string size. */ - std::size_t size() const - { - return size_; - } - - // Lexicographically compare this string reference to other. - int compare(BasicStringRef other) const - { - std::size_t size = size_ < other.size_ ? size_ : other.size_; - int result = std::char_traits<Char>::compare(data_, other.data_, size); - if (result == 0) - result = size_ == other.size_ ? 0 : (size_ < other.size_ ? -1 : 1); - return result; - } - - friend bool operator==(BasicStringRef lhs, BasicStringRef rhs) - { - return lhs.compare(rhs) == 0; - } - friend bool operator!=(BasicStringRef lhs, BasicStringRef rhs) - { - return lhs.compare(rhs) != 0; - } - friend bool operator<(BasicStringRef lhs, BasicStringRef rhs) - { - return lhs.compare(rhs) < 0; - } - friend bool operator<=(BasicStringRef lhs, BasicStringRef rhs) - { - return lhs.compare(rhs) <= 0; - } - friend bool operator>(BasicStringRef lhs, BasicStringRef rhs) - { - return lhs.compare(rhs) > 0; - } - friend bool operator>=(BasicStringRef lhs, BasicStringRef rhs) - { - return lhs.compare(rhs) >= 0; - } -}; - -typedef BasicStringRef<char> StringRef; -typedef BasicStringRef<wchar_t> WStringRef; - -/** -\rst -A reference to a null terminated string. It can be constructed from a C -string or ``std::string``. - -You can use one of the following typedefs for common character types: - -+-------------+--------------------------+ -| Type | Definition | -+=============+==========================+ -| CStringRef | BasicCStringRef<char> | -+-------------+--------------------------+ -| WCStringRef | BasicCStringRef<wchar_t> | -+-------------+--------------------------+ - -This class is most useful as a parameter type to allow passing -different types of strings to a function, for example:: - -template <typename... Args> -std::string format(CStringRef format_str, const Args & ... args); - -format("{}", 42); -format(std::string("{}"), 42); -\endrst -*/ -template <typename Char> -class BasicCStringRef -{ -private: - const Char *data_; - -public: - /** Constructs a string reference object from a C string. */ - BasicCStringRef(const Char *s): data_(s) - {} - - /** - \rst - Constructs a string reference from an ``std::string`` object. - \endrst - */ - BasicCStringRef(const std::basic_string<Char> &s): data_(s.c_str()) - {} - - /** Returns the pointer to a C string. */ - const Char *c_str() const - { - return data_; - } -}; - -typedef BasicCStringRef<char> CStringRef; -typedef BasicCStringRef<wchar_t> WCStringRef; - -/** -A formatting error such as invalid format string. -*/ -class FormatError: public std::runtime_error -{ -public: - explicit FormatError(CStringRef message) - : std::runtime_error(message.c_str()) - {} -}; - -namespace internal -{ - -// MakeUnsigned<T>::Type gives an unsigned type corresponding to integer type T. -template <typename T> -struct MakeUnsigned -{ - typedef T Type; -}; - -#define FMT_SPECIALIZE_MAKE_UNSIGNED(T, U) \ - template <> \ - struct MakeUnsigned<T> { typedef U Type; } - -FMT_SPECIALIZE_MAKE_UNSIGNED(char, unsigned char); -FMT_SPECIALIZE_MAKE_UNSIGNED(signed char, unsigned char); -FMT_SPECIALIZE_MAKE_UNSIGNED(short, unsigned short); -FMT_SPECIALIZE_MAKE_UNSIGNED(int, unsigned); -FMT_SPECIALIZE_MAKE_UNSIGNED(long, unsigned long); -FMT_SPECIALIZE_MAKE_UNSIGNED(LongLong, ULongLong); - -// Casts nonnegative integer to unsigned. -template <typename Int> -inline typename MakeUnsigned<Int>::Type to_unsigned(Int value) -{ - FMT_ASSERT(value >= 0, "negative value"); - return static_cast<typename MakeUnsigned<Int>::Type>(value); -} - -// The number of characters to store in the MemoryBuffer object itself -// to avoid dynamic memory allocation. -enum -{ - INLINE_BUFFER_SIZE = 500 -}; - -#if FMT_SECURE_SCL -// Use checked iterator to avoid warnings on MSVC. -template <typename T> -inline stdext::checked_array_iterator<T*> make_ptr(T *ptr, std::size_t size) -{ - return stdext::checked_array_iterator<T*>(ptr, size); -} -#else -template <typename T> -inline T *make_ptr(T *ptr, std::size_t) -{ - return ptr; -} -#endif -} // namespace internal - -/** -\rst -A buffer supporting a subset of ``std::vector``'s operations. -\endrst -*/ -template <typename T> -class Buffer -{ -private: - FMT_DISALLOW_COPY_AND_ASSIGN(Buffer); - -protected: - T *ptr_; - std::size_t size_; - std::size_t capacity_; - - Buffer(T *ptr = 0, std::size_t capacity = 0) - : ptr_(ptr), size_(0), capacity_(capacity) - {} - - /** - \rst - Increases the buffer capacity to hold at least *size* elements updating - ``ptr_`` and ``capacity_``. - \endrst - */ - virtual void grow(std::size_t size) = 0; - -public: - virtual ~Buffer() - {} - - /** Returns the size of this buffer. */ - std::size_t size() const - { - return size_; - } - - /** Returns the capacity of this buffer. */ - std::size_t capacity() const - { - return capacity_; - } - - /** - Resizes the buffer. If T is a POD type new elements may not be initialized. - */ - void resize(std::size_t new_size) - { - if (new_size > capacity_) - grow(new_size); - size_ = new_size; - } - - /** - \rst - Reserves space to store at least *capacity* elements. - \endrst - */ - void reserve(std::size_t capacity) - { - if (capacity > capacity_) - grow(capacity); - } - - void clear() FMT_NOEXCEPT - { - size_ = 0; - } - - void push_back(const T &value) - { - if (size_ == capacity_) - grow(size_ + 1); - ptr_[size_++] = value; - } - - /** Appends data to the end of the buffer. */ - template <typename U> - void append(const U *begin, const U *end); - - T &operator[](std::size_t index) - { - return ptr_[index]; - } - const T &operator[](std::size_t index) const - { - return ptr_[index]; - } -}; - -template <typename T> -template <typename U> -void Buffer<T>::append(const U *begin, const U *end) -{ - std::size_t new_size = size_ + internal::to_unsigned(end - begin); - if (new_size > capacity_) - grow(new_size); - std::uninitialized_copy(begin, end, - internal::make_ptr(ptr_, capacity_) + size_); - size_ = new_size; -} - -namespace internal -{ - -// A memory buffer for trivially copyable/constructible types with the first SIZE -// elements stored in the object itself. -template <typename T, std::size_t SIZE, typename Allocator = std::allocator<T> > -class MemoryBuffer: private Allocator, public Buffer<T> -{ -private: - T data_[SIZE]; - - // Deallocate memory allocated by the buffer. - void deallocate() - { - if (this->ptr_ != data_) Allocator::deallocate(this->ptr_, this->capacity_); - } - -protected: - void grow(std::size_t size); - -public: - explicit MemoryBuffer(const Allocator &alloc = Allocator()) - : Allocator(alloc), Buffer<T>(data_, SIZE) - {} - ~MemoryBuffer() - { - deallocate(); - } - -#if FMT_USE_RVALUE_REFERENCES -private: - // Move data from other to this buffer. - void move(MemoryBuffer &other) - { - Allocator &this_alloc = *this, &other_alloc = other; - this_alloc = std::move(other_alloc); - this->size_ = other.size_; - this->capacity_ = other.capacity_; - if (other.ptr_ == other.data_) - { - this->ptr_ = data_; - std::uninitialized_copy(other.data_, other.data_ + this->size_, - make_ptr(data_, this->capacity_)); - } - else - { - this->ptr_ = other.ptr_; - // Set pointer to the inline array so that delete is not called - // when deallocating. - other.ptr_ = other.data_; - } - } - -public: - MemoryBuffer(MemoryBuffer &&other) - { - move(other); - } - - MemoryBuffer &operator=(MemoryBuffer &&other) - { - assert(this != &other); - deallocate(); - move(other); - return *this; - } -#endif - - // Returns a copy of the allocator associated with this buffer. - Allocator get_allocator() const - { - return *this; - } -}; - -template <typename T, std::size_t SIZE, typename Allocator> -void MemoryBuffer<T, SIZE, Allocator>::grow(std::size_t size) -{ - std::size_t new_capacity = this->capacity_ + this->capacity_ / 2; - if (size > new_capacity) - new_capacity = size; - T *new_ptr = this->allocate(new_capacity); - // The following code doesn't throw, so the raw pointer above doesn't leak. - std::uninitialized_copy(this->ptr_, this->ptr_ + this->size_, - make_ptr(new_ptr, new_capacity)); - std::size_t old_capacity = this->capacity_; - T *old_ptr = this->ptr_; - this->capacity_ = new_capacity; - this->ptr_ = new_ptr; - // deallocate may throw (at least in principle), but it doesn't matter since - // the buffer already uses the new storage and will deallocate it in case - // of exception. - if (old_ptr != data_) - Allocator::deallocate(old_ptr, old_capacity); -} - -// A fixed-size buffer. -template <typename Char> -class FixedBuffer: public fmt::Buffer<Char> -{ -public: - FixedBuffer(Char *array, std::size_t size): fmt::Buffer<Char>(array, size) - {} - -protected: - FMT_API void grow(std::size_t size); -}; - -template <typename Char> -class BasicCharTraits -{ -public: -#if FMT_SECURE_SCL - typedef stdext::checked_array_iterator<Char*> CharPtr; -#else - typedef Char *CharPtr; -#endif - static Char cast(int value) - { - return static_cast<Char>(value); - } -}; - -template <typename Char> -class CharTraits; - -template <> -class CharTraits<char>: public BasicCharTraits<char> -{ -private: - // Conversion from wchar_t to char is not allowed. - static char convert(wchar_t); - -public: - static char convert(char value) - { - return value; - } - - // Formats a floating-point number. - template <typename T> - FMT_API static int format_float(char *buffer, std::size_t size, - const char *format, unsigned width, int precision, T value); -}; - -template <> -class CharTraits<wchar_t>: public BasicCharTraits<wchar_t> -{ -public: - static wchar_t convert(char value) - { - return value; - } - static wchar_t convert(wchar_t value) - { - return value; - } - - template <typename T> - FMT_API static int format_float(wchar_t *buffer, std::size_t size, - const wchar_t *format, unsigned width, int precision, T value); -}; - -// Checks if a number is negative - used to avoid warnings. -template <bool IsSigned> -struct SignChecker -{ - template <typename T> - static bool is_negative(T value) - { - return value < 0; - } -}; - -template <> -struct SignChecker<false> -{ - template <typename T> - static bool is_negative(T) - { - return false; - } -}; - -// Returns true if value is negative, false otherwise. -// Same as (value < 0) but doesn't produce warnings if T is an unsigned type. -template <typename T> -inline bool is_negative(T value) -{ - return SignChecker<std::numeric_limits<T>::is_signed>::is_negative(value); -} - -// Selects uint32_t if FitsIn32Bits is true, uint64_t otherwise. -template <bool FitsIn32Bits> -struct TypeSelector -{ - typedef uint32_t Type; -}; - -template <> -struct TypeSelector<false> -{ - typedef uint64_t Type; -}; - -template <typename T> -struct IntTraits -{ - // Smallest of uint32_t and uint64_t that is large enough to represent - // all values of T. - typedef typename - TypeSelector<std::numeric_limits<T>::digits <= 32>::Type MainType; -}; - -FMT_API void report_unknown_type(char code, const char *type); - -// Static data is placed in this class template to allow header-only -// configuration. -template <typename T = void> -struct FMT_API BasicData -{ - static const uint32_t POWERS_OF_10_32[]; - static const uint64_t POWERS_OF_10_64[]; - static const char DIGITS[]; -}; - -typedef BasicData<> Data; - -#ifdef FMT_BUILTIN_CLZLL -// Returns the number of decimal digits in n. Leading zeros are not counted -// except for n == 0 in which case count_digits returns 1. -inline unsigned count_digits(uint64_t n) -{ - // Based on http://graphics.stanford.edu/~seander/bithacks.html#IntegerLog10 - // and the benchmark https://github.com/localvoid/cxx-benchmark-count-digits. - int t = (64 - FMT_BUILTIN_CLZLL(n | 1)) * 1233 >> 12; - return to_unsigned(t) - (n < Data::POWERS_OF_10_64[t]) + 1; -} -#else -// Fallback version of count_digits used when __builtin_clz is not available. -inline unsigned count_digits(uint64_t n) -{ - unsigned count = 1; - for (;;) - { - // Integer division is slow so do it for a group of four digits instead - // of for every digit. The idea comes from the talk by Alexandrescu - // "Three Optimization Tips for C++". See speed-test for a comparison. - if (n < 10) return count; - if (n < 100) return count + 1; - if (n < 1000) return count + 2; - if (n < 10000) return count + 3; - n /= 10000u; - count += 4; - } -} -#endif - -#ifdef FMT_BUILTIN_CLZ -// Optional version of count_digits for better performance on 32-bit platforms. -inline unsigned count_digits(uint32_t n) -{ - int t = (32 - FMT_BUILTIN_CLZ(n | 1)) * 1233 >> 12; - return to_unsigned(t) - (n < Data::POWERS_OF_10_32[t]) + 1; -} -#endif - -// Formats a decimal unsigned integer value writing into buffer. -template <typename UInt, typename Char> -inline void format_decimal(Char *buffer, UInt value, unsigned num_digits) -{ - buffer += num_digits; - while (value >= 100) - { - // Integer division is slow so do it for a group of two digits instead - // of for every digit. The idea comes from the talk by Alexandrescu - // "Three Optimization Tips for C++". See speed-test for a comparison. - unsigned index = static_cast<unsigned>((value % 100) * 2); - value /= 100; - *--buffer = Data::DIGITS[index + 1]; - *--buffer = Data::DIGITS[index]; - } - if (value < 10) - { - *--buffer = static_cast<char>('0' + value); - return; - } - unsigned index = static_cast<unsigned>(value * 2); - *--buffer = Data::DIGITS[index + 1]; - *--buffer = Data::DIGITS[index]; -} - -#ifndef _WIN32 -# define FMT_USE_WINDOWS_H 0 -#elif !defined(FMT_USE_WINDOWS_H) -# define FMT_USE_WINDOWS_H 1 -#endif - -// Define FMT_USE_WINDOWS_H to 0 to disable use of windows.h. -// All the functionality that relies on it will be disabled too. -#if FMT_USE_WINDOWS_H -// A converter from UTF-8 to UTF-16. -// It is only provided for Windows since other systems support UTF-8 natively. -class UTF8ToUTF16 -{ -private: - MemoryBuffer<wchar_t, INLINE_BUFFER_SIZE> buffer_; - -public: - FMT_API explicit UTF8ToUTF16(StringRef s); - operator WStringRef() const - { - return WStringRef(&buffer_[0], size()); - } - size_t size() const - { - return buffer_.size() - 1; - } - const wchar_t *c_str() const - { - return &buffer_[0]; - } - std::wstring str() const - { - return std::wstring(&buffer_[0], size()); - } -}; - -// A converter from UTF-16 to UTF-8. -// It is only provided for Windows since other systems support UTF-8 natively. -class UTF16ToUTF8 -{ -private: - MemoryBuffer<char, INLINE_BUFFER_SIZE> buffer_; - -public: - UTF16ToUTF8() - {} - FMT_API explicit UTF16ToUTF8(WStringRef s); - operator StringRef() const - { - return StringRef(&buffer_[0], size()); - } - size_t size() const - { - return buffer_.size() - 1; - } - const char *c_str() const - { - return &buffer_[0]; - } - std::string str() const - { - return std::string(&buffer_[0], size()); - } - - // Performs conversion returning a system error code instead of - // throwing exception on conversion error. This method may still throw - // in case of memory allocation error. - FMT_API int convert(WStringRef s); -}; - -FMT_API void format_windows_error(fmt::Writer &out, int error_code, - fmt::StringRef message) FMT_NOEXCEPT; -#endif - -FMT_API void format_system_error(fmt::Writer &out, int error_code, - fmt::StringRef message) FMT_NOEXCEPT; - -// A formatting argument value. -struct Value -{ - template <typename Char> - struct StringValue - { - const Char *value; - std::size_t size; - }; - - typedef void(*FormatFunc)( - void *formatter, const void *arg, void *format_str_ptr); - - struct CustomValue - { - const void *value; - FormatFunc format; - }; - - union - { - int int_value; - unsigned uint_value; - LongLong long_long_value; - ULongLong ulong_long_value; - double double_value; - long double long_double_value; - const void *pointer; - StringValue<char> string; - StringValue<signed char> sstring; - StringValue<unsigned char> ustring; - StringValue<wchar_t> wstring; - CustomValue custom; - }; - - enum Type - { - NONE, NAMED_ARG, - // Integer types should go first, - INT, UINT, LONG_LONG, ULONG_LONG, BOOL, CHAR, LAST_INTEGER_TYPE = CHAR, - // followed by floating-point types. - DOUBLE, LONG_DOUBLE, LAST_NUMERIC_TYPE = LONG_DOUBLE, - CSTRING, STRING, WSTRING, POINTER, CUSTOM - }; -}; - -// A formatting argument. It is a trivially copyable/constructible type to -// allow storage in internal::MemoryBuffer. -struct Arg: Value -{ - Type type; -}; - -template <typename Char> -struct NamedArg; - -template <typename T = void> -struct Null -{}; - -// A helper class template to enable or disable overloads taking wide -// characters and strings in MakeValue. -template <typename T, typename Char> -struct WCharHelper -{ - typedef Null<T> Supported; - typedef T Unsupported; -}; - -template <typename T> -struct WCharHelper<T, wchar_t> -{ - typedef T Supported; - typedef Null<T> Unsupported; -}; - -typedef char Yes[1]; -typedef char No[2]; - -// These are non-members to workaround an overload resolution bug in bcc32. -Yes &convert(fmt::ULongLong); -Yes &convert(std::ostream &); -No &convert(...); - -template <typename T> -T &get(); - -struct DummyStream: std::ostream -{ - DummyStream(); // Suppress a bogus warning in MSVC. - // Hide all operator<< overloads from std::ostream. - void operator<<(Null<>); -}; - -No &operator<<(std::ostream &, int); - -template<typename T, bool ENABLE_CONVERSION> -struct ConvertToIntImpl -{ - enum - { - value = false - }; -}; - -template<typename T> -struct ConvertToIntImpl<T, true> -{ - // Convert to int only if T doesn't have an overloaded operator<<. - enum - { - value = sizeof(convert(get<DummyStream>() << get<T>())) == sizeof(No) - }; -}; - -template<typename T, bool ENABLE_CONVERSION> -struct ConvertToIntImpl2 -{ - enum - { - value = false - }; -}; - -template<typename T> -struct ConvertToIntImpl2<T, true> -{ - enum - { - // Don't convert numeric types. - value = ConvertToIntImpl<T, !std::numeric_limits<T>::is_specialized>::value - }; -}; - -template<typename T> -struct ConvertToInt -{ - enum - { - enable_conversion = sizeof(convert(get<T>())) == sizeof(Yes) - }; - enum - { - value = ConvertToIntImpl2<T, enable_conversion>::value - }; -}; - -#define FMT_DISABLE_CONVERSION_TO_INT(Type) \ - template <> \ - struct ConvertToInt<Type> { enum { value = 0 }; } - -// Silence warnings about convering float to int. -FMT_DISABLE_CONVERSION_TO_INT(float); -FMT_DISABLE_CONVERSION_TO_INT(double); -FMT_DISABLE_CONVERSION_TO_INT(long double); - -template<bool B, class T = void> -struct EnableIf -{}; - -template<class T> -struct EnableIf<true, T> -{ - typedef T type; -}; - -template<bool B, class T, class F> -struct Conditional -{ - typedef T type; -}; - -template<class T, class F> -struct Conditional<false, T, F> -{ - typedef F type; -}; - -// For bcc32 which doesn't understand ! in template arguments. -template<bool> -struct Not -{ - enum - { - value = 0 - }; -}; - -template<> -struct Not<false> -{ - enum - { - value = 1 - }; -}; - -// Makes an Arg object from any type. -template <typename Formatter> -class MakeValue: public Arg -{ -public: - typedef typename Formatter::Char Char; - -private: - // The following two methods are private to disallow formatting of - // arbitrary pointers. If you want to output a pointer cast it to - // "void *" or "const void *". In particular, this forbids formatting - // of "[const] volatile char *" which is printed as bool by iostreams. - // Do not implement! - template <typename T> - MakeValue(const T *value); - template <typename T> - MakeValue(T *value); - - // The following methods are private to disallow formatting of wide - // characters and strings into narrow strings as in - // fmt::format("{}", L"test"); - // To fix this, use a wide format string: fmt::format(L"{}", L"test"). -#if !defined(_MSC_VER) || defined(_NATIVE_WCHAR_T_DEFINED) - MakeValue(typename WCharHelper<wchar_t, Char>::Unsupported); -#endif - MakeValue(typename WCharHelper<wchar_t *, Char>::Unsupported); - MakeValue(typename WCharHelper<const wchar_t *, Char>::Unsupported); - MakeValue(typename WCharHelper<const std::wstring &, Char>::Unsupported); - MakeValue(typename WCharHelper<WStringRef, Char>::Unsupported); - - void set_string(StringRef str) - { - string.value = str.data(); - string.size = str.size(); - } - - void set_string(WStringRef str) - { - wstring.value = str.data(); - wstring.size = str.size(); - } - - // Formats an argument of a custom type, such as a user-defined class. - template <typename T> - static void format_custom_arg( - void *formatter, const void *arg, void *format_str_ptr) - { - format(*static_cast<Formatter*>(formatter), - *static_cast<const Char**>(format_str_ptr), - *static_cast<const T*>(arg)); - } - -public: - MakeValue() - {} - -#define FMT_MAKE_VALUE_(Type, field, TYPE, rhs) \ - MakeValue(Type value) { field = rhs; } \ - static uint64_t type(Type) { return Arg::TYPE; } - -#define FMT_MAKE_VALUE(Type, field, TYPE) \ - FMT_MAKE_VALUE_(Type, field, TYPE, value) - - FMT_MAKE_VALUE(bool, int_value, BOOL) - FMT_MAKE_VALUE(short, int_value, INT) - FMT_MAKE_VALUE(unsigned short, uint_value, UINT) - FMT_MAKE_VALUE(int, int_value, INT) - FMT_MAKE_VALUE(unsigned, uint_value, UINT) - - MakeValue(long value) - { - // To minimize the number of types we need to deal with, long is - // translated either to int or to long long depending on its size. - if (check(sizeof(long) == sizeof(int))) - int_value = static_cast<int>(value); - else - long_long_value = value; - } - static uint64_t type(long) - { - return sizeof(long) == sizeof(int) ? Arg::INT : Arg::LONG_LONG; - } - - MakeValue(unsigned long value) - { - if (check(sizeof(unsigned long) == sizeof(unsigned))) - uint_value = static_cast<unsigned>(value); - else - ulong_long_value = value; - } - static uint64_t type(unsigned long) - { - return sizeof(unsigned long) == sizeof(unsigned) ? - Arg::UINT : Arg::ULONG_LONG; - } - - FMT_MAKE_VALUE(LongLong, long_long_value, LONG_LONG) - FMT_MAKE_VALUE(ULongLong, ulong_long_value, ULONG_LONG) - FMT_MAKE_VALUE(float, double_value, DOUBLE) - FMT_MAKE_VALUE(double, double_value, DOUBLE) - FMT_MAKE_VALUE(long double, long_double_value, LONG_DOUBLE) - FMT_MAKE_VALUE(signed char, int_value, INT) - FMT_MAKE_VALUE(unsigned char, uint_value, UINT) - FMT_MAKE_VALUE(char, int_value, CHAR) - -#if !defined(_MSC_VER) || defined(_NATIVE_WCHAR_T_DEFINED) - MakeValue(typename WCharHelper<wchar_t, Char>::Supported value) - { - int_value = value; - } - static uint64_t type(wchar_t) - { - return Arg::CHAR; - } -#endif - -#define FMT_MAKE_STR_VALUE(Type, TYPE) \ - MakeValue(Type value) { set_string(value); } \ - static uint64_t type(Type) { return Arg::TYPE; } - - FMT_MAKE_VALUE(char *, string.value, CSTRING) - FMT_MAKE_VALUE(const char *, string.value, CSTRING) - FMT_MAKE_VALUE(const signed char *, sstring.value, CSTRING) - FMT_MAKE_VALUE(const unsigned char *, ustring.value, CSTRING) - FMT_MAKE_STR_VALUE(const std::string &, STRING) - FMT_MAKE_STR_VALUE(StringRef, STRING) - FMT_MAKE_VALUE_(CStringRef, string.value, CSTRING, value.c_str()) - -#define FMT_MAKE_WSTR_VALUE(Type, TYPE) \ - MakeValue(typename WCharHelper<Type, Char>::Supported value) { \ - set_string(value); \ - } \ - static uint64_t type(Type) { return Arg::TYPE; } - - FMT_MAKE_WSTR_VALUE(wchar_t *, WSTRING) - FMT_MAKE_WSTR_VALUE(const wchar_t *, WSTRING) - FMT_MAKE_WSTR_VALUE(const std::wstring &, WSTRING) - FMT_MAKE_WSTR_VALUE(WStringRef, WSTRING) - - FMT_MAKE_VALUE(void *, pointer, POINTER) - FMT_MAKE_VALUE(const void *, pointer, POINTER) - - template <typename T> - MakeValue(const T &value, - typename EnableIf<Not< - ConvertToInt<T>::value>::value, int>::type = 0) - { - custom.value = &value; - custom.format = &format_custom_arg<T>; - } - - template <typename T> - MakeValue(const T &value, - typename EnableIf<ConvertToInt<T>::value, int>::type = 0) - { - int_value = value; - } - - template <typename T> - static uint64_t type(const T &) - { - return ConvertToInt<T>::value ? Arg::INT : Arg::CUSTOM; - } - - // Additional template param `Char_` is needed here because make_type always - // uses char. - template <typename Char_> - MakeValue(const NamedArg<Char_> &value) - { - pointer = &value; - } - - template <typename Char_> - static uint64_t type(const NamedArg<Char_> &) - { - return Arg::NAMED_ARG; - } -}; - -template <typename Formatter> -class MakeArg: public Arg -{ -public: - MakeArg() - { - type = Arg::NONE; - } - - template <typename T> - MakeArg(const T &value) - : Arg(MakeValue<Formatter>(value)) - { - type = static_cast<Arg::Type>(MakeValue<Formatter>::type(value)); - } -}; - -template <typename Char> -struct NamedArg: Arg -{ - BasicStringRef<Char> name; - - template <typename T> - NamedArg(BasicStringRef<Char> argname, const T &value) - : Arg(MakeArg< BasicFormatter<Char> >(value)), name(argname) - {} -}; - -#define FMT_DISPATCH(call) static_cast<Impl*>(this)->call - -// An argument visitor. -// To use ArgVisitor define a subclass that implements some or all of the -// visit methods with the same signatures as the methods in ArgVisitor, -// for example, visit_int(int). -// Specify the subclass name as the Impl template parameter. Then calling -// ArgVisitor::visit for some argument will dispatch to a visit method -// specific to the argument type. For example, if the argument type is -// double then visit_double(double) method of a subclass will be called. -// If the subclass doesn't contain a method with this signature, then -// a corresponding method of ArgVisitor will be called. -// -// Example: -// class MyArgVisitor : public ArgVisitor<MyArgVisitor, void> { -// public: -// void visit_int(int value) { print("{}", value); } -// void visit_double(double value) { print("{}", value ); } -// }; -// -// ArgVisitor uses the curiously recurring template pattern: -// http://en.wikipedia.org/wiki/Curiously_recurring_template_pattern -template <typename Impl, typename Result> -class ArgVisitor -{ -public: - void report_unhandled_arg() - {} - - Result visit_unhandled_arg() - { - FMT_DISPATCH(report_unhandled_arg()); - return Result(); - } - - Result visit_int(int value) - { - return FMT_DISPATCH(visit_any_int(value)); - } - Result visit_long_long(LongLong value) - { - return FMT_DISPATCH(visit_any_int(value)); - } - Result visit_uint(unsigned value) - { - return FMT_DISPATCH(visit_any_int(value)); - } - Result visit_ulong_long(ULongLong value) - { - return FMT_DISPATCH(visit_any_int(value)); - } - Result visit_bool(bool value) - { - return FMT_DISPATCH(visit_any_int(value)); - } - Result visit_char(int value) - { - return FMT_DISPATCH(visit_any_int(value)); - } - template <typename T> - Result visit_any_int(T) - { - return FMT_DISPATCH(visit_unhandled_arg()); - } - - Result visit_double(double value) - { - return FMT_DISPATCH(visit_any_double(value)); - } - Result visit_long_double(long double value) - { - return FMT_DISPATCH(visit_any_double(value)); - } - template <typename T> - Result visit_any_double(T) - { - return FMT_DISPATCH(visit_unhandled_arg()); - } - - Result visit_cstring(const char *) - { - return FMT_DISPATCH(visit_unhandled_arg()); - } - Result visit_string(Arg::StringValue<char>) - { - return FMT_DISPATCH(visit_unhandled_arg()); - } - Result visit_wstring(Arg::StringValue<wchar_t>) - { - return FMT_DISPATCH(visit_unhandled_arg()); - } - Result visit_pointer(const void *) - { - return FMT_DISPATCH(visit_unhandled_arg()); - } - Result visit_custom(Arg::CustomValue) - { - return FMT_DISPATCH(visit_unhandled_arg()); - } - - Result visit(const Arg &arg) - { - switch (arg.type) - { - default: - FMT_ASSERT(false, "invalid argument type"); - return Result(); - case Arg::INT: - return FMT_DISPATCH(visit_int(arg.int_value)); - case Arg::UINT: - return FMT_DISPATCH(visit_uint(arg.uint_value)); - case Arg::LONG_LONG: - return FMT_DISPATCH(visit_long_long(arg.long_long_value)); - case Arg::ULONG_LONG: - return FMT_DISPATCH(visit_ulong_long(arg.ulong_long_value)); - case Arg::BOOL: - return FMT_DISPATCH(visit_bool(arg.int_value != 0)); - case Arg::CHAR: - return FMT_DISPATCH(visit_char(arg.int_value)); - case Arg::DOUBLE: - return FMT_DISPATCH(visit_double(arg.double_value)); - case Arg::LONG_DOUBLE: - return FMT_DISPATCH(visit_long_double(arg.long_double_value)); - case Arg::CSTRING: - return FMT_DISPATCH(visit_cstring(arg.string.value)); - case Arg::STRING: - return FMT_DISPATCH(visit_string(arg.string)); - case Arg::WSTRING: - return FMT_DISPATCH(visit_wstring(arg.wstring)); - case Arg::POINTER: - return FMT_DISPATCH(visit_pointer(arg.pointer)); - case Arg::CUSTOM: - return FMT_DISPATCH(visit_custom(arg.custom)); - } - } -}; - -class RuntimeError: public std::runtime_error -{ -protected: - RuntimeError(): std::runtime_error("") - {} -}; - -template <typename Char> -class PrintfArgFormatter; - -template <typename Char> -class ArgMap; -} // namespace internal - -/** An argument list. */ -class ArgList -{ -private: - // To reduce compiled code size per formatting function call, types of first - // MAX_PACKED_ARGS arguments are passed in the types_ field. - uint64_t types_; - union - { - // If the number of arguments is less than MAX_PACKED_ARGS, the argument - // values are stored in values_, otherwise they are stored in args_. - // This is done to reduce compiled code size as storing larger objects - // may require more code (at least on x86-64) even if the same amount of - // data is actually copied to stack. It saves ~10% on the bloat test. - const internal::Value *values_; - const internal::Arg *args_; - }; - - internal::Arg::Type type(unsigned index) const - { - unsigned shift = index * 4; - uint64_t mask = 0xf; - return static_cast<internal::Arg::Type>( - (types_ & (mask << shift)) >> shift); - } - - template <typename Char> - friend class internal::ArgMap; - -public: - // Maximum number of arguments with packed types. - enum - { - MAX_PACKED_ARGS = 16 - }; - - ArgList(): types_(0) - {} - - ArgList(ULongLong types, const internal::Value *values) - : types_(types), values_(values) - {} - ArgList(ULongLong types, const internal::Arg *args) - : types_(types), args_(args) - {} - - /** Returns the argument at specified index. */ - internal::Arg operator[](unsigned index) const - { - using internal::Arg; - Arg arg; - bool use_values = type(MAX_PACKED_ARGS - 1) == Arg::NONE; - if (index < MAX_PACKED_ARGS) - { - Arg::Type arg_type = type(index); - internal::Value &val = arg; - if (arg_type != Arg::NONE) - val = use_values ? values_[index] : args_[index]; - arg.type = arg_type; - return arg; - } - if (use_values) - { - // The index is greater than the number of arguments that can be stored - // in values, so return a "none" argument. - arg.type = Arg::NONE; - return arg; - } - for (unsigned i = MAX_PACKED_ARGS; i <= index; ++i) - { - if (args_[i].type == Arg::NONE) - return args_[i]; - } - return args_[index]; - } -}; - -enum Alignment -{ - ALIGN_DEFAULT, ALIGN_LEFT, ALIGN_RIGHT, ALIGN_CENTER, ALIGN_NUMERIC -}; - -// Flags. -enum -{ - SIGN_FLAG = 1, PLUS_FLAG = 2, MINUS_FLAG = 4, HASH_FLAG = 8, - CHAR_FLAG = 0x10 // Argument has char type - used in error reporting. -}; - -// An empty format specifier. -struct EmptySpec -{}; - -// A type specifier. -template <char TYPE> -struct TypeSpec: EmptySpec -{ - Alignment align() const - { - return ALIGN_DEFAULT; - } - unsigned width() const - { - return 0; - } - int precision() const - { - return -1; - } - bool flag(unsigned) const - { - return false; - } - char type() const - { - return TYPE; - } - char fill() const - { - return ' '; - } -}; - -// A width specifier. -struct WidthSpec -{ - unsigned width_; - // Fill is always wchar_t and cast to char if necessary to avoid having - // two specialization of WidthSpec and its subclasses. - wchar_t fill_; - - WidthSpec(unsigned width, wchar_t fill): width_(width), fill_(fill) - {} - - unsigned width() const - { - return width_; - } - wchar_t fill() const - { - return fill_; - } -}; - -// An alignment specifier. -struct AlignSpec: WidthSpec -{ - Alignment align_; - - AlignSpec(unsigned width, wchar_t fill, Alignment align = ALIGN_DEFAULT) - : WidthSpec(width, fill), align_(align) - {} - - Alignment align() const - { - return align_; - } - - int precision() const - { - return -1; - } -}; - -// An alignment and type specifier. -template <char TYPE> -struct AlignTypeSpec: AlignSpec -{ - AlignTypeSpec(unsigned width, wchar_t fill): AlignSpec(width, fill) - {} - - bool flag(unsigned) const - { - return false; - } - char type() const - { - return TYPE; - } -}; - -// A full format specifier. -struct FormatSpec: AlignSpec -{ - unsigned flags_; - int precision_; - char type_; - - FormatSpec( - unsigned width = 0, char type = 0, wchar_t fill = ' ') - : AlignSpec(width, fill), flags_(0), precision_(-1), type_(type) - {} - - bool flag(unsigned f) const - { - return (flags_ & f) != 0; - } - int precision() const - { - return precision_; - } - char type() const - { - return type_; - } -}; - -// An integer format specifier. -template <typename T, typename SpecT = TypeSpec<0>, typename Char = char> -class IntFormatSpec: public SpecT -{ -private: - T value_; - -public: - IntFormatSpec(T val, const SpecT &spec = SpecT()) - : SpecT(spec), value_(val) - {} - - T value() const - { - return value_; - } -}; - -// A string format specifier. -template <typename Char> -class StrFormatSpec: public AlignSpec -{ -private: - const Char *str_; - -public: - template <typename FillChar> - StrFormatSpec(const Char *str, unsigned width, FillChar fill) - : AlignSpec(width, fill), str_(str) - { - internal::CharTraits<Char>::convert(FillChar()); - } - - const Char *str() const - { - return str_; - } -}; - -/** -Returns an integer format specifier to format the value in base 2. -*/ -IntFormatSpec<int, TypeSpec<'b'> > bin(int value); - -/** -Returns an integer format specifier to format the value in base 8. -*/ -IntFormatSpec<int, TypeSpec<'o'> > oct(int value); - -/** -Returns an integer format specifier to format the value in base 16 using -lower-case letters for the digits above 9. -*/ -IntFormatSpec<int, TypeSpec<'x'> > hex(int value); - -/** -Returns an integer formatter format specifier to format in base 16 using -upper-case letters for the digits above 9. -*/ -IntFormatSpec<int, TypeSpec<'X'> > hexu(int value); - -/** -\rst -Returns an integer format specifier to pad the formatted argument with the -fill character to the specified width using the default (right) numeric -alignment. - -**Example**:: - -MemoryWriter out; -out << pad(hex(0xcafe), 8, '0'); -// out.str() == "0000cafe" - -\endrst -*/ -template <char TYPE_CODE, typename Char> -IntFormatSpec<int, AlignTypeSpec<TYPE_CODE>, Char> pad( - int value, unsigned width, Char fill = ' '); - -#define FMT_DEFINE_INT_FORMATTERS(TYPE) \ -inline IntFormatSpec<TYPE, TypeSpec<'b'> > bin(TYPE value) { \ - return IntFormatSpec<TYPE, TypeSpec<'b'> >(value, TypeSpec<'b'>()); \ -} \ - \ -inline IntFormatSpec<TYPE, TypeSpec<'o'> > oct(TYPE value) { \ - return IntFormatSpec<TYPE, TypeSpec<'o'> >(value, TypeSpec<'o'>()); \ -} \ - \ -inline IntFormatSpec<TYPE, TypeSpec<'x'> > hex(TYPE value) { \ - return IntFormatSpec<TYPE, TypeSpec<'x'> >(value, TypeSpec<'x'>()); \ -} \ - \ -inline IntFormatSpec<TYPE, TypeSpec<'X'> > hexu(TYPE value) { \ - return IntFormatSpec<TYPE, TypeSpec<'X'> >(value, TypeSpec<'X'>()); \ -} \ - \ -template <char TYPE_CODE> \ -inline IntFormatSpec<TYPE, AlignTypeSpec<TYPE_CODE> > pad( \ - IntFormatSpec<TYPE, TypeSpec<TYPE_CODE> > f, unsigned width) { \ - return IntFormatSpec<TYPE, AlignTypeSpec<TYPE_CODE> >( \ - f.value(), AlignTypeSpec<TYPE_CODE>(width, ' ')); \ -} \ - \ -/* For compatibility with older compilers we provide two overloads for pad, */ \ -/* one that takes a fill character and one that doesn't. In the future this */ \ -/* can be replaced with one overload making the template argument Char */ \ -/* default to char (C++11). */ \ -template <char TYPE_CODE, typename Char> \ -inline IntFormatSpec<TYPE, AlignTypeSpec<TYPE_CODE>, Char> pad( \ - IntFormatSpec<TYPE, TypeSpec<TYPE_CODE>, Char> f, \ - unsigned width, Char fill) { \ - return IntFormatSpec<TYPE, AlignTypeSpec<TYPE_CODE>, Char>( \ - f.value(), AlignTypeSpec<TYPE_CODE>(width, fill)); \ -} \ - \ -inline IntFormatSpec<TYPE, AlignTypeSpec<0> > pad( \ - TYPE value, unsigned width) { \ - return IntFormatSpec<TYPE, AlignTypeSpec<0> >( \ - value, AlignTypeSpec<0>(width, ' ')); \ -} \ - \ -template <typename Char> \ -inline IntFormatSpec<TYPE, AlignTypeSpec<0>, Char> pad( \ - TYPE value, unsigned width, Char fill) { \ - return IntFormatSpec<TYPE, AlignTypeSpec<0>, Char>( \ - value, AlignTypeSpec<0>(width, fill)); \ -} - -FMT_DEFINE_INT_FORMATTERS(int) -FMT_DEFINE_INT_FORMATTERS(long) -FMT_DEFINE_INT_FORMATTERS(unsigned) -FMT_DEFINE_INT_FORMATTERS(unsigned long) -FMT_DEFINE_INT_FORMATTERS(LongLong) -FMT_DEFINE_INT_FORMATTERS(ULongLong) - -/** -\rst -Returns a string formatter that pads the formatted argument with the fill -character to the specified width using the default (left) string alignment. - -**Example**:: - -std::string s = str(MemoryWriter() << pad("abc", 8)); -// s == "abc " - -\endrst -*/ -template <typename Char> -inline StrFormatSpec<Char> pad( - const Char *str, unsigned width, Char fill = ' ') -{ - return StrFormatSpec<Char>(str, width, fill); -} - -inline StrFormatSpec<wchar_t> pad( - const wchar_t *str, unsigned width, char fill = ' ') -{ - return StrFormatSpec<wchar_t>(str, width, fill); -} - -namespace internal -{ - -template <typename Char> -class ArgMap -{ -private: - typedef std::vector<std::pair<fmt::BasicStringRef<Char>, internal::Arg> > MapType; - typedef typename MapType::value_type Pair; - - MapType map_; - -public: - FMT_API void init(const ArgList &args); - - const internal::Arg* find(const fmt::BasicStringRef<Char> &name) const - { - // The list is unsorted, so just return the first matching name. - for (typename MapType::const_iterator it = map_.begin(), end = map_.end(); - it != end; ++it) - { - if (it->first == name) - return &it->second; - } - return 0; - } -}; - -template <typename Impl, typename Char> -class ArgFormatterBase: public ArgVisitor<Impl, void> -{ -private: - BasicWriter<Char> &writer_; - FormatSpec &spec_; - - FMT_DISALLOW_COPY_AND_ASSIGN(ArgFormatterBase); - - void write_pointer(const void *p) - { - spec_.flags_ = HASH_FLAG; - spec_.type_ = 'x'; - writer_.write_int(reinterpret_cast<uintptr_t>(p), spec_); - } - -protected: - BasicWriter<Char> &writer() - { - return writer_; - } - FormatSpec &spec() - { - return spec_; - } - - void write(bool value) - { - const char *str_value = value ? "true" : "false"; - Arg::StringValue<char> str = { str_value, std::strlen(str_value) }; - writer_.write_str(str, spec_); - } - - void write(const char *value) - { - Arg::StringValue<char> str = { value, value != 0 ? std::strlen(value) : 0 }; - writer_.write_str(str, spec_); - } - -public: - ArgFormatterBase(BasicWriter<Char> &w, FormatSpec &s) - : writer_(w), spec_(s) - {} - - template <typename T> - void visit_any_int(T value) - { - writer_.write_int(value, spec_); - } - - template <typename T> - void visit_any_double(T value) - { - writer_.write_double(value, spec_); - } - - void visit_bool(bool value) - { - if (spec_.type_) - return visit_any_int(value); - write(value); - } - - void visit_char(int value) - { - if (spec_.type_ && spec_.type_ != 'c') - { - spec_.flags_ |= CHAR_FLAG; - writer_.write_int(value, spec_); - return; - } - if (spec_.align_ == ALIGN_NUMERIC || spec_.flags_ != 0) - FMT_THROW(FormatError("invalid format specifier for char")); - typedef typename BasicWriter<Char>::CharPtr CharPtr; - Char fill = internal::CharTraits<Char>::cast(spec_.fill()); - CharPtr out = CharPtr(); - const unsigned CHAR_WIDTH = 1; - if (spec_.width_ > CHAR_WIDTH) - { - out = writer_.grow_buffer(spec_.width_); - if (spec_.align_ == ALIGN_RIGHT) - { - std::uninitialized_fill_n(out, spec_.width_ - CHAR_WIDTH, fill); - out += spec_.width_ - CHAR_WIDTH; - } - else if (spec_.align_ == ALIGN_CENTER) - { - out = writer_.fill_padding(out, spec_.width_, - internal::check(CHAR_WIDTH), fill); - } - else - { - std::uninitialized_fill_n(out + CHAR_WIDTH, - spec_.width_ - CHAR_WIDTH, fill); - } - } - else - { - out = writer_.grow_buffer(CHAR_WIDTH); - } - *out = internal::CharTraits<Char>::cast(value); - } - - void visit_cstring(const char *value) - { - if (spec_.type_ == 'p') - return write_pointer(value); - write(value); - } - - void visit_string(Arg::StringValue<char> value) - { - writer_.write_str(value, spec_); - } - - using ArgVisitor<Impl, void>::visit_wstring; - - void visit_wstring(Arg::StringValue<Char> value) - { - writer_.write_str(value, spec_); - } - - void visit_pointer(const void *value) - { - if (spec_.type_ && spec_.type_ != 'p') - report_unknown_type(spec_.type_, "pointer"); - write_pointer(value); - } -}; - -// An argument formatter. -template <typename Char> -class BasicArgFormatter: - public ArgFormatterBase<BasicArgFormatter<Char>, Char> -{ -private: - BasicFormatter<Char> &formatter_; - const Char *format_; - -public: - BasicArgFormatter(BasicFormatter<Char> &f, FormatSpec &s, const Char *fmt) - : ArgFormatterBase<BasicArgFormatter<Char>, Char>(f.writer(), s), - formatter_(f), format_(fmt) - {} - - void visit_custom(Arg::CustomValue c) - { - c.format(&formatter_, c.value, &format_); - } -}; - -class FormatterBase -{ -private: - ArgList args_; - int next_arg_index_; - - // Returns the argument with specified index. - FMT_API Arg do_get_arg(unsigned arg_index, const char *&error); - -protected: - const ArgList &args() const - { - return args_; - } - - explicit FormatterBase(const ArgList &args) - { - args_ = args; - next_arg_index_ = 0; - } - - // Returns the next argument. - Arg next_arg(const char *&error) - { - if (next_arg_index_ >= 0) - return do_get_arg(internal::to_unsigned(next_arg_index_++), error); - error = "cannot switch from manual to automatic argument indexing"; - return Arg(); - } - - // Checks if manual indexing is used and returns the argument with - // specified index. - Arg get_arg(unsigned arg_index, const char *&error) - { - return check_no_auto_index(error) ? do_get_arg(arg_index, error) : Arg(); - } - - bool check_no_auto_index(const char *&error) - { - if (next_arg_index_ > 0) - { - error = "cannot switch from automatic to manual argument indexing"; - return false; - } - next_arg_index_ = -1; - return true; - } - - template <typename Char> - void write(BasicWriter<Char> &w, const Char *start, const Char *end) - { - if (start != end) - w << BasicStringRef<Char>(start, internal::to_unsigned(end - start)); - } -}; - -// A printf formatter. -template <typename Char> -class PrintfFormatter: private FormatterBase -{ -private: - void parse_flags(FormatSpec &spec, const Char *&s); - - // Returns the argument with specified index or, if arg_index is equal - // to the maximum unsigned value, the next argument. - Arg get_arg(const Char *s, - unsigned arg_index = (std::numeric_limits<unsigned>::max)()); - - // Parses argument index, flags and width and returns the argument index. - unsigned parse_header(const Char *&s, FormatSpec &spec); - -public: - explicit PrintfFormatter(const ArgList &args): FormatterBase(args) - {} - FMT_API void format(BasicWriter<Char> &writer, - BasicCStringRef<Char> format_str); -}; -} // namespace internal - -/** This template formats data and writes the output to a writer. */ -template <typename CharType, typename ArgFormatter> -class BasicFormatter: private internal::FormatterBase -{ -public: - /** The character type for the output. */ - typedef CharType Char; - -private: - BasicWriter<Char> &writer_; - internal::ArgMap<Char> map_; - - FMT_DISALLOW_COPY_AND_ASSIGN(BasicFormatter); - - using internal::FormatterBase::get_arg; - - // Checks if manual indexing is used and returns the argument with - // specified name. - internal::Arg get_arg(BasicStringRef<Char> arg_name, const char *&error); - - // Parses argument index and returns corresponding argument. - internal::Arg parse_arg_index(const Char *&s); - - // Parses argument name and returns corresponding argument. - internal::Arg parse_arg_name(const Char *&s); - -public: - /** - \rst - Constructs a ``BasicFormatter`` object. References to the arguments and - the writer are stored in the formatter object so make sure they have - appropriate lifetimes. - \endrst - */ - BasicFormatter(const ArgList &args, BasicWriter<Char> &w) - : internal::FormatterBase(args), writer_(w) - {} - - /** Returns a reference to the writer associated with this formatter. */ - BasicWriter<Char> &writer() - { - return writer_; - } - - /** Formats stored arguments and writes the output to the writer. */ - void format(BasicCStringRef<Char> format_str); - - // Formats a single argument and advances format_str, a format string pointer. - const Char *format(const Char *&format_str, const internal::Arg &arg); -}; - -// Generates a comma-separated list with results of applying f to -// numbers 0..n-1. -# define FMT_GEN(n, f) FMT_GEN##n(f) -# define FMT_GEN1(f) f(0) -# define FMT_GEN2(f) FMT_GEN1(f), f(1) -# define FMT_GEN3(f) FMT_GEN2(f), f(2) -# define FMT_GEN4(f) FMT_GEN3(f), f(3) -# define FMT_GEN5(f) FMT_GEN4(f), f(4) -# define FMT_GEN6(f) FMT_GEN5(f), f(5) -# define FMT_GEN7(f) FMT_GEN6(f), f(6) -# define FMT_GEN8(f) FMT_GEN7(f), f(7) -# define FMT_GEN9(f) FMT_GEN8(f), f(8) -# define FMT_GEN10(f) FMT_GEN9(f), f(9) -# define FMT_GEN11(f) FMT_GEN10(f), f(10) -# define FMT_GEN12(f) FMT_GEN11(f), f(11) -# define FMT_GEN13(f) FMT_GEN12(f), f(12) -# define FMT_GEN14(f) FMT_GEN13(f), f(13) -# define FMT_GEN15(f) FMT_GEN14(f), f(14) - -namespace internal -{ -inline uint64_t make_type() -{ - return 0; -} - -template <typename T> -inline uint64_t make_type(const T &arg) -{ - return MakeValue< BasicFormatter<char> >::type(arg); -} - -template <unsigned N, bool/*IsPacked*/ = (N < ArgList::MAX_PACKED_ARGS)> - struct ArgArray; - -template <unsigned N> -struct ArgArray<N, true/*IsPacked*/> -{ - typedef Value Type[N > 0 ? N : 1]; - -template <typename Formatter, typename T> -static Value make(const T &value) -{ - Value result = MakeValue<Formatter>(value); - // Workaround a bug in Apple LLVM version 4.2 (clang-425.0.28) of clang: - // https://github.com/cppformat/cppformat/issues/276 - (void)result.custom.format; - return result; -} - }; - -template <unsigned N> -struct ArgArray<N, false/*IsPacked*/> -{ - typedef Arg Type[N + 1]; // +1 for the list end Arg::NONE - - template <typename Formatter, typename T> - static Arg make(const T &value) - { - return MakeArg<Formatter>(value); - } -}; - -#if FMT_USE_VARIADIC_TEMPLATES -template <typename Arg, typename... Args> -inline uint64_t make_type(const Arg &first, const Args & ... tail) -{ - return make_type(first) | (make_type(tail...) << 4); -} - -#else - -struct ArgType -{ - uint64_t type; - - ArgType(): type(0) - {} - - template <typename T> - ArgType(const T &arg) : type(make_type(arg)) - {} -}; - -# define FMT_ARG_TYPE_DEFAULT(n) ArgType t##n = ArgType() - -inline uint64_t make_type(FMT_GEN15(FMT_ARG_TYPE_DEFAULT)) -{ - return t0.type | (t1.type << 4) | (t2.type << 8) | (t3.type << 12) | - (t4.type << 16) | (t5.type << 20) | (t6.type << 24) | (t7.type << 28) | - (t8.type << 32) | (t9.type << 36) | (t10.type << 40) | (t11.type << 44) | - (t12.type << 48) | (t13.type << 52) | (t14.type << 56); -} -#endif - -template <class Char> -class FormatBuf: public std::basic_streambuf<Char> -{ -private: - typedef typename std::basic_streambuf<Char>::int_type int_type; - typedef typename std::basic_streambuf<Char>::traits_type traits_type; - - Buffer<Char> &buffer_; - Char *start_; - -public: - FormatBuf(Buffer<Char> &buffer): buffer_(buffer), start_(&buffer[0]) - { - this->setp(start_, start_ + buffer_.capacity()); - } - - int_type overflow(int_type ch = traits_type::eof()) - { - if (!traits_type::eq_int_type(ch, traits_type::eof())) - { - size_t size = this->size(); - buffer_.resize(size); - buffer_.reserve(size * 2); - - start_ = &buffer_[0]; - start_[size] = traits_type::to_char_type(ch); - this->setp(start_ + size + 1, start_ + size * 2); - } - return ch; - } - - size_t size() const - { - return to_unsigned(this->pptr() - start_); - } -}; -} // namespace internal - -# define FMT_MAKE_TEMPLATE_ARG(n) typename T##n -# define FMT_MAKE_ARG_TYPE(n) T##n -# define FMT_MAKE_ARG(n) const T##n &v##n -# define FMT_ASSIGN_char(n) \ - arr[n] = fmt::internal::MakeValue< fmt::BasicFormatter<char> >(v##n) -# define FMT_ASSIGN_wchar_t(n) \ - arr[n] = fmt::internal::MakeValue< fmt::BasicFormatter<wchar_t> >(v##n) - -#if FMT_USE_VARIADIC_TEMPLATES -// Defines a variadic function returning void. -# define FMT_VARIADIC_VOID(func, arg_type) \ - template <typename... Args> \ - void func(arg_type arg0, const Args & ... args) { \ - typedef fmt::internal::ArgArray<sizeof...(Args)> ArgArray; \ - typename ArgArray::Type array{ \ - ArgArray::template make<fmt::BasicFormatter<Char> >(args)...}; \ - func(arg0, fmt::ArgList(fmt::internal::make_type(args...), array)); \ - } - -// Defines a variadic constructor. -# define FMT_VARIADIC_CTOR(ctor, func, arg0_type, arg1_type) \ - template <typename... Args> \ - ctor(arg0_type arg0, arg1_type arg1, const Args & ... args) { \ - typedef fmt::internal::ArgArray<sizeof...(Args)> ArgArray; \ - typename ArgArray::Type array{ \ - ArgArray::template make<fmt::BasicFormatter<Char> >(args)...}; \ - func(arg0, arg1, fmt::ArgList(fmt::internal::make_type(args...), array)); \ - } - -#else - -# define FMT_MAKE_REF(n) \ - fmt::internal::MakeValue< fmt::BasicFormatter<Char> >(v##n) -# define FMT_MAKE_REF2(n) v##n - -// Defines a wrapper for a function taking one argument of type arg_type -// and n additional arguments of arbitrary types. -# define FMT_WRAP1(func, arg_type, n) \ - template <FMT_GEN(n, FMT_MAKE_TEMPLATE_ARG)> \ - inline void func(arg_type arg1, FMT_GEN(n, FMT_MAKE_ARG)) { \ - const fmt::internal::ArgArray<n>::Type array = {FMT_GEN(n, FMT_MAKE_REF)}; \ - func(arg1, fmt::ArgList( \ - fmt::internal::make_type(FMT_GEN(n, FMT_MAKE_REF2)), array)); \ - } - -// Emulates a variadic function returning void on a pre-C++11 compiler. -# define FMT_VARIADIC_VOID(func, arg_type) \ - inline void func(arg_type arg) { func(arg, fmt::ArgList()); } \ - FMT_WRAP1(func, arg_type, 1) FMT_WRAP1(func, arg_type, 2) \ - FMT_WRAP1(func, arg_type, 3) FMT_WRAP1(func, arg_type, 4) \ - FMT_WRAP1(func, arg_type, 5) FMT_WRAP1(func, arg_type, 6) \ - FMT_WRAP1(func, arg_type, 7) FMT_WRAP1(func, arg_type, 8) \ - FMT_WRAP1(func, arg_type, 9) FMT_WRAP1(func, arg_type, 10) - -# define FMT_CTOR(ctor, func, arg0_type, arg1_type, n) \ - template <FMT_GEN(n, FMT_MAKE_TEMPLATE_ARG)> \ - ctor(arg0_type arg0, arg1_type arg1, FMT_GEN(n, FMT_MAKE_ARG)) { \ - const fmt::internal::ArgArray<n>::Type array = {FMT_GEN(n, FMT_MAKE_REF)}; \ - func(arg0, arg1, fmt::ArgList( \ - fmt::internal::make_type(FMT_GEN(n, FMT_MAKE_REF2)), array)); \ - } - -// Emulates a variadic constructor on a pre-C++11 compiler. -# define FMT_VARIADIC_CTOR(ctor, func, arg0_type, arg1_type) \ - FMT_CTOR(ctor, func, arg0_type, arg1_type, 1) \ - FMT_CTOR(ctor, func, arg0_type, arg1_type, 2) \ - FMT_CTOR(ctor, func, arg0_type, arg1_type, 3) \ - FMT_CTOR(ctor, func, arg0_type, arg1_type, 4) \ - FMT_CTOR(ctor, func, arg0_type, arg1_type, 5) \ - FMT_CTOR(ctor, func, arg0_type, arg1_type, 6) \ - FMT_CTOR(ctor, func, arg0_type, arg1_type, 7) \ - FMT_CTOR(ctor, func, arg0_type, arg1_type, 8) \ - FMT_CTOR(ctor, func, arg0_type, arg1_type, 9) \ - FMT_CTOR(ctor, func, arg0_type, arg1_type, 10) -#endif - -// Generates a comma-separated list with results of applying f to pairs -// (argument, index). -#define FMT_FOR_EACH1(f, x0) f(x0, 0) -#define FMT_FOR_EACH2(f, x0, x1) \ - FMT_FOR_EACH1(f, x0), f(x1, 1) -#define FMT_FOR_EACH3(f, x0, x1, x2) \ - FMT_FOR_EACH2(f, x0 ,x1), f(x2, 2) -#define FMT_FOR_EACH4(f, x0, x1, x2, x3) \ - FMT_FOR_EACH3(f, x0, x1, x2), f(x3, 3) -#define FMT_FOR_EACH5(f, x0, x1, x2, x3, x4) \ - FMT_FOR_EACH4(f, x0, x1, x2, x3), f(x4, 4) -#define FMT_FOR_EACH6(f, x0, x1, x2, x3, x4, x5) \ - FMT_FOR_EACH5(f, x0, x1, x2, x3, x4), f(x5, 5) -#define FMT_FOR_EACH7(f, x0, x1, x2, x3, x4, x5, x6) \ - FMT_FOR_EACH6(f, x0, x1, x2, x3, x4, x5), f(x6, 6) -#define FMT_FOR_EACH8(f, x0, x1, x2, x3, x4, x5, x6, x7) \ - FMT_FOR_EACH7(f, x0, x1, x2, x3, x4, x5, x6), f(x7, 7) -#define FMT_FOR_EACH9(f, x0, x1, x2, x3, x4, x5, x6, x7, x8) \ - FMT_FOR_EACH8(f, x0, x1, x2, x3, x4, x5, x6, x7), f(x8, 8) -#define FMT_FOR_EACH10(f, x0, x1, x2, x3, x4, x5, x6, x7, x8, x9) \ - FMT_FOR_EACH9(f, x0, x1, x2, x3, x4, x5, x6, x7, x8), f(x9, 9) - -/** -An error returned by an operating system or a language runtime, -for example a file opening error. -*/ -class SystemError: public internal::RuntimeError -{ -private: - void init(int err_code, CStringRef format_str, ArgList args); - -protected: - int error_code_; - - typedef char Char; // For FMT_VARIADIC_CTOR. - - SystemError() - {} - -public: - /** - \rst - Constructs a :class:`fmt::SystemError` object with the description - of the form - - .. parsed-literal:: - *<message>*: *<system-message>* - - where *<message>* is the formatted message and *<system-message>* is - the system message corresponding to the error code. - *error_code* is a system error code as given by ``errno``. - If *error_code* is not a valid error code such as -1, the system message - may look like "Unknown error -1" and is platform-dependent. - - **Example**:: - - // This throws a SystemError with the description - // cannot open file 'madeup': No such file or directory - // or similar (system message may vary). - const char *filename = "madeup"; - std::FILE *file = std::fopen(filename, "r"); - if (!file) - throw fmt::SystemError(errno, "cannot open file '{}'", filename); - \endrst - */ - SystemError(int error_code, CStringRef message) - { - init(error_code, message, ArgList()); - } - FMT_VARIADIC_CTOR(SystemError, init, int, CStringRef) - - int error_code() const - { - return error_code_; - } -}; - -/** -\rst -This template provides operations for formatting and writing data into -a character stream. The output is stored in a buffer provided by a subclass -such as :class:`fmt::BasicMemoryWriter`. - -You can use one of the following typedefs for common character types: - -+---------+----------------------+ -| Type | Definition | -+=========+======================+ -| Writer | BasicWriter<char> | -+---------+----------------------+ -| WWriter | BasicWriter<wchar_t> | -+---------+----------------------+ - -\endrst -*/ -template <typename Char> -class BasicWriter -{ -private: - // Output buffer. - Buffer<Char> &buffer_; - - FMT_DISALLOW_COPY_AND_ASSIGN(BasicWriter); - - typedef typename internal::CharTraits<Char>::CharPtr CharPtr; - -#if FMT_SECURE_SCL - // Returns pointer value. - static Char *get(CharPtr p) - { - return p.base(); - } -#else - static Char *get(Char *p) - { - return p; - } -#endif - - // Fills the padding around the content and returns the pointer to the - // content area. - static CharPtr fill_padding(CharPtr buffer, - unsigned total_size, std::size_t content_size, wchar_t fill); - - // Grows the buffer by n characters and returns a pointer to the newly - // allocated area. - CharPtr grow_buffer(std::size_t n) - { - std::size_t size = buffer_.size(); - buffer_.resize(size + n); - return internal::make_ptr(&buffer_[size], n); - } - - // Writes an unsigned decimal integer. - template <typename UInt> - Char *write_unsigned_decimal(UInt value, unsigned prefix_size = 0) - { - unsigned num_digits = internal::count_digits(value); - Char *ptr = get(grow_buffer(prefix_size + num_digits)); - internal::format_decimal(ptr + prefix_size, value, num_digits); - return ptr; - } - - // Writes a decimal integer. - template <typename Int> - void write_decimal(Int value) - { - typedef typename internal::IntTraits<Int>::MainType MainType; - MainType abs_value = static_cast<MainType>(value); - if (internal::is_negative(value)) - { - abs_value = 0 - abs_value; - *write_unsigned_decimal(abs_value, 1) = '-'; - } - else - { - write_unsigned_decimal(abs_value, 0); - } - } - - // Prepare a buffer for integer formatting. - CharPtr prepare_int_buffer(unsigned num_digits, - const EmptySpec &, const char *prefix, unsigned prefix_size) - { - unsigned size = prefix_size + num_digits; - CharPtr p = grow_buffer(size); - std::uninitialized_copy(prefix, prefix + prefix_size, p); - return p + size - 1; - } - - template <typename Spec> - CharPtr prepare_int_buffer(unsigned num_digits, - const Spec &spec, const char *prefix, unsigned prefix_size); - - // Formats an integer. - template <typename T, typename Spec> - void write_int(T value, Spec spec); - - // Formats a floating-point number (double or long double). - template <typename T> - void write_double(T value, const FormatSpec &spec); - - // Writes a formatted string. - template <typename StrChar> - CharPtr write_str(const StrChar *s, std::size_t size, const AlignSpec &spec); - - template <typename StrChar> - void write_str(const internal::Arg::StringValue<StrChar> &str, - const FormatSpec &spec); - - // This following methods are private to disallow writing wide characters - // and strings to a char stream. If you want to print a wide string as a - // pointer as std::ostream does, cast it to const void*. - // Do not implement! - void operator<<(typename internal::WCharHelper<wchar_t, Char>::Unsupported); - void operator<<( - typename internal::WCharHelper<const wchar_t *, Char>::Unsupported); - - // Appends floating-point length specifier to the format string. - // The second argument is only used for overload resolution. - void append_float_length(Char *&format_ptr, long double) - { - *format_ptr++ = 'L'; - } - - template<typename T> - void append_float_length(Char *&, T) - {} - - template <typename Impl, typename Char_> - friend class internal::ArgFormatterBase; - - friend class internal::PrintfArgFormatter<Char>; - -protected: - /** - Constructs a ``BasicWriter`` object. - */ - explicit BasicWriter(Buffer<Char> &b): buffer_(b) - {} - -public: - /** - \rst - Destroys a ``BasicWriter`` object. - \endrst - */ - virtual ~BasicWriter() - {} - - /** - Returns the total number of characters written. - */ - std::size_t size() const - { - return buffer_.size(); - } - - /** - Returns a pointer to the output buffer content. No terminating null - character is appended. - */ - const Char *data() const FMT_NOEXCEPT - { - return &buffer_[0]; - } - - /** - Returns a pointer to the output buffer content with terminating null - character appended. - */ - const Char *c_str() const - { - std::size_t size = buffer_.size(); - buffer_.reserve(size + 1); - buffer_[size] = '\0'; - return &buffer_[0]; - } - - /** - \rst - Returns the content of the output buffer as an `std::string`. - \endrst - */ - std::basic_string<Char> str() const - { - return std::basic_string<Char>(&buffer_[0], buffer_.size()); - } - - /** - \rst - Writes formatted data. - - *args* is an argument list representing arbitrary arguments. - - **Example**:: - - MemoryWriter out; - out.write("Current point:\n"); - out.write("({:+f}, {:+f})", -3.14, 3.14); - - This will write the following output to the ``out`` object: - - .. code-block:: none - - Current point: - (-3.140000, +3.140000) - - The output can be accessed using :func:`data()`, :func:`c_str` or - :func:`str` methods. - - See also :ref:`syntax`. - \endrst - */ - void write(BasicCStringRef<Char> format, ArgList args) - { - BasicFormatter<Char>(args, *this).format(format); - } - FMT_VARIADIC_VOID(write, BasicCStringRef<Char>) - - BasicWriter &operator<<(int value) - { - write_decimal(value); - return *this; - } - BasicWriter &operator<<(unsigned value) - { - return *this << IntFormatSpec<unsigned>(value); - } - BasicWriter &operator<<(long value) - { - write_decimal(value); - return *this; - } - BasicWriter &operator<<(unsigned long value) - { - return *this << IntFormatSpec<unsigned long>(value); - } - BasicWriter &operator<<(LongLong value) - { - write_decimal(value); - return *this; - } - - /** - \rst - Formats *value* and writes it to the stream. - \endrst - */ - BasicWriter &operator<<(ULongLong value) - { - return *this << IntFormatSpec<ULongLong>(value); - } - - BasicWriter &operator<<(double value) - { - write_double(value, FormatSpec()); - return *this; - } - - /** - \rst - Formats *value* using the general format for floating-point numbers - (``'g'``) and writes it to the stream. - \endrst - */ - BasicWriter &operator<<(long double value) - { - write_double(value, FormatSpec()); - return *this; - } - - /** - Writes a character to the stream. - */ - BasicWriter &operator<<(char value) - { - buffer_.push_back(value); - return *this; - } - - BasicWriter &operator<<( - typename internal::WCharHelper<wchar_t, Char>::Supported value) - { - buffer_.push_back(value); - return *this; - } - - /** - \rst - Writes *value* to the stream. - \endrst - */ - BasicWriter &operator<<(fmt::BasicStringRef<Char> value) - { - const Char *str = value.data(); - buffer_.append(str, str + value.size()); - return *this; - } - - BasicWriter &operator<<( - typename internal::WCharHelper<StringRef, Char>::Supported value) - { - const char *str = value.data(); - buffer_.append(str, str + value.size()); - return *this; - } - - template <typename T, typename Spec, typename FillChar> - BasicWriter &operator<<(IntFormatSpec<T, Spec, FillChar> spec) - { - internal::CharTraits<Char>::convert(FillChar()); - write_int(spec.value(), spec); - return *this; - } - - template <typename StrChar> - BasicWriter &operator<<(const StrFormatSpec<StrChar> &spec) - { - const StrChar *s = spec.str(); - write_str(s, std::char_traits<Char>::length(s), spec); - return *this; - } - - void clear() FMT_NOEXCEPT - { - buffer_.clear(); - } -}; - -template <typename Char> -template <typename StrChar> -typename BasicWriter<Char>::CharPtr BasicWriter<Char>::write_str( - const StrChar *s, std::size_t size, const AlignSpec &spec) -{ - CharPtr out = CharPtr(); - if (spec.width() > size) - { - out = grow_buffer(spec.width()); - Char fill = internal::CharTraits<Char>::cast(spec.fill()); - if (spec.align() == ALIGN_RIGHT) - { - std::uninitialized_fill_n(out, spec.width() - size, fill); - out += spec.width() - size; - } - else if (spec.align() == ALIGN_CENTER) - { - out = fill_padding(out, spec.width(), size, fill); - } - else - { - std::uninitialized_fill_n(out + size, spec.width() - size, fill); - } - } - else - { - out = grow_buffer(size); - } - std::uninitialized_copy(s, s + size, out); - return out; -} - -template <typename Char> -template <typename StrChar> -void BasicWriter<Char>::write_str( - const internal::Arg::StringValue<StrChar> &s, const FormatSpec &spec) -{ - // Check if StrChar is convertible to Char. - internal::CharTraits<Char>::convert(StrChar()); - if (spec.type_ && spec.type_ != 's') - internal::report_unknown_type(spec.type_, "string"); - const StrChar *str_value = s.value; - std::size_t str_size = s.size; - if (str_size == 0) - { - if (!str_value) - { - FMT_THROW(FormatError("string pointer is null")); - return; - } - } - std::size_t precision = static_cast<std::size_t>(spec.precision_); - if (spec.precision_ >= 0 && precision < str_size) - str_size = precision; - write_str(str_value, str_size, spec); -} - -template <typename Char> -typename BasicWriter<Char>::CharPtr -BasicWriter<Char>::fill_padding( - CharPtr buffer, unsigned total_size, - std::size_t content_size, wchar_t fill) -{ - std::size_t padding = total_size - content_size; - std::size_t left_padding = padding / 2; - Char fill_char = internal::CharTraits<Char>::cast(fill); - std::uninitialized_fill_n(buffer, left_padding, fill_char); - buffer += left_padding; - CharPtr content = buffer; - std::uninitialized_fill_n(buffer + content_size, - padding - left_padding, fill_char); - return content; -} - -template <typename Char> -template <typename Spec> -typename BasicWriter<Char>::CharPtr -BasicWriter<Char>::prepare_int_buffer( - unsigned num_digits, const Spec &spec, - const char *prefix, unsigned prefix_size) -{ - unsigned width = spec.width(); - Alignment align = spec.align(); - Char fill = internal::CharTraits<Char>::cast(spec.fill()); - if (spec.precision() > static_cast<int>(num_digits)) - { - // Octal prefix '0' is counted as a digit, so ignore it if precision - // is specified. - if (prefix_size > 0 && prefix[prefix_size - 1] == '0') - --prefix_size; - unsigned number_size = - prefix_size + internal::to_unsigned(spec.precision()); - AlignSpec subspec(number_size, '0', ALIGN_NUMERIC); - if (number_size >= width) - return prepare_int_buffer(num_digits, subspec, prefix, prefix_size); - buffer_.reserve(width); - unsigned fill_size = width - number_size; - if (align != ALIGN_LEFT) - { - CharPtr p = grow_buffer(fill_size); - std::uninitialized_fill(p, p + fill_size, fill); - } - CharPtr result = prepare_int_buffer( - num_digits, subspec, prefix, prefix_size); - if (align == ALIGN_LEFT) - { - CharPtr p = grow_buffer(fill_size); - std::uninitialized_fill(p, p + fill_size, fill); - } - return result; - } - unsigned size = prefix_size + num_digits; - if (width <= size) - { - CharPtr p = grow_buffer(size); - std::uninitialized_copy(prefix, prefix + prefix_size, p); - return p + size - 1; - } - CharPtr p = grow_buffer(width); - CharPtr end = p + width; - if (align == ALIGN_LEFT) - { - std::uninitialized_copy(prefix, prefix + prefix_size, p); - p += size; - std::uninitialized_fill(p, end, fill); - } - else if (align == ALIGN_CENTER) - { - p = fill_padding(p, width, size, fill); - std::uninitialized_copy(prefix, prefix + prefix_size, p); - p += size; - } - else - { - if (align == ALIGN_NUMERIC) - { - if (prefix_size != 0) - { - p = std::uninitialized_copy(prefix, prefix + prefix_size, p); - size -= prefix_size; - } - } - else - { - std::uninitialized_copy(prefix, prefix + prefix_size, end - size); - } - std::uninitialized_fill(p, end - size, fill); - p = end; - } - return p - 1; -} - -template <typename Char> -template <typename T, typename Spec> -void BasicWriter<Char>::write_int(T value, Spec spec) -{ - unsigned prefix_size = 0; - typedef typename internal::IntTraits<T>::MainType UnsignedType; - UnsignedType abs_value = static_cast<UnsignedType>(value); - char prefix[4] = ""; - if (internal::is_negative(value)) - { - prefix[0] = '-'; - ++prefix_size; - abs_value = 0 - abs_value; - } - else if (spec.flag(SIGN_FLAG)) - { - prefix[0] = spec.flag(PLUS_FLAG) ? '+' : ' '; - ++prefix_size; - } - switch (spec.type()) - { - case 0: - case 'd': - { - unsigned num_digits = internal::count_digits(abs_value); - CharPtr p = prepare_int_buffer( - num_digits, spec, prefix, prefix_size) + 1 - num_digits; - internal::format_decimal(get(p), abs_value, num_digits); - break; - } - case 'x': - case 'X': - { - UnsignedType n = abs_value; - if (spec.flag(HASH_FLAG)) - { - prefix[prefix_size++] = '0'; - prefix[prefix_size++] = spec.type(); - } - unsigned num_digits = 0; - do - { - ++num_digits; - } - while ((n >>= 4) != 0); - Char *p = get(prepare_int_buffer( - num_digits, spec, prefix, prefix_size)); - n = abs_value; - const char *digits = spec.type() == 'x' ? - "0123456789abcdef" : "0123456789ABCDEF"; - do - { - *p-- = digits[n & 0xf]; - } - while ((n >>= 4) != 0); - break; - } - case 'b': - case 'B': - { - UnsignedType n = abs_value; - if (spec.flag(HASH_FLAG)) - { - prefix[prefix_size++] = '0'; - prefix[prefix_size++] = spec.type(); - } - unsigned num_digits = 0; - do - { - ++num_digits; - } - while ((n >>= 1) != 0); - Char *p = get(prepare_int_buffer(num_digits, spec, prefix, prefix_size)); - n = abs_value; - do - { - *p-- = static_cast<Char>('0' + (n & 1)); - } - while ((n >>= 1) != 0); - break; - } - case 'o': - { - UnsignedType n = abs_value; - if (spec.flag(HASH_FLAG)) - prefix[prefix_size++] = '0'; - unsigned num_digits = 0; - do - { - ++num_digits; - } - while ((n >>= 3) != 0); - Char *p = get(prepare_int_buffer(num_digits, spec, prefix, prefix_size)); - n = abs_value; - do - { - *p-- = static_cast<Char>('0' + (n & 7)); - } - while ((n >>= 3) != 0); - break; - } - default: - internal::report_unknown_type( - spec.type(), spec.flag(CHAR_FLAG) ? "char" : "integer"); - break; - } -} - -template <typename Char> -template <typename T> -void BasicWriter<Char>::write_double(T value, const FormatSpec &spec) -{ - // Check type. - char type = spec.type(); - bool upper = false; - switch (type) - { - case 0: - type = 'g'; - break; - case 'e': - case 'f': - case 'g': - case 'a': - break; - case 'F': -#ifdef _MSC_VER - // MSVC's printf doesn't support 'F'. - type = 'f'; -#endif - // Fall through. - case 'E': - case 'G': - case 'A': - upper = true; - break; - default: - internal::report_unknown_type(type, "double"); - break; - } - - char sign = 0; - // Use isnegative instead of value < 0 because the latter is always - // false for NaN. - if (internal::FPUtil::isnegative(static_cast<double>(value))) - { - sign = '-'; - value = -value; - } - else if (spec.flag(SIGN_FLAG)) - { - sign = spec.flag(PLUS_FLAG) ? '+' : ' '; - } - - if (internal::FPUtil::isnotanumber(value)) - { - // Format NaN ourselves because sprintf's output is not consistent - // across platforms. - std::size_t nan_size = 4; - const char *nan = upper ? " NAN" : " nan"; - if (!sign) - { - --nan_size; - ++nan; - } - CharPtr out = write_str(nan, nan_size, spec); - if (sign) - *out = sign; - return; - } - - if (internal::FPUtil::isinfinity(value)) - { - // Format infinity ourselves because sprintf's output is not consistent - // across platforms. - std::size_t inf_size = 4; - const char *inf = upper ? " INF" : " inf"; - if (!sign) - { - --inf_size; - ++inf; - } - CharPtr out = write_str(inf, inf_size, spec); - if (sign) - *out = sign; - return; - } - - std::size_t offset = buffer_.size(); - unsigned width = spec.width(); - if (sign) - { - buffer_.reserve(buffer_.size() + (width > 1u ? width : 1u)); - if (width > 0) - --width; - ++offset; - } - - // Build format string. - enum - { - MAX_FORMAT_SIZE = 10 - }; // longest format: %#-*.*Lg - Char format[MAX_FORMAT_SIZE]; - Char *format_ptr = format; - *format_ptr++ = '%'; - unsigned width_for_sprintf = width; - if (spec.flag(HASH_FLAG)) - *format_ptr++ = '#'; - if (spec.align() == ALIGN_CENTER) - { - width_for_sprintf = 0; - } - else - { - if (spec.align() == ALIGN_LEFT) - *format_ptr++ = '-'; - if (width != 0) - *format_ptr++ = '*'; - } - if (spec.precision() >= 0) - { - *format_ptr++ = '.'; - *format_ptr++ = '*'; - } - - append_float_length(format_ptr, value); - *format_ptr++ = type; - *format_ptr = '\0'; - - // Format using snprintf. - Char fill = internal::CharTraits<Char>::cast(spec.fill()); - unsigned n = 0; - Char *start = 0; - for (;;) - { - std::size_t buffer_size = buffer_.capacity() - offset; -#ifdef _MSC_VER - // MSVC's vsnprintf_s doesn't work with zero size, so reserve - // space for at least one extra character to make the size non-zero. - // Note that the buffer's capacity will increase by more than 1. - if (buffer_size == 0) - { - buffer_.reserve(offset + 1); - buffer_size = buffer_.capacity() - offset; - } -#endif - start = &buffer_[offset]; - int result = internal::CharTraits<Char>::format_float( - start, buffer_size, format, width_for_sprintf, spec.precision(), value); - if (result >= 0) - { - n = internal::to_unsigned(result); - if (offset + n < buffer_.capacity()) - break; // The buffer is large enough - continue with formatting. - buffer_.reserve(offset + n + 1); - } - else - { - // If result is negative we ask to increase the capacity by at least 1, - // but as std::vector, the buffer grows exponentially. - buffer_.reserve(buffer_.capacity() + 1); - } - } - if (sign) - { - if ((spec.align() != ALIGN_RIGHT && spec.align() != ALIGN_DEFAULT) || - *start != ' ') - { - *(start - 1) = sign; - sign = 0; - } - else - { - *(start - 1) = fill; - } - ++n; - } - if (spec.align() == ALIGN_CENTER && spec.width() > n) - { - width = spec.width(); - CharPtr p = grow_buffer(width); - std::memmove(get(p) + (width - n) / 2, get(p), n * sizeof(Char)); - fill_padding(p, spec.width(), n, fill); - return; - } - if (spec.fill() != ' ' || sign) - { - while (*start == ' ') - *start++ = fill; - if (sign) - *(start - 1) = sign; - } - grow_buffer(n); -} - -/** -\rst -This class template provides operations for formatting and writing data -into a character stream. The output is stored in a memory buffer that grows -dynamically. - -You can use one of the following typedefs for common character types -and the standard allocator: - -+---------------+-----------------------------------------------------+ -| Type | Definition | -+===============+=====================================================+ -| MemoryWriter | BasicMemoryWriter<char, std::allocator<char>> | -+---------------+-----------------------------------------------------+ -| WMemoryWriter | BasicMemoryWriter<wchar_t, std::allocator<wchar_t>> | -+---------------+-----------------------------------------------------+ - -**Example**:: - -MemoryWriter out; -out << "The answer is " << 42 << "\n"; -out.write("({:+f}, {:+f})", -3.14, 3.14); - -This will write the following output to the ``out`` object: - -.. code-block:: none - -The answer is 42 -(-3.140000, +3.140000) - -The output can be converted to an ``std::string`` with ``out.str()`` or -accessed as a C string with ``out.c_str()``. -\endrst -*/ -template <typename Char, typename Allocator = std::allocator<Char> > -class BasicMemoryWriter: public BasicWriter<Char> -{ -private: - internal::MemoryBuffer<Char, internal::INLINE_BUFFER_SIZE, Allocator> buffer_; - -public: - explicit BasicMemoryWriter(const Allocator& alloc = Allocator()) - : BasicWriter<Char>(buffer_), buffer_(alloc) - {} - -#if FMT_USE_RVALUE_REFERENCES - /** - \rst - Constructs a :class:`fmt::BasicMemoryWriter` object moving the content - of the other object to it. - \endrst - */ - BasicMemoryWriter(BasicMemoryWriter &&other) - : BasicWriter<Char>(buffer_), buffer_(std::move(other.buffer_)) - {} - - /** - \rst - Moves the content of the other ``BasicMemoryWriter`` object to this one. - \endrst - */ - BasicMemoryWriter &operator=(BasicMemoryWriter &&other) - { - buffer_ = std::move(other.buffer_); - return *this; - } -#endif -}; - -typedef BasicMemoryWriter<char> MemoryWriter; -typedef BasicMemoryWriter<wchar_t> WMemoryWriter; - -/** -\rst -This class template provides operations for formatting and writing data -into a fixed-size array. For writing into a dynamically growing buffer -use :class:`fmt::BasicMemoryWriter`. - -Any write method will throw ``std::runtime_error`` if the output doesn't fit -into the array. - -You can use one of the following typedefs for common character types: - -+--------------+---------------------------+ -| Type | Definition | -+==============+===========================+ -| ArrayWriter | BasicArrayWriter<char> | -+--------------+---------------------------+ -| WArrayWriter | BasicArrayWriter<wchar_t> | -+--------------+---------------------------+ -\endrst -*/ -template <typename Char> -class BasicArrayWriter: public BasicWriter<Char> -{ -private: - internal::FixedBuffer<Char> buffer_; - -public: - /** - \rst - Constructs a :class:`fmt::BasicArrayWriter` object for *array* of the - given size. - \endrst - */ - BasicArrayWriter(Char *array, std::size_t size) - : BasicWriter<Char>(buffer_), buffer_(array, size) - {} - - /** - \rst - Constructs a :class:`fmt::BasicArrayWriter` object for *array* of the - size known at compile time. - \endrst - */ - template <std::size_t SIZE> - explicit BasicArrayWriter(Char(&array)[SIZE]) - : BasicWriter<Char>(buffer_), buffer_(array, SIZE) - {} -}; - -typedef BasicArrayWriter<char> ArrayWriter; -typedef BasicArrayWriter<wchar_t> WArrayWriter; - -// Formats a value. -template <typename Char, typename T> -void format(BasicFormatter<Char> &f, const Char *&format_str, const T &value) -{ - internal::MemoryBuffer<Char, internal::INLINE_BUFFER_SIZE> buffer; - - internal::FormatBuf<Char> format_buf(buffer); - std::basic_ostream<Char> output(&format_buf); - output << value; - - BasicStringRef<Char> str(&buffer[0], format_buf.size()); - typedef internal::MakeArg< BasicFormatter<Char> > MakeArg; - format_str = f.format(format_str, MakeArg(str)); -} - -// Reports a system error without throwing an exception. -// Can be used to report errors from destructors. -FMT_API void report_system_error(int error_code, - StringRef message) FMT_NOEXCEPT; - -#if FMT_USE_WINDOWS_H - -/** A Windows error. */ -class WindowsError: public SystemError -{ -private: - FMT_API void init(int error_code, CStringRef format_str, ArgList args); - -public: - /** - \rst - Constructs a :class:`fmt::WindowsError` object with the description - of the form - - .. parsed-literal:: - *<message>*: *<system-message>* - - where *<message>* is the formatted message and *<system-message>* is the - system message corresponding to the error code. - *error_code* is a Windows error code as given by ``GetLastError``. - If *error_code* is not a valid error code such as -1, the system message - will look like "error -1". - - **Example**:: - - // This throws a WindowsError with the description - // cannot open file 'madeup': The system cannot find the file specified. - // or similar (system message may vary). - const char *filename = "madeup"; - LPOFSTRUCT of = LPOFSTRUCT(); - HFILE file = OpenFile(filename, &of, OF_READ); - if (file == HFILE_ERROR) { - throw fmt::WindowsError(GetLastError(), - "cannot open file '{}'", filename); - } - \endrst - */ - WindowsError(int error_code, CStringRef message) - { - init(error_code, message, ArgList()); - } - FMT_VARIADIC_CTOR(WindowsError, init, int, CStringRef) -}; - -// Reports a Windows error without throwing an exception. -// Can be used to report errors from destructors. -FMT_API void report_windows_error(int error_code, - StringRef message) FMT_NOEXCEPT; - -#endif - -enum Color -{ - BLACK, RED, GREEN, YELLOW, BLUE, MAGENTA, CYAN, WHITE -}; - -/** -Formats a string and prints it to stdout using ANSI escape sequences -to specify color (experimental). -Example: -print_colored(fmt::RED, "Elapsed time: {0:.2f} seconds", 1.23); -*/ -FMT_API void print_colored(Color c, CStringRef format, ArgList args); - -/** -\rst -Formats arguments and returns the result as a string. - -**Example**:: - -std::string message = format("The answer is {}", 42); -\endrst -*/ -inline std::string format(CStringRef format_str, ArgList args) -{ - MemoryWriter w; - w.write(format_str, args); - return w.str(); -} - -inline std::wstring format(WCStringRef format_str, ArgList args) -{ - WMemoryWriter w; - w.write(format_str, args); - return w.str(); -} - -/** -\rst -Prints formatted data to the file *f*. - -**Example**:: - -print(stderr, "Don't {}!", "panic"); -\endrst -*/ -FMT_API void print(std::FILE *f, CStringRef format_str, ArgList args); - -/** -\rst -Prints formatted data to ``stdout``. - -**Example**:: - -print("Elapsed time: {0:.2f} seconds", 1.23); -\endrst -*/ -FMT_API void print(CStringRef format_str, ArgList args); - -template <typename Char> -void printf(BasicWriter<Char> &w, BasicCStringRef<Char> format, ArgList args) -{ - internal::PrintfFormatter<Char>(args).format(w, format); -} - -/** -\rst -Formats arguments and returns the result as a string. - -**Example**:: - -std::string message = fmt::sprintf("The answer is %d", 42); -\endrst -*/ -inline std::string sprintf(CStringRef format, ArgList args) -{ - MemoryWriter w; - printf(w, format, args); - return w.str(); -} - -inline std::wstring sprintf(WCStringRef format, ArgList args) -{ - WMemoryWriter w; - printf(w, format, args); - return w.str(); -} - -/** -\rst -Prints formatted data to the file *f*. - -**Example**:: - -fmt::fprintf(stderr, "Don't %s!", "panic"); -\endrst -*/ -FMT_API int fprintf(std::FILE *f, CStringRef format, ArgList args); - -/** -\rst -Prints formatted data to ``stdout``. - -**Example**:: - -fmt::printf("Elapsed time: %.2f seconds", 1.23); -\endrst -*/ -inline int printf(CStringRef format, ArgList args) -{ - return fprintf(stdout, format, args); -} - -/** -Fast integer formatter. -*/ -class FormatInt -{ -private: - // Buffer should be large enough to hold all digits (digits10 + 1), - // a sign and a null character. - enum - { - BUFFER_SIZE = std::numeric_limits<ULongLong>::digits10 + 3 - }; - mutable char buffer_[BUFFER_SIZE]; - char *str_; - - // Formats value in reverse and returns the number of digits. - char *format_decimal(ULongLong value) - { - char *buffer_end = buffer_ + BUFFER_SIZE - 1; - while (value >= 100) - { - // Integer division is slow so do it for a group of two digits instead - // of for every digit. The idea comes from the talk by Alexandrescu - // "Three Optimization Tips for C++". See speed-test for a comparison. - unsigned index = static_cast<unsigned>((value % 100) * 2); - value /= 100; - *--buffer_end = internal::Data::DIGITS[index + 1]; - *--buffer_end = internal::Data::DIGITS[index]; - } - if (value < 10) - { - *--buffer_end = static_cast<char>('0' + value); - return buffer_end; - } - unsigned index = static_cast<unsigned>(value * 2); - *--buffer_end = internal::Data::DIGITS[index + 1]; - *--buffer_end = internal::Data::DIGITS[index]; - return buffer_end; - } - - void FormatSigned(LongLong value) - { - ULongLong abs_value = static_cast<ULongLong>(value); - bool negative = value < 0; - if (negative) - abs_value = 0 - abs_value; - str_ = format_decimal(abs_value); - if (negative) - *--str_ = '-'; - } - -public: - explicit FormatInt(int value) - { - FormatSigned(value); - } - explicit FormatInt(long value) - { - FormatSigned(value); - } - explicit FormatInt(LongLong value) - { - FormatSigned(value); - } - explicit FormatInt(unsigned value): str_(format_decimal(value)) - {} - explicit FormatInt(unsigned long value): str_(format_decimal(value)) - {} - explicit FormatInt(ULongLong value): str_(format_decimal(value)) - {} - - /** Returns the number of characters written to the output buffer. */ - std::size_t size() const - { - return internal::to_unsigned(buffer_ - str_ + BUFFER_SIZE - 1); - } - - /** - Returns a pointer to the output buffer content. No terminating null - character is appended. - */ - const char *data() const - { - return str_; - } - - /** - Returns a pointer to the output buffer content with terminating null - character appended. - */ - const char *c_str() const - { - buffer_[BUFFER_SIZE - 1] = '\0'; - return str_; - } - - /** - \rst - Returns the content of the output buffer as an ``std::string``. - \endrst - */ - std::string str() const - { - return std::string(str_, size()); - } -}; - -// Formats a decimal integer value writing into buffer and returns -// a pointer to the end of the formatted string. This function doesn't -// write a terminating null character. -template <typename T> -inline void format_decimal(char *&buffer, T value) -{ - typedef typename internal::IntTraits<T>::MainType MainType; - MainType abs_value = static_cast<MainType>(value); - if (internal::is_negative(value)) - { - *buffer++ = '-'; - abs_value = 0 - abs_value; - } - if (abs_value < 100) - { - if (abs_value < 10) - { - *buffer++ = static_cast<char>('0' + abs_value); - return; - } - unsigned index = static_cast<unsigned>(abs_value * 2); - *buffer++ = internal::Data::DIGITS[index]; - *buffer++ = internal::Data::DIGITS[index + 1]; - return; - } - unsigned num_digits = internal::count_digits(abs_value); - internal::format_decimal(buffer, abs_value, num_digits); - buffer += num_digits; -} - -/** -\rst -Returns a named argument for formatting functions. - -**Example**:: - -print("Elapsed time: {s:.2f} seconds", arg("s", 1.23)); - -\endrst -*/ -template <typename T> -inline internal::NamedArg<char> arg(StringRef name, const T &arg) -{ - return internal::NamedArg<char>(name, arg); -} - -template <typename T> -inline internal::NamedArg<wchar_t> arg(WStringRef name, const T &arg) -{ - return internal::NamedArg<wchar_t>(name, arg); -} - -// The following two functions are deleted intentionally to disable -// nested named arguments as in ``format("{}", arg("a", arg("b", 42)))``. -template <typename Char> -void arg(StringRef, const internal::NamedArg<Char>&) FMT_DELETED_OR_UNDEFINED; -template <typename Char> -void arg(WStringRef, const internal::NamedArg<Char>&) FMT_DELETED_OR_UNDEFINED; -} - -#if FMT_GCC_VERSION -// Use the system_header pragma to suppress warnings about variadic macros -// because suppressing -Wvariadic-macros with the diagnostic pragma doesn't -// work. It is used at the end because we want to suppress as little warnings -// as possible. -# pragma GCC system_header -#endif - -// This is used to work around VC++ bugs in handling variadic macros. -#define FMT_EXPAND(args) args - -// Returns the number of arguments. -// Based on https://groups.google.com/forum/#!topic/comp.std.c/d-6Mj5Lko_s. -#define FMT_NARG(...) FMT_NARG_(__VA_ARGS__, FMT_RSEQ_N()) -#define FMT_NARG_(...) FMT_EXPAND(FMT_ARG_N(__VA_ARGS__)) -#define FMT_ARG_N(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, N, ...) N -#define FMT_RSEQ_N() 10, 9, 8, 7, 6, 5, 4, 3, 2, 1, 0 - -#define FMT_CONCAT(a, b) a##b -#define FMT_FOR_EACH_(N, f, ...) \ - FMT_EXPAND(FMT_CONCAT(FMT_FOR_EACH, N)(f, __VA_ARGS__)) -#define FMT_FOR_EACH(f, ...) \ - FMT_EXPAND(FMT_FOR_EACH_(FMT_NARG(__VA_ARGS__), f, __VA_ARGS__)) - -#define FMT_ADD_ARG_NAME(type, index) type arg##index -#define FMT_GET_ARG_NAME(type, index) arg##index - -#if FMT_USE_VARIADIC_TEMPLATES -# define FMT_VARIADIC_(Char, ReturnType, func, call, ...) \ - template <typename... Args> \ - ReturnType func(FMT_FOR_EACH(FMT_ADD_ARG_NAME, __VA_ARGS__), \ - const Args & ... args) { \ - typedef fmt::internal::ArgArray<sizeof...(Args)> ArgArray; \ - typename ArgArray::Type array{ \ - ArgArray::template make<fmt::BasicFormatter<Char> >(args)...}; \ - call(FMT_FOR_EACH(FMT_GET_ARG_NAME, __VA_ARGS__), \ - fmt::ArgList(fmt::internal::make_type(args...), array)); \ - } -#else -// Defines a wrapper for a function taking __VA_ARGS__ arguments -// and n additional arguments of arbitrary types. -# define FMT_WRAP(Char, ReturnType, func, call, n, ...) \ - template <FMT_GEN(n, FMT_MAKE_TEMPLATE_ARG)> \ - inline ReturnType func(FMT_FOR_EACH(FMT_ADD_ARG_NAME, __VA_ARGS__), \ - FMT_GEN(n, FMT_MAKE_ARG)) { \ - fmt::internal::ArgArray<n>::Type arr; \ - FMT_GEN(n, FMT_ASSIGN_##Char); \ - call(FMT_FOR_EACH(FMT_GET_ARG_NAME, __VA_ARGS__), fmt::ArgList( \ - fmt::internal::make_type(FMT_GEN(n, FMT_MAKE_REF2)), arr)); \ - } - -# define FMT_VARIADIC_(Char, ReturnType, func, call, ...) \ - inline ReturnType func(FMT_FOR_EACH(FMT_ADD_ARG_NAME, __VA_ARGS__)) { \ - call(FMT_FOR_EACH(FMT_GET_ARG_NAME, __VA_ARGS__), fmt::ArgList()); \ - } \ - FMT_WRAP(Char, ReturnType, func, call, 1, __VA_ARGS__) \ - FMT_WRAP(Char, ReturnType, func, call, 2, __VA_ARGS__) \ - FMT_WRAP(Char, ReturnType, func, call, 3, __VA_ARGS__) \ - FMT_WRAP(Char, ReturnType, func, call, 4, __VA_ARGS__) \ - FMT_WRAP(Char, ReturnType, func, call, 5, __VA_ARGS__) \ - FMT_WRAP(Char, ReturnType, func, call, 6, __VA_ARGS__) \ - FMT_WRAP(Char, ReturnType, func, call, 7, __VA_ARGS__) \ - FMT_WRAP(Char, ReturnType, func, call, 8, __VA_ARGS__) \ - FMT_WRAP(Char, ReturnType, func, call, 9, __VA_ARGS__) \ - FMT_WRAP(Char, ReturnType, func, call, 10, __VA_ARGS__) \ - FMT_WRAP(Char, ReturnType, func, call, 11, __VA_ARGS__) \ - FMT_WRAP(Char, ReturnType, func, call, 12, __VA_ARGS__) \ - FMT_WRAP(Char, ReturnType, func, call, 13, __VA_ARGS__) \ - FMT_WRAP(Char, ReturnType, func, call, 14, __VA_ARGS__) \ - FMT_WRAP(Char, ReturnType, func, call, 15, __VA_ARGS__) -#endif // FMT_USE_VARIADIC_TEMPLATES - -/** -\rst -Defines a variadic function with the specified return type, function name -and argument types passed as variable arguments to this macro. - -**Example**:: - -void print_error(const char *file, int line, const char *format, -fmt::ArgList args) { -fmt::print("{}: {}: ", file, line); -fmt::print(format, args); -} -FMT_VARIADIC(void, print_error, const char *, int, const char *) - -``FMT_VARIADIC`` is used for compatibility with legacy C++ compilers that -don't implement variadic templates. You don't have to use this macro if -you don't need legacy compiler support and can use variadic templates -directly:: - -template <typename... Args> -void print_error(const char *file, int line, const char *format, -const Args & ... args) { -fmt::print("{}: {}: ", file, line); -fmt::print(format, args...); -} -\endrst -*/ -#define FMT_VARIADIC(ReturnType, func, ...) \ - FMT_VARIADIC_(char, ReturnType, func, return func, __VA_ARGS__) - -#define FMT_VARIADIC_W(ReturnType, func, ...) \ - FMT_VARIADIC_(wchar_t, ReturnType, func, return func, __VA_ARGS__) - -#define FMT_CAPTURE_ARG_(id, index) ::fmt::arg(#id, id) - -#define FMT_CAPTURE_ARG_W_(id, index) ::fmt::arg(L###id, id) - -/** -\rst -Convenient macro to capture the arguments' names and values into several -``fmt::arg(name, value)``. - -**Example**:: - -int x = 1, y = 2; -print("point: ({x}, {y})", FMT_CAPTURE(x, y)); -// same as: -// print("point: ({x}, {y})", arg("x", x), arg("y", y)); - -\endrst -*/ -#define FMT_CAPTURE(...) FMT_FOR_EACH(FMT_CAPTURE_ARG_, __VA_ARGS__) - -#define FMT_CAPTURE_W(...) FMT_FOR_EACH(FMT_CAPTURE_ARG_W_, __VA_ARGS__) - -namespace fmt -{ -FMT_VARIADIC(std::string, format, CStringRef) -FMT_VARIADIC_W(std::wstring, format, WCStringRef) -FMT_VARIADIC(void, print, CStringRef) -FMT_VARIADIC(void, print, std::FILE *, CStringRef) - -FMT_VARIADIC(void, print_colored, Color, CStringRef) -FMT_VARIADIC(std::string, sprintf, CStringRef) -FMT_VARIADIC_W(std::wstring, sprintf, WCStringRef) -FMT_VARIADIC(int, printf, CStringRef) -FMT_VARIADIC(int, fprintf, std::FILE *, CStringRef) - -#if FMT_USE_IOSTREAMS -/** -\rst -Prints formatted data to the stream *os*. - -**Example**:: - -print(cerr, "Don't {}!", "panic"); -\endrst -*/ -FMT_API void print(std::ostream &os, CStringRef format_str, ArgList args); -FMT_VARIADIC(void, print, std::ostream &, CStringRef) - -/** -\rst -Prints formatted data to the stream *os*. - -**Example**:: - -fprintf(cerr, "Don't %s!", "panic"); -\endrst -*/ -FMT_API int fprintf(std::ostream &os, CStringRef format_str, ArgList args); -FMT_VARIADIC(int, fprintf, std::ostream &, CStringRef) -#endif - -namespace internal -{ -template <typename Char> -inline bool is_name_start(Char c) -{ - return ('a' <= c && c <= 'z') || ('A' <= c && c <= 'Z') || '_' == c; -} - -// Parses an unsigned integer advancing s to the end of the parsed input. -// This function assumes that the first character of s is a digit. -template <typename Char> -unsigned parse_nonnegative_int(const Char *&s) -{ - assert('0' <= *s && *s <= '9'); - unsigned value = 0; - do - { - unsigned new_value = value * 10 + (*s++ - '0'); - // Check if value wrapped around. - if (new_value < value) - { - value = (std::numeric_limits<unsigned>::max)(); - break; - } - value = new_value; - } - while ('0' <= *s && *s <= '9'); - // Convert to unsigned to prevent a warning. - unsigned max_int = (std::numeric_limits<int>::max)(); - if (value > max_int) - FMT_THROW(FormatError("number is too big")); - return value; -} - -inline void require_numeric_argument(const Arg &arg, char spec) -{ - if (arg.type > Arg::LAST_NUMERIC_TYPE) - { - std::string message = - fmt::format("format specifier '{}' requires numeric argument", spec); - FMT_THROW(fmt::FormatError(message)); - } -} - -template <typename Char> -void check_sign(const Char *&s, const Arg &arg) -{ - char sign = static_cast<char>(*s); - require_numeric_argument(arg, sign); - if (arg.type == Arg::UINT || arg.type == Arg::ULONG_LONG) - { - FMT_THROW(FormatError(fmt::format( - "format specifier '{}' requires signed argument", sign))); - } - ++s; -} -} // namespace internal - -template <typename Char, typename AF> -inline internal::Arg BasicFormatter<Char, AF>::get_arg( - BasicStringRef<Char> arg_name, const char *&error) -{ - if (check_no_auto_index(error)) - { - map_.init(args()); - const internal::Arg *arg = map_.find(arg_name); - if (arg) - return *arg; - error = "argument not found"; - } - return internal::Arg(); -} - -template <typename Char, typename AF> -inline internal::Arg BasicFormatter<Char, AF>::parse_arg_index(const Char *&s) -{ - const char *error = 0; - internal::Arg arg = *s < '0' || *s > '9' ? - next_arg(error) : get_arg(internal::parse_nonnegative_int(s), error); - if (error) - { - FMT_THROW(FormatError( - *s != '}' && *s != ':' ? "invalid format string" : error)); - } - return arg; -} - -template <typename Char, typename AF> -inline internal::Arg BasicFormatter<Char, AF>::parse_arg_name(const Char *&s) -{ - assert(internal::is_name_start(*s)); - const Char *start = s; - Char c; - do - { - c = *++s; - } - while (internal::is_name_start(c) || ('0' <= c && c <= '9')); - const char *error = 0; - internal::Arg arg = get_arg(BasicStringRef<Char>(start, s - start), error); - if (error) - FMT_THROW(FormatError(error)); - return arg; -} - -template <typename Char, typename ArgFormatter> -const Char *BasicFormatter<Char, ArgFormatter>::format( - const Char *&format_str, const internal::Arg &arg) -{ - using internal::Arg; - const Char *s = format_str; - FormatSpec spec; - if (*s == ':') - { - if (arg.type == Arg::CUSTOM) - { - arg.custom.format(this, arg.custom.value, &s); - return s; - } - ++s; - // Parse fill and alignment. - if (Char c = *s) - { - const Char *p = s + 1; - spec.align_ = ALIGN_DEFAULT; - do - { - switch (*p) - { - case '<': - spec.align_ = ALIGN_LEFT; - break; - case '>': - spec.align_ = ALIGN_RIGHT; - break; - case '=': - spec.align_ = ALIGN_NUMERIC; - break; - case '^': - spec.align_ = ALIGN_CENTER; - break; - } - if (spec.align_ != ALIGN_DEFAULT) - { - if (p != s) - { - if (c == '}') break; - if (c == '{') - FMT_THROW(FormatError("invalid fill character '{'")); - s += 2; - spec.fill_ = c; - } - else ++s; - if (spec.align_ == ALIGN_NUMERIC) - require_numeric_argument(arg, '='); - break; - } - } - while (--p >= s); - } - - // Parse sign. - switch (*s) - { - case '+': - check_sign(s, arg); - spec.flags_ |= SIGN_FLAG | PLUS_FLAG; - break; - case '-': - check_sign(s, arg); - spec.flags_ |= MINUS_FLAG; - break; - case ' ': - check_sign(s, arg); - spec.flags_ |= SIGN_FLAG; - break; - } - - if (*s == '#') - { - require_numeric_argument(arg, '#'); - spec.flags_ |= HASH_FLAG; - ++s; - } - - // Parse zero flag. - if (*s == '0') - { - require_numeric_argument(arg, '0'); - spec.align_ = ALIGN_NUMERIC; - spec.fill_ = '0'; - ++s; - } - - // Parse width. - if ('0' <= *s && *s <= '9') - { - spec.width_ = internal::parse_nonnegative_int(s); - } - else if (*s == '{') - { - ++s; - Arg width_arg = internal::is_name_start(*s) ? - parse_arg_name(s) : parse_arg_index(s); - if (*s++ != '}') - FMT_THROW(FormatError("invalid format string")); - ULongLong value = 0; - switch (width_arg.type) - { - case Arg::INT: - if (width_arg.int_value < 0) - FMT_THROW(FormatError("negative width")); - value = width_arg.int_value; - break; - case Arg::UINT: - value = width_arg.uint_value; - break; - case Arg::LONG_LONG: - if (width_arg.long_long_value < 0) - FMT_THROW(FormatError("negative width")); - value = width_arg.long_long_value; - break; - case Arg::ULONG_LONG: - value = width_arg.ulong_long_value; - break; - default: - FMT_THROW(FormatError("width is not integer")); - } - if (value >(std::numeric_limits<int>::max)()) - FMT_THROW(FormatError("number is too big")); - spec.width_ = static_cast<int>(value); - } - - // Parse precision. - if (*s == '.') - { - ++s; - spec.precision_ = 0; - if ('0' <= *s && *s <= '9') - { - spec.precision_ = internal::parse_nonnegative_int(s); - } - else if (*s == '{') - { - ++s; - Arg precision_arg = internal::is_name_start(*s) ? - parse_arg_name(s) : parse_arg_index(s); - if (*s++ != '}') - FMT_THROW(FormatError("invalid format string")); - ULongLong value = 0; - switch (precision_arg.type) - { - case Arg::INT: - if (precision_arg.int_value < 0) - FMT_THROW(FormatError("negative precision")); - value = precision_arg.int_value; - break; - case Arg::UINT: - value = precision_arg.uint_value; - break; - case Arg::LONG_LONG: - if (precision_arg.long_long_value < 0) - FMT_THROW(FormatError("negative precision")); - value = precision_arg.long_long_value; - break; - case Arg::ULONG_LONG: - value = precision_arg.ulong_long_value; - break; - default: - FMT_THROW(FormatError("precision is not integer")); - } - if (value >(std::numeric_limits<int>::max)()) - FMT_THROW(FormatError("number is too big")); - spec.precision_ = static_cast<int>(value); - } - else - { - FMT_THROW(FormatError("missing precision specifier")); - } - if (arg.type <= Arg::LAST_INTEGER_TYPE || arg.type == Arg::POINTER) - { - FMT_THROW(FormatError( - fmt::format("precision not allowed in {} format specifier", - arg.type == Arg::POINTER ? "pointer" : "integer"))); - } - } - - // Parse type. - if (*s != '}' && *s) - spec.type_ = static_cast<char>(*s++); - } - - if (*s++ != '}') - FMT_THROW(FormatError("missing '}' in format string")); - - // Format argument. - ArgFormatter(*this, spec, s - 1).visit(arg); - return s; -} - -template <typename Char, typename AF> -void BasicFormatter<Char, AF>::format(BasicCStringRef<Char> format_str) -{ - const Char *s = format_str.c_str(); - const Char *start = s; - while (*s) - { - Char c = *s++; - if (c != '{' && c != '}') continue; - if (*s == c) - { - write(writer_, start, s); - start = ++s; - continue; - } - if (c == '}') - FMT_THROW(FormatError("unmatched '}' in format string")); - write(writer_, start, s - 1); - internal::Arg arg = internal::is_name_start(*s) ? - parse_arg_name(s) : parse_arg_index(s); - start = s = format(s, arg); - } - write(writer_, start, s); -} -} // namespace fmt - -#if FMT_USE_USER_DEFINED_LITERALS -namespace fmt -{ -namespace internal -{ - -template <typename Char> -struct UdlFormat -{ - const Char *str; - - template <typename... Args> - auto operator()(Args && ... args) const - -> decltype(format(str, std::forward<Args>(args)...)) - { - return format(str, std::forward<Args>(args)...); - } -}; - -template <typename Char> -struct UdlArg -{ - const Char *str; - - template <typename T> - NamedArg<Char> operator=(T &&value) const - { - return { str, std::forward<T>(value) }; - } -}; - -} // namespace internal - -inline namespace literals -{ - -/** -\rst -C++11 literal equivalent of :func:`fmt::format`. - -**Example**:: - -using namespace fmt::literals; -std::string message = "The answer is {}"_format(42); -\endrst -*/ -inline internal::UdlFormat<char> -operator"" _format(const char *s, std::size_t) -{ - return { s }; -} -inline internal::UdlFormat<wchar_t> -operator"" _format(const wchar_t *s, std::size_t) -{ - return { s }; -} - -/** -\rst -C++11 literal equivalent of :func:`fmt::arg`. - -**Example**:: - -using namespace fmt::literals; -print("Elapsed time: {s:.2f} seconds", "s"_a=1.23); -\endrst -*/ -inline internal::UdlArg<char> -operator"" _a(const char *s, std::size_t) -{ - return { s }; -} -inline internal::UdlArg<wchar_t> -operator"" _a(const wchar_t *s, std::size_t) -{ - return { s }; -} - -} // inline namespace literals -} // namespace fmt -#endif // FMT_USE_USER_DEFINED_LITERALS - -// Restore warnings. -#if FMT_GCC_VERSION >= 406 -# pragma GCC diagnostic pop -#endif - -#if defined(__clang__) && !defined(__INTEL_COMPILER) -# pragma clang diagnostic pop -#endif - -#ifdef FMT_HEADER_ONLY -# include "format.cc" -#endif - -#endif // FMT_FORMAT_H_ - diff --git a/src/spdlog/details/line_logger_fwd.h b/src/spdlog/details/line_logger_fwd.h deleted file mode 100644 index a8bc58ff50a85b47a0fb4b923c1042d8b270c360..0000000000000000000000000000000000000000 --- a/src/spdlog/details/line_logger_fwd.h +++ /dev/null @@ -1,78 +0,0 @@ -// -// Copyright(c) 2015 Gabi Melman. -// Distributed under the MIT License (http://opensource.org/licenses/MIT) -// -#pragma once - -#include <spdlog/common.h> -#include <spdlog/details/log_msg.h> - -#include <string> - -// Line logger class - aggregates operator<< calls to fast ostream -// and logs upon destruction - -namespace spdlog -{ - -// Forward declaration -class logger; - -namespace details -{ -class line_logger -{ -public: - line_logger(logger* callback_logger, level::level_enum msg_level, bool enabled); - - // No copy intended. Only move - line_logger(const line_logger& other) = delete; - line_logger& operator=(const line_logger&) = delete; - line_logger& operator=(line_logger&&) = delete; - - - line_logger(line_logger&& other); - - //Log the log message using the callback logger - ~line_logger(); - - // - // Support for format string with variadic args - // - - - void write(const char* what); - - template <typename... Args> - void write(const char* fmt, const Args&... args); - - // - // Support for operator<< - // - line_logger& operator<<(const char* what); - line_logger& operator<<(const std::string& what); - line_logger& operator<<(int what); - line_logger& operator<<(unsigned int what); - line_logger& operator<<(long what); - line_logger& operator<<(unsigned long what); - line_logger& operator<<(long long what); - line_logger& operator<<(unsigned long long what); - line_logger& operator<<(double what); - line_logger& operator<<(long double what); - line_logger& operator<<(float what); - line_logger& operator<<(char what); - //Support user types which implements operator<< - template<typename T> - line_logger& operator<<(const T& what); - - void disable(); - bool is_enabled() const; - -private: - logger* _callback_logger; - log_msg _log_msg; - bool _enabled; -}; -} //Namespace details -} // Namespace spdlog - diff --git a/src/spdlog/details/line_logger_impl.h b/src/spdlog/details/line_logger_impl.h deleted file mode 100644 index d61225afbcdddf4575786cfb3562cb0849712894..0000000000000000000000000000000000000000 --- a/src/spdlog/details/line_logger_impl.h +++ /dev/null @@ -1,185 +0,0 @@ -// -// Copyright(c) 2015 Gabi Melman. -// Distributed under the MIT License (http://opensource.org/licenses/MIT) -// -#pragma once -#include <type_traits> - -#include <spdlog/details/line_logger_fwd.h> -#include <spdlog/common.h> -#include <spdlog/logger.h> - -#include <string> -#include <utility> - -// Line logger class - aggregates operator<< calls to fast ostream -// and logs upon destruction - -inline spdlog::details::line_logger::line_logger(logger* callback_logger, level::level_enum msg_level, bool enabled): - _callback_logger(callback_logger), - _log_msg(msg_level), - _enabled(enabled) -{} - -inline spdlog::details::line_logger::line_logger(line_logger&& other) : - _callback_logger(other._callback_logger), - _log_msg(std::move(other._log_msg)), - _enabled(other._enabled) -{ - other.disable(); -} - -//Log the log message using the callback logger -inline spdlog::details::line_logger::~line_logger() -{ - if (_enabled) - { -#ifndef SPDLOG_NO_NAME - _log_msg.logger_name = _callback_logger->name(); -#endif -#ifndef SPDLOG_NO_DATETIME - _log_msg.time = os::now(); -#endif - -#ifndef SPDLOG_NO_THREAD_ID - _log_msg.thread_id = os::thread_id(); -#endif - _callback_logger->_log_msg(_log_msg); - } -} - -// -// Support for format string with variadic args -// - - -inline void spdlog::details::line_logger::write(const char* what) -{ - if (_enabled) - _log_msg.raw << what; -} - -template <typename... Args> -inline void spdlog::details::line_logger::write(const char* fmt, const Args&... args) -{ - if (!_enabled) - return; - try - { - _log_msg.raw.write(fmt, args...); - } - catch (const fmt::FormatError& e) - { - throw spdlog_ex(fmt::format("formatting error while processing format string '{}': {}", fmt, e.what())); - } -} - - -// -// Support for operator<< -// -inline spdlog::details::line_logger& spdlog::details::line_logger::operator<<(const char* what) -{ - if (_enabled) - _log_msg.raw << what; - return *this; -} - -inline spdlog::details::line_logger& spdlog::details::line_logger::operator<<(const std::string& what) -{ - if (_enabled) - _log_msg.raw << what; - return *this; -} - -inline spdlog::details::line_logger& spdlog::details::line_logger::operator<<(int what) -{ - if (_enabled) - _log_msg.raw << what; - return *this; -} - -inline spdlog::details::line_logger& spdlog::details::line_logger::operator<<(unsigned int what) -{ - if (_enabled) - _log_msg.raw << what; - return *this; -} - - -inline spdlog::details::line_logger& spdlog::details::line_logger::operator<<(long what) -{ - if (_enabled) - _log_msg.raw << what; - return *this; -} - -inline spdlog::details::line_logger& spdlog::details::line_logger::operator<<(unsigned long what) -{ - if (_enabled) - _log_msg.raw << what; - return *this; -} - -inline spdlog::details::line_logger& spdlog::details::line_logger::operator<<(long long what) -{ - if (_enabled) - _log_msg.raw << what; - return *this; -} - -inline spdlog::details::line_logger& spdlog::details::line_logger::operator<<(unsigned long long what) -{ - if (_enabled) - _log_msg.raw << what; - return *this; -} - -inline spdlog::details::line_logger& spdlog::details::line_logger::operator<<(double what) -{ - if (_enabled) - _log_msg.raw << what; - return *this; -} - -inline spdlog::details::line_logger& spdlog::details::line_logger::operator<<(long double what) -{ - if (_enabled) - _log_msg.raw << what; - return *this; -} - -inline spdlog::details::line_logger& spdlog::details::line_logger::operator<<(float what) -{ - if (_enabled) - _log_msg.raw << what; - return *this; -} - -inline spdlog::details::line_logger& spdlog::details::line_logger::operator<<(char what) -{ - if (_enabled) - _log_msg.raw << what; - return *this; -} - -//Support user types which implements operator<< -template<typename T> -inline spdlog::details::line_logger& spdlog::details::line_logger::operator<<(const T& what) -{ - if (_enabled) - _log_msg.raw.write("{}", what); - return *this; -} - - -inline void spdlog::details::line_logger::disable() -{ - _enabled = false; -} - -inline bool spdlog::details::line_logger::is_enabled() const -{ - return _enabled; -} - diff --git a/src/spdlog/details/log_msg.h b/src/spdlog/details/log_msg.h index 0d50b684888c9ef3a335fefd680230609307a812..a1e8b2ee239e402447c50ae9a225858f1ee667b9 100644 --- a/src/spdlog/details/log_msg.h +++ b/src/spdlog/details/log_msg.h @@ -5,77 +5,39 @@ #pragma once -#include <spdlog/common.h> -#include <spdlog/details/format.h> +#include "../common.h" +#include "../details/os.h" #include <string> #include <utility> -namespace spdlog -{ -namespace details -{ +namespace spdlog { namespace details { struct log_msg { log_msg() = default; - log_msg(level::level_enum l): - logger_name(), - level(l), - raw(), - formatted() {} - - - log_msg(const log_msg& other) : - logger_name(other.logger_name), - level(other.level), - time(other.time), - thread_id(other.thread_id) - { - if (other.raw.size()) - raw << fmt::BasicStringRef<char>(other.raw.data(), other.raw.size()); - if (other.formatted.size()) - formatted << fmt::BasicStringRef<char>(other.formatted.data(), other.formatted.size()); - } - - log_msg(log_msg&& other) : - logger_name(std::move(other.logger_name)), - level(other.level), - time(std::move(other.time)), - thread_id(other.thread_id), - raw(std::move(other.raw)), - formatted(std::move(other.formatted)) - { - other.clear(); - } - - log_msg& operator=(log_msg&& other) + log_msg(const std::string *loggers_name, level::level_enum lvl) + : logger_name(loggers_name) + , level(lvl) { - if (this == &other) - return *this; +#ifndef SPDLOG_NO_DATETIME + time = os::now(); +#endif - logger_name = std::move(other.logger_name); - level = other.level; - time = std::move(other.time); - thread_id = other.thread_id; - raw = std::move(other.raw); - formatted = std::move(other.formatted); - other.clear(); - return *this; +#ifndef SPDLOG_NO_THREAD_ID + thread_id = os::thread_id(); +#endif } - void clear() - { - level = level::off; - raw.clear(); - formatted.clear(); - } + log_msg(const log_msg &other) = delete; + log_msg &operator=(log_msg &&other) = delete; + log_msg(log_msg &&other) = delete; - std::string logger_name; + const std::string *logger_name{nullptr}; level::level_enum level; log_clock::time_point time; size_t thread_id; fmt::MemoryWriter raw; fmt::MemoryWriter formatted; + size_t msg_id{0}; }; -} -} +}} // namespace spdlog::details diff --git a/src/spdlog/details/logger_impl.h b/src/spdlog/details/logger_impl.h index 9f2f13d72bcd870fadee554ce3cf4303e15fbf9e..3b64543d3f20fcc5e8875b2630d7fcdb9068a462 100644 --- a/src/spdlog/details/logger_impl.h +++ b/src/spdlog/details/logger_impl.h @@ -5,259 +5,235 @@ #pragma once -#include <spdlog/logger.h> +#include "../logger.h" +#include "../sinks/stdout_sinks.h" #include <memory> #include <string> // create logger with given name, sinks and the default pattern formatter // all other ctors will call this one -template<class It> -inline spdlog::logger::logger(const std::string& logger_name, const It& begin, const It& end) : - _name(logger_name), - _sinks(begin, end), - _formatter(std::make_shared<pattern_formatter>("%+")) +template <class It> +inline spdlog::logger::logger(std::string logger_name, const It &begin, const It &end) + : _name(std::move(logger_name)) + , _sinks(begin, end) + , _formatter(std::make_shared<pattern_formatter>("%+")) + , _level(level::info) + , _flush_level(level::off) + , _last_err_time(0) + , _msg_counter(1) // message counter will start from 1. 0-message id will be reserved for controll messages { - - // no support under vs2013 for member initialization for std::atomic - _level = level::info; - _flush_level = level::off; + _err_handler = [this](const std::string &msg) { this->_default_err_handler(msg); }; } // ctor with sinks as init list -inline spdlog::logger::logger(const std::string& logger_name, sinks_init_list sinks_list) : - logger(logger_name, sinks_list.begin(), sinks_list.end()) {} - +inline spdlog::logger::logger(const std::string &logger_name, sinks_init_list sinks_list) + : logger(logger_name, sinks_list.begin(), sinks_list.end()) +{ +} // ctor with single sink -inline spdlog::logger::logger(const std::string& logger_name, spdlog::sink_ptr single_sink) : - logger(logger_name, +inline spdlog::logger::logger(const std::string &logger_name, spdlog::sink_ptr single_sink) + : logger(logger_name, {std::move(single_sink)}) { - single_sink -}) {} - +} inline spdlog::logger::~logger() = default; - inline void spdlog::logger::set_formatter(spdlog::formatter_ptr msg_formatter) { - _set_formatter(msg_formatter); + _set_formatter(std::move(msg_formatter)); } -inline void spdlog::logger::set_pattern(const std::string& pattern) +inline void spdlog::logger::set_pattern(const std::string &pattern, pattern_time_type pattern_time) { - _set_pattern(pattern); + _set_pattern(pattern, pattern_time); } -// -// log only if given level>=logger's log level -// - - -template <typename... Args> -inline spdlog::details::line_logger spdlog::logger::_log_if_enabled(level::level_enum lvl, const char* fmt, const Args&... args) +template <typename... Args> inline void spdlog::logger::log(level::level_enum lvl, const char *fmt, const Args &... args) { - bool msg_enabled = should_log(lvl); - details::line_logger l(this, lvl, msg_enabled); - l.write(fmt, args...); - return l; -} + if (!should_log(lvl)) + return; -inline spdlog::details::line_logger spdlog::logger::_log_if_enabled(level::level_enum lvl) -{ - return details::line_logger(this, lvl, should_log(lvl)); -} + try + { + details::log_msg log_msg(&_name, lvl); -template<typename T> -inline spdlog::details::line_logger spdlog::logger::_log_if_enabled(level::level_enum lvl, const T& msg) -{ - bool msg_enabled = should_log(lvl); - details::line_logger l(this, lvl, msg_enabled); - l << msg; - return l; +#if defined(SPDLOG_FMT_PRINTF) + fmt::printf(log_msg.raw, fmt, args...); +#else + log_msg.raw.write(fmt, args...); +#endif + _sink_it(log_msg); + } + catch (const std::exception &ex) + { + _err_handler(ex.what()); + } + catch (...) + { + _err_handler("Unknown exception in logger " + _name); + throw; + } } -// -// logger.info(cppformat_string, arg1, arg2, arg3, ...) call style -// -template <typename... Args> -inline spdlog::details::line_logger spdlog::logger::trace(const char* fmt, const Args&... args) +template <typename... Args> inline void spdlog::logger::log(level::level_enum lvl, const char *msg) { - return _log_if_enabled(level::trace, fmt, args...); + if (!should_log(lvl)) + return; + try + { + details::log_msg log_msg(&_name, lvl); + log_msg.raw << msg; + _sink_it(log_msg); + } + catch (const std::exception &ex) + { + _err_handler(ex.what()); + } + catch (...) + { + _err_handler("Unknown exception in logger " + _name); + throw; + } } -template <typename... Args> -inline spdlog::details::line_logger spdlog::logger::debug(const char* fmt, const Args&... args) +template <typename T> inline void spdlog::logger::log(level::level_enum lvl, const T &msg) { - return _log_if_enabled(level::debug, fmt, args...); + if (!should_log(lvl)) + return; + try + { + details::log_msg log_msg(&_name, lvl); + log_msg.raw << msg; + _sink_it(log_msg); + } + catch (const std::exception &ex) + { + _err_handler(ex.what()); + } + catch (...) + { + _err_handler("Unknown exception in logger " + _name); + throw; + } } -template <typename... Args> -inline spdlog::details::line_logger spdlog::logger::info(const char* fmt, const Args&... args) +template <typename Arg1, typename... Args> inline void spdlog::logger::trace(const char *fmt, const Arg1 &arg1, const Args &... args) { - return _log_if_enabled(level::info, fmt, args...); + log(level::trace, fmt, arg1, args...); } -template <typename... Args> -inline spdlog::details::line_logger spdlog::logger::notice(const char* fmt, const Args&... args) +template <typename Arg1, typename... Args> inline void spdlog::logger::debug(const char *fmt, const Arg1 &arg1, const Args &... args) { - return _log_if_enabled(level::notice, fmt, args...); + log(level::debug, fmt, arg1, args...); } -template <typename... Args> -inline spdlog::details::line_logger spdlog::logger::warn(const char* fmt, const Args&... args) +template <typename Arg1, typename... Args> inline void spdlog::logger::info(const char *fmt, const Arg1 &arg1, const Args &... args) { - return _log_if_enabled(level::warn, fmt, args...); + log(level::info, fmt, arg1, args...); } -template <typename... Args> -inline spdlog::details::line_logger spdlog::logger::error(const char* fmt, const Args&... args) +template <typename Arg1, typename... Args> inline void spdlog::logger::warn(const char *fmt, const Arg1 &arg1, const Args &... args) { - return _log_if_enabled(level::err, fmt, args...); + log(level::warn, fmt, arg1, args...); } -template <typename... Args> -inline spdlog::details::line_logger spdlog::logger::critical(const char* fmt, const Args&... args) +template <typename Arg1, typename... Args> inline void spdlog::logger::error(const char *fmt, const Arg1 &arg1, const Args &... args) { - return _log_if_enabled(level::critical, fmt, args...); + log(level::err, fmt, arg1, args...); } -template <typename... Args> -inline spdlog::details::line_logger spdlog::logger::alert(const char* fmt, const Args&... args) +template <typename Arg1, typename... Args> inline void spdlog::logger::critical(const char *fmt, const Arg1 &arg1, const Args &... args) { - return _log_if_enabled(level::alert, fmt, args...); + log(level::critical, fmt, arg1, args...); } -template <typename... Args> -inline spdlog::details::line_logger spdlog::logger::emerg(const char* fmt, const Args&... args) +template <typename T> inline void spdlog::logger::trace(const T &msg) { - return _log_if_enabled(level::emerg, fmt, args...); + log(level::trace, msg); } -// -// logger.info(msg) << ".." call style -// -template<typename T> -inline spdlog::details::line_logger spdlog::logger::trace(const T& msg) +template <typename T> inline void spdlog::logger::debug(const T &msg) { - return _log_if_enabled(level::trace, msg); + log(level::debug, msg); } -template<typename T> -inline spdlog::details::line_logger spdlog::logger::debug(const T& msg) +template <typename T> inline void spdlog::logger::info(const T &msg) { - return _log_if_enabled(level::debug, msg); + log(level::info, msg); } - -template<typename T> -inline spdlog::details::line_logger spdlog::logger::info(const T& msg) +template <typename T> inline void spdlog::logger::warn(const T &msg) { - return _log_if_enabled(level::info, msg); + log(level::warn, msg); } -template<typename T> -inline spdlog::details::line_logger spdlog::logger::notice(const T& msg) +template <typename T> inline void spdlog::logger::error(const T &msg) { - return _log_if_enabled(level::notice, msg); + log(level::err, msg); } -template<typename T> -inline spdlog::details::line_logger spdlog::logger::warn(const T& msg) +template <typename T> inline void spdlog::logger::critical(const T &msg) { - return _log_if_enabled(level::warn, msg); + log(level::critical, msg); } -template<typename T> -inline spdlog::details::line_logger spdlog::logger::error(const T& msg) -{ - return _log_if_enabled(level::err, msg); -} +#ifdef SPDLOG_WCHAR_TO_UTF8_SUPPORT +#include <codecvt> +#include <locale> -template<typename T> -inline spdlog::details::line_logger spdlog::logger::critical(const T& msg) +template <typename... Args> inline void spdlog::logger::log(level::level_enum lvl, const wchar_t *msg) { - return _log_if_enabled(level::critical, msg); -} + std::wstring_convert<std::codecvt_utf8<wchar_t>> conv; -template<typename T> -inline spdlog::details::line_logger spdlog::logger::alert(const T& msg) -{ - return _log_if_enabled(level::alert, msg); + log(lvl, conv.to_bytes(msg)); } -template<typename T> -inline spdlog::details::line_logger spdlog::logger::emerg(const T& msg) +template <typename... Args> inline void spdlog::logger::log(level::level_enum lvl, const wchar_t *fmt, const Args &... args) { - return _log_if_enabled(level::emerg, msg); -} - - - + fmt::WMemoryWriter wWriter; -// -// logger.info() << ".." call style -// -inline spdlog::details::line_logger spdlog::logger::trace() -{ - return _log_if_enabled(level::trace); + wWriter.write(fmt, args...); + log(lvl, wWriter.c_str()); } -inline spdlog::details::line_logger spdlog::logger::debug() +template <typename... Args> inline void spdlog::logger::trace(const wchar_t *fmt, const Args &... args) { - return _log_if_enabled(level::debug); + log(level::trace, fmt, args...); } -inline spdlog::details::line_logger spdlog::logger::info() +template <typename... Args> inline void spdlog::logger::debug(const wchar_t *fmt, const Args &... args) { - return _log_if_enabled(level::info); + log(level::debug, fmt, args...); } -inline spdlog::details::line_logger spdlog::logger::notice() +template <typename... Args> inline void spdlog::logger::info(const wchar_t *fmt, const Args &... args) { - return _log_if_enabled(level::notice); + log(level::info, fmt, args...); } -inline spdlog::details::line_logger spdlog::logger::warn() +template <typename... Args> inline void spdlog::logger::warn(const wchar_t *fmt, const Args &... args) { - return _log_if_enabled(level::warn); + log(level::warn, fmt, args...); } -inline spdlog::details::line_logger spdlog::logger::error() +template <typename... Args> inline void spdlog::logger::error(const wchar_t *fmt, const Args &... args) { - return _log_if_enabled(level::err); + log(level::err, fmt, args...); } -inline spdlog::details::line_logger spdlog::logger::critical() +template <typename... Args> inline void spdlog::logger::critical(const wchar_t *fmt, const Args &... args) { - return _log_if_enabled(level::critical); + log(level::critical, fmt, args...); } -inline spdlog::details::line_logger spdlog::logger::alert() -{ - return _log_if_enabled(level::alert); -} - -inline spdlog::details::line_logger spdlog::logger::emerg() -{ - return _log_if_enabled(level::emerg); -} - - -// always log, no matter what is the actual logger's log level -template <typename... Args> -inline spdlog::details::line_logger spdlog::logger::force_log(level::level_enum lvl, const char* fmt, const Args&... args) -{ - details::line_logger l(this, lvl, true); - l.write(fmt, args...); - return l; -} +#endif // SPDLOG_WCHAR_TO_UTF8_SUPPORT // // name and level // -inline const std::string& spdlog::logger::name() const +inline const std::string &spdlog::logger::name() const { return _name; } @@ -267,6 +243,16 @@ inline void spdlog::logger::set_level(spdlog::level::level_enum log_level) _level.store(log_level); } +inline void spdlog::logger::set_error_handler(spdlog::log_err_handler err_handler) +{ + _err_handler = std::move(err_handler); +} + +inline spdlog::log_err_handler spdlog::logger::error_handler() +{ + return _err_handler; +} + inline void spdlog::logger::flush_on(level::level_enum log_level) { _flush_level.store(log_level); @@ -285,28 +271,66 @@ inline bool spdlog::logger::should_log(spdlog::level::level_enum msg_level) cons // // protected virtual called at end of each user log call (if enabled) by the line_logger // -inline void spdlog::logger::_log_msg(details::log_msg& msg) +inline void spdlog::logger::_sink_it(details::log_msg &msg) { +#if defined(SPDLOG_ENABLE_MESSAGE_COUNTER) + _incr_msg_counter(msg); +#endif _formatter->format(msg); for (auto &sink : _sinks) - sink->log(msg); - - const auto flush_level = _flush_level.load(std::memory_order_relaxed); - if (msg.level >= flush_level) + { + if (sink->should_log(msg.level)) + { + sink->log(msg); + } + } + + if (_should_flush_on(msg)) flush(); } -inline void spdlog::logger::_set_pattern(const std::string& pattern) +inline void spdlog::logger::_set_pattern(const std::string &pattern, pattern_time_type pattern_time) { - _formatter = std::make_shared<pattern_formatter>(pattern); + _formatter = std::make_shared<pattern_formatter>(pattern, pattern_time); } + inline void spdlog::logger::_set_formatter(formatter_ptr msg_formatter) { - _formatter = msg_formatter; + _formatter = std::move(msg_formatter); } inline void spdlog::logger::flush() { - for (auto& sink : _sinks) + for (auto &sink : _sinks) sink->flush(); } + +inline void spdlog::logger::_default_err_handler(const std::string &msg) +{ + auto now = time(nullptr); + if (now - _last_err_time < 60) + return; + auto tm_time = details::os::localtime(now); + char date_buf[100]; + std::strftime(date_buf, sizeof(date_buf), "%Y-%m-%d %H:%M:%S", &tm_time); + details::log_msg err_msg; + err_msg.formatted.write("[*** LOG ERROR ***] [{}] [{}] [{}]{}", name(), msg, date_buf, details::os::default_eol); + sinks::stderr_sink_mt::instance()->log(err_msg); + _last_err_time = now; +} + +inline bool spdlog::logger::_should_flush_on(const details::log_msg &msg) +{ + const auto flush_level = _flush_level.load(std::memory_order_relaxed); + return (msg.level >= flush_level) && (msg.level != level::off); +} + +inline void spdlog::logger::_incr_msg_counter(details::log_msg &msg) +{ + msg.msg_id = _msg_counter.fetch_add(1, std::memory_order_relaxed); +} + +inline const std::vector<spdlog::sink_ptr> &spdlog::logger::sinks() const +{ + return _sinks; +} diff --git a/src/spdlog/details/mpmc_bounded_q.h b/src/spdlog/details/mpmc_bounded_q.h index ad14d6f259612ffc127cd443e3ee378411797deb..567f292f425cf4f51d0c0d65c797d3cd18662031 100644 --- a/src/spdlog/details/mpmc_bounded_q.h +++ b/src/spdlog/details/mpmc_bounded_q.h @@ -43,28 +43,25 @@ Distributed under the MIT License (http://opensource.org/licenses/MIT) #pragma once -#include <spdlog/common.h> +#include "../common.h" #include <atomic> #include <utility> -namespace spdlog -{ -namespace details -{ +namespace spdlog { namespace details { -template<typename T> -class mpmc_bounded_queue +template <typename T> class mpmc_bounded_queue { public: - using item_type = T; - mpmc_bounded_queue(size_t buffer_size) - : buffer_(new cell_t [buffer_size]), - buffer_mask_(buffer_size - 1) + + explicit mpmc_bounded_queue(size_t buffer_size) + : max_size_(buffer_size) + , buffer_(new cell_t[buffer_size]) + , buffer_mask_(buffer_size - 1) { - //queue size must be power of two - if(!((buffer_size >= 2) && ((buffer_size & (buffer_size - 1)) == 0))) + // queue size must be power of two + if (!((buffer_size >= 2) && ((buffer_size & (buffer_size - 1)) == 0))) throw spdlog_ex("async logger queue size must be power of two"); for (size_t i = 0; i != buffer_size; i += 1) @@ -75,19 +72,21 @@ public: ~mpmc_bounded_queue() { - delete [] buffer_; + delete[] buffer_; } + mpmc_bounded_queue(mpmc_bounded_queue const &) = delete; + void operator=(mpmc_bounded_queue const &) = delete; - bool enqueue(T&& data) + bool enqueue(T &&data) { - cell_t* cell; + cell_t *cell; size_t pos = enqueue_pos_.load(std::memory_order_relaxed); for (;;) { cell = &buffer_[pos & buffer_mask_]; size_t seq = cell->sequence_.load(std::memory_order_acquire); - intptr_t dif = (intptr_t)seq - (intptr_t)pos; + intptr_t dif = static_cast<intptr_t>(seq) - static_cast<intptr_t>(pos); if (dif == 0) { if (enqueue_pos_.compare_exchange_weak(pos, pos + 1, std::memory_order_relaxed)) @@ -107,16 +106,15 @@ public: return true; } - bool dequeue(T& data) + bool dequeue(T &data) { - cell_t* cell; + cell_t *cell; size_t pos = dequeue_pos_.load(std::memory_order_relaxed); for (;;) { cell = &buffer_[pos & buffer_mask_]; - size_t seq = - cell->sequence_.load(std::memory_order_acquire); - intptr_t dif = (intptr_t)seq - (intptr_t)(pos + 1); + size_t seq = cell->sequence_.load(std::memory_order_acquire); + intptr_t dif = static_cast<intptr_t>(seq) - static_cast<intptr_t>(pos + 1); if (dif == 0) { if (dequeue_pos_.compare_exchange_weak(pos, pos + 1, std::memory_order_relaxed)) @@ -132,28 +130,39 @@ public: return true; } + bool is_empty() + { + size_t front, front1, back; + // try to take a consistent snapshot of front/tail. + do + { + front = enqueue_pos_.load(std::memory_order_acquire); + back = dequeue_pos_.load(std::memory_order_acquire); + front1 = enqueue_pos_.load(std::memory_order_relaxed); + } while (front != front1); + return back == front; + } + private: struct cell_t { - std::atomic<size_t> sequence_; - T data_; + std::atomic<size_t> sequence_; + T data_; }; - static size_t const cacheline_size = 64; - typedef char cacheline_pad_t [cacheline_size]; + size_t const max_size_; - cacheline_pad_t pad0_; - cell_t* const buffer_; - size_t const buffer_mask_; - cacheline_pad_t pad1_; - std::atomic<size_t> enqueue_pos_; - cacheline_pad_t pad2_; - std::atomic<size_t> dequeue_pos_; - cacheline_pad_t pad3_; + static size_t const cacheline_size = 64; + using cacheline_pad_t = char[cacheline_size]; - mpmc_bounded_queue(mpmc_bounded_queue const&); - void operator = (mpmc_bounded_queue const&); + cacheline_pad_t pad0_; + cell_t *const buffer_; + size_t const buffer_mask_; + cacheline_pad_t pad1_; + std::atomic<size_t> enqueue_pos_; + cacheline_pad_t pad2_; + std::atomic<size_t> dequeue_pos_; + cacheline_pad_t pad3_; }; -} // ns details -} // ns spdlog +}} // namespace spdlog::details diff --git a/src/spdlog/details/null_mutex.h b/src/spdlog/details/null_mutex.h index 67b0aeee004c3120d33d54e546a18854ac4d0ecc..5f4ce46a1162ceda61fb805dc447044a3bb33f42 100644 --- a/src/spdlog/details/null_mutex.h +++ b/src/spdlog/details/null_mutex.h @@ -8,10 +8,7 @@ #include <atomic> // null, no cost dummy "mutex" and dummy "atomic" int -namespace spdlog -{ -namespace details -{ +namespace spdlog { namespace details { struct null_mutex { void lock() {} @@ -27,8 +24,10 @@ struct null_atomic_int int value; null_atomic_int() = default; - null_atomic_int(int val):value(val) - {} + explicit null_atomic_int(int val) + : value(val) + { + } int load(std::memory_order) const { @@ -41,5 +40,4 @@ struct null_atomic_int } }; -} -} +}} // namespace spdlog::details diff --git a/src/spdlog/details/os.h b/src/spdlog/details/os.h index fda4d0a0c15f0b249561627d02351f34c525f0c0..271348da41505310086808e956a88b2c74c34c51 100644 --- a/src/spdlog/details/os.h +++ b/src/spdlog/details/os.h @@ -4,43 +4,56 @@ // #pragma once -#include <spdlog/common.h> +#include "../common.h" +#include <algorithm> +#include <chrono> #include <cstdio> +#include <cstdlib> +#include <cstring> #include <ctime> #include <functional> #include <string> +#include <sys/stat.h> +#include <sys/types.h> +#include <thread> #ifdef _WIN32 #ifndef NOMINMAX -#define NOMINMAX //prevent windows redefining min/max +#define NOMINMAX // prevent windows redefining min/max #endif #ifndef WIN32_LEAN_AND_MEAN #define WIN32_LEAN_AND_MEAN #endif +#include <io.h> // _get_osfhandle and _isatty support +#include <process.h> // _get_pid support #include <windows.h> #ifdef __MINGW32__ #include <share.h> #endif -#elif __linux__ -#include <sys/syscall.h> //Use gettid() syscall under linux to get thread id -#include <sys/stat.h> +#else // unix + +#include <fcntl.h> #include <unistd.h> -#include <chrono> -#else -#include <thread> + +#ifdef __linux__ +#include <sys/syscall.h> //Use gettid() syscall under linux to get thread id + +#elif __FreeBSD__ +#include <sys/thr.h> //Use thr_self() syscall under FreeBSD to get thread id #endif -namespace spdlog -{ -namespace details -{ -namespace os -{ +#endif // unix + +#ifndef __has_feature // Clang - feature checking macros. +#define __has_feature(x) 0 // Compatibility with non-clang compilers. +#endif + +namespace spdlog { namespace details { namespace os { inline spdlog::log_clock::time_point now() { @@ -49,14 +62,11 @@ inline spdlog::log_clock::time_point now() timespec ts; ::clock_gettime(CLOCK_REALTIME_COARSE, &ts); return std::chrono::time_point<log_clock, typename log_clock::duration>( - std::chrono::duration_cast<typename log_clock::duration>( - std::chrono::seconds(ts.tv_sec) + std::chrono::nanoseconds(ts.tv_nsec))); - + std::chrono::duration_cast<typename log_clock::duration>(std::chrono::seconds(ts.tv_sec) + std::chrono::nanoseconds(ts.tv_nsec))); #else return log_clock::now(); #endif - } inline std::tm localtime(const std::time_t &time_tt) { @@ -77,7 +87,6 @@ inline std::tm localtime() return localtime(now_t); } - inline std::tm gmtime(const std::time_t &time_tt) { @@ -96,24 +105,19 @@ inline std::tm gmtime() std::time_t now_t = time(nullptr); return gmtime(now_t); } -inline bool operator==(const std::tm& tm1, const std::tm& tm2) +inline bool operator==(const std::tm &tm1, const std::tm &tm2) { - return (tm1.tm_sec == tm2.tm_sec && - tm1.tm_min == tm2.tm_min && - tm1.tm_hour == tm2.tm_hour && - tm1.tm_mday == tm2.tm_mday && - tm1.tm_mon == tm2.tm_mon && - tm1.tm_year == tm2.tm_year && - tm1.tm_isdst == tm2.tm_isdst); + return (tm1.tm_sec == tm2.tm_sec && tm1.tm_min == tm2.tm_min && tm1.tm_hour == tm2.tm_hour && tm1.tm_mday == tm2.tm_mday && + tm1.tm_mon == tm2.tm_mon && tm1.tm_year == tm2.tm_year && tm1.tm_isdst == tm2.tm_isdst); } -inline bool operator!=(const std::tm& tm1, const std::tm& tm2) +inline bool operator!=(const std::tm &tm1, const std::tm &tm2) { return !(tm1 == tm2); } // eol definition -#if !defined (SPDLOG_EOL) +#if !defined(SPDLOG_EOL) #ifdef _WIN32 #define SPDLOG_EOL "\r\n" #else @@ -121,13 +125,33 @@ inline bool operator!=(const std::tm& tm1, const std::tm& tm2) #endif #endif -SPDLOG_CONSTEXPR static const char* eol = SPDLOG_EOL; -SPDLOG_CONSTEXPR static int eol_size = sizeof(SPDLOG_EOL) - 1; +SPDLOG_CONSTEXPR static const char *default_eol = SPDLOG_EOL; +// folder separator +#ifdef _WIN32 +SPDLOG_CONSTEXPR static const char folder_sep = '\\'; +#else +SPDLOG_CONSTEXPR static const char folder_sep = '/'; +#endif +inline void prevent_child_fd(FILE *f) +{ -//fopen_s on non windows for writing -inline int fopen_s(FILE** fp, const filename_t& filename, const filename_t& mode) +#ifdef _WIN32 +#if !defined(__cplusplus_winrt) + auto file_handle = (HANDLE)_get_osfhandle(_fileno(f)); + if (!::SetHandleInformation(file_handle, HANDLE_FLAG_INHERIT, 0)) + throw spdlog_ex("SetHandleInformation failed", errno); +#endif +#else + auto fd = fileno(f); + if (fcntl(fd, F_SETFD, FD_CLOEXEC) == -1) + throw spdlog_ex("fcntl with FD_CLOEXEC failed", errno); +#endif +} + +// fopen_s on non windows for writing +inline bool fopen_s(FILE **fp, const filename_t &filename, const filename_t &mode) { #ifdef _WIN32 #ifdef SPDLOG_WCHAR_FILENAMES @@ -135,11 +159,15 @@ inline int fopen_s(FILE** fp, const filename_t& filename, const filename_t& mode #else *fp = _fsopen((filename.c_str()), mode.c_str(), _SH_DENYWR); #endif - return *fp == nullptr; -#else +#else // unix *fp = fopen((filename.c_str()), mode.c_str()); - return *fp == nullptr; #endif + +#ifdef SPDLOG_PREVENT_CHILD_FD + if (*fp != nullptr) + prevent_child_fd(*fp); +#endif + return *fp == nullptr; } inline int remove(const filename_t &filename) @@ -151,7 +179,7 @@ inline int remove(const filename_t &filename) #endif } -inline int rename(const filename_t& filename1, const filename_t& filename2) +inline int rename(const filename_t &filename1, const filename_t &filename2) { #if defined(_WIN32) && defined(SPDLOG_WCHAR_FILENAMES) return _wrename(filename1.c_str(), filename2.c_str()); @@ -160,9 +188,8 @@ inline int rename(const filename_t& filename1, const filename_t& filename2) #endif } - -//Return if file exists -inline bool file_exists(const filename_t& filename) +// Return if file exists +inline bool file_exists(const filename_t &filename) { #ifdef _WIN32 #ifdef SPDLOG_WCHAR_FILENAMES @@ -171,23 +198,48 @@ inline bool file_exists(const filename_t& filename) auto attribs = GetFileAttributesA(filename.c_str()); #endif return (attribs != INVALID_FILE_ATTRIBUTES && !(attribs & FILE_ATTRIBUTE_DIRECTORY)); -#elif __linux__ +#else // common linux/unix all have the stat system call struct stat buffer; - return (stat (filename.c_str(), &buffer) == 0); -#else - auto *file = fopen(filename.c_str(), "r"); - if (file != nullptr) - { - fclose(file); - return true; - } - return false; + return (stat(filename.c_str(), &buffer) == 0); +#endif +} +// Return file size according to open FILE* object +inline size_t filesize(FILE *f) +{ + if (f == nullptr) + throw spdlog_ex("Failed getting file size. fd is null"); +#if defined(_WIN32) && !defined(__CYGWIN__) + int fd = _fileno(f); +#if _WIN64 // 64 bits + struct _stat64 st; + if (_fstat64(fd, &st) == 0) + return st.st_size; + +#else // windows 32 bits + long ret = _filelength(fd); + if (ret >= 0) + return static_cast<size_t>(ret); +#endif + +#else // unix + int fd = fileno(f); + // 64 bits(but not in osx or cygwin, where fstat64 is deprecated) +#if !defined(__FreeBSD__) && !defined(__APPLE__) && (defined(__x86_64__) || defined(__ppc64__)) && !defined(__CYGWIN__) + struct stat64 st; + if (fstat64(fd, &st) == 0) + return static_cast<size_t>(st.st_size); +#else // unix 32 bits or cygwin + struct stat st; + if (fstat(fd, &st) == 0) + return static_cast<size_t>(st.st_size); +#endif #endif + throw spdlog_ex("Failed getting file size from fd", errno); } -//Return utc offset in minutes or throw spdlog_ex on failure -inline int utc_minutes_offset(const std::tm& tm = details::os::localtime()) +// Return utc offset in minutes or throw spdlog_ex on failure +inline int utc_minutes_offset(const std::tm &tm = details::os::localtime()) { #ifdef _WIN32 @@ -199,7 +251,7 @@ inline int utc_minutes_offset(const std::tm& tm = details::os::localtime()) auto rv = GetDynamicTimeZoneInformation(&tzinfo); #endif if (rv == TIME_ZONE_ID_INVALID) - throw spdlog::spdlog_ex("Failed getting timezone info. Last error: " + std::to_string(GetLastError())); + throw spdlog::spdlog_ex("Failed getting timezone info. ", errno); int offset = -tzinfo.Bias; if (tm.tm_isdst) @@ -208,46 +260,189 @@ inline int utc_minutes_offset(const std::tm& tm = details::os::localtime()) offset -= tzinfo.StandardBias; return offset; #else - return static_cast<int>(tm.tm_gmtoff / 60); + +#if defined(sun) || defined(__sun) + // 'tm_gmtoff' field is BSD extension and it's missing on SunOS/Solaris + struct helper + { + static long int calculate_gmt_offset(const std::tm &localtm = details::os::localtime(), const std::tm &gmtm = details::os::gmtime()) + { + int local_year = localtm.tm_year + (1900 - 1); + int gmt_year = gmtm.tm_year + (1900 - 1); + + long int days = ( + // difference in day of year + localtm.tm_yday - + gmtm.tm_yday + + // + intervening leap days + + ((local_year >> 2) - (gmt_year >> 2)) - (local_year / 100 - gmt_year / 100) + + ((local_year / 100 >> 2) - (gmt_year / 100 >> 2)) + + // + difference in years * 365 */ + + (long int)(local_year - gmt_year) * 365); + + long int hours = (24 * days) + (localtm.tm_hour - gmtm.tm_hour); + long int mins = (60 * hours) + (localtm.tm_min - gmtm.tm_min); + long int secs = (60 * mins) + (localtm.tm_sec - gmtm.tm_sec); + + return secs; + } + }; + + auto offset_seconds = helper::calculate_gmt_offset(tm); +#else + auto offset_seconds = tm.tm_gmtoff; +#endif + + return static_cast<int>(offset_seconds / 60); #endif } -//Return current thread id as size_t -//It exists because the std::this_thread::get_id() is much slower(espcially under VS 2013) -inline size_t thread_id() +// Return current thread id as size_t +// It exists because the std::this_thread::get_id() is much slower(especially under VS 2013) +inline size_t _thread_id() { #ifdef _WIN32 - return static_cast<size_t>(::GetCurrentThreadId()); + return static_cast<size_t>(::GetCurrentThreadId()); #elif __linux__ -# if defined(__ANDROID__) && defined(__ANDROID_API__) && (__ANDROID_API__ < 21) -# define SYS_gettid __NR_gettid -# endif - return static_cast<size_t>(syscall(SYS_gettid)); -#else //Default to standard C++11 (OSX and other Unix) +#if defined(__ANDROID__) && defined(__ANDROID_API__) && (__ANDROID_API__ < 21) +#define SYS_gettid __NR_gettid +#endif + return static_cast<size_t>(syscall(SYS_gettid)); +#elif __FreeBSD__ + long tid; + thr_self(&tid); + return static_cast<size_t>(tid); +#elif __APPLE__ + uint64_t tid; + pthread_threadid_np(nullptr, &tid); + return static_cast<size_t>(tid); +#else // Default to standard C++11 (other Unix) return static_cast<size_t>(std::hash<std::thread::id>()(std::this_thread::get_id())); #endif +} +// Return current thread id as size_t (from thread local storage) +inline size_t thread_id() +{ +#if defined(SPDLOG_DISABLE_TID_CACHING) || (defined(_MSC_VER) && (_MSC_VER < 1900)) || defined(__cplusplus_winrt) || \ + (defined(__clang__) && !__has_feature(cxx_thread_local)) + return _thread_id(); +#else // cache thread id in tls + static thread_local const size_t tid = _thread_id(); + return tid; +#endif } +// This is avoid msvc issue in sleep_for that happens if the clock changes. +// See https://github.com/gabime/spdlog/issues/609 +inline void sleep_for_millis(int milliseconds) +{ +#if defined(_WIN32) + ::Sleep(milliseconds); +#else + std::this_thread::sleep_for(std::chrono::milliseconds(milliseconds)); +#endif +} // wchar support for windows file names (SPDLOG_WCHAR_FILENAMES must be defined) #if defined(_WIN32) && defined(SPDLOG_WCHAR_FILENAMES) -#define SPDLOG_FILENAME_T(s) L ## s -inline std::string filename_to_str(const filename_t& filename) +#define SPDLOG_FILENAME_T(s) L##s +inline std::string filename_to_str(const filename_t &filename) { std::wstring_convert<std::codecvt_utf8<wchar_t>, wchar_t> c; return c.to_bytes(filename); } #else #define SPDLOG_FILENAME_T(s) s -inline std::string filename_to_str(const filename_t& filename) +inline std::string filename_to_str(const filename_t &filename) { return filename; } #endif -} //os -} //details -} //spdlog +inline std::string errno_to_string(char[256], char *res) +{ + return std::string(res); +} + +inline std::string errno_to_string(char buf[256], int res) +{ + if (res == 0) + { + return std::string(buf); + } + return "Unknown error"; +} + +// Return errno string (thread safe) +inline std::string errno_str(int err_num) +{ + char buf[256]; + SPDLOG_CONSTEXPR auto buf_size = sizeof(buf); +#ifdef _WIN32 + if (strerror_s(buf, buf_size, err_num) == 0) + return std::string(buf); + else + return "Unknown error"; + +#elif defined(__FreeBSD__) || defined(__APPLE__) || defined(ANDROID) || defined(__SUNPRO_CC) || \ + ((_POSIX_C_SOURCE >= 200112L) && !defined(_GNU_SOURCE)) // posix version + if (strerror_r(err_num, buf, buf_size) == 0) + return std::string(buf); + else + return "Unknown error"; + +#else // gnu version (might not use the given buf, so its retval pointer must be used) + auto err = strerror_r(err_num, buf, buf_size); // let compiler choose type + return errno_to_string(buf, err); // use overloading to select correct stringify function +#endif +} + +inline int pid() +{ + +#ifdef _WIN32 + return static_cast<int>(::GetCurrentProcessId()); +#else + return static_cast<int>(::getpid()); +#endif +} + +// Determine if the terminal supports colors +// Source: https://github.com/agauniyal/rang/ +inline bool is_color_terminal() +{ +#ifdef _WIN32 + return true; +#else + static constexpr const char *Terms[] = { + "ansi", "color", "console", "cygwin", "gnome", "konsole", "kterm", "linux", "msys", "putty", "rxvt", "screen", "vt100", "xterm"}; + + const char *env_p = std::getenv("TERM"); + if (env_p == nullptr) + { + return false; + } + + static const bool result = + std::any_of(std::begin(Terms), std::end(Terms), [&](const char *term) { return std::strstr(env_p, term) != nullptr; }); + return result; +#endif +} + +// Detrmine if the terminal attached +// Source: https://github.com/agauniyal/rang/ +inline bool in_terminal(FILE *file) +{ + +#ifdef _WIN32 + return _isatty(_fileno(file)) != 0; +#else + return isatty(fileno(file)) != 0; +#endif +} +}}} // namespace spdlog::details::os diff --git a/src/spdlog/details/pattern_formatter_impl.h b/src/spdlog/details/pattern_formatter_impl.h index 313bfc8503d5e13496701bac61449064f2011d90..670538d5e65fd069295cbd28be7e7db4fed9f33f 100644 --- a/src/spdlog/details/pattern_formatter_impl.h +++ b/src/spdlog/details/pattern_formatter_impl.h @@ -5,11 +5,12 @@ #pragma once -#include <spdlog/formatter.h> -#include <spdlog/details/log_msg.h> -#include <spdlog/details/os.h> -#include <spdlog/details/format.h> +#include "../details/log_msg.h" +#include "../details/os.h" +#include "../fmt/fmt.h" +#include "../formatter.h" +#include <array> #include <chrono> #include <ctime> #include <memory> @@ -19,44 +20,38 @@ #include <utility> #include <vector> -namespace spdlog -{ -namespace details -{ +namespace spdlog { namespace details { class flag_formatter { public: - virtual ~flag_formatter() {} - virtual void format(details::log_msg& msg, const std::tm& tm_time) = 0; + virtual ~flag_formatter() = default; + virtual void format(details::log_msg &msg, const std::tm &tm_time) = 0; }; /////////////////////////////////////////////////////////////////////// // name & level pattern appenders /////////////////////////////////////////////////////////////////////// -namespace +class name_formatter : public flag_formatter { -class name_formatter :public flag_formatter -{ - void format(details::log_msg& msg, const std::tm&) override + void format(details::log_msg &msg, const std::tm &) override { - msg.formatted << msg.logger_name; + msg.formatted << *msg.logger_name; } }; -} // log level appender -class level_formatter :public flag_formatter +class level_formatter : public flag_formatter { - void format(details::log_msg& msg, const std::tm&) override + void format(details::log_msg &msg, const std::tm &) override { msg.formatted << level::to_str(msg.level); } }; // short log level appender -class short_level_formatter :public flag_formatter +class short_level_formatter : public flag_formatter { - void format(details::log_msg& msg, const std::tm&) override + void format(details::log_msg &msg, const std::tm &) override { msg.formatted << level::to_short_str(msg.level); } @@ -66,171 +61,166 @@ class short_level_formatter :public flag_formatter // Date time pattern appenders /////////////////////////////////////////////////////////////////////// -static const char* ampm(const tm& t) +static const char *ampm(const tm &t) { return t.tm_hour >= 12 ? "PM" : "AM"; } -static int to12h(const tm& t) +static int to12h(const tm &t) { return t.tm_hour > 12 ? t.tm_hour - 12 : t.tm_hour; } -//Abbreviated weekday name -static const std::string days[] { "Sun", "Mon", "Tue", "Wed", "Thu", "Fri", "Sat" }; -class a_formatter :public flag_formatter +// Abbreviated weekday name +static const std::string days[]{"Sun", "Mon", "Tue", "Wed", "Thu", "Fri", "Sat"}; +class a_formatter : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { msg.formatted << days[tm_time.tm_wday]; } }; -//Full weekday name -static const std::string full_days[] { "Sunday", "Monday", "Tuesday", "Wednesday", "Thursday", "Friday", "Saturday" }; -class A_formatter :public flag_formatter +// Full weekday name +static const std::string full_days[]{"Sunday", "Monday", "Tuesday", "Wednesday", "Thursday", "Friday", "Saturday"}; +class A_formatter : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { msg.formatted << full_days[tm_time.tm_wday]; } }; -//Abbreviated month -static const std::string months[] { "Jan", "Feb", "Mar", "Apr", "May", "June", "July", "Aug", "Sept", "Oct", "Nov", "Dec" }; -class b_formatter :public flag_formatter +// Abbreviated month +static const std::string months[]{"Jan", "Feb", "Mar", "Apr", "May", "Jun", "Jul", "Aug", "Sept", "Oct", "Nov", "Dec"}; +class b_formatter : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { - msg.formatted<< months[tm_time.tm_mon]; + msg.formatted << months[tm_time.tm_mon]; } }; -//Full month name -static const std::string full_months[] { "January", "February", "March", "April", "May", "June", "July", "August", "September", "October", "November", "December" }; -class B_formatter :public flag_formatter +// Full month name +static const std::string full_months[]{ + "January", "February", "March", "April", "May", "June", "July", "August", "September", "October", "November", "December"}; +class B_formatter : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { msg.formatted << full_months[tm_time.tm_mon]; } }; - -//write 2 ints seperated by sep with padding of 2 -static fmt::MemoryWriter& pad_n_join(fmt::MemoryWriter& w, int v1, int v2, char sep) +// write 2 ints separated by sep with padding of 2 +static fmt::MemoryWriter &pad_n_join(fmt::MemoryWriter &w, int v1, int v2, char sep) { w << fmt::pad(v1, 2, '0') << sep << fmt::pad(v2, 2, '0'); return w; } -//write 3 ints seperated by sep with padding of 2 -static fmt::MemoryWriter& pad_n_join(fmt::MemoryWriter& w, int v1, int v2, int v3, char sep) +// write 3 ints separated by sep with padding of 2 +static fmt::MemoryWriter &pad_n_join(fmt::MemoryWriter &w, int v1, int v2, int v3, char sep) { w << fmt::pad(v1, 2, '0') << sep << fmt::pad(v2, 2, '0') << sep << fmt::pad(v3, 2, '0'); return w; } - -//Date and time representation (Thu Aug 23 15:35:46 2014) -class c_formatter :public flag_formatter +// Date and time representation (Thu Aug 23 15:35:46 2014) +class c_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { msg.formatted << days[tm_time.tm_wday] << ' ' << months[tm_time.tm_mon] << ' ' << tm_time.tm_mday << ' '; pad_n_join(msg.formatted, tm_time.tm_hour, tm_time.tm_min, tm_time.tm_sec, ':') << ' ' << tm_time.tm_year + 1900; } }; - // year - 2 digit -class C_formatter :public flag_formatter +class C_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { msg.formatted << fmt::pad(tm_time.tm_year % 100, 2, '0'); } }; - - // Short MM/DD/YY date, equivalent to %m/%d/%y 08/23/01 -class D_formatter :public flag_formatter +class D_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { pad_n_join(msg.formatted, tm_time.tm_mon + 1, tm_time.tm_mday, tm_time.tm_year % 100, '/'); } }; - // year - 4 digit -class Y_formatter :public flag_formatter +class Y_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { msg.formatted << tm_time.tm_year + 1900; } }; // month 1-12 -class m_formatter :public flag_formatter +class m_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { msg.formatted << fmt::pad(tm_time.tm_mon + 1, 2, '0'); } }; // day of month 1-31 -class d_formatter :public flag_formatter +class d_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { msg.formatted << fmt::pad(tm_time.tm_mday, 2, '0'); } }; -// hours in 24 format 0-23 -class H_formatter :public flag_formatter +// hours in 24 format 0-23 +class H_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { msg.formatted << fmt::pad(tm_time.tm_hour, 2, '0'); } }; -// hours in 12 format 1-12 -class I_formatter :public flag_formatter +// hours in 12 format 1-12 +class I_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { msg.formatted << fmt::pad(to12h(tm_time), 2, '0'); } }; // minutes 0-59 -class M_formatter :public flag_formatter +class M_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { msg.formatted << fmt::pad(tm_time.tm_min, 2, '0'); } }; // seconds 0-59 -class S_formatter :public flag_formatter +class S_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { msg.formatted << fmt::pad(tm_time.tm_sec, 2, '0'); } }; // milliseconds -class e_formatter :public flag_formatter +class e_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm&) override + void format(details::log_msg &msg, const std::tm &) override { auto duration = msg.time.time_since_epoch(); auto millis = std::chrono::duration_cast<std::chrono::milliseconds>(duration).count() % 1000; @@ -239,9 +229,9 @@ class e_formatter :public flag_formatter }; // microseconds -class f_formatter :public flag_formatter +class f_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm&) override + void format(details::log_msg &msg, const std::tm &) override { auto duration = msg.time.time_since_epoch(); auto micros = std::chrono::duration_cast<std::chrono::microseconds>(duration).count() % 1000000; @@ -250,9 +240,9 @@ class f_formatter :public flag_formatter }; // nanoseconds -class F_formatter :public flag_formatter +class F_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm&) override + void format(details::log_msg &msg, const std::tm &) override { auto duration = msg.time.time_since_epoch(); auto ns = std::chrono::duration_cast<std::chrono::nanoseconds>(duration).count() % 1000000000; @@ -260,55 +250,63 @@ class F_formatter :public flag_formatter } }; +class E_formatter SPDLOG_FINAL : public flag_formatter +{ + void format(details::log_msg &msg, const std::tm &) override + { + auto duration = msg.time.time_since_epoch(); + auto seconds = std::chrono::duration_cast<std::chrono::seconds>(duration).count(); + msg.formatted << seconds; + } +}; + // AM/PM -class p_formatter :public flag_formatter +class p_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { msg.formatted << ampm(tm_time); } }; - // 12 hour clock 02:55:02 pm -class r_formatter :public flag_formatter +class r_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { pad_n_join(msg.formatted, to12h(tm_time), tm_time.tm_min, tm_time.tm_sec, ':') << ' ' << ampm(tm_time); } }; // 24-hour HH:MM time, equivalent to %H:%M -class R_formatter :public flag_formatter +class R_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { pad_n_join(msg.formatted, tm_time.tm_hour, tm_time.tm_min, ':'); } }; // ISO 8601 time format (HH:MM:SS), equivalent to %H:%M:%S -class T_formatter :public flag_formatter +class T_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { pad_n_join(msg.formatted, tm_time.tm_hour, tm_time.tm_min, tm_time.tm_sec, ':'); } }; - // ISO 8601 offset from UTC in timezone (+-HH:MM) -class z_formatter :public flag_formatter +class z_formatter SPDLOG_FINAL : public flag_formatter { public: const std::chrono::seconds cache_refresh = std::chrono::seconds(5); - z_formatter() :_last_update(std::chrono::seconds(0)) {} - z_formatter(const z_formatter&) = delete; - z_formatter& operator=(const z_formatter&) = delete; + z_formatter() = default; + z_formatter(const z_formatter &) = delete; + z_formatter &operator=(const z_formatter &) = delete; - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { #ifdef _WIN32 int total_minutes = get_cached_offset(msg, tm_time); @@ -317,23 +315,31 @@ public: // it is very fast (already stored in tm.tm_gmtoff) int total_minutes = os::utc_minutes_offset(tm_time); #endif + bool is_negative = total_minutes < 0; + char sign; + if (is_negative) + { + total_minutes = -total_minutes; + sign = '-'; + } + else + { + sign = '+'; + } int h = total_minutes / 60; int m = total_minutes % 60; - if (h >= 0) //minus sign will be printed anyway if negative - { - msg.formatted << '+'; - } + msg.formatted << sign; pad_n_join(msg.formatted, h, m, ':'); } + private: - log_clock::time_point _last_update; - int _offset_minutes; + log_clock::time_point _last_update{std::chrono::seconds(0)}; + int _offset_minutes{0}; std::mutex _mutex; - int get_cached_offset(const log_msg& msg, const std::tm& tm_time) + int get_cached_offset(const log_msg &msg, const std::tm &tm_time) { - using namespace std::chrono; std::lock_guard<std::mutex> l(_mutex); if (msg.time - _last_update >= cache_refresh) { @@ -344,63 +350,81 @@ private: } }; - - -//Thread id -class t_formatter :public flag_formatter +// Thread id +class t_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm&) override + void format(details::log_msg &msg, const std::tm &) override { msg.formatted << msg.thread_id; } }; +// Current pid +class pid_formatter SPDLOG_FINAL : public flag_formatter +{ + void format(details::log_msg &msg, const std::tm &) override + { + msg.formatted << details::os::pid(); + } +}; -class v_formatter :public flag_formatter +// message counter formatter +class i_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm&) override + void format(details::log_msg &msg, const std::tm &) override + { + msg.formatted << fmt::pad(msg.msg_id, 6, '0'); + } +}; + +class v_formatter SPDLOG_FINAL : public flag_formatter +{ + void format(details::log_msg &msg, const std::tm &) override { msg.formatted << fmt::StringRef(msg.raw.data(), msg.raw.size()); } }; -class ch_formatter :public flag_formatter +class ch_formatter SPDLOG_FINAL : public flag_formatter { public: - explicit ch_formatter(char ch) : _ch(ch) - {} - void format(details::log_msg& msg, const std::tm&) override + explicit ch_formatter(char ch) + : _ch(ch) + { + } + void format(details::log_msg &msg, const std::tm &) override { msg.formatted << _ch; } + private: char _ch; }; - -//aggregate user chars to display as is -class aggregate_formatter :public flag_formatter +// aggregate user chars to display as is +class aggregate_formatter SPDLOG_FINAL : public flag_formatter { public: - aggregate_formatter() - {} + aggregate_formatter() = default; + void add_ch(char ch) { _str += ch; } - void format(details::log_msg& msg, const std::tm&) override + void format(details::log_msg &msg, const std::tm &) override { msg.formatted << _str; } + private: std::string _str; }; // Full info formatter // pattern: [%Y-%m-%d %H:%M:%S.%e] [%n] [%l] %v -class full_formatter :public flag_formatter +class full_formatter SPDLOG_FINAL : public flag_formatter { - void format(details::log_msg& msg, const std::tm& tm_time) override + void format(details::log_msg &msg, const std::tm &tm_time) override { #ifndef SPDLOG_NO_DATETIME auto duration = msg.time.time_since_epoch(); @@ -419,7 +443,6 @@ class full_formatter :public flag_formatter level::to_str(msg.level), msg.raw.str());*/ - // Faster (albeit uglier) way to format the line (5.6 million lines/sec under 10 threads) msg.formatted << '[' << static_cast<unsigned int>(tm_time.tm_year + 1900) << '-' << fmt::pad(static_cast<unsigned int>(tm_time.tm_mon + 1), 2, '0') << '-' @@ -429,13 +452,13 @@ class full_formatter :public flag_formatter << fmt::pad(static_cast<unsigned int>(tm_time.tm_sec), 2, '0') << '.' << fmt::pad(static_cast<unsigned int>(millis), 3, '0') << "] "; -//no datetime needed + // no datetime needed #else (void)tm_time; #endif #ifndef SPDLOG_NO_NAME - msg.formatted << '[' << msg.logger_name << "] "; + msg.formatted << '[' << *msg.logger_name << "] "; #endif msg.formatted << '[' << level::to_str(msg.level) << "] "; @@ -443,17 +466,18 @@ class full_formatter :public flag_formatter } }; -} -} +}} // namespace spdlog::details /////////////////////////////////////////////////////////////////////////////// // pattern_formatter inline impl /////////////////////////////////////////////////////////////////////////////// -inline spdlog::pattern_formatter::pattern_formatter(const std::string& pattern) +inline spdlog::pattern_formatter::pattern_formatter(const std::string &pattern, pattern_time_type pattern_time, std::string eol) + : _eol(std::move(eol)) + , _pattern_time(pattern_time) { compile_pattern(pattern); } -inline void spdlog::pattern_formatter::compile_pattern(const std::string& pattern) +inline void spdlog::pattern_formatter::compile_pattern(const std::string &pattern) { auto end = pattern.end(); std::unique_ptr<details::aggregate_formatter> user_chars; @@ -461,9 +485,8 @@ inline void spdlog::pattern_formatter::compile_pattern(const std::string& patter { if (*it == '%') { - if (user_chars) //append user chars found so far + if (user_chars) // append user chars found so far _formatters.push_back(std::move(user_chars)); - if (++it != end) handle_flag(*it); else @@ -476,11 +499,10 @@ inline void spdlog::pattern_formatter::compile_pattern(const std::string& patter user_chars->add_ch(*it); } } - if (user_chars) //append raw chars found so far + if (user_chars) // append raw chars found so far { _formatters.push_back(std::move(user_chars)); } - } inline void spdlog::pattern_formatter::handle_flag(char flag) { @@ -488,141 +510,159 @@ inline void spdlog::pattern_formatter::handle_flag(char flag) { // logger name case 'n': - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::name_formatter())); + _formatters.emplace_back(new details::name_formatter()); break; case 'l': - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::level_formatter())); + _formatters.emplace_back(new details::level_formatter()); break; case 'L': - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::short_level_formatter())); + _formatters.emplace_back(new details::short_level_formatter()); break; - case('t') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::t_formatter())); + case ('t'): + _formatters.emplace_back(new details::t_formatter()); break; - case('v') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::v_formatter())); + case ('v'): + _formatters.emplace_back(new details::v_formatter()); break; - case('a') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::a_formatter())); + case ('a'): + _formatters.emplace_back(new details::a_formatter()); break; - case('A') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::A_formatter())); + case ('A'): + _formatters.emplace_back(new details::A_formatter()); break; - case('b') : - case('h') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::b_formatter())); + case ('b'): + case ('h'): + _formatters.emplace_back(new details::b_formatter()); break; - case('B') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::B_formatter())); + case ('B'): + _formatters.emplace_back(new details::B_formatter()); break; - case('c') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::c_formatter())); + case ('c'): + _formatters.emplace_back(new details::c_formatter()); break; - case('C') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::C_formatter())); + case ('C'): + _formatters.emplace_back(new details::C_formatter()); break; - case('Y') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::Y_formatter())); + case ('Y'): + _formatters.emplace_back(new details::Y_formatter()); break; - case('D') : - case('x') : + case ('D'): + case ('x'): - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::D_formatter())); + _formatters.emplace_back(new details::D_formatter()); break; - case('m') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::m_formatter())); + case ('m'): + _formatters.emplace_back(new details::m_formatter()); break; - case('d') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::d_formatter())); + case ('d'): + _formatters.emplace_back(new details::d_formatter()); break; - case('H') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::H_formatter())); + case ('H'): + _formatters.emplace_back(new details::H_formatter()); break; - case('I') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::I_formatter())); + case ('I'): + _formatters.emplace_back(new details::I_formatter()); break; - case('M') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::M_formatter())); + case ('M'): + _formatters.emplace_back(new details::M_formatter()); break; - case('S') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::S_formatter())); + case ('S'): + _formatters.emplace_back(new details::S_formatter()); break; - case('e') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::e_formatter())); + case ('e'): + _formatters.emplace_back(new details::e_formatter()); break; - case('f') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::f_formatter())); + case ('f'): + _formatters.emplace_back(new details::f_formatter()); break; - case('F') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::F_formatter())); + case ('F'): + _formatters.emplace_back(new details::F_formatter()); break; - case('p') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::p_formatter())); + case ('E'): + _formatters.emplace_back(new details::E_formatter()); break; - case('r') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::r_formatter())); + case ('p'): + _formatters.emplace_back(new details::p_formatter()); break; - case('R') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::R_formatter())); + case ('r'): + _formatters.emplace_back(new details::r_formatter()); break; - case('T') : - case('X') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::T_formatter())); + case ('R'): + _formatters.emplace_back(new details::R_formatter()); break; - case('z') : - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::z_formatter())); + case ('T'): + case ('X'): + _formatters.emplace_back(new details::T_formatter()); + break; + + case ('z'): + _formatters.emplace_back(new details::z_formatter()); break; case ('+'): - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::full_formatter())); + _formatters.emplace_back(new details::full_formatter()); + break; + + case ('P'): + _formatters.emplace_back(new details::pid_formatter()); + break; + + case ('i'): + _formatters.emplace_back(new details::i_formatter()); break; - default: //Unkown flag appears as is - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::ch_formatter('%'))); - _formatters.push_back(std::unique_ptr<details::flag_formatter>(new details::ch_formatter(flag))); + default: // Unknown flag appears as is + _formatters.emplace_back(new details::ch_formatter('%')); + _formatters.emplace_back(new details::ch_formatter(flag)); break; } } - -inline void spdlog::pattern_formatter::format(details::log_msg& msg) +inline std::tm spdlog::pattern_formatter::get_time(details::log_msg &msg) { - try + if (_pattern_time == pattern_time_type::local) { - auto tm_time = details::os::localtime(log_clock::to_time_t(msg.time)); - for (auto &f : _formatters) - { - f->format(msg, tm_time); - } - //write eol - msg.formatted.write(details::os::eol, details::os::eol_size); + return details::os::localtime(log_clock::to_time_t(msg.time)); } - catch(const fmt::FormatError& e) + return details::os::gmtime(log_clock::to_time_t(msg.time)); +} + +inline void spdlog::pattern_formatter::format(details::log_msg &msg) +{ + +#ifndef SPDLOG_NO_DATETIME + auto tm_time = get_time(msg); +#else + std::tm tm_time; +#endif + for (auto &f : _formatters) { - throw spdlog_ex(fmt::format("formatting error while processing format string: {}", e.what())); + f->format(msg, tm_time); } + // write eol + msg.formatted.write(_eol.data(), _eol.size()); } diff --git a/src/spdlog/details/registry.h b/src/spdlog/details/registry.h index 7d744f89edaefe73bf164f4686323ec5e7174906..300e4fd7f0b896c4c55489fd7b2ad21b6931c476 100644 --- a/src/spdlog/details/registry.h +++ b/src/spdlog/details/registry.h @@ -10,10 +10,10 @@ // If user requests a non existing logger, nullptr will be returned // This class is thread safe -#include <spdlog/details/null_mutex.h> -#include <spdlog/logger.h> -#include <spdlog/async_logger.h> -#include <spdlog/common.h> +#include "../async_logger.h" +#include "../common.h" +#include "../details/null_mutex.h" +#include "../logger.h" #include <chrono> #include <functional> @@ -22,13 +22,12 @@ #include <string> #include <unordered_map> -namespace spdlog -{ -namespace details -{ +namespace spdlog { namespace details { template <class Mutex> class registry_t { public: + registry_t<Mutex>(const registry_t<Mutex> &) = delete; + registry_t<Mutex> &operator=(const registry_t<Mutex> &) = delete; void register_logger(std::shared_ptr<logger> logger) { @@ -38,35 +37,71 @@ public: _loggers[logger_name] = logger; } - - std::shared_ptr<logger> get(const std::string& logger_name) + std::shared_ptr<logger> get(const std::string &logger_name) { std::lock_guard<Mutex> lock(_mutex); auto found = _loggers.find(logger_name); return found == _loggers.end() ? nullptr : found->second; } - template<class It> - std::shared_ptr<logger> create(const std::string& logger_name, const It& sinks_begin, const It& sinks_end) + template <class It> std::shared_ptr<logger> create(const std::string &logger_name, const It &sinks_begin, const It &sinks_end) { std::lock_guard<Mutex> lock(_mutex); throw_if_exists(logger_name); std::shared_ptr<logger> new_logger; if (_async_mode) - new_logger = std::make_shared<async_logger>(logger_name, sinks_begin, sinks_end, _async_q_size, _overflow_policy, _worker_warmup_cb, _flush_interval_ms, _worker_teardown_cb); + new_logger = std::make_shared<async_logger>(logger_name, sinks_begin, sinks_end, _async_q_size, _overflow_policy, + _worker_warmup_cb, _flush_interval_ms, _worker_teardown_cb); else new_logger = std::make_shared<logger>(logger_name, sinks_begin, sinks_end); if (_formatter) new_logger->set_formatter(_formatter); + if (_err_handler) + new_logger->set_error_handler(_err_handler); + + new_logger->set_level(_level); + new_logger->flush_on(_flush_level); + + // Add to registry + _loggers[logger_name] = new_logger; + return new_logger; + } + + template <class It> + std::shared_ptr<async_logger> create_async(const std::string &logger_name, size_t queue_size, + const async_overflow_policy overflow_policy, const std::function<void()> &worker_warmup_cb, + const std::chrono::milliseconds &flush_interval_ms, const std::function<void()> &worker_teardown_cb, const It &sinks_begin, + const It &sinks_end) + { + std::lock_guard<Mutex> lock(_mutex); + throw_if_exists(logger_name); + auto new_logger = std::make_shared<async_logger>( + logger_name, sinks_begin, sinks_end, queue_size, overflow_policy, worker_warmup_cb, flush_interval_ms, worker_teardown_cb); + + if (_formatter) + new_logger->set_formatter(_formatter); + + if (_err_handler) + new_logger->set_error_handler(_err_handler); + new_logger->set_level(_level); - //Add to registry + new_logger->flush_on(_flush_level); + + // Add to registry _loggers[logger_name] = new_logger; return new_logger; } - void drop(const std::string& logger_name) + void apply_all(std::function<void(std::shared_ptr<logger>)> fun) + { + std::lock_guard<Mutex> lock(_mutex); + for (auto &l : _loggers) + fun(l.second); + } + + void drop(const std::string &logger_name) { std::lock_guard<Mutex> lock(_mutex); _loggers.erase(logger_name); @@ -77,42 +112,73 @@ public: std::lock_guard<Mutex> lock(_mutex); _loggers.clear(); } - std::shared_ptr<logger> create(const std::string& logger_name, sinks_init_list sinks) + + std::shared_ptr<logger> create(const std::string &logger_name, sinks_init_list sinks) { return create(logger_name, sinks.begin(), sinks.end()); } - std::shared_ptr<logger> create(const std::string& logger_name, sink_ptr sink) + std::shared_ptr<logger> create(const std::string &logger_name, sink_ptr sink) + { + return create(logger_name, {sink}); + } + + std::shared_ptr<async_logger> create_async(const std::string &logger_name, size_t queue_size, + const async_overflow_policy overflow_policy, const std::function<void()> &worker_warmup_cb, + const std::chrono::milliseconds &flush_interval_ms, const std::function<void()> &worker_teardown_cb, sinks_init_list sinks) { - return create(logger_name, { sink }); + return create_async( + logger_name, queue_size, overflow_policy, worker_warmup_cb, flush_interval_ms, worker_teardown_cb, sinks.begin(), sinks.end()); } + std::shared_ptr<async_logger> create_async(const std::string &logger_name, size_t queue_size, + const async_overflow_policy overflow_policy, const std::function<void()> &worker_warmup_cb, + const std::chrono::milliseconds &flush_interval_ms, const std::function<void()> &worker_teardown_cb, sink_ptr sink) + { + return create_async(logger_name, queue_size, overflow_policy, worker_warmup_cb, flush_interval_ms, worker_teardown_cb, {sink}); + } void formatter(formatter_ptr f) { std::lock_guard<Mutex> lock(_mutex); _formatter = f; - for (auto& l : _loggers) + for (auto &l : _loggers) l.second->set_formatter(_formatter); } - void set_pattern(const std::string& pattern) + void set_pattern(const std::string &pattern) { std::lock_guard<Mutex> lock(_mutex); _formatter = std::make_shared<pattern_formatter>(pattern); - for (auto& l : _loggers) + for (auto &l : _loggers) l.second->set_formatter(_formatter); } void set_level(level::level_enum log_level) { std::lock_guard<Mutex> lock(_mutex); - for (auto& l : _loggers) + for (auto &l : _loggers) l.second->set_level(log_level); _level = log_level; } - void set_async_mode(size_t q_size, const async_overflow_policy overflow_policy, const std::function<void()>& worker_warmup_cb, const std::chrono::milliseconds& flush_interval_ms, const std::function<void()>& worker_teardown_cb) + void flush_on(level::level_enum log_level) + { + std::lock_guard<Mutex> lock(_mutex); + for (auto &l : _loggers) + l.second->flush_on(log_level); + _flush_level = log_level; + } + + void set_error_handler(log_err_handler handler) + { + for (auto &l : _loggers) + l.second->set_error_handler(handler); + _err_handler = handler; + } + + void set_async_mode(size_t q_size, const async_overflow_policy overflow_policy, const std::function<void()> &worker_warmup_cb, + const std::chrono::milliseconds &flush_interval_ms, const std::function<void()> &worker_teardown_cb) { std::lock_guard<Mutex> lock(_mutex); _async_mode = true; @@ -129,37 +195,39 @@ public: _async_mode = false; } - static registry_t<Mutex>& instance() + static registry_t<Mutex> &instance() { static registry_t<Mutex> s_instance; return s_instance; } private: - registry_t<Mutex>() {} - registry_t<Mutex>(const registry_t<Mutex>&) = delete; - registry_t<Mutex>& operator=(const registry_t<Mutex>&) = delete; + registry_t<Mutex>() = default; void throw_if_exists(const std::string &logger_name) { if (_loggers.find(logger_name) != _loggers.end()) throw spdlog_ex("logger with name '" + logger_name + "' already exists"); } + Mutex _mutex; - std::unordered_map <std::string, std::shared_ptr<logger>> _loggers; + std::unordered_map<std::string, std::shared_ptr<logger>> _loggers; formatter_ptr _formatter; level::level_enum _level = level::info; + level::level_enum _flush_level = level::off; + log_err_handler _err_handler; bool _async_mode = false; size_t _async_q_size = 0; async_overflow_policy _overflow_policy = async_overflow_policy::block_retry; - std::function<void()> _worker_warmup_cb = nullptr; + std::function<void()> _worker_warmup_cb; std::chrono::milliseconds _flush_interval_ms; - std::function<void()> _worker_teardown_cb = nullptr; + std::function<void()> _worker_teardown_cb; }; + #ifdef SPDLOG_NO_REGISTRY_MUTEX -typedef registry_t<spdlog::details::null_mutex> registry; +using registry = registry_t<spdlog::details::null_mutex>; #else -typedef registry_t<std::mutex> registry; +using registry = registry_t<std::mutex>; #endif -} -} + +}} // namespace spdlog::details diff --git a/src/spdlog/details/spdlog_impl.h b/src/spdlog/details/spdlog_impl.h index 8337ab47c7d8ef2819fe801bc7911775fd8dc9bc..114498b14ea24825a1df97462a5cd3e52812fedd 100644 --- a/src/spdlog/details/spdlog_impl.h +++ b/src/spdlog/details/spdlog_impl.h @@ -8,12 +8,23 @@ // // Global registry functions // -#include <spdlog/spdlog.h> -#include <spdlog/details/registry.h> -#include <spdlog/sinks/file_sinks.h> -#include <spdlog/sinks/stdout_sinks.h> -#include <spdlog/sinks/syslog_sink.h> -#include <spdlog/sinks/ansicolor_sink.h> +#include "../details/registry.h" +#include "../sinks/file_sinks.h" +#include "../sinks/stdout_sinks.h" +#include "../spdlog.h" +#ifdef SPDLOG_ENABLE_SYSLOG +#include "../sinks/syslog_sink.h" +#endif + +#if defined _WIN32 && !defined(__cplusplus_winrt) +#include "../sinks/wincolor_sink.h" +#else +#include "../sinks/ansicolor_sink.h" +#endif + +#ifdef __ANDROID__ +#include "../sinks/android_sink.h" +#endif #include <chrono> #include <functional> @@ -22,10 +33,10 @@ inline void spdlog::register_logger(std::shared_ptr<logger> logger) { - return details::registry::instance().register_logger(logger); + return details::registry::instance().register_logger(std::move(logger)); } -inline std::shared_ptr<spdlog::logger> spdlog::get(const std::string& name) +inline std::shared_ptr<spdlog::logger> spdlog::get(const std::string &name) { return details::registry::instance().get(name); } @@ -35,93 +46,196 @@ inline void spdlog::drop(const std::string &name) details::registry::instance().drop(name); } +// Create multi/single threaded simple file logger +inline std::shared_ptr<spdlog::logger> spdlog::basic_logger_mt(const std::string &logger_name, const filename_t &filename, bool truncate) +{ + return create<spdlog::sinks::simple_file_sink_mt>(logger_name, filename, truncate); +} + +inline std::shared_ptr<spdlog::logger> spdlog::basic_logger_st(const std::string &logger_name, const filename_t &filename, bool truncate) +{ + return create<spdlog::sinks::simple_file_sink_st>(logger_name, filename, truncate); +} + // Create multi/single threaded rotating file logger -inline std::shared_ptr<spdlog::logger> spdlog::rotating_logger_mt(const std::string& logger_name, const filename_t& filename, size_t max_file_size, size_t max_files, bool force_flush) +inline std::shared_ptr<spdlog::logger> spdlog::rotating_logger_mt( + const std::string &logger_name, const filename_t &filename, size_t max_file_size, size_t max_files) { - return create<spdlog::sinks::rotating_file_sink_mt>(logger_name, filename, SPDLOG_FILENAME_T("txt"), max_file_size, max_files, force_flush); + return create<spdlog::sinks::rotating_file_sink_mt>(logger_name, filename, max_file_size, max_files); } -inline std::shared_ptr<spdlog::logger> spdlog::rotating_logger_st(const std::string& logger_name, const filename_t& filename, size_t max_file_size, size_t max_files, bool force_flush) +inline std::shared_ptr<spdlog::logger> spdlog::rotating_logger_st( + const std::string &logger_name, const filename_t &filename, size_t max_file_size, size_t max_files) { - return create<spdlog::sinks::rotating_file_sink_st>(logger_name, filename, SPDLOG_FILENAME_T("txt"), max_file_size, max_files, force_flush); + return create<spdlog::sinks::rotating_file_sink_st>(logger_name, filename, max_file_size, max_files); } // Create file logger which creates new file at midnight): -inline std::shared_ptr<spdlog::logger> spdlog::daily_logger_mt(const std::string& logger_name, const filename_t& filename, int hour, int minute, bool force_flush) +inline std::shared_ptr<spdlog::logger> spdlog::daily_logger_mt( + const std::string &logger_name, const filename_t &filename, int hour, int minute) +{ + return create<spdlog::sinks::daily_file_sink_mt>(logger_name, filename, hour, minute); +} + +inline std::shared_ptr<spdlog::logger> spdlog::daily_logger_st( + const std::string &logger_name, const filename_t &filename, int hour, int minute) +{ + return create<spdlog::sinks::daily_file_sink_st>(logger_name, filename, hour, minute); +} + +// +// stdout/stderr loggers +// +inline std::shared_ptr<spdlog::logger> spdlog::stdout_logger_mt(const std::string &logger_name) +{ + return spdlog::details::registry::instance().create(logger_name, spdlog::sinks::stdout_sink_mt::instance()); +} + +inline std::shared_ptr<spdlog::logger> spdlog::stdout_logger_st(const std::string &logger_name) +{ + return spdlog::details::registry::instance().create(logger_name, spdlog::sinks::stdout_sink_st::instance()); +} + +inline std::shared_ptr<spdlog::logger> spdlog::stderr_logger_mt(const std::string &logger_name) +{ + return spdlog::details::registry::instance().create(logger_name, spdlog::sinks::stderr_sink_mt::instance()); +} + +inline std::shared_ptr<spdlog::logger> spdlog::stderr_logger_st(const std::string &logger_name) +{ + return spdlog::details::registry::instance().create(logger_name, spdlog::sinks::stderr_sink_st::instance()); +} + +// +// stdout/stderr color loggers +// +#if defined _WIN32 && !defined(__cplusplus_winrt) + +inline std::shared_ptr<spdlog::logger> spdlog::stdout_color_mt(const std::string &logger_name) +{ + auto sink = std::make_shared<spdlog::sinks::wincolor_stdout_sink_mt>(); + return spdlog::details::registry::instance().create(logger_name, sink); +} + +inline std::shared_ptr<spdlog::logger> spdlog::stdout_color_st(const std::string &logger_name) { - return create<spdlog::sinks::daily_file_sink_mt>(logger_name, filename, SPDLOG_FILENAME_T("txt"), hour, minute, force_flush); + auto sink = std::make_shared<spdlog::sinks::wincolor_stdout_sink_st>(); + return spdlog::details::registry::instance().create(logger_name, sink); } -inline std::shared_ptr<spdlog::logger> spdlog::daily_logger_st(const std::string& logger_name, const filename_t& filename, int hour, int minute, bool force_flush) +inline std::shared_ptr<spdlog::logger> spdlog::stderr_color_mt(const std::string &logger_name) { - return create<spdlog::sinks::daily_file_sink_st>(logger_name, filename, SPDLOG_FILENAME_T("txt"), hour, minute, force_flush); + auto sink = std::make_shared<spdlog::sinks::wincolor_stderr_sink_mt>(); + return spdlog::details::registry::instance().create(logger_name, sink); } -// Create stdout/stderr loggers (with optinal color support) -inline std::shared_ptr<spdlog::logger> create_console_logger(const std::string& logger_name, spdlog::sink_ptr sink, bool color) +inline std::shared_ptr<spdlog::logger> spdlog::stderr_color_st(const std::string &logger_name) { - if (color) //use color wrapper sink - sink = std::make_shared<spdlog::sinks::ansicolor_sink>(sink); + auto sink = std::make_shared<spdlog::sinks::wincolor_stderr_sink_st>(); return spdlog::details::registry::instance().create(logger_name, sink); } -inline std::shared_ptr<spdlog::logger> spdlog::stdout_logger_mt(const std::string& logger_name, bool color) +#else // ansi terminal colors + +inline std::shared_ptr<spdlog::logger> spdlog::stdout_color_mt(const std::string &logger_name) { - return create_console_logger(logger_name, sinks::stdout_sink_mt::instance(), color); + auto sink = std::make_shared<spdlog::sinks::ansicolor_stdout_sink_mt>(); + return spdlog::details::registry::instance().create(logger_name, sink); } -inline std::shared_ptr<spdlog::logger> spdlog::stdout_logger_st(const std::string& logger_name, bool color) +inline std::shared_ptr<spdlog::logger> spdlog::stdout_color_st(const std::string &logger_name) { - return create_console_logger(logger_name, sinks::stdout_sink_st::instance(), color); + auto sink = std::make_shared<spdlog::sinks::ansicolor_stdout_sink_st>(); + return spdlog::details::registry::instance().create(logger_name, sink); } -inline std::shared_ptr<spdlog::logger> spdlog::stderr_logger_mt(const std::string& logger_name, bool color) +inline std::shared_ptr<spdlog::logger> spdlog::stderr_color_mt(const std::string &logger_name) { - return create_console_logger(logger_name, sinks::stderr_sink_mt::instance(), color); + auto sink = std::make_shared<spdlog::sinks::ansicolor_stderr_sink_mt>(); + return spdlog::details::registry::instance().create(logger_name, sink); } -inline std::shared_ptr<spdlog::logger> spdlog::stderr_logger_st(const std::string& logger_name, bool color) +inline std::shared_ptr<spdlog::logger> spdlog::stderr_color_st(const std::string &logger_name) { - return create_console_logger(logger_name, sinks::stderr_sink_st::instance(), color); + auto sink = std::make_shared<spdlog::sinks::ansicolor_stderr_sink_st>(); + return spdlog::details::registry::instance().create(logger_name, sink); } +#endif -#if defined(__linux__) || defined(__APPLE__) +#ifdef SPDLOG_ENABLE_SYSLOG // Create syslog logger -inline std::shared_ptr<spdlog::logger> spdlog::syslog_logger(const std::string& logger_name, const std::string& syslog_ident, int syslog_option) +inline std::shared_ptr<spdlog::logger> spdlog::syslog_logger( + const std::string &logger_name, const std::string &syslog_ident, int syslog_option, int syslog_facility) { - return create<spdlog::sinks::syslog_sink>(logger_name, syslog_ident, syslog_option); + return create<spdlog::sinks::syslog_sink>(logger_name, syslog_ident, syslog_option, syslog_facility); } #endif +#ifdef __ANDROID__ +inline std::shared_ptr<spdlog::logger> spdlog::android_logger(const std::string &logger_name, const std::string &tag) +{ + return create<spdlog::sinks::android_sink>(logger_name, tag); +} +#endif -//Create logger with multiple sinks +// Create and register a logger a single sink +inline std::shared_ptr<spdlog::logger> spdlog::create(const std::string &logger_name, const spdlog::sink_ptr &sink) +{ + return details::registry::instance().create(logger_name, sink); +} -inline std::shared_ptr<spdlog::logger> spdlog::create(const std::string& logger_name, spdlog::sinks_init_list sinks) +// Create logger with multiple sinks +inline std::shared_ptr<spdlog::logger> spdlog::create(const std::string &logger_name, spdlog::sinks_init_list sinks) { return details::registry::instance().create(logger_name, sinks); } - template <typename Sink, typename... Args> -inline std::shared_ptr<spdlog::logger> spdlog::create(const std::string& logger_name, Args... args) +inline std::shared_ptr<spdlog::logger> spdlog::create(const std::string &logger_name, Args... args) { sink_ptr sink = std::make_shared<Sink>(args...); - return details::registry::instance().create(logger_name, { sink }); + return details::registry::instance().create(logger_name, {sink}); } - -template<class It> -inline std::shared_ptr<spdlog::logger> spdlog::create(const std::string& logger_name, const It& sinks_begin, const It& sinks_end) +template <class It> +inline std::shared_ptr<spdlog::logger> spdlog::create(const std::string &logger_name, const It &sinks_begin, const It &sinks_end) { return details::registry::instance().create(logger_name, sinks_begin, sinks_end); } +// Create and register an async logger with a single sink +inline std::shared_ptr<spdlog::logger> spdlog::create_async(const std::string &logger_name, const sink_ptr &sink, size_t queue_size, + const async_overflow_policy overflow_policy, const std::function<void()> &worker_warmup_cb, + const std::chrono::milliseconds &flush_interval_ms, const std::function<void()> &worker_teardown_cb) +{ + return details::registry::instance().create_async( + logger_name, queue_size, overflow_policy, worker_warmup_cb, flush_interval_ms, worker_teardown_cb, sink); +} + +// Create and register an async logger with multiple sinks +inline std::shared_ptr<spdlog::logger> spdlog::create_async(const std::string &logger_name, sinks_init_list sinks, size_t queue_size, + const async_overflow_policy overflow_policy, const std::function<void()> &worker_warmup_cb, + const std::chrono::milliseconds &flush_interval_ms, const std::function<void()> &worker_teardown_cb) +{ + return details::registry::instance().create_async( + logger_name, queue_size, overflow_policy, worker_warmup_cb, flush_interval_ms, worker_teardown_cb, sinks); +} + +template <class It> +inline std::shared_ptr<spdlog::logger> spdlog::create_async(const std::string &logger_name, const It &sinks_begin, const It &sinks_end, + size_t queue_size, const async_overflow_policy overflow_policy, const std::function<void()> &worker_warmup_cb, + const std::chrono::milliseconds &flush_interval_ms, const std::function<void()> &worker_teardown_cb) +{ + return details::registry::instance().create_async( + logger_name, queue_size, overflow_policy, worker_warmup_cb, flush_interval_ms, worker_teardown_cb, sinks_begin, sinks_end); +} + inline void spdlog::set_formatter(spdlog::formatter_ptr f) { - details::registry::instance().formatter(f); + details::registry::instance().formatter(std::move(f)); } -inline void spdlog::set_pattern(const std::string& format_string) +inline void spdlog::set_pattern(const std::string &format_string) { return details::registry::instance().set_pattern(format_string); } @@ -131,8 +245,19 @@ inline void spdlog::set_level(level::level_enum log_level) return details::registry::instance().set_level(log_level); } +inline void spdlog::flush_on(level::level_enum log_level) +{ + return details::registry::instance().flush_on(log_level); +} + +inline void spdlog::set_error_handler(log_err_handler handler) +{ + return details::registry::instance().set_error_handler(std::move(handler)); +} -inline void spdlog::set_async_mode(size_t queue_size, const async_overflow_policy overflow_policy, const std::function<void()>& worker_warmup_cb, const std::chrono::milliseconds& flush_interval_ms, const std::function<void()>& worker_teardown_cb) +inline void spdlog::set_async_mode(size_t queue_size, const async_overflow_policy overflow_policy, + const std::function<void()> &worker_warmup_cb, const std::chrono::milliseconds &flush_interval_ms, + const std::function<void()> &worker_teardown_cb) { details::registry::instance().set_async_mode(queue_size, overflow_policy, worker_warmup_cb, flush_interval_ms, worker_teardown_cb); } @@ -142,6 +267,11 @@ inline void spdlog::set_sync_mode() details::registry::instance().set_sync_mode(); } +inline void spdlog::apply_all(std::function<void(std::shared_ptr<logger>)> fun) +{ + details::registry::instance().apply_all(std::move(fun)); +} + inline void spdlog::drop_all() { details::registry::instance().drop_all(); diff --git a/src/spdlog/fmt/bundled/LICENSE.rst b/src/spdlog/fmt/bundled/LICENSE.rst new file mode 100644 index 0000000000000000000000000000000000000000..eb6be6503e93cc384f912841c7c7cbc65e5aa709 --- /dev/null +++ b/src/spdlog/fmt/bundled/LICENSE.rst @@ -0,0 +1,23 @@ +Copyright (c) 2012 - 2016, Victor Zverovich + +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/src/spdlog/fmt/bundled/format.cc b/src/spdlog/fmt/bundled/format.cc new file mode 100644 index 0000000000000000000000000000000000000000..2d236bc641d98c9c49800f5595f7a8d0ab978438 --- /dev/null +++ b/src/spdlog/fmt/bundled/format.cc @@ -0,0 +1,495 @@ +/* + Formatting library for C++ + + Copyright (c) 2012 - 2016, Victor Zverovich + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + 2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND + ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED + WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR + ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES + (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; + LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND + ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS + SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + */ + +#include "format.h" + +#include <string.h> + +#include <cctype> +#include <cerrno> +#include <climits> +#include <cmath> +#include <cstdarg> +#include <cstddef> // for std::ptrdiff_t + +#if defined(_WIN32) && defined(__MINGW32__) +# include <cstring> +#endif + +#if FMT_USE_WINDOWS_H +# if !defined(FMT_HEADER_ONLY) && !defined(WIN32_LEAN_AND_MEAN) +# define WIN32_LEAN_AND_MEAN +# endif +# if defined(NOMINMAX) || defined(FMT_WIN_MINMAX) +# include <windows.h> +# else +# define NOMINMAX +# include <windows.h> +# undef NOMINMAX +# endif +#endif + +#if FMT_EXCEPTIONS +# define FMT_TRY try +# define FMT_CATCH(x) catch (x) +#else +# define FMT_TRY if (true) +# define FMT_CATCH(x) if (false) +#endif + +#ifdef _MSC_VER +# pragma warning(push) +# pragma warning(disable: 4127) // conditional expression is constant +# pragma warning(disable: 4702) // unreachable code +// Disable deprecation warning for strerror. The latter is not called but +// MSVC fails to detect it. +# pragma warning(disable: 4996) +#endif + +// Dummy implementations of strerror_r and strerror_s called if corresponding +// system functions are not available. +FMT_MAYBE_UNUSED +static inline fmt::internal::Null<> strerror_r(int, char *, ...) { + return fmt::internal::Null<>(); +} +FMT_MAYBE_UNUSED +static inline fmt::internal::Null<> strerror_s(char *, std::size_t, ...) { + return fmt::internal::Null<>(); +} + +namespace fmt { + +FMT_FUNC internal::RuntimeError::~RuntimeError() FMT_DTOR_NOEXCEPT {} +FMT_FUNC FormatError::~FormatError() FMT_DTOR_NOEXCEPT {} +FMT_FUNC SystemError::~SystemError() FMT_DTOR_NOEXCEPT {} + +namespace { + +#ifndef _MSC_VER +# define FMT_SNPRINTF snprintf +#else // _MSC_VER +inline int fmt_snprintf(char *buffer, size_t size, const char *format, ...) { + va_list args; + va_start(args, format); + int result = vsnprintf_s(buffer, size, _TRUNCATE, format, args); + va_end(args); + return result; +} +# define FMT_SNPRINTF fmt_snprintf +#endif // _MSC_VER + +#if defined(_WIN32) && defined(__MINGW32__) && !defined(__NO_ISOCEXT) +# define FMT_SWPRINTF snwprintf +#else +# define FMT_SWPRINTF swprintf +#endif // defined(_WIN32) && defined(__MINGW32__) && !defined(__NO_ISOCEXT) + +const char RESET_COLOR[] = "\x1b[0m"; + +typedef void (*FormatFunc)(Writer &, int, StringRef); + +// Portable thread-safe version of strerror. +// Sets buffer to point to a string describing the error code. +// This can be either a pointer to a string stored in buffer, +// or a pointer to some static immutable string. +// Returns one of the following values: +// 0 - success +// ERANGE - buffer is not large enough to store the error message +// other - failure +// Buffer should be at least of size 1. +int safe_strerror( + int error_code, char *&buffer, std::size_t buffer_size) FMT_NOEXCEPT { + FMT_ASSERT(buffer != FMT_NULL && buffer_size != 0, "invalid buffer"); + + class StrError { + private: + int error_code_; + char *&buffer_; + std::size_t buffer_size_; + + // A noop assignment operator to avoid bogus warnings. + void operator=(const StrError &) {} + + // Handle the result of XSI-compliant version of strerror_r. + int handle(int result) { + // glibc versions before 2.13 return result in errno. + return result == -1 ? errno : result; + } + + // Handle the result of GNU-specific version of strerror_r. + int handle(char *message) { + // If the buffer is full then the message is probably truncated. + if (message == buffer_ && strlen(buffer_) == buffer_size_ - 1) + return ERANGE; + buffer_ = message; + return 0; + } + + // Handle the case when strerror_r is not available. + int handle(internal::Null<>) { + return fallback(strerror_s(buffer_, buffer_size_, error_code_)); + } + + // Fallback to strerror_s when strerror_r is not available. + int fallback(int result) { + // If the buffer is full then the message is probably truncated. + return result == 0 && strlen(buffer_) == buffer_size_ - 1 ? + ERANGE : result; + } + +#ifdef __c2__ +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Wdeprecated-declarations" +#endif + + // Fallback to strerror if strerror_r and strerror_s are not available. + int fallback(internal::Null<>) { + errno = 0; + buffer_ = strerror(error_code_); + return errno; + } + +#ifdef __c2__ +# pragma clang diagnostic pop +#endif + + public: + StrError(int err_code, char *&buf, std::size_t buf_size) + : error_code_(err_code), buffer_(buf), buffer_size_(buf_size) {} + + int run() { + return handle(strerror_r(error_code_, buffer_, buffer_size_)); + } + }; + return StrError(error_code, buffer, buffer_size).run(); +} + +void format_error_code(Writer &out, int error_code, + StringRef message) FMT_NOEXCEPT { + // Report error code making sure that the output fits into + // INLINE_BUFFER_SIZE to avoid dynamic memory allocation and potential + // bad_alloc. + out.clear(); + static const char SEP[] = ": "; + static const char ERROR_STR[] = "error "; + // Subtract 2 to account for terminating null characters in SEP and ERROR_STR. + std::size_t error_code_size = sizeof(SEP) + sizeof(ERROR_STR) - 2; + typedef internal::IntTraits<int>::MainType MainType; + MainType abs_value = static_cast<MainType>(error_code); + if (internal::is_negative(error_code)) { + abs_value = 0 - abs_value; + ++error_code_size; + } + error_code_size += internal::count_digits(abs_value); + if (message.size() <= internal::INLINE_BUFFER_SIZE - error_code_size) + out << message << SEP; + out << ERROR_STR << error_code; + assert(out.size() <= internal::INLINE_BUFFER_SIZE); +} + +void report_error(FormatFunc func, int error_code, + StringRef message) FMT_NOEXCEPT { + MemoryWriter full_message; + func(full_message, error_code, message); + // Use Writer::data instead of Writer::c_str to avoid potential memory + // allocation. + std::fwrite(full_message.data(), full_message.size(), 1, stderr); + std::fputc('\n', stderr); +} +} // namespace + +FMT_FUNC void SystemError::init( + int err_code, CStringRef format_str, ArgList args) { + error_code_ = err_code; + MemoryWriter w; + format_system_error(w, err_code, format(format_str, args)); + std::runtime_error &base = *this; + base = std::runtime_error(w.str()); +} + +template <typename T> +int internal::CharTraits<char>::format_float( + char *buffer, std::size_t size, const char *format, + unsigned width, int precision, T value) { + if (width == 0) { + return precision < 0 ? + FMT_SNPRINTF(buffer, size, format, value) : + FMT_SNPRINTF(buffer, size, format, precision, value); + } + return precision < 0 ? + FMT_SNPRINTF(buffer, size, format, width, value) : + FMT_SNPRINTF(buffer, size, format, width, precision, value); +} + +template <typename T> +int internal::CharTraits<wchar_t>::format_float( + wchar_t *buffer, std::size_t size, const wchar_t *format, + unsigned width, int precision, T value) { + if (width == 0) { + return precision < 0 ? + FMT_SWPRINTF(buffer, size, format, value) : + FMT_SWPRINTF(buffer, size, format, precision, value); + } + return precision < 0 ? + FMT_SWPRINTF(buffer, size, format, width, value) : + FMT_SWPRINTF(buffer, size, format, width, precision, value); +} + +template <typename T> +const char internal::BasicData<T>::DIGITS[] = + "0001020304050607080910111213141516171819" + "2021222324252627282930313233343536373839" + "4041424344454647484950515253545556575859" + "6061626364656667686970717273747576777879" + "8081828384858687888990919293949596979899"; + +#define FMT_POWERS_OF_10(factor) \ + factor * 10, \ + factor * 100, \ + factor * 1000, \ + factor * 10000, \ + factor * 100000, \ + factor * 1000000, \ + factor * 10000000, \ + factor * 100000000, \ + factor * 1000000000 + +template <typename T> +const uint32_t internal::BasicData<T>::POWERS_OF_10_32[] = { + 0, FMT_POWERS_OF_10(1) +}; + +template <typename T> +const uint64_t internal::BasicData<T>::POWERS_OF_10_64[] = { + 0, + FMT_POWERS_OF_10(1), + FMT_POWERS_OF_10(ULongLong(1000000000)), + // Multiply several constants instead of using a single long long constant + // to avoid warnings about C++98 not supporting long long. + ULongLong(1000000000) * ULongLong(1000000000) * 10 +}; + +FMT_FUNC void internal::report_unknown_type(char code, const char *type) { + (void)type; + if (std::isprint(static_cast<unsigned char>(code))) { + FMT_THROW(FormatError( + format("unknown format code '{}' for {}", code, type))); + } + FMT_THROW(FormatError( + format("unknown format code '\\x{:02x}' for {}", + static_cast<unsigned>(code), type))); +} + +#if FMT_USE_WINDOWS_H + +FMT_FUNC internal::UTF8ToUTF16::UTF8ToUTF16(StringRef s) { + static const char ERROR_MSG[] = "cannot convert string from UTF-8 to UTF-16"; + if (s.size() > INT_MAX) + FMT_THROW(WindowsError(ERROR_INVALID_PARAMETER, ERROR_MSG)); + int s_size = static_cast<int>(s.size()); + int length = MultiByteToWideChar( + CP_UTF8, MB_ERR_INVALID_CHARS, s.data(), s_size, FMT_NULL, 0); + if (length == 0) + FMT_THROW(WindowsError(GetLastError(), ERROR_MSG)); + buffer_.resize(length + 1); + length = MultiByteToWideChar( + CP_UTF8, MB_ERR_INVALID_CHARS, s.data(), s_size, &buffer_[0], length); + if (length == 0) + FMT_THROW(WindowsError(GetLastError(), ERROR_MSG)); + buffer_[length] = 0; +} + +FMT_FUNC internal::UTF16ToUTF8::UTF16ToUTF8(WStringRef s) { + if (int error_code = convert(s)) { + FMT_THROW(WindowsError(error_code, + "cannot convert string from UTF-16 to UTF-8")); + } +} + +FMT_FUNC int internal::UTF16ToUTF8::convert(WStringRef s) { + if (s.size() > INT_MAX) + return ERROR_INVALID_PARAMETER; + int s_size = static_cast<int>(s.size()); + int length = WideCharToMultiByte( + CP_UTF8, 0, s.data(), s_size, FMT_NULL, 0, FMT_NULL, FMT_NULL); + if (length == 0) + return GetLastError(); + buffer_.resize(length + 1); + length = WideCharToMultiByte( + CP_UTF8, 0, s.data(), s_size, &buffer_[0], length, FMT_NULL, FMT_NULL); + if (length == 0) + return GetLastError(); + buffer_[length] = 0; + return 0; +} + +FMT_FUNC void WindowsError::init( + int err_code, CStringRef format_str, ArgList args) { + error_code_ = err_code; + MemoryWriter w; + internal::format_windows_error(w, err_code, format(format_str, args)); + std::runtime_error &base = *this; + base = std::runtime_error(w.str()); +} + +FMT_FUNC void internal::format_windows_error( + Writer &out, int error_code, StringRef message) FMT_NOEXCEPT { + FMT_TRY { + MemoryBuffer<wchar_t, INLINE_BUFFER_SIZE> buffer; + buffer.resize(INLINE_BUFFER_SIZE); + for (;;) { + wchar_t *system_message = &buffer[0]; + int result = FormatMessageW( + FORMAT_MESSAGE_FROM_SYSTEM | FORMAT_MESSAGE_IGNORE_INSERTS, + FMT_NULL, error_code, MAKELANGID(LANG_NEUTRAL, SUBLANG_DEFAULT), + system_message, static_cast<uint32_t>(buffer.size()), FMT_NULL); + if (result != 0) { + UTF16ToUTF8 utf8_message; + if (utf8_message.convert(system_message) == ERROR_SUCCESS) { + out << message << ": " << utf8_message; + return; + } + break; + } + if (GetLastError() != ERROR_INSUFFICIENT_BUFFER) + break; // Can't get error message, report error code instead. + buffer.resize(buffer.size() * 2); + } + } FMT_CATCH(...) {} + fmt::format_error_code(out, error_code, message); // 'fmt::' is for bcc32. +} + +#endif // FMT_USE_WINDOWS_H + +FMT_FUNC void format_system_error( + Writer &out, int error_code, StringRef message) FMT_NOEXCEPT { + FMT_TRY { + internal::MemoryBuffer<char, internal::INLINE_BUFFER_SIZE> buffer; + buffer.resize(internal::INLINE_BUFFER_SIZE); + for (;;) { + char *system_message = &buffer[0]; + int result = safe_strerror(error_code, system_message, buffer.size()); + if (result == 0) { + out << message << ": " << system_message; + return; + } + if (result != ERANGE) + break; // Can't get error message, report error code instead. + buffer.resize(buffer.size() * 2); + } + } FMT_CATCH(...) {} + fmt::format_error_code(out, error_code, message); // 'fmt::' is for bcc32. +} + +template <typename Char> +void internal::FixedBuffer<Char>::grow(std::size_t) { + FMT_THROW(std::runtime_error("buffer overflow")); +} + +FMT_FUNC internal::Arg internal::FormatterBase::do_get_arg( + unsigned arg_index, const char *&error) { + internal::Arg arg = args_[arg_index]; + switch (arg.type) { + case internal::Arg::NONE: + error = "argument index out of range"; + break; + case internal::Arg::NAMED_ARG: + arg = *static_cast<const internal::Arg*>(arg.pointer); + break; + default: + /*nothing*/; + } + return arg; +} + +FMT_FUNC void report_system_error( + int error_code, fmt::StringRef message) FMT_NOEXCEPT { + // 'fmt::' is for bcc32. + report_error(format_system_error, error_code, message); +} + +#if FMT_USE_WINDOWS_H +FMT_FUNC void report_windows_error( + int error_code, fmt::StringRef message) FMT_NOEXCEPT { + // 'fmt::' is for bcc32. + report_error(internal::format_windows_error, error_code, message); +} +#endif + +FMT_FUNC void print(std::FILE *f, CStringRef format_str, ArgList args) { + MemoryWriter w; + w.write(format_str, args); + std::fwrite(w.data(), 1, w.size(), f); +} + +FMT_FUNC void print(CStringRef format_str, ArgList args) { + print(stdout, format_str, args); +} + +FMT_FUNC void print_colored(Color c, CStringRef format, ArgList args) { + char escape[] = "\x1b[30m"; + escape[3] = static_cast<char>('0' + c); + std::fputs(escape, stdout); + print(format, args); + std::fputs(RESET_COLOR, stdout); +} + +#ifndef FMT_HEADER_ONLY + +template struct internal::BasicData<void>; + +// Explicit instantiations for char. + +template void internal::FixedBuffer<char>::grow(std::size_t); + +template FMT_API int internal::CharTraits<char>::format_float( + char *buffer, std::size_t size, const char *format, + unsigned width, int precision, double value); + +template FMT_API int internal::CharTraits<char>::format_float( + char *buffer, std::size_t size, const char *format, + unsigned width, int precision, long double value); + +// Explicit instantiations for wchar_t. + +template void internal::FixedBuffer<wchar_t>::grow(std::size_t); + +template FMT_API int internal::CharTraits<wchar_t>::format_float( + wchar_t *buffer, std::size_t size, const wchar_t *format, + unsigned width, int precision, double value); + +template FMT_API int internal::CharTraits<wchar_t>::format_float( + wchar_t *buffer, std::size_t size, const wchar_t *format, + unsigned width, int precision, long double value); + +#endif // FMT_HEADER_ONLY + +} // namespace fmt + +#ifdef _MSC_VER +# pragma warning(pop) +#endif diff --git a/src/spdlog/fmt/bundled/format.h b/src/spdlog/fmt/bundled/format.h new file mode 100644 index 0000000000000000000000000000000000000000..1396d6b37f91eb71e382b6bd45e4dd263031aa97 --- /dev/null +++ b/src/spdlog/fmt/bundled/format.h @@ -0,0 +1,4769 @@ +/* + Formatting library for C++ + + Copyright (c) 2012 - 2016, Victor Zverovich + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + 2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND + ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED + WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR + ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES + (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; + LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND + ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS + SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + */ + +#ifndef FMT_FORMAT_H_ +#define FMT_FORMAT_H_ + +#define FMT_INCLUDE +#include <cassert> +#include <clocale> +#include <cmath> +#include <cstdio> +#include <cstring> +#include <limits> +#include <memory> +#include <stdexcept> +#include <string> +#include <utility> // for std::pair +#include <vector> +#undef FMT_INCLUDE + +// The fmt library version in the form major * 10000 + minor * 100 + patch. +#define FMT_VERSION 40100 + +#if defined(__has_include) +#define FMT_HAS_INCLUDE(x) __has_include(x) +#else +#define FMT_HAS_INCLUDE(x) 0 +#endif + +#if (FMT_HAS_INCLUDE(<string_view>) && __cplusplus > 201402L) || (defined(_MSVC_LANG) && _MSVC_LANG > 201402L && _MSC_VER >= 1910) +#include <string_view> +#define FMT_HAS_STRING_VIEW 1 +#else +#define FMT_HAS_STRING_VIEW 0 +#endif + +#if defined _SECURE_SCL && _SECURE_SCL +#define FMT_SECURE_SCL _SECURE_SCL +#else +#define FMT_SECURE_SCL 0 +#endif + +#if FMT_SECURE_SCL +#include <iterator> +#endif + +#ifdef _MSC_VER +#define FMT_MSC_VER _MSC_VER +#else +#define FMT_MSC_VER 0 +#endif + +#if FMT_MSC_VER && FMT_MSC_VER <= 1500 +typedef unsigned __int32 uint32_t; +typedef unsigned __int64 uint64_t; +typedef __int64 intmax_t; +#else +#include <stdint.h> +#endif + +#if !defined(FMT_HEADER_ONLY) && defined(_WIN32) +#ifdef FMT_EXPORT +#define FMT_API __declspec(dllexport) +#elif defined(FMT_SHARED) +#define FMT_API __declspec(dllimport) +#endif +#endif +#ifndef FMT_API +#define FMT_API +#endif + +#ifdef __GNUC__ +#define FMT_GCC_VERSION (__GNUC__ * 100 + __GNUC_MINOR__) +#define FMT_GCC_EXTENSION __extension__ +#if FMT_GCC_VERSION >= 406 +#pragma GCC diagnostic push +// Disable the warning about "long long" which is sometimes reported even +// when using __extension__. +#pragma GCC diagnostic ignored "-Wlong-long" +// Disable the warning about declaration shadowing because it affects too +// many valid cases. +#pragma GCC diagnostic ignored "-Wshadow" +// Disable the warning about implicit conversions that may change the sign of +// an integer; silencing it otherwise would require many explicit casts. +#pragma GCC diagnostic ignored "-Wsign-conversion" +#endif +#if __cplusplus >= 201103L || defined __GXX_EXPERIMENTAL_CXX0X__ +#define FMT_HAS_GXX_CXX11 1 +#endif +#else +#define FMT_GCC_VERSION 0 +#define FMT_GCC_EXTENSION +#define FMT_HAS_GXX_CXX11 0 +#endif + +#if defined(__INTEL_COMPILER) +#define FMT_ICC_VERSION __INTEL_COMPILER +#elif defined(__ICL) +#define FMT_ICC_VERSION __ICL +#endif + +#if defined(__clang__) && !defined(FMT_ICC_VERSION) +#define FMT_CLANG_VERSION (__clang_major__ * 100 + __clang_minor__) +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wdocumentation-unknown-command" +#pragma clang diagnostic ignored "-Wpadded" +#endif + +#ifdef __GNUC_LIBSTD__ +#define FMT_GNUC_LIBSTD_VERSION (__GNUC_LIBSTD__ * 100 + __GNUC_LIBSTD_MINOR__) +#endif + +#ifdef __has_feature +#define FMT_HAS_FEATURE(x) __has_feature(x) +#else +#define FMT_HAS_FEATURE(x) 0 +#endif + +#ifdef __has_builtin +#define FMT_HAS_BUILTIN(x) __has_builtin(x) +#else +#define FMT_HAS_BUILTIN(x) 0 +#endif + +#ifdef __has_cpp_attribute +#define FMT_HAS_CPP_ATTRIBUTE(x) __has_cpp_attribute(x) +#else +#define FMT_HAS_CPP_ATTRIBUTE(x) 0 +#endif + +#if FMT_HAS_CPP_ATTRIBUTE(maybe_unused) +#define FMT_HAS_CXX17_ATTRIBUTE_MAYBE_UNUSED +// VC++ 1910 support /std: option and that will set _MSVC_LANG macro +// Clang with Microsoft CodeGen doesn't define _MSVC_LANG macro +#elif defined(_MSVC_LANG) && _MSVC_LANG > 201402 && _MSC_VER >= 1910 +#define FMT_HAS_CXX17_ATTRIBUTE_MAYBE_UNUSED +#endif + +#ifdef FMT_HAS_CXX17_ATTRIBUTE_MAYBE_UNUSED +#define FMT_MAYBE_UNUSED [[maybe_unused]] +// g++/clang++ also support [[gnu::unused]]. However, we don't use it. +#elif defined(__GNUC__) +#define FMT_MAYBE_UNUSED __attribute__((unused)) +#else +#define FMT_MAYBE_UNUSED +#endif + +// Use the compiler's attribute noreturn +#if defined(__MINGW32__) || defined(__MINGW64__) +#define FMT_NORETURN __attribute__((noreturn)) +#elif FMT_HAS_CPP_ATTRIBUTE(noreturn) && __cplusplus >= 201103L +#define FMT_NORETURN [[noreturn]] +#else +#define FMT_NORETURN +#endif + +#ifndef FMT_USE_VARIADIC_TEMPLATES +// Variadic templates are available in GCC since version 4.4 +// (http://gcc.gnu.org/projects/cxx0x.html) and in Visual C++ +// since version 2013. +#define FMT_USE_VARIADIC_TEMPLATES \ + (FMT_HAS_FEATURE(cxx_variadic_templates) || (FMT_GCC_VERSION >= 404 && FMT_HAS_GXX_CXX11) || FMT_MSC_VER >= 1800) +#endif + +#ifndef FMT_USE_RVALUE_REFERENCES +// Don't use rvalue references when compiling with clang and an old libstdc++ +// as the latter doesn't provide std::move. +#if defined(FMT_GNUC_LIBSTD_VERSION) && FMT_GNUC_LIBSTD_VERSION <= 402 +#define FMT_USE_RVALUE_REFERENCES 0 +#else +#define FMT_USE_RVALUE_REFERENCES \ + (FMT_HAS_FEATURE(cxx_rvalue_references) || (FMT_GCC_VERSION >= 403 && FMT_HAS_GXX_CXX11) || FMT_MSC_VER >= 1600) +#endif +#endif + +#if __cplusplus >= 201103L || FMT_MSC_VER >= 1700 +#define FMT_USE_ALLOCATOR_TRAITS 1 +#else +#define FMT_USE_ALLOCATOR_TRAITS 0 +#endif + +// Check if exceptions are disabled. +#if defined(__GNUC__) && !defined(__EXCEPTIONS) +#define FMT_EXCEPTIONS 0 +#endif +#if FMT_MSC_VER && !_HAS_EXCEPTIONS +#define FMT_EXCEPTIONS 0 +#endif +#ifndef FMT_EXCEPTIONS +#define FMT_EXCEPTIONS 1 +#endif + +#ifndef FMT_THROW +#if FMT_EXCEPTIONS +#define FMT_THROW(x) throw x +#else +#define FMT_THROW(x) assert(false) +#endif +#endif + +// Define FMT_USE_NOEXCEPT to make fmt use noexcept (C++11 feature). +#ifndef FMT_USE_NOEXCEPT +#define FMT_USE_NOEXCEPT 0 +#endif + +#if FMT_USE_NOEXCEPT || FMT_HAS_FEATURE(cxx_noexcept) || (FMT_GCC_VERSION >= 408 && FMT_HAS_GXX_CXX11) || FMT_MSC_VER >= 1900 +#define FMT_DETECTED_NOEXCEPT noexcept +#else +#define FMT_DETECTED_NOEXCEPT throw() +#endif + +#ifndef FMT_NOEXCEPT +#if FMT_EXCEPTIONS +#define FMT_NOEXCEPT FMT_DETECTED_NOEXCEPT +#else +#define FMT_NOEXCEPT +#endif +#endif + +// This is needed because GCC still uses throw() in its headers when exceptions +// are disabled. +#if FMT_GCC_VERSION +#define FMT_DTOR_NOEXCEPT FMT_DETECTED_NOEXCEPT +#else +#define FMT_DTOR_NOEXCEPT FMT_NOEXCEPT +#endif + +#ifndef FMT_OVERRIDE +#if (defined(FMT_USE_OVERRIDE) && FMT_USE_OVERRIDE) || FMT_HAS_FEATURE(cxx_override) || (FMT_GCC_VERSION >= 408 && FMT_HAS_GXX_CXX11) || \ + FMT_MSC_VER >= 1900 +#define FMT_OVERRIDE override +#else +#define FMT_OVERRIDE +#endif +#endif + +#ifndef FMT_NULL +#if FMT_HAS_FEATURE(cxx_nullptr) || (FMT_GCC_VERSION >= 408 && FMT_HAS_GXX_CXX11) || FMT_MSC_VER >= 1600 +#define FMT_NULL nullptr +#else +#define FMT_NULL NULL +#endif +#endif + +// A macro to disallow the copy constructor and operator= functions +// This should be used in the private: declarations for a class +#ifndef FMT_USE_DELETED_FUNCTIONS +#define FMT_USE_DELETED_FUNCTIONS 0 +#endif + +#if FMT_USE_DELETED_FUNCTIONS || FMT_HAS_FEATURE(cxx_deleted_functions) || (FMT_GCC_VERSION >= 404 && FMT_HAS_GXX_CXX11) || \ + FMT_MSC_VER >= 1800 +#define FMT_DELETED_OR_UNDEFINED = delete +#define FMT_DISALLOW_COPY_AND_ASSIGN(TypeName) \ + TypeName(const TypeName &) = delete; \ + TypeName &operator=(const TypeName &) = delete +#else +#define FMT_DELETED_OR_UNDEFINED +#define FMT_DISALLOW_COPY_AND_ASSIGN(TypeName) \ + TypeName(const TypeName &); \ + TypeName &operator=(const TypeName &) +#endif + +#ifndef FMT_USE_DEFAULTED_FUNCTIONS +#define FMT_USE_DEFAULTED_FUNCTIONS 0 +#endif + +#ifndef FMT_DEFAULTED_COPY_CTOR +#if FMT_USE_DEFAULTED_FUNCTIONS || FMT_HAS_FEATURE(cxx_defaulted_functions) || (FMT_GCC_VERSION >= 404 && FMT_HAS_GXX_CXX11) || \ + FMT_MSC_VER >= 1800 +#define FMT_DEFAULTED_COPY_CTOR(TypeName) TypeName(const TypeName &) = default; +#else +#define FMT_DEFAULTED_COPY_CTOR(TypeName) +#endif +#endif + +#ifndef FMT_USE_USER_DEFINED_LITERALS +// All compilers which support UDLs also support variadic templates. This +// makes the fmt::literals implementation easier. However, an explicit check +// for variadic templates is added here just in case. +// For Intel's compiler both it and the system gcc/msc must support UDLs. +#if FMT_USE_VARIADIC_TEMPLATES && FMT_USE_RVALUE_REFERENCES && \ + (FMT_HAS_FEATURE(cxx_user_literals) || (FMT_GCC_VERSION >= 407 && FMT_HAS_GXX_CXX11) || FMT_MSC_VER >= 1900) && \ + (!defined(FMT_ICC_VERSION) || FMT_ICC_VERSION >= 1500) +#define FMT_USE_USER_DEFINED_LITERALS 1 +#else +#define FMT_USE_USER_DEFINED_LITERALS 0 +#endif +#endif + +#ifndef FMT_USE_EXTERN_TEMPLATES +#define FMT_USE_EXTERN_TEMPLATES (FMT_CLANG_VERSION >= 209 || (FMT_GCC_VERSION >= 303 && FMT_HAS_GXX_CXX11)) +#endif + +#ifdef FMT_HEADER_ONLY +// If header only do not use extern templates. +#undef FMT_USE_EXTERN_TEMPLATES +#define FMT_USE_EXTERN_TEMPLATES 0 +#endif + +#ifndef FMT_ASSERT +#define FMT_ASSERT(condition, message) assert((condition) && message) +#endif + +// __builtin_clz is broken in clang with Microsoft CodeGen: +// https://github.com/fmtlib/fmt/issues/519 +#ifndef _MSC_VER +#if FMT_GCC_VERSION >= 400 || FMT_HAS_BUILTIN(__builtin_clz) +#define FMT_BUILTIN_CLZ(n) __builtin_clz(n) +#endif + +#if FMT_GCC_VERSION >= 400 || FMT_HAS_BUILTIN(__builtin_clzll) +#define FMT_BUILTIN_CLZLL(n) __builtin_clzll(n) +#endif +#endif + +// Some compilers masquerade as both MSVC and GCC-likes or +// otherwise support __builtin_clz and __builtin_clzll, so +// only define FMT_BUILTIN_CLZ using the MSVC intrinsics +// if the clz and clzll builtins are not available. +#if FMT_MSC_VER && !defined(FMT_BUILTIN_CLZLL) && !defined(_MANAGED) +#include <intrin.h> // _BitScanReverse, _BitScanReverse64 + +namespace fmt { +namespace internal { +// avoid Clang with Microsoft CodeGen's -Wunknown-pragmas warning +#ifndef __clang__ +#pragma intrinsic(_BitScanReverse) +#endif +inline uint32_t clz(uint32_t x) +{ + unsigned long r = 0; + _BitScanReverse(&r, x); + + assert(x != 0); + // Static analysis complains about using uninitialized data + // "r", but the only way that can happen is if "x" is 0, + // which the callers guarantee to not happen. +#pragma warning(suppress : 6102) + return 31 - r; +} +#define FMT_BUILTIN_CLZ(n) fmt::internal::clz(n) + +// avoid Clang with Microsoft CodeGen's -Wunknown-pragmas warning +#if defined(_WIN64) && !defined(__clang__) +#pragma intrinsic(_BitScanReverse64) +#endif + +inline uint32_t clzll(uint64_t x) +{ + unsigned long r = 0; +#ifdef _WIN64 + _BitScanReverse64(&r, x); +#else + // Scan the high 32 bits. + if (_BitScanReverse(&r, static_cast<uint32_t>(x >> 32))) + return 63 - (r + 32); + + // Scan the low 32 bits. + _BitScanReverse(&r, static_cast<uint32_t>(x)); +#endif + + assert(x != 0); + // Static analysis complains about using uninitialized data + // "r", but the only way that can happen is if "x" is 0, + // which the callers guarantee to not happen. +#pragma warning(suppress : 6102) + return 63 - r; +} +#define FMT_BUILTIN_CLZLL(n) fmt::internal::clzll(n) +} +} // namespace fmt +#endif + +namespace fmt { namespace internal { +struct DummyInt +{ + int data[2]; + operator int() const + { + return 0; + } +}; +typedef std::numeric_limits<fmt::internal::DummyInt> FPUtil; + +// Dummy implementations of system functions such as signbit and ecvt called +// if the latter are not available. +inline DummyInt signbit(...) +{ + return DummyInt(); +} +inline DummyInt _ecvt_s(...) +{ + return DummyInt(); +} +inline DummyInt isinf(...) +{ + return DummyInt(); +} +inline DummyInt _finite(...) +{ + return DummyInt(); +} +inline DummyInt isnan(...) +{ + return DummyInt(); +} +inline DummyInt _isnan(...) +{ + return DummyInt(); +} + +// A helper function to suppress bogus "conditional expression is constant" +// warnings. +template <typename T> inline T const_check(T value) +{ + return value; +} +}} // namespace fmt::internal + +namespace std { +// Standard permits specialization of std::numeric_limits. This specialization +// is used to resolve ambiguity between isinf and std::isinf in glibc: +// https://gcc.gnu.org/bugzilla/show_bug.cgi?id=48891 +// and the same for isnan and signbit. +template <> class numeric_limits<fmt::internal::DummyInt> : public std::numeric_limits<int> +{ +public: + // Portable version of isinf. + template <typename T> static bool isinfinity(T x) + { + using namespace fmt::internal; + // The resolution "priority" is: + // isinf macro > std::isinf > ::isinf > fmt::internal::isinf + if (const_check(sizeof(isinf(x)) == sizeof(bool) || sizeof(isinf(x)) == sizeof(int))) + { + return isinf(x) != 0; + } + return !_finite(static_cast<double>(x)); + } + + // Portable version of isnan. + template <typename T> static bool isnotanumber(T x) + { + using namespace fmt::internal; + if (const_check(sizeof(isnan(x)) == sizeof(bool) || sizeof(isnan(x)) == sizeof(int))) + { + return isnan(x) != 0; + } + return _isnan(static_cast<double>(x)) != 0; + } + + // Portable version of signbit. + static bool isnegative(double x) + { + using namespace fmt::internal; + if (const_check(sizeof(signbit(x)) == sizeof(bool) || sizeof(signbit(x)) == sizeof(int))) + { + return signbit(x) != 0; + } + if (x < 0) + return true; + if (!isnotanumber(x)) + return false; + int dec = 0, sign = 0; + char buffer[2]; // The buffer size must be >= 2 or _ecvt_s will fail. + _ecvt_s(buffer, sizeof(buffer), x, 0, &dec, &sign); + return sign != 0; + } +}; +} // namespace std + +namespace fmt { + +// Fix the warning about long long on older versions of GCC +// that don't support the diagnostic pragma. +FMT_GCC_EXTENSION typedef long long LongLong; +FMT_GCC_EXTENSION typedef unsigned long long ULongLong; + +#if FMT_USE_RVALUE_REFERENCES +using std::move; +#endif + +template <typename Char> class BasicWriter; + +typedef BasicWriter<char> Writer; +typedef BasicWriter<wchar_t> WWriter; + +template <typename Char> class ArgFormatter; + +struct FormatSpec; + +template <typename Impl, typename Char, typename Spec = fmt::FormatSpec> class BasicPrintfArgFormatter; + +template <typename CharType, typename ArgFormatter = fmt::ArgFormatter<CharType>> class BasicFormatter; + +/** + \rst + A string reference. It can be constructed from a C string or + ``std::basic_string``. + + You can use one of the following typedefs for common character types: + + +------------+-------------------------+ + | Type | Definition | + +============+=========================+ + | StringRef | BasicStringRef<char> | + +------------+-------------------------+ + | WStringRef | BasicStringRef<wchar_t> | + +------------+-------------------------+ + + This class is most useful as a parameter type to allow passing + different types of strings to a function, for example:: + + template <typename... Args> + std::string format(StringRef format_str, const Args & ... args); + + format("{}", 42); + format(std::string("{}"), 42); + \endrst + */ +template <typename Char> class BasicStringRef +{ +private: + const Char *data_; + std::size_t size_; + +public: + /** Constructs a string reference object from a C string and a size. */ + BasicStringRef(const Char *s, std::size_t size) + : data_(s) + , size_(size) + { + } + + /** + \rst + Constructs a string reference object from a C string computing + the size with ``std::char_traits<Char>::length``. + \endrst + */ + BasicStringRef(const Char *s) + : data_(s) + , size_(std::char_traits<Char>::length(s)) + { + } + + /** + \rst + Constructs a string reference from a ``std::basic_string`` object. + \endrst + */ + template <typename Allocator> + BasicStringRef(const std::basic_string<Char, std::char_traits<Char>, Allocator> &s) + : data_(s.c_str()) + , size_(s.size()) + { + } + +#if FMT_HAS_STRING_VIEW + /** + \rst + Constructs a string reference from a ``std::basic_string_view`` object. + \endrst + */ + BasicStringRef(const std::basic_string_view<Char, std::char_traits<Char>> &s) + : data_(s.data()) + , size_(s.size()) + { + } + + /** + \rst + Converts a string reference to an ``std::string_view`` object. + \endrst + */ + explicit operator std::basic_string_view<Char>() const FMT_NOEXCEPT + { + return std::basic_string_view<Char>(data_, size_); + } +#endif + + /** + \rst + Converts a string reference to an ``std::string`` object. + \endrst + */ + std::basic_string<Char> to_string() const + { + return std::basic_string<Char>(data_, size_); + } + + /** Returns a pointer to the string data. */ + const Char *data() const + { + return data_; + } + + /** Returns the string size. */ + std::size_t size() const + { + return size_; + } + + // Lexicographically compare this string reference to other. + int compare(BasicStringRef other) const + { + std::size_t size = size_ < other.size_ ? size_ : other.size_; + int result = std::char_traits<Char>::compare(data_, other.data_, size); + if (result == 0) + result = size_ == other.size_ ? 0 : (size_ < other.size_ ? -1 : 1); + return result; + } + + friend bool operator==(BasicStringRef lhs, BasicStringRef rhs) + { + return lhs.compare(rhs) == 0; + } + friend bool operator!=(BasicStringRef lhs, BasicStringRef rhs) + { + return lhs.compare(rhs) != 0; + } + friend bool operator<(BasicStringRef lhs, BasicStringRef rhs) + { + return lhs.compare(rhs) < 0; + } + friend bool operator<=(BasicStringRef lhs, BasicStringRef rhs) + { + return lhs.compare(rhs) <= 0; + } + friend bool operator>(BasicStringRef lhs, BasicStringRef rhs) + { + return lhs.compare(rhs) > 0; + } + friend bool operator>=(BasicStringRef lhs, BasicStringRef rhs) + { + return lhs.compare(rhs) >= 0; + } +}; + +typedef BasicStringRef<char> StringRef; +typedef BasicStringRef<wchar_t> WStringRef; + +/** + \rst + A reference to a null terminated string. It can be constructed from a C + string or ``std::basic_string``. + + You can use one of the following typedefs for common character types: + + +-------------+--------------------------+ + | Type | Definition | + +=============+==========================+ + | CStringRef | BasicCStringRef<char> | + +-------------+--------------------------+ + | WCStringRef | BasicCStringRef<wchar_t> | + +-------------+--------------------------+ + + This class is most useful as a parameter type to allow passing + different types of strings to a function, for example:: + + template <typename... Args> + std::string format(CStringRef format_str, const Args & ... args); + + format("{}", 42); + format(std::string("{}"), 42); + \endrst + */ +template <typename Char> class BasicCStringRef +{ +private: + const Char *data_; + +public: + /** Constructs a string reference object from a C string. */ + BasicCStringRef(const Char *s) + : data_(s) + { + } + + /** + \rst + Constructs a string reference from a ``std::basic_string`` object. + \endrst + */ + template <typename Allocator> + BasicCStringRef(const std::basic_string<Char, std::char_traits<Char>, Allocator> &s) + : data_(s.c_str()) + { + } + + /** Returns the pointer to a C string. */ + const Char *c_str() const + { + return data_; + } +}; + +typedef BasicCStringRef<char> CStringRef; +typedef BasicCStringRef<wchar_t> WCStringRef; + +/** A formatting error such as invalid format string. */ +class FormatError : public std::runtime_error +{ +public: + explicit FormatError(CStringRef message) + : std::runtime_error(message.c_str()) + { + } + FormatError(const FormatError &ferr) + : std::runtime_error(ferr) + { + } + FMT_API ~FormatError() FMT_DTOR_NOEXCEPT FMT_OVERRIDE; +}; + +namespace internal { + +// MakeUnsigned<T>::Type gives an unsigned type corresponding to integer type T. +template <typename T> struct MakeUnsigned +{ + typedef T Type; +}; + +#define FMT_SPECIALIZE_MAKE_UNSIGNED(T, U) \ + template <> struct MakeUnsigned<T> \ + { \ + typedef U Type; \ + } + +FMT_SPECIALIZE_MAKE_UNSIGNED(char, unsigned char); +FMT_SPECIALIZE_MAKE_UNSIGNED(signed char, unsigned char); +FMT_SPECIALIZE_MAKE_UNSIGNED(short, unsigned short); +FMT_SPECIALIZE_MAKE_UNSIGNED(int, unsigned); +FMT_SPECIALIZE_MAKE_UNSIGNED(long, unsigned long); +FMT_SPECIALIZE_MAKE_UNSIGNED(LongLong, ULongLong); + +// Casts nonnegative integer to unsigned. +template <typename Int> inline typename MakeUnsigned<Int>::Type to_unsigned(Int value) +{ + FMT_ASSERT(value >= 0, "negative value"); + return static_cast<typename MakeUnsigned<Int>::Type>(value); +} + +// The number of characters to store in the MemoryBuffer object itself +// to avoid dynamic memory allocation. +enum +{ + INLINE_BUFFER_SIZE = 500 +}; + +#if FMT_SECURE_SCL +// Use checked iterator to avoid warnings on MSVC. +template <typename T> inline stdext::checked_array_iterator<T *> make_ptr(T *ptr, std::size_t size) +{ + return stdext::checked_array_iterator<T *>(ptr, size); +} +#else +template <typename T> inline T *make_ptr(T *ptr, std::size_t) +{ + return ptr; +} +#endif +} // namespace internal + +/** + \rst + A buffer supporting a subset of ``std::vector``'s operations. + \endrst + */ +template <typename T> class Buffer +{ +private: + FMT_DISALLOW_COPY_AND_ASSIGN(Buffer); + +protected: + T *ptr_; + std::size_t size_; + std::size_t capacity_; + + Buffer(T *ptr = FMT_NULL, std::size_t capacity = 0) + : ptr_(ptr) + , size_(0) + , capacity_(capacity) + { + } + + /** + \rst + Increases the buffer capacity to hold at least *size* elements updating + ``ptr_`` and ``capacity_``. + \endrst + */ + virtual void grow(std::size_t size) = 0; + +public: + virtual ~Buffer() {} + + /** Returns the size of this buffer. */ + std::size_t size() const + { + return size_; + } + + /** Returns the capacity of this buffer. */ + std::size_t capacity() const + { + return capacity_; + } + + /** + Resizes the buffer. If T is a POD type new elements may not be initialized. + */ + void resize(std::size_t new_size) + { + if (new_size > capacity_) + grow(new_size); + size_ = new_size; + } + + /** + \rst + Reserves space to store at least *capacity* elements. + \endrst + */ + void reserve(std::size_t capacity) + { + if (capacity > capacity_) + grow(capacity); + } + + void clear() FMT_NOEXCEPT + { + size_ = 0; + } + + void push_back(const T &value) + { + if (size_ == capacity_) + grow(size_ + 1); + ptr_[size_++] = value; + } + + /** Appends data to the end of the buffer. */ + template <typename U> void append(const U *begin, const U *end); + + T &operator[](std::size_t index) + { + return ptr_[index]; + } + const T &operator[](std::size_t index) const + { + return ptr_[index]; + } +}; + +template <typename T> template <typename U> void Buffer<T>::append(const U *begin, const U *end) +{ + FMT_ASSERT(end >= begin, "negative value"); + std::size_t new_size = size_ + static_cast<std::size_t>(end - begin); + if (new_size > capacity_) + grow(new_size); + std::uninitialized_copy(begin, end, internal::make_ptr(ptr_, capacity_) + size_); + size_ = new_size; +} + +namespace internal { + +// A memory buffer for trivially copyable/constructible types with the first +// SIZE elements stored in the object itself. +template <typename T, std::size_t SIZE, typename Allocator = std::allocator<T>> class MemoryBuffer : private Allocator, public Buffer<T> +{ +private: + T data_[SIZE]; + + // Deallocate memory allocated by the buffer. + void deallocate() + { + if (this->ptr_ != data_) + Allocator::deallocate(this->ptr_, this->capacity_); + } + +protected: + void grow(std::size_t size) FMT_OVERRIDE; + +public: + explicit MemoryBuffer(const Allocator &alloc = Allocator()) + : Allocator(alloc) + , Buffer<T>(data_, SIZE) + { + } + ~MemoryBuffer() FMT_OVERRIDE + { + deallocate(); + } + +#if FMT_USE_RVALUE_REFERENCES +private: + // Move data from other to this buffer. + void move(MemoryBuffer &other) + { + Allocator &this_alloc = *this, &other_alloc = other; + this_alloc = std::move(other_alloc); + this->size_ = other.size_; + this->capacity_ = other.capacity_; + if (other.ptr_ == other.data_) + { + this->ptr_ = data_; + std::uninitialized_copy(other.data_, other.data_ + this->size_, make_ptr(data_, this->capacity_)); + } + else + { + this->ptr_ = other.ptr_; + // Set pointer to the inline array so that delete is not called + // when deallocating. + other.ptr_ = other.data_; + } + } + +public: + MemoryBuffer(MemoryBuffer &&other) + { + move(other); + } + + MemoryBuffer &operator=(MemoryBuffer &&other) + { + assert(this != &other); + deallocate(); + move(other); + return *this; + } +#endif + + // Returns a copy of the allocator associated with this buffer. + Allocator get_allocator() const + { + return *this; + } +}; + +template <typename T, std::size_t SIZE, typename Allocator> void MemoryBuffer<T, SIZE, Allocator>::grow(std::size_t size) +{ + std::size_t new_capacity = this->capacity_ + this->capacity_ / 2; + if (size > new_capacity) + new_capacity = size; +#if FMT_USE_ALLOCATOR_TRAITS + T *new_ptr = std::allocator_traits<Allocator>::allocate(*this, new_capacity, FMT_NULL); +#else + T *new_ptr = this->allocate(new_capacity, FMT_NULL); +#endif + // The following code doesn't throw, so the raw pointer above doesn't leak. + std::uninitialized_copy(this->ptr_, this->ptr_ + this->size_, make_ptr(new_ptr, new_capacity)); + std::size_t old_capacity = this->capacity_; + T *old_ptr = this->ptr_; + this->capacity_ = new_capacity; + this->ptr_ = new_ptr; + // deallocate may throw (at least in principle), but it doesn't matter since + // the buffer already uses the new storage and will deallocate it in case + // of exception. + if (old_ptr != data_) + Allocator::deallocate(old_ptr, old_capacity); +} + +// A fixed-size buffer. +template <typename Char> class FixedBuffer : public fmt::Buffer<Char> +{ +public: + FixedBuffer(Char *array, std::size_t size) + : fmt::Buffer<Char>(array, size) + { + } + +protected: + FMT_API void grow(std::size_t size) FMT_OVERRIDE; +}; + +template <typename Char> class BasicCharTraits +{ +public: +#if FMT_SECURE_SCL + typedef stdext::checked_array_iterator<Char *> CharPtr; +#else + typedef Char *CharPtr; +#endif + static Char cast(int value) + { + return static_cast<Char>(value); + } +}; + +template <typename Char> class CharTraits; + +template <> class CharTraits<char> : public BasicCharTraits<char> +{ +private: + // Conversion from wchar_t to char is not allowed. + static char convert(wchar_t); + +public: + static char convert(char value) + { + return value; + } + + // Formats a floating-point number. + template <typename T> + FMT_API static int format_float(char *buffer, std::size_t size, const char *format, unsigned width, int precision, T value); +}; + +#if FMT_USE_EXTERN_TEMPLATES +extern template int CharTraits<char>::format_float<double>( + char *buffer, std::size_t size, const char *format, unsigned width, int precision, double value); +extern template int CharTraits<char>::format_float<long double>( + char *buffer, std::size_t size, const char *format, unsigned width, int precision, long double value); +#endif + +template <> class CharTraits<wchar_t> : public BasicCharTraits<wchar_t> +{ +public: + static wchar_t convert(char value) + { + return value; + } + static wchar_t convert(wchar_t value) + { + return value; + } + + template <typename T> + FMT_API static int format_float(wchar_t *buffer, std::size_t size, const wchar_t *format, unsigned width, int precision, T value); +}; + +#if FMT_USE_EXTERN_TEMPLATES +extern template int CharTraits<wchar_t>::format_float<double>( + wchar_t *buffer, std::size_t size, const wchar_t *format, unsigned width, int precision, double value); +extern template int CharTraits<wchar_t>::format_float<long double>( + wchar_t *buffer, std::size_t size, const wchar_t *format, unsigned width, int precision, long double value); +#endif + +// Checks if a number is negative - used to avoid warnings. +template <bool IsSigned> struct SignChecker +{ + template <typename T> static bool is_negative(T value) + { + return value < 0; + } +}; + +template <> struct SignChecker<false> +{ + template <typename T> static bool is_negative(T) + { + return false; + } +}; + +// Returns true if value is negative, false otherwise. +// Same as (value < 0) but doesn't produce warnings if T is an unsigned type. +template <typename T> inline bool is_negative(T value) +{ + return SignChecker<std::numeric_limits<T>::is_signed>::is_negative(value); +} + +// Selects uint32_t if FitsIn32Bits is true, uint64_t otherwise. +template <bool FitsIn32Bits> struct TypeSelector +{ + typedef uint32_t Type; +}; + +template <> struct TypeSelector<false> +{ + typedef uint64_t Type; +}; + +template <typename T> struct IntTraits +{ + // Smallest of uint32_t and uint64_t that is large enough to represent + // all values of T. + typedef typename TypeSelector<std::numeric_limits<T>::digits <= 32>::Type MainType; +}; + +FMT_API FMT_NORETURN void report_unknown_type(char code, const char *type); + +// Static data is placed in this class template to allow header-only +// configuration. +template <typename T = void> struct FMT_API BasicData +{ + static const uint32_t POWERS_OF_10_32[]; + static const uint64_t POWERS_OF_10_64[]; + static const char DIGITS[]; +}; + +#if FMT_USE_EXTERN_TEMPLATES +extern template struct BasicData<void>; +#endif + +typedef BasicData<> Data; + +#ifdef FMT_BUILTIN_CLZLL +// Returns the number of decimal digits in n. Leading zeros are not counted +// except for n == 0 in which case count_digits returns 1. +inline unsigned count_digits(uint64_t n) +{ + // Based on http://graphics.stanford.edu/~seander/bithacks.html#IntegerLog10 + // and the benchmark https://github.com/localvoid/cxx-benchmark-count-digits. + int t = (64 - FMT_BUILTIN_CLZLL(n | 1)) * 1233 >> 12; + return to_unsigned(t) - (n < Data::POWERS_OF_10_64[t]) + 1; +} +#else +// Fallback version of count_digits used when __builtin_clz is not available. +inline unsigned count_digits(uint64_t n) +{ + unsigned count = 1; + for (;;) + { + // Integer division is slow so do it for a group of four digits instead + // of for every digit. The idea comes from the talk by Alexandrescu + // "Three Optimization Tips for C++". See speed-test for a comparison. + if (n < 10) + return count; + if (n < 100) + return count + 1; + if (n < 1000) + return count + 2; + if (n < 10000) + return count + 3; + n /= 10000u; + count += 4; + } +} +#endif + +#ifdef FMT_BUILTIN_CLZ +// Optional version of count_digits for better performance on 32-bit platforms. +inline unsigned count_digits(uint32_t n) +{ + int t = (32 - FMT_BUILTIN_CLZ(n | 1)) * 1233 >> 12; + return to_unsigned(t) - (n < Data::POWERS_OF_10_32[t]) + 1; +} +#endif + +// A functor that doesn't add a thousands separator. +struct NoThousandsSep +{ + template <typename Char> void operator()(Char *) {} +}; + +// A functor that adds a thousands separator. +class ThousandsSep +{ +private: + fmt::StringRef sep_; + + // Index of a decimal digit with the least significant digit having index 0. + unsigned digit_index_; + +public: + explicit ThousandsSep(fmt::StringRef sep) + : sep_(sep) + , digit_index_(0) + { + } + + template <typename Char> void operator()(Char *&buffer) + { + if (++digit_index_ % 3 != 0) + return; + buffer -= sep_.size(); + std::uninitialized_copy(sep_.data(), sep_.data() + sep_.size(), internal::make_ptr(buffer, sep_.size())); + } +}; + +// Formats a decimal unsigned integer value writing into buffer. +// thousands_sep is a functor that is called after writing each char to +// add a thousands separator if necessary. +template <typename UInt, typename Char, typename ThousandsSep> +inline void format_decimal(Char *buffer, UInt value, unsigned num_digits, ThousandsSep thousands_sep) +{ + buffer += num_digits; + while (value >= 100) + { + // Integer division is slow so do it for a group of two digits instead + // of for every digit. The idea comes from the talk by Alexandrescu + // "Three Optimization Tips for C++". See speed-test for a comparison. + unsigned index = static_cast<unsigned>((value % 100) * 2); + value /= 100; + *--buffer = Data::DIGITS[index + 1]; + thousands_sep(buffer); + *--buffer = Data::DIGITS[index]; + thousands_sep(buffer); + } + if (value < 10) + { + *--buffer = static_cast<char>('0' + value); + return; + } + unsigned index = static_cast<unsigned>(value * 2); + *--buffer = Data::DIGITS[index + 1]; + thousands_sep(buffer); + *--buffer = Data::DIGITS[index]; +} + +template <typename UInt, typename Char> inline void format_decimal(Char *buffer, UInt value, unsigned num_digits) +{ + format_decimal(buffer, value, num_digits, NoThousandsSep()); + return; +} + +#ifndef _WIN32 +#define FMT_USE_WINDOWS_H 0 +#elif !defined(FMT_USE_WINDOWS_H) +#define FMT_USE_WINDOWS_H 1 +#endif + +// Define FMT_USE_WINDOWS_H to 0 to disable use of windows.h. +// All the functionality that relies on it will be disabled too. +#if FMT_USE_WINDOWS_H +// A converter from UTF-8 to UTF-16. +// It is only provided for Windows since other systems support UTF-8 natively. +class UTF8ToUTF16 +{ +private: + MemoryBuffer<wchar_t, INLINE_BUFFER_SIZE> buffer_; + +public: + FMT_API explicit UTF8ToUTF16(StringRef s); + operator WStringRef() const + { + return WStringRef(&buffer_[0], size()); + } + size_t size() const + { + return buffer_.size() - 1; + } + const wchar_t *c_str() const + { + return &buffer_[0]; + } + std::wstring str() const + { + return std::wstring(&buffer_[0], size()); + } +}; + +// A converter from UTF-16 to UTF-8. +// It is only provided for Windows since other systems support UTF-8 natively. +class UTF16ToUTF8 +{ +private: + MemoryBuffer<char, INLINE_BUFFER_SIZE> buffer_; + +public: + UTF16ToUTF8() {} + FMT_API explicit UTF16ToUTF8(WStringRef s); + operator StringRef() const + { + return StringRef(&buffer_[0], size()); + } + size_t size() const + { + return buffer_.size() - 1; + } + const char *c_str() const + { + return &buffer_[0]; + } + std::string str() const + { + return std::string(&buffer_[0], size()); + } + + // Performs conversion returning a system error code instead of + // throwing exception on conversion error. This method may still throw + // in case of memory allocation error. + FMT_API int convert(WStringRef s); +}; + +FMT_API void format_windows_error(fmt::Writer &out, int error_code, fmt::StringRef message) FMT_NOEXCEPT; +#endif + +// A formatting argument value. +struct Value +{ + template <typename Char> struct StringValue + { + const Char *value; + std::size_t size; + }; + + typedef void (*FormatFunc)(void *formatter, const void *arg, void *format_str_ptr); + + struct CustomValue + { + const void *value; + FormatFunc format; + }; + + union + { + int int_value; + unsigned uint_value; + LongLong long_long_value; + ULongLong ulong_long_value; + double double_value; + long double long_double_value; + const void *pointer; + StringValue<char> string; + StringValue<signed char> sstring; + StringValue<unsigned char> ustring; + StringValue<wchar_t> wstring; + CustomValue custom; + }; + + enum Type + { + NONE, + NAMED_ARG, + // Integer types should go first, + INT, + UINT, + LONG_LONG, + ULONG_LONG, + BOOL, + CHAR, + LAST_INTEGER_TYPE = CHAR, + // followed by floating-point types. + DOUBLE, + LONG_DOUBLE, + LAST_NUMERIC_TYPE = LONG_DOUBLE, + CSTRING, + STRING, + WSTRING, + POINTER, + CUSTOM + }; +}; + +// A formatting argument. It is a trivially copyable/constructible type to +// allow storage in internal::MemoryBuffer. +struct Arg : Value +{ + Type type; +}; + +template <typename Char> struct NamedArg; +template <typename Char, typename T> struct NamedArgWithType; + +template <typename T = void> struct Null +{ +}; + +// A helper class template to enable or disable overloads taking wide +// characters and strings in MakeValue. +template <typename T, typename Char> struct WCharHelper +{ + typedef Null<T> Supported; + typedef T Unsupported; +}; + +template <typename T> struct WCharHelper<T, wchar_t> +{ + typedef T Supported; + typedef Null<T> Unsupported; +}; + +typedef char Yes[1]; +typedef char No[2]; + +template <typename T> T &get(); + +// These are non-members to workaround an overload resolution bug in bcc32. +Yes &convert(fmt::ULongLong); +No &convert(...); + +template <typename T, bool ENABLE_CONVERSION> struct ConvertToIntImpl +{ + enum + { + value = ENABLE_CONVERSION + }; +}; + +template <typename T, bool ENABLE_CONVERSION> struct ConvertToIntImpl2 +{ + enum + { + value = false + }; +}; + +template <typename T> struct ConvertToIntImpl2<T, true> +{ + enum + { + // Don't convert numeric types. + value = ConvertToIntImpl<T, !std::numeric_limits<T>::is_specialized>::value + }; +}; + +template <typename T> struct ConvertToInt +{ + enum + { + enable_conversion = sizeof(fmt::internal::convert(get<T>())) == sizeof(Yes) + }; + enum + { + value = ConvertToIntImpl2<T, enable_conversion>::value + }; +}; + +#define FMT_DISABLE_CONVERSION_TO_INT(Type) \ + template <> struct ConvertToInt<Type> \ + { \ + enum \ + { \ + value = 0 \ + }; \ + } + +// Silence warnings about convering float to int. +FMT_DISABLE_CONVERSION_TO_INT(float); +FMT_DISABLE_CONVERSION_TO_INT(double); +FMT_DISABLE_CONVERSION_TO_INT(long double); + +template <bool B, class T = void> struct EnableIf +{ +}; + +template <class T> struct EnableIf<true, T> +{ + typedef T type; +}; + +template <bool B, class T, class F> struct Conditional +{ + typedef T type; +}; + +template <class T, class F> struct Conditional<false, T, F> +{ + typedef F type; +}; + +// For bcc32 which doesn't understand ! in template arguments. +template <bool> struct Not +{ + enum + { + value = 0 + }; +}; + +template <> struct Not<false> +{ + enum + { + value = 1 + }; +}; + +template <typename T> struct FalseType +{ + enum + { + value = 0 + }; +}; + +template <typename T, T> struct LConvCheck +{ + LConvCheck(int) {} +}; + +// Returns the thousands separator for the current locale. +// We check if ``lconv`` contains ``thousands_sep`` because on Android +// ``lconv`` is stubbed as an empty struct. +template <typename LConv> inline StringRef thousands_sep(LConv *lc, LConvCheck<char * LConv::*, &LConv::thousands_sep> = 0) +{ + return lc->thousands_sep; +} + +inline fmt::StringRef thousands_sep(...) +{ + return ""; +} + +#define FMT_CONCAT(a, b) a##b + +#if FMT_GCC_VERSION >= 303 +#define FMT_UNUSED __attribute__((unused)) +#else +#define FMT_UNUSED +#endif + +#ifndef FMT_USE_STATIC_ASSERT +#define FMT_USE_STATIC_ASSERT 0 +#endif + +#if FMT_USE_STATIC_ASSERT || FMT_HAS_FEATURE(cxx_static_assert) || (FMT_GCC_VERSION >= 403 && FMT_HAS_GXX_CXX11) || _MSC_VER >= 1600 +#define FMT_STATIC_ASSERT(cond, message) static_assert(cond, message) +#else +#define FMT_CONCAT_(a, b) FMT_CONCAT(a, b) +#define FMT_STATIC_ASSERT(cond, message) typedef int FMT_CONCAT_(Assert, __LINE__)[(cond) ? 1 : -1] FMT_UNUSED +#endif + +template <typename Formatter> void format_arg(Formatter &, ...) +{ + FMT_STATIC_ASSERT(FalseType<Formatter>::value, "Cannot format argument. To enable the use of ostream " + "operator<< include fmt/ostream.h. Otherwise provide " + "an overload of format_arg."); +} + +// Makes an Arg object from any type. +template <typename Formatter> class MakeValue : public Arg +{ +public: + typedef typename Formatter::Char Char; + +private: + // The following two methods are private to disallow formatting of + // arbitrary pointers. If you want to output a pointer cast it to + // "void *" or "const void *". In particular, this forbids formatting + // of "[const] volatile char *" which is printed as bool by iostreams. + // Do not implement! + template <typename T> MakeValue(const T *value); + template <typename T> MakeValue(T *value); + + // The following methods are private to disallow formatting of wide + // characters and strings into narrow strings as in + // fmt::format("{}", L"test"); + // To fix this, use a wide format string: fmt::format(L"{}", L"test"). +#if !FMT_MSC_VER || defined(_NATIVE_WCHAR_T_DEFINED) + MakeValue(typename WCharHelper<wchar_t, Char>::Unsupported); +#endif + MakeValue(typename WCharHelper<wchar_t *, Char>::Unsupported); + MakeValue(typename WCharHelper<const wchar_t *, Char>::Unsupported); + MakeValue(typename WCharHelper<const std::wstring &, Char>::Unsupported); +#if FMT_HAS_STRING_VIEW + MakeValue(typename WCharHelper<const std::wstring_view &, Char>::Unsupported); +#endif + MakeValue(typename WCharHelper<WStringRef, Char>::Unsupported); + + void set_string(StringRef str) + { + string.value = str.data(); + string.size = str.size(); + } + + void set_string(WStringRef str) + { + wstring.value = str.data(); + wstring.size = str.size(); + } + + // Formats an argument of a custom type, such as a user-defined class. + template <typename T> static void format_custom_arg(void *formatter, const void *arg, void *format_str_ptr) + { + format_arg(*static_cast<Formatter *>(formatter), *static_cast<const Char **>(format_str_ptr), *static_cast<const T *>(arg)); + } + +public: + MakeValue() {} + +#define FMT_MAKE_VALUE_(Type, field, TYPE, rhs) \ + MakeValue(Type value) \ + { \ + field = rhs; \ + } \ + static uint64_t type(Type) \ + { \ + return Arg::TYPE; \ + } + +#define FMT_MAKE_VALUE(Type, field, TYPE) FMT_MAKE_VALUE_(Type, field, TYPE, value) + + FMT_MAKE_VALUE(bool, int_value, BOOL) + FMT_MAKE_VALUE(short, int_value, INT) + FMT_MAKE_VALUE(unsigned short, uint_value, UINT) + FMT_MAKE_VALUE(int, int_value, INT) + FMT_MAKE_VALUE(unsigned, uint_value, UINT) + + MakeValue(long value) + { + // To minimize the number of types we need to deal with, long is + // translated either to int or to long long depending on its size. + if (const_check(sizeof(long) == sizeof(int))) + int_value = static_cast<int>(value); + else + long_long_value = value; + } + static uint64_t type(long) + { + return sizeof(long) == sizeof(int) ? Arg::INT : Arg::LONG_LONG; + } + + MakeValue(unsigned long value) + { + if (const_check(sizeof(unsigned long) == sizeof(unsigned))) + uint_value = static_cast<unsigned>(value); + else + ulong_long_value = value; + } + static uint64_t type(unsigned long) + { + return sizeof(unsigned long) == sizeof(unsigned) ? Arg::UINT : Arg::ULONG_LONG; + } + + FMT_MAKE_VALUE(LongLong, long_long_value, LONG_LONG) + FMT_MAKE_VALUE(ULongLong, ulong_long_value, ULONG_LONG) + FMT_MAKE_VALUE(float, double_value, DOUBLE) + FMT_MAKE_VALUE(double, double_value, DOUBLE) + FMT_MAKE_VALUE(long double, long_double_value, LONG_DOUBLE) + FMT_MAKE_VALUE(signed char, int_value, INT) + FMT_MAKE_VALUE(unsigned char, uint_value, UINT) + FMT_MAKE_VALUE(char, int_value, CHAR) + +#if __cplusplus >= 201103L + template <typename T, typename = typename std::enable_if<std::is_enum<T>::value && ConvertToInt<T>::value>::type> MakeValue(T value) + { + int_value = value; + } + + template <typename T, typename = typename std::enable_if<std::is_enum<T>::value && ConvertToInt<T>::value>::type> + static uint64_t type(T) + { + return Arg::INT; + } +#endif + +#if !defined(_MSC_VER) || defined(_NATIVE_WCHAR_T_DEFINED) + MakeValue(typename WCharHelper<wchar_t, Char>::Supported value) + { + int_value = value; + } + static uint64_t type(wchar_t) + { + return Arg::CHAR; + } +#endif + +#define FMT_MAKE_STR_VALUE(Type, TYPE) \ + MakeValue(Type value) \ + { \ + set_string(value); \ + } \ + static uint64_t type(Type) \ + { \ + return Arg::TYPE; \ + } + + FMT_MAKE_VALUE(char *, string.value, CSTRING) + FMT_MAKE_VALUE(const char *, string.value, CSTRING) + FMT_MAKE_VALUE(signed char *, sstring.value, CSTRING) + FMT_MAKE_VALUE(const signed char *, sstring.value, CSTRING) + FMT_MAKE_VALUE(unsigned char *, ustring.value, CSTRING) + FMT_MAKE_VALUE(const unsigned char *, ustring.value, CSTRING) + FMT_MAKE_STR_VALUE(const std::string &, STRING) +#if FMT_HAS_STRING_VIEW + FMT_MAKE_STR_VALUE(const std::string_view &, STRING) +#endif + FMT_MAKE_STR_VALUE(StringRef, STRING) + FMT_MAKE_VALUE_(CStringRef, string.value, CSTRING, value.c_str()) + +#define FMT_MAKE_WSTR_VALUE(Type, TYPE) \ + MakeValue(typename WCharHelper<Type, Char>::Supported value) \ + { \ + set_string(value); \ + } \ + static uint64_t type(Type) \ + { \ + return Arg::TYPE; \ + } + + FMT_MAKE_WSTR_VALUE(wchar_t *, WSTRING) + FMT_MAKE_WSTR_VALUE(const wchar_t *, WSTRING) + FMT_MAKE_WSTR_VALUE(const std::wstring &, WSTRING) +#if FMT_HAS_STRING_VIEW + FMT_MAKE_WSTR_VALUE(const std::wstring_view &, WSTRING) +#endif + FMT_MAKE_WSTR_VALUE(WStringRef, WSTRING) + + FMT_MAKE_VALUE(void *, pointer, POINTER) + FMT_MAKE_VALUE(const void *, pointer, POINTER) + + template <typename T> MakeValue(const T &value, typename EnableIf<Not<ConvertToInt<T>::value>::value, int>::type = 0) + { + custom.value = &value; + custom.format = &format_custom_arg<T>; + } + + template <typename T> static typename EnableIf<Not<ConvertToInt<T>::value>::value, uint64_t>::type type(const T &) + { + return Arg::CUSTOM; + } + + // Additional template param `Char_` is needed here because make_type always + // uses char. + template <typename Char_> MakeValue(const NamedArg<Char_> &value) + { + pointer = &value; + } + template <typename Char_, typename T> MakeValue(const NamedArgWithType<Char_, T> &value) + { + pointer = &value; + } + + template <typename Char_> static uint64_t type(const NamedArg<Char_> &) + { + return Arg::NAMED_ARG; + } + template <typename Char_, typename T> static uint64_t type(const NamedArgWithType<Char_, T> &) + { + return Arg::NAMED_ARG; + } +}; + +template <typename Formatter> class MakeArg : public Arg +{ +public: + MakeArg() + { + type = Arg::NONE; + } + + template <typename T> + MakeArg(const T &value) + : Arg(MakeValue<Formatter>(value)) + { + type = static_cast<Arg::Type>(MakeValue<Formatter>::type(value)); + } +}; + +template <typename Char> struct NamedArg : Arg +{ + BasicStringRef<Char> name; + + template <typename T> + NamedArg(BasicStringRef<Char> argname, const T &value) + : Arg(MakeArg<BasicFormatter<Char>>(value)) + , name(argname) + { + } +}; + +template <typename Char, typename T> struct NamedArgWithType : NamedArg<Char> +{ + NamedArgWithType(BasicStringRef<Char> argname, const T &value) + : NamedArg<Char>(argname, value) + { + } +}; + +class RuntimeError : public std::runtime_error +{ +protected: + RuntimeError() + : std::runtime_error("") + { + } + RuntimeError(const RuntimeError &rerr) + : std::runtime_error(rerr) + { + } + FMT_API ~RuntimeError() FMT_DTOR_NOEXCEPT FMT_OVERRIDE; +}; + +template <typename Char> class ArgMap; +} // namespace internal + +/** An argument list. */ +class ArgList +{ +private: + // To reduce compiled code size per formatting function call, types of first + // MAX_PACKED_ARGS arguments are passed in the types_ field. + uint64_t types_; + union + { + // If the number of arguments is less than MAX_PACKED_ARGS, the argument + // values are stored in values_, otherwise they are stored in args_. + // This is done to reduce compiled code size as storing larger objects + // may require more code (at least on x86-64) even if the same amount of + // data is actually copied to stack. It saves ~10% on the bloat test. + const internal::Value *values_; + const internal::Arg *args_; + }; + + internal::Arg::Type type(unsigned index) const + { + return type(types_, index); + } + + template <typename Char> friend class internal::ArgMap; + +public: + // Maximum number of arguments with packed types. + enum + { + MAX_PACKED_ARGS = 16 + }; + + ArgList() + : types_(0) + { + } + + ArgList(ULongLong types, const internal::Value *values) + : types_(types) + , values_(values) + { + } + ArgList(ULongLong types, const internal::Arg *args) + : types_(types) + , args_(args) + { + } + + uint64_t types() const + { + return types_; + } + + /** Returns the argument at specified index. */ + internal::Arg operator[](unsigned index) const + { + using internal::Arg; + Arg arg; + bool use_values = type(MAX_PACKED_ARGS - 1) == Arg::NONE; + if (index < MAX_PACKED_ARGS) + { + Arg::Type arg_type = type(index); + internal::Value &val = arg; + if (arg_type != Arg::NONE) + val = use_values ? values_[index] : args_[index]; + arg.type = arg_type; + return arg; + } + if (use_values) + { + // The index is greater than the number of arguments that can be stored + // in values, so return a "none" argument. + arg.type = Arg::NONE; + return arg; + } + for (unsigned i = MAX_PACKED_ARGS; i <= index; ++i) + { + if (args_[i].type == Arg::NONE) + return args_[i]; + } + return args_[index]; + } + + static internal::Arg::Type type(uint64_t types, unsigned index) + { + unsigned shift = index * 4; + uint64_t mask = 0xf; + return static_cast<internal::Arg::Type>((types & (mask << shift)) >> shift); + } +}; + +#define FMT_DISPATCH(call) static_cast<Impl *>(this)->call + +/** + \rst + An argument visitor based on the `curiously recurring template pattern + <http://en.wikipedia.org/wiki/Curiously_recurring_template_pattern>`_. + + To use `~fmt::ArgVisitor` define a subclass that implements some or all of the + visit methods with the same signatures as the methods in `~fmt::ArgVisitor`, + for example, `~fmt::ArgVisitor::visit_int()`. + Pass the subclass as the *Impl* template parameter. Then calling + `~fmt::ArgVisitor::visit` for some argument will dispatch to a visit method + specific to the argument type. For example, if the argument type is + ``double`` then the `~fmt::ArgVisitor::visit_double()` method of a subclass + will be called. If the subclass doesn't contain a method with this signature, + then a corresponding method of `~fmt::ArgVisitor` will be called. + + **Example**:: + + class MyArgVisitor : public fmt::ArgVisitor<MyArgVisitor, void> { + public: + void visit_int(int value) { fmt::print("{}", value); } + void visit_double(double value) { fmt::print("{}", value ); } + }; + \endrst + */ +template <typename Impl, typename Result> class ArgVisitor +{ +private: + typedef internal::Arg Arg; + +public: + void report_unhandled_arg() {} + + Result visit_unhandled_arg() + { + FMT_DISPATCH(report_unhandled_arg()); + return Result(); + } + + /** Visits an ``int`` argument. **/ + Result visit_int(int value) + { + return FMT_DISPATCH(visit_any_int(value)); + } + + /** Visits a ``long long`` argument. **/ + Result visit_long_long(LongLong value) + { + return FMT_DISPATCH(visit_any_int(value)); + } + + /** Visits an ``unsigned`` argument. **/ + Result visit_uint(unsigned value) + { + return FMT_DISPATCH(visit_any_int(value)); + } + + /** Visits an ``unsigned long long`` argument. **/ + Result visit_ulong_long(ULongLong value) + { + return FMT_DISPATCH(visit_any_int(value)); + } + + /** Visits a ``bool`` argument. **/ + Result visit_bool(bool value) + { + return FMT_DISPATCH(visit_any_int(value)); + } + + /** Visits a ``char`` or ``wchar_t`` argument. **/ + Result visit_char(int value) + { + return FMT_DISPATCH(visit_any_int(value)); + } + + /** Visits an argument of any integral type. **/ + template <typename T> Result visit_any_int(T) + { + return FMT_DISPATCH(visit_unhandled_arg()); + } + + /** Visits a ``double`` argument. **/ + Result visit_double(double value) + { + return FMT_DISPATCH(visit_any_double(value)); + } + + /** Visits a ``long double`` argument. **/ + Result visit_long_double(long double value) + { + return FMT_DISPATCH(visit_any_double(value)); + } + + /** Visits a ``double`` or ``long double`` argument. **/ + template <typename T> Result visit_any_double(T) + { + return FMT_DISPATCH(visit_unhandled_arg()); + } + + /** Visits a null-terminated C string (``const char *``) argument. **/ + Result visit_cstring(const char *) + { + return FMT_DISPATCH(visit_unhandled_arg()); + } + + /** Visits a string argument. **/ + Result visit_string(Arg::StringValue<char>) + { + return FMT_DISPATCH(visit_unhandled_arg()); + } + + /** Visits a wide string argument. **/ + Result visit_wstring(Arg::StringValue<wchar_t>) + { + return FMT_DISPATCH(visit_unhandled_arg()); + } + + /** Visits a pointer argument. **/ + Result visit_pointer(const void *) + { + return FMT_DISPATCH(visit_unhandled_arg()); + } + + /** Visits an argument of a custom (user-defined) type. **/ + Result visit_custom(Arg::CustomValue) + { + return FMT_DISPATCH(visit_unhandled_arg()); + } + + /** + \rst + Visits an argument dispatching to the appropriate visit method based on + the argument type. For example, if the argument type is ``double`` then + the `~fmt::ArgVisitor::visit_double()` method of the *Impl* class will be + called. + \endrst + */ + Result visit(const Arg &arg) + { + switch (arg.type) + { + case Arg::NONE: + case Arg::NAMED_ARG: + FMT_ASSERT(false, "invalid argument type"); + break; + case Arg::INT: + return FMT_DISPATCH(visit_int(arg.int_value)); + case Arg::UINT: + return FMT_DISPATCH(visit_uint(arg.uint_value)); + case Arg::LONG_LONG: + return FMT_DISPATCH(visit_long_long(arg.long_long_value)); + case Arg::ULONG_LONG: + return FMT_DISPATCH(visit_ulong_long(arg.ulong_long_value)); + case Arg::BOOL: + return FMT_DISPATCH(visit_bool(arg.int_value != 0)); + case Arg::CHAR: + return FMT_DISPATCH(visit_char(arg.int_value)); + case Arg::DOUBLE: + return FMT_DISPATCH(visit_double(arg.double_value)); + case Arg::LONG_DOUBLE: + return FMT_DISPATCH(visit_long_double(arg.long_double_value)); + case Arg::CSTRING: + return FMT_DISPATCH(visit_cstring(arg.string.value)); + case Arg::STRING: + return FMT_DISPATCH(visit_string(arg.string)); + case Arg::WSTRING: + return FMT_DISPATCH(visit_wstring(arg.wstring)); + case Arg::POINTER: + return FMT_DISPATCH(visit_pointer(arg.pointer)); + case Arg::CUSTOM: + return FMT_DISPATCH(visit_custom(arg.custom)); + } + return Result(); + } +}; + +enum Alignment +{ + ALIGN_DEFAULT, + ALIGN_LEFT, + ALIGN_RIGHT, + ALIGN_CENTER, + ALIGN_NUMERIC +}; + +// Flags. +enum +{ + SIGN_FLAG = 1, + PLUS_FLAG = 2, + MINUS_FLAG = 4, + HASH_FLAG = 8, + CHAR_FLAG = 0x10 // Argument has char type - used in error reporting. +}; + +// An empty format specifier. +struct EmptySpec +{ +}; + +// A type specifier. +template <char TYPE> struct TypeSpec : EmptySpec +{ + Alignment align() const + { + return ALIGN_DEFAULT; + } + unsigned width() const + { + return 0; + } + int precision() const + { + return -1; + } + bool flag(unsigned) const + { + return false; + } + char type() const + { + return TYPE; + } + char type_prefix() const + { + return TYPE; + } + char fill() const + { + return ' '; + } +}; + +// A width specifier. +struct WidthSpec +{ + unsigned width_; + // Fill is always wchar_t and cast to char if necessary to avoid having + // two specialization of WidthSpec and its subclasses. + wchar_t fill_; + + WidthSpec(unsigned width, wchar_t fill) + : width_(width) + , fill_(fill) + { + } + + unsigned width() const + { + return width_; + } + wchar_t fill() const + { + return fill_; + } +}; + +// An alignment specifier. +struct AlignSpec : WidthSpec +{ + Alignment align_; + + AlignSpec(unsigned width, wchar_t fill, Alignment align = ALIGN_DEFAULT) + : WidthSpec(width, fill) + , align_(align) + { + } + + Alignment align() const + { + return align_; + } + + int precision() const + { + return -1; + } +}; + +// An alignment and type specifier. +template <char TYPE> struct AlignTypeSpec : AlignSpec +{ + AlignTypeSpec(unsigned width, wchar_t fill) + : AlignSpec(width, fill) + { + } + + bool flag(unsigned) const + { + return false; + } + char type() const + { + return TYPE; + } + char type_prefix() const + { + return TYPE; + } +}; + +// A full format specifier. +struct FormatSpec : AlignSpec +{ + unsigned flags_; + int precision_; + char type_; + + FormatSpec(unsigned width = 0, char type = 0, wchar_t fill = ' ') + : AlignSpec(width, fill) + , flags_(0) + , precision_(-1) + , type_(type) + { + } + + bool flag(unsigned f) const + { + return (flags_ & f) != 0; + } + int precision() const + { + return precision_; + } + char type() const + { + return type_; + } + char type_prefix() const + { + return type_; + } +}; + +// An integer format specifier. +template <typename T, typename SpecT = TypeSpec<0>, typename Char = char> class IntFormatSpec : public SpecT +{ +private: + T value_; + +public: + IntFormatSpec(T val, const SpecT &spec = SpecT()) + : SpecT(spec) + , value_(val) + { + } + + T value() const + { + return value_; + } +}; + +// A string format specifier. +template <typename Char> class StrFormatSpec : public AlignSpec +{ +private: + const Char *str_; + +public: + template <typename FillChar> + StrFormatSpec(const Char *str, unsigned width, FillChar fill) + : AlignSpec(width, fill) + , str_(str) + { + internal::CharTraits<Char>::convert(FillChar()); + } + + const Char *str() const + { + return str_; + } +}; + +/** + Returns an integer format specifier to format the value in base 2. + */ +IntFormatSpec<int, TypeSpec<'b'>> bin(int value); + +/** + Returns an integer format specifier to format the value in base 8. + */ +IntFormatSpec<int, TypeSpec<'o'>> oct(int value); + +/** + Returns an integer format specifier to format the value in base 16 using + lower-case letters for the digits above 9. + */ +IntFormatSpec<int, TypeSpec<'x'>> hex(int value); + +/** + Returns an integer formatter format specifier to format in base 16 using + upper-case letters for the digits above 9. + */ +IntFormatSpec<int, TypeSpec<'X'>> hexu(int value); + +/** + \rst + Returns an integer format specifier to pad the formatted argument with the + fill character to the specified width using the default (right) numeric + alignment. + + **Example**:: + + MemoryWriter out; + out << pad(hex(0xcafe), 8, '0'); + // out.str() == "0000cafe" + + \endrst + */ +template <char TYPE_CODE, typename Char> IntFormatSpec<int, AlignTypeSpec<TYPE_CODE>, Char> pad(int value, unsigned width, Char fill = ' '); + +#define FMT_DEFINE_INT_FORMATTERS(TYPE) \ + inline IntFormatSpec<TYPE, TypeSpec<'b'>> bin(TYPE value) \ + { \ + return IntFormatSpec<TYPE, TypeSpec<'b'>>(value, TypeSpec<'b'>()); \ + } \ + \ + inline IntFormatSpec<TYPE, TypeSpec<'o'>> oct(TYPE value) \ + { \ + return IntFormatSpec<TYPE, TypeSpec<'o'>>(value, TypeSpec<'o'>()); \ + } \ + \ + inline IntFormatSpec<TYPE, TypeSpec<'x'>> hex(TYPE value) \ + { \ + return IntFormatSpec<TYPE, TypeSpec<'x'>>(value, TypeSpec<'x'>()); \ + } \ + \ + inline IntFormatSpec<TYPE, TypeSpec<'X'>> hexu(TYPE value) \ + { \ + return IntFormatSpec<TYPE, TypeSpec<'X'>>(value, TypeSpec<'X'>()); \ + } \ + \ + template <char TYPE_CODE> \ + inline IntFormatSpec<TYPE, AlignTypeSpec<TYPE_CODE>> pad(IntFormatSpec<TYPE, TypeSpec<TYPE_CODE>> f, unsigned width) \ + { \ + return IntFormatSpec<TYPE, AlignTypeSpec<TYPE_CODE>>(f.value(), AlignTypeSpec<TYPE_CODE>(width, ' ')); \ + } \ + \ + /* For compatibility with older compilers we provide two overloads for pad, */ \ + /* one that takes a fill character and one that doesn't. In the future this */ \ + /* can be replaced with one overload making the template argument Char */ \ + /* default to char (C++11). */ \ + template <char TYPE_CODE, typename Char> \ + inline IntFormatSpec<TYPE, AlignTypeSpec<TYPE_CODE>, Char> pad( \ + IntFormatSpec<TYPE, TypeSpec<TYPE_CODE>, Char> f, unsigned width, Char fill) \ + { \ + return IntFormatSpec<TYPE, AlignTypeSpec<TYPE_CODE>, Char>(f.value(), AlignTypeSpec<TYPE_CODE>(width, fill)); \ + } \ + \ + inline IntFormatSpec<TYPE, AlignTypeSpec<0>> pad(TYPE value, unsigned width) \ + { \ + return IntFormatSpec<TYPE, AlignTypeSpec<0>>(value, AlignTypeSpec<0>(width, ' ')); \ + } \ + \ + template <typename Char> inline IntFormatSpec<TYPE, AlignTypeSpec<0>, Char> pad(TYPE value, unsigned width, Char fill) \ + { \ + return IntFormatSpec<TYPE, AlignTypeSpec<0>, Char>(value, AlignTypeSpec<0>(width, fill)); \ + } + +FMT_DEFINE_INT_FORMATTERS(int) +FMT_DEFINE_INT_FORMATTERS(long) +FMT_DEFINE_INT_FORMATTERS(unsigned) +FMT_DEFINE_INT_FORMATTERS(unsigned long) +FMT_DEFINE_INT_FORMATTERS(LongLong) +FMT_DEFINE_INT_FORMATTERS(ULongLong) + +/** + \rst + Returns a string formatter that pads the formatted argument with the fill + character to the specified width using the default (left) string alignment. + + **Example**:: + + std::string s = str(MemoryWriter() << pad("abc", 8)); + // s == "abc " + + \endrst + */ +template <typename Char> inline StrFormatSpec<Char> pad(const Char *str, unsigned width, Char fill = ' ') +{ + return StrFormatSpec<Char>(str, width, fill); +} + +inline StrFormatSpec<wchar_t> pad(const wchar_t *str, unsigned width, char fill = ' ') +{ + return StrFormatSpec<wchar_t>(str, width, fill); +} + +namespace internal { + +template <typename Char> class ArgMap +{ +private: + typedef std::vector<std::pair<fmt::BasicStringRef<Char>, internal::Arg>> MapType; + typedef typename MapType::value_type Pair; + + MapType map_; + +public: + void init(const ArgList &args); + + const internal::Arg *find(const fmt::BasicStringRef<Char> &name) const + { + // The list is unsorted, so just return the first matching name. + for (typename MapType::const_iterator it = map_.begin(), end = map_.end(); it != end; ++it) + { + if (it->first == name) + return &it->second; + } + return FMT_NULL; + } +}; + +template <typename Char> void ArgMap<Char>::init(const ArgList &args) +{ + if (!map_.empty()) + return; + typedef internal::NamedArg<Char> NamedArg; + const NamedArg *named_arg = FMT_NULL; + bool use_values = args.type(ArgList::MAX_PACKED_ARGS - 1) == internal::Arg::NONE; + if (use_values) + { + for (unsigned i = 0; /*nothing*/; ++i) + { + internal::Arg::Type arg_type = args.type(i); + switch (arg_type) + { + case internal::Arg::NONE: + return; + case internal::Arg::NAMED_ARG: + named_arg = static_cast<const NamedArg *>(args.values_[i].pointer); + map_.push_back(Pair(named_arg->name, *named_arg)); + break; + default: + /*nothing*/ + ; + } + } + return; + } + for (unsigned i = 0; i != ArgList::MAX_PACKED_ARGS; ++i) + { + internal::Arg::Type arg_type = args.type(i); + if (arg_type == internal::Arg::NAMED_ARG) + { + named_arg = static_cast<const NamedArg *>(args.args_[i].pointer); + map_.push_back(Pair(named_arg->name, *named_arg)); + } + } + for (unsigned i = ArgList::MAX_PACKED_ARGS; /*nothing*/; ++i) + { + switch (args.args_[i].type) + { + case internal::Arg::NONE: + return; + case internal::Arg::NAMED_ARG: + named_arg = static_cast<const NamedArg *>(args.args_[i].pointer); + map_.push_back(Pair(named_arg->name, *named_arg)); + break; + default: + /*nothing*/ + ; + } + } +} + +template <typename Impl, typename Char, typename Spec = fmt::FormatSpec> class ArgFormatterBase : public ArgVisitor<Impl, void> +{ +private: + BasicWriter<Char> &writer_; + Spec &spec_; + + FMT_DISALLOW_COPY_AND_ASSIGN(ArgFormatterBase); + + void write_pointer(const void *p) + { + spec_.flags_ = HASH_FLAG; + spec_.type_ = 'x'; + writer_.write_int(reinterpret_cast<uintptr_t>(p), spec_); + } + + // workaround MSVC two-phase lookup issue + typedef internal::Arg Arg; + +protected: + BasicWriter<Char> &writer() + { + return writer_; + } + Spec &spec() + { + return spec_; + } + + void write(bool value) + { + const char *str_value = value ? "true" : "false"; + Arg::StringValue<char> str = {str_value, std::strlen(str_value)}; + writer_.write_str(str, spec_); + } + + void write(const char *value) + { + Arg::StringValue<char> str = {value, value ? std::strlen(value) : 0}; + writer_.write_str(str, spec_); + } + +public: + typedef Spec SpecType; + + ArgFormatterBase(BasicWriter<Char> &w, Spec &s) + : writer_(w) + , spec_(s) + { + } + + template <typename T> void visit_any_int(T value) + { + writer_.write_int(value, spec_); + } + + template <typename T> void visit_any_double(T value) + { + writer_.write_double(value, spec_); + } + + void visit_bool(bool value) + { + if (spec_.type_) + { + visit_any_int(value); + return; + } + write(value); + } + + void visit_char(int value) + { + if (spec_.type_ && spec_.type_ != 'c') + { + spec_.flags_ |= CHAR_FLAG; + writer_.write_int(value, spec_); + return; + } + if (spec_.align_ == ALIGN_NUMERIC || spec_.flags_ != 0) + FMT_THROW(FormatError("invalid format specifier for char")); + typedef typename BasicWriter<Char>::CharPtr CharPtr; + Char fill = internal::CharTraits<Char>::cast(spec_.fill()); + CharPtr out = CharPtr(); + const unsigned CHAR_SIZE = 1; + if (spec_.width_ > CHAR_SIZE) + { + out = writer_.grow_buffer(spec_.width_); + if (spec_.align_ == ALIGN_RIGHT) + { + std::uninitialized_fill_n(out, spec_.width_ - CHAR_SIZE, fill); + out += spec_.width_ - CHAR_SIZE; + } + else if (spec_.align_ == ALIGN_CENTER) + { + out = writer_.fill_padding(out, spec_.width_, internal::const_check(CHAR_SIZE), fill); + } + else + { + std::uninitialized_fill_n(out + CHAR_SIZE, spec_.width_ - CHAR_SIZE, fill); + } + } + else + { + out = writer_.grow_buffer(CHAR_SIZE); + } + *out = internal::CharTraits<Char>::cast(value); + } + + void visit_cstring(const char *value) + { + if (spec_.type_ == 'p') + return write_pointer(value); + write(value); + } + + // Qualification with "internal" here and below is a workaround for nvcc. + void visit_string(internal::Arg::StringValue<char> value) + { + writer_.write_str(value, spec_); + } + + using ArgVisitor<Impl, void>::visit_wstring; + + void visit_wstring(internal::Arg::StringValue<Char> value) + { + writer_.write_str(value, spec_); + } + + void visit_pointer(const void *value) + { + if (spec_.type_ && spec_.type_ != 'p') + report_unknown_type(spec_.type_, "pointer"); + write_pointer(value); + } +}; + +class FormatterBase +{ +private: + ArgList args_; + int next_arg_index_; + + // Returns the argument with specified index. + FMT_API Arg do_get_arg(unsigned arg_index, const char *&error); + +protected: + const ArgList &args() const + { + return args_; + } + + explicit FormatterBase(const ArgList &args) + { + args_ = args; + next_arg_index_ = 0; + } + + // Returns the next argument. + Arg next_arg(const char *&error) + { + if (next_arg_index_ >= 0) + return do_get_arg(internal::to_unsigned(next_arg_index_++), error); + error = "cannot switch from manual to automatic argument indexing"; + return Arg(); + } + + // Checks if manual indexing is used and returns the argument with + // specified index. + Arg get_arg(unsigned arg_index, const char *&error) + { + return check_no_auto_index(error) ? do_get_arg(arg_index, error) : Arg(); + } + + bool check_no_auto_index(const char *&error) + { + if (next_arg_index_ > 0) + { + error = "cannot switch from automatic to manual argument indexing"; + return false; + } + next_arg_index_ = -1; + return true; + } + + template <typename Char> void write(BasicWriter<Char> &w, const Char *start, const Char *end) + { + if (start != end) + w << BasicStringRef<Char>(start, internal::to_unsigned(end - start)); + } +}; +} // namespace internal + +/** + \rst + An argument formatter based on the `curiously recurring template pattern + <http://en.wikipedia.org/wiki/Curiously_recurring_template_pattern>`_. + + To use `~fmt::BasicArgFormatter` define a subclass that implements some or + all of the visit methods with the same signatures as the methods in + `~fmt::ArgVisitor`, for example, `~fmt::ArgVisitor::visit_int()`. + Pass the subclass as the *Impl* template parameter. When a formatting + function processes an argument, it will dispatch to a visit method + specific to the argument type. For example, if the argument type is + ``double`` then the `~fmt::ArgVisitor::visit_double()` method of a subclass + will be called. If the subclass doesn't contain a method with this signature, + then a corresponding method of `~fmt::BasicArgFormatter` or its superclass + will be called. + \endrst + */ +template <typename Impl, typename Char, typename Spec = fmt::FormatSpec> +class BasicArgFormatter : public internal::ArgFormatterBase<Impl, Char, Spec> +{ +private: + BasicFormatter<Char, Impl> &formatter_; + const Char *format_; + +public: + /** + \rst + Constructs an argument formatter object. + *formatter* is a reference to the main formatter object, *spec* contains + format specifier information for standard argument types, and *fmt* points + to the part of the format string being parsed for custom argument types. + \endrst + */ + BasicArgFormatter(BasicFormatter<Char, Impl> &formatter, Spec &spec, const Char *fmt) + : internal::ArgFormatterBase<Impl, Char, Spec>(formatter.writer(), spec) + , formatter_(formatter) + , format_(fmt) + { + } + + /** Formats an argument of a custom (user-defined) type. */ + void visit_custom(internal::Arg::CustomValue c) + { + c.format(&formatter_, c.value, &format_); + } +}; + +/** The default argument formatter. */ +template <typename Char> class ArgFormatter : public BasicArgFormatter<ArgFormatter<Char>, Char, FormatSpec> +{ +public: + /** Constructs an argument formatter object. */ + ArgFormatter(BasicFormatter<Char> &formatter, FormatSpec &spec, const Char *fmt) + : BasicArgFormatter<ArgFormatter<Char>, Char, FormatSpec>(formatter, spec, fmt) + { + } +}; + +/** This template formats data and writes the output to a writer. */ +template <typename CharType, typename ArgFormatter> class BasicFormatter : private internal::FormatterBase +{ +public: + /** The character type for the output. */ + typedef CharType Char; + +private: + BasicWriter<Char> &writer_; + internal::ArgMap<Char> map_; + + FMT_DISALLOW_COPY_AND_ASSIGN(BasicFormatter); + + using internal::FormatterBase::get_arg; + + // Checks if manual indexing is used and returns the argument with + // specified name. + internal::Arg get_arg(BasicStringRef<Char> arg_name, const char *&error); + + // Parses argument index and returns corresponding argument. + internal::Arg parse_arg_index(const Char *&s); + + // Parses argument name and returns corresponding argument. + internal::Arg parse_arg_name(const Char *&s); + +public: + /** + \rst + Constructs a ``BasicFormatter`` object. References to the arguments and + the writer are stored in the formatter object so make sure they have + appropriate lifetimes. + \endrst + */ + BasicFormatter(const ArgList &args, BasicWriter<Char> &w) + : internal::FormatterBase(args) + , writer_(w) + { + } + + /** Returns a reference to the writer associated with this formatter. */ + BasicWriter<Char> &writer() + { + return writer_; + } + + /** Formats stored arguments and writes the output to the writer. */ + void format(BasicCStringRef<Char> format_str); + + // Formats a single argument and advances format_str, a format string pointer. + const Char *format(const Char *&format_str, const internal::Arg &arg); +}; + +// Generates a comma-separated list with results of applying f to +// numbers 0..n-1. +#define FMT_GEN(n, f) FMT_GEN##n(f) +#define FMT_GEN1(f) f(0) +#define FMT_GEN2(f) FMT_GEN1(f), f(1) +#define FMT_GEN3(f) FMT_GEN2(f), f(2) +#define FMT_GEN4(f) FMT_GEN3(f), f(3) +#define FMT_GEN5(f) FMT_GEN4(f), f(4) +#define FMT_GEN6(f) FMT_GEN5(f), f(5) +#define FMT_GEN7(f) FMT_GEN6(f), f(6) +#define FMT_GEN8(f) FMT_GEN7(f), f(7) +#define FMT_GEN9(f) FMT_GEN8(f), f(8) +#define FMT_GEN10(f) FMT_GEN9(f), f(9) +#define FMT_GEN11(f) FMT_GEN10(f), f(10) +#define FMT_GEN12(f) FMT_GEN11(f), f(11) +#define FMT_GEN13(f) FMT_GEN12(f), f(12) +#define FMT_GEN14(f) FMT_GEN13(f), f(13) +#define FMT_GEN15(f) FMT_GEN14(f), f(14) + +namespace internal { +inline uint64_t make_type() +{ + return 0; +} + +template <typename T> inline uint64_t make_type(const T &arg) +{ + return MakeValue<BasicFormatter<char>>::type(arg); +} + +template <std::size_t N, bool /*IsPacked*/ = (N < ArgList::MAX_PACKED_ARGS)> struct ArgArray; + +template <std::size_t N> struct ArgArray<N, true /*IsPacked*/> +{ + // '+' is used to silence GCC -Wduplicated-branches warning. + typedef Value Type[N > 0 ? N : +1]; + + template <typename Formatter, typename T> static Value make(const T &value) + { +#ifdef __clang__ + Value result = MakeValue<Formatter>(value); + // Workaround a bug in Apple LLVM version 4.2 (clang-425.0.28) of clang: + // https://github.com/fmtlib/fmt/issues/276 + (void)result.custom.format; + return result; +#else + return MakeValue<Formatter>(value); +#endif + } +}; + +template <std::size_t N> struct ArgArray<N, false /*IsPacked*/> +{ + typedef Arg Type[N + 1]; // +1 for the list end Arg::NONE + + template <typename Formatter, typename T> static Arg make(const T &value) + { + return MakeArg<Formatter>(value); + } +}; + +#if FMT_USE_VARIADIC_TEMPLATES +template <typename Arg, typename... Args> inline uint64_t make_type(const Arg &first, const Args &... tail) +{ + return make_type(first) | (make_type(tail...) << 4); +} + +#else + +struct ArgType +{ + uint64_t type; + + ArgType() + : type(0) + { + } + + template <typename T> + ArgType(const T &arg) + : type(make_type(arg)) + { + } +}; + +#define FMT_ARG_TYPE_DEFAULT(n) ArgType t##n = ArgType() + +inline uint64_t make_type(FMT_GEN15(FMT_ARG_TYPE_DEFAULT)) +{ + return t0.type | (t1.type << 4) | (t2.type << 8) | (t3.type << 12) | (t4.type << 16) | (t5.type << 20) | (t6.type << 24) | + (t7.type << 28) | (t8.type << 32) | (t9.type << 36) | (t10.type << 40) | (t11.type << 44) | (t12.type << 48) | (t13.type << 52) | + (t14.type << 56); +} +#endif +} // namespace internal + +#define FMT_MAKE_TEMPLATE_ARG(n) typename T##n +#define FMT_MAKE_ARG_TYPE(n) T##n +#define FMT_MAKE_ARG(n) const T##n &v##n +#define FMT_ASSIGN_char(n) arr[n] = fmt::internal::MakeValue<fmt::BasicFormatter<char>>(v##n) +#define FMT_ASSIGN_wchar_t(n) arr[n] = fmt::internal::MakeValue<fmt::BasicFormatter<wchar_t>>(v##n) + +#if FMT_USE_VARIADIC_TEMPLATES +// Defines a variadic function returning void. +#define FMT_VARIADIC_VOID(func, arg_type) \ + template <typename... Args> void func(arg_type arg0, const Args &... args) \ + { \ + typedef fmt::internal::ArgArray<sizeof...(Args)> ArgArray; \ + typename ArgArray::Type array{ArgArray::template make<fmt::BasicFormatter<Char>>(args)...}; \ + func(arg0, fmt::ArgList(fmt::internal::make_type(args...), array)); \ + } + +// Defines a variadic constructor. +#define FMT_VARIADIC_CTOR(ctor, func, arg0_type, arg1_type) \ + template <typename... Args> ctor(arg0_type arg0, arg1_type arg1, const Args &... args) \ + { \ + typedef fmt::internal::ArgArray<sizeof...(Args)> ArgArray; \ + typename ArgArray::Type array{ArgArray::template make<fmt::BasicFormatter<Char>>(args)...}; \ + func(arg0, arg1, fmt::ArgList(fmt::internal::make_type(args...), array)); \ + } + +#else + +#define FMT_MAKE_REF(n) fmt::internal::MakeValue<fmt::BasicFormatter<Char>>(v##n) +#define FMT_MAKE_REF2(n) v##n + +// Defines a wrapper for a function taking one argument of type arg_type +// and n additional arguments of arbitrary types. +#define FMT_WRAP1(func, arg_type, n) \ + template <FMT_GEN(n, FMT_MAKE_TEMPLATE_ARG)> inline void func(arg_type arg1, FMT_GEN(n, FMT_MAKE_ARG)) \ + { \ + const fmt::internal::ArgArray<n>::Type array = {FMT_GEN(n, FMT_MAKE_REF)}; \ + func(arg1, fmt::ArgList(fmt::internal::make_type(FMT_GEN(n, FMT_MAKE_REF2)), array)); \ + } + +// Emulates a variadic function returning void on a pre-C++11 compiler. +#define FMT_VARIADIC_VOID(func, arg_type) \ + inline void func(arg_type arg) \ + { \ + func(arg, fmt::ArgList()); \ + } \ + FMT_WRAP1(func, arg_type, 1) \ + FMT_WRAP1(func, arg_type, 2) \ + FMT_WRAP1(func, arg_type, 3) \ + FMT_WRAP1(func, arg_type, 4) \ + FMT_WRAP1(func, arg_type, 5) FMT_WRAP1(func, arg_type, 6) FMT_WRAP1(func, arg_type, 7) FMT_WRAP1(func, arg_type, 8) \ + FMT_WRAP1(func, arg_type, 9) FMT_WRAP1(func, arg_type, 10) + +#define FMT_CTOR(ctor, func, arg0_type, arg1_type, n) \ + template <FMT_GEN(n, FMT_MAKE_TEMPLATE_ARG)> ctor(arg0_type arg0, arg1_type arg1, FMT_GEN(n, FMT_MAKE_ARG)) \ + { \ + const fmt::internal::ArgArray<n>::Type array = {FMT_GEN(n, FMT_MAKE_REF)}; \ + func(arg0, arg1, fmt::ArgList(fmt::internal::make_type(FMT_GEN(n, FMT_MAKE_REF2)), array)); \ + } + +// Emulates a variadic constructor on a pre-C++11 compiler. +#define FMT_VARIADIC_CTOR(ctor, func, arg0_type, arg1_type) \ + FMT_CTOR(ctor, func, arg0_type, arg1_type, 1) \ + FMT_CTOR(ctor, func, arg0_type, arg1_type, 2) \ + FMT_CTOR(ctor, func, arg0_type, arg1_type, 3) \ + FMT_CTOR(ctor, func, arg0_type, arg1_type, 4) \ + FMT_CTOR(ctor, func, arg0_type, arg1_type, 5) \ + FMT_CTOR(ctor, func, arg0_type, arg1_type, 6) \ + FMT_CTOR(ctor, func, arg0_type, arg1_type, 7) \ + FMT_CTOR(ctor, func, arg0_type, arg1_type, 8) \ + FMT_CTOR(ctor, func, arg0_type, arg1_type, 9) \ + FMT_CTOR(ctor, func, arg0_type, arg1_type, 10) +#endif + +// Generates a comma-separated list with results of applying f to pairs +// (argument, index). +#define FMT_FOR_EACH1(f, x0) f(x0, 0) +#define FMT_FOR_EACH2(f, x0, x1) FMT_FOR_EACH1(f, x0), f(x1, 1) +#define FMT_FOR_EACH3(f, x0, x1, x2) FMT_FOR_EACH2(f, x0, x1), f(x2, 2) +#define FMT_FOR_EACH4(f, x0, x1, x2, x3) FMT_FOR_EACH3(f, x0, x1, x2), f(x3, 3) +#define FMT_FOR_EACH5(f, x0, x1, x2, x3, x4) FMT_FOR_EACH4(f, x0, x1, x2, x3), f(x4, 4) +#define FMT_FOR_EACH6(f, x0, x1, x2, x3, x4, x5) FMT_FOR_EACH5(f, x0, x1, x2, x3, x4), f(x5, 5) +#define FMT_FOR_EACH7(f, x0, x1, x2, x3, x4, x5, x6) FMT_FOR_EACH6(f, x0, x1, x2, x3, x4, x5), f(x6, 6) +#define FMT_FOR_EACH8(f, x0, x1, x2, x3, x4, x5, x6, x7) FMT_FOR_EACH7(f, x0, x1, x2, x3, x4, x5, x6), f(x7, 7) +#define FMT_FOR_EACH9(f, x0, x1, x2, x3, x4, x5, x6, x7, x8) FMT_FOR_EACH8(f, x0, x1, x2, x3, x4, x5, x6, x7), f(x8, 8) +#define FMT_FOR_EACH10(f, x0, x1, x2, x3, x4, x5, x6, x7, x8, x9) FMT_FOR_EACH9(f, x0, x1, x2, x3, x4, x5, x6, x7, x8), f(x9, 9) + +/** + An error returned by an operating system or a language runtime, + for example a file opening error. +*/ +class SystemError : public internal::RuntimeError +{ +private: + FMT_API void init(int err_code, CStringRef format_str, ArgList args); + +protected: + int error_code_; + + typedef char Char; // For FMT_VARIADIC_CTOR. + + SystemError() {} + +public: + /** + \rst + Constructs a :class:`fmt::SystemError` object with a description + formatted with `fmt::format_system_error`. *message* and additional + arguments passed into the constructor are formatted similarly to + `fmt::format`. + + **Example**:: + + // This throws a SystemError with the description + // cannot open file 'madeup': No such file or directory + // or similar (system message may vary). + const char *filename = "madeup"; + std::FILE *file = std::fopen(filename, "r"); + if (!file) + throw fmt::SystemError(errno, "cannot open file '{}'", filename); + \endrst + */ + SystemError(int error_code, CStringRef message) + { + init(error_code, message, ArgList()); + } + FMT_DEFAULTED_COPY_CTOR(SystemError) + FMT_VARIADIC_CTOR(SystemError, init, int, CStringRef) + + FMT_API ~SystemError() FMT_DTOR_NOEXCEPT FMT_OVERRIDE; + + int error_code() const + { + return error_code_; + } +}; + +/** + \rst + Formats an error returned by an operating system or a language runtime, + for example a file opening error, and writes it to *out* in the following + form: + + .. parsed-literal:: + *<message>*: *<system-message>* + + where *<message>* is the passed message and *<system-message>* is + the system message corresponding to the error code. + *error_code* is a system error code as given by ``errno``. + If *error_code* is not a valid error code such as -1, the system message + may look like "Unknown error -1" and is platform-dependent. + \endrst + */ +FMT_API void format_system_error(fmt::Writer &out, int error_code, fmt::StringRef message) FMT_NOEXCEPT; + +/** + \rst + This template provides operations for formatting and writing data into + a character stream. The output is stored in a buffer provided by a subclass + such as :class:`fmt::BasicMemoryWriter`. + + You can use one of the following typedefs for common character types: + + +---------+----------------------+ + | Type | Definition | + +=========+======================+ + | Writer | BasicWriter<char> | + +---------+----------------------+ + | WWriter | BasicWriter<wchar_t> | + +---------+----------------------+ + + \endrst + */ +template <typename Char> class BasicWriter +{ +private: + // Output buffer. + Buffer<Char> &buffer_; + + FMT_DISALLOW_COPY_AND_ASSIGN(BasicWriter); + + typedef typename internal::CharTraits<Char>::CharPtr CharPtr; + +#if FMT_SECURE_SCL + // Returns pointer value. + static Char *get(CharPtr p) + { + return p.base(); + } +#else + static Char *get(Char *p) + { + return p; + } +#endif + + // Fills the padding around the content and returns the pointer to the + // content area. + static CharPtr fill_padding(CharPtr buffer, unsigned total_size, std::size_t content_size, wchar_t fill); + + // Grows the buffer by n characters and returns a pointer to the newly + // allocated area. + CharPtr grow_buffer(std::size_t n) + { + std::size_t size = buffer_.size(); + buffer_.resize(size + n); + return internal::make_ptr(&buffer_[size], n); + } + + // Writes an unsigned decimal integer. + template <typename UInt> Char *write_unsigned_decimal(UInt value, unsigned prefix_size = 0) + { + unsigned num_digits = internal::count_digits(value); + Char *ptr = get(grow_buffer(prefix_size + num_digits)); + internal::format_decimal(ptr + prefix_size, value, num_digits); + return ptr; + } + + // Writes a decimal integer. + template <typename Int> void write_decimal(Int value) + { + typedef typename internal::IntTraits<Int>::MainType MainType; + MainType abs_value = static_cast<MainType>(value); + if (internal::is_negative(value)) + { + abs_value = 0 - abs_value; + *write_unsigned_decimal(abs_value, 1) = '-'; + } + else + { + write_unsigned_decimal(abs_value, 0); + } + } + + // Prepare a buffer for integer formatting. + CharPtr prepare_int_buffer(unsigned num_digits, const EmptySpec &, const char *prefix, unsigned prefix_size) + { + unsigned size = prefix_size + num_digits; + CharPtr p = grow_buffer(size); + std::uninitialized_copy(prefix, prefix + prefix_size, p); + return p + size - 1; + } + + template <typename Spec> CharPtr prepare_int_buffer(unsigned num_digits, const Spec &spec, const char *prefix, unsigned prefix_size); + + // Formats an integer. + template <typename T, typename Spec> void write_int(T value, Spec spec); + + // Formats a floating-point number (double or long double). + template <typename T, typename Spec> void write_double(T value, const Spec &spec); + + // Writes a formatted string. + template <typename StrChar> CharPtr write_str(const StrChar *s, std::size_t size, const AlignSpec &spec); + + template <typename StrChar, typename Spec> void write_str(const internal::Arg::StringValue<StrChar> &str, const Spec &spec); + + // This following methods are private to disallow writing wide characters + // and strings to a char stream. If you want to print a wide string as a + // pointer as std::ostream does, cast it to const void*. + // Do not implement! + void operator<<(typename internal::WCharHelper<wchar_t, Char>::Unsupported); + void operator<<(typename internal::WCharHelper<const wchar_t *, Char>::Unsupported); + + // Appends floating-point length specifier to the format string. + // The second argument is only used for overload resolution. + void append_float_length(Char *&format_ptr, long double) + { + *format_ptr++ = 'L'; + } + + template <typename T> void append_float_length(Char *&, T) {} + + template <typename Impl, typename Char_, typename Spec_> friend class internal::ArgFormatterBase; + + template <typename Impl, typename Char_, typename Spec_> friend class BasicPrintfArgFormatter; + +protected: + /** + Constructs a ``BasicWriter`` object. + */ + explicit BasicWriter(Buffer<Char> &b) + : buffer_(b) + { + } + +public: + /** + \rst + Destroys a ``BasicWriter`` object. + \endrst + */ + virtual ~BasicWriter() {} + + /** + Returns the total number of characters written. + */ + std::size_t size() const + { + return buffer_.size(); + } + + /** + Returns a pointer to the output buffer content. No terminating null + character is appended. + */ + const Char *data() const FMT_NOEXCEPT + { + return &buffer_[0]; + } + + /** + Returns a pointer to the output buffer content with terminating null + character appended. + */ + const Char *c_str() const + { + std::size_t size = buffer_.size(); + buffer_.reserve(size + 1); + buffer_[size] = '\0'; + return &buffer_[0]; + } + + /** + \rst + Returns the content of the output buffer as an `std::string`. + \endrst + */ + std::basic_string<Char> str() const + { + return std::basic_string<Char>(&buffer_[0], buffer_.size()); + } + + /** + \rst + Writes formatted data. + + *args* is an argument list representing arbitrary arguments. + + **Example**:: + + MemoryWriter out; + out.write("Current point:\n"); + out.write("({:+f}, {:+f})", -3.14, 3.14); + + This will write the following output to the ``out`` object: + + .. code-block:: none + + Current point: + (-3.140000, +3.140000) + + The output can be accessed using :func:`data()`, :func:`c_str` or + :func:`str` methods. + + See also :ref:`syntax`. + \endrst + */ + void write(BasicCStringRef<Char> format, ArgList args) + { + BasicFormatter<Char>(args, *this).format(format); + } + FMT_VARIADIC_VOID(write, BasicCStringRef<Char>) + + BasicWriter &operator<<(int value) + { + write_decimal(value); + return *this; + } + BasicWriter &operator<<(unsigned value) + { + return *this << IntFormatSpec<unsigned>(value); + } + BasicWriter &operator<<(long value) + { + write_decimal(value); + return *this; + } + BasicWriter &operator<<(unsigned long value) + { + return *this << IntFormatSpec<unsigned long>(value); + } + BasicWriter &operator<<(LongLong value) + { + write_decimal(value); + return *this; + } + + /** + \rst + Formats *value* and writes it to the stream. + \endrst + */ + BasicWriter &operator<<(ULongLong value) + { + return *this << IntFormatSpec<ULongLong>(value); + } + + BasicWriter &operator<<(double value) + { + write_double(value, FormatSpec()); + return *this; + } + + /** + \rst + Formats *value* using the general format for floating-point numbers + (``'g'``) and writes it to the stream. + \endrst + */ + BasicWriter &operator<<(long double value) + { + write_double(value, FormatSpec()); + return *this; + } + + /** + Writes a character to the stream. + */ + BasicWriter &operator<<(char value) + { + buffer_.push_back(value); + return *this; + } + + BasicWriter &operator<<(typename internal::WCharHelper<wchar_t, Char>::Supported value) + { + buffer_.push_back(value); + return *this; + } + + /** + \rst + Writes *value* to the stream. + \endrst + */ + BasicWriter &operator<<(fmt::BasicStringRef<Char> value) + { + const Char *str = value.data(); + buffer_.append(str, str + value.size()); + return *this; + } + + BasicWriter &operator<<(typename internal::WCharHelper<StringRef, Char>::Supported value) + { + const char *str = value.data(); + buffer_.append(str, str + value.size()); + return *this; + } + + template <typename T, typename Spec, typename FillChar> BasicWriter &operator<<(IntFormatSpec<T, Spec, FillChar> spec) + { + internal::CharTraits<Char>::convert(FillChar()); + write_int(spec.value(), spec); + return *this; + } + + template <typename StrChar> BasicWriter &operator<<(const StrFormatSpec<StrChar> &spec) + { + const StrChar *s = spec.str(); + write_str(s, std::char_traits<Char>::length(s), spec); + return *this; + } + + void clear() FMT_NOEXCEPT + { + buffer_.clear(); + } + + Buffer<Char> &buffer() FMT_NOEXCEPT + { + return buffer_; + } +}; + +template <typename Char> +template <typename StrChar> +typename BasicWriter<Char>::CharPtr BasicWriter<Char>::write_str(const StrChar *s, std::size_t size, const AlignSpec &spec) +{ + CharPtr out = CharPtr(); + if (spec.width() > size) + { + out = grow_buffer(spec.width()); + Char fill = internal::CharTraits<Char>::cast(spec.fill()); + if (spec.align() == ALIGN_RIGHT) + { + std::uninitialized_fill_n(out, spec.width() - size, fill); + out += spec.width() - size; + } + else if (spec.align() == ALIGN_CENTER) + { + out = fill_padding(out, spec.width(), size, fill); + } + else + { + std::uninitialized_fill_n(out + size, spec.width() - size, fill); + } + } + else + { + out = grow_buffer(size); + } + std::uninitialized_copy(s, s + size, out); + return out; +} + +template <typename Char> +template <typename StrChar, typename Spec> +void BasicWriter<Char>::write_str(const internal::Arg::StringValue<StrChar> &s, const Spec &spec) +{ + // Check if StrChar is convertible to Char. + internal::CharTraits<Char>::convert(StrChar()); + if (spec.type_ && spec.type_ != 's') + internal::report_unknown_type(spec.type_, "string"); + const StrChar *str_value = s.value; + std::size_t str_size = s.size; + if (str_size == 0) + { + if (!str_value) + { + FMT_THROW(FormatError("string pointer is null")); + } + } + std::size_t precision = static_cast<std::size_t>(spec.precision_); + if (spec.precision_ >= 0 && precision < str_size) + str_size = precision; + write_str(str_value, str_size, spec); +} + +template <typename Char> +typename BasicWriter<Char>::CharPtr BasicWriter<Char>::fill_padding( + CharPtr buffer, unsigned total_size, std::size_t content_size, wchar_t fill) +{ + std::size_t padding = total_size - content_size; + std::size_t left_padding = padding / 2; + Char fill_char = internal::CharTraits<Char>::cast(fill); + std::uninitialized_fill_n(buffer, left_padding, fill_char); + buffer += left_padding; + CharPtr content = buffer; + std::uninitialized_fill_n(buffer + content_size, padding - left_padding, fill_char); + return content; +} + +template <typename Char> +template <typename Spec> +typename BasicWriter<Char>::CharPtr BasicWriter<Char>::prepare_int_buffer( + unsigned num_digits, const Spec &spec, const char *prefix, unsigned prefix_size) +{ + unsigned width = spec.width(); + Alignment align = spec.align(); + Char fill = internal::CharTraits<Char>::cast(spec.fill()); + if (spec.precision() > static_cast<int>(num_digits)) + { + // Octal prefix '0' is counted as a digit, so ignore it if precision + // is specified. + if (prefix_size > 0 && prefix[prefix_size - 1] == '0') + --prefix_size; + unsigned number_size = prefix_size + internal::to_unsigned(spec.precision()); + AlignSpec subspec(number_size, '0', ALIGN_NUMERIC); + if (number_size >= width) + return prepare_int_buffer(num_digits, subspec, prefix, prefix_size); + buffer_.reserve(width); + unsigned fill_size = width - number_size; + if (align != ALIGN_LEFT) + { + CharPtr p = grow_buffer(fill_size); + std::uninitialized_fill(p, p + fill_size, fill); + } + CharPtr result = prepare_int_buffer(num_digits, subspec, prefix, prefix_size); + if (align == ALIGN_LEFT) + { + CharPtr p = grow_buffer(fill_size); + std::uninitialized_fill(p, p + fill_size, fill); + } + return result; + } + unsigned size = prefix_size + num_digits; + if (width <= size) + { + CharPtr p = grow_buffer(size); + std::uninitialized_copy(prefix, prefix + prefix_size, p); + return p + size - 1; + } + CharPtr p = grow_buffer(width); + CharPtr end = p + width; + if (align == ALIGN_LEFT) + { + std::uninitialized_copy(prefix, prefix + prefix_size, p); + p += size; + std::uninitialized_fill(p, end, fill); + } + else if (align == ALIGN_CENTER) + { + p = fill_padding(p, width, size, fill); + std::uninitialized_copy(prefix, prefix + prefix_size, p); + p += size; + } + else + { + if (align == ALIGN_NUMERIC) + { + if (prefix_size != 0) + { + p = std::uninitialized_copy(prefix, prefix + prefix_size, p); + size -= prefix_size; + } + } + else + { + std::uninitialized_copy(prefix, prefix + prefix_size, end - size); + } + std::uninitialized_fill(p, end - size, fill); + p = end; + } + return p - 1; +} + +template <typename Char> template <typename T, typename Spec> void BasicWriter<Char>::write_int(T value, Spec spec) +{ + unsigned prefix_size = 0; + typedef typename internal::IntTraits<T>::MainType UnsignedType; + UnsignedType abs_value = static_cast<UnsignedType>(value); + char prefix[4] = ""; + if (internal::is_negative(value)) + { + prefix[0] = '-'; + ++prefix_size; + abs_value = 0 - abs_value; + } + else if (spec.flag(SIGN_FLAG)) + { + prefix[0] = spec.flag(PLUS_FLAG) ? '+' : ' '; + ++prefix_size; + } + switch (spec.type()) + { + case 0: + case 'd': + { + unsigned num_digits = internal::count_digits(abs_value); + CharPtr p = prepare_int_buffer(num_digits, spec, prefix, prefix_size) + 1; + internal::format_decimal(get(p), abs_value, 0); + break; + } + case 'x': + case 'X': + { + UnsignedType n = abs_value; + if (spec.flag(HASH_FLAG)) + { + prefix[prefix_size++] = '0'; + prefix[prefix_size++] = spec.type_prefix(); + } + unsigned num_digits = 0; + do + { + ++num_digits; + } while ((n >>= 4) != 0); + Char *p = get(prepare_int_buffer(num_digits, spec, prefix, prefix_size)); + n = abs_value; + const char *digits = spec.type() == 'x' ? "0123456789abcdef" : "0123456789ABCDEF"; + do + { + *p-- = digits[n & 0xf]; + } while ((n >>= 4) != 0); + break; + } + case 'b': + case 'B': + { + UnsignedType n = abs_value; + if (spec.flag(HASH_FLAG)) + { + prefix[prefix_size++] = '0'; + prefix[prefix_size++] = spec.type_prefix(); + } + unsigned num_digits = 0; + do + { + ++num_digits; + } while ((n >>= 1) != 0); + Char *p = get(prepare_int_buffer(num_digits, spec, prefix, prefix_size)); + n = abs_value; + do + { + *p-- = static_cast<Char>('0' + (n & 1)); + } while ((n >>= 1) != 0); + break; + } + case 'o': + { + UnsignedType n = abs_value; + if (spec.flag(HASH_FLAG)) + prefix[prefix_size++] = '0'; + unsigned num_digits = 0; + do + { + ++num_digits; + } while ((n >>= 3) != 0); + Char *p = get(prepare_int_buffer(num_digits, spec, prefix, prefix_size)); + n = abs_value; + do + { + *p-- = static_cast<Char>('0' + (n & 7)); + } while ((n >>= 3) != 0); + break; + } + case 'n': + { + unsigned num_digits = internal::count_digits(abs_value); + fmt::StringRef sep = ""; +#if !(defined(ANDROID) || defined(__ANDROID__)) + sep = internal::thousands_sep(std::localeconv()); +#endif + unsigned size = static_cast<unsigned>(num_digits + sep.size() * ((num_digits - 1) / 3)); + CharPtr p = prepare_int_buffer(size, spec, prefix, prefix_size) + 1; + internal::format_decimal(get(p), abs_value, 0, internal::ThousandsSep(sep)); + break; + } + default: + internal::report_unknown_type(spec.type(), spec.flag(CHAR_FLAG) ? "char" : "integer"); + break; + } +} + +template <typename Char> template <typename T, typename Spec> void BasicWriter<Char>::write_double(T value, const Spec &spec) +{ + // Check type. + char type = spec.type(); + bool upper = false; + switch (type) + { + case 0: + type = 'g'; + break; + case 'e': + case 'f': + case 'g': + case 'a': + break; + case 'F': +#if FMT_MSC_VER + // MSVC's printf doesn't support 'F'. + type = 'f'; +#endif + // Fall through. + case 'E': + case 'G': + case 'A': + upper = true; + break; + default: + internal::report_unknown_type(type, "double"); + break; + } + + char sign = 0; + // Use isnegative instead of value < 0 because the latter is always + // false for NaN. + if (internal::FPUtil::isnegative(static_cast<double>(value))) + { + sign = '-'; + value = -value; + } + else if (spec.flag(SIGN_FLAG)) + { + sign = spec.flag(PLUS_FLAG) ? '+' : ' '; + } + + if (internal::FPUtil::isnotanumber(value)) + { + // Format NaN ourselves because sprintf's output is not consistent + // across platforms. + std::size_t nan_size = 4; + const char *nan = upper ? " NAN" : " nan"; + if (!sign) + { + --nan_size; + ++nan; + } + CharPtr out = write_str(nan, nan_size, spec); + if (sign) + *out = sign; + return; + } + + if (internal::FPUtil::isinfinity(value)) + { + // Format infinity ourselves because sprintf's output is not consistent + // across platforms. + std::size_t inf_size = 4; + const char *inf = upper ? " INF" : " inf"; + if (!sign) + { + --inf_size; + ++inf; + } + CharPtr out = write_str(inf, inf_size, spec); + if (sign) + *out = sign; + return; + } + + std::size_t offset = buffer_.size(); + unsigned width = spec.width(); + if (sign) + { + buffer_.reserve(buffer_.size() + (width > 1u ? width : 1u)); + if (width > 0) + --width; + ++offset; + } + + // Build format string. + enum + { + MAX_FORMAT_SIZE = 10 + }; // longest format: %#-*.*Lg + Char format[MAX_FORMAT_SIZE]; + Char *format_ptr = format; + *format_ptr++ = '%'; + unsigned width_for_sprintf = width; + if (spec.flag(HASH_FLAG)) + *format_ptr++ = '#'; + if (spec.align() == ALIGN_CENTER) + { + width_for_sprintf = 0; + } + else + { + if (spec.align() == ALIGN_LEFT) + *format_ptr++ = '-'; + if (width != 0) + *format_ptr++ = '*'; + } + if (spec.precision() >= 0) + { + *format_ptr++ = '.'; + *format_ptr++ = '*'; + } + + append_float_length(format_ptr, value); + *format_ptr++ = type; + *format_ptr = '\0'; + + // Format using snprintf. + Char fill = internal::CharTraits<Char>::cast(spec.fill()); + unsigned n = 0; + Char *start = FMT_NULL; + for (;;) + { + std::size_t buffer_size = buffer_.capacity() - offset; +#if FMT_MSC_VER + // MSVC's vsnprintf_s doesn't work with zero size, so reserve + // space for at least one extra character to make the size non-zero. + // Note that the buffer's capacity will increase by more than 1. + if (buffer_size == 0) + { + buffer_.reserve(offset + 1); + buffer_size = buffer_.capacity() - offset; + } +#endif + start = &buffer_[offset]; + int result = internal::CharTraits<Char>::format_float(start, buffer_size, format, width_for_sprintf, spec.precision(), value); + if (result >= 0) + { + n = internal::to_unsigned(result); + if (offset + n < buffer_.capacity()) + break; // The buffer is large enough - continue with formatting. + buffer_.reserve(offset + n + 1); + } + else + { + // If result is negative we ask to increase the capacity by at least 1, + // but as std::vector, the buffer grows exponentially. + buffer_.reserve(buffer_.capacity() + 1); + } + } + if (sign) + { + if ((spec.align() != ALIGN_RIGHT && spec.align() != ALIGN_DEFAULT) || *start != ' ') + { + *(start - 1) = sign; + sign = 0; + } + else + { + *(start - 1) = fill; + } + ++n; + } + if (spec.align() == ALIGN_CENTER && spec.width() > n) + { + width = spec.width(); + CharPtr p = grow_buffer(width); + std::memmove(get(p) + (width - n) / 2, get(p), n * sizeof(Char)); + fill_padding(p, spec.width(), n, fill); + return; + } + if (spec.fill() != ' ' || sign) + { + while (*start == ' ') + *start++ = fill; + if (sign) + *(start - 1) = sign; + } + grow_buffer(n); +} + +/** + \rst + This class template provides operations for formatting and writing data + into a character stream. The output is stored in a memory buffer that grows + dynamically. + + You can use one of the following typedefs for common character types + and the standard allocator: + + +---------------+-----------------------------------------------------+ + | Type | Definition | + +===============+=====================================================+ + | MemoryWriter | BasicMemoryWriter<char, std::allocator<char>> | + +---------------+-----------------------------------------------------+ + | WMemoryWriter | BasicMemoryWriter<wchar_t, std::allocator<wchar_t>> | + +---------------+-----------------------------------------------------+ + + **Example**:: + + MemoryWriter out; + out << "The answer is " << 42 << "\n"; + out.write("({:+f}, {:+f})", -3.14, 3.14); + + This will write the following output to the ``out`` object: + + .. code-block:: none + + The answer is 42 + (-3.140000, +3.140000) + + The output can be converted to an ``std::string`` with ``out.str()`` or + accessed as a C string with ``out.c_str()``. + \endrst + */ +template <typename Char, typename Allocator = std::allocator<Char>> class BasicMemoryWriter : public BasicWriter<Char> +{ +private: + internal::MemoryBuffer<Char, internal::INLINE_BUFFER_SIZE, Allocator> buffer_; + +public: + explicit BasicMemoryWriter(const Allocator &alloc = Allocator()) + : BasicWriter<Char>(buffer_) + , buffer_(alloc) + { + } + +#if FMT_USE_RVALUE_REFERENCES + /** + \rst + Constructs a :class:`fmt::BasicMemoryWriter` object moving the content + of the other object to it. + \endrst + */ + BasicMemoryWriter(BasicMemoryWriter &&other) + : BasicWriter<Char>(buffer_) + , buffer_(std::move(other.buffer_)) + { + } + + /** + \rst + Moves the content of the other ``BasicMemoryWriter`` object to this one. + \endrst + */ + BasicMemoryWriter &operator=(BasicMemoryWriter &&other) + { + buffer_ = std::move(other.buffer_); + return *this; + } +#endif +}; + +typedef BasicMemoryWriter<char> MemoryWriter; +typedef BasicMemoryWriter<wchar_t> WMemoryWriter; + +/** + \rst + This class template provides operations for formatting and writing data + into a fixed-size array. For writing into a dynamically growing buffer + use :class:`fmt::BasicMemoryWriter`. + + Any write method will throw ``std::runtime_error`` if the output doesn't fit + into the array. + + You can use one of the following typedefs for common character types: + + +--------------+---------------------------+ + | Type | Definition | + +==============+===========================+ + | ArrayWriter | BasicArrayWriter<char> | + +--------------+---------------------------+ + | WArrayWriter | BasicArrayWriter<wchar_t> | + +--------------+---------------------------+ + \endrst + */ +template <typename Char> class BasicArrayWriter : public BasicWriter<Char> +{ +private: + internal::FixedBuffer<Char> buffer_; + +public: + /** + \rst + Constructs a :class:`fmt::BasicArrayWriter` object for *array* of the + given size. + \endrst + */ + BasicArrayWriter(Char *array, std::size_t size) + : BasicWriter<Char>(buffer_) + , buffer_(array, size) + { + } + + /** + \rst + Constructs a :class:`fmt::BasicArrayWriter` object for *array* of the + size known at compile time. + \endrst + */ + template <std::size_t SIZE> + explicit BasicArrayWriter(Char (&array)[SIZE]) + : BasicWriter<Char>(buffer_) + , buffer_(array, SIZE) + { + } +}; + +typedef BasicArrayWriter<char> ArrayWriter; +typedef BasicArrayWriter<wchar_t> WArrayWriter; + +// Reports a system error without throwing an exception. +// Can be used to report errors from destructors. +FMT_API void report_system_error(int error_code, StringRef message) FMT_NOEXCEPT; + +#if FMT_USE_WINDOWS_H + +/** A Windows error. */ +class WindowsError : public SystemError +{ +private: + FMT_API void init(int error_code, CStringRef format_str, ArgList args); + +public: + /** + \rst + Constructs a :class:`fmt::WindowsError` object with the description + of the form + + .. parsed-literal:: + *<message>*: *<system-message>* + + where *<message>* is the formatted message and *<system-message>* is the + system message corresponding to the error code. + *error_code* is a Windows error code as given by ``GetLastError``. + If *error_code* is not a valid error code such as -1, the system message + will look like "error -1". + + **Example**:: + + // This throws a WindowsError with the description + // cannot open file 'madeup': The system cannot find the file specified. + // or similar (system message may vary). + const char *filename = "madeup"; + LPOFSTRUCT of = LPOFSTRUCT(); + HFILE file = OpenFile(filename, &of, OF_READ); + if (file == HFILE_ERROR) { + throw fmt::WindowsError(GetLastError(), + "cannot open file '{}'", filename); + } + \endrst + */ + WindowsError(int error_code, CStringRef message) + { + init(error_code, message, ArgList()); + } + FMT_VARIADIC_CTOR(WindowsError, init, int, CStringRef) +}; + +// Reports a Windows error without throwing an exception. +// Can be used to report errors from destructors. +FMT_API void report_windows_error(int error_code, StringRef message) FMT_NOEXCEPT; + +#endif + +enum Color +{ + BLACK, + RED, + GREEN, + YELLOW, + BLUE, + MAGENTA, + CYAN, + WHITE +}; + +/** + Formats a string and prints it to stdout using ANSI escape sequences + to specify color (experimental). + Example: + print_colored(fmt::RED, "Elapsed time: {0:.2f} seconds", 1.23); + */ +FMT_API void print_colored(Color c, CStringRef format, ArgList args); + +/** + \rst + Formats arguments and returns the result as a string. + + **Example**:: + + std::string message = format("The answer is {}", 42); + \endrst +*/ +inline std::string format(CStringRef format_str, ArgList args) +{ + MemoryWriter w; + w.write(format_str, args); + return w.str(); +} + +inline std::wstring format(WCStringRef format_str, ArgList args) +{ + WMemoryWriter w; + w.write(format_str, args); + return w.str(); +} + +/** + \rst + Prints formatted data to the file *f*. + + **Example**:: + + print(stderr, "Don't {}!", "panic"); + \endrst + */ +FMT_API void print(std::FILE *f, CStringRef format_str, ArgList args); + +/** + \rst + Prints formatted data to ``stdout``. + + **Example**:: + + print("Elapsed time: {0:.2f} seconds", 1.23); + \endrst + */ +FMT_API void print(CStringRef format_str, ArgList args); + +/** + Fast integer formatter. + */ +class FormatInt +{ +private: + // Buffer should be large enough to hold all digits (digits10 + 1), + // a sign and a null character. + enum + { + BUFFER_SIZE = std::numeric_limits<ULongLong>::digits10 + 3 + }; + mutable char buffer_[BUFFER_SIZE]; + char *str_; + + // Formats value in reverse and returns the number of digits. + char *format_decimal(ULongLong value) + { + char *buffer_end = buffer_ + BUFFER_SIZE - 1; + while (value >= 100) + { + // Integer division is slow so do it for a group of two digits instead + // of for every digit. The idea comes from the talk by Alexandrescu + // "Three Optimization Tips for C++". See speed-test for a comparison. + unsigned index = static_cast<unsigned>((value % 100) * 2); + value /= 100; + *--buffer_end = internal::Data::DIGITS[index + 1]; + *--buffer_end = internal::Data::DIGITS[index]; + } + if (value < 10) + { + *--buffer_end = static_cast<char>('0' + value); + return buffer_end; + } + unsigned index = static_cast<unsigned>(value * 2); + *--buffer_end = internal::Data::DIGITS[index + 1]; + *--buffer_end = internal::Data::DIGITS[index]; + return buffer_end; + } + + void FormatSigned(LongLong value) + { + ULongLong abs_value = static_cast<ULongLong>(value); + bool negative = value < 0; + if (negative) + abs_value = 0 - abs_value; + str_ = format_decimal(abs_value); + if (negative) + *--str_ = '-'; + } + +public: + explicit FormatInt(int value) + { + FormatSigned(value); + } + explicit FormatInt(long value) + { + FormatSigned(value); + } + explicit FormatInt(LongLong value) + { + FormatSigned(value); + } + explicit FormatInt(unsigned value) + : str_(format_decimal(value)) + { + } + explicit FormatInt(unsigned long value) + : str_(format_decimal(value)) + { + } + explicit FormatInt(ULongLong value) + : str_(format_decimal(value)) + { + } + + /** Returns the number of characters written to the output buffer. */ + std::size_t size() const + { + return internal::to_unsigned(buffer_ - str_ + BUFFER_SIZE - 1); + } + + /** + Returns a pointer to the output buffer content. No terminating null + character is appended. + */ + const char *data() const + { + return str_; + } + + /** + Returns a pointer to the output buffer content with terminating null + character appended. + */ + const char *c_str() const + { + buffer_[BUFFER_SIZE - 1] = '\0'; + return str_; + } + + /** + \rst + Returns the content of the output buffer as an ``std::string``. + \endrst + */ + std::string str() const + { + return std::string(str_, size()); + } +}; + +// Formats a decimal integer value writing into buffer and returns +// a pointer to the end of the formatted string. This function doesn't +// write a terminating null character. +template <typename T> inline void format_decimal(char *&buffer, T value) +{ + typedef typename internal::IntTraits<T>::MainType MainType; + MainType abs_value = static_cast<MainType>(value); + if (internal::is_negative(value)) + { + *buffer++ = '-'; + abs_value = 0 - abs_value; + } + if (abs_value < 100) + { + if (abs_value < 10) + { + *buffer++ = static_cast<char>('0' + abs_value); + return; + } + unsigned index = static_cast<unsigned>(abs_value * 2); + *buffer++ = internal::Data::DIGITS[index]; + *buffer++ = internal::Data::DIGITS[index + 1]; + return; + } + unsigned num_digits = internal::count_digits(abs_value); + internal::format_decimal(buffer, abs_value, num_digits); + buffer += num_digits; +} + +/** + \rst + Returns a named argument for formatting functions. + + **Example**:: + + print("Elapsed time: {s:.2f} seconds", arg("s", 1.23)); + + \endrst + */ +template <typename T> inline internal::NamedArgWithType<char, T> arg(StringRef name, const T &arg) +{ + return internal::NamedArgWithType<char, T>(name, arg); +} + +template <typename T> inline internal::NamedArgWithType<wchar_t, T> arg(WStringRef name, const T &arg) +{ + return internal::NamedArgWithType<wchar_t, T>(name, arg); +} + +// The following two functions are deleted intentionally to disable +// nested named arguments as in ``format("{}", arg("a", arg("b", 42)))``. +template <typename Char> void arg(StringRef, const internal::NamedArg<Char> &) FMT_DELETED_OR_UNDEFINED; +template <typename Char> void arg(WStringRef, const internal::NamedArg<Char> &) FMT_DELETED_OR_UNDEFINED; +} // namespace fmt + +#if FMT_GCC_VERSION +// Use the system_header pragma to suppress warnings about variadic macros +// because suppressing -Wvariadic-macros with the diagnostic pragma doesn't +// work. It is used at the end because we want to suppress as little warnings +// as possible. +#pragma GCC system_header +#endif + +// This is used to work around VC++ bugs in handling variadic macros. +#define FMT_EXPAND(args) args + +// Returns the number of arguments. +// Based on https://groups.google.com/forum/#!topic/comp.std.c/d-6Mj5Lko_s. +#define FMT_NARG(...) FMT_NARG_(__VA_ARGS__, FMT_RSEQ_N()) +#define FMT_NARG_(...) FMT_EXPAND(FMT_ARG_N(__VA_ARGS__)) +#define FMT_ARG_N(_1, _2, _3, _4, _5, _6, _7, _8, _9, _10, N, ...) N +#define FMT_RSEQ_N() 10, 9, 8, 7, 6, 5, 4, 3, 2, 1, 0 + +#define FMT_FOR_EACH_(N, f, ...) FMT_EXPAND(FMT_CONCAT(FMT_FOR_EACH, N)(f, __VA_ARGS__)) +#define FMT_FOR_EACH(f, ...) FMT_EXPAND(FMT_FOR_EACH_(FMT_NARG(__VA_ARGS__), f, __VA_ARGS__)) + +#define FMT_ADD_ARG_NAME(type, index) type arg##index +#define FMT_GET_ARG_NAME(type, index) arg##index + +#if FMT_USE_VARIADIC_TEMPLATES +#define FMT_VARIADIC_(Const, Char, ReturnType, func, call, ...) \ + template <typename... Args> ReturnType func(FMT_FOR_EACH(FMT_ADD_ARG_NAME, __VA_ARGS__), const Args &... args) Const \ + { \ + typedef fmt::internal::ArgArray<sizeof...(Args)> ArgArray; \ + typename ArgArray::Type array{ArgArray::template make<fmt::BasicFormatter<Char>>(args)...}; \ + call(FMT_FOR_EACH(FMT_GET_ARG_NAME, __VA_ARGS__), fmt::ArgList(fmt::internal::make_type(args...), array)); \ + } +#else +// Defines a wrapper for a function taking __VA_ARGS__ arguments +// and n additional arguments of arbitrary types. +#define FMT_WRAP(Const, Char, ReturnType, func, call, n, ...) \ + template <FMT_GEN(n, FMT_MAKE_TEMPLATE_ARG)> \ + inline ReturnType func(FMT_FOR_EACH(FMT_ADD_ARG_NAME, __VA_ARGS__), FMT_GEN(n, FMT_MAKE_ARG)) Const \ + { \ + fmt::internal::ArgArray<n>::Type arr; \ + FMT_GEN(n, FMT_ASSIGN_##Char); \ + call(FMT_FOR_EACH(FMT_GET_ARG_NAME, __VA_ARGS__), fmt::ArgList(fmt::internal::make_type(FMT_GEN(n, FMT_MAKE_REF2)), arr)); \ + } + +#define FMT_VARIADIC_(Const, Char, ReturnType, func, call, ...) \ + inline ReturnType func(FMT_FOR_EACH(FMT_ADD_ARG_NAME, __VA_ARGS__)) Const \ + { \ + call(FMT_FOR_EACH(FMT_GET_ARG_NAME, __VA_ARGS__), fmt::ArgList()); \ + } \ + FMT_WRAP(Const, Char, ReturnType, func, call, 1, __VA_ARGS__) \ + FMT_WRAP(Const, Char, ReturnType, func, call, 2, __VA_ARGS__) \ + FMT_WRAP(Const, Char, ReturnType, func, call, 3, __VA_ARGS__) \ + FMT_WRAP(Const, Char, ReturnType, func, call, 4, __VA_ARGS__) \ + FMT_WRAP(Const, Char, ReturnType, func, call, 5, __VA_ARGS__) \ + FMT_WRAP(Const, Char, ReturnType, func, call, 6, __VA_ARGS__) \ + FMT_WRAP(Const, Char, ReturnType, func, call, 7, __VA_ARGS__) \ + FMT_WRAP(Const, Char, ReturnType, func, call, 8, __VA_ARGS__) \ + FMT_WRAP(Const, Char, ReturnType, func, call, 9, __VA_ARGS__) \ + FMT_WRAP(Const, Char, ReturnType, func, call, 10, __VA_ARGS__) \ + FMT_WRAP(Const, Char, ReturnType, func, call, 11, __VA_ARGS__) \ + FMT_WRAP(Const, Char, ReturnType, func, call, 12, __VA_ARGS__) \ + FMT_WRAP(Const, Char, ReturnType, func, call, 13, __VA_ARGS__) \ + FMT_WRAP(Const, Char, ReturnType, func, call, 14, __VA_ARGS__) \ + FMT_WRAP(Const, Char, ReturnType, func, call, 15, __VA_ARGS__) +#endif // FMT_USE_VARIADIC_TEMPLATES + +/** + \rst + Defines a variadic function with the specified return type, function name + and argument types passed as variable arguments to this macro. + + **Example**:: + + void print_error(const char *file, int line, const char *format, + fmt::ArgList args) { + fmt::print("{}: {}: ", file, line); + fmt::print(format, args); + } + FMT_VARIADIC(void, print_error, const char *, int, const char *) + + ``FMT_VARIADIC`` is used for compatibility with legacy C++ compilers that + don't implement variadic templates. You don't have to use this macro if + you don't need legacy compiler support and can use variadic templates + directly:: + + template <typename... Args> + void print_error(const char *file, int line, const char *format, + const Args & ... args) { + fmt::print("{}: {}: ", file, line); + fmt::print(format, args...); + } + \endrst + */ +#define FMT_VARIADIC(ReturnType, func, ...) FMT_VARIADIC_(, char, ReturnType, func, return func, __VA_ARGS__) + +#define FMT_VARIADIC_CONST(ReturnType, func, ...) FMT_VARIADIC_(const, char, ReturnType, func, return func, __VA_ARGS__) + +#define FMT_VARIADIC_W(ReturnType, func, ...) FMT_VARIADIC_(, wchar_t, ReturnType, func, return func, __VA_ARGS__) + +#define FMT_VARIADIC_CONST_W(ReturnType, func, ...) FMT_VARIADIC_(const, wchar_t, ReturnType, func, return func, __VA_ARGS__) + +#define FMT_CAPTURE_ARG_(id, index) ::fmt::arg(#id, id) + +#define FMT_CAPTURE_ARG_W_(id, index) ::fmt::arg(L## #id, id) + +/** + \rst + Convenient macro to capture the arguments' names and values into several + ``fmt::arg(name, value)``. + + **Example**:: + + int x = 1, y = 2; + print("point: ({x}, {y})", FMT_CAPTURE(x, y)); + // same as: + // print("point: ({x}, {y})", arg("x", x), arg("y", y)); + + \endrst + */ +#define FMT_CAPTURE(...) FMT_FOR_EACH(FMT_CAPTURE_ARG_, __VA_ARGS__) + +#define FMT_CAPTURE_W(...) FMT_FOR_EACH(FMT_CAPTURE_ARG_W_, __VA_ARGS__) + +namespace fmt { +FMT_VARIADIC(std::string, format, CStringRef) +FMT_VARIADIC_W(std::wstring, format, WCStringRef) +FMT_VARIADIC(void, print, CStringRef) +FMT_VARIADIC(void, print, std::FILE *, CStringRef) +FMT_VARIADIC(void, print_colored, Color, CStringRef) + +namespace internal { +template <typename Char> inline bool is_name_start(Char c) +{ + return ('a' <= c && c <= 'z') || ('A' <= c && c <= 'Z') || '_' == c; +} + +// Parses an unsigned integer advancing s to the end of the parsed input. +// This function assumes that the first character of s is a digit. +template <typename Char> unsigned parse_nonnegative_int(const Char *&s) +{ + assert('0' <= *s && *s <= '9'); + unsigned value = 0; + // Convert to unsigned to prevent a warning. + unsigned max_int = (std::numeric_limits<int>::max)(); + unsigned big = max_int / 10; + do + { + // Check for overflow. + if (value > big) + { + value = max_int + 1; + break; + } + value = value * 10 + (*s - '0'); + ++s; + } while ('0' <= *s && *s <= '9'); + // Convert to unsigned to prevent a warning. + if (value > max_int) + FMT_THROW(FormatError("number is too big")); + return value; +} + +inline void require_numeric_argument(const Arg &arg, char spec) +{ + if (arg.type > Arg::LAST_NUMERIC_TYPE) + { + std::string message = fmt::format("format specifier '{}' requires numeric argument", spec); + FMT_THROW(fmt::FormatError(message)); + } +} + +template <typename Char> void check_sign(const Char *&s, const Arg &arg) +{ + char sign = static_cast<char>(*s); + require_numeric_argument(arg, sign); + if (arg.type == Arg::UINT || arg.type == Arg::ULONG_LONG) + { + FMT_THROW(FormatError(fmt::format("format specifier '{}' requires signed argument", sign))); + } + ++s; +} +} // namespace internal + +template <typename Char, typename AF> +inline internal::Arg BasicFormatter<Char, AF>::get_arg(BasicStringRef<Char> arg_name, const char *&error) +{ + if (check_no_auto_index(error)) + { + map_.init(args()); + const internal::Arg *arg = map_.find(arg_name); + if (arg) + return *arg; + error = "argument not found"; + } + return internal::Arg(); +} + +template <typename Char, typename AF> inline internal::Arg BasicFormatter<Char, AF>::parse_arg_index(const Char *&s) +{ + const char *error = FMT_NULL; + internal::Arg arg = *s < '0' || *s > '9' ? next_arg(error) : get_arg(internal::parse_nonnegative_int(s), error); + if (error) + { + FMT_THROW(FormatError(*s != '}' && *s != ':' ? "invalid format string" : error)); + } + return arg; +} + +template <typename Char, typename AF> inline internal::Arg BasicFormatter<Char, AF>::parse_arg_name(const Char *&s) +{ + assert(internal::is_name_start(*s)); + const Char *start = s; + Char c; + do + { + c = *++s; + } while (internal::is_name_start(c) || ('0' <= c && c <= '9')); + const char *error = FMT_NULL; + internal::Arg arg = get_arg(BasicStringRef<Char>(start, s - start), error); + if (error) + FMT_THROW(FormatError(error)); + return arg; +} + +template <typename Char, typename ArgFormatter> +const Char *BasicFormatter<Char, ArgFormatter>::format(const Char *&format_str, const internal::Arg &arg) +{ + using internal::Arg; + const Char *s = format_str; + typename ArgFormatter::SpecType spec; + if (*s == ':') + { + if (arg.type == Arg::CUSTOM) + { + arg.custom.format(this, arg.custom.value, &s); + return s; + } + ++s; + // Parse fill and alignment. + if (Char c = *s) + { + const Char *p = s + 1; + spec.align_ = ALIGN_DEFAULT; + do + { + switch (*p) + { + case '<': + spec.align_ = ALIGN_LEFT; + break; + case '>': + spec.align_ = ALIGN_RIGHT; + break; + case '=': + spec.align_ = ALIGN_NUMERIC; + break; + case '^': + spec.align_ = ALIGN_CENTER; + break; + } + if (spec.align_ != ALIGN_DEFAULT) + { + if (p != s) + { + if (c == '}') + break; + if (c == '{') + FMT_THROW(FormatError("invalid fill character '{'")); + s += 2; + spec.fill_ = c; + } + else + ++s; + if (spec.align_ == ALIGN_NUMERIC) + require_numeric_argument(arg, '='); + break; + } + } while (--p >= s); + } + + // Parse sign. + switch (*s) + { + case '+': + check_sign(s, arg); + spec.flags_ |= SIGN_FLAG | PLUS_FLAG; + break; + case '-': + check_sign(s, arg); + spec.flags_ |= MINUS_FLAG; + break; + case ' ': + check_sign(s, arg); + spec.flags_ |= SIGN_FLAG; + break; + } + + if (*s == '#') + { + require_numeric_argument(arg, '#'); + spec.flags_ |= HASH_FLAG; + ++s; + } + + // Parse zero flag. + if (*s == '0') + { + require_numeric_argument(arg, '0'); + spec.align_ = ALIGN_NUMERIC; + spec.fill_ = '0'; + ++s; + } + + // Parse width. + if ('0' <= *s && *s <= '9') + { + spec.width_ = internal::parse_nonnegative_int(s); + } + else if (*s == '{') + { + ++s; + Arg width_arg = internal::is_name_start(*s) ? parse_arg_name(s) : parse_arg_index(s); + if (*s++ != '}') + FMT_THROW(FormatError("invalid format string")); + ULongLong value = 0; + switch (width_arg.type) + { + case Arg::INT: + if (width_arg.int_value < 0) + FMT_THROW(FormatError("negative width")); + value = width_arg.int_value; + break; + case Arg::UINT: + value = width_arg.uint_value; + break; + case Arg::LONG_LONG: + if (width_arg.long_long_value < 0) + FMT_THROW(FormatError("negative width")); + value = width_arg.long_long_value; + break; + case Arg::ULONG_LONG: + value = width_arg.ulong_long_value; + break; + default: + FMT_THROW(FormatError("width is not integer")); + } + unsigned max_int = (std::numeric_limits<int>::max)(); + if (value > max_int) + FMT_THROW(FormatError("number is too big")); + spec.width_ = static_cast<int>(value); + } + + // Parse precision. + if (*s == '.') + { + ++s; + spec.precision_ = 0; + if ('0' <= *s && *s <= '9') + { + spec.precision_ = internal::parse_nonnegative_int(s); + } + else if (*s == '{') + { + ++s; + Arg precision_arg = internal::is_name_start(*s) ? parse_arg_name(s) : parse_arg_index(s); + if (*s++ != '}') + FMT_THROW(FormatError("invalid format string")); + ULongLong value = 0; + switch (precision_arg.type) + { + case Arg::INT: + if (precision_arg.int_value < 0) + FMT_THROW(FormatError("negative precision")); + value = precision_arg.int_value; + break; + case Arg::UINT: + value = precision_arg.uint_value; + break; + case Arg::LONG_LONG: + if (precision_arg.long_long_value < 0) + FMT_THROW(FormatError("negative precision")); + value = precision_arg.long_long_value; + break; + case Arg::ULONG_LONG: + value = precision_arg.ulong_long_value; + break; + default: + FMT_THROW(FormatError("precision is not integer")); + } + unsigned max_int = (std::numeric_limits<int>::max)(); + if (value > max_int) + FMT_THROW(FormatError("number is too big")); + spec.precision_ = static_cast<int>(value); + } + else + { + FMT_THROW(FormatError("missing precision specifier")); + } + if (arg.type <= Arg::LAST_INTEGER_TYPE || arg.type == Arg::POINTER) + { + FMT_THROW(FormatError( + fmt::format("precision not allowed in {} format specifier", arg.type == Arg::POINTER ? "pointer" : "integer"))); + } + } + + // Parse type. + if (*s != '}' && *s) + spec.type_ = static_cast<char>(*s++); + } + + if (*s++ != '}') + FMT_THROW(FormatError("missing '}' in format string")); + + // Format argument. + ArgFormatter(*this, spec, s - 1).visit(arg); + return s; +} + +template <typename Char, typename AF> void BasicFormatter<Char, AF>::format(BasicCStringRef<Char> format_str) +{ + const Char *s = format_str.c_str(); + const Char *start = s; + while (*s) + { + Char c = *s++; + if (c != '{' && c != '}') + continue; + if (*s == c) + { + write(writer_, start, s); + start = ++s; + continue; + } + if (c == '}') + FMT_THROW(FormatError("unmatched '}' in format string")); + write(writer_, start, s - 1); + internal::Arg arg = internal::is_name_start(*s) ? parse_arg_name(s) : parse_arg_index(s); + start = s = format(s, arg); + } + write(writer_, start, s); +} + +template <typename Char, typename It> struct ArgJoin +{ + It first; + It last; + BasicCStringRef<Char> sep; + + ArgJoin(It first, It last, const BasicCStringRef<Char> &sep) + : first(first) + , last(last) + , sep(sep) + { + } +}; + +template <typename It> ArgJoin<char, It> join(It first, It last, const BasicCStringRef<char> &sep) +{ + return ArgJoin<char, It>(first, last, sep); +} + +template <typename It> ArgJoin<wchar_t, It> join(It first, It last, const BasicCStringRef<wchar_t> &sep) +{ + return ArgJoin<wchar_t, It>(first, last, sep); +} + +#if FMT_HAS_GXX_CXX11 +template <typename Range> auto join(const Range &range, const BasicCStringRef<char> &sep) -> ArgJoin<char, decltype(std::begin(range))> +{ + return join(std::begin(range), std::end(range), sep); +} + +template <typename Range> +auto join(const Range &range, const BasicCStringRef<wchar_t> &sep) -> ArgJoin<wchar_t, decltype(std::begin(range))> +{ + return join(std::begin(range), std::end(range), sep); +} +#endif + +template <typename ArgFormatter, typename Char, typename It> +void format_arg(fmt::BasicFormatter<Char, ArgFormatter> &f, const Char *&format_str, const ArgJoin<Char, It> &e) +{ + const Char *end = format_str; + if (*end == ':') + ++end; + while (*end && *end != '}') + ++end; + if (*end != '}') + FMT_THROW(FormatError("missing '}' in format string")); + + It it = e.first; + if (it != e.last) + { + const Char *save = format_str; + f.format(format_str, internal::MakeArg<fmt::BasicFormatter<Char, ArgFormatter>>(*it++)); + while (it != e.last) + { + f.writer().write(e.sep); + format_str = save; + f.format(format_str, internal::MakeArg<fmt::BasicFormatter<Char, ArgFormatter>>(*it++)); + } + } + format_str = end + 1; +} +} // namespace fmt + +#if FMT_USE_USER_DEFINED_LITERALS +namespace fmt { +namespace internal { + +template <typename Char> struct UdlFormat +{ + const Char *str; + + template <typename... Args> auto operator()(Args &&... args) const -> decltype(format(str, std::forward<Args>(args)...)) + { + return format(str, std::forward<Args>(args)...); + } +}; + +template <typename Char> struct UdlArg +{ + const Char *str; + + template <typename T> NamedArgWithType<Char, T> operator=(T &&value) const + { + return {str, std::forward<T>(value)}; + } +}; + +} // namespace internal + +inline namespace literals { + +/** + \rst + C++11 literal equivalent of :func:`fmt::format`. + + **Example**:: + + using namespace fmt::literals; + std::string message = "The answer is {}"_format(42); + \endrst + */ +inline internal::UdlFormat<char> operator"" _format(const char *s, std::size_t) +{ + return {s}; +} +inline internal::UdlFormat<wchar_t> operator"" _format(const wchar_t *s, std::size_t) +{ + return {s}; +} + +/** + \rst + C++11 literal equivalent of :func:`fmt::arg`. + + **Example**:: + + using namespace fmt::literals; + print("Elapsed time: {s:.2f} seconds", "s"_a=1.23); + \endrst + */ +inline internal::UdlArg<char> operator"" _a(const char *s, std::size_t) +{ + return {s}; +} +inline internal::UdlArg<wchar_t> operator"" _a(const wchar_t *s, std::size_t) +{ + return {s}; +} + +} // namespace literals +} // namespace fmt +#endif // FMT_USE_USER_DEFINED_LITERALS + +// Restore warnings. +#if FMT_GCC_VERSION >= 406 +#pragma GCC diagnostic pop +#endif + +#if defined(__clang__) && !defined(FMT_ICC_VERSION) +#pragma clang diagnostic pop +#endif + +#ifdef FMT_HEADER_ONLY +#define FMT_FUNC inline +#include "format.cc" +#else +#define FMT_FUNC +#endif + +#endif // FMT_FORMAT_H_ diff --git a/src/spdlog/fmt/bundled/ostream.cc b/src/spdlog/fmt/bundled/ostream.cc new file mode 100644 index 0000000000000000000000000000000000000000..2d443f73054ed1439e238cb1aa6a0f46b964f131 --- /dev/null +++ b/src/spdlog/fmt/bundled/ostream.cc @@ -0,0 +1,35 @@ +/* + Formatting library for C++ - std::ostream support + + Copyright (c) 2012 - 2016, Victor Zverovich + All rights reserved. + + For the license information refer to format.h. + */ + +#include "ostream.h" + +namespace fmt { + +namespace internal { +FMT_FUNC void write(std::ostream &os, Writer &w) { + const char *data = w.data(); + typedef internal::MakeUnsigned<std::streamsize>::Type UnsignedStreamSize; + UnsignedStreamSize size = w.size(); + UnsignedStreamSize max_size = + internal::to_unsigned((std::numeric_limits<std::streamsize>::max)()); + do { + UnsignedStreamSize n = size <= max_size ? size : max_size; + os.write(data, static_cast<std::streamsize>(n)); + data += n; + size -= n; + } while (size != 0); +} +} + +FMT_FUNC void print(std::ostream &os, CStringRef format_str, ArgList args) { + MemoryWriter w; + w.write(format_str, args); + internal::write(os, w); +} +} // namespace fmt diff --git a/src/spdlog/fmt/bundled/ostream.h b/src/spdlog/fmt/bundled/ostream.h new file mode 100644 index 0000000000000000000000000000000000000000..1a21f9849b9419f8f37f2245fb6a3bf19cc2ec6b --- /dev/null +++ b/src/spdlog/fmt/bundled/ostream.h @@ -0,0 +1,114 @@ +/* + Formatting library for C++ - std::ostream support + + Copyright (c) 2012 - 2016, Victor Zverovich + All rights reserved. + + For the license information refer to format.h. + */ + +#ifndef FMT_OSTREAM_H_ +#define FMT_OSTREAM_H_ + +#include "format.h" +#include <ostream> + +namespace fmt { + +namespace internal { + +template <class Char> class FormatBuf : public std::basic_streambuf<Char> +{ +private: + typedef typename std::basic_streambuf<Char>::int_type int_type; + typedef typename std::basic_streambuf<Char>::traits_type traits_type; + + Buffer<Char> &buffer_; + +public: + FormatBuf(Buffer<Char> &buffer) + : buffer_(buffer) + { + } + +protected: + // The put-area is actually always empty. This makes the implementation + // simpler and has the advantage that the streambuf and the buffer are always + // in sync and sputc never writes into uninitialized memory. The obvious + // disadvantage is that each call to sputc always results in a (virtual) call + // to overflow. There is no disadvantage here for sputn since this always + // results in a call to xsputn. + + int_type overflow(int_type ch = traits_type::eof()) FMT_OVERRIDE + { + if (!traits_type::eq_int_type(ch, traits_type::eof())) + buffer_.push_back(static_cast<Char>(ch)); + return ch; + } + + std::streamsize xsputn(const Char *s, std::streamsize count) FMT_OVERRIDE + { + buffer_.append(s, s + count); + return count; + } +}; + +Yes &convert(std::ostream &); + +struct DummyStream : std::ostream +{ + DummyStream(); // Suppress a bogus warning in MSVC. + + // Hide all operator<< overloads from std::ostream. + template <typename T> typename EnableIf<sizeof(T) == 0>::type operator<<(const T &); +}; + +No &operator<<(std::ostream &, int); + +template <typename T> struct ConvertToIntImpl<T, true> +{ + // Convert to int only if T doesn't have an overloaded operator<<. + enum + { + value = sizeof(convert(get<DummyStream>() << get<T>())) == sizeof(No) + }; +}; + +// Write the content of w to os. +FMT_API void write(std::ostream &os, Writer &w); +} // namespace internal + +// Formats a value. +template <typename Char, typename ArgFormatter_, typename T> +void format_arg(BasicFormatter<Char, ArgFormatter_> &f, const Char *&format_str, const T &value) +{ + internal::MemoryBuffer<Char, internal::INLINE_BUFFER_SIZE> buffer; + + internal::FormatBuf<Char> format_buf(buffer); + std::basic_ostream<Char> output(&format_buf); + output.exceptions(std::ios_base::failbit | std::ios_base::badbit); + output << value; + + BasicStringRef<Char> str(&buffer[0], buffer.size()); + typedef internal::MakeArg<BasicFormatter<Char>> MakeArg; + format_str = f.format(format_str, MakeArg(str)); +} + +/** + \rst + Prints formatted data to the stream *os*. + + **Example**:: + + print(cerr, "Don't {}!", "panic"); + \endrst + */ +FMT_API void print(std::ostream &os, CStringRef format_str, ArgList args); +FMT_VARIADIC(void, print, std::ostream &, CStringRef) +} // namespace fmt + +#ifdef FMT_HEADER_ONLY +#include "ostream.cc" +#endif + +#endif // FMT_OSTREAM_H_ diff --git a/src/spdlog/fmt/bundled/posix.cc b/src/spdlog/fmt/bundled/posix.cc new file mode 100644 index 0000000000000000000000000000000000000000..356668c135ab0704d2a3538117732957342ba68d --- /dev/null +++ b/src/spdlog/fmt/bundled/posix.cc @@ -0,0 +1,241 @@ +/* + A C++ interface to POSIX functions. + + Copyright (c) 2012 - 2016, Victor Zverovich + All rights reserved. + + For the license information refer to format.h. + */ + +// Disable bogus MSVC warnings. +#ifndef _CRT_SECURE_NO_WARNINGS +# define _CRT_SECURE_NO_WARNINGS +#endif + +#include "posix.h" + +#include <limits.h> +#include <sys/types.h> +#include <sys/stat.h> + +#ifndef _WIN32 +# include <unistd.h> +#else +# ifndef WIN32_LEAN_AND_MEAN +# define WIN32_LEAN_AND_MEAN +# endif +# include <windows.h> +# include <io.h> + +# define O_CREAT _O_CREAT +# define O_TRUNC _O_TRUNC + +# ifndef S_IRUSR +# define S_IRUSR _S_IREAD +# endif + +# ifndef S_IWUSR +# define S_IWUSR _S_IWRITE +# endif + +# ifdef __MINGW32__ +# define _SH_DENYNO 0x40 +# endif + +#endif // _WIN32 + +#ifdef fileno +# undef fileno +#endif + +namespace { +#ifdef _WIN32 +// Return type of read and write functions. +typedef int RWResult; + +// On Windows the count argument to read and write is unsigned, so convert +// it from size_t preventing integer overflow. +inline unsigned convert_rwcount(std::size_t count) { + return count <= UINT_MAX ? static_cast<unsigned>(count) : UINT_MAX; +} +#else +// Return type of read and write functions. +typedef ssize_t RWResult; + +inline std::size_t convert_rwcount(std::size_t count) { return count; } +#endif +} + +fmt::BufferedFile::~BufferedFile() FMT_NOEXCEPT { + if (file_ && FMT_SYSTEM(fclose(file_)) != 0) + fmt::report_system_error(errno, "cannot close file"); +} + +fmt::BufferedFile::BufferedFile( + fmt::CStringRef filename, fmt::CStringRef mode) { + FMT_RETRY_VAL(file_, FMT_SYSTEM(fopen(filename.c_str(), mode.c_str())), 0); + if (!file_) + FMT_THROW(SystemError(errno, "cannot open file {}", filename)); +} + +void fmt::BufferedFile::close() { + if (!file_) + return; + int result = FMT_SYSTEM(fclose(file_)); + file_ = FMT_NULL; + if (result != 0) + FMT_THROW(SystemError(errno, "cannot close file")); +} + +// A macro used to prevent expansion of fileno on broken versions of MinGW. +#define FMT_ARGS + +int fmt::BufferedFile::fileno() const { + int fd = FMT_POSIX_CALL(fileno FMT_ARGS(file_)); + if (fd == -1) + FMT_THROW(SystemError(errno, "cannot get file descriptor")); + return fd; +} + +fmt::File::File(fmt::CStringRef path, int oflag) { + int mode = S_IRUSR | S_IWUSR; +#if defined(_WIN32) && !defined(__MINGW32__) + fd_ = -1; + FMT_POSIX_CALL(sopen_s(&fd_, path.c_str(), oflag, _SH_DENYNO, mode)); +#else + FMT_RETRY(fd_, FMT_POSIX_CALL(open(path.c_str(), oflag, mode))); +#endif + if (fd_ == -1) + FMT_THROW(SystemError(errno, "cannot open file {}", path)); +} + +fmt::File::~File() FMT_NOEXCEPT { + // Don't retry close in case of EINTR! + // See http://linux.derkeiler.com/Mailing-Lists/Kernel/2005-09/3000.html + if (fd_ != -1 && FMT_POSIX_CALL(close(fd_)) != 0) + fmt::report_system_error(errno, "cannot close file"); +} + +void fmt::File::close() { + if (fd_ == -1) + return; + // Don't retry close in case of EINTR! + // See http://linux.derkeiler.com/Mailing-Lists/Kernel/2005-09/3000.html + int result = FMT_POSIX_CALL(close(fd_)); + fd_ = -1; + if (result != 0) + FMT_THROW(SystemError(errno, "cannot close file")); +} + +fmt::LongLong fmt::File::size() const { +#ifdef _WIN32 + // Use GetFileSize instead of GetFileSizeEx for the case when _WIN32_WINNT + // is less than 0x0500 as is the case with some default MinGW builds. + // Both functions support large file sizes. + DWORD size_upper = 0; + HANDLE handle = reinterpret_cast<HANDLE>(_get_osfhandle(fd_)); + DWORD size_lower = FMT_SYSTEM(GetFileSize(handle, &size_upper)); + if (size_lower == INVALID_FILE_SIZE) { + DWORD error = GetLastError(); + if (error != NO_ERROR) + FMT_THROW(WindowsError(GetLastError(), "cannot get file size")); + } + fmt::ULongLong long_size = size_upper; + return (long_size << sizeof(DWORD) * CHAR_BIT) | size_lower; +#else + typedef struct stat Stat; + Stat file_stat = Stat(); + if (FMT_POSIX_CALL(fstat(fd_, &file_stat)) == -1) + FMT_THROW(SystemError(errno, "cannot get file attributes")); + FMT_STATIC_ASSERT(sizeof(fmt::LongLong) >= sizeof(file_stat.st_size), + "return type of File::size is not large enough"); + return file_stat.st_size; +#endif +} + +std::size_t fmt::File::read(void *buffer, std::size_t count) { + RWResult result = 0; + FMT_RETRY(result, FMT_POSIX_CALL(read(fd_, buffer, convert_rwcount(count)))); + if (result < 0) + FMT_THROW(SystemError(errno, "cannot read from file")); + return internal::to_unsigned(result); +} + +std::size_t fmt::File::write(const void *buffer, std::size_t count) { + RWResult result = 0; + FMT_RETRY(result, FMT_POSIX_CALL(write(fd_, buffer, convert_rwcount(count)))); + if (result < 0) + FMT_THROW(SystemError(errno, "cannot write to file")); + return internal::to_unsigned(result); +} + +fmt::File fmt::File::dup(int fd) { + // Don't retry as dup doesn't return EINTR. + // http://pubs.opengroup.org/onlinepubs/009695399/functions/dup.html + int new_fd = FMT_POSIX_CALL(dup(fd)); + if (new_fd == -1) + FMT_THROW(SystemError(errno, "cannot duplicate file descriptor {}", fd)); + return File(new_fd); +} + +void fmt::File::dup2(int fd) { + int result = 0; + FMT_RETRY(result, FMT_POSIX_CALL(dup2(fd_, fd))); + if (result == -1) { + FMT_THROW(SystemError(errno, + "cannot duplicate file descriptor {} to {}", fd_, fd)); + } +} + +void fmt::File::dup2(int fd, ErrorCode &ec) FMT_NOEXCEPT { + int result = 0; + FMT_RETRY(result, FMT_POSIX_CALL(dup2(fd_, fd))); + if (result == -1) + ec = ErrorCode(errno); +} + +void fmt::File::pipe(File &read_end, File &write_end) { + // Close the descriptors first to make sure that assignments don't throw + // and there are no leaks. + read_end.close(); + write_end.close(); + int fds[2] = {}; +#ifdef _WIN32 + // Make the default pipe capacity same as on Linux 2.6.11+. + enum { DEFAULT_CAPACITY = 65536 }; + int result = FMT_POSIX_CALL(pipe(fds, DEFAULT_CAPACITY, _O_BINARY)); +#else + // Don't retry as the pipe function doesn't return EINTR. + // http://pubs.opengroup.org/onlinepubs/009696799/functions/pipe.html + int result = FMT_POSIX_CALL(pipe(fds)); +#endif + if (result != 0) + FMT_THROW(SystemError(errno, "cannot create pipe")); + // The following assignments don't throw because read_fd and write_fd + // are closed. + read_end = File(fds[0]); + write_end = File(fds[1]); +} + +fmt::BufferedFile fmt::File::fdopen(const char *mode) { + // Don't retry as fdopen doesn't return EINTR. + FILE *f = FMT_POSIX_CALL(fdopen(fd_, mode)); + if (!f) + FMT_THROW(SystemError(errno, "cannot associate stream with file descriptor")); + BufferedFile file(f); + fd_ = -1; + return file; +} + +long fmt::getpagesize() { +#ifdef _WIN32 + SYSTEM_INFO si; + GetSystemInfo(&si); + return si.dwPageSize; +#else + long size = FMT_POSIX_CALL(sysconf(_SC_PAGESIZE)); + if (size < 0) + FMT_THROW(SystemError(errno, "cannot get memory page size")); + return size; +#endif +} diff --git a/src/spdlog/fmt/bundled/posix.h b/src/spdlog/fmt/bundled/posix.h new file mode 100644 index 0000000000000000000000000000000000000000..20af016c487bac8ad05241921b80218ff707ba58 --- /dev/null +++ b/src/spdlog/fmt/bundled/posix.h @@ -0,0 +1,423 @@ +/* + A C++ interface to POSIX functions. + + Copyright (c) 2012 - 2016, Victor Zverovich + All rights reserved. + + For the license information refer to format.h. + */ + +#ifndef FMT_POSIX_H_ +#define FMT_POSIX_H_ + +#if defined(__MINGW32__) || defined(__CYGWIN__) +// Workaround MinGW bug https://sourceforge.net/p/mingw/bugs/2024/. +#undef __STRICT_ANSI__ +#endif + +#include <errno.h> +#include <fcntl.h> // for O_RDONLY +#include <locale.h> // for locale_t +#include <stdio.h> +#include <stdlib.h> // for strtod_l + +#include <cstddef> + +#if defined __APPLE__ || defined(__FreeBSD__) +#include <xlocale.h> // for LC_NUMERIC_MASK on OS X +#endif + +#include "format.h" + +#ifndef FMT_POSIX +#if defined(_WIN32) && !defined(__MINGW32__) +// Fix warnings about deprecated symbols. +#define FMT_POSIX(call) _##call +#else +#define FMT_POSIX(call) call +#endif +#endif + +// Calls to system functions are wrapped in FMT_SYSTEM for testability. +#ifdef FMT_SYSTEM +#define FMT_POSIX_CALL(call) FMT_SYSTEM(call) +#else +#define FMT_SYSTEM(call) call +#ifdef _WIN32 +// Fix warnings about deprecated symbols. +#define FMT_POSIX_CALL(call) ::_##call +#else +#define FMT_POSIX_CALL(call) ::call +#endif +#endif + +// Retries the expression while it evaluates to error_result and errno +// equals to EINTR. +#ifndef _WIN32 +#define FMT_RETRY_VAL(result, expression, error_result) \ + do \ + { \ + result = (expression); \ + } while (result == error_result && errno == EINTR) +#else +#define FMT_RETRY_VAL(result, expression, error_result) result = (expression) +#endif + +#define FMT_RETRY(result, expression) FMT_RETRY_VAL(result, expression, -1) + +namespace fmt { + +// An error code. +class ErrorCode +{ +private: + int value_; + +public: + explicit ErrorCode(int value = 0) FMT_NOEXCEPT : value_(value) {} + + int get() const FMT_NOEXCEPT + { + return value_; + } +}; + +// A buffered file. +class BufferedFile +{ +private: + FILE *file_; + + friend class File; + + explicit BufferedFile(FILE *f) + : file_(f) + { + } + +public: + // Constructs a BufferedFile object which doesn't represent any file. + BufferedFile() FMT_NOEXCEPT : file_(FMT_NULL) {} + + // Destroys the object closing the file it represents if any. + FMT_API ~BufferedFile() FMT_NOEXCEPT; + +#if !FMT_USE_RVALUE_REFERENCES + // Emulate a move constructor and a move assignment operator if rvalue + // references are not supported. + +private: + // A proxy object to emulate a move constructor. + // It is private to make it impossible call operator Proxy directly. + struct Proxy + { + FILE *file; + }; + +public: + // A "move constructor" for moving from a temporary. + BufferedFile(Proxy p) FMT_NOEXCEPT : file_(p.file) {} + + // A "move constructor" for moving from an lvalue. + BufferedFile(BufferedFile &f) FMT_NOEXCEPT : file_(f.file_) + { + f.file_ = FMT_NULL; + } + + // A "move assignment operator" for moving from a temporary. + BufferedFile &operator=(Proxy p) + { + close(); + file_ = p.file; + return *this; + } + + // A "move assignment operator" for moving from an lvalue. + BufferedFile &operator=(BufferedFile &other) + { + close(); + file_ = other.file_; + other.file_ = FMT_NULL; + return *this; + } + + // Returns a proxy object for moving from a temporary: + // BufferedFile file = BufferedFile(...); + operator Proxy() FMT_NOEXCEPT + { + Proxy p = {file_}; + file_ = FMT_NULL; + return p; + } + +#else +private: + FMT_DISALLOW_COPY_AND_ASSIGN(BufferedFile); + +public: + BufferedFile(BufferedFile &&other) FMT_NOEXCEPT : file_(other.file_) + { + other.file_ = FMT_NULL; + } + + BufferedFile &operator=(BufferedFile &&other) + { + close(); + file_ = other.file_; + other.file_ = FMT_NULL; + return *this; + } +#endif + + // Opens a file. + FMT_API BufferedFile(CStringRef filename, CStringRef mode); + + // Closes the file. + FMT_API void close(); + + // Returns the pointer to a FILE object representing this file. + FILE *get() const FMT_NOEXCEPT + { + return file_; + } + + // We place parentheses around fileno to workaround a bug in some versions + // of MinGW that define fileno as a macro. + FMT_API int(fileno)() const; + + void print(CStringRef format_str, const ArgList &args) + { + fmt::print(file_, format_str, args); + } + FMT_VARIADIC(void, print, CStringRef) +}; + +// A file. Closed file is represented by a File object with descriptor -1. +// Methods that are not declared with FMT_NOEXCEPT may throw +// fmt::SystemError in case of failure. Note that some errors such as +// closing the file multiple times will cause a crash on Windows rather +// than an exception. You can get standard behavior by overriding the +// invalid parameter handler with _set_invalid_parameter_handler. +class File +{ +private: + int fd_; // File descriptor. + + // Constructs a File object with a given descriptor. + explicit File(int fd) + : fd_(fd) + { + } + +public: + // Possible values for the oflag argument to the constructor. + enum + { + RDONLY = FMT_POSIX(O_RDONLY), // Open for reading only. + WRONLY = FMT_POSIX(O_WRONLY), // Open for writing only. + RDWR = FMT_POSIX(O_RDWR) // Open for reading and writing. + }; + + // Constructs a File object which doesn't represent any file. + File() FMT_NOEXCEPT : fd_(-1) {} + + // Opens a file and constructs a File object representing this file. + FMT_API File(CStringRef path, int oflag); + +#if !FMT_USE_RVALUE_REFERENCES + // Emulate a move constructor and a move assignment operator if rvalue + // references are not supported. + +private: + // A proxy object to emulate a move constructor. + // It is private to make it impossible call operator Proxy directly. + struct Proxy + { + int fd; + }; + +public: + // A "move constructor" for moving from a temporary. + File(Proxy p) FMT_NOEXCEPT : fd_(p.fd) {} + + // A "move constructor" for moving from an lvalue. + File(File &other) FMT_NOEXCEPT : fd_(other.fd_) + { + other.fd_ = -1; + } + + // A "move assignment operator" for moving from a temporary. + File &operator=(Proxy p) + { + close(); + fd_ = p.fd; + return *this; + } + + // A "move assignment operator" for moving from an lvalue. + File &operator=(File &other) + { + close(); + fd_ = other.fd_; + other.fd_ = -1; + return *this; + } + + // Returns a proxy object for moving from a temporary: + // File file = File(...); + operator Proxy() FMT_NOEXCEPT + { + Proxy p = {fd_}; + fd_ = -1; + return p; + } + +#else +private: + FMT_DISALLOW_COPY_AND_ASSIGN(File); + +public: + File(File &&other) FMT_NOEXCEPT : fd_(other.fd_) + { + other.fd_ = -1; + } + + File &operator=(File &&other) + { + close(); + fd_ = other.fd_; + other.fd_ = -1; + return *this; + } +#endif + + // Destroys the object closing the file it represents if any. + FMT_API ~File() FMT_NOEXCEPT; + + // Returns the file descriptor. + int descriptor() const FMT_NOEXCEPT + { + return fd_; + } + + // Closes the file. + FMT_API void close(); + + // Returns the file size. The size has signed type for consistency with + // stat::st_size. + FMT_API LongLong size() const; + + // Attempts to read count bytes from the file into the specified buffer. + FMT_API std::size_t read(void *buffer, std::size_t count); + + // Attempts to write count bytes from the specified buffer to the file. + FMT_API std::size_t write(const void *buffer, std::size_t count); + + // Duplicates a file descriptor with the dup function and returns + // the duplicate as a file object. + FMT_API static File dup(int fd); + + // Makes fd be the copy of this file descriptor, closing fd first if + // necessary. + FMT_API void dup2(int fd); + + // Makes fd be the copy of this file descriptor, closing fd first if + // necessary. + FMT_API void dup2(int fd, ErrorCode &ec) FMT_NOEXCEPT; + + // Creates a pipe setting up read_end and write_end file objects for reading + // and writing respectively. + FMT_API static void pipe(File &read_end, File &write_end); + + // Creates a BufferedFile object associated with this file and detaches + // this File object from the file. + FMT_API BufferedFile fdopen(const char *mode); +}; + +// Returns the memory page size. +long getpagesize(); + +#if (defined(LC_NUMERIC_MASK) || defined(_MSC_VER)) && !defined(__ANDROID__) && !defined(__CYGWIN__) +#define FMT_LOCALE +#endif + +#ifdef FMT_LOCALE +// A "C" numeric locale. +class Locale +{ +private: +#ifdef _MSC_VER + typedef _locale_t locale_t; + + enum + { + LC_NUMERIC_MASK = LC_NUMERIC + }; + + static locale_t newlocale(int category_mask, const char *locale, locale_t) + { + return _create_locale(category_mask, locale); + } + + static void freelocale(locale_t locale) + { + _free_locale(locale); + } + + static double strtod_l(const char *nptr, char **endptr, _locale_t locale) + { + return _strtod_l(nptr, endptr, locale); + } +#endif + + locale_t locale_; + + FMT_DISALLOW_COPY_AND_ASSIGN(Locale); + +public: + typedef locale_t Type; + + Locale() + : locale_(newlocale(LC_NUMERIC_MASK, "C", FMT_NULL)) + { + if (!locale_) + FMT_THROW(fmt::SystemError(errno, "cannot create locale")); + } + ~Locale() + { + freelocale(locale_); + } + + Type get() const + { + return locale_; + } + + // Converts string to floating-point number and advances str past the end + // of the parsed input. + double strtod(const char *&str) const + { + char *end = FMT_NULL; + double result = strtod_l(str, &end, locale_); + str = end; + return result; + } +}; +#endif // FMT_LOCALE +} // namespace fmt + +#if !FMT_USE_RVALUE_REFERENCES +namespace std { +// For compatibility with C++98. +inline fmt::BufferedFile &move(fmt::BufferedFile &f) +{ + return f; +} +inline fmt::File &move(fmt::File &f) +{ + return f; +} +} // namespace std +#endif + +#endif // FMT_POSIX_H_ diff --git a/src/spdlog/fmt/bundled/printf.cc b/src/spdlog/fmt/bundled/printf.cc new file mode 100644 index 0000000000000000000000000000000000000000..95d7a36ab67600f5f486111e5bf47796cbe04d91 --- /dev/null +++ b/src/spdlog/fmt/bundled/printf.cc @@ -0,0 +1,32 @@ +/* + Formatting library for C++ + + Copyright (c) 2012 - 2016, Victor Zverovich + All rights reserved. + + For the license information refer to format.h. + */ + +#include "format.h" +#include "printf.h" + +namespace fmt { + +template <typename Char> +void printf(BasicWriter<Char> &w, BasicCStringRef<Char> format, ArgList args); + +FMT_FUNC int fprintf(std::FILE *f, CStringRef format, ArgList args) { + MemoryWriter w; + printf(w, format, args); + std::size_t size = w.size(); + return std::fwrite(w.data(), 1, size, f) < size ? -1 : static_cast<int>(size); +} + +#ifndef FMT_HEADER_ONLY + +template void PrintfFormatter<char>::format(CStringRef format); +template void PrintfFormatter<wchar_t>::format(WCStringRef format); + +#endif // FMT_HEADER_ONLY + +} // namespace fmt diff --git a/src/spdlog/fmt/bundled/printf.h b/src/spdlog/fmt/bundled/printf.h new file mode 100644 index 0000000000000000000000000000000000000000..4f540048e7c6291d99ea11b4328c714b1bddcbdb --- /dev/null +++ b/src/spdlog/fmt/bundled/printf.h @@ -0,0 +1,702 @@ +/* + Formatting library for C++ + + Copyright (c) 2012 - 2016, Victor Zverovich + All rights reserved. + + For the license information refer to format.h. + */ + +#ifndef FMT_PRINTF_H_ +#define FMT_PRINTF_H_ + +#include <algorithm> // std::fill_n +#include <limits> // std::numeric_limits + +#include "ostream.h" + +namespace fmt { +namespace internal { + +// Checks if a value fits in int - used to avoid warnings about comparing +// signed and unsigned integers. +template <bool IsSigned> struct IntChecker +{ + template <typename T> static bool fits_in_int(T value) + { + unsigned max = std::numeric_limits<int>::max(); + return value <= max; + } + static bool fits_in_int(bool) + { + return true; + } +}; + +template <> struct IntChecker<true> +{ + template <typename T> static bool fits_in_int(T value) + { + return value >= std::numeric_limits<int>::min() && value <= std::numeric_limits<int>::max(); + } + static bool fits_in_int(int) + { + return true; + } +}; + +class PrecisionHandler : public ArgVisitor<PrecisionHandler, int> +{ +public: + void report_unhandled_arg() + { + FMT_THROW(FormatError("precision is not integer")); + } + + template <typename T> int visit_any_int(T value) + { + if (!IntChecker<std::numeric_limits<T>::is_signed>::fits_in_int(value)) + FMT_THROW(FormatError("number is too big")); + return static_cast<int>(value); + } +}; + +// IsZeroInt::visit(arg) returns true iff arg is a zero integer. +class IsZeroInt : public ArgVisitor<IsZeroInt, bool> +{ +public: + template <typename T> bool visit_any_int(T value) + { + return value == 0; + } +}; + +// returns the default type for format specific "%s" +class DefaultType : public ArgVisitor<DefaultType, char> +{ +public: + char visit_char(int) + { + return 'c'; + } + + char visit_bool(bool) + { + return 's'; + } + + char visit_pointer(const void *) + { + return 'p'; + } + + template <typename T> char visit_any_int(T) + { + return 'd'; + } + + template <typename T> char visit_any_double(T) + { + return 'g'; + } + + char visit_unhandled_arg() + { + return 's'; + } +}; + +template <typename T, typename U> struct is_same +{ + enum + { + value = 0 + }; +}; + +template <typename T> struct is_same<T, T> +{ + enum + { + value = 1 + }; +}; + +// An argument visitor that converts an integer argument to T for printf, +// if T is an integral type. If T is void, the argument is converted to +// corresponding signed or unsigned type depending on the type specifier: +// 'd' and 'i' - signed, other - unsigned) +template <typename T = void> class ArgConverter : public ArgVisitor<ArgConverter<T>, void> +{ +private: + internal::Arg &arg_; + wchar_t type_; + + FMT_DISALLOW_COPY_AND_ASSIGN(ArgConverter); + +public: + ArgConverter(internal::Arg &arg, wchar_t type) + : arg_(arg) + , type_(type) + { + } + + void visit_bool(bool value) + { + if (type_ != 's') + visit_any_int(value); + } + + void visit_char(int value) + { + if (type_ != 's') + visit_any_int(value); + } + + template <typename U> void visit_any_int(U value) + { + bool is_signed = type_ == 'd' || type_ == 'i'; + if (type_ == 's') + { + is_signed = std::numeric_limits<U>::is_signed; + } + + using internal::Arg; + typedef typename internal::Conditional<is_same<T, void>::value, U, T>::type TargetType; + if (const_check(sizeof(TargetType) <= sizeof(int))) + { + // Extra casts are used to silence warnings. + if (is_signed) + { + arg_.type = Arg::INT; + arg_.int_value = static_cast<int>(static_cast<TargetType>(value)); + } + else + { + arg_.type = Arg::UINT; + typedef typename internal::MakeUnsigned<TargetType>::Type Unsigned; + arg_.uint_value = static_cast<unsigned>(static_cast<Unsigned>(value)); + } + } + else + { + if (is_signed) + { + arg_.type = Arg::LONG_LONG; + // glibc's printf doesn't sign extend arguments of smaller types: + // std::printf("%lld", -42); // prints "4294967254" + // but we don't have to do the same because it's a UB. + arg_.long_long_value = static_cast<LongLong>(value); + } + else + { + arg_.type = Arg::ULONG_LONG; + arg_.ulong_long_value = static_cast<typename internal::MakeUnsigned<U>::Type>(value); + } + } + } +}; + +// Converts an integer argument to char for printf. +class CharConverter : public ArgVisitor<CharConverter, void> +{ +private: + internal::Arg &arg_; + + FMT_DISALLOW_COPY_AND_ASSIGN(CharConverter); + +public: + explicit CharConverter(internal::Arg &arg) + : arg_(arg) + { + } + + template <typename T> void visit_any_int(T value) + { + arg_.type = internal::Arg::CHAR; + arg_.int_value = static_cast<char>(value); + } +}; + +// Checks if an argument is a valid printf width specifier and sets +// left alignment if it is negative. +class WidthHandler : public ArgVisitor<WidthHandler, unsigned> +{ +private: + FormatSpec &spec_; + + FMT_DISALLOW_COPY_AND_ASSIGN(WidthHandler); + +public: + explicit WidthHandler(FormatSpec &spec) + : spec_(spec) + { + } + + void report_unhandled_arg() + { + FMT_THROW(FormatError("width is not integer")); + } + + template <typename T> unsigned visit_any_int(T value) + { + typedef typename internal::IntTraits<T>::MainType UnsignedType; + UnsignedType width = static_cast<UnsignedType>(value); + if (internal::is_negative(value)) + { + spec_.align_ = ALIGN_LEFT; + width = 0 - width; + } + unsigned int_max = std::numeric_limits<int>::max(); + if (width > int_max) + FMT_THROW(FormatError("number is too big")); + return static_cast<unsigned>(width); + } +}; +} // namespace internal + +/** + \rst + A ``printf`` argument formatter based on the `curiously recurring template + pattern <http://en.wikipedia.org/wiki/Curiously_recurring_template_pattern>`_. + + To use `~fmt::BasicPrintfArgFormatter` define a subclass that implements some + or all of the visit methods with the same signatures as the methods in + `~fmt::ArgVisitor`, for example, `~fmt::ArgVisitor::visit_int()`. + Pass the subclass as the *Impl* template parameter. When a formatting + function processes an argument, it will dispatch to a visit method + specific to the argument type. For example, if the argument type is + ``double`` then the `~fmt::ArgVisitor::visit_double()` method of a subclass + will be called. If the subclass doesn't contain a method with this signature, + then a corresponding method of `~fmt::BasicPrintfArgFormatter` or its + superclass will be called. + \endrst + */ +template <typename Impl, typename Char, typename Spec> class BasicPrintfArgFormatter : public internal::ArgFormatterBase<Impl, Char, Spec> +{ +private: + void write_null_pointer() + { + this->spec().type_ = 0; + this->write("(nil)"); + } + + typedef internal::ArgFormatterBase<Impl, Char, Spec> Base; + +public: + /** + \rst + Constructs an argument formatter object. + *writer* is a reference to the output writer and *spec* contains format + specifier information for standard argument types. + \endrst + */ + BasicPrintfArgFormatter(BasicWriter<Char> &w, Spec &s) + : internal::ArgFormatterBase<Impl, Char, Spec>(w, s) + { + } + + /** Formats an argument of type ``bool``. */ + void visit_bool(bool value) + { + Spec &fmt_spec = this->spec(); + if (fmt_spec.type_ != 's') + return this->visit_any_int(value); + fmt_spec.type_ = 0; + this->write(value); + } + + /** Formats a character. */ + void visit_char(int value) + { + const Spec &fmt_spec = this->spec(); + BasicWriter<Char> &w = this->writer(); + if (fmt_spec.type_ && fmt_spec.type_ != 'c') + w.write_int(value, fmt_spec); + typedef typename BasicWriter<Char>::CharPtr CharPtr; + CharPtr out = CharPtr(); + if (fmt_spec.width_ > 1) + { + Char fill = ' '; + out = w.grow_buffer(fmt_spec.width_); + if (fmt_spec.align_ != ALIGN_LEFT) + { + std::fill_n(out, fmt_spec.width_ - 1, fill); + out += fmt_spec.width_ - 1; + } + else + { + std::fill_n(out + 1, fmt_spec.width_ - 1, fill); + } + } + else + { + out = w.grow_buffer(1); + } + *out = static_cast<Char>(value); + } + + /** Formats a null-terminated C string. */ + void visit_cstring(const char *value) + { + if (value) + Base::visit_cstring(value); + else if (this->spec().type_ == 'p') + write_null_pointer(); + else + this->write("(null)"); + } + + /** Formats a pointer. */ + void visit_pointer(const void *value) + { + if (value) + return Base::visit_pointer(value); + this->spec().type_ = 0; + write_null_pointer(); + } + + /** Formats an argument of a custom (user-defined) type. */ + void visit_custom(internal::Arg::CustomValue c) + { + BasicFormatter<Char> formatter(ArgList(), this->writer()); + const Char format_str[] = {'}', 0}; + const Char *format = format_str; + c.format(&formatter, c.value, &format); + } +}; + +/** The default printf argument formatter. */ +template <typename Char> class PrintfArgFormatter : public BasicPrintfArgFormatter<PrintfArgFormatter<Char>, Char, FormatSpec> +{ +public: + /** Constructs an argument formatter object. */ + PrintfArgFormatter(BasicWriter<Char> &w, FormatSpec &s) + : BasicPrintfArgFormatter<PrintfArgFormatter<Char>, Char, FormatSpec>(w, s) + { + } +}; + +/** This template formats data and writes the output to a writer. */ +template <typename Char, typename ArgFormatter = PrintfArgFormatter<Char>> class PrintfFormatter : private internal::FormatterBase +{ +private: + BasicWriter<Char> &writer_; + + void parse_flags(FormatSpec &spec, const Char *&s); + + // Returns the argument with specified index or, if arg_index is equal + // to the maximum unsigned value, the next argument. + internal::Arg get_arg(const Char *s, unsigned arg_index = (std::numeric_limits<unsigned>::max)()); + + // Parses argument index, flags and width and returns the argument index. + unsigned parse_header(const Char *&s, FormatSpec &spec); + +public: + /** + \rst + Constructs a ``PrintfFormatter`` object. References to the arguments and + the writer are stored in the formatter object so make sure they have + appropriate lifetimes. + \endrst + */ + explicit PrintfFormatter(const ArgList &al, BasicWriter<Char> &w) + : FormatterBase(al) + , writer_(w) + { + } + + /** Formats stored arguments and writes the output to the writer. */ + void format(BasicCStringRef<Char> format_str); +}; + +template <typename Char, typename AF> void PrintfFormatter<Char, AF>::parse_flags(FormatSpec &spec, const Char *&s) +{ + for (;;) + { + switch (*s++) + { + case '-': + spec.align_ = ALIGN_LEFT; + break; + case '+': + spec.flags_ |= SIGN_FLAG | PLUS_FLAG; + break; + case '0': + spec.fill_ = '0'; + break; + case ' ': + spec.flags_ |= SIGN_FLAG; + break; + case '#': + spec.flags_ |= HASH_FLAG; + break; + default: + --s; + return; + } + } +} + +template <typename Char, typename AF> internal::Arg PrintfFormatter<Char, AF>::get_arg(const Char *s, unsigned arg_index) +{ + (void)s; + const char *error = FMT_NULL; + internal::Arg arg = arg_index == std::numeric_limits<unsigned>::max() ? next_arg(error) : FormatterBase::get_arg(arg_index - 1, error); + if (error) + FMT_THROW(FormatError(!*s ? "invalid format string" : error)); + return arg; +} + +template <typename Char, typename AF> unsigned PrintfFormatter<Char, AF>::parse_header(const Char *&s, FormatSpec &spec) +{ + unsigned arg_index = std::numeric_limits<unsigned>::max(); + Char c = *s; + if (c >= '0' && c <= '9') + { + // Parse an argument index (if followed by '$') or a width possibly + // preceded with '0' flag(s). + unsigned value = internal::parse_nonnegative_int(s); + if (*s == '$') // value is an argument index + { + ++s; + arg_index = value; + } + else + { + if (c == '0') + spec.fill_ = '0'; + if (value != 0) + { + // Nonzero value means that we parsed width and don't need to + // parse it or flags again, so return now. + spec.width_ = value; + return arg_index; + } + } + } + parse_flags(spec, s); + // Parse width. + if (*s >= '0' && *s <= '9') + { + spec.width_ = internal::parse_nonnegative_int(s); + } + else if (*s == '*') + { + ++s; + spec.width_ = internal::WidthHandler(spec).visit(get_arg(s)); + } + return arg_index; +} + +template <typename Char, typename AF> void PrintfFormatter<Char, AF>::format(BasicCStringRef<Char> format_str) +{ + const Char *start = format_str.c_str(); + const Char *s = start; + while (*s) + { + Char c = *s++; + if (c != '%') + continue; + if (*s == c) + { + write(writer_, start, s); + start = ++s; + continue; + } + write(writer_, start, s - 1); + + FormatSpec spec; + spec.align_ = ALIGN_RIGHT; + + // Parse argument index, flags and width. + unsigned arg_index = parse_header(s, spec); + + // Parse precision. + if (*s == '.') + { + ++s; + if ('0' <= *s && *s <= '9') + { + spec.precision_ = static_cast<int>(internal::parse_nonnegative_int(s)); + } + else if (*s == '*') + { + ++s; + spec.precision_ = internal::PrecisionHandler().visit(get_arg(s)); + } + else + { + spec.precision_ = 0; + } + } + + using internal::Arg; + Arg arg = get_arg(s, arg_index); + if (spec.flag(HASH_FLAG) && internal::IsZeroInt().visit(arg)) + spec.flags_ &= ~internal::to_unsigned<int>(HASH_FLAG); + if (spec.fill_ == '0') + { + if (arg.type <= Arg::LAST_NUMERIC_TYPE) + spec.align_ = ALIGN_NUMERIC; + else + spec.fill_ = ' '; // Ignore '0' flag for non-numeric types. + } + + // Parse length and convert the argument to the required type. + using internal::ArgConverter; + switch (*s++) + { + case 'h': + if (*s == 'h') + ArgConverter<signed char>(arg, *++s).visit(arg); + else + ArgConverter<short>(arg, *s).visit(arg); + break; + case 'l': + if (*s == 'l') + ArgConverter<fmt::LongLong>(arg, *++s).visit(arg); + else + ArgConverter<long>(arg, *s).visit(arg); + break; + case 'j': + ArgConverter<intmax_t>(arg, *s).visit(arg); + break; + case 'z': + ArgConverter<std::size_t>(arg, *s).visit(arg); + break; + case 't': + ArgConverter<std::ptrdiff_t>(arg, *s).visit(arg); + break; + case 'L': + // printf produces garbage when 'L' is omitted for long double, no + // need to do the same. + break; + default: + --s; + ArgConverter<void>(arg, *s).visit(arg); + } + + // Parse type. + if (!*s) + FMT_THROW(FormatError("invalid format string")); + spec.type_ = static_cast<char>(*s++); + + if (spec.type_ == 's') + { + // set the format type to the default if 's' is specified + spec.type_ = internal::DefaultType().visit(arg); + } + + if (arg.type <= Arg::LAST_INTEGER_TYPE) + { + // Normalize type. + switch (spec.type_) + { + case 'i': + case 'u': + spec.type_ = 'd'; + break; + case 'c': + // TODO: handle wchar_t + internal::CharConverter(arg).visit(arg); + break; + } + } + + start = s; + + // Format argument. + AF(writer_, spec).visit(arg); + } + write(writer_, start, s); +} + +inline void printf(Writer &w, CStringRef format, ArgList args) +{ + PrintfFormatter<char>(args, w).format(format); +} +FMT_VARIADIC(void, printf, Writer &, CStringRef) + +inline void printf(WWriter &w, WCStringRef format, ArgList args) +{ + PrintfFormatter<wchar_t>(args, w).format(format); +} +FMT_VARIADIC(void, printf, WWriter &, WCStringRef) + +/** + \rst + Formats arguments and returns the result as a string. + + **Example**:: + + std::string message = fmt::sprintf("The answer is %d", 42); + \endrst +*/ +inline std::string sprintf(CStringRef format, ArgList args) +{ + MemoryWriter w; + printf(w, format, args); + return w.str(); +} +FMT_VARIADIC(std::string, sprintf, CStringRef) + +inline std::wstring sprintf(WCStringRef format, ArgList args) +{ + WMemoryWriter w; + printf(w, format, args); + return w.str(); +} +FMT_VARIADIC_W(std::wstring, sprintf, WCStringRef) + +/** + \rst + Prints formatted data to the file *f*. + + **Example**:: + + fmt::fprintf(stderr, "Don't %s!", "panic"); + \endrst + */ +FMT_API int fprintf(std::FILE *f, CStringRef format, ArgList args); +FMT_VARIADIC(int, fprintf, std::FILE *, CStringRef) + +/** + \rst + Prints formatted data to ``stdout``. + + **Example**:: + + fmt::printf("Elapsed time: %.2f seconds", 1.23); + \endrst + */ +inline int printf(CStringRef format, ArgList args) +{ + return fprintf(stdout, format, args); +} +FMT_VARIADIC(int, printf, CStringRef) + +/** + \rst + Prints formatted data to the stream *os*. + + **Example**:: + + fprintf(cerr, "Don't %s!", "panic"); + \endrst + */ +inline int fprintf(std::ostream &os, CStringRef format_str, ArgList args) +{ + MemoryWriter w; + printf(w, format_str, args); + internal::write(os, w); + return static_cast<int>(w.size()); +} +FMT_VARIADIC(int, fprintf, std::ostream &, CStringRef) +} // namespace fmt + +#ifdef FMT_HEADER_ONLY +#include "printf.cc" +#endif + +#endif // FMT_PRINTF_H_ diff --git a/src/spdlog/fmt/bundled/time.h b/src/spdlog/fmt/bundled/time.h new file mode 100644 index 0000000000000000000000000000000000000000..e06fa5b496890e226fdadb3f2fb309d859f6d627 --- /dev/null +++ b/src/spdlog/fmt/bundled/time.h @@ -0,0 +1,187 @@ +/* + Formatting library for C++ - time formatting + + Copyright (c) 2012 - 2016, Victor Zverovich + All rights reserved. + + For the license information refer to format.h. + */ + +#ifndef FMT_TIME_H_ +#define FMT_TIME_H_ + +#include "format.h" +#include <ctime> + +#ifdef _MSC_VER +#pragma warning(push) +#pragma warning(disable : 4702) // unreachable code +#pragma warning(disable : 4996) // "deprecated" functions +#endif + +namespace fmt { +template <typename ArgFormatter> void format_arg(BasicFormatter<char, ArgFormatter> &f, const char *&format_str, const std::tm &tm) +{ + if (*format_str == ':') + ++format_str; + const char *end = format_str; + while (*end && *end != '}') + ++end; + if (*end != '}') + FMT_THROW(FormatError("missing '}' in format string")); + internal::MemoryBuffer<char, internal::INLINE_BUFFER_SIZE> format; + format.append(format_str, end + 1); + format[format.size() - 1] = '\0'; + Buffer<char> &buffer = f.writer().buffer(); + std::size_t start = buffer.size(); + for (;;) + { + std::size_t size = buffer.capacity() - start; + std::size_t count = std::strftime(&buffer[start], size, &format[0], &tm); + if (count != 0) + { + buffer.resize(start + count); + break; + } + if (size >= format.size() * 256) + { + // If the buffer is 256 times larger than the format string, assume + // that `strftime` gives an empty result. There doesn't seem to be a + // better way to distinguish the two cases: + // https://github.com/fmtlib/fmt/issues/367 + break; + } + const std::size_t MIN_GROWTH = 10; + buffer.reserve(buffer.capacity() + (size > MIN_GROWTH ? size : MIN_GROWTH)); + } + format_str = end + 1; +} + +namespace internal { +inline Null<> localtime_r(...) +{ + return Null<>(); +} +inline Null<> localtime_s(...) +{ + return Null<>(); +} +inline Null<> gmtime_r(...) +{ + return Null<>(); +} +inline Null<> gmtime_s(...) +{ + return Null<>(); +} +} // namespace internal + +// Thread-safe replacement for std::localtime +inline std::tm localtime(std::time_t time) +{ + struct LocalTime + { + std::time_t time_; + std::tm tm_; + + LocalTime(std::time_t t) + : time_(t) + { + } + + bool run() + { + using namespace fmt::internal; + return handle(localtime_r(&time_, &tm_)); + } + + bool handle(std::tm *tm) + { + return tm != FMT_NULL; + } + + bool handle(internal::Null<>) + { + using namespace fmt::internal; + return fallback(localtime_s(&tm_, &time_)); + } + + bool fallback(int res) + { + return res == 0; + } + + bool fallback(internal::Null<>) + { + using namespace fmt::internal; + std::tm *tm = std::localtime(&time_); + if (tm) + tm_ = *tm; + return tm != FMT_NULL; + } + }; + LocalTime lt(time); + if (lt.run()) + return lt.tm_; + // Too big time values may be unsupported. + FMT_THROW(fmt::FormatError("time_t value out of range")); + return std::tm(); +} + +// Thread-safe replacement for std::gmtime +inline std::tm gmtime(std::time_t time) +{ + struct GMTime + { + std::time_t time_; + std::tm tm_; + + GMTime(std::time_t t) + : time_(t) + { + } + + bool run() + { + using namespace fmt::internal; + return handle(gmtime_r(&time_, &tm_)); + } + + bool handle(std::tm *tm) + { + return tm != FMT_NULL; + } + + bool handle(internal::Null<>) + { + using namespace fmt::internal; + return fallback(gmtime_s(&tm_, &time_)); + } + + bool fallback(int res) + { + return res == 0; + } + + bool fallback(internal::Null<>) + { + std::tm *tm = std::gmtime(&time_); + if (tm != FMT_NULL) + tm_ = *tm; + return tm != FMT_NULL; + } + }; + GMTime gt(time); + if (gt.run()) + return gt.tm_; + // Too big time values may be unsupported. + FMT_THROW(fmt::FormatError("time_t value out of range")); + return std::tm(); +} +} // namespace fmt + +#ifdef _MSC_VER +#pragma warning(pop) +#endif + +#endif // FMT_TIME_H_ diff --git a/src/spdlog/fmt/fmt.h b/src/spdlog/fmt/fmt.h new file mode 100644 index 0000000000000000000000000000000000000000..e2e048546940c99441dc0f9c00df5ff74f8f8dd2 --- /dev/null +++ b/src/spdlog/fmt/fmt.h @@ -0,0 +1,33 @@ +// +// Copyright(c) 2016 Gabi Melman. +// Distributed under the MIT License (http://opensource.org/licenses/MIT) +// + +#pragma once + +// +// Include a bundled header-only copy of fmtlib or an external one. +// By default spdlog include its own copy. +// + +#if !defined(SPDLOG_FMT_EXTERNAL) + +#ifndef FMT_HEADER_ONLY +#define FMT_HEADER_ONLY +#endif +#ifndef FMT_USE_WINDOWS_H +#define FMT_USE_WINDOWS_H 0 +#endif +#include "bundled/format.h" +#if defined(SPDLOG_FMT_PRINTF) +#include "bundled/printf.h" +#endif + +#else // external fmtlib + +#include <fmt/format.h> +#if defined(SPDLOG_FMT_PRINTF) +#include <fmt/printf.h> +#endif + +#endif diff --git a/src/spdlog/fmt/ostr.h b/src/spdlog/fmt/ostr.h new file mode 100644 index 0000000000000000000000000000000000000000..cf4c32c51501a1c8b34909ff7079a70c86e281ca --- /dev/null +++ b/src/spdlog/fmt/ostr.h @@ -0,0 +1,15 @@ +// +// Copyright(c) 2016 Gabi Melman. +// Distributed under the MIT License (http://opensource.org/licenses/MIT) +// + +#pragma once + +// include external or bundled copy of fmtlib's ostream support +// +#if !defined(SPDLOG_FMT_EXTERNAL) +#include "bundled/ostream.h" +#include "fmt.h" +#else +#include <fmt/ostream.h> +#endif diff --git a/src/spdlog/formatter.h b/src/spdlog/formatter.h index 0ffcec03e05856215227bd5e388aa5adafa6e2e0..55388ec56947b8d1e9c37253f8c1f49d325f16df 100644 --- a/src/spdlog/formatter.h +++ b/src/spdlog/formatter.h @@ -5,41 +5,42 @@ #pragma once -#include <spdlog/details/log_msg.h> +#include "details/log_msg.h" -#include <vector> -#include <string> #include <memory> +#include <string> +#include <vector> -namespace spdlog -{ -namespace details -{ +namespace spdlog { +namespace details { class flag_formatter; } class formatter { public: - virtual ~formatter() {} - virtual void format(details::log_msg& msg) = 0; + virtual ~formatter() = default; + virtual void format(details::log_msg &msg) = 0; }; -class pattern_formatter : public formatter +class pattern_formatter SPDLOG_FINAL : public formatter { - public: - explicit pattern_formatter(const std::string& pattern); - pattern_formatter(const pattern_formatter&) = delete; - pattern_formatter& operator=(const pattern_formatter&) = delete; - void format(details::log_msg& msg) override; + explicit pattern_formatter(const std::string &pattern, pattern_time_type pattern_time = pattern_time_type::local, + std::string eol = spdlog::details::os::default_eol); + pattern_formatter(const pattern_formatter &) = delete; + pattern_formatter &operator=(const pattern_formatter &) = delete; + void format(details::log_msg &msg) override; + private: + const std::string _eol; const std::string _pattern; + const pattern_time_type _pattern_time; std::vector<std::unique_ptr<details::flag_formatter>> _formatters; + std::tm get_time(details::log_msg &msg); void handle_flag(char flag); - void compile_pattern(const std::string& pattern); + void compile_pattern(const std::string &pattern); }; -} - -#include <spdlog/details/pattern_formatter_impl.h> +} // namespace spdlog +#include "details/pattern_formatter_impl.h" diff --git a/src/spdlog/logger.h b/src/spdlog/logger.h index ca94e554273a54aa22cd7b2c659ef247805716f4..4f3bfa56cedc7f4e4e541ea03b6688d3912e1a84 100644 --- a/src/spdlog/logger.h +++ b/src/spdlog/logger.h @@ -5,112 +5,104 @@ #pragma once -// Thread safe logger +// Thread safe logger (except for set_pattern(..), set_formatter(..) and set_error_handler()) // Has name, log level, vector of std::shared sink pointers and formatter // Upon each log write the logger: // 1. Checks if its log level is enough to log the message // 2. Format the message using the formatter function // 3. Pass the formatted message to its sinks to performa the actual logging -#include <spdlog/sinks/base_sink.h> -#include <spdlog/common.h> -#include <spdlog/details/line_logger_fwd.h> +#include "common.h" +#include "sinks/base_sink.h" -#include <vector> #include <memory> #include <string> +#include <vector> -namespace spdlog -{ +namespace spdlog { class logger { public: - logger(const std::string& logger_name, sink_ptr single_sink); - logger(const std::string& name, sinks_init_list); - template<class It> - logger(const std::string& name, const It& begin, const It& end); + logger(const std::string &name, sink_ptr single_sink); + logger(const std::string &name, sinks_init_list sinks); + + template <class It> logger(std::string name, const It &begin, const It &end); virtual ~logger(); - logger(const logger&) = delete; - logger& operator=(const logger&) = delete; - void set_level(level::level_enum); + logger(const logger &) = delete; + logger &operator=(const logger &) = delete; + + template <typename... Args> void log(level::level_enum lvl, const char *fmt, const Args &... args); + template <typename... Args> void log(level::level_enum lvl, const char *msg); + template <typename Arg1, typename... Args> void trace(const char *fmt, const Arg1 &, const Args &... args); + template <typename Arg1, typename... Args> void debug(const char *fmt, const Arg1 &, const Args &... args); + template <typename Arg1, typename... Args> void info(const char *fmt, const Arg1 &, const Args &... args); + template <typename Arg1, typename... Args> void warn(const char *fmt, const Arg1 &, const Args &... args); + template <typename Arg1, typename... Args> void error(const char *fmt, const Arg1 &, const Args &... args); + template <typename Arg1, typename... Args> void critical(const char *fmt, const Arg1 &, const Args &... args); + +#ifdef SPDLOG_WCHAR_TO_UTF8_SUPPORT + template <typename... Args> void log(level::level_enum lvl, const wchar_t *msg); + template <typename... Args> void log(level::level_enum lvl, const wchar_t *fmt, const Args &... args); + template <typename... Args> void trace(const wchar_t *fmt, const Args &... args); + template <typename... Args> void debug(const wchar_t *fmt, const Args &... args); + template <typename... Args> void info(const wchar_t *fmt, const Args &... args); + template <typename... Args> void warn(const wchar_t *fmt, const Args &... args); + template <typename... Args> void error(const wchar_t *fmt, const Args &... args); + template <typename... Args> void critical(const wchar_t *fmt, const Args &... args); +#endif // SPDLOG_WCHAR_TO_UTF8_SUPPORT + + template <typename T> void log(level::level_enum lvl, const T &); + template <typename T> void trace(const T &msg); + template <typename T> void debug(const T &msg); + template <typename T> void info(const T &msg); + template <typename T> void warn(const T &msg); + template <typename T> void error(const T &msg); + template <typename T> void critical(const T &msg); + + bool should_log(level::level_enum msg_level) const; + void set_level(level::level_enum log_level); level::level_enum level() const; + const std::string &name() const; + void set_pattern(const std::string &pattern, pattern_time_type pattern_time = pattern_time_type::local); + void set_formatter(formatter_ptr msg_formatter); - const std::string& name() const; - bool should_log(level::level_enum) const; - - // automatically call flush() after a message of level log_level or higher is emitted + // automatically call flush() if message level >= log_level void flush_on(level::level_enum log_level); - // logger.info(cppformat_string, arg1, arg2, arg3, ...) call style - template <typename... Args> details::line_logger trace(const char* fmt, const Args&... args); - template <typename... Args> details::line_logger debug(const char* fmt, const Args&... args); - template <typename... Args> details::line_logger info(const char* fmt, const Args&... args); - template <typename... Args> details::line_logger notice(const char* fmt, const Args&... args); - template <typename... Args> details::line_logger warn(const char* fmt, const Args&... args); - template <typename... Args> details::line_logger error(const char* fmt, const Args&... args); - template <typename... Args> details::line_logger critical(const char* fmt, const Args&... args); - template <typename... Args> details::line_logger alert(const char* fmt, const Args&... args); - template <typename... Args> details::line_logger emerg(const char* fmt, const Args&... args); - - - // logger.info(msg) << ".." call style - template <typename T> details::line_logger trace(const T&); - template <typename T> details::line_logger debug(const T&); - template <typename T> details::line_logger info(const T&); - template <typename T> details::line_logger notice(const T&); - template <typename T> details::line_logger warn(const T&); - template <typename T> details::line_logger error(const T&); - template <typename T> details::line_logger critical(const T&); - template <typename T> details::line_logger alert(const T&); - template <typename T> details::line_logger emerg(const T&); - - - // logger.info() << ".." call style - details::line_logger trace(); - details::line_logger debug(); - details::line_logger info(); - details::line_logger notice(); - details::line_logger warn(); - details::line_logger error(); - details::line_logger critical(); - details::line_logger alert(); - details::line_logger emerg(); - - - - // Create log message with the given level, no matter what is the actual logger's level - template <typename... Args> - details::line_logger force_log(level::level_enum lvl, const char* fmt, const Args&... args); - - // Set the format of the log messages from this logger - void set_pattern(const std::string&); - void set_formatter(formatter_ptr); - virtual void flush(); + const std::vector<sink_ptr> &sinks() const; + + // error handler + virtual void set_error_handler(log_err_handler err_handler); + virtual log_err_handler error_handler(); + protected: - virtual void _log_msg(details::log_msg&); - virtual void _set_pattern(const std::string&); - virtual void _set_formatter(formatter_ptr); - details::line_logger _log_if_enabled(level::level_enum lvl); - template <typename... Args> - details::line_logger _log_if_enabled(level::level_enum lvl, const char* fmt, const Args&... args); - template<typename T> - inline details::line_logger _log_if_enabled(level::level_enum lvl, const T& msg); - - - friend details::line_logger; - std::string _name; + virtual void _sink_it(details::log_msg &msg); + virtual void _set_pattern(const std::string &pattern, pattern_time_type pattern_time); + virtual void _set_formatter(formatter_ptr msg_formatter); + + // default error handler: print the error to stderr with the max rate of 1 message/minute + virtual void _default_err_handler(const std::string &msg); + + // return true if the given message level should trigger a flush + bool _should_flush_on(const details::log_msg &msg); + + // increment the message count (only if defined(SPDLOG_ENABLE_MESSAGE_COUNTER)) + void _incr_msg_counter(details::log_msg &msg); + + const std::string _name; std::vector<sink_ptr> _sinks; formatter_ptr _formatter; spdlog::level_t _level; spdlog::level_t _flush_level; + log_err_handler _err_handler; + std::atomic<time_t> _last_err_time; + std::atomic<size_t> _msg_counter; }; -} - -#include <spdlog/details/logger_impl.h> -#include <spdlog/details/line_logger_impl.h> +} // namespace spdlog +#include "details/logger_impl.h" diff --git a/src/spdlog/sinks/android_sink.h b/src/spdlog/sinks/android_sink.h index 885f78da7395f04e921ed89f08253b521c511ccc..1284441e83598b1949aa31414e73a2c983caa090 100644 --- a/src/spdlog/sinks/android_sink.h +++ b/src/spdlog/sinks/android_sink.h @@ -7,55 +7,61 @@ #if defined(__ANDROID__) -#include <spdlog/sinks/base_sink.h> -#include <spdlog/details/null_mutex.h> +#include "../details/os.h" +#include "sink.h" #include <android/log.h> - +#include <chrono> #include <mutex> #include <string> +#include <thread> + +#if !defined(SPDLOG_ANDROID_RETRIES) +#define SPDLOG_ANDROID_RETRIES 2 +#endif + +namespace spdlog { namespace sinks { -namespace spdlog -{ -namespace sinks -{ /* -* Android sink (logging using __android_log_write) -*/ -template<class Mutex> -class base_android_sink : public base_sink < Mutex > + * Android sink (logging using __android_log_write) + * __android_log_write is thread-safe. No lock is needed. + */ +class android_sink : public sink { public: - explicit base_android_sink(std::string tag="spdlog"): _tag(tag) - { - } - - void flush() override + explicit android_sink(const std::string &tag = "spdlog", bool use_raw_msg = false) + : _tag(tag) + , _use_raw_msg(use_raw_msg) { } -protected: - void _sink_it(const details::log_msg& msg) override + void log(const details::log_msg &msg) override { const android_LogPriority priority = convert_to_android(msg.level); - const int expected_size = msg.formatted.size(); - const int size = __android_log_write( - priority, _tag.c_str(), msg.formatted.c_str() - ); - if (size > expected_size) + const char *msg_output = (_use_raw_msg ? msg.raw.c_str() : msg.formatted.c_str()); + + // See system/core/liblog/logger_write.c for explanation of return value + int ret = __android_log_write(priority, _tag.c_str(), msg_output); + int retry_count = 0; + while ((ret == -11 /*EAGAIN*/) && (retry_count < SPDLOG_ANDROID_RETRIES)) { - // Will write a little bit more than original message + details::os::sleep_for_millis(5); + ret = __android_log_write(priority, _tag.c_str(), msg_output); + retry_count++; } - else + + if (ret < 0) { - throw spdlog_ex("Send to Android logcat failed"); + throw spdlog_ex("__android_log_write() failed", ret); } } + void flush() override {} + private: static android_LogPriority convert_to_android(spdlog::level::level_enum level) { - switch(level) + switch (level) { case spdlog::level::trace: return ANDROID_LOG_VERBOSE; @@ -63,30 +69,21 @@ private: return ANDROID_LOG_DEBUG; case spdlog::level::info: return ANDROID_LOG_INFO; - case spdlog::level::notice: - return ANDROID_LOG_INFO; case spdlog::level::warn: return ANDROID_LOG_WARN; case spdlog::level::err: return ANDROID_LOG_ERROR; case spdlog::level::critical: return ANDROID_LOG_FATAL; - case spdlog::level::alert: - return ANDROID_LOG_FATAL; - case spdlog::level::emerg: - return ANDROID_LOG_FATAL; default: - throw spdlog_ex("Incorrect level value"); + return ANDROID_LOG_DEFAULT; } } std::string _tag; + bool _use_raw_msg; }; -typedef base_android_sink<std::mutex> android_sink_mt; -typedef base_android_sink<details::null_mutex> android_sink_st; - -} -} +}} // namespace spdlog::sinks #endif diff --git a/src/spdlog/sinks/ansicolor_sink.h b/src/spdlog/sinks/ansicolor_sink.h index 664b2599221e281f0dc3142c2ddf2cc2dde5170c..7cb2b3e78d5e22dd3e79d9ec4bbdab8dc66b974a 100644 --- a/src/spdlog/sinks/ansicolor_sink.h +++ b/src/spdlog/sinks/ansicolor_sink.h @@ -1,115 +1,133 @@ // -// Copyright(c) 2016 Kevin M. Godby (a modified version by spdlog). +// Copyright(c) 2017 spdlog authors. // Distributed under the MIT License (http://opensource.org/licenses/MIT) // #pragma once -#include <spdlog/sinks/base_sink.h> -#include <spdlog/common.h> +#include "../common.h" +#include "../details/os.h" +#include "base_sink.h" #include <string> -#include <map> +#include <unordered_map> -namespace spdlog -{ -namespace sinks -{ +namespace spdlog { namespace sinks { /** - * @brief The ansi_color_sink is a decorator around another sink and prefixes - * the output with an ANSI escape sequence color code depending on the severity + * This sink prefixes the output with an ANSI escape sequence color code depending on the severity * of the message. + * If no color terminal detected, omit the escape codes. */ -class ansicolor_sink : public sink +template <class Mutex> class ansicolor_sink : public base_sink<Mutex> { public: - ansicolor_sink(sink_ptr wrapped_sink); - virtual ~ansicolor_sink(); - - ansicolor_sink(const ansicolor_sink& other) = delete; - ansicolor_sink& operator=(const ansicolor_sink& other) = delete; - - virtual void log(const details::log_msg& msg) override; - virtual void flush() override; - - void set_color(level::level_enum level, const std::string& color); + explicit ansicolor_sink(FILE *file) + : target_file_(file) + { + should_do_colors_ = details::os::in_terminal(file) && details::os::is_color_terminal(); + colors_[level::trace] = cyan; + colors_[level::debug] = cyan; + colors_[level::info] = reset; + colors_[level::warn] = yellow + bold; + colors_[level::err] = red + bold; + colors_[level::critical] = bold + on_red; + colors_[level::off] = reset; + } + + ~ansicolor_sink() override + { + _flush(); + } + + void set_color(level::level_enum color_level, const std::string &color) + { + std::lock_guard<Mutex> lock(base_sink<Mutex>::_mutex); + colors_[color_level] = color; + } /// Formatting codes - const std::string reset = "\033[00m"; - const std::string bold = "\033[1m"; - const std::string dark = "\033[2m"; - const std::string underline = "\033[4m"; - const std::string blink = "\033[5m"; - const std::string reverse = "\033[7m"; - const std::string concealed = "\033[8m"; + const std::string reset = "\033[m"; + const std::string bold = "\033[1m"; + const std::string dark = "\033[2m"; + const std::string underline = "\033[4m"; + const std::string blink = "\033[5m"; + const std::string reverse = "\033[7m"; + const std::string concealed = "\033[8m"; + const std::string clear_line = "\033[K"; // Foreground colors - const std::string grey = "\033[30m"; - const std::string red = "\033[31m"; - const std::string green = "\033[32m"; - const std::string yellow = "\033[33m"; - const std::string blue = "\033[34m"; - const std::string magenta = "\033[35m"; - const std::string cyan = "\033[36m"; - const std::string white = "\033[37m"; + const std::string black = "\033[30m"; + const std::string red = "\033[31m"; + const std::string green = "\033[32m"; + const std::string yellow = "\033[33m"; + const std::string blue = "\033[34m"; + const std::string magenta = "\033[35m"; + const std::string cyan = "\033[36m"; + const std::string white = "\033[37m"; /// Background colors - const std::string on_grey = "\033[40m"; - const std::string on_red = "\033[41m"; - const std::string on_green = "\033[42m"; - const std::string on_yellow = "\033[43m"; - const std::string on_blue = "\033[44m"; + const std::string on_black = "\033[40m"; + const std::string on_red = "\033[41m"; + const std::string on_green = "\033[42m"; + const std::string on_yellow = "\033[43m"; + const std::string on_blue = "\033[44m"; const std::string on_magenta = "\033[45m"; - const std::string on_cyan = "\033[46m"; - const std::string on_white = "\033[47m"; - + const std::string on_cyan = "\033[46m"; + const std::string on_white = "\033[47m"; protected: - sink_ptr sink_; - std::map<level::level_enum, std::string> colors_; + void _sink_it(const details::log_msg &msg) override + { + // Wrap the originally formatted message in color codes. + // If color is not supported in the terminal, log as is instead. + if (should_do_colors_) + { + const std::string &prefix = colors_[msg.level]; + fwrite(prefix.data(), sizeof(char), prefix.size(), target_file_); + fwrite(msg.formatted.data(), sizeof(char), msg.formatted.size(), target_file_); + fwrite(reset.data(), sizeof(char), reset.size(), target_file_); + fwrite(clear_line.data(), sizeof(char), clear_line.size(), target_file_); + } + else + { + fwrite(msg.formatted.data(), sizeof(char), msg.formatted.size(), target_file_); + } + _flush(); + } + + void _flush() override + { + fflush(target_file_); + } + + FILE *target_file_; + bool should_do_colors_; + std::unordered_map<level::level_enum, std::string, level::level_hasher> colors_; }; -inline ansicolor_sink::ansicolor_sink(sink_ptr wrapped_sink) : sink_(wrapped_sink) -{ - colors_[level::trace] = cyan; - colors_[level::debug] = cyan; - colors_[level::info] = white; - colors_[level::notice] = bold + white; - colors_[level::warn] = bold + yellow; - colors_[level::err] = red; - colors_[level::critical] = bold + red; - colors_[level::alert] = bold + white + on_red; - colors_[level::emerg] = bold + yellow + on_red; - colors_[level::off] = reset; -} - -inline void ansicolor_sink::log(const details::log_msg& msg) +template <class Mutex> class ansicolor_stdout_sink : public ansicolor_sink<Mutex> { - // Wrap the originally formatted message in color codes - const std::string& prefix = colors_[msg.level]; - const std::string& s = msg.formatted.str(); - const std::string& suffix = reset; - details::log_msg m; - m.formatted << prefix << s << suffix; - sink_->log(m); -} - -inline void ansicolor_sink::flush() -{ - sink_->flush(); -} +public: + ansicolor_stdout_sink() + : ansicolor_sink<Mutex>(stdout) + { + } +}; -inline void ansicolor_sink::set_color(level::level_enum level, const std::string& color) -{ - colors_[level] = color; -} +using ansicolor_stdout_sink_mt = ansicolor_stdout_sink<std::mutex>; +using ansicolor_stdout_sink_st = ansicolor_stdout_sink<details::null_mutex>; -inline ansicolor_sink::~ansicolor_sink() +template <class Mutex> class ansicolor_stderr_sink : public ansicolor_sink<Mutex> { - flush(); -} +public: + ansicolor_stderr_sink() + : ansicolor_sink<Mutex>(stderr) + { + } +}; -} // namespace sinks -} // namespace spdlog +using ansicolor_stderr_sink_mt = ansicolor_stderr_sink<std::mutex>; +using ansicolor_stderr_sink_st = ansicolor_stderr_sink<details::null_mutex>; +}} // namespace spdlog::sinks diff --git a/src/spdlog/sinks/base_sink.h b/src/spdlog/sinks/base_sink.h index 615bb6f0c1e43a2a77d4cae1b95136fecb84c30f..bf67a59ca8337e479a23243761ecf0b908b57809 100644 --- a/src/spdlog/sinks/base_sink.h +++ b/src/spdlog/sinks/base_sink.h @@ -5,41 +5,42 @@ #pragma once // -// base sink templated over a mutex (either dummy or realy) -// concrete implementation should only overrid the _sink_it method. -// all locking is taken care of here so no locking needed by the implementors.. +// base sink templated over a mutex (either dummy or real) +// concrete implementation should only override the _sink_it method. +// all locking is taken care of here so no locking needed by the implementers.. // -#include <spdlog/sinks/sink.h> -#include <spdlog/formatter.h> -#include <spdlog/common.h> -#include <spdlog/details/log_msg.h> +#include "../common.h" +#include "../details/log_msg.h" +#include "../formatter.h" +#include "sink.h" #include <mutex> -namespace spdlog -{ -namespace sinks -{ -template<class Mutex> -class base_sink:public sink +namespace spdlog { namespace sinks { +template <class Mutex> class base_sink : public sink { public: - base_sink():_mutex() {} - virtual ~base_sink() = default; + base_sink() = default; - base_sink(const base_sink&) = delete; - base_sink& operator=(const base_sink&) = delete; + base_sink(const base_sink &) = delete; + base_sink &operator=(const base_sink &) = delete; - void log(const details::log_msg& msg) override + void log(const details::log_msg &msg) SPDLOG_FINAL override { std::lock_guard<Mutex> lock(_mutex); _sink_it(msg); } + void flush() SPDLOG_FINAL override + { + std::lock_guard<Mutex> lock(_mutex); + _flush(); + } + protected: - virtual void _sink_it(const details::log_msg& msg) = 0; + virtual void _sink_it(const details::log_msg &msg) = 0; + virtual void _flush() = 0; Mutex _mutex; }; -} -} +}} // namespace spdlog::sinks diff --git a/src/spdlog/sinks/dist_sink.h b/src/spdlog/sinks/dist_sink.h index 0e7cfc1e9bc6b55619a45ad4669a88edf9f4b1b4..b204790b5331dda8281b572b71375972dbc19a8b 100644 --- a/src/spdlog/sinks/dist_sink.h +++ b/src/spdlog/sinks/dist_sink.h @@ -5,68 +5,64 @@ #pragma once -#include <spdlog/details/log_msg.h> -#include <spdlog/details/null_mutex.h> -#include <spdlog/sinks/base_sink.h> -#include <spdlog/sinks/sink.h> +#include "../details/log_msg.h" +#include "../details/null_mutex.h" +#include "base_sink.h" +#include "sink.h" #include <algorithm> #include <memory> #include <mutex> #include <vector> -namespace spdlog -{ -namespace sinks -{ -template<class Mutex> -class dist_sink: public base_sink<Mutex> +// Distribution sink (mux). Stores a vector of sinks which get called when log is called + +namespace spdlog { namespace sinks { +template <class Mutex> class dist_sink : public base_sink<Mutex> { public: - explicit dist_sink() :_sinks() {} - dist_sink(const dist_sink&) = delete; - dist_sink& operator=(const dist_sink&) = delete; - virtual ~dist_sink() = default; - -protected: - void _sink_it(const details::log_msg& msg) override + explicit dist_sink() + : _sinks() { - for (auto iter = _sinks.begin(); iter != _sinks.end(); iter++) - (*iter)->log(msg); } + dist_sink(const dist_sink &) = delete; + dist_sink &operator=(const dist_sink &) = delete; +protected: std::vector<std::shared_ptr<sink>> _sinks; -public: - void flush() override + void _sink_it(const details::log_msg &msg) override { - std::lock_guard<Mutex> lock(base_sink<Mutex>::_mutex); - for (auto iter = _sinks.begin(); iter != _sinks.end(); iter++) - (*iter)->flush(); + for (auto &sink : _sinks) + { + if (sink->should_log(msg.level)) + { + sink->log(msg); + } + } } + void _flush() override + { + for (auto &sink : _sinks) + sink->flush(); + } + +public: void add_sink(std::shared_ptr<sink> sink) { std::lock_guard<Mutex> lock(base_sink<Mutex>::_mutex); - if (sink && - _sinks.end() == std::find(_sinks.begin(), _sinks.end(), sink)) - { - _sinks.push_back(sink); - } + _sinks.push_back(sink); } void remove_sink(std::shared_ptr<sink> sink) { std::lock_guard<Mutex> lock(base_sink<Mutex>::_mutex); - auto pos = std::find(_sinks.begin(), _sinks.end(), sink); - if (pos != _sinks.end()) - { - _sinks.erase(pos); - } + _sinks.erase(std::remove(_sinks.begin(), _sinks.end(), sink), _sinks.end()); } }; -typedef dist_sink<std::mutex> dist_sink_mt; -typedef dist_sink<details::null_mutex> dist_sink_st; -} -} +using dist_sink_mt = dist_sink<std::mutex>; +using dist_sink_st = dist_sink<details::null_mutex>; + +}} // namespace spdlog::sinks diff --git a/src/spdlog/sinks/file_sinks.h b/src/spdlog/sinks/file_sinks.h index 14b3cbfff3683747c86efa36a1e3d65daf200d4e..48322e83067583e0a10c2a7e9c8639ad4d71ef54 100644 --- a/src/spdlog/sinks/file_sinks.h +++ b/src/spdlog/sinks/file_sinks.h @@ -5,80 +5,93 @@ #pragma once -#include <spdlog/sinks/base_sink.h> -#include <spdlog/details/null_mutex.h> -#include <spdlog/details/file_helper.h> -#include <spdlog/details/format.h> +#include "../details/file_helper.h" +#include "../details/null_mutex.h" +#include "../fmt/fmt.h" +#include "base_sink.h" #include <algorithm> +#include <cerrno> #include <chrono> #include <cstdio> #include <ctime> #include <mutex> #include <string> -namespace spdlog -{ -namespace sinks -{ +namespace spdlog { namespace sinks { /* -* Trivial file sink with single file as target -*/ -template<class Mutex> -class simple_file_sink : public base_sink < Mutex > + * Trivial file sink with single file as target + */ +template <class Mutex> class simple_file_sink SPDLOG_FINAL : public base_sink<Mutex> { public: - explicit simple_file_sink(const filename_t &filename, - bool force_flush = false) : - _file_helper(force_flush) + explicit simple_file_sink(const filename_t &filename, bool truncate = false) + : _force_flush(false) { - _file_helper.open(filename); + _file_helper.open(filename, truncate); } - void flush() override + + void set_force_flush(bool force_flush) { - _file_helper.flush(); + _force_flush = force_flush; } protected: - void _sink_it(const details::log_msg& msg) override + void _sink_it(const details::log_msg &msg) override { _file_helper.write(msg); + if (_force_flush) + _file_helper.flush(); } + + void _flush() override + { + _file_helper.flush(); + } + private: details::file_helper _file_helper; + bool _force_flush; }; -typedef simple_file_sink<std::mutex> simple_file_sink_mt; -typedef simple_file_sink<details::null_mutex> simple_file_sink_st; +using simple_file_sink_mt = simple_file_sink<std::mutex>; +using simple_file_sink_st = simple_file_sink<details::null_mutex>; /* -* Rotating file sink based on size -*/ -template<class Mutex> -class rotating_file_sink : public base_sink < Mutex > + * Rotating file sink based on size + */ +template <class Mutex> class rotating_file_sink SPDLOG_FINAL : public base_sink<Mutex> { public: - rotating_file_sink(const filename_t &base_filename, const filename_t &extension, - std::size_t max_size, std::size_t max_files, - bool force_flush = false) : - _base_filename(base_filename), - _extension(extension), - _max_size(max_size), - _max_files(max_files), - _current_size(0), - _file_helper(force_flush) + rotating_file_sink(filename_t base_filename, std::size_t max_size, std::size_t max_files) + : _base_filename(std::move(base_filename)) + , _max_size(max_size) + , _max_files(max_files) { - _file_helper.open(calc_filename(_base_filename, 0, _extension)); - _current_size = _file_helper.size(); //expensive. called only once + _file_helper.open(calc_filename(_base_filename, 0)); + _current_size = _file_helper.size(); // expensive. called only once } - void flush() override + // calc filename according to index and file extension if exists. + // e.g. calc_filename("logs/mylog.txt, 3) => "logs/mylog.3.txt". + static filename_t calc_filename(const filename_t &filename, std::size_t index) { - _file_helper.flush(); + typename std::conditional<std::is_same<filename_t::value_type, char>::value, fmt::MemoryWriter, fmt::WMemoryWriter>::type w; + if (index != 0u) + { + filename_t basename, ext; + std::tie(basename, ext) = details::file_helper::split_by_extenstion(filename); + w.write(SPDLOG_FILENAME_T("{}.{}{}"), basename, index, ext); + } + else + { + w.write(SPDLOG_FILENAME_T("{}"), filename); + } + return w.str(); } protected: - void _sink_it(const details::log_msg& msg) override + void _sink_it(const details::log_msg &msg) override { _current_size += msg.formatted.size(); if (_current_size > _max_size) @@ -89,133 +102,124 @@ protected: _file_helper.write(msg); } -private: - static filename_t calc_filename(const filename_t& filename, std::size_t index, const filename_t& extension) + void _flush() override { - std::conditional<std::is_same<filename_t::value_type, char>::value, fmt::MemoryWriter, fmt::WMemoryWriter>::type w; - if (index) - w.write(SPDLOG_FILENAME_T("{}.{}.{}"), filename, index, extension); - else - w.write(SPDLOG_FILENAME_T("{}.{}"), filename, extension); - return w.str(); + _file_helper.flush(); } +private: // Rotate files: // log.txt -> log.1.txt - // log.1.txt -> log2.txt - // log.2.txt -> log3.txt + // log.1.txt -> log.2.txt + // log.2.txt -> log.3.txt // log.3.txt -> delete - void _rotate() { using details::os::filename_to_str; _file_helper.close(); for (auto i = _max_files; i > 0; --i) { - filename_t src = calc_filename(_base_filename, i - 1, _extension); - filename_t target = calc_filename(_base_filename, i, _extension); + filename_t src = calc_filename(_base_filename, i - 1); + filename_t target = calc_filename(_base_filename, i); if (details::file_helper::file_exists(target)) { if (details::os::remove(target) != 0) { - throw spdlog_ex("rotating_file_sink: failed removing " + filename_to_str(target)); + throw spdlog_ex("rotating_file_sink: failed removing " + filename_to_str(target), errno); } } - if (details::file_helper::file_exists(src) && details::os::rename(src, target)) + if (details::file_helper::file_exists(src) && details::os::rename(src, target) != 0) { - throw spdlog_ex("rotating_file_sink: failed renaming " + filename_to_str(src) + " to " + filename_to_str(target)); + throw spdlog_ex("rotating_file_sink: failed renaming " + filename_to_str(src) + " to " + filename_to_str(target), errno); } } _file_helper.reopen(true); } + filename_t _base_filename; - filename_t _extension; std::size_t _max_size; std::size_t _max_files; std::size_t _current_size; details::file_helper _file_helper; }; -typedef rotating_file_sink<std::mutex> rotating_file_sink_mt; -typedef rotating_file_sink<details::null_mutex>rotating_file_sink_st; +using rotating_file_sink_mt = rotating_file_sink<std::mutex>; +using rotating_file_sink_st = rotating_file_sink<details::null_mutex>; /* -* Default generator of daily log file names. -*/ + * Default generator of daily log file names. + */ struct default_daily_file_name_calculator { - //Create filename for the form basename.YYYY-MM-DD_hh-mm.extension - static filename_t calc_filename(const filename_t& basename, const filename_t& extension) + // Create filename for the form filename.YYYY-MM-DD_hh-mm.ext + static filename_t calc_filename(const filename_t &filename) { std::tm tm = spdlog::details::os::localtime(); + filename_t basename, ext; + std::tie(basename, ext) = details::file_helper::split_by_extenstion(filename); std::conditional<std::is_same<filename_t::value_type, char>::value, fmt::MemoryWriter, fmt::WMemoryWriter>::type w; - w.write(SPDLOG_FILENAME_T("{}_{:04d}-{:02d}-{:02d}_{:02d}-{:02d}.{}"), basename, tm.tm_year + 1900, tm.tm_mon + 1, tm.tm_mday, tm.tm_hour, tm.tm_min, extension); + w.write(SPDLOG_FILENAME_T("{}_{:04d}-{:02d}-{:02d}_{:02d}-{:02d}{}"), basename, tm.tm_year + 1900, tm.tm_mon + 1, tm.tm_mday, + tm.tm_hour, tm.tm_min, ext); return w.str(); } }; /* -* Generator of daily log file names in format basename.YYYY-MM-DD.extension -*/ + * Generator of daily log file names in format basename.YYYY-MM-DD.ext + */ struct dateonly_daily_file_name_calculator { - //Create filename for the form basename.YYYY-MM-DD.extension - static filename_t calc_filename(const filename_t& basename, const filename_t& extension) + // Create filename for the form basename.YYYY-MM-DD + static filename_t calc_filename(const filename_t &filename) { std::tm tm = spdlog::details::os::localtime(); + filename_t basename, ext; + std::tie(basename, ext) = details::file_helper::split_by_extenstion(filename); std::conditional<std::is_same<filename_t::value_type, char>::value, fmt::MemoryWriter, fmt::WMemoryWriter>::type w; - w.write(SPDLOG_FILENAME_T("{}_{:04d}-{:02d}-{:02d}.{}"), basename, tm.tm_year + 1900, tm.tm_mon + 1, tm.tm_mday, extension); + w.write(SPDLOG_FILENAME_T("{}_{:04d}-{:02d}-{:02d}{}"), basename, tm.tm_year + 1900, tm.tm_mon + 1, tm.tm_mday, ext); return w.str(); } }; /* -* Rotating file sink based on date. rotates at midnight -*/ -template<class Mutex, class FileNameCalc = default_daily_file_name_calculator> -class daily_file_sink :public base_sink < Mutex > + * Rotating file sink based on date. rotates at midnight + */ +template <class Mutex, class FileNameCalc = default_daily_file_name_calculator> class daily_file_sink SPDLOG_FINAL : public base_sink<Mutex> { public: - //create daily file sink which rotates on given time - daily_file_sink( - const filename_t& base_filename, - const filename_t& extension, - int rotation_hour, - int rotation_minute, - bool force_flush = false) : _base_filename(base_filename), - _extension(extension), - _rotation_h(rotation_hour), - _rotation_m(rotation_minute), - _file_helper(force_flush) + // create daily file sink which rotates on given time + daily_file_sink(filename_t base_filename, int rotation_hour, int rotation_minute) + : _base_filename(std::move(base_filename)) + , _rotation_h(rotation_hour) + , _rotation_m(rotation_minute) { if (rotation_hour < 0 || rotation_hour > 23 || rotation_minute < 0 || rotation_minute > 59) throw spdlog_ex("daily_file_sink: Invalid rotation time in ctor"); _rotation_tp = _next_rotation_tp(); - _file_helper.open(FileNameCalc::calc_filename(_base_filename, _extension)); - } - - void flush() override - { - _file_helper.flush(); + _file_helper.open(FileNameCalc::calc_filename(_base_filename)); } protected: - void _sink_it(const details::log_msg& msg) override + void _sink_it(const details::log_msg &msg) override { if (std::chrono::system_clock::now() >= _rotation_tp) { - _file_helper.open(FileNameCalc::calc_filename(_base_filename, _extension)); + _file_helper.open(FileNameCalc::calc_filename(_base_filename)); _rotation_tp = _next_rotation_tp(); } _file_helper.write(msg); } + void _flush() override + { + _file_helper.flush(); + } + private: std::chrono::system_clock::time_point _next_rotation_tp() { - using namespace std::chrono; - auto now = system_clock::now(); + auto now = std::chrono::system_clock::now(); time_t tnow = std::chrono::system_clock::to_time_t(now); tm date = spdlog::details::os::localtime(tnow); date.tm_hour = _rotation_h; @@ -223,20 +227,20 @@ private: date.tm_sec = 0; auto rotation_time = std::chrono::system_clock::from_time_t(std::mktime(&date)); if (rotation_time > now) + { return rotation_time; - else - return system_clock::time_point(rotation_time + hours(24)); + } + return {rotation_time + std::chrono::hours(24)}; } filename_t _base_filename; - filename_t _extension; int _rotation_h; int _rotation_m; std::chrono::system_clock::time_point _rotation_tp; details::file_helper _file_helper; }; -typedef daily_file_sink<std::mutex> daily_file_sink_mt; -typedef daily_file_sink<details::null_mutex> daily_file_sink_st; -} -} +using daily_file_sink_mt = daily_file_sink<std::mutex>; +using daily_file_sink_st = daily_file_sink<details::null_mutex>; + +}} // namespace spdlog::sinks diff --git a/src/spdlog/sinks/msvc_sink.h b/src/spdlog/sinks/msvc_sink.h index 16342ca26435c8aab8dce40a241995f1ea986762..e58d49ce377f086a567b4017879fedaddaf414a7 100644 --- a/src/spdlog/sinks/msvc_sink.h +++ b/src/spdlog/sinks/msvc_sink.h @@ -5,46 +5,37 @@ #pragma once -#if defined(_MSC_VER) +#if defined(_WIN32) -#include <spdlog/sinks/base_sink.h> -#include <spdlog/details/null_mutex.h> +#include "../details/null_mutex.h" +#include "base_sink.h" -#include <WinBase.h> +#include <winbase.h> #include <mutex> #include <string> -namespace spdlog -{ -namespace sinks -{ +namespace spdlog { namespace sinks { /* -* MSVC sink (logging using OutputDebugStringA) -*/ -template<class Mutex> -class msvc_sink : public base_sink < Mutex > + * MSVC sink (logging using OutputDebugStringA) + */ +template <class Mutex> class msvc_sink : public base_sink<Mutex> { public: - explicit msvc_sink() - { - } - - void flush() override - { - } + explicit msvc_sink() {} protected: - void _sink_it(const details::log_msg& msg) override + void _sink_it(const details::log_msg &msg) override { OutputDebugStringA(msg.formatted.c_str()); } + + void _flush() override {} }; -typedef msvc_sink<std::mutex> msvc_sink_mt; -typedef msvc_sink<details::null_mutex> msvc_sink_st; +using msvc_sink_mt = msvc_sink<std::mutex>; +using msvc_sink_st = msvc_sink<details::null_mutex>; -} -} +}} // namespace spdlog::sinks #endif diff --git a/src/spdlog/sinks/null_sink.h b/src/spdlog/sinks/null_sink.h index 68bd9c94da13591b5bd596c5c7a1c8f2ff9fb56f..d8eab97c304a4f40332b55e8a6b584040c17ca1c 100644 --- a/src/spdlog/sinks/null_sink.h +++ b/src/spdlog/sinks/null_sink.h @@ -5,30 +5,22 @@ #pragma once -#include <spdlog/sinks/base_sink.h> -#include <spdlog/details/null_mutex.h> +#include "../details/null_mutex.h" +#include "base_sink.h" #include <mutex> -namespace spdlog -{ -namespace sinks -{ +namespace spdlog { namespace sinks { -template <class Mutex> -class null_sink : public base_sink < Mutex > +template <class Mutex> class null_sink : public base_sink<Mutex> { protected: - void _sink_it(const details::log_msg&) override - {} - - void flush() override - {} + void _sink_it(const details::log_msg &) override {} + void _flush() override {} }; -typedef null_sink<details::null_mutex> null_sink_st; -typedef null_sink<std::mutex> null_sink_mt; -} -} +using null_sink_mt = null_sink<details::null_mutex>; +using null_sink_st = null_sink<details::null_mutex>; +}} // namespace spdlog::sinks diff --git a/src/spdlog/sinks/ostream_sink.h b/src/spdlog/sinks/ostream_sink.h index feb5efa18dd17e91b3add638a30bccd6112f49e9..bb79de768d06a1dbdd35f9f1030fc85571049ced 100644 --- a/src/spdlog/sinks/ostream_sink.h +++ b/src/spdlog/sinks/ostream_sink.h @@ -5,43 +5,42 @@ #pragma once -#include <spdlog/details/null_mutex.h> -#include <spdlog/sinks/base_sink.h> +#include "../details/null_mutex.h" +#include "base_sink.h" -#include <ostream> #include <mutex> +#include <ostream> -namespace spdlog -{ -namespace sinks -{ -template<class Mutex> -class ostream_sink: public base_sink<Mutex> +namespace spdlog { namespace sinks { +template <class Mutex> class ostream_sink : public base_sink<Mutex> { public: - explicit ostream_sink(std::ostream& os, bool force_flush=false) :_ostream(os), _force_flush(force_flush) {} - ostream_sink(const ostream_sink&) = delete; - ostream_sink& operator=(const ostream_sink&) = delete; - virtual ~ostream_sink() = default; + explicit ostream_sink(std::ostream &os, bool force_flush = false) + : _ostream(os) + , _force_flush(force_flush) + { + } + ostream_sink(const ostream_sink &) = delete; + ostream_sink &operator=(const ostream_sink &) = delete; protected: - void _sink_it(const details::log_msg& msg) override + void _sink_it(const details::log_msg &msg) override { _ostream.write(msg.formatted.data(), msg.formatted.size()); if (_force_flush) _ostream.flush(); } - void flush() override + void _flush() override { _ostream.flush(); } - std::ostream& _ostream; + std::ostream &_ostream; bool _force_flush; }; -typedef ostream_sink<std::mutex> ostream_sink_mt; -typedef ostream_sink<details::null_mutex> ostream_sink_st; -} -} +using ostream_sink_mt = ostream_sink<std::mutex>; +using ostream_sink_st = ostream_sink<details::null_mutex>; + +}} // namespace spdlog::sinks diff --git a/src/spdlog/sinks/sink.h b/src/spdlog/sinks/sink.h index 39dc771add1cd3ea851b3a541ac5553c3dae6253..f3c146d993bcd18ac25e2f1226f0429155ccbcd2 100644 --- a/src/spdlog/sinks/sink.h +++ b/src/spdlog/sinks/sink.h @@ -3,22 +3,40 @@ // Distributed under the MIT License (http://opensource.org/licenses/MIT) // - #pragma once -#include <spdlog/details/log_msg.h> +#include "../details/log_msg.h" -namespace spdlog -{ -namespace sinks -{ +namespace spdlog { namespace sinks { class sink { public: - virtual ~sink() {} - virtual void log(const details::log_msg& msg) = 0; + virtual ~sink() = default; + + virtual void log(const details::log_msg &msg) = 0; virtual void flush() = 0; + + bool should_log(level::level_enum msg_level) const; + void set_level(level::level_enum log_level); + level::level_enum level() const; + +private: + level_t _level{level::trace}; }; + +inline bool sink::should_log(level::level_enum msg_level) const +{ + return msg_level >= _level.load(std::memory_order_relaxed); +} + +inline void sink::set_level(level::level_enum log_level) +{ + _level.store(log_level); } + +inline level::level_enum sink::level() const +{ + return static_cast<spdlog::level::level_enum>(_level.load(std::memory_order_relaxed)); } +}} // namespace spdlog::sinks diff --git a/src/spdlog/sinks/stdout_sinks.h b/src/spdlog/sinks/stdout_sinks.h index ca4c55ac85d11edfa011782705922bfd125de64c..4494323bc17d48c26fb45b5388a477368c847286 100644 --- a/src/spdlog/sinks/stdout_sinks.h +++ b/src/spdlog/sinks/stdout_sinks.h @@ -5,70 +5,71 @@ #pragma once -#include <spdlog/details/null_mutex.h> +#include "../details/null_mutex.h" +#include "base_sink.h" #include <cstdio> #include <memory> #include <mutex> -namespace spdlog -{ -namespace sinks -{ +namespace spdlog { namespace sinks { -template <class Mutex> -class stdout_sink : public base_sink<Mutex> +template <class Mutex> class stdout_sink SPDLOG_FINAL : public base_sink<Mutex> { using MyType = stdout_sink<Mutex>; + public: - stdout_sink() {} + explicit stdout_sink() = default; + static std::shared_ptr<MyType> instance() { static std::shared_ptr<MyType> instance = std::make_shared<MyType>(); return instance; } - void _sink_it(const details::log_msg& msg) override +protected: + void _sink_it(const details::log_msg &msg) override { fwrite(msg.formatted.data(), sizeof(char), msg.formatted.size(), stdout); - flush(); + _flush(); } - void flush() override + void _flush() override { fflush(stdout); } }; -typedef stdout_sink<details::null_mutex> stdout_sink_st; -typedef stdout_sink<std::mutex> stdout_sink_mt; +using stdout_sink_mt = stdout_sink<std::mutex>; +using stdout_sink_st = stdout_sink<details::null_mutex>; - -template <class Mutex> -class stderr_sink : public base_sink<Mutex> +template <class Mutex> class stderr_sink SPDLOG_FINAL : public base_sink<Mutex> { using MyType = stderr_sink<Mutex>; + public: - stderr_sink() {} + explicit stderr_sink() = default; + static std::shared_ptr<MyType> instance() { static std::shared_ptr<MyType> instance = std::make_shared<MyType>(); return instance; } - void _sink_it(const details::log_msg& msg) override +protected: + void _sink_it(const details::log_msg &msg) override { fwrite(msg.formatted.data(), sizeof(char), msg.formatted.size(), stderr); - flush(); + _flush(); } - void flush() override + void _flush() override { fflush(stderr); } }; -typedef stderr_sink<std::mutex> stderr_sink_mt; -typedef stderr_sink<details::null_mutex> stderr_sink_st; -} -} +using stderr_sink_mt = stderr_sink<std::mutex>; +using stderr_sink_st = stderr_sink<details::null_mutex>; + +}} // namespace spdlog::sinks diff --git a/src/spdlog/sinks/syslog_sink.h b/src/spdlog/sinks/syslog_sink.h index 5d7ccf871f108b8c4fbdf2a594aafb8e4b2828c7..e4472f2f4891c93b2acbed614e1801944482d750 100644 --- a/src/spdlog/sinks/syslog_sink.h +++ b/src/spdlog/sinks/syslog_sink.h @@ -5,21 +5,18 @@ #pragma once -#if defined(__linux__) || defined(__APPLE__) +#include "../common.h" -#include <spdlog/sinks/sink.h> -#include <spdlog/common.h> -#include <spdlog/details/log_msg.h> +#ifdef SPDLOG_ENABLE_SYSLOG + +#include "../details/log_msg.h" +#include "sink.h" #include <array> #include <string> #include <syslog.h> - -namespace spdlog -{ -namespace sinks -{ +namespace spdlog { namespace sinks { /** * Sink that write to syslog using the `syscall()` library call. * @@ -29,44 +26,39 @@ class syslog_sink : public sink { public: // - syslog_sink(const std::string& ident = "", int syslog_option=0, int syslog_facility=LOG_USER): - _ident(ident) + syslog_sink(const std::string &ident = "", int syslog_option = 0, int syslog_facility = LOG_USER) + : _ident(ident) { - _priorities[static_cast<int>(level::trace)] = LOG_DEBUG; - _priorities[static_cast<int>(level::debug)] = LOG_DEBUG; - _priorities[static_cast<int>(level::info)] = LOG_INFO; - _priorities[static_cast<int>(level::notice)] = LOG_NOTICE; - _priorities[static_cast<int>(level::warn)] = LOG_WARNING; - _priorities[static_cast<int>(level::err)] = LOG_ERR; - _priorities[static_cast<int>(level::critical)] = LOG_CRIT; - _priorities[static_cast<int>(level::alert)] = LOG_ALERT; - _priorities[static_cast<int>(level::emerg)] = LOG_EMERG; - _priorities[static_cast<int>(level::off)] = LOG_INFO; + _priorities[static_cast<size_t>(level::trace)] = LOG_DEBUG; + _priorities[static_cast<size_t>(level::debug)] = LOG_DEBUG; + _priorities[static_cast<size_t>(level::info)] = LOG_INFO; + _priorities[static_cast<size_t>(level::warn)] = LOG_WARNING; + _priorities[static_cast<size_t>(level::err)] = LOG_ERR; + _priorities[static_cast<size_t>(level::critical)] = LOG_CRIT; + _priorities[static_cast<size_t>(level::off)] = LOG_INFO; - //set ident to be program name if empty - ::openlog(_ident.empty()? nullptr:_ident.c_str(), syslog_option, syslog_facility); + // set ident to be program name if empty + ::openlog(_ident.empty() ? nullptr : _ident.c_str(), syslog_option, syslog_facility); } - ~syslog_sink() + + ~syslog_sink() override { ::closelog(); } - syslog_sink(const syslog_sink&) = delete; - syslog_sink& operator=(const syslog_sink&) = delete; + syslog_sink(const syslog_sink &) = delete; + syslog_sink &operator=(const syslog_sink &) = delete; void log(const details::log_msg &msg) override { ::syslog(syslog_prio_from_level(msg), "%s", msg.raw.str().c_str()); } - void flush() override - { - } - + void flush() override {} private: - std::array<int, 10> _priorities; - //must store the ident because the man says openlog might use the pointer as is and not a string copy + std::array<int, 7> _priorities; + // must store the ident because the man says openlog might use the pointer as is and not a string copy const std::string _ident; // @@ -74,10 +66,9 @@ private: // int syslog_prio_from_level(const details::log_msg &msg) const { - return _priorities[static_cast<int>(msg.level)]; + return _priorities[static_cast<size_t>(msg.level)]; } }; -} -} +}} // namespace spdlog::sinks #endif diff --git a/src/spdlog/sinks/wincolor_sink.h b/src/spdlog/sinks/wincolor_sink.h new file mode 100644 index 0000000000000000000000000000000000000000..77ae59e150a24880363b668c3ab682dd1989cb45 --- /dev/null +++ b/src/spdlog/sinks/wincolor_sink.h @@ -0,0 +1,119 @@ +// +// Copyright(c) 2016 spdlog +// Distributed under the MIT License (http://opensource.org/licenses/MIT) +// + +#pragma once + +#include "../common.h" +#include "../details/null_mutex.h" +#include "base_sink.h" + +#include <mutex> +#include <string> +#include <unordered_map> +#include <wincon.h> + +namespace spdlog { namespace sinks { +/* + * Windows color console sink. Uses WriteConsoleA to write to the console with colors + */ +template <class Mutex> class wincolor_sink : public base_sink<Mutex> +{ +public: + const WORD BOLD = FOREGROUND_INTENSITY; + const WORD RED = FOREGROUND_RED; + const WORD CYAN = FOREGROUND_GREEN | FOREGROUND_BLUE; + const WORD WHITE = FOREGROUND_RED | FOREGROUND_GREEN | FOREGROUND_BLUE; + const WORD YELLOW = FOREGROUND_RED | FOREGROUND_GREEN; + + wincolor_sink(HANDLE std_handle) + : out_handle_(std_handle) + { + colors_[level::trace] = CYAN; + colors_[level::debug] = CYAN; + colors_[level::info] = WHITE | BOLD; + colors_[level::warn] = YELLOW | BOLD; + colors_[level::err] = RED | BOLD; // red bold + colors_[level::critical] = BACKGROUND_RED | WHITE | BOLD; // white bold on red background + colors_[level::off] = 0; + } + + ~wincolor_sink() override + { + this->flush(); + } + + wincolor_sink(const wincolor_sink &other) = delete; + wincolor_sink &operator=(const wincolor_sink &other) = delete; + + // change the color for the given level + void set_color(level::level_enum level, WORD color) + { + std::lock_guard<Mutex> lock(base_sink<Mutex>::_mutex); + colors_[level] = color; + } + +protected: + void _sink_it(const details::log_msg &msg) override + { + auto color = colors_[msg.level]; + auto orig_attribs = set_console_attribs(color); + WriteConsoleA(out_handle_, msg.formatted.data(), static_cast<DWORD>(msg.formatted.size()), nullptr, nullptr); + SetConsoleTextAttribute(out_handle_, orig_attribs); // reset to orig colors + } + + void _flush() override + { + // windows console always flushed? + } + +private: + HANDLE out_handle_; + std::unordered_map<level::level_enum, WORD, level::level_hasher> colors_; + + // set color and return the orig console attributes (for resetting later) + WORD set_console_attribs(WORD attribs) + { + CONSOLE_SCREEN_BUFFER_INFO orig_buffer_info; + GetConsoleScreenBufferInfo(out_handle_, &orig_buffer_info); + WORD back_color = orig_buffer_info.wAttributes; + // retrieve the current background color + back_color &= static_cast<WORD>(~(FOREGROUND_RED | FOREGROUND_GREEN | FOREGROUND_BLUE | FOREGROUND_INTENSITY)); + // keep the background color unchanged + SetConsoleTextAttribute(out_handle_, attribs | back_color); + return orig_buffer_info.wAttributes; // return orig attribs + } +}; + +// +// windows color console to stdout +// +template <class Mutex> class wincolor_stdout_sink : public wincolor_sink<Mutex> +{ +public: + wincolor_stdout_sink() + : wincolor_sink<Mutex>(GetStdHandle(STD_OUTPUT_HANDLE)) + { + } +}; + +using wincolor_stdout_sink_mt = wincolor_stdout_sink<std::mutex>; +using wincolor_stdout_sink_st = wincolor_stdout_sink<details::null_mutex>; + +// +// windows color console to stderr +// +template <class Mutex> class wincolor_stderr_sink : public wincolor_sink<Mutex> +{ +public: + wincolor_stderr_sink() + : wincolor_sink<Mutex>(GetStdHandle(STD_ERROR_HANDLE)) + { + } +}; + +using wincolor_stderr_sink_mt = wincolor_stderr_sink<std::mutex>; +using wincolor_stderr_sink_st = wincolor_stderr_sink<details::null_mutex>; + +}} // namespace spdlog::sinks diff --git a/src/spdlog/sinks/windebug_sink.h b/src/spdlog/sinks/windebug_sink.h new file mode 100644 index 0000000000000000000000000000000000000000..5bd58042f6fc8292a235715c0ca1b051def1c0fa --- /dev/null +++ b/src/spdlog/sinks/windebug_sink.h @@ -0,0 +1,24 @@ +// +// Copyright(c) 2017 Alexander Dalshov. +// Distributed under the MIT License (http://opensource.org/licenses/MIT) +// + +#pragma once + +#if defined(_WIN32) + +#include "msvc_sink.h" + +namespace spdlog { namespace sinks { + +/* + * Windows debug sink (logging using OutputDebugStringA, synonym for msvc_sink) + */ +template <class Mutex> using windebug_sink = msvc_sink<Mutex>; + +using windebug_sink_mt = msvc_sink_mt; +using windebug_sink_st = msvc_sink_st; + +}} // namespace spdlog::sinks + +#endif diff --git a/src/spdlog/spdlog.h b/src/spdlog/spdlog.h index 043e81419cf03384370c2c6437321b4ea30d2161..34706a38ec615ad47f92b6363ba6095f0f8fc53b 100644 --- a/src/spdlog/spdlog.h +++ b/src/spdlog/spdlog.h @@ -2,45 +2,49 @@ // Copyright(c) 2015 Gabi Melman. // Distributed under the MIT License (http://opensource.org/licenses/MIT) // - // spdlog main header file. // see example.cpp for usage example #pragma once -#include <spdlog/tweakme.h> -#include <spdlog/common.h> -#include <spdlog/logger.h> +#include "common.h" +#include "logger.h" -#include <memory> -#include <functional> #include <chrono> +#include <functional> +#include <memory> #include <string> -namespace spdlog -{ +namespace spdlog { +// // Return an existing logger or nullptr if a logger with such name doesn't exist. -// Examples: +// example: spdlog::get("my_logger")->info("hello {}", "world"); // -// spdlog::get("mylog")->info("Hello"); -// auto logger = spdlog::get("mylog"); -// logger.info("This is another message" , x, y, z); -// logger.info() << "This is another message" << x << y << z; -std::shared_ptr<logger> get(const std::string& name); +std::shared_ptr<logger> get(const std::string &name); // // Set global formatting // example: spdlog::set_pattern("%Y-%m-%d %H:%M:%S.%e %l : %v"); // -void set_pattern(const std::string& format_string); +void set_pattern(const std::string &format_string); void set_formatter(formatter_ptr f); // -// Set global logging level for +// Set global logging level // void set_level(level::level_enum log_level); +// +// Set global flush level +// +void flush_on(level::level_enum log_level); + +// +// Set global error handler +// +void set_error_handler(log_err_handler handler); + // // Turn on async mode (off by default) and set the queue size for each async_logger. // effective only for loggers created after this call. @@ -49,7 +53,7 @@ void set_level(level::level_enum log_level); // // async_overflow_policy (optional, block_retry by default): // async_overflow_policy::block_retry - if queue is full, block until queue has room for the new log entry. -// async_overflow_policy::discard_log_msg - never block and discard any new messages when queue overflows. +// async_overflow_policy::discard_log_msg - never block and discard any new messages when queue overflows. // // worker_warmup_cb (optional): // callback function that will be called in worker thread upon start (can be used to init stuff like thread affinity) @@ -57,87 +61,139 @@ void set_level(level::level_enum log_level); // worker_teardown_cb (optional): // callback function that will be called in worker thread upon exit // -void set_async_mode(size_t queue_size, const async_overflow_policy overflow_policy = async_overflow_policy::block_retry, const std::function<void()>& worker_warmup_cb = nullptr, const std::chrono::milliseconds& flush_interval_ms = std::chrono::milliseconds::zero(), const std::function<void()>& worker_teardown_cb = nullptr); +void set_async_mode(size_t queue_size, const async_overflow_policy overflow_policy = async_overflow_policy::block_retry, + const std::function<void()> &worker_warmup_cb = nullptr, + const std::chrono::milliseconds &flush_interval_ms = std::chrono::milliseconds::zero(), + const std::function<void()> &worker_teardown_cb = nullptr); // Turn off async mode void set_sync_mode(); +// +// Create and register multi/single threaded basic file logger. +// Basic logger simply writes to given file without any limitations or rotations. +// +std::shared_ptr<logger> basic_logger_mt(const std::string &logger_name, const filename_t &filename, bool truncate = false); +std::shared_ptr<logger> basic_logger_st(const std::string &logger_name, const filename_t &filename, bool truncate = false); + // // Create and register multi/single threaded rotating file logger // -std::shared_ptr<logger> rotating_logger_mt(const std::string& logger_name, const filename_t& filename, size_t max_file_size, size_t max_files, bool force_flush = false); -std::shared_ptr<logger> rotating_logger_st(const std::string& logger_name, const filename_t& filename, size_t max_file_size, size_t max_files, bool force_flush = false); +std::shared_ptr<logger> rotating_logger_mt( + const std::string &logger_name, const filename_t &filename, size_t max_file_size, size_t max_files); +std::shared_ptr<logger> rotating_logger_st( + const std::string &logger_name, const filename_t &filename, size_t max_file_size, size_t max_files); // -// Create file logger which creates new file on the given time (default in midnight): +// Create file logger which creates new file on the given time (default in midnight): // -std::shared_ptr<logger> daily_logger_mt(const std::string& logger_name, const filename_t& filename, int hour=0, int minute=0, bool force_flush = false); -std::shared_ptr<logger> daily_logger_st(const std::string& logger_name, const filename_t& filename, int hour=0, int minute=0, bool force_flush = false); +std::shared_ptr<logger> daily_logger_mt(const std::string &logger_name, const filename_t &filename, int hour = 0, int minute = 0); +std::shared_ptr<logger> daily_logger_st(const std::string &logger_name, const filename_t &filename, int hour = 0, int minute = 0); // // Create and register stdout/stderr loggers // -std::shared_ptr<logger> stdout_logger_mt(const std::string& logger_name, bool color = false); -std::shared_ptr<logger> stdout_logger_st(const std::string& logger_name, bool color = false); -std::shared_ptr<logger> stderr_logger_mt(const std::string& logger_name, bool color = false); -std::shared_ptr<logger> stderr_logger_st(const std::string& logger_name, bool color = false); - +std::shared_ptr<logger> stdout_logger_mt(const std::string &logger_name); +std::shared_ptr<logger> stdout_logger_st(const std::string &logger_name); +std::shared_ptr<logger> stderr_logger_mt(const std::string &logger_name); +std::shared_ptr<logger> stderr_logger_st(const std::string &logger_name); +// +// Create and register colored stdout/stderr loggers +// +std::shared_ptr<logger> stdout_color_mt(const std::string &logger_name); +std::shared_ptr<logger> stdout_color_st(const std::string &logger_name); +std::shared_ptr<logger> stderr_color_mt(const std::string &logger_name); +std::shared_ptr<logger> stderr_color_st(const std::string &logger_name); // // Create and register a syslog logger // -#if defined(__linux__) || defined(__APPLE__) -std::shared_ptr<logger> syslog_logger(const std::string& logger_name, const std::string& ident = "", int syslog_option = 0); +#ifdef SPDLOG_ENABLE_SYSLOG +std::shared_ptr<logger> syslog_logger( + const std::string &logger_name, const std::string &ident = "", int syslog_option = 0, int syslog_facilty = (1 << 3)); +#endif + +#if defined(__ANDROID__) +std::shared_ptr<logger> android_logger(const std::string &logger_name, const std::string &tag = "spdlog"); #endif +// Create and register a logger with a single sink +std::shared_ptr<logger> create(const std::string &logger_name, const sink_ptr &sink); // Create and register a logger with multiple sinks -std::shared_ptr<logger> create(const std::string& logger_name, sinks_init_list sinks); -template<class It> -std::shared_ptr<logger> create(const std::string& logger_name, const It& sinks_begin, const It& sinks_end); - +std::shared_ptr<logger> create(const std::string &logger_name, sinks_init_list sinks); +template <class It> std::shared_ptr<logger> create(const std::string &logger_name, const It &sinks_begin, const It &sinks_end); // Create and register a logger with templated sink type -// Example: spdlog::create<daily_file_sink_st>("mylog", "dailylog_filename", "txt"); -template <typename Sink, typename... Args> -std::shared_ptr<spdlog::logger> create(const std::string& logger_name, Args...); - +// Example: +// spdlog::create<daily_file_sink_st>("mylog", "dailylog_filename"); +template <typename Sink, typename... Args> std::shared_ptr<spdlog::logger> create(const std::string &logger_name, Args... args); + +// Create and register an async logger with a single sink +std::shared_ptr<logger> create_async(const std::string &logger_name, const sink_ptr &sink, size_t queue_size, + const async_overflow_policy overflow_policy = async_overflow_policy::block_retry, + const std::function<void()> &worker_warmup_cb = nullptr, + const std::chrono::milliseconds &flush_interval_ms = std::chrono::milliseconds::zero(), + const std::function<void()> &worker_teardown_cb = nullptr); + +// Create and register an async logger with multiple sinks +std::shared_ptr<logger> create_async(const std::string &logger_name, sinks_init_list sinks, size_t queue_size, + const async_overflow_policy overflow_policy = async_overflow_policy::block_retry, + const std::function<void()> &worker_warmup_cb = nullptr, + const std::chrono::milliseconds &flush_interval_ms = std::chrono::milliseconds::zero(), + const std::function<void()> &worker_teardown_cb = nullptr); +template <class It> +std::shared_ptr<logger> create_async(const std::string &logger_name, const It &sinks_begin, const It &sinks_end, size_t queue_size, + const async_overflow_policy overflow_policy = async_overflow_policy::block_retry, + const std::function<void()> &worker_warmup_cb = nullptr, + const std::chrono::milliseconds &flush_interval_ms = std::chrono::milliseconds::zero(), + const std::function<void()> &worker_teardown_cb = nullptr); // Register the given logger with the given name void register_logger(std::shared_ptr<logger> logger); +// Apply a user defined function on all registered loggers +// Example: +// spdlog::apply_all([&](std::shared_ptr<spdlog::logger> l) {l->flush();}); +void apply_all(std::function<void(std::shared_ptr<logger>)> fun); + // Drop the reference to the given logger void drop(const std::string &name); -// Drop all references +// Drop all references from the registry void drop_all(); - /////////////////////////////////////////////////////////////////////////////// // -// Macros to be display source file & line // Trace & Debug can be switched on/off at compile time for zero cost debug statements. -// Uncomment SPDLOG_DEBUG_ON/SPDLOG_TRACE_ON in teakme.h to enable. +// Uncomment SPDLOG_DEBUG_ON/SPDLOG_TRACE_ON in tweakme.h to enable. +// SPDLOG_TRACE(..) will also print current file and line. // // Example: -// spdlog::set_level(spdlog::level::debug); -// SPDLOG_DEBUG(my_logger, "Some debug message {} {}", 1, 3.2); +// spdlog::set_level(spdlog::level::trace); +// SPDLOG_TRACE(my_logger, "some trace message"); +// SPDLOG_TRACE(my_logger, "another trace message {} {}", 1, 2); +// SPDLOG_DEBUG(my_logger, "some debug message {} {}", 3, 4); /////////////////////////////////////////////////////////////////////////////// #ifdef SPDLOG_TRACE_ON -#define SPDLOG_TRACE(logger, ...) logger->trace(__VA_ARGS__) << " (" << __FILE__ << " #" << __LINE__ <<")"; +#define SPDLOG_STR_H(x) #x +#define SPDLOG_STR_HELPER(x) SPDLOG_STR_H(x) +#ifdef _MSC_VER +#define SPDLOG_TRACE(logger, ...) logger->trace("[ " __FILE__ "(" SPDLOG_STR_HELPER(__LINE__) ") ] " __VA_ARGS__) #else -#define SPDLOG_TRACE(logger, ...) +#define SPDLOG_TRACE(logger, ...) logger->trace("[ " __FILE__ ":" SPDLOG_STR_HELPER(__LINE__) " ] " __VA_ARGS__) +#endif +#else +#define SPDLOG_TRACE(logger, ...) (void)0 #endif #ifdef SPDLOG_DEBUG_ON -#define SPDLOG_DEBUG(logger, ...) logger->debug(__VA_ARGS__) << " (" << __FILE__ << " #" << __LINE__ <<")"; +#define SPDLOG_DEBUG(logger, ...) logger->debug(__VA_ARGS__) #else -#define SPDLOG_DEBUG(logger, ...) +#define SPDLOG_DEBUG(logger, ...) (void)0 #endif +} // namespace spdlog -} - - -#include <spdlog/details/spdlog_impl.h> +#include "details/spdlog_impl.h" diff --git a/src/spdlog/tweakme.h b/src/spdlog/tweakme.h index 2d6f606c9db11a49ea0a15f48421a3c4f9b73796..50ad3fb46c7f187a0838f98c9f807aac07cc985a 100644 --- a/src/spdlog/tweakme.h +++ b/src/spdlog/tweakme.h @@ -5,66 +5,139 @@ #pragma once +/////////////////////////////////////////////////////////////////////////////// // -// Edit this file to squeeze every last drop of performance out of spdlog. +// Edit this file to squeeze more performance, and to customize supported features // +/////////////////////////////////////////////////////////////////////////////// /////////////////////////////////////////////////////////////////////////////// // Under Linux, the much faster CLOCK_REALTIME_COARSE clock can be used. // This clock is less accurate - can be off by dozens of millis - depending on the kernel HZ. -// Uncomment to use it instead of the regular (but slower) clock. +// Uncomment to use it instead of the regular clock. +// // #define SPDLOG_CLOCK_COARSE /////////////////////////////////////////////////////////////////////////////// - /////////////////////////////////////////////////////////////////////////////// -// Uncomment if date/time logging is not needed. -// This will prevent spdlog from quering the clock on each log call. +// Uncomment if date/time logging is not needed and never appear in the log pattern. +// This will prevent spdlog from querying the clock on each log call. +// +// WARNING: If the log pattern contains any date/time while this flag is on, the result is undefined. +// You must set new pattern(spdlog::set_pattern(..") without any date/time in it +// // #define SPDLOG_NO_DATETIME /////////////////////////////////////////////////////////////////////////////// - /////////////////////////////////////////////////////////////////////////////// // Uncomment if thread id logging is not needed (i.e. no %t in the log pattern). -// This will prevent spdlog from quering the thread id on each log call. +// This will prevent spdlog from querying the thread id on each log call. +// +// WARNING: If the log pattern contains thread id (i.e, %t) while this flag is on, the result is undefined. +// // #define SPDLOG_NO_THREAD_ID /////////////////////////////////////////////////////////////////////////////// +/////////////////////////////////////////////////////////////////////////////// +// Uncomment to prevent spdlog from caching thread ids in thread local storage. +// By default spdlog saves thread ids in tls to gain a few micros for each call. +// +// WARNING: if your program forks, UNCOMMENT this flag to prevent undefined thread ids in the children logs. +// +// #define SPDLOG_DISABLE_TID_CACHING +/////////////////////////////////////////////////////////////////////////////// /////////////////////////////////////////////////////////////////////////////// // Uncomment if logger name logging is not needed. -// This will prevent spdlog from copying the logger name on each log call. +// This will prevent spdlog from copying the logger name on each log call. +// // #define SPDLOG_NO_NAME /////////////////////////////////////////////////////////////////////////////// - /////////////////////////////////////////////////////////////////////////////// // Uncomment to enable the SPDLOG_DEBUG/SPDLOG_TRACE macros. +// // #define SPDLOG_DEBUG_ON // #define SPDLOG_TRACE_ON /////////////////////////////////////////////////////////////////////////////// - /////////////////////////////////////////////////////////////////////////////// // Uncomment to avoid locking in the registry operations (spdlog::get(), spdlog::drop() spdlog::register()). -// Use only if your code never modifes concurrently the registry. +// Use only if your code never modifies concurrently the registry. // Note that upon creating a logger the registry is modified by spdlog.. +// // #define SPDLOG_NO_REGISTRY_MUTEX /////////////////////////////////////////////////////////////////////////////// - /////////////////////////////////////////////////////////////////////////////// // Uncomment to avoid spdlog's usage of atomic log levels -// Use only if your code never modifies a logger's log levels concurrently. +// Use only if your code never modifies a logger's log levels concurrently by different threads. +// // #define SPDLOG_NO_ATOMIC_LEVELS /////////////////////////////////////////////////////////////////////////////// /////////////////////////////////////////////////////////////////////////////// // Uncomment to enable usage of wchar_t for file names on Windows. +// // #define SPDLOG_WCHAR_FILENAMES /////////////////////////////////////////////////////////////////////////////// /////////////////////////////////////////////////////////////////////////////// // Uncomment to override default eol ("\n" or "\r\n" under Linux/Windows) +// // #define SPDLOG_EOL ";-)\n" /////////////////////////////////////////////////////////////////////////////// + +/////////////////////////////////////////////////////////////////////////////// +// Uncomment to use your own copy of the fmt library instead of spdlog's copy. +// In this case spdlog will try to include <fmt/format.h> so set your -I flag accordingly. +// +// #define SPDLOG_FMT_EXTERNAL +/////////////////////////////////////////////////////////////////////////////// + +/////////////////////////////////////////////////////////////////////////////// +// Uncomment to use printf-style messages in your logs instead of the usual +// format-style used by default. +// +// #define SPDLOG_FMT_PRINTF +/////////////////////////////////////////////////////////////////////////////// + +/////////////////////////////////////////////////////////////////////////////// +// Uncomment to enable syslog (disabled by default) +// +// #define SPDLOG_ENABLE_SYSLOG +/////////////////////////////////////////////////////////////////////////////// + +/////////////////////////////////////////////////////////////////////////////// +// Uncomment to enable wchar_t support (convert to utf8) +// +// #define SPDLOG_WCHAR_TO_UTF8_SUPPORT +/////////////////////////////////////////////////////////////////////////////// + +/////////////////////////////////////////////////////////////////////////////// +// Uncomment to prevent child processes from inheriting log file descriptors +// +// #define SPDLOG_PREVENT_CHILD_FD +/////////////////////////////////////////////////////////////////////////////// + +/////////////////////////////////////////////////////////////////////////////// +// Uncomment if your compiler doesn't support the "final" keyword. +// The final keyword allows more optimizations in release +// mode with recent compilers. See GCC's documentation for -Wsuggest-final-types +// for instance. +// +// #define SPDLOG_NO_FINAL +/////////////////////////////////////////////////////////////////////////////// + +/////////////////////////////////////////////////////////////////////////////// +// Uncomment to enable message counting feature. +// Use the %i in the logger pattern to display log message sequence id. +// +// #define SPDLOG_ENABLE_MESSAGE_COUNTER +/////////////////////////////////////////////////////////////////////////////// + +/////////////////////////////////////////////////////////////////////////////// +// Uncomment to customize level names (e.g. "MT TRACE") +// +// #define SPDLOG_LEVEL_NAMES { "MY TRACE", "MY DEBUG", "MY INFO", "MY WARNING", "MY ERROR", "MY CRITICAL", "OFF" } +/////////////////////////////////////////////////////////////////////////////// diff --git a/tests/client/loomenv.py b/tests/client/loomenv.py index 698629a44fa5aaf0e59bf9282289d54437a4b1f8..4f7dde5c3ae7ac2e41f137d593706f38c2237a20 100644 --- a/tests/client/loomenv.py +++ b/tests/client/loomenv.py @@ -140,12 +140,12 @@ class LoomEnv(Env): self.check_stats() return self._client - def submit_and_gather(self, tasks, check=True): + def submit_and_gather(self, tasks, check=True, load=False): if isinstance(tasks, Task): - future = self.client.submit_one(tasks) + future = self.client.submit_one(tasks, load=load) return self.client.gather_one(future) else: - futures = self.client.submit(tasks) + futures = self.client.submit(tasks, load=load) return self.client.gather(futures) if check: self.check_final_state() diff --git a/tests/client/test_checkpoint.py b/tests/client/test_checkpoint.py new file mode 100644 index 0000000000000000000000000000000000000000..2c5d2c4c6986d55e4faaf30e836b45e3153c12e3 --- /dev/null +++ b/tests/client/test_checkpoint.py @@ -0,0 +1,54 @@ +from loomenv import loom_env, LOOM_TESTPROG, LOOM_TEST_BUILD_DIR # noqa +import loom.client.tasks as tasks # noqa + +import os + +loom_env # silence flake8 + + +def test_checkpoint_basic(loom_env): + loom_env.start(1) + t1 = tasks.const("abcd") + t2 = tasks.const("XYZ") + t3 = tasks.merge((t1, t2)) + path = os.path.join(LOOM_TEST_BUILD_DIR, "test.data") + t3.checkpoint_path = path + assert b"abcdXYZ" == loom_env.submit_and_gather(t3) + with open(path, "rb") as f: + assert f.read() == b"abcdXYZ" + assert not os.path.isfile(path + ".loom.tmp") + + +def test_checkpoint_load(loom_env): + loom_env.start(1) + + path1 = os.path.join(LOOM_TEST_BUILD_DIR, "f1.txt") + path2 = os.path.join(LOOM_TEST_BUILD_DIR, "f2.txt") + path3 = os.path.join(LOOM_TEST_BUILD_DIR, "f3.txt") + path4 = os.path.join(LOOM_TEST_BUILD_DIR, "f4.txt") + path5 = os.path.join(LOOM_TEST_BUILD_DIR, "nonexisting") + + for i, p in enumerate((path1, path2, path3, path4)): + with open(p, "w") as f: + f.write("[{}]".format(i + 1)) + + t1 = tasks.const("$t1$") + t1.checkpoint_path = path1 # This shoud load: [1] + + t2 = tasks.const("$t2$") + t2.checkpoint_path = path2 # This shoud load: [2] + + t3 = tasks.const("$t3$") + t4 = tasks.const("$t4$") + + x1 = tasks.merge((t1, t2, t3)) # [1][2]$t3$ + x2 = tasks.merge((t1, x1)) + x2.checkpoint_path = path3 # loaded as [3] + + x3 = tasks.merge((t4, t4)) + x3.checkpoint_path = path4 # loaded as [4] + + x4 = tasks.merge((x3, x1, x2, t1, t2, t3)) + x4.checkpoint_path = path5 + + assert loom_env.submit_and_gather(x4, load=True) == b'[4][1][2]$t3$[3][1][2]$t3$' \ No newline at end of file diff --git a/tests/cpp/test_scheduler.cpp b/tests/cpp/test_scheduler.cpp index be296c7ebc50f95a3d6190d1483bf17ff6413034..5e8000ad9680f4d1b78d9f4a386774a513f0163c 100644 --- a/tests/cpp/test_scheduler.cpp +++ b/tests/cpp/test_scheduler.cpp @@ -317,10 +317,16 @@ static std::vector<TaskNode*> nodes(ComputationState &s, std::vector<loom::base: return result; } +static void add_plan(ComputationState &s, const loom::pb::comm::Plan &plan) { + std::vector<TaskNode*> to_load; + s.add_plan(plan, false, to_load); + assert(to_load.empty()); +} + TEST_CASE("basic-plan", "[scheduling]") { Server server(NULL, 0); ComputationState s(server); - s.add_plan(make_simple_plan(server)); + add_plan(s, make_simple_plan(server)); auto w1 = simple_worker(server, "w1"); auto w2 = simple_worker(server, "w2"); @@ -388,7 +394,7 @@ TEST_CASE("basic-plan", "[scheduling]") { TEST_CASE("plan4", "[scheduling]") { Server server(NULL, 0); ComputationState s(server); - s.add_plan(make_plan4(server)); + add_plan(s, make_plan4(server)); SECTION("More narrow") { auto w1 = simple_worker(server, "w1", 1); @@ -473,7 +479,7 @@ TEST_CASE("plan4", "[scheduling]") { TEST_CASE("Plan2", "[scheduling]") { Server server(NULL, 0); ComputationState s(server); - s.add_plan(make_plan2(server)); + add_plan(s, make_plan2(server)); SECTION("Two simple workers") { auto w1 = simple_worker(server, "w1"); @@ -562,7 +568,7 @@ TEST_CASE("big-plan", "[scheduling]") { Server server(NULL, 0); ComputationState s(server); - s.add_plan(make_big_plan(server, BIG_PLAN_SIZE)); + add_plan(s, make_big_plan(server, BIG_PLAN_SIZE)); std::vector<WorkerConnection*> ws; ws.reserve(BIG_PLAN_WORKERS); @@ -593,7 +599,7 @@ TEST_CASE("big-simple-plan", "[scheduling]") { Server server(NULL, 0); ComputationState s(server); - s.add_plan(make_big_trivial_plan(server, BIG_PLAN_SIZE)); + add_plan(s, make_big_trivial_plan(server, BIG_PLAN_SIZE)); std::vector<WorkerConnection*> ws; ws.reserve(BIG_PLAN_WORKERS); @@ -620,7 +626,7 @@ TEST_CASE("big-simple-plan", "[scheduling]") { TEST_CASE("request-plan", "[scheduling]") { Server server(NULL, 0); ComputationState s(server); - s.add_plan(make_request_plan(server)); + add_plan(s, make_request_plan(server)); SECTION("0 cpus - include free tasks") { auto w1 = simple_worker(server, "w1", 0); @@ -693,7 +699,7 @@ TEST_CASE("request-plan", "[scheduling]") { TEST_CASE("continuation2", "[scheduling]") { Server server(NULL, 0); ComputationState s(server); - s.add_plan(make_plan2(server)); + add_plan(s, make_plan2(server)); /*SECTION("Stick together") { auto w1 = simple_worker(server, "w1", 2); @@ -719,7 +725,7 @@ TEST_CASE("continuation2", "[scheduling]") { TEST_CASE("continuation", "[scheduling]") { Server server(NULL, 0); ComputationState s(server); - s.add_plan(make_plan3(server)); + add_plan(s, make_plan3(server)); SECTION("Stick together - inputs dominant") { auto w1 = simple_worker(server, "w1", 2); @@ -821,7 +827,7 @@ TEST_CASE("benchmark1", "[benchmark][!hide]") { for (size_t n_workers = 10; n_workers < 600; n_workers *= 2) { Server server(NULL, 0); ComputationState s(server); - s.add_plan(plan); + add_plan(s, plan); std::vector<WorkerConnection*> ws; ws.reserve(n_workers); @@ -845,7 +851,6 @@ TEST_CASE("benchmark1", "[benchmark][!hide]") { } } - TEST_CASE("benchmark2", "[benchmark][!hide]") { using namespace std::chrono; const size_t CPUS = 24; @@ -857,7 +862,7 @@ TEST_CASE("benchmark2", "[benchmark][!hide]") { for (size_t n_workers = 10; n_workers <= 160; n_workers *= 2) { Server server(NULL, 0); ComputationState s(server); - s.add_plan(plan); + add_plan(s, plan); std::vector<WorkerConnection*> ws; ws.reserve(n_workers); for (size_t i = 0; i < n_workers; i++) { diff --git a/version b/version index eb49d7c7fdcbb1b4745de39837864aa7f78570ac..aec258df73d39d2122706793921981f4a0f672f8 100644 --- a/version +++ b/version @@ -1 +1 @@ -0.7 +0.8