2021-09-03 16:46:09 +00:00
|
|
|
#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>
|
2021-09-03 17:31:29 +00:00
|
|
|
#include <Parsers/ASTInsertQuery.h>
|
2021-09-03 16:46:09 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
static constexpr auto TIME_SCALE = 6;
|
|
|
|
|
|
|
|
NamesAndTypesList StorageSystemAsynchronousInserts::getNamesAndTypes()
|
|
|
|
{
|
|
|
|
return
|
|
|
|
{
|
|
|
|
{"query", std::make_shared<DataTypeString>()},
|
2021-09-03 17:31:29 +00:00
|
|
|
{"database", std::make_shared<DataTypeString>()},
|
|
|
|
{"table", std::make_shared<DataTypeString>()},
|
|
|
|
{"format", std::make_shared<DataTypeString>()},
|
2021-09-03 16:46:09 +00:00
|
|
|
{"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;
|
|
|
|
|
Fix race between INSERT async_insert=1 and system.asynchronous_inserts
CI report [1]:
[c190f600f8c6] 2022.03.02 01:07:34.553012 [ 23552 ] {76b6113b-1479-46c9-90ab-e78a3c9f3dbb} executeQuery: Code: 60. DB::Exception: Both table name and UUID are empty. (UNKNOWN_TABLE) (version 22.3.1.1) (from [::1]:42040) (comment: '02015_async_inserts_stress_long.sh') (in query: SELECT * FROM system.asynchronous_inserts FORMAT Null), Stack trace (when copying this message, always include the lines below):
0. ClickHouse/contrib/libcxx/include/exception:133: Poco::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int) @ 0xf50e04c in /fasttest-workspace/build/programs/clickhouse
1. ClickHouse/src/Common/Exception.cpp:58: DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int, bool) @ 0x663ebfa in /fasttest-workspace/build/programs/clickhouse
2. DB::StorageID::assertNotEmpty() const @ 0xbc08591 in /fasttest-workspace/build/programs/clickhouse
3. ClickHouse/contrib/libcxx/include/string:1444: DB::StorageID::getDatabaseName() const @ 0xe50d2b6 in /fasttest-workspace/build/programs/clickhouse
4. ClickHouse/contrib/libcxx/include/string:1957: DB::StorageSystemAsynchronousInserts::fillData(std::__1::vector::mutable_ptr, std::__1::allocator::mutable_ptr > >&, std::__1::shared_ptr, DB::SelectQueryInfo const&) const @ 0xdac636c in /fasttest-workspace/build/programs/clickhouse
[1]: https://s3.amazonaws.com/clickhouse-test-reports/34973/e6fc6a22d5c018961c18247242dd3a40b8c54ff2/fast_test__actions_.html
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-03-02 12:14:56 +00:00
|
|
|
auto [queue, queue_lock] = insert_queue->getQueueLocked();
|
2021-09-03 16:46:09 +00:00
|
|
|
for (const auto & [key, elem] : queue)
|
|
|
|
{
|
Fix race between INSERT async_insert=1 and system.asynchronous_inserts
CI report [1]:
[c190f600f8c6] 2022.03.02 01:07:34.553012 [ 23552 ] {76b6113b-1479-46c9-90ab-e78a3c9f3dbb} executeQuery: Code: 60. DB::Exception: Both table name and UUID are empty. (UNKNOWN_TABLE) (version 22.3.1.1) (from [::1]:42040) (comment: '02015_async_inserts_stress_long.sh') (in query: SELECT * FROM system.asynchronous_inserts FORMAT Null), Stack trace (when copying this message, always include the lines below):
0. ClickHouse/contrib/libcxx/include/exception:133: Poco::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int) @ 0xf50e04c in /fasttest-workspace/build/programs/clickhouse
1. ClickHouse/src/Common/Exception.cpp:58: DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int, bool) @ 0x663ebfa in /fasttest-workspace/build/programs/clickhouse
2. DB::StorageID::assertNotEmpty() const @ 0xbc08591 in /fasttest-workspace/build/programs/clickhouse
3. ClickHouse/contrib/libcxx/include/string:1444: DB::StorageID::getDatabaseName() const @ 0xe50d2b6 in /fasttest-workspace/build/programs/clickhouse
4. ClickHouse/contrib/libcxx/include/string:1957: DB::StorageSystemAsynchronousInserts::fillData(std::__1::vector::mutable_ptr, std::__1::allocator::mutable_ptr > >&, std::__1::shared_ptr, DB::SelectQueryInfo const&) const @ 0xdac636c in /fasttest-workspace/build/programs/clickhouse
[1]: https://s3.amazonaws.com/clickhouse-test-reports/34973/e6fc6a22d5c018961c18247242dd3a40b8c54ff2/fast_test__actions_.html
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-03-02 12:14:56 +00:00
|
|
|
std::lock_guard elem_lock(elem->mutex);
|
2021-09-03 16:46:09 +00:00
|
|
|
|
|
|
|
if (!elem->data)
|
|
|
|
continue;
|
|
|
|
|
|
|
|
auto time_in_microseconds = [](const time_point<steady_clock> & timestamp)
|
|
|
|
{
|
|
|
|
auto time_diff = duration_cast<microseconds>(steady_clock::now() - timestamp);
|
2021-09-04 01:11:37 +00:00
|
|
|
auto time_us = (system_clock::now() - time_diff).time_since_epoch().count();
|
2021-09-03 16:46:09 +00:00
|
|
|
|
2021-09-04 01:11:37 +00:00
|
|
|
DecimalUtils::DecimalComponents<DateTime64> components{time_us / 1'000'000, time_us % 1'000'000};
|
|
|
|
return DecimalField(DecimalUtils::decimalFromComponents<DateTime64>(components, TIME_SCALE), TIME_SCALE);
|
|
|
|
};
|
2021-09-03 16:46:09 +00:00
|
|
|
|
2021-09-03 17:31:29 +00:00
|
|
|
const auto & insert_query = key.query->as<const ASTInsertQuery &>();
|
2021-09-04 01:11:37 +00:00
|
|
|
size_t i = 0;
|
2021-09-03 17:31:29 +00:00
|
|
|
|
|
|
|
res_columns[i++]->insert(queryToString(insert_query));
|
|
|
|
res_columns[i++]->insert(insert_query.table_id.getDatabaseName());
|
|
|
|
res_columns[i++]->insert(insert_query.table_id.getTableName());
|
|
|
|
res_columns[i++]->insert(insert_query.format);
|
2021-09-03 16:46:09 +00:00
|
|
|
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());
|
|
|
|
|
2021-09-04 01:11:37 +00:00
|
|
|
if (auto exception = entry->getException())
|
2021-09-03 16:46:09 +00:00
|
|
|
{
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|