mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Better
This commit is contained in:
parent
056b95d7e5
commit
b899bd07cf
@ -4,30 +4,28 @@
|
||||
|
||||
#include <Coordination/CoordinationSettings.h>
|
||||
#include <Coordination/KeeperContext.h>
|
||||
#include <Coordination/KeeperSnapshotManager.h>
|
||||
#include <Coordination/KeeperStorage.h>
|
||||
#include "Common/ConcurrentBoundedQueue.h"
|
||||
#include "Common/Exception.h"
|
||||
#include "Common/ZooKeeper/IKeeper.h"
|
||||
#include "Common/ZooKeeper/ZooKeeperArgs.h"
|
||||
#include "Common/ZooKeeper/ZooKeeperCommon.h"
|
||||
#include "Common/ZooKeeper/ZooKeeperConstants.h"
|
||||
#include <Common/EventNotifier.h>
|
||||
#include <Common/Config/ConfigProcessor.h>
|
||||
#include "Coordination/KeeperSnapshotManager.h"
|
||||
#include "Core/ColumnWithTypeAndName.h"
|
||||
#include "Core/ColumnsWithTypeAndName.h"
|
||||
#include <Core/ColumnWithTypeAndName.h>
|
||||
#include <Core/ColumnsWithTypeAndName.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include "IO/ReadBuffer.h"
|
||||
#include "IO/ReadBufferFromFile.h"
|
||||
#include "base/Decimal.h"
|
||||
#include "base/types.h"
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Formats/registerFormats.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/copyData.h>
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Formats/registerFormats.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
#include <Common/Config/ConfigProcessor.h>
|
||||
#include <Common/EventNotifier.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ZooKeeper/IKeeper.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperArgs.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperCommon.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperConstants.h>
|
||||
|
||||
|
||||
namespace CurrentMetrics
|
||||
@ -884,6 +882,7 @@ struct SetupNodeCollector
|
||||
if (initial_storage->container.contains(path))
|
||||
return;
|
||||
|
||||
new_nodes = true;
|
||||
std::cerr << "Adding expected node " << path << std::endl;
|
||||
|
||||
Coordination::Requests create_ops;
|
||||
@ -923,11 +922,19 @@ struct SetupNodeCollector
|
||||
|
||||
void generateSnapshot()
|
||||
{
|
||||
std::cerr << "Generating snapshot with starting data" << std::endl;
|
||||
std::lock_guard lock(nodes_mutex);
|
||||
if (!new_nodes)
|
||||
{
|
||||
std::cerr << "No new nodes added" << std::endl;
|
||||
return;
|
||||
}
|
||||
|
||||
std::cerr << "Generating snapshot with starting data" << std::endl;
|
||||
DB::SnapshotMetadataPtr snapshot_meta = std::make_shared<DB::SnapshotMetadata>(initial_storage->getZXID(), 1, std::make_shared<nuraft::cluster_config>());
|
||||
DB::KeeperStorageSnapshot snapshot(initial_storage.get(), snapshot_meta);
|
||||
snapshot_manager->serializeSnapshotToDisk(snapshot);
|
||||
|
||||
new_nodes = false;
|
||||
}
|
||||
|
||||
std::mutex nodes_mutex;
|
||||
@ -935,6 +942,7 @@ struct SetupNodeCollector
|
||||
Coordination::KeeperStoragePtr initial_storage;
|
||||
std::unordered_set<std::string> nodes_created_during_replay;
|
||||
std::optional<Coordination::KeeperSnapshotManager> snapshot_manager;
|
||||
bool new_nodes = false;
|
||||
};
|
||||
|
||||
void dumpStats(std::string_view type, const RequestFromLogStats::Stats & stats_for_type)
|
||||
@ -972,23 +980,25 @@ void requestFromLogExecutor(std::shared_ptr<ConcurrentBoundedQueue<RequestFromLo
|
||||
if (*expected_result != response.error)
|
||||
stats.unexpected_results.fetch_add(1, std::memory_order_relaxed);
|
||||
|
||||
//if (*expected_result != response.error)
|
||||
//{
|
||||
// std::cerr << fmt::format(
|
||||
// "Unexpected result for {}\ngot {}, expected {}\n", request->toString(), response.error, *expected_result)
|
||||
// << std::endl;
|
||||
#if 0
|
||||
if (*expected_result != response.error)
|
||||
{
|
||||
std::cerr << fmt::format(
|
||||
"Unexpected result for {}\ngot {}, expected {}\n", request->toString(), response.error, *expected_result)
|
||||
<< std::endl;
|
||||
|
||||
// if (const auto * multi_response = dynamic_cast<const Coordination::ZooKeeperMultiResponse *>(&response))
|
||||
// {
|
||||
// std::string subresponses;
|
||||
// for (size_t i = 0; i < multi_response->responses.size(); ++i)
|
||||
// {
|
||||
// subresponses += fmt::format("{} = {}\n", i, multi_response->responses[i]->error);
|
||||
// }
|
||||
if (const auto * multi_response = dynamic_cast<const Coordination::ZooKeeperMultiResponse *>(&response))
|
||||
{
|
||||
std::string subresponses;
|
||||
for (size_t i = 0; i < multi_response->responses.size(); ++i)
|
||||
{
|
||||
subresponses += fmt::format("{} = {}\n", i, multi_response->responses[i]->error);
|
||||
}
|
||||
|
||||
// std::cerr << "Subresponses\n" << subresponses << std::endl;
|
||||
// }
|
||||
//}
|
||||
std::cerr << "Subresponses\n" << subresponses << std::endl;
|
||||
}
|
||||
}
|
||||
#endif
|
||||
}
|
||||
|
||||
request_promise->set_value();
|
||||
@ -1048,8 +1058,16 @@ void Runner::runBenchmarkFromLog()
|
||||
|
||||
pool->wait();
|
||||
|
||||
dumpStats("Write", stats.write_requests);
|
||||
dumpStats("Read", stats.read_requests);
|
||||
|
||||
if (setup_nodes_collector)
|
||||
{
|
||||
setup_nodes_collector->generateSnapshot();
|
||||
}
|
||||
else
|
||||
{
|
||||
dumpStats("Write", stats.write_requests);
|
||||
dumpStats("Read", stats.read_requests);
|
||||
}
|
||||
});
|
||||
|
||||
auto push_request = [&](RequestFromLog request)
|
||||
|
Loading…
Reference in New Issue
Block a user