add system table asynchronous_inserts

This commit is contained in:
Anton Popov 2021-09-03 19:46:09 +03:00
parent c9a0cc5ce9
commit 7d57aba4f9
4 changed files with 134 additions and 4 deletions

View File

@ -91,11 +91,11 @@ private:
using Queue = std::unordered_map<InsertQuery, std::shared_ptr<Container>, InsertQuery::Hash>; using Queue = std::unordered_map<InsertQuery, std::shared_ptr<Container>, InsertQuery::Hash>;
using QueueIterator = Queue::iterator; using QueueIterator = Queue::iterator;
std::shared_mutex rwlock; mutable std::shared_mutex rwlock;
Queue queue; Queue queue;
using QueryIdToEntry = std::unordered_map<String, InsertData::EntryPtr>; using QueryIdToEntry = std::unordered_map<String, InsertData::EntryPtr>;
std::mutex currently_processing_mutex; mutable std::mutex currently_processing_mutex;
QueryIdToEntry currently_processing_queries; QueryIdToEntry currently_processing_queries;
/// Logic and events behind queue are as follows: /// Logic and events behind queue are as follows:
@ -128,8 +128,11 @@ private:
static void processData(InsertQuery key, InsertDataPtr data, ContextPtr global_context); static void processData(InsertQuery key, InsertDataPtr data, ContextPtr global_context);
public: public:
Queue getQueue() const { return queue; } Queue getQueue() const
QueryIdToEntry getCurrentlyProcessingQueries() const { return currently_processing_queries; } {
std::shared_lock lock(rwlock);
return queue;
}
}; };
} }

View File

@ -0,0 +1,99 @@
#include <Storages/System/StorageSystemAsynchronousInserts.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/AsynchronousInsertQueue.h>
#include <Interpreters/Context.h>
#include <Parsers/queryToString.h>
#include <Core/DecimalFunctions.h>
namespace DB
{
static constexpr auto TIME_SCALE = 6;
NamesAndTypesList StorageSystemAsynchronousInserts::getNamesAndTypes()
{
return
{
{"query", std::make_shared<DataTypeString>()},
{"first_update", std::make_shared<DataTypeDateTime64>(TIME_SCALE)},
{"last_update", std::make_shared<DataTypeDateTime64>(TIME_SCALE)},
{"total_bytes", std::make_shared<DataTypeUInt64>()},
{"entries.query_id", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{"entries.bytes", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())},
{"entries.finished", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt8>())},
{"entries.exception", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
};
}
void StorageSystemAsynchronousInserts::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
using namespace std::chrono;
auto * insert_queue = context->getAsynchronousInsertQueue();
if (!insert_queue)
return;
auto queue = insert_queue->getQueue();
for (const auto & [key, elem] : queue)
{
std::lock_guard lock(elem->mutex);
if (!elem->data)
continue;
auto time_in_microseconds = [](const time_point<steady_clock> & timestamp)
{
auto time_diff = duration_cast<microseconds>(steady_clock::now() - timestamp);
return (system_clock::now() - time_diff).time_since_epoch().count();
};
size_t i = 0;
res_columns[i++]->insert(queryToString(key.query));
res_columns[i++]->insert(time_in_microseconds(elem->data->first_update));
res_columns[i++]->insert(time_in_microseconds(elem->data->last_update));
res_columns[i++]->insert(elem->data->size);
Array arr_query_id;
Array arr_bytes;
Array arr_finished;
Array arr_exception;
for (const auto & entry : elem->data->entries)
{
arr_query_id.push_back(entry->query_id);
arr_bytes.push_back(entry->bytes.size());
arr_finished.push_back(entry->isFinished());
auto exception = entry->getException();
if (exception)
{
try
{
std::rethrow_exception(exception);
}
catch (const Exception & e)
{
arr_exception.push_back(e.displayText());
}
catch (...)
{
arr_exception.push_back("Unknown exception");
}
}
else
arr_exception.push_back("");
}
res_columns[i++]->insert(arr_query_id);
res_columns[i++]->insert(arr_bytes);
res_columns[i++]->insert(arr_finished);
res_columns[i++]->insert(arr_exception);
}
}
}

View File

@ -0,0 +1,26 @@
#pragma once
#include <common/shared_ptr_helper.h>
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
{
/** Implements the system table `asynhronous_inserts`,
* which contains information about pending asynchronous inserts in queue.
*/
class StorageSystemAsynchronousInserts final :
public shared_ptr_helper<StorageSystemAsynchronousInserts>,
public IStorageSystemOneBlock<StorageSystemAsynchronousInserts>
{
public:
std::string getName() const override { return "AsynchronousInserts"; }
static NamesAndTypesList getNamesAndTypes();
protected:
friend struct shared_ptr_helper<StorageSystemAsynchronousInserts>;
using IStorageSystemOneBlock::IStorageSystemOneBlock;
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override;
};
}

View File

@ -74,6 +74,7 @@
#include <Storages/System/StorageSystemQuotasUsage.h> #include <Storages/System/StorageSystemQuotasUsage.h>
#include <Storages/System/StorageSystemUserDirectories.h> #include <Storages/System/StorageSystemUserDirectories.h>
#include <Storages/System/StorageSystemPrivileges.h> #include <Storages/System/StorageSystemPrivileges.h>
#include <Storages/System/StorageSystemAsynchronousInserts.h>
#ifdef OS_LINUX #ifdef OS_LINUX
#include <Storages/System/StorageSystemStackTrace.h> #include <Storages/System/StorageSystemStackTrace.h>
@ -167,6 +168,7 @@ void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper)
attach<StorageSystemMacros>(system_database, "macros"); attach<StorageSystemMacros>(system_database, "macros");
attach<StorageSystemReplicatedFetches>(system_database, "replicated_fetches"); attach<StorageSystemReplicatedFetches>(system_database, "replicated_fetches");
attach<StorageSystemPartMovesBetweenShards>(system_database, "part_moves_between_shards"); attach<StorageSystemPartMovesBetweenShards>(system_database, "part_moves_between_shards");
attach<StorageSystemAsynchronousInserts>(system_database, "asynchronous_inserts");
if (has_zookeeper) if (has_zookeeper)
attach<StorageSystemZooKeeper>(system_database, "zookeeper"); attach<StorageSystemZooKeeper>(system_database, "zookeeper");