Model-based testing of simulated full cluster ()

This PR adds support for generating randomized workloads that will be executed
against a simulated cluster, as well as against a correctness model. Initially
this just generates ScanAll and CreateVertex requests, and anything that it
creates, it also inserts into a `std::set`, and when we do a ScanAll, it asserts
that we get the same number of requests back. This will become much more
sophisticated over time, but it's already hitting pay-dirt.
This commit is contained in:
Tyler Neely 2022-10-24 19:54:09 +02:00 committed by GitHub
parent 12e7a261aa
commit acc655f4fd
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 613 additions and 40 deletions

View File

@ -172,14 +172,17 @@ struct ShardMap {
for (auto &aas : shard) {
if (initialized.contains(aas.address.unique_id)) {
spdlog::info("marking shard as full consensus participant: {}", aas.address.unique_id);
aas.status = Status::CONSENSUS_PARTICIPANT;
machine_contains_shard = true;
if (aas.status != Status::CONSENSUS_PARTICIPANT) {
spdlog::info("marking shard as full consensus participant: {}", aas.address.unique_id);
aas.status = Status::CONSENSUS_PARTICIPANT;
}
} else {
const bool same_machine = aas.address.last_known_ip == storage_manager.last_known_ip &&
aas.address.last_known_port == storage_manager.last_known_port;
if (same_machine) {
machine_contains_shard = true;
spdlog::info("reminding shard manager that they should begin participating in shard");
ret.push_back(ShardToInitialize{
.uuid = aas.address.unique_id,
.label_id = label_id,
@ -198,12 +201,16 @@ struct ShardMap {
// TODO(tyler) use deterministic UUID so that coordinators don't diverge here
address.unique_id = boost::uuids::uuid{boost::uuids::random_generator()()},
ret.push_back(ShardToInitialize{.uuid = address.unique_id,
.label_id = label_id,
.min_key = low_key,
.max_key = high_key,
.schema = schemas[label_id],
.config = Config{}});
spdlog::info("assigning shard manager to shard");
ret.push_back(ShardToInitialize{
.uuid = address.unique_id,
.label_id = label_id,
.min_key = low_key,
.max_key = high_key,
.schema = schemas[label_id],
.config = Config{},
});
AddressAndStatus aas = {
.address = address,
@ -398,6 +405,29 @@ struct ShardMap {
return ret;
}
/// Returns true if all shards have the desired number of replicas and they are in
/// the CONSENSUS_PARTICIPANT state. Note that this does not necessarily mean that
/// there is also an active leader for each shard.
bool ClusterInitialized() const {
for (const auto &[label_id, label_space] : label_spaces) {
for (const auto &[low_key, shard] : label_space.shards) {
if (shard.size() < label_space.replication_factor) {
spdlog::info("label_space below desired replication factor");
return false;
}
for (const auto &aas : shard) {
if (aas.status != Status::CONSENSUS_PARTICIPANT) {
spdlog::info("shard member not yet a CONSENSUS_PARTICIPANT");
return false;
}
}
}
}
return true;
}
};
} // namespace memgraph::coordinator

View File

@ -15,6 +15,7 @@
#include <fmt/format.h>
#include <boost/asio/ip/tcp.hpp>
#include <boost/functional/hash.hpp>
#include <boost/uuid/uuid.hpp>
#include <boost/uuid/uuid_generators.hpp>
#include <boost/uuid/uuid_io.hpp>
@ -78,6 +79,13 @@ struct Address {
};
}
PartialAddress ToPartialAddress() const {
return PartialAddress{
.ip = last_known_ip,
.port = last_known_port,
};
}
friend bool operator==(const Address &lhs, const Address &rhs) = default;
/// unique_id is most dominant for ordering, then last_known_ip, then last_known_port
@ -103,4 +111,30 @@ struct Address {
return in;
}
};
}; // namespace memgraph::io
namespace std {
template <>
struct hash<memgraph::io::PartialAddress> {
size_t operator()(const memgraph::io::PartialAddress &pa) const {
using boost::hash_combine;
using boost::hash_value;
// Start with a hash value of 0 .
std::size_t seed = 0;
if (pa.ip.is_v4()) {
auto h = std::hash<boost::asio::ip::address_v4>()(pa.ip.to_v4());
hash_combine(seed, h);
} else {
auto h = std::hash<boost::asio::ip::address_v6>()(pa.ip.to_v6());
hash_combine(seed, h);
}
hash_combine(seed, hash_value(pa.port));
// Return the result.
return seed;
}
};
} // namespace std

View File

@ -29,6 +29,8 @@ class Simulator {
explicit Simulator(SimulatorConfig config)
: rng_(std::mt19937{config.rng_seed}), simulator_handle_{std::make_shared<SimulatorHandle>(config)} {}
~Simulator() { ShutDown(); }
void ShutDown() { simulator_handle_->ShutDown(); }
Io<SimulatorTransport> RegisterNew() {

View File

@ -16,12 +16,10 @@
#include "io/simulator/simulator_stats.hpp"
#include "io/time.hpp"
#include "io/transport.hpp"
#include "utils/exceptions.hpp"
namespace memgraph::io::simulator {
using memgraph::io::Duration;
using memgraph::io::Time;
void SimulatorHandle::ShutDown() {
std::unique_lock<std::mutex> lock(mu_);
should_shut_down_ = true;
@ -76,9 +74,15 @@ bool SimulatorHandle::MaybeTickSimulator() {
const Duration clock_advance = std::chrono::microseconds{time_distrib_(rng_)};
cluster_wide_time_microseconds_ += clock_advance;
MG_ASSERT(cluster_wide_time_microseconds_ < config_.abort_time,
"Cluster has executed beyond its configured abort_time, and something may be failing to make progress "
"in an expected amount of time.");
if (cluster_wide_time_microseconds_ >= config_.abort_time) {
if (should_shut_down_) {
return false;
}
spdlog::error(
"Cluster has executed beyond its configured abort_time, and something may be failing to make progress "
"in an expected amount of time.");
throw utils::BasicException{"Cluster has executed beyond its configured abort_time"};
}
return true;
}
@ -121,7 +125,8 @@ bool SimulatorHandle::MaybeTickSimulator() {
// don't add it anywhere, let it drop
} else {
// add to can_receive_ if not
const auto &[om_vec, inserted] = can_receive_.try_emplace(to_address, std::vector<OpaqueMessage>());
const auto &[om_vec, inserted] =
can_receive_.try_emplace(to_address.ToPartialAddress(), std::vector<OpaqueMessage>());
om_vec->second.emplace_back(std::move(opaque_message));
}

View File

@ -43,7 +43,7 @@ class SimulatorHandle {
std::map<PromiseKey, DeadlineAndOpaquePromise> promises_;
// messages that are sent to servers that may later receive them
std::map<Address, std::vector<OpaqueMessage>> can_receive_;
std::map<PartialAddress, std::vector<OpaqueMessage>> can_receive_;
Time cluster_wide_time_microseconds_;
bool should_shut_down_ = false;
@ -59,7 +59,7 @@ class SimulatorHandle {
const Time now = cluster_wide_time_microseconds_;
for (auto it = promises_.begin(); it != promises_.end();) {
auto &[promise_key, dop] = *it;
if (dop.deadline < now) {
if (dop.deadline < now && config_.perform_timeouts) {
spdlog::info("timing out request from requester {} to replier {}.", promise_key.requester_address.ToString(),
promise_key.replier_address.ToString());
std::move(dop).promise.TimeOut();
@ -76,6 +76,14 @@ class SimulatorHandle {
explicit SimulatorHandle(SimulatorConfig config)
: cluster_wide_time_microseconds_(config.start_time), rng_(config.rng_seed), config_(config) {}
~SimulatorHandle() {
for (auto it = promises_.begin(); it != promises_.end();) {
auto &[promise_key, dop] = *it;
std::move(dop).promise.TimeOut();
it = promises_.erase(it);
}
}
void IncrementServerCountAndWaitForQuiescentState(Address address);
/// This method causes most of the interesting simulation logic to happen, wrt network behavior.
@ -121,9 +129,11 @@ class SimulatorHandle {
const Time deadline = cluster_wide_time_microseconds_ + timeout;
auto partial_address = receiver.ToPartialAddress();
while (!should_shut_down_ && (cluster_wide_time_microseconds_ < deadline)) {
if (can_receive_.contains(receiver)) {
std::vector<OpaqueMessage> &can_rx = can_receive_.at(receiver);
if (can_receive_.contains(partial_address)) {
std::vector<OpaqueMessage> &can_rx = can_receive_.at(partial_address);
if (!can_rx.empty()) {
OpaqueMessage message = std::move(can_rx.back());
can_rx.pop_back();

View File

@ -65,7 +65,7 @@ class MachineManager {
MachineConfig config_;
CoordinatorRsm<IoImpl> coordinator_;
ShardManager<IoImpl> shard_manager_;
Time next_cron_;
Time next_cron_ = Time::min();
public:
// TODO initialize ShardManager with "real" coordinator addresses instead of io.GetAddress
@ -95,7 +95,7 @@ class MachineManager {
WriteResponse<CoordinatorWriteResponses>, ReadRequest<StorageReadRequest>,
AppendRequest<StorageWriteRequest>, WriteRequest<StorageWriteRequest>>;
spdlog::info("MM waiting on Receive");
spdlog::info("MM waiting on Receive on address {}", io_.GetAddress().ToString());
// Note: this parameter pack must be kept in-sync with the AllMessages parameter pack above
auto request_result = io_.template ReceiveWithTimeout<
@ -106,7 +106,6 @@ class MachineManager {
if (request_result.HasError()) {
// time to do Cron
spdlog::info("MM got timeout");
continue;
}
@ -116,7 +115,6 @@ class MachineManager {
// If message is for the coordinator, cast it to subset and pass it to the coordinator
bool to_coordinator = coordinator_.GetAddress() == request_envelope.to_address;
spdlog::info("coordinator: {}", coordinator_.GetAddress().ToString());
if (to_coordinator) {
std::optional<CoordinatorMessages> conversion_attempt =
ConvertVariant<AllMessages, ReadRequest<CoordinatorReadRequests>, AppendRequest<CoordinatorWriteRequests>,

View File

@ -246,6 +246,7 @@ class ShardRequestManager : public ShardRequestManagerInterface {
// TODO(kostasrim) Simplify return result
std::vector<VertexAccessor> Request(ExecutionState<ScanVerticesRequest> &state) override {
spdlog::info("shards_map_.size(): {}", shards_map_.GetShards(*state.label).size());
MaybeInitializeExecutionState(state);
std::vector<ScanVerticesResponse> responses;
@ -260,6 +261,7 @@ class ShardRequestManager : public ShardRequestManagerInterface {
for (const auto &shard : state.shard_cache) {
paginated_response_tracker.insert(std::make_pair(shard, PaginatedResponseState::Pending));
}
do {
AwaitOnPaginatedRequests(state, responses, paginated_response_tracker);
} while (!all_requests_gathered(paginated_response_tracker));

View File

@ -62,8 +62,8 @@ template <typename IoImpl>
using ShardRaft = Raft<IoImpl, ShardRsm, WriteRequests, WriteResponses, ReadRequests, ReadResponses>;
using namespace std::chrono_literals;
static constexpr Duration kMinimumCronInterval = 1000ms;
static constexpr Duration kMaximumCronInterval = 2000ms;
static constexpr Duration kMinimumCronInterval = 100ms;
static constexpr Duration kMaximumCronInterval = 200ms;
static_assert(kMinimumCronInterval < kMaximumCronInterval,
"The minimum cron interval has to be smaller than the maximum cron interval!");
@ -135,7 +135,7 @@ class ShardManager {
io::Io<IoImpl> io_;
std::map<uuid, ShardRaft<IoImpl>> rsm_map_;
std::priority_queue<std::pair<Time, uuid>, std::vector<std::pair<Time, uuid>>, std::greater<>> cron_schedule_;
Time next_cron_;
Time next_cron_ = Time::min();
Address coordinator_leader_;
coordinator::ShardMap shard_map_;
std::optional<ResponseFuture<WriteResponse<CoordinatorWriteResponses>>> heartbeat_res_;

View File

@ -17,20 +17,18 @@ function(add_simulation_test test_cpp)
# requires unique logical target names
set_target_properties(${target_name} PROPERTIES OUTPUT_NAME ${exec_name})
# sanitize
target_compile_options(${target_name} PRIVATE -fsanitize=${san})
target_link_options(${target_name} PRIVATE -fsanitize=${san})
target_link_libraries(${target_name} mg-storage-v3 mg-communication gtest gmock mg-utils mg-io mg-io-simulator mg-coordinator Boost::headers mg-query-v2)
target_link_libraries(${target_name} mg-storage-v3 mg-communication mg-utils mg-io mg-io-simulator mg-coordinator mg-query-v2)
target_link_libraries(${target_name} Boost::headers)
target_link_libraries(${target_name} gtest gtest_main gmock rapidcheck rapidcheck_gtest)
# register test
add_test(${target_name} ${exec_name})
add_dependencies(memgraph__simulation ${target_name})
endfunction(add_simulation_test)
add_simulation_test(basic_request.cpp address)
add_simulation_test(raft.cpp address)
add_simulation_test(trial_query_storage/query_storage_test.cpp address)
add_simulation_test(sharded_map.cpp address)
add_simulation_test(shard_request_manager.cpp address)
add_simulation_test(basic_request.cpp)
add_simulation_test(raft.cpp)
add_simulation_test(trial_query_storage/query_storage_test.cpp)
add_simulation_test(sharded_map.cpp)
add_simulation_test(shard_rsm.cpp)
add_simulation_test(cluster_property_test.cpp)

View File

@ -0,0 +1,51 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <rapidcheck.h>
#include "testing_constants.hpp"
namespace memgraph::tests::simulation {
struct ClusterConfig {
int servers;
int replication_factor;
int shards;
friend std::ostream &operator<<(std::ostream &in, const ClusterConfig &cluster) {
in << "ClusterConfig { servers: " << cluster.servers << ", replication_factor: " << cluster.replication_factor
<< ", shards: " << cluster.shards << " }";
return in;
}
};
} // namespace memgraph::tests::simulation
// Required namespace for rapidcheck generator
namespace rc {
using memgraph::tests::simulation::ClusterConfig;
template <>
struct Arbitrary<ClusterConfig> {
static Gen<ClusterConfig> arbitrary() {
return gen::build<ClusterConfig>(
// gen::inRange is [inclusive min, exclusive max)
gen::set(&ClusterConfig::servers, gen::inRange(kMinimumServers, kMaximumServers)),
gen::set(&ClusterConfig::replication_factor,
gen::inRange(kMinimumReplicationFactor, kMaximumReplicationFactor)),
gen::set(&ClusterConfig::shards, gen::inRange(kMinimumShards, kMaximumShards)));
}
};
} // namespace rc

View File

@ -0,0 +1,50 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
// This test serves as an example of a property-based model test.
// It generates a cluster configuration and a set of operations to
// apply against both the real system and a greatly simplified model.
#include <chrono>
#include <gtest/gtest.h>
#include <rapidcheck.h>
#include <rapidcheck/gtest.h>
#include "generated_operations.hpp"
#include "io/simulator/simulator_config.hpp"
#include "io/time.hpp"
#include "storage/v3/shard_manager.hpp"
#include "test_cluster.hpp"
namespace memgraph::tests::simulation {
using io::Duration;
using io::Time;
using io::simulator::SimulatorConfig;
using storage::v3::kMaximumCronInterval;
RC_GTEST_PROP(RandomClusterConfig, HappyPath, (ClusterConfig cluster_config, NonEmptyOpVec ops)) {
// TODO(tyler) set abort_time to something more restrictive than Time::max()
SimulatorConfig sim_config{
.drop_percent = 0,
.perform_timeouts = false,
.scramble_messages = true,
.rng_seed = 0,
.start_time = Time::min(),
.abort_time = Time::max(),
};
RunClusterSimulation(sim_config, cluster_config, ops.ops);
}
} // namespace memgraph::tests::simulation

View File

@ -0,0 +1,114 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <map>
#include <optional>
#include <variant>
#include <rapidcheck.h>
#include <rapidcheck/gtest.h>
#include "storage/v2/storage.hpp"
#include "testing_constants.hpp"
#include "utils/logging.hpp"
namespace memgraph::tests::simulation {
struct CreateVertex {
int first;
int second;
friend std::ostream &operator<<(std::ostream &in, const CreateVertex &add) {
in << "CreateVertex { first: " << add.first << ", second: " << add.second << " }";
return in;
}
};
struct ScanAll {
friend std::ostream &operator<<(std::ostream &in, const ScanAll &get) {
in << "ScanAll {}";
return in;
}
};
using OpVariant = std::variant<CreateVertex, ScanAll>;
struct Op {
OpVariant inner;
friend std::ostream &operator<<(std::ostream &in, const Op &op) {
std::visit([&](const auto &x) { in << x; }, op.inner);
return in;
}
};
struct NonEmptyOpVec {
std::vector<memgraph::tests::simulation::Op> ops;
friend std::ostream &operator<<(std::ostream &in, const NonEmptyOpVec &op) {
in << "[";
bool first = true;
for (const auto &op : op.ops) {
if (!first) {
in << ", ";
}
in << op;
first = false;
}
in << "]";
return in;
}
};
} // namespace memgraph::tests::simulation
// Required namespace for rapidcheck generators
namespace rc {
using namespace memgraph::tests::simulation;
template <>
struct Arbitrary<CreateVertex> {
static Gen<CreateVertex> arbitrary() {
return gen::build<CreateVertex>(gen::set(&CreateVertex::first, gen::inRange(0, kMaximumShards + 1)),
gen::set(&CreateVertex::second, gen::inRange(0, kMaximumShards + 1)));
}
};
template <>
struct Arbitrary<ScanAll> {
static Gen<ScanAll> arbitrary() { return gen::just(ScanAll{}); }
};
OpVariant opHoist(ScanAll op) { return op; }
OpVariant opHoist(CreateVertex op) { return op; }
template <>
struct ::rc::Arbitrary<Op> {
static Gen<Op> arbitrary() {
return gen::build<Op>(gen::set(
&Op::inner, gen::oneOf(gen::map(gen::arbitrary<CreateVertex>(), [](CreateVertex op) { return opHoist(op); }),
gen::map(gen::arbitrary<ScanAll>(), [](ScanAll op) { return opHoist(op); }))));
}
};
template <>
struct Arbitrary<NonEmptyOpVec> {
static Gen<NonEmptyOpVec> arbitrary() {
return gen::build<NonEmptyOpVec>(
gen::set(&NonEmptyOpVec::ops, gen::nonEmpty<std::vector<memgraph::tests::simulation::Op>>()));
}
};
} // namespace rc

View File

@ -130,7 +130,7 @@ void RunSimulation() {
.scramble_messages = true,
.rng_seed = 0,
.start_time = Time::min() + std::chrono::microseconds{256 * 1024},
.abort_time = Time::min() + std::chrono::microseconds{8 * 1024 * 128},
.abort_time = Time::max(),
};
auto simulator = Simulator(config);

View File

@ -1074,7 +1074,7 @@ int TestMessages() {
.scramble_messages = false,
.rng_seed = 0,
.start_time = Time::min() + std::chrono::microseconds{256 * 1024},
.abort_time = Time::min() + std::chrono::microseconds{4 * 8 * 1024 * 1024},
.abort_time = Time::max(),
};
auto simulator = Simulator(config);

View File

@ -0,0 +1,251 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include <chrono>
#include <iostream>
#include <limits>
#include <memory>
#include <set>
#include <thread>
#include <rapidcheck.h>
#include "cluster_config.hpp"
#include "coordinator/coordinator_client.hpp"
#include "coordinator/coordinator_rsm.hpp"
#include "coordinator/shard_map.hpp"
#include "generated_operations.hpp"
#include "io/address.hpp"
#include "io/simulator/simulator.hpp"
#include "io/simulator/simulator_config.hpp"
#include "io/simulator/simulator_transport.hpp"
#include "machine_manager/machine_config.hpp"
#include "machine_manager/machine_manager.hpp"
#include "query/v2/requests.hpp"
#include "query/v2/shard_request_manager.hpp"
#include "testing_constants.hpp"
#include "utils/variant_helpers.hpp"
namespace memgraph::tests::simulation {
using coordinator::Coordinator;
using coordinator::CoordinatorClient;
using coordinator::CoordinatorReadRequests;
using coordinator::CoordinatorWriteRequests;
using coordinator::CoordinatorWriteResponses;
using coordinator::GetShardMapRequest;
using coordinator::GetShardMapResponse;
using coordinator::Hlc;
using coordinator::HlcResponse;
using coordinator::Shard;
using coordinator::ShardMap;
using io::Address;
using io::Io;
using io::rsm::RsmClient;
using io::simulator::Simulator;
using io::simulator::SimulatorConfig;
using io::simulator::SimulatorStats;
using io::simulator::SimulatorTransport;
using machine_manager::MachineConfig;
using machine_manager::MachineManager;
using msgs::ReadRequests;
using msgs::ReadResponses;
using msgs::WriteRequests;
using msgs::WriteResponses;
using storage::v3::LabelId;
using storage::v3::SchemaProperty;
using CompoundKey = std::pair<int, int>;
using ShardClient = RsmClient<SimulatorTransport, WriteRequests, WriteResponses, ReadRequests, ReadResponses>;
MachineManager<SimulatorTransport> MkMm(Simulator &simulator, std::vector<Address> coordinator_addresses, Address addr,
ShardMap shard_map) {
MachineConfig config{
.coordinator_addresses = coordinator_addresses,
.is_storage = true,
.is_coordinator = true,
.listen_ip = addr.last_known_ip,
.listen_port = addr.last_known_port,
};
Io<SimulatorTransport> io = simulator.Register(addr);
Coordinator coordinator{shard_map};
return MachineManager{io, config, coordinator, shard_map};
}
void RunMachine(MachineManager<SimulatorTransport> mm) { mm.Run(); }
void WaitForShardsToInitialize(CoordinatorClient<SimulatorTransport> &coordinator_client) {
// Call coordinator client's read method for GetShardMap and keep
// reading it until the shard map contains proper replicas for
// each shard in the label space.
while (true) {
GetShardMapRequest req{};
CoordinatorReadRequests read_req = req;
auto read_res = coordinator_client.SendReadRequest(read_req);
if (read_res.HasError()) {
// timed out
continue;
}
auto response_result = read_res.GetValue();
auto response = std::get<GetShardMapResponse>(response_result);
auto shard_map = response.shard_map;
if (shard_map.ClusterInitialized()) {
spdlog::info("cluster stabilized - beginning workload");
return;
}
}
}
ShardMap TestShardMap(int n_splits, int replication_factor) {
ShardMap sm{};
const std::string label_name = std::string("test_label");
// register new properties
const std::vector<std::string> property_names = {"property_1", "property_2"};
const auto properties = sm.AllocatePropertyIds(property_names);
const auto property_id_1 = properties.at("property_1");
const auto property_id_2 = properties.at("property_2");
const auto type_1 = memgraph::common::SchemaType::INT;
const auto type_2 = memgraph::common::SchemaType::INT;
// register new label space
std::vector<SchemaProperty> schema = {
SchemaProperty{.property_id = property_id_1, .type = type_1},
SchemaProperty{.property_id = property_id_2, .type = type_2},
};
std::optional<LabelId> label_id = sm.InitializeNewLabel(label_name, schema, replication_factor, sm.shard_map_version);
RC_ASSERT(label_id.has_value());
// split the shard at N split points
for (int64_t i = 1; i < n_splits; ++i) {
const auto key1 = memgraph::storage::v3::PropertyValue(i);
const auto key2 = memgraph::storage::v3::PropertyValue(0);
const auto split_point = {key1, key2};
const bool split_success = sm.SplitShard(sm.shard_map_version, label_id.value(), split_point);
RC_ASSERT(split_success);
}
return sm;
}
void ExecuteOp(msgs::ShardRequestManager<SimulatorTransport> &shard_request_manager,
std::set<CompoundKey> &correctness_model, CreateVertex create_vertex) {
const auto key1 = memgraph::storage::v3::PropertyValue(create_vertex.first);
const auto key2 = memgraph::storage::v3::PropertyValue(create_vertex.second);
std::vector<msgs::Value> primary_key = {msgs::Value(int64_t(create_vertex.first)),
msgs::Value(int64_t(create_vertex.second))};
if (correctness_model.contains(std::make_pair(create_vertex.first, create_vertex.second))) {
// TODO(tyler) remove this early-return when we have properly handled setting non-unique vertexes
return;
}
msgs::ExecutionState<msgs::CreateVerticesRequest> state;
auto label_id = shard_request_manager.NameToLabel("test_label");
msgs::NewVertex nv{.primary_key = primary_key};
nv.label_ids.push_back({label_id});
std::vector<msgs::NewVertex> new_vertices;
new_vertices.push_back(std::move(nv));
auto result = shard_request_manager.Request(state, std::move(new_vertices));
RC_ASSERT(result.size() == 1);
RC_ASSERT(result[0].success);
correctness_model.emplace(std::make_pair(create_vertex.first, create_vertex.second));
}
void ExecuteOp(msgs::ShardRequestManager<SimulatorTransport> &shard_request_manager,
std::set<CompoundKey> &correctness_model, ScanAll scan_all) {
msgs::ExecutionState<msgs::ScanVerticesRequest> request{.label = "test_label"};
auto results = shard_request_manager.Request(request);
RC_ASSERT(results.size() == correctness_model.size());
for (const auto &vertex_accessor : results) {
const auto properties = vertex_accessor.Properties();
const auto primary_key = vertex_accessor.Id().second;
const CompoundKey model_key = std::make_pair(primary_key[0].int_v, primary_key[1].int_v);
RC_ASSERT(correctness_model.contains(model_key));
}
}
void RunClusterSimulation(const SimulatorConfig &sim_config, const ClusterConfig &cluster_config,
const std::vector<Op> &ops) {
spdlog::info("========================== NEW SIMULATION ==========================");
auto simulator = Simulator(sim_config);
auto cli_addr = Address::TestAddress(1);
auto machine_1_addr = cli_addr.ForkUniqueAddress();
Io<SimulatorTransport> cli_io = simulator.Register(cli_addr);
auto coordinator_addresses = std::vector{
machine_1_addr,
};
ShardMap initialization_sm = TestShardMap(cluster_config.shards - 1, cluster_config.replication_factor);
auto mm_1 = MkMm(simulator, coordinator_addresses, machine_1_addr, initialization_sm);
Address coordinator_address = mm_1.CoordinatorAddress();
auto mm_thread_1 = std::jthread(RunMachine, std::move(mm_1));
// Need to detach this thread so that the destructor does not
// block before we can propagate assertion failures.
mm_thread_1.detach();
// TODO(tyler) clarify addresses of coordinator etc... as it's a mess
CoordinatorClient<SimulatorTransport> coordinator_client(cli_io, coordinator_address, {coordinator_address});
WaitForShardsToInitialize(coordinator_client);
msgs::ShardRequestManager<SimulatorTransport> shard_request_manager(std::move(coordinator_client), std::move(cli_io));
shard_request_manager.StartTransaction();
auto correctness_model = std::set<CompoundKey>{};
for (const Op &op : ops) {
std::visit([&](auto &o) { ExecuteOp(shard_request_manager, correctness_model, o); }, op.inner);
}
simulator.ShutDown();
SimulatorStats stats = simulator.Stats();
spdlog::info("total messages: {}", stats.total_messages);
spdlog::info("dropped messages: {}", stats.dropped_messages);
spdlog::info("timed out requests: {}", stats.timed_out_requests);
spdlog::info("total requests: {}", stats.total_requests);
spdlog::info("total responses: {}", stats.total_responses);
spdlog::info("simulator ticks: {}", stats.simulator_ticks);
spdlog::info("========================== SUCCESS :) ==========================");
}
} // namespace memgraph::tests::simulation

View File

@ -0,0 +1,28 @@
// Copyright 2022 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
namespace memgraph::tests::simulation {
// TODO(tyler) increase this when we start standing up multiple machines in cluster tests
static constexpr auto kMinimumShards = 1;
static constexpr auto kMaximumShards = kMinimumShards + 10;
// TODO(tyler) increase this when we start standing up multiple machines in cluster tests
static constexpr auto kMinimumServers = 1;
static constexpr auto kMaximumServers = kMinimumServers + 1;
// TODO(tyler) increase this when we start standing up multiple machines in cluster tests
static constexpr auto kMinimumReplicationFactor = 1;
static constexpr auto kMaximumReplicationFactor = kMinimumReplicationFactor + 1;
} // namespace memgraph::tests::simulation

View File

@ -20,8 +20,8 @@
#include "messages.hpp"
namespace memgraph::tests::simulation {
using memgraph::io::Io;
using memgraph::io::simulator::SimulatorTransport;
using io::Io;
using io::simulator::SimulatorTransport;
void run_server(Io<SimulatorTransport> io) {
while (!io.ShouldShutDown()) {