This commit is contained in:
Antonio Andelic 2024-05-22 16:12:33 +02:00
parent 056b95d7e5
commit b899bd07cf

View File

@ -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();
if (setup_nodes_collector)
{
setup_nodes_collector->generateSnapshot();
}
else
{
dumpStats("Write", stats.write_requests);
dumpStats("Read", stats.read_requests);
}
});
auto push_request = [&](RequestFromLog request)