Prepare codebase for high availability

Summary:
Initial for fork (in form of a c/p) form the current single node
version. Once we finish HA we plan to re-link the files to the single node
versions if they don't change.

Reviewers: ipaljak, buda, mferencevic

Reviewed By: ipaljak

Subscribers: pullbot

Differential Revision: https://phabricator.memgraph.io/D1705
This commit is contained in:
Matija Santl 2018-10-26 16:18:20 +02:00
parent 25adfdb90c
commit 43cb506f06
47 changed files with 6595 additions and 0 deletions

View File

@ -238,6 +238,78 @@ target_compile_definitions(mg-distributed PUBLIC MG_DISTRIBUTED)
# END Memgraph Distributed
# ----------------------------------------------------------------------------
# ----------------------------------------------------------------------------
# Memgraph Single Node High Availability
# ----------------------------------------------------------------------------
set(mg_single_node_ha_sources
data_structures/concurrent/skiplist_gc.cpp
database/single_node_ha/config.cpp
database/single_node_ha/graph_db.cpp
database/single_node_ha/graph_db_accessor.cpp
durability/single_node_ha/state_delta.cpp
durability/single_node_ha/paths.cpp
durability/single_node_ha/recovery.cpp
durability/single_node_ha/snapshooter.cpp
durability/single_node_ha/wal.cpp
glue/auth.cpp
glue/communication.cpp
query/common.cpp
query/frontend/ast/ast.cpp
query/frontend/ast/cypher_main_visitor.cpp
query/frontend/semantic/required_privileges.cpp
query/frontend/semantic/symbol_generator.cpp
query/frontend/stripped.cpp
query/interpret/awesome_memgraph_functions.cpp
query/interpreter.cpp
query/plan/operator.cpp
query/plan/preprocess.cpp
query/plan/pretty_print.cpp
query/plan/rule_based_planner.cpp
query/plan/variable_start_planner.cpp
query/repl.cpp
query/typed_value.cpp
storage/common/property_value.cpp
storage/common/property_value_store.cpp
storage/locking/record_lock.cpp
storage/single_node_ha/edge_accessor.cpp
storage/single_node_ha/record_accessor.cpp
storage/single_node_ha/vertex_accessor.cpp
transactions/single_node_ha/engine.cpp
memgraph_init.cpp
)
define_add_lcp(add_lcp_single_node_ha mg_single_node_ha_sources generated_lcp_single_node_ha_files)
add_lcp_single_node_ha(durability/single_node_ha/state_delta.lcp)
add_lcp_single_node_ha(query/frontend/ast/ast.lcp)
add_lcp_single_node_ha(query/plan/operator.lcp)
add_custom_target(generate_lcp_single_node_ha DEPENDS ${generated_lcp_single_node_ha_files})
set(MG_SINGLE_NODE_HA_LIBS stdc++fs Threads::Threads fmt cppitertools
antlr_opencypher_parser_lib dl glog gflags capnp kj
mg-utils mg-io mg-integrations-kafka mg-requests mg-communication mg-auth mg-stats)
if (USE_LTALLOC)
list(APPEND MG_SINGLE_NODE_HA_LIBS ltalloc)
# TODO(mferencevic): Enable this when clang is updated on apollo.
# set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -flto")
endif()
if (READLINE_FOUND)
list(APPEND MG_SINGLE_NODE_HA_LIBS readline)
endif()
add_library(mg-single-node-ha STATIC ${mg_single_node_ha_sources})
target_link_libraries(mg-single-node-ha ${MG_SINGLE_NODE_HA_LIBS})
add_dependencies(mg-single-node-ha generate_opencypher_parser)
add_dependencies(mg-single-node-ha generate_lcp_single_node_ha)
target_compile_definitions(mg-single-node-ha PUBLIC MG_SINGLE_NODE_HA)
# ----------------------------------------------------------------------------
# END Memgraph Single Node High Availability
# ----------------------------------------------------------------------------
string(TOLOWER ${CMAKE_BUILD_TYPE} lower_build_type)
# STATIC library used to store key-value pairs
@ -326,3 +398,17 @@ add_custom_command(TARGET memgraph_distributed POST_BUILD
COMMAND ${CMAKE_COMMAND} -E create_symlink $<TARGET_FILE:memgraph_distributed> ${CMAKE_BINARY_DIR}/memgraph_distributed
BYPRODUCTS ${CMAKE_BINARY_DIR}/memgraph_distributed
COMMENT Creating symlink to memgraph distributed executable)
# memgraph single node high availability executable
add_executable(memgraph_ha memgraph_ha.cpp)
target_link_libraries(memgraph_ha mg-single-node-ha kvstore_lib telemetry_lib)
set_target_properties(memgraph_ha PROPERTIES
# Set the executable output name to include version information.
OUTPUT_NAME "memgraph_ha-${memgraph_VERSION}-${COMMIT_HASH}_${CMAKE_BUILD_TYPE}"
# Output the executable in main binary dir.
RUNTIME_OUTPUT_DIRECTORY ${CMAKE_BINARY_DIR})
# Create symlink to the built executable.
add_custom_command(TARGET memgraph_ha POST_BUILD
COMMAND ${CMAKE_COMMAND} -E create_symlink $<TARGET_FILE:memgraph_ha> ${CMAKE_BINARY_DIR}/memgraph_ha
BYPRODUCTS ${CMAKE_BINARY_DIR}/memgraph_ha
COMMENT Creating symlink to memgraph single node high availability executable)

View File

@ -3,6 +3,11 @@
#ifdef MG_SINGLE_NODE
#include "database/single_node/graph_db.hpp"
#endif
#ifdef MG_SINGLE_NODE_HA
#include "database/single_node_ha/graph_db.hpp"
#endif
#ifdef MG_DISTRIBUTED
#include "database/distributed/graph_db.hpp"
#endif

View File

@ -3,6 +3,11 @@
#ifdef MG_SINGLE_NODE
#include "database/single_node/graph_db_accessor.hpp"
#endif
#ifdef MG_SINGLE_NODE_HA
#include "database/single_node_ha/graph_db_accessor.hpp"
#endif
#ifdef MG_DISTRIBUTED
#include "database/distributed/graph_db_accessor.hpp"
#endif

View File

@ -0,0 +1,52 @@
#include <limits>
#include "database/single_node_ha/graph_db.hpp"
#include "utils/flag_validation.hpp"
#include "utils/string.hpp"
// Durability flags.
DEFINE_bool(durability_enabled, false,
"If durability (database persistence) should be enabled");
DEFINE_string(
durability_directory, "durability",
"Path to directory in which to save snapshots and write-ahead log files.");
DEFINE_bool(db_recover_on_startup, false, "Recover database on startup.");
DEFINE_VALIDATED_int32(
snapshot_cycle_sec, 3600,
"Amount of time between two snapshots, in seconds (min 60).",
FLAG_IN_RANGE(1, std::numeric_limits<int32_t>::max()));
DEFINE_int32(snapshot_max_retained, -1,
"Number of retained snapshots, -1 means without limit.");
DEFINE_bool(snapshot_on_exit, false, "Snapshot on exiting the database.");
// Misc flags
DEFINE_int32(query_execution_time_sec, 180,
"Maximum allowed query execution time. Queries exceeding this "
"limit will be aborted. Value of -1 means no limit.");
DEFINE_int32(gc_cycle_sec, 30,
"Amount of time between starts of two cleaning cycles in seconds. "
"-1 to turn off.");
// Data location.
DEFINE_string(properties_on_disk, "",
"Property names of properties which will be stored on available "
"disk. Property names have to be separated with comma (,).");
// Full durability.
DEFINE_bool(synchronous_commit, false,
"Should a transaction end wait for WAL records to be written to "
"disk before the transaction finishes.");
database::Config::Config()
// Durability flags.
: durability_enabled{FLAGS_durability_enabled},
durability_directory{FLAGS_durability_directory},
db_recover_on_startup{FLAGS_db_recover_on_startup},
snapshot_cycle_sec{FLAGS_snapshot_cycle_sec},
snapshot_max_retained{FLAGS_snapshot_max_retained},
snapshot_on_exit{FLAGS_snapshot_on_exit},
synchronous_commit{FLAGS_synchronous_commit},
// Misc flags.
gc_cycle_sec{FLAGS_gc_cycle_sec},
query_execution_time_sec{FLAGS_query_execution_time_sec},
// Data location.
properties_on_disk(utils::Split(FLAGS_properties_on_disk, ",")) {}

View File

@ -0,0 +1,31 @@
/// @file
#pragma once
#include <atomic>
#include <cstdint>
#include <string>
#include "data_structures/concurrent/concurrent_map.hpp"
namespace database {
/// Implementation for the single-node memgraph
class Counters {
public:
int64_t Get(const std::string &name) {
return counters_.access()
.emplace(name, std::make_tuple(name), std::make_tuple(0))
.first->second.fetch_add(1);
}
void Set(const std::string &name, int64_t value) {
auto name_counter_pair = counters_.access().emplace(
name, std::make_tuple(name), std::make_tuple(value));
if (!name_counter_pair.second) name_counter_pair.first->second.store(value);
}
private:
ConcurrentMap<std::string, std::atomic<int64_t>> counters_;
};
} // namespace database

View File

@ -0,0 +1,157 @@
#include "database/single_node_ha/graph_db.hpp"
#include <experimental/optional>
#include <glog/logging.h>
#include "database/single_node_ha/counters.hpp"
#include "database/single_node_ha/graph_db_accessor.hpp"
#include "durability/single_node_ha/paths.hpp"
#include "durability/single_node_ha/recovery.hpp"
#include "durability/single_node_ha/snapshooter.hpp"
#include "storage/single_node_ha/concurrent_id_mapper.hpp"
#include "storage/single_node_ha/storage_gc.hpp"
#include "transactions/single_node_ha/engine.hpp"
#include "utils/file.hpp"
namespace database {
GraphDb::GraphDb(Config config) : config_(config) {
if (config_.durability_enabled) utils::CheckDir(config_.durability_directory);
// Durability recovery.
if (config_.db_recover_on_startup) {
CHECK(durability::VersionConsistency(config_.durability_directory))
<< "Contents of durability directory are not compatible with the "
"current version of Memgraph binary!";
// What we recover.
std::experimental::optional<durability::RecoveryInfo> recovery_info;
durability::RecoveryData recovery_data;
recovery_info = durability::RecoverOnlySnapshot(
config_.durability_directory, this, &recovery_data,
std::experimental::nullopt);
// Post-recovery setup and checking.
if (recovery_info) {
recovery_data.wal_tx_to_recover = recovery_info->wal_recovered;
durability::RecoveryTransactions recovery_transactions(this);
durability::RecoverWal(config_.durability_directory, this, &recovery_data,
&recovery_transactions);
durability::RecoverIndexes(this, recovery_data.indexes);
}
}
if (config_.durability_enabled) {
// move any existing snapshots or wal files to a deprecated folder.
if (!config_.db_recover_on_startup &&
durability::ContainsDurabilityFiles(config_.durability_directory)) {
durability::MoveToBackup(config_.durability_directory);
LOG(WARNING) << "Since Memgraph was not supposed to recover on startup "
"and durability is enabled, your current durability "
"files will likely be overriden. To prevent important "
"data loss, Memgraph has stored those files into a "
".backup directory inside durability directory";
}
wal_.Init();
snapshot_creator_ = std::make_unique<utils::Scheduler>();
snapshot_creator_->Run(
"Snapshot", std::chrono::seconds(config_.snapshot_cycle_sec), [this] {
auto dba = this->Access();
this->MakeSnapshot(*dba);
});
}
// Start transaction killer.
if (config_.query_execution_time_sec != -1) {
transaction_killer_.Run(
"TX killer",
std::chrono::seconds(
std::max(1, std::min(5, config_.query_execution_time_sec / 4))),
[this]() {
tx_engine_.LocalForEachActiveTransaction([this](tx::Transaction &t) {
if (t.creation_time() +
std::chrono::seconds(config_.query_execution_time_sec) <
std::chrono::steady_clock::now()) {
t.set_should_abort();
};
});
});
}
}
GraphDb::~GraphDb() {
snapshot_creator_ = nullptr;
is_accepting_transactions_ = false;
tx_engine_.LocalForEachActiveTransaction(
[](auto &t) { t.set_should_abort(); });
if (config_.snapshot_on_exit) {
auto dba = this->Access();
MakeSnapshot(*dba);
}
}
std::unique_ptr<GraphDbAccessor> GraphDb::Access() {
// NOTE: We are doing a heap allocation to allow polymorphism. If this poses
// performance issues, we may want to have a stack allocated GraphDbAccessor
// which is constructed with a pointer to some global implementation struct
// which contains only pure functions (without any state).
return std::unique_ptr<GraphDbAccessor>(new GraphDbAccessor(*this));
}
std::unique_ptr<GraphDbAccessor> GraphDb::Access(tx::TransactionId tx_id) {
return std::unique_ptr<GraphDbAccessor>(new GraphDbAccessor(*this, tx_id));
}
std::unique_ptr<GraphDbAccessor> GraphDb::AccessBlocking(
std::experimental::optional<tx::TransactionId> parent_tx) {
return std::unique_ptr<GraphDbAccessor>(
new GraphDbAccessor(*this, parent_tx));
}
Storage &GraphDb::storage() { return *storage_; }
durability::WriteAheadLog &GraphDb::wal() { return wal_; }
tx::Engine &GraphDb::tx_engine() { return tx_engine_; }
storage::ConcurrentIdMapper<storage::Label> &GraphDb::label_mapper() {
return label_mapper_;
}
storage::ConcurrentIdMapper<storage::EdgeType> &GraphDb::edge_type_mapper() {
return edge_mapper_;
}
storage::ConcurrentIdMapper<storage::Property> &GraphDb::property_mapper() {
return property_mapper_;
}
database::Counters &GraphDb::counters() { return counters_; }
void GraphDb::CollectGarbage() { storage_gc_->CollectGarbage(); }
bool GraphDb::MakeSnapshot(GraphDbAccessor &accessor) {
const bool status = durability::MakeSnapshot(
*this, accessor, fs::path(config_.durability_directory),
config_.snapshot_max_retained);
if (status) {
LOG(INFO) << "Snapshot created successfully.";
} else {
LOG(ERROR) << "Snapshot creation failed!";
}
return status;
}
void GraphDb::ReinitializeStorage() {
// Release gc scheduler to stop it from touching storage
storage_gc_ = nullptr;
storage_ = std::make_unique<Storage>(config_.properties_on_disk);
storage_gc_ =
std::make_unique<StorageGc>(*storage_, tx_engine_, config_.gc_cycle_sec);
}
} // namespace database

View File

@ -0,0 +1,129 @@
/// @file
#pragma once
#include <atomic>
#include <experimental/optional>
#include <memory>
#include <vector>
#include "database/single_node_ha/counters.hpp"
#include "durability/single_node_ha/recovery.hpp"
#include "durability/single_node_ha/wal.hpp"
#include "io/network/endpoint.hpp"
#include "storage/common/types.hpp"
#include "storage/single_node_ha/concurrent_id_mapper.hpp"
#include "storage/single_node_ha/storage.hpp"
#include "storage/single_node_ha/storage_gc.hpp"
#include "transactions/single_node_ha/engine.hpp"
#include "utils/scheduler.hpp"
namespace database {
/// Database configuration. Initialized from flags, but modifiable.
struct Config {
Config();
// Durability flags.
bool durability_enabled;
std::string durability_directory;
bool db_recover_on_startup;
int snapshot_cycle_sec;
int snapshot_max_retained;
int snapshot_on_exit;
bool synchronous_commit;
// Misc flags.
int gc_cycle_sec;
int query_execution_time_sec;
// set of properties which will be stored on disk
std::vector<std::string> properties_on_disk;
};
class GraphDbAccessor;
/// An abstract base class providing the interface for a graph database.
///
/// Always be sure that GraphDb object is destructed before main exits, i. e.
/// GraphDb object shouldn't be part of global/static variable, except if its
/// destructor is explicitly called before main exits. Consider code:
///
/// GraphDb db; // KeyIndex is created as a part of database::Storage
/// int main() {
/// GraphDbAccessor dba(db);
/// auto v = dba.InsertVertex();
/// v.add_label(dba.Label(
/// "Start")); // New SkipList is created in KeyIndex for LabelIndex.
/// // That SkipList creates SkipListGc which
/// // initialises static Executor object.
/// return 0;
/// }
///
/// After main exits: 1. Executor is destructed, 2. KeyIndex is destructed.
/// Destructor of KeyIndex calls delete on created SkipLists which destroy
/// SkipListGc that tries to use Excutioner object that doesn't exist anymore.
/// -> CRASH
class GraphDb {
public:
explicit GraphDb(Config config = Config());
~GraphDb();
GraphDb(const GraphDb &) = delete;
GraphDb(GraphDb &&) = delete;
GraphDb &operator=(const GraphDb &) = delete;
GraphDb &operator=(GraphDb &&) = delete;
/// Create a new accessor by starting a new transaction.
std::unique_ptr<GraphDbAccessor> Access();
std::unique_ptr<GraphDbAccessor> AccessBlocking(
std::experimental::optional<tx::TransactionId> parent_tx);
/// Create an accessor for a running transaction.
std::unique_ptr<GraphDbAccessor> Access(tx::TransactionId);
Storage &storage();
durability::WriteAheadLog &wal();
tx::Engine &tx_engine();
storage::ConcurrentIdMapper<storage::Label> &label_mapper();
storage::ConcurrentIdMapper<storage::EdgeType> &edge_type_mapper();
storage::ConcurrentIdMapper<storage::Property> &property_mapper();
database::Counters &counters();
void CollectGarbage();
/// Makes a snapshot from the visibility of the given accessor
bool MakeSnapshot(GraphDbAccessor &accessor);
/// Releases the storage object safely and creates a new object.
/// This is needed because of recovery, otherwise we might try to recover into
/// a storage which has already been polluted because of a failed previous
/// recovery
void ReinitializeStorage();
/// When this is false, no new transactions should be created.
bool is_accepting_transactions() const { return is_accepting_transactions_; }
protected:
std::atomic<bool> is_accepting_transactions_{true};
std::unique_ptr<utils::Scheduler> snapshot_creator_;
utils::Scheduler transaction_killer_;
Config config_;
std::unique_ptr<Storage> storage_ =
std::make_unique<Storage>(config_.properties_on_disk);
durability::WriteAheadLog wal_{config_.durability_directory,
config_.durability_enabled,
config_.synchronous_commit};
tx::Engine tx_engine_{&wal_};
std::unique_ptr<StorageGc> storage_gc_ =
std::make_unique<StorageGc>(*storage_, tx_engine_, config_.gc_cycle_sec);
storage::ConcurrentIdMapper<storage::Label> label_mapper_{
storage_->PropertiesOnDisk()};
storage::ConcurrentIdMapper<storage::EdgeType> edge_mapper_{
storage_->PropertiesOnDisk()};
storage::ConcurrentIdMapper<storage::Property> property_mapper_{
storage_->PropertiesOnDisk()};
database::Counters counters_;
};
} // namespace database

View File

@ -0,0 +1,422 @@
#include "database/single_node_ha/graph_db_accessor.hpp"
#include <chrono>
#include <thread>
#include <glog/logging.h>
#include "durability/single_node_ha/state_delta.hpp"
#include "storage/single_node_ha/edge.hpp"
#include "storage/single_node_ha/edge_accessor.hpp"
#include "storage/single_node_ha/vertex.hpp"
#include "storage/single_node_ha/vertex_accessor.hpp"
#include "utils/cast.hpp"
#include "utils/on_scope_exit.hpp"
namespace database {
GraphDbAccessor::GraphDbAccessor(GraphDb &db)
: db_(db),
transaction_(*db.tx_engine().Begin()),
transaction_starter_{true} {}
GraphDbAccessor::GraphDbAccessor(GraphDb &db, tx::TransactionId tx_id)
: db_(db),
transaction_(*db.tx_engine().RunningTransaction(tx_id)),
transaction_starter_{false} {}
GraphDbAccessor::GraphDbAccessor(
GraphDb &db, std::experimental::optional<tx::TransactionId> parent_tx)
: db_(db),
transaction_(*db.tx_engine().BeginBlocking(parent_tx)),
transaction_starter_{true} {}
GraphDbAccessor::~GraphDbAccessor() {
if (transaction_starter_ && !commited_ && !aborted_) {
this->Abort();
}
}
tx::TransactionId GraphDbAccessor::transaction_id() const {
return transaction_.id_;
}
void GraphDbAccessor::AdvanceCommand() {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
db_.tx_engine().Advance(transaction_.id_);
}
void GraphDbAccessor::Commit() {
DCHECK(!commited_ && !aborted_) << "Already aborted or commited transaction.";
db_.tx_engine().Commit(transaction_);
commited_ = true;
}
void GraphDbAccessor::Abort() {
DCHECK(!commited_ && !aborted_) << "Already aborted or commited transaction.";
db_.tx_engine().Abort(transaction_);
aborted_ = true;
}
bool GraphDbAccessor::should_abort() const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return transaction_.should_abort();
}
durability::WriteAheadLog &GraphDbAccessor::wal() { return db_.wal(); }
VertexAccessor GraphDbAccessor::InsertVertex(
std::experimental::optional<gid::Gid> requested_gid) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
auto gid = db_.storage().vertex_generator_.Next(requested_gid);
auto vertex_vlist = new mvcc::VersionList<Vertex>(transaction_, gid);
bool success =
db_.storage().vertices_.access().insert(gid, vertex_vlist).second;
CHECK(success) << "Attempting to insert a vertex with an existing GID: "
<< gid;
wal().Emplace(
database::StateDelta::CreateVertex(transaction_.id_, vertex_vlist->gid_));
auto va = VertexAccessor(vertex_vlist, *this);
return va;
}
std::experimental::optional<VertexAccessor> GraphDbAccessor::FindVertexOptional(
gid::Gid gid, bool current_state) {
VertexAccessor record_accessor(db_.storage().LocalAddress<Vertex>(gid),
*this);
if (!record_accessor.Visible(transaction(), current_state))
return std::experimental::nullopt;
return record_accessor;
}
VertexAccessor GraphDbAccessor::FindVertex(gid::Gid gid, bool current_state) {
auto found = FindVertexOptional(gid, current_state);
CHECK(found) << "Unable to find vertex for id: " << gid;
return *found;
}
std::experimental::optional<EdgeAccessor> GraphDbAccessor::FindEdgeOptional(
gid::Gid gid, bool current_state) {
EdgeAccessor record_accessor(db_.storage().LocalAddress<Edge>(gid), *this);
if (!record_accessor.Visible(transaction(), current_state))
return std::experimental::nullopt;
return record_accessor;
}
EdgeAccessor GraphDbAccessor::FindEdge(gid::Gid gid, bool current_state) {
auto found = FindEdgeOptional(gid, current_state);
CHECK(found) << "Unable to find edge for id: " << gid;
return *found;
}
void GraphDbAccessor::BuildIndex(storage::Label label,
storage::Property property, bool unique) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
// Create the index
const LabelPropertyIndex::Key key(label, property, unique);
if (db_.storage().label_property_index_.CreateIndex(key) == false) {
throw IndexExistsException(
"Index is either being created by another transaction or already "
"exists.");
}
try {
auto dba =
db_.AccessBlocking(std::experimental::make_optional(transaction_.id_));
dba->PopulateIndex(key);
dba->EnableIndex(key);
dba->Commit();
} catch (const IndexConstraintViolationException &) {
db_.storage().label_property_index_.DeleteIndex(key);
throw;
} catch (const tx::TransactionEngineError &e) {
db_.storage().label_property_index_.DeleteIndex(key);
throw IndexTransactionException(e.what());
}
}
void GraphDbAccessor::EnableIndex(const LabelPropertyIndex::Key &key) {
// Commit transaction as we finished applying method on newest visible
// records. Write that transaction's ID to the WAL as the index has been
// built at this point even if this DBA's transaction aborts for some
// reason.
wal().Emplace(database::StateDelta::BuildIndex(
transaction_id(), key.label_, LabelName(key.label_), key.property_,
PropertyName(key.property_), key.unique_));
}
void GraphDbAccessor::PopulateIndex(const LabelPropertyIndex::Key &key) {
for (auto vertex : Vertices(key.label_, false)) {
if (vertex.PropsAt(key.property_).type() == PropertyValue::Type::Null)
continue;
if (!db_.storage().label_property_index_.UpdateOnLabelProperty(
vertex.address(), vertex.current_)) {
throw IndexConstraintViolationException(
"Index couldn't be created due to constraint violation!");
}
}
}
void GraphDbAccessor::DeleteIndex(storage::Label label,
storage::Property property) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
LabelPropertyIndex::Key key(label, property);
try {
auto dba =
db_.AccessBlocking(std::experimental::make_optional(transaction_.id_));
db_.storage().label_property_index_.DeleteIndex(key);
dba->wal().Emplace(database::StateDelta::DropIndex(
dba->transaction_id(), key.label_, LabelName(key.label_), key.property_,
PropertyName(key.property_)));
dba->Commit();
} catch (const tx::TransactionEngineError &e) {
throw IndexTransactionException(e.what());
}
}
void GraphDbAccessor::UpdateLabelIndices(storage::Label label,
const VertexAccessor &vertex_accessor,
const Vertex *const vertex) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
auto *vlist_ptr = vertex_accessor.address();
if (!db_.storage().label_property_index_.UpdateOnLabel(label, vlist_ptr,
vertex)) {
throw IndexConstraintViolationException(
"Node couldn't be updated due to index constraint violation!");
}
db_.storage().labels_index_.Update(label, vlist_ptr, vertex);
}
void GraphDbAccessor::UpdatePropertyIndex(
storage::Property property, const RecordAccessor<Vertex> &vertex_accessor,
const Vertex *const vertex) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
if (!db_.storage().label_property_index_.UpdateOnProperty(
property, vertex_accessor.address(), vertex)) {
throw IndexConstraintViolationException(
"Node couldn't be updated due to index constraint violation!");
}
}
int64_t GraphDbAccessor::VerticesCount() const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.storage().vertices_.access().size();
}
int64_t GraphDbAccessor::VerticesCount(storage::Label label) const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.storage().labels_index_.Count(label);
}
int64_t GraphDbAccessor::VerticesCount(storage::Label label,
storage::Property property) const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
const LabelPropertyIndex::Key key(label, property);
DCHECK(db_.storage().label_property_index_.IndexExists(key))
<< "Index doesn't exist.";
return db_.storage().label_property_index_.Count(key);
}
int64_t GraphDbAccessor::VerticesCount(storage::Label label,
storage::Property property,
const PropertyValue &value) const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
const LabelPropertyIndex::Key key(label, property);
DCHECK(db_.storage().label_property_index_.IndexExists(key))
<< "Index doesn't exist.";
return db_.storage()
.label_property_index_.PositionAndCount(key, value)
.second;
}
int64_t GraphDbAccessor::VerticesCount(
storage::Label label, storage::Property property,
const std::experimental::optional<utils::Bound<PropertyValue>> lower,
const std::experimental::optional<utils::Bound<PropertyValue>> upper)
const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
const LabelPropertyIndex::Key key(label, property);
DCHECK(db_.storage().label_property_index_.IndexExists(key))
<< "Index doesn't exist.";
CHECK(lower || upper) << "At least one bound must be provided";
CHECK(!lower || lower.value().value().type() != PropertyValue::Type::Null)
<< "Null value is not a valid index bound";
CHECK(!upper || upper.value().value().type() != PropertyValue::Type::Null)
<< "Null value is not a valid index bound";
if (!upper) {
auto lower_pac = db_.storage().label_property_index_.PositionAndCount(
key, lower.value().value());
int64_t size = db_.storage().label_property_index_.Count(key);
return std::max(0l,
size - lower_pac.first -
(lower.value().IsInclusive() ? 0l : lower_pac.second));
} else if (!lower) {
auto upper_pac = db_.storage().label_property_index_.PositionAndCount(
key, upper.value().value());
return upper.value().IsInclusive() ? upper_pac.first + upper_pac.second
: upper_pac.first;
} else {
auto lower_pac = db_.storage().label_property_index_.PositionAndCount(
key, lower.value().value());
auto upper_pac = db_.storage().label_property_index_.PositionAndCount(
key, upper.value().value());
auto result = upper_pac.first - lower_pac.first;
if (lower.value().IsExclusive()) result -= lower_pac.second;
if (upper.value().IsInclusive()) result += upper_pac.second;
return std::max(0l, result);
}
}
bool GraphDbAccessor::RemoveVertex(VertexAccessor &vertex_accessor,
bool check_empty) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
vertex_accessor.SwitchNew();
// it's possible the vertex was removed already in this transaction
// due to it getting matched multiple times by some patterns
// we can only delete it once, so check if it's already deleted
if (vertex_accessor.current().is_expired_by(transaction_)) return true;
if (check_empty &&
vertex_accessor.out_degree() + vertex_accessor.in_degree() > 0)
return false;
auto *vlist_ptr = vertex_accessor.address();
wal().Emplace(database::StateDelta::RemoveVertex(
transaction_.id_, vlist_ptr->gid_, check_empty));
vlist_ptr->remove(vertex_accessor.current_, transaction_);
return true;
}
void GraphDbAccessor::DetachRemoveVertex(VertexAccessor &vertex_accessor) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
vertex_accessor.SwitchNew();
// Note that when we call RemoveEdge we must take care not to delete from the
// collection we are iterating over. This invalidates the iterator in a subtle
// way that does not fail in tests, but is NOT correct.
for (auto edge_accessor : vertex_accessor.in())
RemoveEdge(edge_accessor, true, false);
vertex_accessor.SwitchNew();
for (auto edge_accessor : vertex_accessor.out())
RemoveEdge(edge_accessor, false, true);
RemoveVertex(vertex_accessor, false);
}
EdgeAccessor GraphDbAccessor::InsertEdge(
VertexAccessor &from, VertexAccessor &to, storage::EdgeType edge_type,
std::experimental::optional<gid::Gid> requested_gid) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
auto gid = db_.storage().edge_generator_.Next(requested_gid);
auto edge_vlist = new mvcc::VersionList<Edge>(
transaction_, gid, from.address(), to.address(), edge_type);
// We need to insert edge_vlist to edges_ before calling update since update
// can throw and edge_vlist will not be garbage collected if it is not in
// edges_ skiplist.
bool success = db_.storage().edges_.access().insert(gid, edge_vlist).second;
CHECK(success) << "Attempting to insert an edge with an existing GID: "
<< gid;
// ensure that the "from" accessor has the latest version
from.SwitchNew();
from.update().out_.emplace(to.address(), edge_vlist, edge_type);
// ensure that the "to" accessor has the latest version (Switch new)
// WARNING: must do that after the above "from.update()" for cases when
// we are creating a cycle and "from" and "to" are the same vlist
to.SwitchNew();
to.update().in_.emplace(from.address(), edge_vlist, edge_type);
wal().Emplace(database::StateDelta::CreateEdge(
transaction_.id_, edge_vlist->gid_, from.gid(), to.gid(), edge_type,
EdgeTypeName(edge_type)));
return EdgeAccessor(edge_vlist, *this, from.address(), to.address(),
edge_type);
}
int64_t GraphDbAccessor::EdgesCount() const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.storage().edges_.access().size();
}
void GraphDbAccessor::RemoveEdge(EdgeAccessor &edge, bool remove_out_edge,
bool remove_in_edge) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
// it's possible the edge was removed already in this transaction
// due to it getting matched multiple times by some patterns
// we can only delete it once, so check if it's already deleted
edge.SwitchNew();
if (edge.current().is_expired_by(transaction_)) return;
if (remove_out_edge) edge.from().RemoveOutEdge(edge.address());
if (remove_in_edge) edge.to().RemoveInEdge(edge.address());
edge.address()->remove(edge.current_, transaction_);
wal().Emplace(database::StateDelta::RemoveEdge(transaction_.id_, edge.gid()));
}
storage::Label GraphDbAccessor::Label(const std::string &label_name) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.label_mapper().value_to_id(label_name);
}
const std::string &GraphDbAccessor::LabelName(storage::Label label) const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.label_mapper().id_to_value(label);
}
storage::EdgeType GraphDbAccessor::EdgeType(const std::string &edge_type_name) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.edge_type_mapper().value_to_id(edge_type_name);
}
const std::string &GraphDbAccessor::EdgeTypeName(
storage::EdgeType edge_type) const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.edge_type_mapper().id_to_value(edge_type);
}
storage::Property GraphDbAccessor::Property(const std::string &property_name) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.property_mapper().value_to_id(property_name);
}
const std::string &GraphDbAccessor::PropertyName(
storage::Property property) const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.property_mapper().id_to_value(property);
}
int64_t GraphDbAccessor::Counter(const std::string &name) {
return db_.counters().Get(name);
}
void GraphDbAccessor::CounterSet(const std::string &name, int64_t value) {
db_.counters().Set(name, value);
}
std::vector<std::string> GraphDbAccessor::IndexInfo() const {
std::vector<std::string> info;
for (storage::Label label : db_.storage().labels_index_.Keys()) {
info.emplace_back(":" + LabelName(label));
}
for (LabelPropertyIndex::Key key :
db_.storage().label_property_index_.Keys()) {
info.emplace_back(fmt::format(":{}({}){}", LabelName(key.label_),
PropertyName(key.property_),
key.unique_ ? " unique" : ""));
}
return info;
}
} // namespace database

View File

@ -0,0 +1,641 @@
/// @file
#pragma once
#include <experimental/optional>
#include <string>
#include <vector>
#include <glog/logging.h>
#include <cppitertools/filter.hpp>
#include <cppitertools/imap.hpp>
#include "database/single_node_ha/graph_db.hpp"
#include "storage/common/types.hpp"
#include "storage/single_node_ha/edge_accessor.hpp"
#include "storage/single_node_ha/vertex_accessor.hpp"
#include "transactions/transaction.hpp"
#include "transactions/type.hpp"
#include "utils/bound.hpp"
#include "utils/exceptions.hpp"
namespace database {
/** Thrown when inserting in an index with constraint. */
class IndexConstraintViolationException : public utils::BasicException {
using utils::BasicException::BasicException;
};
/** Thrown when creating an index which already exists. */
class IndexExistsException : public utils::BasicException {
using utils::BasicException::BasicException;
};
/** Thrown when creating an index which already exists. */
class IndexCreationOnWorkerException : public utils::BasicException {
using utils::BasicException::BasicException;
};
/// Thrown on concurrent index creation when the transaction engine fails to
/// start a new transaction.
class IndexTransactionException : public utils::BasicException {
using utils::BasicException::BasicException;
};
/**
* Base accessor for the database object: exposes functions for operating on the
* database. All the functions in this class should be self-sufficient: for
* example the function for creating a new Vertex should take care of all the
* book-keeping around the creation.
*/
class GraphDbAccessor {
// We need to make friends with this guys since they need to access private
// methods for updating indices.
// TODO: Rethink this, we have too much long-distance friendship complicating
// the code.
friend class ::RecordAccessor<Vertex>;
friend class ::VertexAccessor;
friend class GraphDb;
protected:
// Construction should only be done through GraphDb::Access function and
// concrete GraphDbAccessor type.
/// Creates a new accessor by starting a new transaction.
explicit GraphDbAccessor(GraphDb &db);
/// Creates an accessor for a running transaction.
GraphDbAccessor(GraphDb &db, tx::TransactionId tx_id);
GraphDbAccessor(GraphDb &db,
std::experimental::optional<tx::TransactionId> parent_tx);
public:
~GraphDbAccessor();
GraphDbAccessor(const GraphDbAccessor &other) = delete;
GraphDbAccessor(GraphDbAccessor &&other) = delete;
GraphDbAccessor &operator=(const GraphDbAccessor &other) = delete;
GraphDbAccessor &operator=(GraphDbAccessor &&other) = delete;
/**
* Creates a new Vertex and returns an accessor to it. If the ID is
* provided, the created Vertex will have that local ID, and the ID counter
* will be increased to it so collisions are avoided. This should only be used
* by durability recovery, normal vertex creation should not provide the ID.
*
* You should NOT make interleaved recovery and normal DB op calls to this
* function. Doing so will likely mess up the ID generation and crash MG.
* Always perform recovery only once, immediately when the database is
* created, before any transactional ops start.
*
* @param requested_gid The requested GID. Should only be provided when
* recovering from durability.
*
* @return See above.
*/
VertexAccessor InsertVertex(std::experimental::optional<gid::Gid>
requested_gid = std::experimental::nullopt);
/**
* Removes the vertex of the given accessor. If the vertex has any outgoing or
* incoming edges, it is not deleted. See `DetachRemoveVertex` if you want to
* remove a vertex regardless of connectivity.
*
* If the vertex has already been deleted by the current transaction+command,
* this function will not do anything and will return true.
*
* @param vertex_accessor Accessor to vertex.
* @param check_empty If the vertex should be checked for existing edges
* before deletion.
* @return If or not the vertex was deleted.
*/
bool RemoveVertex(VertexAccessor &vertex_accessor, bool check_empty = true);
/**
* Removes the vertex of the given accessor along with all it's outgoing
* and incoming connections.
*
* @param vertex_accessor Accessor to a vertex.
*/
void DetachRemoveVertex(VertexAccessor &vertex_accessor);
/**
* Obtains the vertex for the given ID. If there is no vertex for the given
* ID, or it's not visible to this accessor's transaction, nullopt is
* returned.
*
* @param gid - The GID of the sought vertex.
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
*/
std::experimental::optional<VertexAccessor> FindVertexOptional(
gid::Gid gid, bool current_state);
/**
* Obtains the vertex for the given ID. If there is no vertex for the given
* ID, or it's not visible to this accessor's transaction, MG is crashed
* using a CHECK.
*
* @param gid - The GID of the sought vertex.
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
*/
VertexAccessor FindVertex(gid::Gid gid, bool current_state);
/**
* Returns iterable over accessors to all the vertices in the graph
* visible to the current transaction.
*
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
*/
auto Vertices(bool current_state) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
// wrap version lists into accessors, which will look for visible versions
auto accessors = iter::imap(
[this](auto id_vlist) {
return VertexAccessor(id_vlist.second, *this);
},
db_.storage().vertices_.access());
// filter out the accessors not visible to the current transaction
return iter::filter(
[this, current_state](const VertexAccessor &accessor) {
return accessor.Visible(transaction(), current_state);
},
std::move(accessors));
}
/**
* Return VertexAccessors which contain the current label for the current
* transaction visibilty.
* @param label - label for which to return VertexAccessors
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
* @return iterable collection
*/
auto Vertices(storage::Label label, bool current_state) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return iter::imap(
[this](auto vlist) {
return VertexAccessor(vlist, *this);
},
db_.storage().labels_index_.GetVlists(label, transaction_,
current_state));
}
/**
* Return VertexAccessors which contain the current label and property for the
* given transaction visibility.
*
* @param label - label for which to return VertexAccessors
* @param property - property for which to return VertexAccessors
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
* @return iterable collection
*/
auto Vertices(storage::Label label, storage::Property property,
bool current_state) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
DCHECK(db_.storage().label_property_index_.IndexExists(
LabelPropertyIndex::Key(label, property)))
<< "Label+property index doesn't exist.";
return iter::imap(
[this](auto vlist) {
return VertexAccessor(vlist, *this);
},
db_.storage().label_property_index_.GetVlists(
LabelPropertyIndex::Key(label, property), transaction_,
current_state));
}
/**
* Return VertexAccessors which contain the current label + property, and
* those properties are equal to this 'value' for the given transaction
* visibility.
* @param label - label for which to return VertexAccessors
* @param property - property for which to return VertexAccessors
* @param value - property value for which to return VertexAccessors
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
* @return iterable collection
*/
auto Vertices(storage::Label label, storage::Property property,
const PropertyValue &value, bool current_state) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
DCHECK(db_.storage().label_property_index_.IndexExists(
LabelPropertyIndex::Key(label, property)))
<< "Label+property index doesn't exist.";
CHECK(value.type() != PropertyValue::Type::Null)
<< "Can't query index for propery value type null.";
return iter::imap(
[this](auto vlist) {
return VertexAccessor(vlist, *this);
},
db_.storage().label_property_index_.GetVlists(
LabelPropertyIndex::Key(label, property), value, transaction_,
current_state));
}
/**
* Return an iterable over VertexAccessors which contain the
* given label and whose property value (for the given property)
* falls within the given (lower, upper) @c Bound.
*
* The returned iterator will only contain
* vertices/edges whose property value is comparable with the
* given bounds (w.r.t. type). This has implications on Cypher
* query execuction semantics which have not been resovled yet.
*
* At least one of the bounds must be specified. Bonds can't be
* @c PropertyValue::Null. If both bounds are
* specified, their PropertyValue elments must be of comparable
* types.
*
* @param label - label for which to return VertexAccessors
* @param property - property for which to return VertexAccessors
* @param lower - Lower bound of the interval.
* @param upper - Upper bound of the interval.
* @param value - property value for which to return VertexAccessors
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
* @return iterable collection of record accessors
* satisfy the bounds and are visible to the current transaction.
*/
auto Vertices(
storage::Label label, storage::Property property,
const std::experimental::optional<utils::Bound<PropertyValue>> lower,
const std::experimental::optional<utils::Bound<PropertyValue>> upper,
bool current_state) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
DCHECK(db_.storage().label_property_index_.IndexExists(
LabelPropertyIndex::Key(label, property)))
<< "Label+property index doesn't exist.";
return iter::imap(
[this](auto vlist) {
return VertexAccessor(vlist, *this);
},
db_.storage().label_property_index_.GetVlists(
LabelPropertyIndex::Key(label, property), lower, upper,
transaction_, current_state));
}
/**
* Creates a new Edge and returns an accessor to it. If the ID is
* provided, the created Edge will have that ID, and the ID counter will be
* increased to it so collisions are avoided. This should only be used by
* durability recovery, normal edge creation should not provide the ID.
*
* You should NOT make interleaved recovery and normal DB op calls to this
* function. Doing so will likely mess up the ID generation and crash MG.
* Always perform recovery only once, immediately when the database is
* created, before any transactional ops start.
*
* @param from The 'from' vertex.
* @param to The 'to' vertex'
* @param type Edge type.
* @param requested_gid The requested GID. Should only be provided when
* recovering from durability.
*
* @return An accessor to the edge.
*/
EdgeAccessor InsertEdge(VertexAccessor &from, VertexAccessor &to,
storage::EdgeType type,
std::experimental::optional<gid::Gid> requested_gid =
std::experimental::nullopt);
/**
* Removes an edge from the graph. Parameters can indicate if the edge should
* be removed from data structures in vertices it connects. When removing an
* edge both arguments should be `true`. `false` is only used when
* detach-deleting a vertex.
*
* @param edge The accessor to an edge.
* @param remove_out_edge If the edge should be removed from the its origin
* side.
* @param remove_in_edge If the edge should be removed from the its
* destination side.
*/
void RemoveEdge(EdgeAccessor &edge, bool remove_out_edge = true,
bool remove_in_edge = true);
/**
* Obtains the edge for the given ID. If there is no edge for the given
* ID, or it's not visible to this accessor's transaction, nullopt is
* returned.
*
* @param gid - The GID of the sought edge.
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
*/
std::experimental::optional<EdgeAccessor> FindEdgeOptional(
gid::Gid gid, bool current_state);
/**
* Obtains the edge for the given ID. If there is no edge for the given
* ID, or it's not visible to this accessor's transaction, MG is crashed
* using a CHECK.
*
* @param gid - The GID of the sought edge.
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
*/
EdgeAccessor FindEdge(gid::Gid gid, bool current_state);
/**
* Returns iterable over accessors to all the edges in the graph
* visible to the current transaction.
*
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
*/
auto Edges(bool current_state) {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
// wrap version lists into accessors, which will look for visible versions
auto accessors = iter::imap(
[this](auto id_vlist) {
return EdgeAccessor(id_vlist.second, *this);
},
db_.storage().edges_.access());
// filter out the accessors not visible to the current transaction
return iter::filter(
[this, current_state](const EdgeAccessor &accessor) {
return accessor.Visible(transaction(), current_state);
},
std::move(accessors));
}
/**
* Creates and returns a new accessor that represents the same graph element
* (node / version) as the given `accessor`, but in this `GraphDbAccessor`.
*
* It is possible that the given `accessor` graph element is not visible in
* this `GraphDbAccessor`'s transaction. If that is the case, a `nullopt` is
* returned.
*
* The returned accessor does NOT have the same `current_` set as the given
* `accessor`. It has default post-construction `current_` set (`old` if
* available, otherwise `new`).
*
* @param accessor The [Vertex/Edge]Accessor whose underlying graph element we
* want in this GraphDbAccessor.
* @return See above.
* @tparam TAccessor Either VertexAccessor or EdgeAccessor
*/
template <typename TAccessor>
std::experimental::optional<TAccessor> Transfer(const TAccessor &accessor) {
if (accessor.db_accessor_ == this)
return std::experimental::make_optional(accessor);
TAccessor accessor_in_this(accessor.address(), *this);
if (accessor_in_this.current_)
return std::experimental::make_optional(std::move(accessor_in_this));
else
return std::experimental::nullopt;
}
/**
* Adds an index for the given (label, property) and populates it with
* existing vertices that belong to it.
*
* You should never call BuildIndex on a GraphDbAccessor (transaction) on
* which new vertices have been inserted or existing ones updated. Do it
* in a new accessor instead.
*
* Build index throws if an index for the given (label, property) already
* exists (even if it's being built by a concurrent transaction and is not yet
* ready for use).
*
* It also throws if there is another index being built concurrently on the
* same database this accessor is for.
*
* @param label - label to build for
* @param property - property to build for
*/
void BuildIndex(storage::Label label, storage::Property property,
bool unique);
/// Deletes the index responisble for (label, property).
///
/// @throws IndexTransactionException if it can't obtain a blocking
/// transaction.
void DeleteIndex(storage::Label label, storage::Property property);
/// Populates index with vertices containing the key
void PopulateIndex(const LabelPropertyIndex::Key &key);
/// Writes Index (key) creation to wal, marks it as ready for usage
void EnableIndex(const LabelPropertyIndex::Key &key);
/**
* @brief - Returns true if the given label+property index already exists and
* is ready for use.
*/
bool LabelPropertyIndexExists(storage::Label label,
storage::Property property) const {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.storage().label_property_index_.IndexExists(
LabelPropertyIndex::Key(label, property));
}
/**
* @brief - Returns vector of keys of label-property indices.
*/
std::vector<LabelPropertyIndex::Key> GetIndicesKeys() {
DCHECK(!commited_ && !aborted_) << "Accessor committed or aborted";
return db_.storage().label_property_index_.Keys();
}
/**
* Return approximate number of all vertices in the database.
* Note that this is always an over-estimate and never an under-estimate.
*/
int64_t VerticesCount() const;
/*
* Return approximate number of all edges in the database.
* Note that this is always an over-estimate and never an under-estimate.
*/
int64_t EdgesCount() const;
/**
* Return approximate number of vertices under indexes with the given label.
* Note that this is always an over-estimate and never an under-estimate.
*
* @param label - label to check for
* @return number of vertices with the given label
*/
int64_t VerticesCount(storage::Label label) const;
/**
* Return approximate number of vertices under indexes with the given label
* and property. Note that this is always an over-estimate and never an
* under-estimate.
*
* @param label - label to check for
* @param property - property to check for
* @return number of vertices with the given label, fails if no such
* label+property index exists.
*/
int64_t VerticesCount(storage::Label label, storage::Property property) const;
/**
* Returns approximate number of vertices that have the given label
* and the given value for the given property.
*
* Assumes that an index for that (label, property) exists.
*/
int64_t VerticesCount(storage::Label label, storage::Property property,
const PropertyValue &value) const;
/**
* Returns approximate number of vertices that have the given label
* and whose vaue is in the range defined by upper and lower @c Bound.
*
* At least one bound must be specified. Neither can be
* PropertyValue::Null.
*
* Assumes that an index for that (label, property) exists.
*/
int64_t VerticesCount(
storage::Label label, storage::Property property,
const std::experimental::optional<utils::Bound<PropertyValue>> lower,
const std::experimental::optional<utils::Bound<PropertyValue>> upper)
const;
/**
* Obtains the Label for the label's name.
* @return See above.
*/
storage::Label Label(const std::string &label_name);
/**
* Obtains the label name (a string) for the given label.
*
* @param label a Label.
* @return See above.
*/
const std::string &LabelName(storage::Label label) const;
/**
* Obtains the EdgeType for it's name.
* @return See above.
*/
storage::EdgeType EdgeType(const std::string &edge_type_name);
/**
* Obtains the edge type name (a string) for the given edge type.
*
* @param edge_type an EdgeType.
* @return See above.
*/
const std::string &EdgeTypeName(storage::EdgeType edge_type) const;
/**
* Obtains the Property for it's name.
* @return See above.
*/
storage::Property Property(const std::string &property_name);
/**
* Obtains the property name (a string) for the given property.
*
* @param property a Property.
* @return See above.
*/
const std::string &PropertyName(storage::Property property) const;
/** Returns the id of this accessor's transaction */
tx::TransactionId transaction_id() const;
/** Advances transaction's command id by 1. */
void AdvanceCommand();
/** Commit transaction. */
void Commit();
/** Abort transaction. */
void Abort();
/** Return true if transaction is hinted to abort. */
bool should_abort() const;
const tx::Transaction &transaction() const { return transaction_; }
durability::WriteAheadLog &wal();
auto &db() { return db_; }
const auto &db() const { return db_; }
/**
* Returns the current value of the counter with the given name, and
* increments that counter. If the counter with the given name does not exist,
* a new counter is created and this function returns 0.
*/
int64_t Counter(const std::string &name);
/**
* Sets the counter with the given name to the given value. Returns nothing.
* If the counter with the given name does not exist, a new counter is created
* and set to the given value.
*/
void CounterSet(const std::string &name, int64_t value);
/* Returns a list of index names present in the database. */
std::vector<std::string> IndexInfo() const;
/**
* Insert this vertex into corresponding label and label+property (if it
* exists) index.
*
* @param label - label with which to insert vertex label record
* @param vertex_accessor - vertex_accessor to insert
* @param vertex - vertex record to insert
*/
void UpdateLabelIndices(storage::Label label,
const VertexAccessor &vertex_accessor,
const Vertex *const vertex);
private:
GraphDb &db_;
tx::Transaction &transaction_;
// Indicates if this db-accessor started the transaction and should Abort it
// upon destruction.
bool transaction_starter_;
bool commited_{false};
bool aborted_{false};
/**
* Insert this vertex into corresponding any label + 'property' index.
* @param property - vertex will be inserted into indexes which contain this
* property
* @param vertex_accessor - vertex accessor to insert
* @param vertex - vertex to insert
*/
void UpdatePropertyIndex(storage::Property property,
const RecordAccessor<Vertex> &vertex_accessor,
const Vertex *const vertex);
};
} // namespace database

View File

@ -0,0 +1,90 @@
#include "durability/single_node_ha/paths.hpp"
#include <experimental/filesystem>
#include <experimental/optional>
#include <string>
#include "glog/logging.h"
#include "transactions/type.hpp"
#include "utils/string.hpp"
#include "utils/timestamp.hpp"
namespace durability {
namespace fs = std::experimental::filesystem;
std::experimental::optional<tx::TransactionId> TransactionIdFromWalFilename(
const std::string &name) {
auto nullopt = std::experimental::nullopt;
// Get the max_transaction_id from the file name that has format
// "XXXXX__max_transaction_<MAX_TRANS_ID>"
auto file_name_split = utils::RSplit(name, "__", 1);
if (file_name_split.size() != 2) {
LOG(WARNING) << "Unable to parse WAL file name: " << name;
return nullopt;
}
if (utils::StartsWith(file_name_split[1], "current"))
return std::numeric_limits<tx::TransactionId>::max();
file_name_split = utils::Split(file_name_split[1], "_");
if (file_name_split.size() != 3) {
LOG(WARNING) << "Unable to parse WAL file name: " << name;
return nullopt;
}
auto &tx_id_str = file_name_split[2];
try {
return std::stoll(tx_id_str);
} catch (std::invalid_argument &) {
LOG(WARNING) << "Unable to parse WAL file name tx ID: " << tx_id_str;
return nullopt;
} catch (std::out_of_range &) {
LOG(WARNING) << "WAL file name tx ID too large: " << tx_id_str;
return nullopt;
}
}
/// Generates a file path for a write-ahead log file. If given a transaction ID
/// the file name will contain it. Otherwise the file path is for the "current"
/// WAL file for which the max tx id is still unknown.
fs::path WalFilenameForTransactionId(
const std::experimental::filesystem::path &wal_dir,
std::experimental::optional<tx::TransactionId> tx_id) {
auto file_name = utils::Timestamp::Now().ToIso8601();
if (tx_id) {
file_name += "__max_transaction_" + std::to_string(*tx_id);
} else {
file_name += "__current";
}
return wal_dir / file_name;
}
fs::path MakeSnapshotPath(const fs::path &durability_dir,
tx::TransactionId tx_id) {
std::string date_str =
utils::Timestamp(utils::Timestamp::Now())
.ToString("{:04d}_{:02d}_{:02d}__{:02d}_{:02d}_{:02d}_{:05d}");
auto file_name = date_str + "_tx_" + std::to_string(tx_id);
return durability_dir / kSnapshotDir / file_name;
}
std::experimental::optional<tx::TransactionId>
TransactionIdFromSnapshotFilename(const std::string &name) {
auto nullopt = std::experimental::nullopt;
auto file_name_split = utils::RSplit(name, "_tx_", 1);
if (file_name_split.size() != 2) {
LOG(WARNING) << "Unable to parse snapshot file name: " << name;
return nullopt;
}
try {
return std::stoll(file_name_split[1]);
} catch (std::invalid_argument &) {
LOG(WARNING) << "Unable to parse snapshot file name tx ID: "
<< file_name_split[1];
return nullopt;
} catch (std::out_of_range &) {
LOG(WARNING) << "Unable to parse snapshot file name tx ID: "
<< file_name_split[1];
return nullopt;
}
}
} // namespace durability

View File

@ -0,0 +1,41 @@
#pragma once
#include <experimental/filesystem>
#include <experimental/optional>
#include "transactions/type.hpp"
namespace durability {
const std::string kSnapshotDir = "snapshots";
const std::string kWalDir = "wal";
const std::string kBackupDir = ".backup";
/// Returns the transaction id contained in the file name. If the filename is
/// not a parseable WAL file name, nullopt is returned. If the filename
/// represents the "current" WAL file, then the maximum possible transaction ID
/// is returned because that's appropriate for the recovery logic (the current
/// WAL does not yet have a maximum transaction ID and can't be discarded by
/// the recovery regardless of the snapshot from which the transaction starts).
std::experimental::optional<tx::TransactionId> TransactionIdFromWalFilename(
const std::string &name);
/// Generates a file path for a write-ahead log file. If given a transaction ID
/// the file name will contain it. Otherwise the file path is for the "current"
/// WAL file for which the max tx id is still unknown.
std::experimental::filesystem::path WalFilenameForTransactionId(
const std::experimental::filesystem::path &wal_dir,
std::experimental::optional<tx::TransactionId> tx_id =
std::experimental::nullopt);
/// Generates a path for a DB snapshot in the given folder in a well-defined
/// sortable format with transaction from which the snapshot is created appended
/// to the file name.
std::experimental::filesystem::path MakeSnapshotPath(
const std::experimental::filesystem::path &durability_dir,
tx::TransactionId tx_id);
/// Returns the transaction id contained in the file name. If the filename is
/// not a parseable WAL file name, nullopt is returned.
std::experimental::optional<tx::TransactionId>
TransactionIdFromSnapshotFilename(const std::string &name);
} // namespace durability

View File

@ -0,0 +1,444 @@
#include "durability/single_node_ha/recovery.hpp"
#include <experimental/filesystem>
#include <experimental/optional>
#include <limits>
#include <unordered_map>
#include "communication/bolt/v1/decoder/decoder.hpp"
#include "database/single_node_ha/graph_db_accessor.hpp"
#include "durability/hashed_file_reader.hpp"
#include "durability/single_node_ha/paths.hpp"
#include "durability/single_node_ha/version.hpp"
#include "durability/single_node_ha/wal.hpp"
#include "glue/communication.hpp"
#include "storage/single_node_ha/indexes/label_property_index.hpp"
#include "transactions/type.hpp"
#include "utils/algorithm.hpp"
#include "utils/file.hpp"
namespace fs = std::experimental::filesystem;
namespace durability {
using communication::bolt::Value;
bool ReadSnapshotSummary(HashedFileReader &buffer, int64_t &vertex_count,
int64_t &edge_count, uint64_t &hash) {
auto pos = buffer.Tellg();
auto offset = sizeof(vertex_count) + sizeof(edge_count) + sizeof(hash);
buffer.Seek(-offset, std::ios_base::end);
bool r_val = buffer.ReadType(vertex_count, false) &&
buffer.ReadType(edge_count, false) &&
buffer.ReadType(hash, false);
buffer.Seek(pos);
return r_val;
}
bool VersionConsistency(const fs::path &durability_dir) {
for (const auto &durability_type : {kSnapshotDir, kWalDir}) {
auto recovery_dir = durability_dir / durability_type;
if (!fs::exists(recovery_dir) || !fs::is_directory(recovery_dir)) continue;
for (const auto &file : fs::directory_iterator(recovery_dir)) {
HashedFileReader reader;
communication::bolt::Decoder<HashedFileReader> decoder(reader);
// The following checks are ok because we are only trying to detect
// version inconsistencies.
if (!reader.Open(fs::path(file))) continue;
std::array<uint8_t, 4> target_magic_number =
(durability_type == kSnapshotDir) ? durability::kSnapshotMagic
: durability::kWalMagic;
std::array<uint8_t, 4> magic_number;
if (!reader.Read(magic_number.data(), magic_number.size())) continue;
if (magic_number != target_magic_number) continue;
if (reader.EndOfFile()) continue;
Value dv;
if (!decoder.ReadValue(&dv, Value::Type::Int) ||
dv.ValueInt() != durability::kVersion)
return false;
}
}
return true;
}
bool DistributedVersionConsistency(const int64_t master_version) {
return durability::kVersion == master_version;
}
bool ContainsDurabilityFiles(const fs::path &durability_dir) {
for (const auto &durability_type : {kSnapshotDir, kWalDir}) {
auto recovery_dir = durability_dir / durability_type;
if (fs::exists(recovery_dir) && fs::is_directory(recovery_dir) &&
!fs::is_empty(recovery_dir))
return true;
}
return false;
}
void MoveToBackup(const fs::path &durability_dir) {
auto backup_dir = durability_dir / kBackupDir;
utils::CheckDir(backup_dir);
utils::CheckDir(backup_dir / kSnapshotDir);
utils::CheckDir(backup_dir / kWalDir);
for (const auto &durability_type : {kSnapshotDir, kWalDir}) {
auto recovery_dir = durability_dir / durability_type;
if (!fs::exists(recovery_dir) || !fs::is_directory(recovery_dir)) continue;
for (const auto &file : fs::directory_iterator(recovery_dir)) {
auto filename = fs::path(file).filename();
fs::rename(file, backup_dir / durability_type / filename);
}
}
}
namespace {
using communication::bolt::Value;
#define RETURN_IF_NOT(condition) \
if (!(condition)) { \
reader.Close(); \
return false; \
}
bool RecoverSnapshot(const fs::path &snapshot_file, database::GraphDb *db,
RecoveryData *recovery_data) {
HashedFileReader reader;
communication::bolt::Decoder<HashedFileReader> decoder(reader);
RETURN_IF_NOT(reader.Open(snapshot_file));
auto magic_number = durability::kSnapshotMagic;
reader.Read(magic_number.data(), magic_number.size());
RETURN_IF_NOT(magic_number == durability::kSnapshotMagic);
// Read the vertex and edge count, and the hash, from the end of the snapshot.
int64_t vertex_count;
int64_t edge_count;
uint64_t hash;
RETURN_IF_NOT(
durability::ReadSnapshotSummary(reader, vertex_count, edge_count, hash));
Value dv;
RETURN_IF_NOT(decoder.ReadValue(&dv, Value::Type::Int) &&
dv.ValueInt() == durability::kVersion);
RETURN_IF_NOT(decoder.ReadValue(&dv, Value::Type::Int));
recovery_data->snapshooter_tx_id = dv.ValueInt();
// Transaction snapshot of the transaction that created the snapshot.
RETURN_IF_NOT(decoder.ReadValue(&dv, Value::Type::List));
for (const auto &value : dv.ValueList()) {
RETURN_IF_NOT(value.IsInt());
recovery_data->snapshooter_tx_snapshot.emplace_back(value.ValueInt());
}
// A list of label+property indexes.
RETURN_IF_NOT(decoder.ReadValue(&dv, Value::Type::List));
auto index_value = dv.ValueList();
for (auto it = index_value.begin(); it != index_value.end();) {
auto label = *it++;
RETURN_IF_NOT(it != index_value.end());
auto property = *it++;
RETURN_IF_NOT(it != index_value.end());
auto unique = *it++;
RETURN_IF_NOT(label.IsString() && property.IsString() && unique.IsBool());
recovery_data->indexes.emplace_back(
IndexRecoveryData{label.ValueString(), property.ValueString(),
/*create = */ true, unique.ValueBool()});
}
auto dba = db->Access();
std::unordered_map<uint64_t, VertexAccessor> vertices;
for (int64_t i = 0; i < vertex_count; ++i) {
Value vertex_dv;
RETURN_IF_NOT(decoder.ReadValue(&vertex_dv, Value::Type::Vertex));
auto &vertex = vertex_dv.ValueVertex();
auto vertex_accessor = dba->InsertVertex(vertex.id.AsUint());
for (const auto &label : vertex.labels) {
vertex_accessor.add_label(dba->Label(label));
}
for (const auto &property_pair : vertex.properties) {
vertex_accessor.PropsSet(dba->Property(property_pair.first),
glue::ToPropertyValue(property_pair.second));
}
vertices.insert({vertex.id.AsUint(), vertex_accessor});
}
for (int64_t i = 0; i < edge_count; ++i) {
Value edge_dv;
RETURN_IF_NOT(decoder.ReadValue(&edge_dv, Value::Type::Edge));
auto &edge = edge_dv.ValueEdge();
auto it_from = vertices.find(edge.from.AsUint());
auto it_to = vertices.find(edge.to.AsUint());
RETURN_IF_NOT(it_from != vertices.end() && it_to != vertices.end());
auto edge_accessor =
dba->InsertEdge(it_from->second, it_to->second,
dba->EdgeType(edge.type), edge.id.AsUint());
for (const auto &property_pair : edge.properties)
edge_accessor.PropsSet(dba->Property(property_pair.first),
glue::ToPropertyValue(property_pair.second));
}
// Vertex and edge counts are included in the hash. Re-read them to update the
// hash.
reader.ReadType(vertex_count);
reader.ReadType(edge_count);
if (!reader.Close() || reader.hash() != hash) {
dba->Abort();
return false;
}
// Ensure that the next transaction ID in the recovered DB will be greater
// than the latest one we have recovered. Do this to make sure that
// subsequently created snapshots and WAL files will have transactional info
// that does not interfere with that found in previous snapshots and WAL.
tx::TransactionId max_id = recovery_data->snapshooter_tx_id;
auto &snap = recovery_data->snapshooter_tx_snapshot;
if (!snap.empty()) max_id = *std::max_element(snap.begin(), snap.end());
dba->db().tx_engine().EnsureNextIdGreater(max_id);
dba->Commit();
return true;
}
#undef RETURN_IF_NOT
std::vector<fs::path> GetWalFiles(const fs::path &wal_dir) {
// Get paths to all the WAL files and sort them (on date).
std::vector<fs::path> wal_files;
if (!fs::exists(wal_dir)) return {};
for (auto &wal_file : fs::directory_iterator(wal_dir))
wal_files.emplace_back(wal_file);
std::sort(wal_files.begin(), wal_files.end());
return wal_files;
}
bool ApplyOverDeltas(
const std::vector<fs::path> &wal_files, tx::TransactionId first_to_recover,
const std::function<void(const database::StateDelta &)> &f) {
for (auto &wal_file : wal_files) {
auto wal_file_max_tx_id = TransactionIdFromWalFilename(wal_file.filename());
if (!wal_file_max_tx_id || *wal_file_max_tx_id < first_to_recover) continue;
HashedFileReader wal_reader;
if (!wal_reader.Open(wal_file)) return false;
communication::bolt::Decoder<HashedFileReader> decoder(wal_reader);
auto magic_number = durability::kWalMagic;
wal_reader.Read(magic_number.data(), magic_number.size());
if (magic_number != durability::kWalMagic) return false;
Value dv;
if (!decoder.ReadValue(&dv, Value::Type::Int) ||
dv.ValueInt() != durability::kVersion)
return false;
while (true) {
auto delta = database::StateDelta::Decode(wal_reader, decoder);
if (!delta) break;
f(*delta);
}
}
return true;
}
auto FirstWalTxToRecover(const RecoveryData &recovery_data) {
auto &tx_sn = recovery_data.snapshooter_tx_snapshot;
auto first_to_recover = tx_sn.empty() ? recovery_data.snapshooter_tx_id + 1
: *std::min(tx_sn.begin(), tx_sn.end());
return first_to_recover;
}
std::vector<tx::TransactionId> ReadWalRecoverableTransactions(
const fs::path &wal_dir, database::GraphDb *db,
const RecoveryData &recovery_data) {
auto wal_files = GetWalFiles(wal_dir);
std::unordered_set<tx::TransactionId> committed_set;
auto first_to_recover = FirstWalTxToRecover(recovery_data);
ApplyOverDeltas(
wal_files, first_to_recover, [&](const database::StateDelta &delta) {
if (delta.transaction_id >= first_to_recover &&
delta.type == database::StateDelta::Type::TRANSACTION_COMMIT) {
committed_set.insert(delta.transaction_id);
}
});
std::vector<tx::TransactionId> committed_tx_ids(committed_set.size());
for (auto id : committed_set) committed_tx_ids.push_back(id);
return committed_tx_ids;
}
} // anonymous namespace
RecoveryInfo RecoverOnlySnapshot(
const fs::path &durability_dir, database::GraphDb *db,
RecoveryData *recovery_data,
std::experimental::optional<tx::TransactionId> required_snapshot_tx_id) {
// Attempt to recover from snapshot files in reverse order (from newest
// backwards).
const auto snapshot_dir = durability_dir / kSnapshotDir;
std::vector<fs::path> snapshot_files;
if (fs::exists(snapshot_dir) && fs::is_directory(snapshot_dir))
for (auto &file : fs::directory_iterator(snapshot_dir))
snapshot_files.emplace_back(file);
std::sort(snapshot_files.rbegin(), snapshot_files.rend());
for (auto &snapshot_file : snapshot_files) {
if (required_snapshot_tx_id) {
auto snapshot_file_tx_id =
TransactionIdFromSnapshotFilename(snapshot_file);
if (!snapshot_file_tx_id ||
snapshot_file_tx_id.value() != *required_snapshot_tx_id) {
LOG(INFO) << "Skipping snapshot file '" << snapshot_file
<< "' because it does not match the required snapshot tx id: "
<< *required_snapshot_tx_id;
continue;
}
}
LOG(INFO) << "Starting snapshot recovery from: " << snapshot_file;
if (!RecoverSnapshot(snapshot_file, db, recovery_data)) {
db->ReinitializeStorage();
recovery_data->Clear();
LOG(WARNING) << "Snapshot recovery failed, trying older snapshot...";
continue;
} else {
LOG(INFO) << "Snapshot recovery successful.";
break;
}
}
// If snapshot recovery is required, and we failed, don't even deal with
// the WAL recovery.
if (required_snapshot_tx_id &&
recovery_data->snapshooter_tx_id != *required_snapshot_tx_id)
return {durability::kVersion, recovery_data->snapshooter_tx_id, {}};
return {durability::kVersion, recovery_data->snapshooter_tx_id,
ReadWalRecoverableTransactions(durability_dir / kWalDir, db,
*recovery_data)};
}
RecoveryTransactions::RecoveryTransactions(database::GraphDb *db) : db_(db) {}
void RecoveryTransactions::Begin(const tx::TransactionId &tx_id) {
CHECK(accessors_.find(tx_id) == accessors_.end())
<< "Double transaction start";
accessors_.emplace(tx_id, db_->Access());
}
void RecoveryTransactions::Abort(const tx::TransactionId &tx_id) {
GetAccessor(tx_id)->Abort();
accessors_.erase(accessors_.find(tx_id));
}
void RecoveryTransactions::Commit(const tx::TransactionId &tx_id) {
GetAccessor(tx_id)->Commit();
accessors_.erase(accessors_.find(tx_id));
}
void RecoveryTransactions::Apply(const database::StateDelta &delta) {
delta.Apply(*GetAccessor(delta.transaction_id));
}
database::GraphDbAccessor *RecoveryTransactions::GetAccessor(
const tx::TransactionId &tx_id) {
auto found = accessors_.find(tx_id);
CHECK(found != accessors_.end())
<< "Accessor does not exist for transaction: " << tx_id;
return found->second.get();
}
// TODO - finer-grained recovery feedback could be useful here.
void RecoverWal(const fs::path &durability_dir, database::GraphDb *db,
RecoveryData *recovery_data,
RecoveryTransactions *transactions) {
auto wal_dir = durability_dir / kWalDir;
auto wal_files = GetWalFiles(wal_dir);
// Track which transaction should be recovered first, and define logic for
// which transactions should be skipped in recovery.
auto &tx_sn = recovery_data->snapshooter_tx_snapshot;
auto first_to_recover = FirstWalTxToRecover(*recovery_data);
// Set of transactions which can be recovered, since not every transaction in
// wal can be recovered because it might not be present on some workers (there
// wasn't enough time for it to flush to disk or similar)
std::unordered_set<tx::TransactionId> common_wal_tx;
for (auto tx_id : recovery_data->wal_tx_to_recover)
common_wal_tx.insert(tx_id);
auto should_skip = [&tx_sn, recovery_data, &common_wal_tx,
first_to_recover](tx::TransactionId tx_id) {
return tx_id < first_to_recover ||
(tx_id < recovery_data->snapshooter_tx_id &&
!utils::Contains(tx_sn, tx_id)) ||
!utils::Contains(common_wal_tx, tx_id);
};
// Ensure that the next transaction ID in the recovered DB will be greater
// than the latest one we have recovered. Do this to make sure that
// subsequently created snapshots and WAL files will have transactional info
// that does not interfere with that found in previous snapshots and WAL.
tx::TransactionId max_observed_tx_id{0};
// Read all the WAL files whose max_tx_id is not smaller than
// min_tx_to_recover.
ApplyOverDeltas(
wal_files, first_to_recover, [&](const database::StateDelta &delta) {
max_observed_tx_id = std::max(max_observed_tx_id, delta.transaction_id);
if (should_skip(delta.transaction_id)) return;
switch (delta.type) {
case database::StateDelta::Type::TRANSACTION_BEGIN:
transactions->Begin(delta.transaction_id);
break;
case database::StateDelta::Type::TRANSACTION_ABORT:
transactions->Abort(delta.transaction_id);
break;
case database::StateDelta::Type::TRANSACTION_COMMIT:
transactions->Commit(delta.transaction_id);
break;
case database::StateDelta::Type::BUILD_INDEX:
// TODO index building might still be problematic in HA
recovery_data->indexes.emplace_back(
IndexRecoveryData{delta.label_name, delta.property_name,
/*create = */ true, delta.unique});
break;
case database::StateDelta::Type::DROP_INDEX:
recovery_data->indexes.emplace_back(
IndexRecoveryData{delta.label_name, delta.property_name,
/*create = */ false});
break;
default:
transactions->Apply(delta);
}
});
// TODO when implementing proper error handling return one of the following:
// - WAL fully recovered
// - WAL partially recovered
// - WAL recovery error
db->tx_engine().EnsureNextIdGreater(max_observed_tx_id);
}
void RecoverIndexes(database::GraphDb *db,
const std::vector<IndexRecoveryData> &indexes) {
auto dba = db->Access();
for (const auto &index : indexes) {
auto label = dba->Label(index.label);
auto property = dba->Property(index.property);
if (index.create) {
dba->BuildIndex(label, property, index.unique);
} else {
dba->DeleteIndex(label, property);
}
}
dba->Commit();
}
} // namespace durability

View File

@ -0,0 +1,151 @@
#pragma once
#include <experimental/filesystem>
#include <experimental/optional>
#include <unordered_map>
#include <vector>
#include "durability/hashed_file_reader.hpp"
#include "durability/single_node_ha/state_delta.hpp"
#include "transactions/type.hpp"
namespace database {
class GraphDb;
};
namespace durability {
/// Stores info on what was (or needs to be) recovered from durability.
struct RecoveryInfo {
RecoveryInfo() {}
RecoveryInfo(const int64_t durability_version,
tx::TransactionId snapshot_tx_id,
const std::vector<tx::TransactionId> &wal_recovered)
: durability_version(durability_version),
snapshot_tx_id(snapshot_tx_id),
wal_recovered(wal_recovered) {}
int64_t durability_version;
tx::TransactionId snapshot_tx_id;
std::vector<tx::TransactionId> wal_recovered;
bool operator==(const RecoveryInfo &other) const {
return durability_version == other.durability_version &&
snapshot_tx_id == other.snapshot_tx_id &&
wal_recovered == other.wal_recovered;
}
bool operator!=(const RecoveryInfo &other) const { return !(*this == other); }
};
struct IndexRecoveryData {
std::string label;
std::string property;
bool create; // distinguish between creating and dropping index
bool unique; // used only when creating an index
};
// A data structure for exchanging info between main recovery function and
// snapshot and WAL recovery functions.
struct RecoveryData {
tx::TransactionId snapshooter_tx_id{0};
std::vector<tx::TransactionId> wal_tx_to_recover{};
std::vector<tx::TransactionId> snapshooter_tx_snapshot;
// A collection into which the indexes should be added so they
// can be rebuilt at the end of the recovery transaction.
std::vector<IndexRecoveryData> indexes;
void Clear() {
snapshooter_tx_id = 0;
snapshooter_tx_snapshot.clear();
indexes.clear();
}
};
/** Reads snapshot metadata from the end of the file without messing up the
* hash. */
bool ReadSnapshotSummary(HashedFileReader &buffer, int64_t &vertex_count,
int64_t &edge_count, uint64_t &hash);
/**
* Checks version consistency within the durability directory.
*
* @param durability_dir - Path to durability directory.
* @return - True if snapshot and WAL versions are compatible with
* ` current memgraph binary.
*/
bool VersionConsistency(
const std::experimental::filesystem::path &durability_dir);
/**
* Checks whether the current memgraph binary (on a worker) is
* version consistent with the cluster master.
*
* @param master_version - Version of the master.
* @return - True if versions match.
*/
bool DistributedVersionConsistency(const int64_t master_version);
/**
* Checks whether the durability directory contains snapshot
* or write-ahead log file.
*
* @param durability_dir - Path to durability directory.
* @return - True if durability directory contains either a snapshot
* or WAL file.
*/
bool ContainsDurabilityFiles(
const std::experimental::filesystem::path &durabilty_dir);
/**
* Backup snapshots and WAL files to a backup folder.
*
* @param durability_dir - Path to durability directory.
*/
void MoveToBackup(const std::experimental::filesystem::path &durability_dir);
/**
* Recovers database from the latest possible snapshot. If recovering fails,
* false is returned and db_accessor aborts transaction, else true is returned
* and transaction is commited.
*
* @param durability_dir - Path to durability directory.
* @param db - The database to recover into.
* @param required_snapshot_tx_id - Only used on distributed worker. Indicates
* what the master recovered. The same snapshot must be recovered on the
* worker.
* @return - recovery info
*/
RecoveryInfo RecoverOnlySnapshot(
const std::experimental::filesystem::path &durability_dir,
database::GraphDb *db, durability::RecoveryData *recovery_data,
std::experimental::optional<tx::TransactionId> required_snapshot_tx_id);
/** Interface for accessing transactions during WAL recovery. */
class RecoveryTransactions {
public:
explicit RecoveryTransactions(database::GraphDb *db);
void Begin(const tx::TransactionId &tx_id);
void Abort(const tx::TransactionId &tx_id);
void Commit(const tx::TransactionId &tx_id);
void Apply(const database::StateDelta &delta);
private:
database::GraphDbAccessor *GetAccessor(const tx::TransactionId &tx_id);
database::GraphDb *db_;
std::unordered_map<tx::TransactionId,
std::unique_ptr<database::GraphDbAccessor>>
accessors_;
};
void RecoverWal(const std::experimental::filesystem::path &durability_dir,
database::GraphDb *db, RecoveryData *recovery_data,
RecoveryTransactions *transactions);
void RecoverIndexes(database::GraphDb *db,
const std::vector<IndexRecoveryData> &indexes);
} // namespace durability

View File

@ -0,0 +1,133 @@
#include "durability/single_node_ha/snapshooter.hpp"
#include <algorithm>
#include <glog/logging.h>
#include "communication/bolt/v1/encoder/base_encoder.hpp"
#include "database/single_node_ha/graph_db_accessor.hpp"
#include "durability/hashed_file_writer.hpp"
#include "durability/single_node_ha/paths.hpp"
#include "durability/single_node_ha/version.hpp"
#include "glue/communication.hpp"
#include "utils/file.hpp"
namespace fs = std::experimental::filesystem;
namespace durability {
// Snapshot layout is described in durability/version.hpp
static_assert(durability::kVersion == 7,
"Wrong snapshot version, please update!");
namespace {
bool Encode(const fs::path &snapshot_file, database::GraphDb &db,
database::GraphDbAccessor &dba) {
try {
HashedFileWriter buffer(snapshot_file);
communication::bolt::BaseEncoder<HashedFileWriter> encoder(buffer);
int64_t vertex_num = 0, edge_num = 0;
encoder.WriteRAW(durability::kSnapshotMagic.data(),
durability::kSnapshotMagic.size());
encoder.WriteInt(durability::kVersion);
// Write the ID of the transaction doing the snapshot.
encoder.WriteInt(dba.transaction_id());
// Write the transaction snapshot into the snapshot. It's used when
// recovering from the combination of snapshot and write-ahead-log.
{
std::vector<communication::bolt::Value> tx_snapshot;
for (int64_t tx : dba.transaction().snapshot())
tx_snapshot.emplace_back(tx);
encoder.WriteList(tx_snapshot);
}
// Write label+property indexes as list ["label", "property", ...]
{
std::vector<communication::bolt::Value> index_vec;
for (const auto &key : dba.GetIndicesKeys()) {
index_vec.emplace_back(dba.LabelName(key.label_));
index_vec.emplace_back(dba.PropertyName(key.property_));
index_vec.emplace_back(key.unique_);
}
encoder.WriteList(index_vec);
}
for (const auto &vertex : dba.Vertices(false)) {
encoder.WriteVertex(glue::ToBoltVertex(vertex));
vertex_num++;
}
for (const auto &edge : dba.Edges(false)) {
encoder.WriteEdge(glue::ToBoltEdge(edge));
edge_num++;
}
buffer.WriteValue(vertex_num);
buffer.WriteValue(edge_num);
buffer.WriteValue(buffer.hash());
buffer.Close();
} catch (const std::ifstream::failure &) {
if (fs::exists(snapshot_file) && !fs::remove(snapshot_file)) {
LOG(ERROR) << "Error while removing corrupted snapshot file: "
<< snapshot_file;
}
return false;
}
return true;
}
// Removes snapshot files so that only `max_retained` latest ones are kept. If
// `max_retained == -1`, all the snapshots are retained.
void RemoveOldSnapshots(const fs::path &snapshot_dir, int max_retained) {
if (max_retained == -1) return;
std::vector<fs::path> files;
for (auto &file : fs::directory_iterator(snapshot_dir))
files.push_back(file.path());
if (static_cast<int>(files.size()) <= max_retained) return;
sort(files.begin(), files.end());
for (int i = 0; i < static_cast<int>(files.size()) - max_retained; ++i) {
if (!fs::remove(files[i])) {
LOG(ERROR) << "Error while removing file: " << files[i];
}
}
}
// Removes write-ahead log files that are no longer necessary (they don't get
// used when recovering from the latest snapshot.
void RemoveOldWals(const fs::path &wal_dir,
const tx::Transaction &snapshot_transaction) {
if (!fs::exists(wal_dir)) return;
// We can remove all the WAL files that will not be used when restoring from
// the snapshot created in the given transaction.
auto min_trans_id = snapshot_transaction.snapshot().empty()
? snapshot_transaction.id_ + 1
: snapshot_transaction.snapshot().front();
for (auto &wal_file : fs::directory_iterator(wal_dir)) {
auto tx_id = TransactionIdFromWalFilename(wal_file.path().filename());
if (tx_id && tx_id.value() < min_trans_id) {
bool result = fs::remove(wal_file);
DCHECK(result) << "Unable to delete old wal file: " << wal_file;
}
}
}
} // namespace
bool MakeSnapshot(database::GraphDb &db, database::GraphDbAccessor &dba,
const fs::path &durability_dir, int snapshot_max_retained) {
if (!utils::EnsureDir(durability_dir / kSnapshotDir)) return false;
const auto snapshot_file =
MakeSnapshotPath(durability_dir, dba.transaction_id());
if (fs::exists(snapshot_file)) return false;
if (Encode(snapshot_file, db, dba)) {
RemoveOldSnapshots(durability_dir / kSnapshotDir, snapshot_max_retained);
RemoveOldWals(durability_dir / kWalDir, dba.transaction());
return true;
} else {
std::error_code error_code; // Just for exception suppression.
fs::remove(snapshot_file, error_code);
return false;
}
}
} // namespace durability

View File

@ -0,0 +1,20 @@
#pragma once
#include <experimental/filesystem>
#include "database/single_node_ha/graph_db.hpp"
namespace durability {
/**
* Make snapshot and save it in snapshots folder. Returns true if successful.
* @param db - database for which we are creating a snapshot
* @param dba - db accessor with which we are creating a snapshot (reading data)
* @param durability_dir - directory where durability data is stored.
* @param snapshot_max_retained - maximum number of snapshots to retain.
*/
bool MakeSnapshot(database::GraphDb &db, database::GraphDbAccessor &dba,
const std::experimental::filesystem::path &durability_dir,
int snapshot_max_retained);
} // namespace durability

View File

@ -0,0 +1,340 @@
#include "durability/single_node_ha/state_delta.hpp"
#include <string>
#include "communication/bolt/v1/value.hpp"
#include "database/single_node_ha/graph_db_accessor.hpp"
#include "glue/communication.hpp"
namespace database {
StateDelta StateDelta::TxBegin(tx::TransactionId tx_id) {
return {StateDelta::Type::TRANSACTION_BEGIN, tx_id};
}
StateDelta StateDelta::TxCommit(tx::TransactionId tx_id) {
return {StateDelta::Type::TRANSACTION_COMMIT, tx_id};
}
StateDelta StateDelta::TxAbort(tx::TransactionId tx_id) {
return {StateDelta::Type::TRANSACTION_ABORT, tx_id};
}
StateDelta StateDelta::CreateVertex(tx::TransactionId tx_id,
gid::Gid vertex_id) {
StateDelta op(StateDelta::Type::CREATE_VERTEX, tx_id);
op.vertex_id = vertex_id;
return op;
}
StateDelta StateDelta::CreateEdge(tx::TransactionId tx_id, gid::Gid edge_id,
gid::Gid vertex_from_id,
gid::Gid vertex_to_id,
storage::EdgeType edge_type,
const std::string &edge_type_name) {
StateDelta op(StateDelta::Type::CREATE_EDGE, tx_id);
op.edge_id = edge_id;
op.vertex_from_id = vertex_from_id;
op.vertex_to_id = vertex_to_id;
op.edge_type = edge_type;
op.edge_type_name = edge_type_name;
return op;
}
StateDelta StateDelta::PropsSetVertex(tx::TransactionId tx_id,
gid::Gid vertex_id,
storage::Property property,
const std::string &property_name,
const PropertyValue &value) {
StateDelta op(StateDelta::Type::SET_PROPERTY_VERTEX, tx_id);
op.vertex_id = vertex_id;
op.property = property;
op.property_name = property_name;
op.value = value;
return op;
}
StateDelta StateDelta::PropsSetEdge(tx::TransactionId tx_id, gid::Gid edge_id,
storage::Property property,
const std::string &property_name,
const PropertyValue &value) {
StateDelta op(StateDelta::Type::SET_PROPERTY_EDGE, tx_id);
op.edge_id = edge_id;
op.property = property;
op.property_name = property_name;
op.value = value;
return op;
}
StateDelta StateDelta::AddLabel(tx::TransactionId tx_id, gid::Gid vertex_id,
storage::Label label,
const std::string &label_name) {
StateDelta op(StateDelta::Type::ADD_LABEL, tx_id);
op.vertex_id = vertex_id;
op.label = label;
op.label_name = label_name;
return op;
}
StateDelta StateDelta::RemoveLabel(tx::TransactionId tx_id, gid::Gid vertex_id,
storage::Label label,
const std::string &label_name) {
StateDelta op(StateDelta::Type::REMOVE_LABEL, tx_id);
op.vertex_id = vertex_id;
op.label = label;
op.label_name = label_name;
return op;
}
StateDelta StateDelta::RemoveVertex(tx::TransactionId tx_id, gid::Gid vertex_id,
bool check_empty) {
StateDelta op(StateDelta::Type::REMOVE_VERTEX, tx_id);
op.vertex_id = vertex_id;
op.check_empty = check_empty;
return op;
}
StateDelta StateDelta::RemoveEdge(tx::TransactionId tx_id, gid::Gid edge_id) {
StateDelta op(StateDelta::Type::REMOVE_EDGE, tx_id);
op.edge_id = edge_id;
return op;
}
StateDelta StateDelta::BuildIndex(tx::TransactionId tx_id, storage::Label label,
const std::string &label_name,
storage::Property property,
const std::string &property_name,
bool unique) {
StateDelta op(StateDelta::Type::BUILD_INDEX, tx_id);
op.label = label;
op.label_name = label_name;
op.property = property;
op.property_name = property_name;
op.unique = unique;
return op;
}
StateDelta StateDelta::DropIndex(tx::TransactionId tx_id, storage::Label label,
const std::string &label_name,
storage::Property property,
const std::string &property_name) {
StateDelta op(StateDelta::Type::DROP_INDEX, tx_id);
op.label = label;
op.label_name = label_name;
op.property = property;
op.property_name = property_name;
return op;
}
void StateDelta::Encode(
HashedFileWriter &writer,
communication::bolt::BaseEncoder<HashedFileWriter> &encoder) const {
encoder.WriteInt(static_cast<int64_t>(type));
encoder.WriteInt(static_cast<int64_t>(transaction_id));
switch (type) {
case Type::TRANSACTION_BEGIN:
case Type::TRANSACTION_COMMIT:
case Type::TRANSACTION_ABORT:
break;
case Type::CREATE_VERTEX:
encoder.WriteInt(vertex_id);
break;
case Type::CREATE_EDGE:
encoder.WriteInt(edge_id);
encoder.WriteInt(vertex_from_id);
encoder.WriteInt(vertex_to_id);
encoder.WriteInt(edge_type.Id());
encoder.WriteString(edge_type_name);
break;
case Type::SET_PROPERTY_VERTEX:
encoder.WriteInt(vertex_id);
encoder.WriteInt(property.Id());
encoder.WriteString(property_name);
encoder.WriteValue(glue::ToBoltValue(value));
break;
case Type::SET_PROPERTY_EDGE:
encoder.WriteInt(edge_id);
encoder.WriteInt(property.Id());
encoder.WriteString(property_name);
encoder.WriteValue(glue::ToBoltValue(value));
break;
case Type::ADD_LABEL:
case Type::REMOVE_LABEL:
encoder.WriteInt(vertex_id);
encoder.WriteInt(label.Id());
encoder.WriteString(label_name);
break;
case Type::REMOVE_VERTEX:
encoder.WriteInt(vertex_id);
break;
case Type::REMOVE_EDGE:
encoder.WriteInt(edge_id);
break;
case Type::BUILD_INDEX:
encoder.WriteInt(label.Id());
encoder.WriteString(label_name);
encoder.WriteInt(property.Id());
encoder.WriteString(property_name);
encoder.WriteBool(unique);
break;
case Type::DROP_INDEX:
encoder.WriteInt(label.Id());
encoder.WriteString(label_name);
encoder.WriteInt(property.Id());
encoder.WriteString(property_name);
break;
}
writer.WriteValue(writer.hash());
}
#define DECODE_MEMBER(member, value_f) \
if (!decoder.ReadValue(&dv)) return nullopt; \
r_val.member = dv.value_f();
#define DECODE_MEMBER_CAST(member, value_f, type) \
if (!decoder.ReadValue(&dv)) return nullopt; \
r_val.member = static_cast<type>(dv.value_f());
std::experimental::optional<StateDelta> StateDelta::Decode(
HashedFileReader &reader,
communication::bolt::Decoder<HashedFileReader> &decoder) {
using std::experimental::nullopt;
StateDelta r_val;
// The decoded value used as a temporary while decoding.
communication::bolt::Value dv;
try {
if (!decoder.ReadValue(&dv)) return nullopt;
r_val.type = static_cast<enum StateDelta::Type>(dv.ValueInt());
DECODE_MEMBER(transaction_id, ValueInt)
switch (r_val.type) {
case Type::TRANSACTION_BEGIN:
case Type::TRANSACTION_COMMIT:
case Type::TRANSACTION_ABORT:
break;
case Type::CREATE_VERTEX:
DECODE_MEMBER(vertex_id, ValueInt)
break;
case Type::CREATE_EDGE:
DECODE_MEMBER(edge_id, ValueInt)
DECODE_MEMBER(vertex_from_id, ValueInt)
DECODE_MEMBER(vertex_to_id, ValueInt)
DECODE_MEMBER_CAST(edge_type, ValueInt, storage::EdgeType)
DECODE_MEMBER(edge_type_name, ValueString)
break;
case Type::SET_PROPERTY_VERTEX:
DECODE_MEMBER(vertex_id, ValueInt)
DECODE_MEMBER_CAST(property, ValueInt, storage::Property)
DECODE_MEMBER(property_name, ValueString)
if (!decoder.ReadValue(&dv)) return nullopt;
r_val.value = glue::ToPropertyValue(dv);
break;
case Type::SET_PROPERTY_EDGE:
DECODE_MEMBER(edge_id, ValueInt)
DECODE_MEMBER_CAST(property, ValueInt, storage::Property)
DECODE_MEMBER(property_name, ValueString)
if (!decoder.ReadValue(&dv)) return nullopt;
r_val.value = glue::ToPropertyValue(dv);
break;
case Type::ADD_LABEL:
case Type::REMOVE_LABEL:
DECODE_MEMBER(vertex_id, ValueInt)
DECODE_MEMBER_CAST(label, ValueInt, storage::Label)
DECODE_MEMBER(label_name, ValueString)
break;
case Type::REMOVE_VERTEX:
DECODE_MEMBER(vertex_id, ValueInt)
break;
case Type::REMOVE_EDGE:
DECODE_MEMBER(edge_id, ValueInt)
break;
case Type::BUILD_INDEX:
DECODE_MEMBER_CAST(label, ValueInt, storage::Label)
DECODE_MEMBER(label_name, ValueString)
DECODE_MEMBER_CAST(property, ValueInt, storage::Property)
DECODE_MEMBER(property_name, ValueString)
DECODE_MEMBER(unique, ValueBool)
break;
case Type::DROP_INDEX:
DECODE_MEMBER_CAST(label, ValueInt, storage::Label)
DECODE_MEMBER(label_name, ValueString)
DECODE_MEMBER_CAST(property, ValueInt, storage::Property)
DECODE_MEMBER(property_name, ValueString)
break;
}
auto decoder_hash = reader.hash();
uint64_t encoded_hash;
if (!reader.ReadType(encoded_hash, true)) return nullopt;
if (decoder_hash != encoded_hash) return nullopt;
return r_val;
} catch (communication::bolt::ValueException &) {
return nullopt;
} catch (std::ifstream::failure &) {
return nullopt;
}
}
#undef DECODE_MEMBER
void StateDelta::Apply(GraphDbAccessor &dba) const {
switch (type) {
// Transactional state is not recovered.
case Type::TRANSACTION_BEGIN:
case Type::TRANSACTION_COMMIT:
case Type::TRANSACTION_ABORT:
LOG(FATAL) << "Transaction handling not handled in Apply";
break;
case Type::CREATE_VERTEX:
dba.InsertVertex(vertex_id);
break;
case Type::CREATE_EDGE: {
auto from = dba.FindVertex(vertex_from_id, true);
auto to = dba.FindVertex(vertex_to_id, true);
dba.InsertEdge(from, to, dba.EdgeType(edge_type_name), edge_id);
break;
}
case Type::SET_PROPERTY_VERTEX: {
auto vertex = dba.FindVertex(vertex_id, true);
vertex.PropsSet(dba.Property(property_name), value);
break;
}
case Type::SET_PROPERTY_EDGE: {
auto edge = dba.FindEdge(edge_id, true);
edge.PropsSet(dba.Property(property_name), value);
break;
}
case Type::ADD_LABEL: {
auto vertex = dba.FindVertex(vertex_id, true);
vertex.add_label(dba.Label(label_name));
break;
}
case Type::REMOVE_LABEL: {
auto vertex = dba.FindVertex(vertex_id, true);
vertex.remove_label(dba.Label(label_name));
break;
}
case Type::REMOVE_VERTEX: {
auto vertex = dba.FindVertex(vertex_id, true);
dba.DetachRemoveVertex(vertex);
break;
}
case Type::REMOVE_EDGE: {
auto edge = dba.FindEdge(edge_id, true);
dba.RemoveEdge(edge);
break;
}
case Type::BUILD_INDEX:
case Type::DROP_INDEX: {
LOG(FATAL) << "Index handling not handled in Apply";
break;
}
}
}
}; // namespace database

View File

@ -0,0 +1,129 @@
// -*- buffer-read-only: t; -*-
// vim: readonly
// DO NOT EDIT! Generated using LCP from 'state_delta.lcp'
#pragma once
#include "communication/bolt/v1/decoder/decoder.hpp"
#include "communication/bolt/v1/encoder/base_encoder.hpp"
#include "durability/hashed_file_reader.hpp"
#include "durability/hashed_file_writer.hpp"
#include "mvcc/single_node_ha/version_list.hpp"
#include "storage/common/property_value.hpp"
#include "storage/common/types.hpp"
#include "storage/single_node_ha/gid.hpp"
class Vertex;
class Edge;
namespace database {
class GraphDbAccessor;
/// Describes single change to the database state. Used for durability (WAL) and
/// state communication over network in HA and for distributed remote storage
/// changes.
///
/// Labels, Properties and EdgeTypes are stored both as values (integers) and
/// strings (their names). The values are used when applying deltas in a running
/// database. Names are used when recovering the database as it's not guaranteed
/// that after recovery the old name<->value mapping will be preserved.
///
/// TODO: ensure the mapping is preserved after recovery and don't save strings
/// in StateDeltas.
struct StateDelta {
/// Defines StateDelta type. For each type the comment indicates which values
/// need to be stored. All deltas have the transaction_id member, so that's
/// omitted in the comment.
enum class Type {
TRANSACTION_BEGIN,
TRANSACTION_COMMIT,
TRANSACTION_ABORT,
CREATE_VERTEX,
CREATE_EDGE,
SET_PROPERTY_VERTEX,
SET_PROPERTY_EDGE,
ADD_LABEL,
REMOVE_LABEL,
REMOVE_VERTEX,
REMOVE_EDGE,
BUILD_INDEX,
DROP_INDEX
};
StateDelta() = default;
StateDelta(const enum Type &type, tx::TransactionId tx_id)
: type(type), transaction_id(tx_id) {}
/** Attempts to decode a StateDelta from the given decoder. Returns the
* decoded value if successful, otherwise returns nullopt. */
static std::experimental::optional<StateDelta> Decode(
HashedFileReader &reader,
communication::bolt::Decoder<HashedFileReader> &decoder);
/** Encodes the delta using primitive encoder, and writes out the new hash
* with delta to the writer */
void Encode(
HashedFileWriter &writer,
communication::bolt::BaseEncoder<HashedFileWriter> &encoder) const;
static StateDelta TxBegin(tx::TransactionId tx_id);
static StateDelta TxCommit(tx::TransactionId tx_id);
static StateDelta TxAbort(tx::TransactionId tx_id);
static StateDelta CreateVertex(tx::TransactionId tx_id, gid::Gid vertex_id);
static StateDelta CreateEdge(tx::TransactionId tx_id, gid::Gid edge_id,
gid::Gid vertex_from_id, gid::Gid vertex_to_id,
storage::EdgeType edge_type,
const std::string &edge_type_name);
static StateDelta PropsSetVertex(tx::TransactionId tx_id, gid::Gid vertex_id,
storage::Property property,
const std::string &property_name,
const PropertyValue &value);
static StateDelta PropsSetEdge(tx::TransactionId tx_id, gid::Gid edge_id,
storage::Property property,
const std::string &property_name,
const PropertyValue &value);
static StateDelta AddLabel(tx::TransactionId tx_id, gid::Gid vertex_id,
storage::Label label,
const std::string &label_name);
static StateDelta RemoveLabel(tx::TransactionId tx_id, gid::Gid vertex_id,
storage::Label label,
const std::string &label_name);
static StateDelta RemoveVertex(tx::TransactionId tx_id, gid::Gid vertex_id,
bool check_empty);
static StateDelta RemoveEdge(tx::TransactionId tx_id, gid::Gid edge_id);
static StateDelta BuildIndex(tx::TransactionId tx_id, storage::Label label,
const std::string &label_name,
storage::Property property,
const std::string &property_name, bool unique);
static StateDelta DropIndex(tx::TransactionId tx_id, storage::Label label,
const std::string &label_name,
storage::Property property,
const std::string &property_name);
/// Applies CRUD delta to database accessor. Fails on other types of deltas
void Apply(GraphDbAccessor &dba) const;
Type type;
tx::TransactionId transaction_id;
gid::Gid vertex_id;
gid::Gid edge_id;
mvcc::VersionList<Edge> *edge_address;
gid::Gid vertex_from_id;
mvcc::VersionList<Vertex> *vertex_from_address;
gid::Gid vertex_to_id;
mvcc::VersionList<Vertex> *vertex_to_address;
storage::EdgeType edge_type;
std::string edge_type_name;
storage::Property property;
std::string property_name;
PropertyValue value{PropertyValue::Null};
storage::Label label;
std::string label_name;
bool check_empty;
bool unique;
};
} // namespace database
// Cap'n Proto serialization declarations

View File

@ -0,0 +1,138 @@
#>cpp
#pragma once
#include "communication/bolt/v1/decoder/decoder.hpp"
#include "communication/bolt/v1/encoder/base_encoder.hpp"
#include "durability/hashed_file_reader.hpp"
#include "durability/hashed_file_writer.hpp"
#include "mvcc/single_node_ha/version_list.hpp"
#include "storage/common/property_value.hpp"
#include "storage/common/types.hpp"
#include "storage/single_node_ha/gid.hpp"
class Vertex;
class Edge;
cpp<#
(lcp:namespace database)
#>cpp
class GraphDbAccessor;
cpp<#
(lcp:define-struct state-delta ()
(
;; Members valid for every delta.
(type "Type")
(transaction-id "tx::TransactionId")
;; Members valid only for some deltas, see StateDelta::Type comments above.
;; TODO: when preparing the WAL for distributed, most likely remove Gids and
;; only keep addresses.
(vertex-id "gid::Gid")
(edge-id "gid::Gid")
(edge-address "mvcc::VersionList<Edge> *")
(vertex-from-id "gid::Gid")
(vertex-from-address "mvcc::VersionList<Vertex> *")
(vertex-to-id "gid::Gid")
(vertex-to-address "mvcc::VersionList<Vertex> *")
(edge-type "storage::EdgeType")
(edge-type-name "std::string")
(property "storage::Property")
(property-name "std::string")
(value "PropertyValue" :initval "PropertyValue::Null")
(label "storage::Label")
(label-name "std::string")
(check-empty :bool)
(unique :bool))
(:documentation
"Describes single change to the database state. Used for durability (WAL) and
state communication over network in HA and for distributed remote storage
changes.
Labels, Properties and EdgeTypes are stored both as values (integers) and
strings (their names). The values are used when applying deltas in a running
database. Names are used when recovering the database as it's not guaranteed
that after recovery the old name<->value mapping will be preserved.
TODO: ensure the mapping is preserved after recovery and don't save strings
in StateDeltas.")
(:public
(lcp:define-enum type
(transaction-begin
transaction-commit
transaction-abort
create-vertex ;; vertex_id
create-edge ;; edge_id, from_vertex_id, to_vertex_id, edge_type, edge_type_name
set-property-vertex ;; vertex_id, property, property_name, property_value
set-property-edge ;; edge_id, property, property_name, property_value
;; remove property is done by setting a PropertyValue::Null
add-label ;; vertex_id, label, label_name
remove-label ;; vertex_id, label, label_name
remove-vertex ;; vertex_id, check_empty
remove-edge ;; edge_id
build-index ;; label, label_name, property, property_name, unique
drop-index ;; label, label_name, property, property_name
)
(:documentation
"Defines StateDelta type. For each type the comment indicates which values
need to be stored. All deltas have the transaction_id member, so that's
omitted in the comment."))
#>cpp
StateDelta() = default;
StateDelta(const enum Type &type, tx::TransactionId tx_id)
: type(type), transaction_id(tx_id) {}
/** Attempts to decode a StateDelta from the given decoder. Returns the
* decoded value if successful, otherwise returns nullopt. */
static std::experimental::optional<StateDelta> Decode(
HashedFileReader &reader,
communication::bolt::Decoder<HashedFileReader> &decoder);
/** Encodes the delta using primitive encoder, and writes out the new hash
* with delta to the writer */
void Encode(
HashedFileWriter &writer,
communication::bolt::BaseEncoder<HashedFileWriter> &encoder) const;
static StateDelta TxBegin(tx::TransactionId tx_id);
static StateDelta TxCommit(tx::TransactionId tx_id);
static StateDelta TxAbort(tx::TransactionId tx_id);
static StateDelta CreateVertex(tx::TransactionId tx_id,
gid::Gid vertex_id);
static StateDelta CreateEdge(tx::TransactionId tx_id, gid::Gid edge_id,
gid::Gid vertex_from_id,
gid::Gid vertex_to_id,
storage::EdgeType edge_type,
const std::string &edge_type_name);
static StateDelta PropsSetVertex(tx::TransactionId tx_id,
gid::Gid vertex_id,
storage::Property property,
const std::string &property_name,
const PropertyValue &value);
static StateDelta PropsSetEdge(tx::TransactionId tx_id, gid::Gid edge_id,
storage::Property property,
const std::string &property_name,
const PropertyValue &value);
static StateDelta AddLabel(tx::TransactionId tx_id, gid::Gid vertex_id,
storage::Label label,
const std::string &label_name);
static StateDelta RemoveLabel(tx::TransactionId tx_id, gid::Gid vertex_id,
storage::Label label,
const std::string &label_name);
static StateDelta RemoveVertex(tx::TransactionId tx_id, gid::Gid vertex_id,
bool check_empty);
static StateDelta RemoveEdge(tx::TransactionId tx_id, gid::Gid edge_id);
static StateDelta BuildIndex(tx::TransactionId tx_id, storage::Label label,
const std::string &label_name,
storage::Property property,
const std::string &property_name, bool unique);
static StateDelta DropIndex(tx::TransactionId tx_id, storage::Label label,
const std::string &label_name,
storage::Property property,
const std::string &property_name);
/// Applies CRUD delta to database accessor. Fails on other types of deltas
void Apply(GraphDbAccessor &dba) const;
cpp<#))
(lcp:pop-namespace) ;; database

View File

@ -0,0 +1,43 @@
#pragma once
///
///
/// IMPORTANT: Please update this file for every snapshot format change!!!
/// TODO (buda): This is not rock solid.
///
#include <array>
#include <cstdint>
namespace durability {
constexpr std::array<uint8_t, 4> kSnapshotMagic{{'M', 'G', 's', 'n'}};
constexpr std::array<uint8_t, 4> kWalMagic{{'M', 'G', 'w', 'l'}};
// The current default version of snapshot and WAL encoding / decoding.
constexpr int64_t kVersion{7};
// Snapshot format (version 7):
// 1) Magic number + snapshot version
//
// The following two entries are required when recovering from snapshot combined
// with WAL to determine record visibility.
// 2) Transactional ID of the snapshooter
// 3) Transactional snapshot of the snapshooter
//
// 4) A list of label+property indices.
//
// We must inline edges with nodes because some edges might be stored on other
// worker (edges are always stored only on the worker of the edge source).
// 5) Bolt encoded nodes. Each node is written in the following format:
// * gid, labels, properties
// * inlined edges (edge address, other endpoint address and edge type)
// 6) Bolt encoded edges. Each edge is written in the following format:
// * gid
// * from, to
// * edge_type
// * properties
//
// 7) Snapshot summary (number of nodes, number of edges, hash)
} // namespace durability

View File

@ -0,0 +1,159 @@
#include "durability/single_node_ha/wal.hpp"
#include "durability/single_node_ha/paths.hpp"
#include "durability/single_node_ha/version.hpp"
#include "utils/file.hpp"
#include "utils/flag_validation.hpp"
DEFINE_HIDDEN_int32(
wal_flush_interval_millis, 2,
"Interval between two write-ahead log flushes, in milliseconds.");
DEFINE_HIDDEN_int32(
wal_rotate_deltas_count, 10000,
"How many write-ahead deltas should be stored in a single WAL file "
"before rotating it.");
DEFINE_VALIDATED_HIDDEN_int32(wal_buffer_size, 4096,
"Write-ahead log buffer size.",
FLAG_IN_RANGE(1, 1 << 30));
namespace durability {
WriteAheadLog::WriteAheadLog(
const std::experimental::filesystem::path &durability_dir,
bool durability_enabled, bool synchronous_commit)
: deltas_{FLAGS_wal_buffer_size},
wal_file_{durability_dir},
durability_enabled_(durability_enabled),
synchronous_commit_(synchronous_commit) {
if (durability_enabled_) {
utils::CheckDir(durability_dir);
}
}
WriteAheadLog::~WriteAheadLog() {
if (durability_enabled_) {
if (!synchronous_commit_) scheduler_.Stop();
wal_file_.Flush(deltas_);
}
}
WriteAheadLog::WalFile::WalFile(
const std::experimental::filesystem::path &durability_dir)
: wal_dir_{durability_dir / kWalDir} {}
WriteAheadLog::WalFile::~WalFile() {
if (!current_wal_file_.empty()) writer_.Close();
}
void WriteAheadLog::WalFile::Init() {
if (!utils::EnsureDir(wal_dir_)) {
LOG(ERROR) << "Can't write to WAL directory: " << wal_dir_;
current_wal_file_ = std::experimental::filesystem::path();
} else {
current_wal_file_ = WalFilenameForTransactionId(wal_dir_);
// TODO: Fix error handling, the encoder_ returns `true` or `false`.
try {
writer_.Open(current_wal_file_);
encoder_.WriteRAW(durability::kWalMagic.data(),
durability::kWalMagic.size());
encoder_.WriteInt(durability::kVersion);
writer_.Flush();
} catch (std::ios_base::failure &) {
LOG(ERROR) << "Failed to open write-ahead log file: "
<< current_wal_file_;
current_wal_file_ = std::experimental::filesystem::path();
}
}
latest_tx_ = 0;
current_wal_file_delta_count_ = 0;
}
void WriteAheadLog::WalFile::Flush(RingBuffer<database::StateDelta> &buffer) {
std::lock_guard<std::mutex> flush_lock(flush_mutex_);
if (current_wal_file_.empty()) {
LOG(ERROR) << "Write-ahead log file uninitialized, discarding data.";
buffer.clear();
return;
}
try {
while (true) {
auto delta = buffer.pop();
if (!delta) break;
latest_tx_ = std::max(latest_tx_, delta->transaction_id);
delta->Encode(writer_, encoder_);
writer_.Flush();
if (++current_wal_file_delta_count_ >= FLAGS_wal_rotate_deltas_count)
RotateFile();
}
writer_.Flush();
} catch (std::ios_base::failure &) {
LOG(ERROR) << "Failed to write to write-ahead log, discarding data.";
buffer.clear();
return;
} catch (std::experimental::filesystem::filesystem_error &) {
LOG(ERROR) << "Failed to rotate write-ahead log.";
buffer.clear();
return;
}
}
void WriteAheadLog::WalFile::RotateFile() {
writer_.Flush();
writer_.Close();
std::experimental::filesystem::rename(
current_wal_file_,
WalFilenameForTransactionId(wal_dir_, latest_tx_));
Init();
}
void WriteAheadLog::Init() {
if (durability_enabled_) {
enabled_ = true;
wal_file_.Init();
if (!synchronous_commit_) {
scheduler_.Run("WAL",
std::chrono::milliseconds(FLAGS_wal_flush_interval_millis),
[this]() { wal_file_.Flush(deltas_); });
}
}
}
void WriteAheadLog::Emplace(const database::StateDelta &delta) {
if (durability_enabled_ && enabled_) {
deltas_.emplace(delta);
if (synchronous_commit_ && IsStateDeltaTransactionEnd(delta)) {
wal_file_.Flush(deltas_);
}
}
}
bool WriteAheadLog::IsStateDeltaTransactionEnd(
const database::StateDelta &delta) {
switch (delta.type) {
case database::StateDelta::Type::TRANSACTION_COMMIT:
case database::StateDelta::Type::TRANSACTION_ABORT:
return true;
case database::StateDelta::Type::TRANSACTION_BEGIN:
case database::StateDelta::Type::CREATE_VERTEX:
case database::StateDelta::Type::CREATE_EDGE:
case database::StateDelta::Type::SET_PROPERTY_VERTEX:
case database::StateDelta::Type::SET_PROPERTY_EDGE:
case database::StateDelta::Type::ADD_LABEL:
case database::StateDelta::Type::REMOVE_LABEL:
case database::StateDelta::Type::REMOVE_VERTEX:
case database::StateDelta::Type::REMOVE_EDGE:
case database::StateDelta::Type::BUILD_INDEX:
case database::StateDelta::Type::DROP_INDEX:
return false;
}
}
void WriteAheadLog::Flush() {
if (enabled_) {
wal_file_.Flush(deltas_);
}
}
} // namespace durability

View File

@ -0,0 +1,98 @@
#pragma once
#include <chrono>
#include <cstdint>
#include <experimental/filesystem>
#include <gflags/gflags.h>
#include <glog/logging.h>
#include "communication/bolt/v1/encoder/base_encoder.hpp"
#include "data_structures/ring_buffer.hpp"
#include "durability/single_node_ha/state_delta.hpp"
#include "storage/common/property_value.hpp"
#include "storage/common/types.hpp"
#include "storage/single_node_ha/gid.hpp"
#include "transactions/type.hpp"
#include "utils/scheduler.hpp"
namespace durability {
/// A database StateDelta log for durability. Buffers and periodically
/// serializes small-granulation database deltas (StateDelta).
///
/// The order is not deterministic in a multithreaded scenario (multiple DB
/// transactions). This is fine, the recovery process should be immune to this
/// indeterminism.
class WriteAheadLog {
public:
WriteAheadLog(const std::experimental::filesystem::path &durability_dir,
bool durability_enabled, bool synchronous_commit);
~WriteAheadLog();
/// Initializes the WAL. Called at the end of GraphDb construction, after
/// (optional) recovery. Also responsible for initializing the wal_file.
void Init();
/// Emplaces the given DeltaState onto the buffer, if the WAL is enabled.
/// If the WAL is configured to work in synchronous commit mode, emplace will
/// flush the buffers if a delta represents a transaction end.
void Emplace(const database::StateDelta &delta);
/// Flushes every delta currently in the ring buffer.
/// This method should only be called from tests.
void Flush();
private:
/// Groups the logic of WAL file handling (flushing, naming, rotating)
class WalFile {
public:
explicit WalFile(const std::experimental::filesystem::path &durability_dir);
~WalFile();
/// Initializes the WAL file. Must be called before first flush. Can be
/// called after Flush() to re-initialize stuff.
void Init();
/// Flushes all the deltas in the buffer to the WAL file. If necessary
/// rotates the file.
void Flush(RingBuffer<database::StateDelta> &buffer);
private:
/// Mutex used for flushing wal data
std::mutex flush_mutex_;
const std::experimental::filesystem::path wal_dir_;
HashedFileWriter writer_;
communication::bolt::BaseEncoder<HashedFileWriter> encoder_{writer_};
/// The file to which the WAL flushes data. The path is fixed, the file gets
/// moved when the WAL gets rotated.
std::experimental::filesystem::path current_wal_file_;
/// Number of deltas in the current wal file.
int current_wal_file_delta_count_{0};
/// The latest transaction whose delta is recorded in the current WAL file.
/// Zero indicates that no deltas have so far been written to the current
/// WAL file.
tx::TransactionId latest_tx_{0};
void RotateFile();
};
RingBuffer<database::StateDelta> deltas_;
utils::Scheduler scheduler_;
WalFile wal_file_;
/// Used for disabling the durability feature of the DB.
bool durability_enabled_{false};
/// Used for disabling the WAL during DB recovery.
bool enabled_{false};
/// Should every WAL write be synced with the underlying storage.
bool synchronous_commit_{false};
/// Checks whether the given state delta represents a transaction end,
/// TRANSACTION_COMMIT and TRANSACTION_ABORT.
bool IsStateDeltaTransactionEnd(const database::StateDelta &delta);
};
} // namespace durability

109
src/memgraph_ha.cpp Normal file
View File

@ -0,0 +1,109 @@
#include <algorithm>
#include <chrono>
#include <cstdint>
#include <exception>
#include <functional>
#include <limits>
#include <thread>
#include <gflags/gflags.h>
#include <glog/logging.h>
#include "communication/server.hpp"
#include "database/single_node_ha/graph_db.hpp"
#include "integrations/kafka/exceptions.hpp"
#include "integrations/kafka/streams.hpp"
#include "memgraph_init.hpp"
#include "query/exceptions.hpp"
#include "telemetry/telemetry.hpp"
#include "utils/flag_validation.hpp"
// General purpose flags.
DEFINE_string(interface, "0.0.0.0",
"Communication interface on which to listen.");
DEFINE_VALIDATED_int32(port, 7687, "Communication port on which to listen.",
FLAG_IN_RANGE(0, std::numeric_limits<uint16_t>::max()));
DEFINE_VALIDATED_int32(num_workers,
std::max(std::thread::hardware_concurrency(), 1U),
"Number of workers (Bolt)", FLAG_IN_RANGE(1, INT32_MAX));
DEFINE_VALIDATED_int32(session_inactivity_timeout, 1800,
"Time in seconds after which inactive sessions will be "
"closed.",
FLAG_IN_RANGE(1, INT32_MAX));
DEFINE_string(cert_file, "", "Certificate file to use.");
DEFINE_string(key_file, "", "Key file to use.");
DEFINE_bool(telemetry_enabled, false,
"Set to true to enable telemetry. We collect information about the "
"running system (CPU and memory information) and information about "
"the database runtime (vertex and edge counts and resource usage) "
"to allow for easier improvement of the product.");
using ServerT = communication::Server<BoltSession, SessionData>;
using communication::ServerContext;
void SingleNodeMain() {
google::SetUsageMessage("Memgraph single-node database server");
database::GraphDb db;
query::Interpreter interpreter;
SessionData session_data{&db, &interpreter};
integrations::kafka::Streams kafka_streams{
std::experimental::filesystem::path(FLAGS_durability_directory) /
"streams",
[&session_data](
const std::string &query,
const std::map<std::string, communication::bolt::Value> &params) {
KafkaStreamWriter(session_data, query, params);
}};
try {
// Recover possible streams.
kafka_streams.Recover();
} catch (const integrations::kafka::KafkaStreamException &e) {
LOG(ERROR) << e.what();
}
session_data.interpreter->auth_ = &session_data.auth;
session_data.interpreter->kafka_streams_ = &kafka_streams;
ServerContext context;
std::string service_name = "Bolt";
if (FLAGS_key_file != "" && FLAGS_cert_file != "") {
context = ServerContext(FLAGS_key_file, FLAGS_cert_file);
service_name = "BoltS";
}
ServerT server({FLAGS_interface, static_cast<uint16_t>(FLAGS_port)},
&session_data, &context, FLAGS_session_inactivity_timeout,
service_name, FLAGS_num_workers);
// Setup telemetry
std::experimental::optional<telemetry::Telemetry> telemetry;
if (FLAGS_telemetry_enabled) {
telemetry.emplace(
"https://telemetry.memgraph.com/88b5e7e8-746a-11e8-9f85-538a9e9690cc/",
std::experimental::filesystem::path(FLAGS_durability_directory) /
"telemetry",
std::chrono::minutes(10));
telemetry->AddCollector("db", [&db]() -> nlohmann::json {
auto dba = db.Access();
return {{"vertices", dba->VerticesCount()}, {"edges", dba->EdgesCount()}};
});
}
// Handler for regular termination signals
auto shutdown = [&server] {
// Server needs to be shutdown first and then the database. This prevents a
// race condition when a transaction is accepted during server shutdown.
server.Shutdown();
};
InitSignalHandlers(shutdown);
CHECK(server.Start()) << "Couldn't start the Bolt server!";
server.AwaitShutdown();
}
int main(int argc, char **argv) {
return WithInit(argc, argv, []() { return "memgraph"; }, SingleNodeMain);
}

View File

@ -0,0 +1,335 @@
#pragma once
#include <atomic>
#include <experimental/optional>
#include <iostream>
#include "transactions/commit_log.hpp"
#include "transactions/single_node_ha/engine.hpp"
#include "transactions/transaction.hpp"
#include "mvcc/common/version.hpp"
#include "storage/locking/record_lock.hpp"
// the mvcc implementation used here is very much like postgresql's
// more info: https://momjian.us/main/writings/pgsql/mvcc.pdf
namespace mvcc {
template <class T>
class Record : public Version<T> {
public:
Record() = default;
Record(const Record &) = delete;
Record &operator=(const Record &) = delete;
Record(Record &&) = delete;
Record &operator=(Record &&) = delete;
// check if this record is visible to the transaction t
bool visible(const tx::Transaction &t) {
// Mike Olson says 17 march 1993: the tests in this routine are correct;
// if you think they're not, you're wrong, and you should think about it
// again. i know, it happened to me.
// fetch expiration info in a safe way (see fetch_exp for details)
tx::TransactionId tx_exp;
tx::CommandId cmd_exp;
std::tie(tx_exp, cmd_exp) = fetch_exp();
return ((tx_.cre == t.id_ && // inserted by the current transaction
cmd_.cre < t.cid() && // before this command, and
(tx_exp == 0 || // the row has not been deleted, or
(tx_exp == t.id_ && // it was deleted by the current
// transaction
cmd_exp >= t.cid()))) // but not before this command,
|| // or
(visible_from(Hints::kCre, tx_.cre,
t) && // the record was inserted by a
// committed transaction, and
(tx_exp == 0 || // the record has not been deleted, or
(tx_exp == t.id_ && // the row is being deleted by this
// transaction
cmd_exp >= t.cid()) || // but it's not deleted "yet", or
(tx_exp != t.id_ && // the row was deleted by another
// transaction
!visible_from(Hints::kExp, tx_exp,
t) // that has not been committed
))));
}
void mark_created(const tx::Transaction &t) {
DCHECK(tx_.cre == 0) << "Marking node as created twice.";
tx_.cre = t.id_;
cmd_.cre = t.cid();
}
void mark_expired(const tx::Transaction &t) {
tx_.exp = t.id_;
cmd_.exp = t.cid();
}
bool exp_committed(tx::Engine &engine) {
return committed(Hints::kExp, engine);
}
/**
* Check if this record is visible w.r.t. to the given garbage collection
* snapshot. See source comments for exact logic.
*
* @param snapshot - the GC snapshot. Consists of the oldest active
* transaction's snapshot, with that transaction's id appened as last.
*/
bool is_not_visible_from(const tx::Snapshot &snapshot,
const tx::Engine &engine) const {
// first get tx.exp so that all the subsequent checks operate on
// the same id. otherwise there could be a race condition
auto exp_id = tx_.exp.load();
// a record is NOT visible if:
// 1. it creating transaction aborted (last check), and is also older than
// the current oldest active transaction (optimization) OR
// 2. a) it's expiration is not 0 (some transaction expired it)
// AND
// b) the expiring transaction is older than latest active
// AND
// c) that transaction committed (as opposed to aborted)
// AND
// d) that transaction is not in oldest active transaction's
// snapshot (consequently also not in the snapshots of
// newer transactions)
return (exp_id != 0 && exp_id < snapshot.back() &&
committed(Hints::kExp, engine) && !snapshot.contains(exp_id)) ||
(tx_.cre.load() < snapshot.back() && cre_aborted(engine));
}
// TODO: Test this
// True if this record is visible for write.
// Note that this logic is different from the one above
// in the sense that a record is visible if created before
// OR DURING this command. this is done to support cypher's
// queries which can match, update and return in the same query
bool is_visible_write(const tx::Transaction &t) {
// fetch expiration info in a safe way (see fetch_exp for details)
tx::TransactionId tx_exp;
tx::CommandId cmd_exp;
std::tie(tx_exp, cmd_exp) = fetch_exp();
return (tx_.cre == t.id_ && // inserted by the current transaction
cmd_.cre <= t.cid() && // before OR DURING this command, and
(tx_exp == 0 || // the row has not been deleted, or
(tx_exp == t.id_ && // it was deleted by the current
// transaction
cmd_exp >= t.cid()))); // but not before this command,
}
/**
* True if this record is created in the current command
* of the given transaction.
*/
bool is_created_by(const tx::Transaction &t) {
return tx_.cre == t.id_ && cmd_.cre == t.cid();
}
/**
* True if this record is expired in the current command
* of the given transaction.
*/
bool is_expired_by(const tx::Transaction &t) const {
return std::make_pair(t.id_, t.cid()) == fetch_exp();
}
const auto &tx() const { return tx_; }
const auto &cmd() const { return cmd_; }
/**
* Makes sure that create and expiry are in sync with hints if they are
* committed or aborted and are before the `tx_cutoff`.
* `tx_cutoff` exists as a performance optimization to avoid setting hint bits
* on records for which we don't need to have a guarantee that they are set as
* part of GC hints setting procedure
*/
void populate_hints(const tx::Engine &engine, tx::TransactionId tx_cutoff) {
populate_hint_if_possible(engine, Hints::kCre, tx_cutoff);
if (!populate_hint_if_possible(engine, Hints::kExp, tx_cutoff)) {
// Exp is aborted and we can't set the hint, this way we don't have to set
// the hint because an aborted transaction which expires a record is the
// same thing as a non-expired record
tx::TransactionId expected;
do {
expected = tx_.exp;
// If the transaction expiry is no longer aborted we don't need to
// update it anymore, and hints can't be set since it's obviously an
// active transaction - there might be a case where this transaction
// gets finished and committed in the meantime and hints could be set,
// but since we are not going to delete info for this transaction from
// the commit log since it wasn't older than the oldest active
// transaction at the time, or before the invocation of this method;
// we are in the clear
if (!engine.Info(expected).is_aborted()) break;
} while (!tx_.exp.compare_exchange_weak(expected, 0));
// Ideally we should set the command id as well, but by setting it we
// can't guarantee that some new update won't change the transaction id
// and command id before we had a chance to set it, and just leaving it
// unchanged and relying on all methods to operate on [tx_id: 0, cmd_id:
// some cmd] as a non-transaction doesn't seem too crazy
}
}
private:
/**
* Fast indicators if a transaction has committed or aborted. It is possible
* the hints do not have that information, in which case the commit log needs
* to be consulted (a slower operation).
*/
class Hints {
public:
/// Masks for the creation/expration and commit/abort positions.
static constexpr uint8_t kCre = 0b0011;
static constexpr uint8_t kExp = 0b1100;
static constexpr uint8_t kCmt = 0b0101;
static constexpr uint8_t kAbt = 0b1010;
/** Returns true if any bit under the given mask is set. */
bool Get(uint8_t mask) const { return bits_ & mask; }
/** Sets all the bits under the given mask. */
void Set(uint8_t mask) { bits_.fetch_or(mask); }
/** Clears all the bits under the given mask. */
void Clear(uint8_t mask) { bits_.fetch_and(~mask); }
private:
std::atomic<uint8_t> bits_{0};
};
template <typename TId>
struct CreExp {
std::atomic<TId> cre{0};
std::atomic<TId> exp{0};
};
// tx.cre is the id of the transaction that created the record
// and tx.exp is the id of the transaction that deleted the record
// These values are used to determine the visibility of the record
// to the current transaction.
CreExp<tx::TransactionId> tx_;
// cmd.cre is the id of the command in this transaction that created the
// record and cmd.exp is the id of the command in this transaction that
// deleted the record. These values are used to determine the visibility
// of the record to the current command in the running transaction.
CreExp<tx::CommandId> cmd_;
mutable Hints hints_;
/** Fetch the (transaction, command) expiration before the check
* because they can be concurrently modified by multiple transactions.
* Do it in a loop to ensure that command is consistent with transaction.
*/
auto fetch_exp() const {
tx::TransactionId tx_exp;
tx::CommandId cmd_exp;
do {
tx_exp = tx_.exp;
cmd_exp = cmd_.exp;
} while (tx_exp != tx_.exp);
return std::make_pair(tx_exp, cmd_exp);
}
/**
* Populates hint if it is not set for the given create/expiry mask and is
* before the `tx_cutoff` if specified. Note that it doesn't set hint bits for
* expiry transactions which abort because it's too expensive to maintain
* correctness of those hints with regards to race conditions
* @returns - true if hints are now equal to transaction status
* (committed/aborted), will only be false if we are trying to set hint for
* aborted transaction which is this records expiry
*/
bool populate_hint_if_possible(
const tx::Engine &engine, const uint8_t mask,
const std::experimental::optional<tx::TransactionId> tx_cutoff =
std::experimental::nullopt) const {
DCHECK(mask == Hints::kCre || mask == Hints::kExp)
<< "Mask should be either for creation or expiration";
if (hints_.Get(mask)) return true;
auto id = mask == Hints::kCre ? tx_.cre.load() : tx_.exp.load();
// Nothing to do here if there is no id or id is larger than tx_cutoff
if (!id || (tx_cutoff && id >= *tx_cutoff)) return true;
auto info = engine.Info(id);
if (info.is_committed()) {
hints_.Set(mask & Hints::kCmt);
} else if (info.is_aborted()) {
// Abort hints can only be updated for creation hints because only one
// transaction can be creating a single record, so there is no races
if (mask == Hints::kCre)
hints_.Set(mask & Hints::kAbt);
else
return false;
}
return true;
}
/**
* @brief - Check if the transaciton `id` has comitted before `t` started
* (that means that edits done by transaction `id` are visible in `t`)
*
* Evaluates to true if that transaction has committed,
* it started before `t` and it's not in it's snapshot.
*
* about transactions commit/abort status
* @param mask - Hint bits mask (either Hints::kCre or Hints::kExp).
* @param id - id to check if it's commited and visible
* @return true if the id is commited and visible for the transaction t.
*/
bool visible_from(uint8_t mask, tx::TransactionId id,
const tx::Transaction &t) {
DCHECK(mask == Hints::kCre || mask == Hints::kExp)
<< "Mask must be either kCre or kExp";
// Dominik Gleich says 4 april 2017: the tests in this routine are correct;
// if you think they're not, you're wrong, and you should think about it
// again. I know, it happened to me (and also to Matej Gradicek).
// You certainly can't see the transaction with id greater than yours as
// that means it started after this transaction and if it commited, it
// commited after this transaction has started.
if (id >= t.id_) return false;
// The creating transaction is still in progress (examine snapshot)
if (t.snapshot().contains(id)) return false;
return committed(mask, t.engine_);
}
/**
* @brief - Check if the transaction with the given `id` is committed.
*
* @param mask - Hint bits mask (either Hints::kCre or Hints::kExp).
* @param id - id to check if commited
* statuses
* @return true if it's commited, false otherwise
*/
bool committed(uint8_t mask, const tx::Engine &engine) const {
DCHECK(mask == Hints::kCre || mask == Hints::kExp)
<< "Mask must be either kCre or kExp";
populate_hint_if_possible(engine, mask);
return hints_.Get(Hints::kCmt & mask);
}
/**
* @brief - Check if tx_.cre is aborted. If you need to check for exp
* transaction do it manually by looking at commit log. This function can't do
* that for you since hints can't be used for exp transaction (reason is
* described in function above).
*
* @param engine - engine instance with information about transaction
* statuses
* @return true if it's aborted, false otherwise
*/
bool cre_aborted(const tx::Engine &engine) const {
// Populate hints if not set and return result from hints
DCHECK(populate_hint_if_possible(engine, Hints::kCre))
<< "Hints not populated";
return hints_.Get(Hints::kAbt & Hints::kCre);
}
};
} // namespace mvcc

View File

@ -0,0 +1,270 @@
#pragma once
#include "storage/single_node_ha/gid.hpp"
#include "storage/locking/record_lock.hpp"
#include "transactions/transaction.hpp"
#include "utils/cast.hpp"
#include "utils/exceptions.hpp"
namespace mvcc {
class SerializationError : public utils::BasicException {
static constexpr const char *default_message =
"Can't serialize due to concurrent operations.";
public:
using utils::BasicException::BasicException;
SerializationError() : BasicException(default_message) {}
};
template <class T>
class VersionList {
public:
/**
* @brief Constructor that is used to insert one item into VersionList.
*
* @param t - transaction
* @param gid - Version list identifier. Uniqueness guaranteed by the code
* creating this version list.
* @param args - args forwarded to constructor of item T (for
* creating the first Record (Version) in this VersionList.
*/
template <typename... Args>
VersionList(const tx::Transaction &t, gid::Gid gid, Args &&... args)
: gid_(gid) {
// TODO replace 'new' with something better
auto *v1 = new T(std::forward<Args>(args)...);
v1->mark_created(t);
head_ = v1;
}
VersionList() = delete;
VersionList(const VersionList &) = delete;
VersionList &operator=(const VersionList &) = delete;
// We do a lot of raw-pointer ops with VLists, and these ops assume that a
// VList's address identifies a vertex/edge absolutely and during it's whole
// lifteme. We also assume that the VList owner is the database and that
// ownership is also handled via raw pointers so this shouldn't be moved or
// move assigned.
VersionList(VersionList &&other) = delete;
VersionList &operator=(VersionList &&other) = delete;
~VersionList() { delete head_.load(); }
friend std::ostream &operator<<(std::ostream &stream,
const VersionList<T> &vlist) {
stream << "VersionList" << std::endl;
T *record = vlist.head_;
while (record != nullptr) {
stream << "-- " << *record << std::endl;
record = record->next();
}
return stream;
}
/**
* Garbage collects records that are not reachable/visible anymore.
*
* Relinks this version-list so that garbage collected records are no
* longer reachable through this version list.
* Visibility is defined in mvcc::Record::is_not_visible_from,
* to which the given `snapshot` is passed.
*
* This method is NOT thread-safe.
*
* @param snapshot - the GC snapshot. Consists of the oldest active
* transaction's snapshot, with that transaction's id appened as last.
* @param engine - transaction engine to use - we need it to check which
* records were commited and which weren't
* @return pair<status, to_delete>; status is true - If version list is empty
* after garbage collection. to_delete points to the newest record that is not
* visible anymore. If none exists to_delete will point to nullptr.
*/
std::pair<bool, T *> GcDeleted(const tx::Snapshot &snapshot,
const tx::Engine &engine) {
// nullptr
// |
// [v1] ... all of this gets deleted!
// |
// [v2] <------+ head_of_deletable_records
// | |
// [v3] <------+ oldest_visible_record
// | | Jump backwards until you find the oldest visible
// [VerList] ----+ record, or you reach the end of the list
//
T *head = head_;
T *current = head;
T *oldest_visible_record = nullptr;
while (current) {
// Populate hints only when needed to avoid excessive rpc calls on
// workers.
// snapshot.back() corresponds to the oldest active transaction,
// and this makes it set only hint bits when the creating or expiring
// transaction of a record is older than that)
current->populate_hints(engine, snapshot.back());
if (!current->is_not_visible_from(snapshot, engine))
oldest_visible_record = current;
current = current->next();
}
if (oldest_visible_record) {
T *head_of_deletable_records = oldest_visible_record->next();
// oldest_visible_record might be visible to some transaction but
// head_of_deletable_records is not and will never be visted by the find
// function and as such doesn't represent pointer invalidation
// race-condition risk.
oldest_visible_record->next(nullptr); // No transaction will look
// further than this record and
// that's why it's safe to set
// next to nullptr.
// Calling destructor of head_of_deletable_records will clean everything
// older than this record since they are called recursively.
return std::make_pair(false, head_of_deletable_records);
}
// This can happen only if the head points to a expired record. Since there
// is no visible records in this version_list we can remove it.
head_ = nullptr;
// This is safe to return as ready for deletion since we unlinked head
// above and this will only be deleted after the last active transaction
// ends.
return std::make_pair(true, head);
}
/**
* @brief - returns oldest record
* @return nullptr if none exist
*/
T *Oldest() {
T *r = head_;
while (r && r->next(std::memory_order_seq_cst))
r = r->next(std::memory_order_seq_cst);
return r;
}
T *find(const tx::Transaction &t) {
T *r = head_;
// nullptr
// |
// [v1] ...
// |
// [v2] <------+
// | |
// [v3] <------+
// | | Jump backwards until you find a first visible
// [VerList] ----+ version, or you reach the end of the list
//
while (r != nullptr && !r->visible(t))
r = r->next(std::memory_order_seq_cst);
return r;
}
/**
* Looks for and sets two versions. The 'old' version is the
* newest version that is visible by the current transaction+command,
* but has not been created by it. The 'new' version is the version
* that has been created by current transaction+command.
*
* It is possible that both, either or neither are found:
* - both are found when an existing record has been modified
* - only old is found when an existing record has not been modified
* - only new is found when the whole vlist was created
* - neither is found when for example the record has been deleted but not
* garbage collected yet
*
* @param t The transaction
*/
void find_set_old_new(const tx::Transaction &t, T **old_ref, T **new_ref) {
// assume that the sought old record is further down the list
// from new record, so that if we found old we can stop looking
*new_ref = nullptr;
*old_ref = head_;
while (*old_ref != nullptr && !(*old_ref)->visible(t)) {
if (!*new_ref && (*old_ref)->is_created_by(t)) *new_ref = *old_ref;
*old_ref = (*old_ref)->next(std::memory_order_seq_cst);
}
}
/**
* Looks for the first visible record seen by this transaction. If the current
* transaction has already created new record in the current command then that
* record is returned, else first older visible record is updated. New record
* becomes head of the version list and it is returned. There should always be
* older visible record when this update is called.
*
* @param t The transaction
*/
T *update(const tx::Transaction &t) {
DCHECK(head_ != nullptr) << "Head is nullptr on update.";
T *old_record = nullptr;
T *new_record = nullptr;
find_set_old_new(t, &old_record, &new_record);
// check if current transaction in current cmd has
// already updated version list
if (new_record) return new_record;
// check if we found any visible records
CHECK(old_record != nullptr) << "Updating nullptr record";
return update(old_record, t);
}
/** Makes the given record as being expired by the given transaction. */
void remove(T *record, const tx::Transaction &t) {
DCHECK(record != nullptr) << "Record is nullptr on removal.";
lock_and_validate(record, t);
record->mark_expired(t);
}
const gid::Gid gid_;
int64_t cypher_id() { return utils::MemcpyCast<int64_t>(gid_); }
private:
void lock_and_validate(T *record, const tx::Transaction &t) {
DCHECK(record != nullptr) << "Record is nullptr on lock and validation.";
// take a lock on this node
t.TakeLock(lock_);
// if the record hasn't been deleted yet or the deleting transaction
// has aborted, it's ok to modify it
if (!record->tx().exp || !record->exp_committed(t.engine_)) return;
// if it committed, then we have a serialization conflict
throw SerializationError();
}
T *update(T *record, const tx::Transaction &t) {
DCHECK(record != nullptr) << "Record is nullptr on update.";
lock_and_validate(record, t);
// It could be done with unique_ptr but while this could mean memory
// leak on exception, unique_ptr could mean use after free. Memory
// leak is less dangerous.
auto *updated = record->CloneData();
updated->mark_created(t);
record->mark_expired(t);
// Updated version should point to the latest available version. Older
// versions that can be deleted will be removed during the GC phase.
updated->next(head_.load(), std::memory_order_seq_cst);
// Store the updated version as the first version point to by head.
head_.store(updated, std::memory_order_seq_cst);
return updated;
}
std::atomic<T *> head_{nullptr};
RecordLock lock_;
};
} // namespace mvcc

View File

@ -4,6 +4,10 @@
#include "storage/single_node/edge_accessor.hpp"
#endif
#ifdef MG_SINGLE_NODE_HA
#include "storage/single_node_ha/edge_accessor.hpp"
#endif
#ifdef MG_DISTRIBUTED
#include "storage/distributed/edge_accessor.hpp"
#endif

View File

@ -0,0 +1,68 @@
#pragma once
#include <mutex>
#include "data_structures/concurrent/concurrent_map.hpp"
#include "storage/common/types.hpp"
#include "utils/algorithm.hpp"
namespace storage {
/** SingleNode implementation of ConcurrentIdMapper. */
template <typename TId>
class ConcurrentIdMapper {
using StorageT = typename TId::IdT;
public:
ConcurrentIdMapper() = default;
explicit ConcurrentIdMapper(
const std::vector<std::string> &properties_on_disk)
: properties_on_disk_(properties_on_disk) {}
TId value_to_id(const std::string &value) {
auto value_to_id_acc = value_to_id_.access();
auto found = value_to_id_acc.find(value);
TId inserted_id(0);
if (found == value_to_id_acc.end()) {
StorageT new_id = id_.fetch_add(1);
// After we tried to insert value with our id we either got our id, or the
// id created by the thread which succesfully inserted (value, id) pair
// because that's ConcurrentMap's behaviour
if (std::is_same<TId, Property>::value)
inserted_id =
value_to_id_acc.insert(value, TId(new_id, PropertyLocation(value)))
.first->second;
else
inserted_id = value_to_id_acc.insert(value, TId(new_id)).first->second;
} else {
inserted_id = found->second;
}
auto id_to_value_acc = id_to_value_.access();
// We have to try to insert the inserted_id and value even if we are not the
// one who assigned id because we have to make sure that after this method
// returns that both mappings between id->value and value->id exist.
id_to_value_acc.insert(inserted_id, value);
return inserted_id;
}
const std::string &id_to_value(const TId &id) {
auto id_to_value_acc = id_to_value_.access();
auto result = id_to_value_acc.find(id);
DCHECK(result != id_to_value_acc.end());
return result->second;
}
private:
ConcurrentMap<std::string, TId> value_to_id_;
ConcurrentMap<TId, std::string> id_to_value_;
std::atomic<StorageT> id_{0};
std::vector<std::string> properties_on_disk_;
std::mutex mutex_;
Location PropertyLocation(const std::string &name) {
std::unique_lock<std::mutex> lock(mutex_);
if (utils::Contains(properties_on_disk_, name)) return Location::Disk;
return Location::Memory;
}
};
} // namespace storage

View File

@ -0,0 +1,78 @@
#pragma once
#include <malloc.h>
#include <limits>
#include <list>
#include "glog/logging.h"
#include "mvcc/single_node_ha/record.hpp"
#include "transactions/transaction.hpp"
/**
* @brief - Implements deferred deletion.
* @Tparam T - type of object to delete (Vertex/Edge/VersionList...)
* This is NOT a thread-safe class.
*/
template <typename T>
class DeferredDeleter {
public:
/**
* @brief - keep track of what object was deleted at which time.
*/
struct DeletedObject {
const T *object;
const tx::TransactionId deleted_at;
DeletedObject(const T *object, tx::TransactionId deleted_at)
: object(object), deleted_at(deleted_at) {}
};
/**
* @brief - check if everything is freed
*/
~DeferredDeleter() {
CHECK(objects_.size() == 0U)
<< "Objects are not freed when calling the destructor.";
}
/**
* @brief - Add objects to this deleter. This method assumes that it will
* always be called with a non-decreasing sequence of `deleted_at`.
* @param objects - vector of objects to add
* @param last_transaction - nothing newer or equal to it can see these
* objects
*/
void AddObjects(const std::vector<DeletedObject> &objects) {
auto previous_tx_id = objects_.empty()
? std::numeric_limits<tx::TransactionId>::min()
: objects_.back().deleted_at;
for (auto object : objects) {
CHECK(previous_tx_id <= object.deleted_at)
<< "deleted_at must be non-decreasing";
previous_tx_id = object.deleted_at;
objects_.push_back(object);
}
}
/**
* @brief - Free memory of objects deleted before the id.
* @param id - delete before this id
*/
void FreeExpiredObjects(tx::TransactionId id) {
auto it = objects_.begin();
while (it != objects_.end() && it->deleted_at < id) {
delete it->object;
++it;
}
objects_.erase(objects_.begin(), it);
}
/**
* @brief - Return number of stored objects.
*/
size_t Count() { return objects_.size(); }
private:
// Ascendingly sorted list of deleted objects by `deleted_at`.
std::list<DeletedObject> objects_;
};

View File

@ -0,0 +1,32 @@
#pragma once
#include "mvcc/single_node_ha/record.hpp"
#include "mvcc/single_node_ha/version_list.hpp"
#include "storage/common/property_value_store.hpp"
#include "storage/common/types.hpp"
class Vertex;
class Edge : public mvcc::Record<Edge> {
public:
Edge(mvcc::VersionList<Vertex> *from, mvcc::VersionList<Vertex> *to,
storage::EdgeType edge_type)
: from_(from), to_(to), edge_type_(edge_type) {}
// Returns new Edge with copy of data stored in this Edge, but without
// copying superclass' members.
Edge *CloneData() { return new Edge(*this); }
mvcc::VersionList<Vertex> *from_;
mvcc::VersionList<Vertex> *to_;
storage::EdgeType edge_type_;
PropertyValueStore properties_;
private:
Edge(const Edge &other)
: mvcc::Record<Edge>(),
from_(other.from_),
to_(other.to_),
edge_type_(other.edge_type_),
properties_(other.properties_) {}
};

View File

@ -0,0 +1,59 @@
#include "storage/edge_accessor.hpp"
#include "database/single_node_ha/graph_db_accessor.hpp"
#include "storage/vertex_accessor.hpp"
#include "utils/algorithm.hpp"
EdgeAccessor::EdgeAccessor(mvcc::VersionList<Edge> *address,
database::GraphDbAccessor &db_accessor)
: RecordAccessor(address, db_accessor),
from_(nullptr),
to_(nullptr),
edge_type_() {
RecordAccessor::Reconstruct();
if (current_ != nullptr) {
from_ = current_->from_;
to_ = current_->to_;
edge_type_ = current_->edge_type_;
}
}
EdgeAccessor::EdgeAccessor(mvcc::VersionList<Edge> *address,
database::GraphDbAccessor &db_accessor,
mvcc::VersionList<Vertex> *from,
mvcc::VersionList<Vertex> *to,
storage::EdgeType edge_type)
: RecordAccessor(address, db_accessor),
from_(from),
to_(to),
edge_type_(edge_type) {}
storage::EdgeType EdgeAccessor::EdgeType() const { return edge_type_; }
VertexAccessor EdgeAccessor::from() const {
return VertexAccessor(from_, db_accessor());
}
bool EdgeAccessor::from_is(const VertexAccessor &v) const {
return v.address() == from_;
}
VertexAccessor EdgeAccessor::to() const {
return VertexAccessor(to_, db_accessor());
}
bool EdgeAccessor::to_is(const VertexAccessor &v) const {
return v.address() == to_;
}
bool EdgeAccessor::is_cycle() const { return to_ == from_; }
std::ostream &operator<<(std::ostream &os, const EdgeAccessor &ea) {
os << "E[" << ea.db_accessor().EdgeTypeName(ea.EdgeType());
os << " {";
utils::PrintIterable(os, ea.Properties(), ", ", [&](auto &stream,
const auto &pair) {
stream << ea.db_accessor().PropertyName(pair.first) << ": " << pair.second;
});
return os << "}]";
}

View File

@ -0,0 +1,77 @@
#pragma once
#include "storage/single_node_ha/edge.hpp"
#include "storage/single_node_ha/record_accessor.hpp"
// forward declaring the VertexAccessor because it's returned
// by some functions
class VertexAccessor;
/**
* Provides ways for the client programmer (i.e. code generated by the compiler)
* to interact with an Edge.
*
* Note that EdgeAccessors do not necessary read versioned (MVCC) data. This is
* possible because edge endpoints (from and to), as well as the edge type, are
* all immutable. These are the most often used aspects of an edge, and are
* stored also in the vertex endpoints of the edge. Using them when creating an
* EdgeAccessor means that data does not have to be read from a random memory
* location, which is often a performance bottleneck in traversals.
*/
class EdgeAccessor final : public RecordAccessor<Edge> {
public:
/** Constructor that reads data from the random memory location (lower
* performance, see class docs). */
EdgeAccessor(mvcc::VersionList<Edge> *address,
database::GraphDbAccessor &db_accessor);
/**
* Constructor that does NOT read data from the random memory location
* (better performance, see class docs).
*/
EdgeAccessor(mvcc::VersionList<Edge> *address,
database::GraphDbAccessor &db_accessor,
mvcc::VersionList<Vertex> *from, mvcc::VersionList<Vertex> *to,
storage::EdgeType edge_type);
storage::EdgeType EdgeType() const;
/** Returns an accessor to the originating Vertex of this edge. */
VertexAccessor from() const;
/** Returns the address of the originating Vertex of this edge. */
auto from_addr() const { return from_; }
/** Checks if the given vertex is the source of this edge, without
* creating an additional accessor to perform the check. */
bool from_is(const VertexAccessor &v) const;
/** Returns an accessor to the destination Vertex of this edge. */
VertexAccessor to() const;
/** Returns the address of the destination Vertex of this edge. */
auto to_addr() const { return to_; }
/** Checks if the given vertex is the destination of this edge, without
* creating an additional accessor to perform the check. */
bool to_is(const VertexAccessor &v) const;
/** Returns true if this edge is a cycle (start and end node are
* the same. */
bool is_cycle() const;
private:
mvcc::VersionList<Vertex> *from_;
mvcc::VersionList<Vertex> *to_;
storage::EdgeType edge_type_;
};
std::ostream &operator<<(std::ostream &, const EdgeAccessor &);
// hash function for the edge accessor
namespace std {
template <>
struct hash<EdgeAccessor> {
size_t operator()(const EdgeAccessor &e) const { return e.gid(); };
};
} // namespace std

View File

@ -0,0 +1,153 @@
#pragma once
#include <experimental/optional>
#include <utility>
#include <vector>
#include "glog/logging.h"
#include "mvcc/single_node_ha/version_list.hpp"
#include "storage/common/types.hpp"
#include "utils/algorithm.hpp"
/**
* A data stucture that holds a number of edges. This implementation assumes
* that separate Edges instances are used for incoming and outgoing edges in a
* vertex (and consequently that edge Addresses are unique in it).
*/
class Edges {
private:
struct Element {
mvcc::VersionList<Vertex> *vertex;
mvcc::VersionList<Edge> *edge;
storage::EdgeType edge_type;
};
/** Custom iterator that takes care of skipping edges when the destination
* vertex or edge types are known. */
class Iterator {
public:
/** Ctor that just sets the position. Used for normal iteration (that does
* not skip any edges), and for end-iterator creation in both normal and
* skipping iteration.
*
* @param iterator - Iterator in the underlying storage.
*/
explicit Iterator(std::vector<Element>::const_iterator iterator)
: position_(iterator) {}
/** Ctor used for creating the beginning iterator with known destination
* vertex.
*
* @param iterator - Iterator in the underlying storage.
* @param end - End iterator in the underlying storage.
* @param vertex - The destination vertex address. If empty the
* edges are not filtered on destination.
* @param edge_types - The edge types at least one of which must be matched.
* If nullptr edges are not filtered on type.
*/
Iterator(std::vector<Element>::const_iterator position,
std::vector<Element>::const_iterator end,
mvcc::VersionList<Vertex> *vertex,
const std::vector<storage::EdgeType> *edge_types)
: position_(position),
end_(end),
vertex_(vertex),
edge_types_(edge_types) {
update_position();
}
Iterator &operator++() {
++position_;
update_position();
return *this;
}
const Element &operator*() const { return *position_; }
const Element *operator->() const { return &(*position_); }
bool operator==(const Iterator &other) const {
return position_ == other.position_;
}
bool operator!=(const Iterator &other) const { return !(*this == other); }
private:
std::vector<Element>::const_iterator position_;
// end_ is used only in update_position() to limit find.
std::vector<Element>::const_iterator end_;
// Optional predicates. If set they define which edges are skipped by the
// iterator.
mvcc::VersionList<Vertex> *vertex_{nullptr};
// For edge types we use a vector pointer because it's optional.
const std::vector<storage::EdgeType> *edge_types_ = nullptr;
/** Helper function that skips edges that don't satisfy the predicate
* present in this iterator. */
void update_position() {
if (vertex_) {
position_ = std::find_if(position_,
end_, [v = this->vertex_](const Element &e) {
return e.vertex == v;
});
}
if (edge_types_) {
position_ = std::find_if(position_, end_, [this](const Element &e) {
return utils::Contains(*edge_types_, e.edge_type);
});
}
}
};
public:
/**
* Adds an edge to this structure.
*
* @param vertex - The destination vertex of the edge. That's the one
* opposite from the vertex that contains this `Edges` instance.
* @param edge - The edge.
* @param edge_type - Type of the edge.
*/
void emplace(mvcc::VersionList<Vertex> *vertex, mvcc::VersionList<Edge> *edge,
storage::EdgeType edge_type) {
storage_.emplace_back(Element{vertex, edge, edge_type});
}
/**
* Removes an edge from this structure.
*/
void RemoveEdge(mvcc::VersionList<Edge> *edge) {
auto found = std::find_if(
storage_.begin(), storage_.end(),
[edge](const Element &element) { return edge == element.edge; });
// If the edge is not in the structure we don't care and can simply return
if (found == storage_.end()) return;
*found = std::move(storage_.back());
storage_.pop_back();
}
auto size() const { return storage_.size(); }
auto begin() const { return Iterator(storage_.begin()); }
auto end() const { return Iterator(storage_.end()); }
auto &storage() { return storage_; }
/**
* Creates a beginning iterator that will skip edges whose destination
* vertex is not equal to the given vertex.
*
* @param vertex - The destination vertex Address. If empty the
* edges are not filtered on destination.
* @param edge_types - The edge types at least one of which must be matched.
* If nullptr edges are not filtered on type.
*/
auto begin(mvcc::VersionList<Vertex> *vertex,
const std::vector<storage::EdgeType> *edge_types) const {
if (edge_types && edge_types->empty()) edge_types = nullptr;
return Iterator(storage_.begin(), storage_.end(), vertex, edge_types);
}
private:
std::vector<Element> storage_;
};

View File

@ -0,0 +1,70 @@
#pragma once
#include <glog/logging.h>
#include "data_structures/concurrent/skiplist.hpp"
#include "mvcc/single_node_ha/version_list.hpp"
#include "storage/single_node_ha/deferred_deleter.hpp"
#include "transactions/single_node_ha/engine.hpp"
/**
* @brief - Garbage collects deleted records.
* @tparam TCollection - type of collection. Must have a SkipList-like API
* (accessors).
* @tparam TRecord - type of underlying record in mvcc.
*/
template <typename TCollection, typename TRecord>
class GarbageCollector {
public:
GarbageCollector(
TCollection &collection, DeferredDeleter<TRecord> &record_deleter,
DeferredDeleter<mvcc::VersionList<TRecord>> &version_list_deleter)
: collection_(collection),
record_deleter_(record_deleter),
version_list_deleter_(version_list_deleter) {}
/**
* @brief - Runs garbage collector. Populates deferred deleters with version
* lists and records.
*
* @param snapshot - the GC snapshot. Consists of the oldest active
* transaction's snapshot, with that transaction's id appened as last.
* @param engine - reference to engine object
*/
void Run(const tx::Snapshot &snapshot, const tx::Engine &engine) {
auto collection_accessor = collection_.access();
uint64_t count = 0;
std::vector<typename DeferredDeleter<TRecord>::DeletedObject>
deleted_records;
std::vector<
typename DeferredDeleter<mvcc::VersionList<TRecord>>::DeletedObject>
deleted_version_lists;
for (auto id_vlist : collection_accessor) {
mvcc::VersionList<TRecord> *vlist = id_vlist.second;
// If the version_list is empty, i.e. there is nothing else to be read
// from it we can delete it.
auto ret = vlist->GcDeleted(snapshot, engine);
if (ret.first) {
deleted_version_lists.emplace_back(vlist, engine.LocalLast());
count += collection_accessor.remove(id_vlist.first);
}
if (ret.second != nullptr)
deleted_records.emplace_back(ret.second, engine.LocalLast());
}
DLOG_IF(INFO, count > 0)
<< "GC started cleaning with snapshot: " << snapshot;
DLOG_IF(INFO, count > 0) << "Destroyed: " << count;
// Add records to deleter, with the id larger or equal than the last active
// transaction.
record_deleter_.AddObjects(deleted_records);
// Add version_lists to deleter, with the id larger or equal than the last
// active transaction.
version_list_deleter_.AddObjects(deleted_version_lists);
}
private:
TCollection &collection_;
DeferredDeleter<TRecord> &record_deleter_;
DeferredDeleter<mvcc::VersionList<TRecord>> &version_list_deleter_;
};

View File

@ -0,0 +1,48 @@
#pragma once
#include <atomic>
#include <cstdint>
#include <experimental/optional>
#include "glog/logging.h"
#include "utils/atomic.hpp"
namespace gid {
/**
* Global ids are created by taking both the `local` object id, and `worker` id.
* A global ID has 64 bits. The lower kWorkerIdSize bits contain the worker ID.
* All the other (upper) bits contain the local ID.
*/
using Gid = uint64_t;
/**
* Threadsafe generation of new global ids which belong to the
* worker_id machine. Never call SetId after calling Next without an Id you are
* sure is going to be used for gid, i.e. SetId should only be called before
* first Next call. We want to make sure that every id that we generate is
* larger than the id set by SetId, we can ensure that by not allowing calls to
* SetId after Next which generated new id (incremented internal id counter).
*/
class Generator {
public:
/**
* Returns a globally unique identifier.
*
* @param requested_gid - The desired gid. If given, it will be returned and
* this generator's state updated accordingly.
*/
gid::Gid Next(std::experimental::optional<gid::Gid> requested_gid =
std::experimental::nullopt) {
if (requested_gid) {
utils::EnsureAtomicGe(next_local_id_, *requested_gid + 1);
return *requested_gid;
} else {
return next_local_id_++;
}
}
private:
std::atomic<uint64_t> next_local_id_{0};
};
} // namespace gid

View File

@ -0,0 +1,188 @@
#pragma once
#include "glog/logging.h"
#include "data_structures/concurrent/concurrent_map.hpp"
#include "mvcc/single_node_ha/version_list.hpp"
#include "storage/common/index.hpp"
#include "storage/common/types.hpp"
#include "storage/single_node_ha/edge.hpp"
#include "storage/single_node_ha/vertex.hpp"
#include "transactions/transaction.hpp"
#include "utils/total_ordering.hpp"
namespace database {
/**
* @brief Implements index update and acquire.
* @Tparam TKey - underlying type by which to key objects
* @Tparam TRecord - object stored under the given key
*/
template <typename TKey, typename TRecord>
class KeyIndex {
public:
KeyIndex() {}
KeyIndex(const KeyIndex &other) = delete;
KeyIndex(KeyIndex &&other) = delete;
KeyIndex &operator=(const KeyIndex &other) = delete;
KeyIndex &operator=(KeyIndex &&other) = delete;
/**
* @brief - Add record, vlist, if new, to TKey specific storage.
* @param key - TKey index to update.
* @param vlist - pointer to vlist entry to add
* @param record - pointer to record entry to add (contained in vlist)
*/
void Update(const TKey &key, mvcc::VersionList<TRecord> *vlist,
const TRecord *const record) {
GetKeyStorage(key)->access().insert(IndexEntry(vlist, record));
}
/**
* @brief - Get all the inserted vlists in TKey specific storage which
* still have that label visible in this transaction.
* @param key - key to query.
* @param t - current transaction, which determines visibility.
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
* @return iterable collection of vlists records<TRecord> with the requested
* TKey.
*/
auto GetVlists(const TKey &key, tx::Transaction &t, bool current_state) {
auto access = GetKeyStorage(key)->access();
auto begin = access.begin();
return index::GetVlists<typename SkipList<IndexEntry>::Iterator, IndexEntry,
TRecord>(
std::move(access), begin, [](const IndexEntry &) { return true; }, t,
[key](const IndexEntry &, const TRecord *record) {
return KeyIndex::Exists(key, record);
},
current_state);
}
/**
* @brief - Return number of items in skiplist associated with the given
* TKey. This number could be imprecise because of the underlying skiplist
* storage. Use this as a hint, and not as a rule.
* Moreover, some transaction probably sees only part of the skiplist since
* not all versions are visible for it. Also, garbage collection might now
* have been run for some time so the index might have accumulated garbage.
* @param key - key to query for.
* @return number of items
*/
auto Count(const TKey &key) { return GetKeyStorage(key)->access().size(); }
/**
* @brief - Removes from the index all entries for which records don't contain
* the given label anymore. Update all record which are not visible for any
* transaction with an id larger or equal to `id`.
*
* @param snapshot - the GC snapshot. Consists of the oldest active
* transaction's snapshot, with that transaction's id appened as last.
* @param engine - transaction engine to see which records are commited
*/
void Refresh(const tx::Snapshot &snapshot, tx::Engine &engine) {
return index::Refresh<TKey, IndexEntry, TRecord>(
indices_, snapshot, engine,
[](const TKey &key, const IndexEntry &entry) {
return KeyIndex::Exists(key, entry.record_);
});
}
/**
* Returns a vector of keys present in this index.
*/
std::vector<TKey> Keys() {
std::vector<TKey> keys;
for (auto &kv : indices_.access()) keys.push_back(kv.first);
return keys;
}
private:
/**
* @brief - Contains vlist and record pointers.
*/
class IndexEntry : public utils::TotalOrdering<IndexEntry> {
public:
IndexEntry(const IndexEntry &entry, const TRecord *const new_record)
: IndexEntry(entry.vlist_, new_record) {}
IndexEntry(mvcc::VersionList<TRecord> *const vlist,
const TRecord *const record)
: vlist_(vlist), record_(record) {}
// Comparision operators - we need them to keep this sorted inside
// skiplist.
// This needs to be sorted first by vlist and second record because we
// want to keep same vlists close together since we need to filter them to
// get only the unique ones.
bool operator<(const IndexEntry &other) const {
if (this->vlist_ != other.vlist_) return this->vlist_ < other.vlist_;
return this->record_ < other.record_;
}
bool operator==(const IndexEntry &other) const {
return this->vlist_ == other.vlist_ && this->record_ == other.record_;
}
/**
* @brief - Checks if previous IndexEntry has the same vlist as this
* IndexEntry.
* @return - true if the vlists match.
*/
bool IsAlreadyChecked(const IndexEntry &previous) const {
return previous.vlist_ == this->vlist_;
}
mvcc::VersionList<TRecord> *const vlist_;
const TRecord *const record_;
};
/**
* @brief - Get storage for this label. Creates new
* storage if this key is not yet indexed.
* @param key - key for which to access storage.
* @return pointer to skiplist of version list records<T>.
*/
auto GetKeyStorage(const TKey &key) {
auto access = indices_.access();
// Avoid excessive new/delete by first checking if it exists.
auto iter = access.find(key);
if (iter == access.end()) {
auto ret = access.insert(key, std::make_unique<SkipList<IndexEntry>>());
return ret.first->second.get();
}
return iter->second.get();
}
/**
* @brief - Check if Vertex contains label.
* @param label - label to check for.
* @return true if it contains, false otherwise.
*/
static bool Exists(storage::Label label, const Vertex *const v) {
DCHECK(v != nullptr) << "Vertex is nullptr.";
// We have to check for existance of label because the transaction
// might not see the label, or the label was deleted and not yet
// removed from the index.
const auto &labels = v->labels_;
return std::find(labels.begin(), labels.end(), label) != labels.end();
}
/**
* @brief - Check if Edge has edge_type.
* @param edge_type - edge_type to check for.
* @return true if it has that edge_type, false otherwise.
*/
static bool Exists(storage::EdgeType edge_type, const Edge *const e) {
DCHECK(e != nullptr) << "Edge is nullptr.";
// We have to check for equality of edge types because the transaction
// might not see the edge type, or the edge type was deleted and not yet
// removed from the index.
return e->edge_type_ == edge_type;
}
ConcurrentMap<TKey, std::unique_ptr<SkipList<IndexEntry>>> indices_;
};
} // namespace database

View File

@ -0,0 +1,616 @@
#pragma once
#include <experimental/optional>
#include "data_structures/concurrent/concurrent_map.hpp"
#include "data_structures/concurrent/skiplist.hpp"
#include "mvcc/single_node_ha/version_list.hpp"
#include "storage/common/index.hpp"
#include "storage/common/types.hpp"
#include "storage/single_node_ha/edge.hpp"
#include "storage/single_node_ha/vertex.hpp"
#include "transactions/transaction.hpp"
#include "utils/bound.hpp"
#include "utils/total_ordering.hpp"
namespace database {
/**
* @brief Implements LabelPropertyIndex.
* Currently this provides implementation for:
* acquiring all entries which contain the given label, and a given property
* sorted by the property value
* acquiring all non-unique entries with the given label, and property, with
* exactly one property value
*/
class LabelPropertyIndex {
public:
LabelPropertyIndex(){};
LabelPropertyIndex(const LabelPropertyIndex &other) = delete;
LabelPropertyIndex(LabelPropertyIndex &&other) = delete;
LabelPropertyIndex &operator=(const LabelPropertyIndex &other) = delete;
LabelPropertyIndex &operator=(LabelPropertyIndex &&other) = delete;
/**
* @brief - Contain Label + property, to be used as an index key.
*/
class Key : public utils::TotalOrdering<Key> {
public:
const storage::Label label_;
const storage::Property property_;
bool unique_{false};
Key(storage::Label label, storage::Property property)
: label_(label), property_(property) {}
Key(storage::Label label, storage::Property property, bool unique)
: label_(label), property_(property), unique_(unique) {}
// Comparison operators - we need them to keep this sorted inside skiplist.
bool operator<(const Key &other) const {
if (this->label_ != other.label_) return this->label_ < other.label_;
return this->property_ < other.property_;
}
bool operator==(const Key &other) const {
return this->label_ == other.label_ && this->property_ == other.property_;
}
};
/**
* @brief - Creates index with the given key if it doesn't exist. Note that
* you still need to populate the index with existing records.
* @return - True if it created the index, false if it already exists.
*/
bool CreateIndex(const Key &key) {
auto access = indices_.access();
// Avoid creation if it already exists.
auto iter = access.find(key);
if (iter != access.end()) return false;
auto ret = access.insert(key, std::make_unique<SkipList<IndexEntry>>());
return ret.second;
}
/**
* Returns if it succeeded in deleting the index and freeing the index memory
*/
void DeleteIndex(const Key &key) {
indices_.access().remove(key);
}
/** NOTE: All update methods aren't supporting the case where two threads
* try to update the index with the same value. If both of them conclude that
* the insert is valid, one will insert first and that makes the second insert
* invalid if the unique constraint set.
*/
/**
* @brief - Updates all indexes which should contain this vertex.
* @param vlist - pointer to vlist entry to add
* @param vertex - pointer to vertex record entry to add (contained in vlist)
*/
bool UpdateOnLabelProperty(mvcc::VersionList<Vertex> *const vlist,
const Vertex *const vertex) {
const auto &labels = vertex->labels_;
// We need to check if the given vertex can be inserted in all indexes
auto access = indices_.access();
for (auto &index : access) {
if (!index.first.unique_) continue;
// Vertex has the given label
if (std::find(labels.begin(), labels.end(), index.first.label_) ==
labels.end())
continue;
auto prop = vertex->properties_.at(index.first.property_);
if (prop.type() != PropertyValue::Type::Null) {
if (!CheckUniqueConstraint(*index.second, prop, vlist, vertex)) {
return false;
}
}
}
for (auto &index : access) {
// Vertex has the given label
if (std::find(labels.begin(), labels.end(), index.first.label_) ==
labels.end())
continue;
auto prop = vertex->properties_.at(index.first.property_);
if (prop.type() != PropertyValue::Type::Null) {
Insert(*index.second, prop, vlist, vertex);
}
}
return true;
}
/**
* @brief - Updates all indexes with `label` and any property in `vertex` that
* exists.
* @param label - indexes with this label might be updated if vertex contains
* the corresponding property.
* @param vlist - pointer to vlist entry to add
* @param vertex - pointer to vertex record entry to add (contained in vlist)
*/
bool UpdateOnLabel(storage::Label label,
mvcc::VersionList<Vertex> *const vlist,
const Vertex *const vertex) {
// We need to check if the given vertex can be inserted in all indexes
auto access = indices_.access();
for (auto &index : access) {
if (!index.first.unique_) continue;
if (index.first.label_ != label) continue;
auto prop = vertex->properties_.at(index.first.property_);
if (prop.type() != PropertyValue::Type::Null) {
if (!CheckUniqueConstraint(*index.second, prop, vlist, vertex)) {
return false;
}
}
}
for (auto &index : access) {
if (index.first.label_ != label) continue;
auto prop = vertex->properties_.at(index.first.property_);
if (prop.type() != PropertyValue::Type::Null) {
// Property exists and vertex should be added to skiplist.
Insert(*index.second, prop, vlist, vertex);
}
}
return true;
}
/**
* @brief - Updates all indexes with `property` and any label in `vertex` that
* exists.
* @param property - indexes with this property might be updated if vertex
* contains the corresponding label.
* @param vlist - pointer to vlist entry to add
* @param vertex - pointer to vertex record entry to add (contained in vlist)
*/
bool UpdateOnProperty(storage::Property property,
mvcc::VersionList<Vertex> *const vlist,
const Vertex *const vertex) {
const auto &labels = vertex->labels_;
// We need to check if the given vertex can be inserted in all indexes
auto access = indices_.access();
for (auto &index : access) {
if (!index.first.unique_) continue;
if (index.first.property_ != property) continue;
if (std::find(labels.begin(), labels.end(), index.first.label_) !=
labels.end()) {
// Label exists and vertex should be added to skiplist.
if (!CheckUniqueConstraint(*index.second,
vertex->properties_.at(property), vlist,
vertex)) {
return false;
}
}
}
for (auto &index : access) {
if (index.first.property_ != property) continue;
if (std::find(labels.begin(), labels.end(), index.first.label_) !=
labels.end()) {
// Label exists and vertex should be added to skiplist.
Insert(*index.second, vertex->properties_.at(property), vlist, vertex);
}
}
return true;
}
/**
* @brief - Get all the inserted vlists in key specific storage which still
* have that label and property visible in this transaction.
* @param key - Label+Property to query.
* @param t - current transaction, which determines visibility.
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
* @return iterable collection of vlists of vertex records with the requested
* key sorted ascendingly by the property value.
*/
auto GetVlists(const Key &key, const tx::Transaction &t, bool current_state) {
DCHECK(IndexExists(key)) << "Index not yet ready.";
auto access = GetKeyStorage(key)->access();
auto begin = access.begin();
return index::GetVlists<typename SkipList<IndexEntry>::Iterator, IndexEntry,
Vertex, SkipList<IndexEntry>>(
std::move(access), begin, [](const IndexEntry &) { return true; }, t,
[key](const IndexEntry &entry, const Vertex *const vertex) {
return LabelPropertyIndex::Exists(key, entry.value_, vertex);
},
current_state);
}
/**
* @brief - Get all the inserted vlists in key specific storage which still
* have that label and property visible in this transaction with property
* value equal to 'value'.
* @param key - Label+Property to query.
* @param value - vlists with this value will be returned
* @param t - current transaction, which determines visibility.
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
* @return iterable collection of vlists of vertex records with the requested
* key and value
*/
auto GetVlists(const Key &key, const PropertyValue &value,
const tx::Transaction &t, bool current_state) {
DCHECK(IndexExists(key)) << "Index not yet ready.";
auto access = GetKeyStorage(key)->access();
auto min_ptr = std::numeric_limits<std::uintptr_t>::min();
auto start_iter = access.find_or_larger(IndexEntry(
value, reinterpret_cast<mvcc::VersionList<Vertex> *>(min_ptr),
reinterpret_cast<const Vertex *>(min_ptr)));
return index::GetVlists<typename SkipList<IndexEntry>::Iterator, IndexEntry,
Vertex>(
std::move(access), start_iter,
[value](const IndexEntry &entry) {
return !IndexEntry::Less(value, entry.value_) &&
!IndexEntry::Less(entry.value_, value);
},
t,
[key](const IndexEntry &entry, const Vertex *const vertex) {
return LabelPropertyIndex::Exists(key, entry.value_, vertex);
},
current_state);
}
/**
* Get an iterable over all mvcc::VersionLists that are contained in this
* index and satisfy the given bounds.
*
* The returned iterator will only contain vertices/edges whose property value
* is comparable with the given bounds (w.r.t. type). This has implications on
* Cypher query execuction semantics which have not been resolved yet.
*
* At least one of the bounds must be specified. Bounds can't be @c
* PropertyValue::Null. If both bounds are specified, their PropertyValue
* elements must be of comparable types.
*
* @param key - Label+Property to query.
* @param lower - Lower bound of the interval.
* @param upper - Upper bound of the interval.
* @param t - current transaction, which determines visibility.
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
* @return iterable collection of mvcc:VersionLists pointers that
* satisfy the bounds and are visible to the given transaction.
*/
auto GetVlists(
const Key &key,
const std::experimental::optional<utils::Bound<PropertyValue>> lower,
const std::experimental::optional<utils::Bound<PropertyValue>> upper,
const tx::Transaction &transaction, bool current_state) {
DCHECK(IndexExists(key)) << "Index not yet ready.";
auto type = [](const auto &bound) { return bound.value().value().type(); };
CHECK(lower || upper) << "At least one bound must be provided";
CHECK(!lower || type(lower) != PropertyValue::Type::Null)
<< "Null value is not a valid index bound";
CHECK(!upper || type(upper) != PropertyValue::Type::Null)
<< "Null value is not a valid index bound";
// helper function for creating a bound with an IndexElement
auto make_index_bound = [](const auto &optional_bound, bool bottom) {
std::uintptr_t ptr_bound =
bottom ? std::numeric_limits<std::uintptr_t>::min()
: std::numeric_limits<std::uintptr_t>::max();
return IndexEntry(
optional_bound.value().value(),
reinterpret_cast<mvcc::VersionList<Vertex> *>(ptr_bound),
reinterpret_cast<const Vertex *>(ptr_bound));
};
auto access = GetKeyStorage(key)->access();
// create the iterator startpoint based on the lower bound
auto start_iter = lower
? access.find_or_larger(make_index_bound(
lower, lower.value().IsInclusive()))
: access.begin();
// a function that defines if an entry staisfies the filtering predicate.
// since we already handled the lower bound, we only need to deal with the
// upper bound and value type
std::function<bool(const IndexEntry &entry)> predicate;
if (lower && upper &&
!PropertyValue::AreComparableTypes(type(lower), type(upper)))
predicate = [](const IndexEntry &) { return false; };
else if (upper) {
auto upper_index_entry =
make_index_bound(upper, upper.value().IsExclusive());
predicate = [upper_index_entry](const IndexEntry &entry) {
return PropertyValue::AreComparableTypes(
entry.value_.type(), upper_index_entry.value_.type()) &&
entry < upper_index_entry;
};
} else {
auto lower_type = type(lower);
make_index_bound(lower, lower.value().IsExclusive());
predicate = [lower_type](const IndexEntry &entry) {
return PropertyValue::AreComparableTypes(entry.value_.type(),
lower_type);
};
}
return index::GetVlists<typename SkipList<IndexEntry>::Iterator, IndexEntry,
Vertex>(
std::move(access), start_iter, predicate, transaction,
[key](const IndexEntry &entry, const Vertex *const vertex) {
return LabelPropertyIndex::Exists(key, entry.value_, vertex);
},
current_state);
}
/**
* @brief - Check for existance of index.
* @param key - Index key
* @return true if the index with that key exists
*/
bool IndexExists(const Key &key) {
auto access = indices_.access();
return access.find(key) != access.end();
}
/**
* @brief - Return number of items in skiplist associated with the given
* key. This number could be imprecise because of the underlying skiplist
* storage. Use this as a hint, and not as a rule. Fails if index doesn't
* exist.
* Moreover, some transaction probably sees only part of the skiplist since
* not all versions are visible for it. Also, garbage collection might now
* have been run for some time so the index might have accumulated garbage.
* @param key - key to query for.
* @return number of items
*/
int64_t Count(const Key &key) {
auto index = GetKeyStorage(key);
CHECK(index != nullptr) << "Index doesn't exist.";
return index->access().size();
}
/**
* Returns the approximate position and count of the given value in the
* index for the given Key.
*
* Both are approximations for several reasons. Initially the position
* and count are obtained from the skipist (the index) and as such are
* not exact for perfromance reasons. At the same time the position
* and count are calculated based on property value comparison: an
* additional error is accumulated because the index could contain
* the same vertex with the same value multiple times,
* as well as the same vertex with different values.
*/
auto PositionAndCount(const Key &key, const PropertyValue &value) {
auto access = GetKeyStorage(key)->access();
return access.position_and_count(
value,
// the 'less' function
[](const PropertyValue &a, const IndexEntry &b) {
return IndexEntry::Less(a, b.value_);
},
// the 'equal_to' function
[](const PropertyValue &a, const IndexEntry &b) {
return !(IndexEntry::Less(a, b.value_) ||
IndexEntry::Less(b.value_, a));
});
}
/**
* @brief - Removes from the index all entries for which records don't contain
* the given label anymore, or the record was deleted before this transaction
* id.
*
* @param snapshot - the GC snapshot. Consists of the oldest active
* transaction's snapshot, with that transaction's id appened as last.
*/
void Refresh(const tx::Snapshot &snapshot, tx::Engine &engine) {
return index::Refresh<Key, IndexEntry, Vertex>(
indices_, snapshot, engine,
[](const Key &key, const IndexEntry &entry) {
return LabelPropertyIndex::Exists(key, entry.value_, entry.record_);
});
}
/**
* Returns a vector of keys present in this index.
*/
std::vector<Key> Keys() {
std::vector<Key> keys;
for (auto &kv : indices_.access()) keys.push_back(kv.first);
return keys;
}
private:
/**
* @brief - Contains value, vlist and vertex record to distinguish between
* index entries.
*/
class IndexEntry : public utils::TotalOrdering<IndexEntry> {
public:
IndexEntry(const IndexEntry &entry, const Vertex *new_record)
: IndexEntry(entry.value_, entry.vlist_, new_record) {}
IndexEntry(const PropertyValue &value, mvcc::VersionList<Vertex> *vlist,
const Vertex *record)
: value_(value), vlist_(vlist), record_(record) {}
// Comparision operators - we need them to keep this sorted inside
// skiplist.
bool operator<(const IndexEntry &other) const {
bool this_value_smaller = Less(this->value_, other.value_);
if (this_value_smaller || Less(other.value_, this->value_))
return this_value_smaller;
if (this->vlist_ != other.vlist_) return this->vlist_ < other.vlist_;
return this->record_ < other.record_;
}
bool operator==(const IndexEntry &other) const {
return !(*this < other) && !(other < *this);
}
/**
* @brief - For two property values - orders the records by type and then by
* value. Except for integers and doubles - those are both converted to
* double and then compared.
* @return true if the first property value is smaller( should be before)
* than the second one
*/
static bool Less(const PropertyValue &a, const PropertyValue &b) {
if (!PropertyValue::AreComparableTypes(a.type(), b.type()))
return a.type() < b.type();
if (a.type() == b.type()) {
switch (a.type()) {
case PropertyValue::Type::Null:
return false;
case PropertyValue::Type::String:
return a.Value<std::string>() < b.Value<std::string>();
case PropertyValue::Type::Bool:
return a.Value<bool>() < b.Value<bool>();
case PropertyValue::Type::Int:
return a.Value<int64_t>() < b.Value<int64_t>();
case PropertyValue::Type::Double:
return a.Value<double>() < b.Value<double>();
case PropertyValue::Type::List: {
auto va = a.Value<std::vector<PropertyValue>>();
auto vb = b.Value<std::vector<PropertyValue>>();
if (va.size() != vb.size()) return va.size() < vb.size();
return lexicographical_compare(va.begin(), va.end(), vb.begin(),
vb.end(), Less);
}
case PropertyValue::Type::Map: {
auto ma = a.Value<std::map<std::string, PropertyValue>>();
auto mb = b.Value<std::map<std::string, PropertyValue>>();
if (ma.size() != mb.size()) return ma.size() < mb.size();
const auto cmp = [](const auto &a, const auto &b) {
if (a.first != b.first)
return a.first < b.first;
else
return Less(a.second, b.second);
};
return lexicographical_compare(ma.begin(), ma.end(), mb.begin(),
mb.end(), cmp);
}
}
}
// helper for getting a double from PropertyValue, if possible
auto get_double = [](const PropertyValue &value) {
DCHECK(value.type() == PropertyValue::Type::Int ||
value.type() == PropertyValue::Type::Double)
<< "Invalid data type.";
if (value.type() == PropertyValue::Type::Int)
return static_cast<double>(value.Value<int64_t>());
return value.Value<double>();
};
// Types are int and double - convert int to double
return get_double(a) < get_double(b);
}
/**
* @brief - Check if previous IndexEntry represents the same vlist/value
* pair.
* @return - true if IndexEntries are equal by the vlist/value pair.
*/
bool IsAlreadyChecked(const IndexEntry &previous) const {
return previous.vlist_ == this->vlist_ &&
!Less(previous.value_, this->value_) &&
!Less(this->value_, previous.value_);
}
const PropertyValue value_;
mvcc::VersionList<Vertex> *const vlist_{nullptr};
const Vertex *const record_{nullptr};
};
/**
* @brief - Check if an insert is valid due to the unique constraint
* @param index - into which index to add
* @param value - value which to add
* @param vlist - pointer to vlist entry to add
* @param vertex - pointer to vertex record entry to add (contained in
* vlist)
* @param unique - unique constraint on index
* @return bool - true if valid, false otherwise
*/
bool CheckUniqueConstraint(SkipList<IndexEntry> &index,
const PropertyValue &value,
mvcc::VersionList<Vertex> *const vlist,
const Vertex *const vertex) {
auto access = index.access();
auto it = access.find_or_larger(IndexEntry{value, nullptr, nullptr});
// If not found.
if (it == access.end()) {
return true;
}
// If not equal.
if (IndexEntry::Less(it->value_, value) ||
IndexEntry::Less(value, it->value_)) {
return true;
}
return vlist->cypher_id() == it->vlist_->cypher_id();
}
/**
* @brief - Insert value, vlist, vertex into corresponding index (key) if
* the index exists.
* @param index - into which index to add
* @param value - value which to add
* @param vlist - pointer to vlist entry to add
* @param vertex - pointer to vertex record entry to add (contained in
* vlist)
* @param unique - unique constraint on index
*/
void Insert(SkipList<IndexEntry> &index, const PropertyValue &value,
mvcc::VersionList<Vertex> *const vlist,
const Vertex *const vertex) {
// Property exists and vertex should be added to skiplist.
index.access().insert(IndexEntry{value, vlist, vertex});
}
/**
* @brief - Get storage for this key.
* @param key - Label and and property for which to query.
* @return pointer to skiplist of IndexEntries, if none which matches key
* exists return nullptr
*/
SkipList<IndexEntry> *GetKeyStorage(const Key &key) {
auto access = indices_.access();
auto iter = access.find(key);
if (iter == access.end()) return nullptr;
return iter->second.get();
}
/**
* @brief - Check if Vertex contains label and property with the given
* value.
* @param key - label and property to check for.
* @param value - value of property to compare
* @return true if it contains, false otherwise.
*/
static bool Exists(const Key &key, const PropertyValue &value,
const Vertex *const v) {
DCHECK(v != nullptr) << "Vertex is nullptr.";
// We have to check for existance of label because the transaction
// might not see the label, or the label was deleted and not yet
// removed from the index.
const auto &labels = v->labels_;
if (std::find(labels.begin(), labels.end(), key.label_) == labels.end())
return false;
auto prop = v->properties_.at(key.property_);
// Property doesn't exists.
if (prop.type() == PropertyValue::Type::Null) return false;
// Property value is the same as expected.
return !IndexEntry::Less(prop, value) && !IndexEntry::Less(value, prop);
}
ConcurrentMap<Key, std::unique_ptr<SkipList<IndexEntry>>> indices_;
};
} // namespace database

View File

@ -0,0 +1,171 @@
#include "storage/single_node_ha/record_accessor.hpp"
#include <glog/logging.h>
#include "database/single_node_ha/graph_db_accessor.hpp"
#include "durability/single_node_ha/state_delta.hpp"
#include "storage/single_node_ha/edge.hpp"
#include "storage/single_node_ha/vertex.hpp"
using database::StateDelta;
template <typename TRecord>
RecordAccessor<TRecord>::RecordAccessor(mvcc::VersionList<TRecord> *address,
database::GraphDbAccessor &db_accessor)
: db_accessor_(&db_accessor), address_(address) {}
template <typename TRecord>
PropertyValue RecordAccessor<TRecord>::PropsAt(storage::Property key) const {
return current().properties_.at(key);
}
template <>
void RecordAccessor<Vertex>::PropsSet(storage::Property key,
PropertyValue value) {
auto &dba = db_accessor();
auto delta = StateDelta::PropsSetVertex(dba.transaction_id(), gid(), key,
dba.PropertyName(key), value);
update().properties_.set(key, value);
dba.UpdatePropertyIndex(key, *this, &update());
db_accessor().wal().Emplace(delta);
}
template <>
void RecordAccessor<Edge>::PropsSet(storage::Property key,
PropertyValue value) {
auto &dba = db_accessor();
auto delta = StateDelta::PropsSetEdge(dba.transaction_id(), gid(), key,
dba.PropertyName(key), value);
update().properties_.set(key, value);
db_accessor().wal().Emplace(delta);
}
template <>
void RecordAccessor<Vertex>::PropsErase(storage::Property key) {
auto &dba = db_accessor();
auto delta =
StateDelta::PropsSetVertex(dba.transaction_id(), gid(), key,
dba.PropertyName(key), PropertyValue::Null);
update().properties_.set(key, PropertyValue::Null);
db_accessor().wal().Emplace(delta);
}
template <>
void RecordAccessor<Edge>::PropsErase(storage::Property key) {
auto &dba = db_accessor();
auto delta =
StateDelta::PropsSetEdge(dba.transaction_id(), gid(), key,
dba.PropertyName(key), PropertyValue::Null);
update().properties_.set(key, PropertyValue::Null);
db_accessor().wal().Emplace(delta);
}
template <typename TRecord>
void RecordAccessor<TRecord>::PropsClear() {
std::vector<storage::Property> to_remove;
for (const auto &kv : update().properties_) to_remove.emplace_back(kv.first);
for (const auto &prop : to_remove) {
PropsErase(prop);
}
}
template <typename TRecord>
const PropertyValueStore &RecordAccessor<TRecord>::Properties() const {
return current().properties_;
}
template <typename TRecord>
bool RecordAccessor<TRecord>::operator==(const RecordAccessor &other) const {
DCHECK(db_accessor_->transaction_id() == other.db_accessor_->transaction_id())
<< "Not in the same transaction.";
return address_ == other.address_;
}
template <typename TRecord>
database::GraphDbAccessor &RecordAccessor<TRecord>::db_accessor() const {
return *db_accessor_;
}
template <typename TRecord>
gid::Gid RecordAccessor<TRecord>::gid() const {
return address_->gid_;
}
template <typename TRecord>
typename mvcc::VersionList<TRecord> *RecordAccessor<TRecord>::address() const {
return address_;
}
template <typename TRecord>
RecordAccessor<TRecord> &RecordAccessor<TRecord>::SwitchNew() {
if (!new_) {
// if new_ is not set yet, look for it
// we can just Reconstruct the pointers, old_ will get initialized
// to the same value as it has now, and the amount of work is the
// same as just looking for a new_ record
if (!Reconstruct())
DLOG(FATAL)
<< "RecordAccessor::SwitchNew - accessor invalid after Reconstruct";
}
current_ = new_ ? new_ : old_;
return *this;
}
template <typename TRecord>
RecordAccessor<TRecord> &RecordAccessor<TRecord>::SwitchOld() {
current_ = old_ ? old_ : new_;
return *this;
}
template <typename TRecord>
bool RecordAccessor<TRecord>::Reconstruct() const {
auto &dba = db_accessor();
const auto &addr = address();
addr->find_set_old_new(dba.transaction(), &old_, &new_);
current_ = old_ ? old_ : new_;
return old_ != nullptr || new_ != nullptr;
}
template <typename TRecord>
TRecord &RecordAccessor<TRecord>::update() const {
auto &dba = db_accessor();
// Edges have lazily initialize mutable, versioned data (properties).
if (std::is_same<TRecord, Edge>::value && current_ == nullptr) {
bool reconstructed = Reconstruct();
DCHECK(reconstructed) << "Unable to initialize record";
}
const auto &t = dba.transaction();
if (!new_ && old_->is_expired_by(t))
throw RecordDeletedError();
else if (new_ && new_->is_expired_by(t))
throw RecordDeletedError();
if (new_) return *new_;
const auto &addr = address();
new_ = addr->update(dba.transaction());
DCHECK(new_ != nullptr) << "RecordAccessor.new_ is null after update";
return *new_;
}
template <typename TRecord>
int64_t RecordAccessor<TRecord>::CypherId() const {
return address()->cypher_id();
}
template <typename TRecord>
const TRecord &RecordAccessor<TRecord>::current() const {
// Edges have lazily initialize mutable, versioned data (properties).
if (std::is_same<TRecord, Edge>::value && current_ == nullptr) {
bool reconstructed = Reconstruct();
DCHECK(reconstructed) << "Unable to initialize record";
}
DCHECK(current_ != nullptr) << "RecordAccessor.current_ pointer is nullptr";
return *current_;
}
template class RecordAccessor<Vertex>;
template class RecordAccessor<Edge>;

View File

@ -0,0 +1,202 @@
/// @file
#pragma once
#include <glog/logging.h>
#include "mvcc/single_node_ha/version_list.hpp"
#include "storage/common/property_value.hpp"
#include "storage/common/property_value_store.hpp"
#include "storage/common/types.hpp"
#include "storage/single_node_ha/gid.hpp"
#include "utils/total_ordering.hpp"
namespace database {
class GraphDbAccessor;
struct StateDelta;
}; // namespace database
/**
* An accessor to a database record (an Edge or a Vertex).
*
* Exposes view and update functions to the client programmer.
* Assumes responsibility of doing all the relevant book-keeping
* (such as index updates etc).
*
* @tparam TRecord Type of record (MVCC Version) of the accessor.
*/
template <typename TRecord>
class RecordAccessor : public utils::TotalOrdering<RecordAccessor<TRecord>> {
protected:
/**
* The database::GraphDbAccessor is friend to this accessor so it can
* operate on it's data (mvcc version-list and the record itself).
* This is legitimate because database::GraphDbAccessor creates
* RecordAccessors
* and is semantically their parent/owner. It is necessary because
* the database::GraphDbAccessor handles insertions and deletions, and these
* operations modify data intensively.
*/
friend database::GraphDbAccessor;
public:
/**
* @param address Address (local or global) of the Vertex/Edge of this
* accessor.
* @param db_accessor The DB accessor that "owns" this record accessor.
*/
RecordAccessor(mvcc::VersionList<TRecord> *address, database::GraphDbAccessor &db_accessor);
// this class is default copyable, movable and assignable
RecordAccessor(const RecordAccessor &other) = default;
RecordAccessor(RecordAccessor &&other) = default;
RecordAccessor &operator=(const RecordAccessor &other) = default;
RecordAccessor &operator=(RecordAccessor &&other) = default;
/** Gets the property for the given key. */
PropertyValue PropsAt(storage::Property key) const;
/** Sets a value on the record for the given property. */
void PropsSet(storage::Property key, PropertyValue value);
/** Erases the property for the given key. */
void PropsErase(storage::Property key);
/** Removes all the properties from this record. */
void PropsClear();
/** Returns the properties of this record. */
const PropertyValueStore &Properties() const;
bool operator==(const RecordAccessor &other) const;
/** Returns a GraphDB accessor of this record accessor. */
database::GraphDbAccessor &db_accessor() const;
/**
* Returns a globally-unique ID of this vertex or edge. Note that vertices
* and edges have separate ID domains, there can be a vertex with ID X and an
* edge with the same id.
*/
gid::Gid gid() const;
mvcc::VersionList<TRecord> *address() const;
/*
* Switches this record accessor to use the latest version visible to the
* current transaction+command. Possibly the one that was created by this
* transaction+command.
*
* @return A reference to this.
*/
RecordAccessor<TRecord> &SwitchNew();
/** Returns the new record pointer. */
TRecord *GetNew() const { return new_; }
/**
* Attempts to switch this accessor to use the latest version not updated by
* the current transaction+command. If that is not possible (vertex/edge was
* created by the current transaction/command), it does nothing (current
* remains pointing to the new version).
*
* @return A reference to this.
*/
RecordAccessor<TRecord> &SwitchOld();
/** Returns the old record pointer. */
TRecord *GetOld() const { return old_; }
/**
* Reconstructs the internal state of the record accessor so it uses the
* versions appropriate to this transaction+command.
*
* @return True if this accessor is valid after reconstruction. This means
* that at least one record pointer was found (either new_ or old_), possibly
* both.
*/
bool Reconstruct() const;
/**
* Ensures there is an updateable version of the record in the version_list,
* and that the `new_` pointer points to it. Returns a reference to that
* version.
*
* It is not legal to call this function on a Vertex/Edge that has been
* deleted in the current transaction+command.
*
* @throws RecordDeletedError
*/
TRecord &update() const;
/**
* Returns true if the given accessor is visible to the given transaction.
*
* @param current_state If true then the graph state for the
* current transaction+command is returned (insertions, updates and
* deletions performed in the current transaction+command are not
* ignored).
*/
bool Visible(const tx::Transaction &t, bool current_state) const {
return (old_ && !(current_state && old_->is_expired_by(t))) ||
(current_state && new_ && !new_->is_expired_by(t));
}
/**
* Returns Cypher Id of this record.
*/
int64_t CypherId() const;
protected:
/**
* Pointer to the version (either old_ or new_) that READ operations
* in the accessor should take data from. Note that WRITE operations
* should always use new_.
*
* This pointer can be null if created by an accessor which lazily reads from
* mvcc.
*/
mutable TRecord *current_{nullptr};
/** Returns the current version (either new_ or old_) set on this
* RecordAccessor. */
const TRecord &current() const;
private:
// The database accessor for which this record accessor is created
// Provides means of getting to the transaction and database functions.
// Immutable, set in the constructor and never changed.
database::GraphDbAccessor *db_accessor_;
mvcc::VersionList<TRecord> *address_;
/**
* Latest version which is visible to the current transaction+command
* but has not been created nor modified by the current transaction+command.
*
* Can be null only when the record itself (the version-list) has
* been created by the current transaction+command.
*/
mutable TRecord *old_{nullptr};
/**
* Version that has been modified (created or updated) by the current
* transaction+command.
*
* Can be null when the record has not been modified in the current
* transaction+command. It is also possible that the modification
* has happened, but this RecordAccessor does not know this. To
* ensure correctness, the `SwitchNew` function must check if this
* is null, and if it is it must check with the vlist_ if there is
* an update.
*/
mutable TRecord *new_{nullptr};
};
/** Error when trying to update a deleted record */
class RecordDeletedError : public utils::BasicException {
public:
RecordDeletedError()
: utils::BasicException(
"Can't update a record deleted in the current transaction+commad") {
}
};

View File

@ -0,0 +1,95 @@
#pragma once
#include <experimental/filesystem>
#include <experimental/optional>
#include "data_structures/concurrent/concurrent_map.hpp"
#include "mvcc/single_node_ha/version_list.hpp"
#include "storage/common/types.hpp"
#include "storage/kvstore/kvstore.hpp"
#include "storage/single_node_ha/edge.hpp"
#include "storage/single_node_ha/indexes/key_index.hpp"
#include "storage/single_node_ha/indexes/label_property_index.hpp"
#include "storage/single_node_ha/vertex.hpp"
#include "transactions/type.hpp"
namespace database {
class GraphDb;
};
namespace database {
/** A data structure containing the main data members of a graph database. */
class Storage {
public:
explicit Storage(const std::vector<std::string> &properties_on_disk)
: properties_on_disk_{properties_on_disk} {}
public:
~Storage() {
// Delete vertices and edges which weren't collected before, also deletes
// records inside version list
for (auto &id_vlist : vertices_.access()) delete id_vlist.second;
for (auto &id_vlist : edges_.access()) delete id_vlist.second;
}
Storage(const Storage &) = delete;
Storage(Storage &&) = delete;
Storage &operator=(const Storage &) = delete;
Storage &operator=(Storage &&) = delete;
gid::Generator &VertexGenerator() { return vertex_generator_; }
gid::Generator &EdgeGenerator() { return edge_generator_; }
LabelPropertyIndex &label_property_index() { return label_property_index_; }
/// Gets the local address for the given gid. Fails if not present.
template <typename TRecord>
mvcc::VersionList<TRecord> *LocalAddress(gid::Gid gid) const {
const auto &map = GetMap<TRecord>();
auto access = map.access();
auto found = access.find(gid);
CHECK(found != access.end())
<< "Failed to find "
<< (std::is_same<TRecord, Vertex>::value ? "vertex" : "edge")
<< " for gid: " << gid;
return found->second;
}
/// Gets names of properties stored on disk
std::vector<std::string> &PropertiesOnDisk() { return properties_on_disk_; }
private:
friend class GraphDbAccessor;
friend class StorageGc;
gid::Generator vertex_generator_;
gid::Generator edge_generator_;
// main storage for the graph
ConcurrentMap<gid::Gid, mvcc::VersionList<Vertex> *> vertices_;
ConcurrentMap<gid::Gid, mvcc::VersionList<Edge> *> edges_;
// indexes
KeyIndex<storage::Label, Vertex> labels_index_;
LabelPropertyIndex label_property_index_;
std::vector<std::string> properties_on_disk_;
/// Gets the Vertex/Edge main storage map.
template <typename TRecord>
const ConcurrentMap<gid::Gid, mvcc::VersionList<TRecord> *> &GetMap() const;
};
template <>
inline const ConcurrentMap<gid::Gid, mvcc::VersionList<Vertex> *>
&Storage::GetMap() const {
return vertices_;
}
template <>
inline const ConcurrentMap<gid::Gid, mvcc::VersionList<Edge> *>
&Storage::GetMap() const {
return edges_;
}
} // namespace database

View File

@ -0,0 +1,169 @@
#pragma once
#include <chrono>
#include <queue>
#include "data_structures/concurrent/concurrent_map.hpp"
#include "mvcc/single_node_ha/version_list.hpp"
#include "stats/metrics.hpp"
#include "storage/single_node_ha/deferred_deleter.hpp"
#include "storage/single_node_ha/edge.hpp"
#include "storage/single_node_ha/garbage_collector.hpp"
#include "storage/single_node_ha/gid.hpp"
#include "storage/single_node_ha/storage.hpp"
#include "storage/single_node_ha/vertex.hpp"
#include "transactions/single_node_ha/engine.hpp"
#include "utils/exceptions.hpp"
#include "utils/scheduler.hpp"
#include "utils/timer.hpp"
namespace database {
/** Garbage collection capabilities for database::Storage. Extracted into a
* separate class for better code organization, and because the GC requires a
* tx::Engine, while the Storage itself can exist without it. Even though, a
* database::Storage is always acompanied by a Gc.
*/
class StorageGc {
template <typename TRecord>
class MvccDeleter {
using VlistT = mvcc::VersionList<TRecord>;
public:
explicit MvccDeleter(ConcurrentMap<gid::Gid, VlistT *> &collection)
: gc_(collection, record_deleter_, version_list_deleter_) {}
DeferredDeleter<TRecord> record_deleter_;
DeferredDeleter<mvcc::VersionList<TRecord>> version_list_deleter_;
GarbageCollector<ConcurrentMap<gid::Gid, VlistT *>, TRecord> gc_;
};
public:
/** Creates a garbage collector for the given storage that uses the given
* tx::Engine. If `pause_sec` is greater then zero, then GC gets triggered
* periodically. */
StorageGc(Storage &storage, tx::Engine &tx_engine, int pause_sec)
: tx_engine_(tx_engine),
storage_(storage),
vertices_(storage.vertices_),
edges_(storage.edges_) {
if (pause_sec > 0)
scheduler_.Run(
"Storage GC", std::chrono::seconds(pause_sec), [this] {
try {
CollectGarbage();
} catch (const utils::BasicException &e) {
DLOG(WARNING)
<< "Couldn't perform storage garbage collection due to: "
<< e.what();
}
});
}
~StorageGc() {
// We have to stop the scheduler before destroying this class.
scheduler_.Stop();
edges_.record_deleter_.FreeExpiredObjects(tx::Transaction::MaxId());
vertices_.record_deleter_.FreeExpiredObjects(tx::Transaction::MaxId());
edges_.version_list_deleter_.FreeExpiredObjects(tx::Transaction::MaxId());
vertices_.version_list_deleter_.FreeExpiredObjects(
tx::Transaction::MaxId());
}
StorageGc(const StorageGc &) = delete;
StorageGc(StorageGc &&) = delete;
StorageGc &operator=(const StorageGc &) = delete;
StorageGc &operator=(StorageGc &&) = delete;
void CollectCommitLogGarbage(tx::TransactionId oldest_active) {
auto safe_to_delete = GetClogSafeTransaction(oldest_active);
if (safe_to_delete) tx_engine_.GarbageCollectCommitLog(*safe_to_delete);
}
void CollectGarbage() {
// main garbage collection logic
// see wiki documentation for logic explanation
VLOG(21) << "Garbage collector started";
const auto snapshot_gc = tx_engine_.GlobalGcSnapshot();
{
// This can be run concurrently
utils::Timer x;
vertices_.gc_.Run(snapshot_gc, tx_engine_);
edges_.gc_.Run(snapshot_gc, tx_engine_);
VLOG(21) << "Garbage collector mvcc phase time: " << x.Elapsed().count();
}
// This has to be run sequentially after gc because gc modifies
// version_lists and changes the oldest visible record, on which Refresh
// depends.
{
// This can be run concurrently
utils::Timer x;
storage_.labels_index_.Refresh(snapshot_gc, tx_engine_);
storage_.label_property_index_.Refresh(snapshot_gc, tx_engine_);
VLOG(21) << "Garbage collector index phase time: " << x.Elapsed().count();
}
{
// We free expired objects with snapshot.back(), which is
// the ID of the oldest active transaction (or next active, if there
// are no currently active). That's legal because that was the
// last possible transaction that could have obtained pointers
// to those records. New snapshot can be used, different than one used for
// first two phases of gc.
utils::Timer x;
const auto snapshot_gc = tx_engine_.GlobalGcSnapshot();
edges_.record_deleter_.FreeExpiredObjects(snapshot_gc.back());
vertices_.record_deleter_.FreeExpiredObjects(snapshot_gc.back());
edges_.version_list_deleter_.FreeExpiredObjects(snapshot_gc.back());
vertices_.version_list_deleter_.FreeExpiredObjects(snapshot_gc.back());
VLOG(21) << "Garbage collector deferred deletion phase time: "
<< x.Elapsed().count();
}
CollectCommitLogGarbage(snapshot_gc.back());
gc_txid_ranges_.emplace(snapshot_gc.back(), tx_engine_.GlobalLast());
VLOG(21) << "gc snapshot: " << snapshot_gc;
VLOG(21) << "edge_record_deleter_ size: " << edges_.record_deleter_.Count();
VLOG(21) << "vertex record deleter_ size: "
<< vertices_.record_deleter_.Count();
VLOG(21) << "edge_version_list_deleter_ size: "
<< edges_.version_list_deleter_.Count();
VLOG(21) << "vertex_version_list_deleter_ size: "
<< vertices_.version_list_deleter_.Count();
VLOG(21) << "vertices_ size: " << storage_.vertices_.access().size();
VLOG(21) << "edges_ size: " << storage_.edges_.access().size();
VLOG(21) << "Garbage collector finished.";
}
protected:
// Find the largest transaction from which everything older is safe to
// delete, ones for which the hints have been set in the gc phase, and no
// alive transaction from the time before the hints were set is still alive
// (otherwise that transaction could still be waiting for a resolution of
// the query to the commit log about some old transaction)
std::experimental::optional<tx::TransactionId> GetClogSafeTransaction(
tx::TransactionId oldest_active) {
std::experimental::optional<tx::TransactionId> safe_to_delete;
while (!gc_txid_ranges_.empty() &&
gc_txid_ranges_.front().second < oldest_active) {
safe_to_delete = gc_txid_ranges_.front().first;
gc_txid_ranges_.pop();
}
return safe_to_delete;
}
tx::Engine &tx_engine_;
utils::Scheduler scheduler_;
private:
Storage &storage_;
MvccDeleter<Vertex> vertices_;
MvccDeleter<Edge> edges_;
// History of <oldest active transaction, next transaction to be ran> ranges
// that gc operated on at some previous time - used to clear commit log
std::queue<std::pair<tx::TransactionId, tx::TransactionId>> gc_txid_ranges_;
};
} // namespace database

View File

@ -0,0 +1,28 @@
#pragma once
#include "mvcc/single_node_ha/record.hpp"
#include "mvcc/single_node_ha/version_list.hpp"
#include "storage/common/property_value_store.hpp"
#include "storage/common/types.hpp"
#include "storage/single_node_ha/edges.hpp"
class Vertex : public mvcc::Record<Vertex> {
public:
Vertex() = default;
// Returns new Vertex with copy of data stored in this Vertex, but without
// copying superclass' members.
Vertex *CloneData() { return new Vertex(*this); }
Edges out_;
Edges in_;
std::vector<storage::Label> labels_;
PropertyValueStore properties_;
private:
Vertex(const Vertex &other)
: mvcc::Record<Vertex>(),
out_(other.out_),
in_(other.in_),
labels_(other.labels_),
properties_(other.properties_) {}
};

View File

@ -0,0 +1,85 @@
#include "storage/single_node_ha/vertex_accessor.hpp"
#include <algorithm>
#include "database/single_node_ha/graph_db_accessor.hpp"
#include "durability/single_node_ha/state_delta.hpp"
#include "utils/algorithm.hpp"
VertexAccessor::VertexAccessor(mvcc::VersionList<Vertex> *address,
database::GraphDbAccessor &db_accessor)
: RecordAccessor(address, db_accessor) {
Reconstruct();
}
size_t VertexAccessor::out_degree() const { return current().out_.size(); }
size_t VertexAccessor::in_degree() const { return current().in_.size(); }
void VertexAccessor::add_label(storage::Label label) {
auto &dba = db_accessor();
auto delta = database::StateDelta::AddLabel(dba.transaction_id(), gid(),
label, dba.LabelName(label));
Vertex &vertex = update();
// not a duplicate label, add it
if (!utils::Contains(vertex.labels_, label)) {
vertex.labels_.emplace_back(label);
dba.wal().Emplace(delta);
dba.UpdateLabelIndices(label, *this, &vertex);
}
}
void VertexAccessor::remove_label(storage::Label label) {
auto &dba = db_accessor();
auto delta = database::StateDelta::RemoveLabel(dba.transaction_id(), gid(),
label, dba.LabelName(label));
Vertex &vertex = update();
if (utils::Contains(vertex.labels_, label)) {
auto &labels = vertex.labels_;
auto found = std::find(labels.begin(), labels.end(), delta.label);
std::swap(*found, labels.back());
labels.pop_back();
dba.wal().Emplace(delta);
}
}
bool VertexAccessor::has_label(storage::Label label) const {
auto &labels = this->current().labels_;
return std::find(labels.begin(), labels.end(), label) != labels.end();
}
const std::vector<storage::Label> &VertexAccessor::labels() const {
return this->current().labels_;
}
void VertexAccessor::RemoveOutEdge(mvcc::VersionList<Edge> *edge) {
auto &dba = db_accessor();
SwitchNew();
if (current().is_expired_by(dba.transaction())) return;
update().out_.RemoveEdge(edge);
}
void VertexAccessor::RemoveInEdge(mvcc::VersionList<Edge> *edge) {
auto &dba = db_accessor();
SwitchNew();
if (current().is_expired_by(dba.transaction())) return;
update().in_.RemoveEdge(edge);
}
std::ostream &operator<<(std::ostream &os, const VertexAccessor &va) {
os << "V(";
utils::PrintIterable(os, va.labels(), ":", [&](auto &stream, auto label) {
stream << va.db_accessor().LabelName(label);
});
os << " {";
utils::PrintIterable(os, va.Properties(), ", ",
[&](auto &stream, const auto &pair) {
stream << va.db_accessor().PropertyName(pair.first)
<< ": " << pair.second;
});
return os << "})";
}

View File

@ -0,0 +1,158 @@
#pragma once
#include <limits>
#include <set>
#include <vector>
#include <cppitertools/chain.hpp>
#include <cppitertools/imap.hpp>
#include "storage/single_node_ha/edge_accessor.hpp"
#include "storage/single_node_ha/record_accessor.hpp"
#include "storage/single_node_ha/vertex.hpp"
#include "utils/algorithm.hpp"
/**
* Provides ways for the client programmer (i.e. code generated
* by the compiler) to interact with a Vertex.
*
* This class indirectly inherits MVCC data structures and
* takes care of MVCC versioning.
*/
class VertexAccessor final : public RecordAccessor<Vertex> {
// Helper function for creating an iterator over edges.
// @param begin - begin iterator
// @param end - end iterator
// @param from - if true specifies that the vertex represents `from` part of
// the edge, otherwise it specifies `to` part of the edge
// @param vertex - one endpoint of every edge
// @param db_accessor - database accessor
// @return - Iterator over EdgeAccessors
template <typename TIterator>
static inline auto MakeAccessorIterator(
TIterator &&begin, TIterator &&end, bool from,
mvcc::VersionList<Vertex> *vertex,
database::GraphDbAccessor &db_accessor) {
return iter::imap(
[from, vertex, &db_accessor](auto &edges_element) {
if (from) {
return EdgeAccessor(edges_element.edge, db_accessor, vertex,
edges_element.vertex, edges_element.edge_type);
} else {
return EdgeAccessor(edges_element.edge, db_accessor,
edges_element.vertex, vertex,
edges_element.edge_type);
}
},
utils::Iterable<TIterator>(std::forward<TIterator>(begin),
std::forward<TIterator>(end)));
}
public:
VertexAccessor(mvcc::VersionList<Vertex> *address,
database::GraphDbAccessor &db_accessor);
/** Returns the number of outgoing edges. */
size_t out_degree() const;
/** Returns the number of incoming edges. */
size_t in_degree() const;
/** Adds a label to the Vertex. If the Vertex already has that label the call
* has no effect. */
void add_label(storage::Label label);
/** Removes a label from the Vertex. */
void remove_label(storage::Label label);
/** Indicates if the Vertex has the given label. */
bool has_label(storage::Label label) const;
/** Returns all the Labels of the Vertex. */
const std::vector<storage::Label> &labels() const;
/** Returns EdgeAccessors for all incoming edges. */
auto in() const {
return MakeAccessorIterator(current().in_.begin(), current().in_.end(),
false, address(), db_accessor());
}
/**
* Returns EdgeAccessors for all incoming edges.
*
* @param dest - The destination vertex filter.
* @param edge_types - Edge types filter. At least one be matched. If nullptr
* or empty, the parameter is ignored.
*/
auto in(const VertexAccessor &dest,
const std::vector<storage::EdgeType> *edge_types = nullptr) const {
return MakeAccessorIterator(current().in_.begin(dest.address(), edge_types),
current().in_.end(), false, address(),
db_accessor());
}
/**
* Returns EdgeAccessors for all incoming edges.
*
* @param edge_types - Edge types filter. At least one be matched. If nullptr
* or empty, the parameter is ignored.
*/
auto in(const std::vector<storage::EdgeType> *edge_types) const {
return MakeAccessorIterator(current().in_.begin(nullptr, edge_types),
current().in_.end(), false, address(),
db_accessor());
}
/** Returns EdgeAccessors for all outgoing edges. */
auto out() const {
return MakeAccessorIterator(current().out_.begin(), current().out_.end(),
true, address(), db_accessor());
}
/**
* Returns EdgeAccessors for all outgoing edges whose destination is the given
* vertex.
*
* @param dest - The destination vertex filter.
* @param edge_types - Edge types filter. At least one be matched. If nullptr
* or empty, the parameter is ignored.
*/
auto out(const VertexAccessor &dest,
const std::vector<storage::EdgeType> *edge_types = nullptr) const {
return MakeAccessorIterator(
current().out_.begin(dest.address(), edge_types), current().out_.end(),
true, address(), db_accessor());
}
/**
* Returns EdgeAccessors for all outgoing edges.
*
* @param edge_types - Edge types filter. At least one be matched. If nullptr
* or empty, the parameter is ignored.
*/
auto out(const std::vector<storage::EdgeType> *edge_types) const {
return MakeAccessorIterator(current().out_.begin(nullptr, edge_types),
current().out_.end(), true, address(),
db_accessor());
}
/** Removes the given edge from the outgoing edges of this vertex. Note that
* this operation should always be accompanied by the removal of the edge from
* the incoming edges on the other side and edge deletion. */
void RemoveOutEdge(mvcc::VersionList<Edge> *edge);
/** Removes the given edge from the incoming edges of this vertex. Note that
* this operation should always be accompanied by the removal of the edge from
* the outgoing edges on the other side and edge deletion. */
void RemoveInEdge(mvcc::VersionList<Edge> *edge);
};
std::ostream &operator<<(std::ostream &, const VertexAccessor &);
// hash function for the vertex accessor
namespace std {
template <>
struct hash<VertexAccessor> {
size_t operator()(const VertexAccessor &v) const { return v.gid(); };
};
} // namespace std

View File

@ -4,6 +4,10 @@
#include "storage/single_node/vertex_accessor.hpp"
#endif
#ifdef MG_SINGLE_NODE_HA
#include "storage/single_node_ha/vertex_accessor.hpp"
#endif
#ifdef MG_DISTRIBUTED
#include "storage/distributed/vertex_accessor.hpp"
#endif

View File

@ -4,6 +4,10 @@
#include "transactions/single_node/engine.hpp"
#endif
#ifdef MG_SINGLE_NODE_HA
#include "transactions/single_node_ha/engine.hpp"
#endif
#ifdef MG_DISTRIBUTED
#include "transactions/distributed/engine.hpp"
#endif

View File

@ -0,0 +1,178 @@
#include "transactions/single_node_ha/engine.hpp"
#include <limits>
#include <mutex>
#include "glog/logging.h"
#include "durability/single_node_ha/state_delta.hpp"
namespace tx {
Engine::Engine(durability::WriteAheadLog *wal) : wal_(wal) {}
Transaction *Engine::Begin() {
VLOG(11) << "[Tx] Starting transaction " << counter_ + 1;
std::lock_guard<utils::SpinLock> guard(lock_);
if (!accepting_transactions_.load())
throw TransactionEngineError(
"The transaction engine currently isn't accepting new transactions.");
return BeginTransaction(false);
}
Transaction *Engine::BeginBlocking(
std::experimental::optional<TransactionId> parent_tx) {
Snapshot wait_for_txs;
{
std::lock_guard<utils::SpinLock> guard(lock_);
if (!accepting_transactions_.load())
throw TransactionEngineError("Engine is not accepting new transactions");
// Block the engine from acceping new transactions.
accepting_transactions_.store(false);
// Set active transactions to abort ASAP.
for (auto transaction : active_) {
store_.find(transaction)->second->set_should_abort();
}
wait_for_txs = active_;
}
// Wait for all active transactions except the parent (optional) and ourselves
// to end.
for (auto id : wait_for_txs) {
if (parent_tx && *parent_tx == id) continue;
while (Info(id).is_active()) {
// TODO reconsider this constant, currently rule-of-thumb chosen
std::this_thread::sleep_for(std::chrono::microseconds(100));
}
}
// Only after all transactions have finished, start the blocking transaction.
std::lock_guard<utils::SpinLock> guard(lock_);
return BeginTransaction(true);
}
CommandId Engine::Advance(TransactionId id) {
std::lock_guard<utils::SpinLock> guard(lock_);
auto it = store_.find(id);
DCHECK(it != store_.end())
<< "Transaction::advance on non-existing transaction";
return it->second.get()->AdvanceCommand();
}
CommandId Engine::UpdateCommand(TransactionId id) {
std::lock_guard<utils::SpinLock> guard(lock_);
auto it = store_.find(id);
DCHECK(it != store_.end())
<< "Transaction::advance on non-existing transaction";
return it->second->cid();
}
void Engine::Commit(const Transaction &t) {
VLOG(11) << "[Tx] Commiting transaction " << t.id_;
std::lock_guard<utils::SpinLock> guard(lock_);
clog_.set_committed(t.id_);
active_.remove(t.id_);
if (wal_) {
wal_->Emplace(database::StateDelta::TxCommit(t.id_));
}
store_.erase(store_.find(t.id_));
if (t.blocking()) {
accepting_transactions_.store(true);
}
}
void Engine::Abort(const Transaction &t) {
VLOG(11) << "[Tx] Aborting transaction " << t.id_;
std::lock_guard<utils::SpinLock> guard(lock_);
clog_.set_aborted(t.id_);
active_.remove(t.id_);
if (wal_) {
wal_->Emplace(database::StateDelta::TxAbort(t.id_));
}
store_.erase(store_.find(t.id_));
if (t.blocking()) {
accepting_transactions_.store(true);
}
}
CommitLog::Info Engine::Info(TransactionId tx) const {
return clog_.fetch_info(tx);
}
Snapshot Engine::GlobalGcSnapshot() {
std::lock_guard<utils::SpinLock> guard(lock_);
// No active transactions.
if (active_.size() == 0) {
auto snapshot_copy = active_;
snapshot_copy.insert(counter_ + 1);
return snapshot_copy;
}
// There are active transactions.
auto snapshot_copy = store_.find(active_.front())->second->snapshot();
snapshot_copy.insert(active_.front());
return snapshot_copy;
}
Snapshot Engine::GlobalActiveTransactions() {
std::lock_guard<utils::SpinLock> guard(lock_);
Snapshot active_transactions = active_;
return active_transactions;
}
TransactionId Engine::LocalLast() const {
std::lock_guard<utils::SpinLock> guard(lock_);
return counter_;
}
TransactionId Engine::GlobalLast() const { return LocalLast(); }
TransactionId Engine::LocalOldestActive() const {
std::lock_guard<utils::SpinLock> guard(lock_);
return active_.empty() ? counter_ + 1 : active_.front();
}
void Engine::GarbageCollectCommitLog(TransactionId tx_id) {
clog_.garbage_collect_older(tx_id);
}
void Engine::LocalForEachActiveTransaction(
std::function<void(Transaction &)> f) {
std::lock_guard<utils::SpinLock> guard(lock_);
for (auto transaction : active_) {
f(*store_.find(transaction)->second);
}
}
Transaction *Engine::RunningTransaction(TransactionId tx_id) {
std::lock_guard<utils::SpinLock> guard(lock_);
auto found = store_.find(tx_id);
CHECK(found != store_.end())
<< "Can't return snapshot for an inactive transaction";
return found->second.get();
}
void Engine::EnsureNextIdGreater(TransactionId tx_id) {
std::lock_guard<utils::SpinLock> guard(lock_);
counter_ = std::max(tx_id, counter_);
}
Transaction *Engine::BeginTransaction(bool blocking) {
TransactionId id{++counter_};
Transaction *t = new Transaction(id, active_, *this, blocking);
active_.insert(id);
store_.emplace(id, t);
if (wal_) {
wal_->Emplace(database::StateDelta::TxBegin(id));
}
return t;
}
} // namespace tx

View File

@ -0,0 +1,76 @@
/// @file
#pragma once
#include <atomic>
#include <experimental/optional>
#include <unordered_map>
#include "durability/single_node_ha/wal.hpp"
#include "transactions/commit_log.hpp"
#include "transactions/transaction.hpp"
#include "utils/thread/sync.hpp"
namespace tx {
class TransactionEngineError : public utils::BasicException {
using utils::BasicException::BasicException;
};
/// Single-node deployment transaction engine. Has complete functionality.
class Engine final {
public:
/// @param wal - Optional. If present, the Engine will write tx
/// Begin/Commit/Abort atomically (while under lock).
explicit Engine(durability::WriteAheadLog *wal = nullptr);
Engine(const Engine &) = delete;
Engine(Engine &&) = delete;
Engine &operator=(const Engine &) = delete;
Engine &operator=(Engine &&) = delete;
Transaction *Begin();
/// Blocking transactions are used when we can't allow any other transaction to
/// run (besides this one). This is the reason why this transactions blocks the
/// engine from creating new transactions and waits for the existing ones to
/// finish.
Transaction *BeginBlocking(
std::experimental::optional<TransactionId> parent_tx);
CommandId Advance(TransactionId id);
CommandId UpdateCommand(TransactionId id);
void Commit(const Transaction &t);
void Abort(const Transaction &t);
CommitLog::Info Info(TransactionId tx) const;
Snapshot GlobalGcSnapshot();
Snapshot GlobalActiveTransactions();
TransactionId GlobalLast() const;
TransactionId LocalLast() const;
TransactionId LocalOldestActive() const;
void LocalForEachActiveTransaction(std::function<void(Transaction &)> f);
Transaction *RunningTransaction(TransactionId tx_id);
void EnsureNextIdGreater(TransactionId tx_id);
void GarbageCollectCommitLog(TransactionId tx_id);
auto &local_lock_graph() { return local_lock_graph_; }
const auto &local_lock_graph() const { return local_lock_graph_; }
private:
// Map lock dependencies. Each entry maps (tx_that_wants_lock,
// tx_that_holds_lock). Used for local deadlock resolution.
// TODO consider global deadlock resolution.
ConcurrentMap<TransactionId, TransactionId> local_lock_graph_;
TransactionId counter_{0};
CommitLog clog_;
std::unordered_map<TransactionId, std::unique_ptr<Transaction>> store_;
Snapshot active_;
mutable utils::SpinLock lock_;
// Optional. If present, the Engine will write tx Begin/Commit/Abort
// atomically (while under lock).
durability::WriteAheadLog *wal_{nullptr};
std::atomic<bool> accepting_transactions_{true};
// Helper method for transaction begin.
Transaction *BeginTransaction(bool blocking);
};
} // namespace tx

View File

@ -28,6 +28,10 @@ void PrintTo(const query::EdgeAtom::Direction &dir, std::ostream *os) {
using VertexAddress = mvcc::VersionList<Vertex> *;
using EdgeAddress = mvcc::VersionList<Edge> *;
#endif
#ifdef MG_SINGLE_NODE_HA
using VertexAddress = mvcc::VersionList<Vertex> *;
using EdgeAddress = mvcc::VersionList<Edge> *;
#endif
#ifdef MG_DISTRIBUTED
using VertexAddress = storage::Address<mvcc::VersionList<Vertex>>;
using EdgeAddress = storage::Address<mvcc::VersionList<Edge>>;