Merge pull request #60774 from canhld94/zookeeper_log_us

Change column name from duration_ms to duration_us in system.zookeeper
This commit is contained in:
Alexey Milovidov 2024-03-25 18:07:58 +03:00 committed by GitHub
commit dc6d713658
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 18 additions and 21 deletions

View File

@ -812,7 +812,7 @@ void ZooKeeper::receiveEvent()
RequestInfo request_info;
ZooKeeperResponsePtr response;
UInt64 elapsed_ms = 0;
UInt64 elapsed_microseconds = 0;
maybeInjectRecvFault();
@ -875,8 +875,8 @@ void ZooKeeper::receiveEvent()
CurrentMetrics::sub(CurrentMetrics::ZooKeeperRequest);
}
elapsed_ms = std::chrono::duration_cast<std::chrono::microseconds>(clock::now() - request_info.time).count();
ProfileEvents::increment(ProfileEvents::ZooKeeperWaitMicroseconds, elapsed_ms);
elapsed_microseconds = std::chrono::duration_cast<std::chrono::microseconds>(clock::now() - request_info.time).count();
ProfileEvents::increment(ProfileEvents::ZooKeeperWaitMicroseconds, elapsed_microseconds);
}
try
@ -935,7 +935,7 @@ void ZooKeeper::receiveEvent()
length, actual_length);
}
logOperationIfNeeded(request_info.request, response, /* finalize= */ false, elapsed_ms);
logOperationIfNeeded(request_info.request, response, /* finalize= */ false, elapsed_microseconds);
}
catch (...)
{
@ -954,7 +954,7 @@ void ZooKeeper::receiveEvent()
if (request_info.callback)
request_info.callback(*response);
logOperationIfNeeded(request_info.request, response, /* finalize= */ false, elapsed_ms);
logOperationIfNeeded(request_info.request, response, /* finalize= */ false, elapsed_microseconds);
}
catch (...)
{
@ -1048,14 +1048,14 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea
? Error::ZCONNECTIONLOSS
: Error::ZSESSIONEXPIRED;
response->xid = request_info.request->xid;
UInt64 elapsed_ms = std::chrono::duration_cast<std::chrono::microseconds>(clock::now() - request_info.time).count();
UInt64 elapsed_microseconds = std::chrono::duration_cast<std::chrono::microseconds>(clock::now() - request_info.time).count();
if (request_info.callback)
{
try
{
request_info.callback(*response);
logOperationIfNeeded(request_info.request, response, true, elapsed_ms);
logOperationIfNeeded(request_info.request, response, true, elapsed_microseconds);
}
catch (...)
{
@ -1115,8 +1115,8 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea
try
{
info.callback(*response);
UInt64 elapsed_ms = std::chrono::duration_cast<std::chrono::microseconds>(clock::now() - info.time).count();
logOperationIfNeeded(info.request, response, true, elapsed_ms);
UInt64 elapsed_microseconds = std::chrono::duration_cast<std::chrono::microseconds>(clock::now() - info.time).count();
logOperationIfNeeded(info.request, response, true, elapsed_microseconds);
}
catch (...)
{
@ -1498,7 +1498,7 @@ void ZooKeeper::setZooKeeperLog(std::shared_ptr<DB::ZooKeeperLog> zk_log_)
}
#ifdef ZOOKEEPER_LOG
void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr & request, const ZooKeeperResponsePtr & response, bool finalize, UInt64 elapsed_ms)
void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr & request, const ZooKeeperResponsePtr & response, bool finalize, UInt64 elapsed_microseconds)
{
auto maybe_zk_log = std::atomic_load(&zk_log);
if (!maybe_zk_log)
@ -1536,7 +1536,7 @@ void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr & request, const
elem.event_time = event_time;
elem.address = socket_address;
elem.session_id = session_id;
elem.duration_ms = elapsed_ms;
elem.duration_microseconds = elapsed_microseconds;
if (request)
{
elem.thread_id = request->thread_id;

View File

@ -343,7 +343,7 @@ private:
void flushWriteBuffer();
ReadBuffer & getReadBuffer();
void logOperationIfNeeded(const ZooKeeperRequestPtr & request, const ZooKeeperResponsePtr & response = nullptr, bool finalize = false, UInt64 elapsed_ms = 0);
void logOperationIfNeeded(const ZooKeeperRequestPtr & request, const ZooKeeperResponsePtr & response = nullptr, bool finalize = false, UInt64 elapsed_microseconds = 0);
void initFeatureFlags();

View File

@ -13,7 +13,6 @@
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h>
#include <Parsers/FunctionParameterValuesVisitor.h>
#include <Access/Common/AccessFlags.h>
#include <Access/ContextAccess.h>
@ -73,7 +72,6 @@
#include <Processors/Transforms/FilterTransform.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
#include <Storages/StorageDistributed.h>
#include <Storages/StorageDummy.h>
@ -85,7 +83,6 @@
#include <Core/ColumnNumbers.h>
#include <Core/Field.h>
#include <Core/ProtocolDefines.h>
#include <Functions/IFunction.h>
#include <Interpreters/Aggregator.h>
#include <Interpreters/IJoin.h>
#include <QueryPipeline/SizeLimits.h>

View File

@ -131,7 +131,7 @@ ColumnsDescription ZooKeeperLogElement::getColumnsDescription()
{"address", DataTypeFactory::instance().get("IPv6"), "IP address of ZooKeeper server that was used to make the request."},
{"port", std::make_shared<DataTypeUInt16>(), "The port of ZooKeeper server that was used to make the request."},
{"session_id", std::make_shared<DataTypeInt64>(), "The session ID that the ZooKeeper server sets for each connection."},
{"duration_ms", std::make_shared<DataTypeUInt64>(), "The time taken by ZooKeeper to execute the request."},
{"duration_microseconds", std::make_shared<DataTypeUInt64>(), "The time taken by ZooKeeper to execute the request."},
{"xid", std::make_shared<DataTypeInt32>(), "The ID of the request within the session. This is usually a sequential request number. It is the same for the request row and the paired response/finalize row."},
{"has_watch", std::make_shared<DataTypeUInt8>(), "The request whether the watch has been set."},
@ -183,7 +183,7 @@ void ZooKeeperLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insertData(IPv6ToBinary(address.host()).data(), 16);
columns[i++]->insert(address.port());
columns[i++]->insert(session_id);
columns[i++]->insert(duration_ms);
columns[i++]->insert(duration_microseconds);
columns[i++]->insert(xid);
columns[i++]->insert(has_watch);

View File

@ -28,7 +28,7 @@ struct ZooKeeperLogElement
Poco::Net::SocketAddress address;
Int64 session_id = 0;
UInt64 duration_ms = 0;
UInt64 duration_microseconds = 0;
/// Common request info
Int32 xid = 0;

View File

@ -34,5 +34,5 @@ Response 0 Error /test/01158/default/rmt/blocks/all_6308706741995381342_24957917
Response 0 Error /test/01158/default/rmt/block_numbers/all/block- 1 1 \N 0 2 ZRUNTIMEINCONSISTENCY \N \N 0 0 0 0
Request 0 Get /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 0 \N \N \N 0 0 0 0
Response 0 Get /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 0 ZOK \N \N 0 0 9 0
duration_ms
duration_microseconds
1

View File

@ -42,5 +42,5 @@ order by xid, type, request_idx;
drop table rmt sync;
system flush logs;
select 'duration_ms';
select count()>0 from system.zookeeper_log where path like '/test/01158/' || currentDatabase() || '/rmt%' and duration_ms > 0;
select 'duration_microseconds';
select count()>0 from system.zookeeper_log where path like '/test/01158/' || currentDatabase() || '/rmt%' and duration_microseconds > 0;