diff --git a/src/index/DeltaTriples.cpp b/src/index/DeltaTriples.cpp index d85c1ba0b4..719daafb7c 100644 --- a/src/index/DeltaTriples.cpp +++ b/src/index/DeltaTriples.cpp @@ -22,30 +22,56 @@ #include "util/Serializer/TripleSerializer.h" // ____________________________________________________________________________ -LocatedTriples::iterator& DeltaTriples::LocatedTripleHandles::forPermutation( +template +LocatedTriples::iterator& +DeltaTriples::State::LocatedTripleHandles::forPermutation( Permutation::Enum permutation) { return handles_[static_cast(permutation)]; } // ____________________________________________________________________________ void DeltaTriples::clear() { - triplesInserted_.clear(); - triplesDeleted_.clear(); - ql::ranges::for_each(locatedTriples(), &LocatedTriplesPerBlock::clear); + auto clearImpl = [](auto& state) { + state.triplesInserted_.clear(); + state.triplesDeleted_.clear(); + ql::ranges::for_each(state.locatedTriples_, &LocatedTriplesPerBlock::clear); + }; + clearImpl(state_); + clearImpl(internalState_); +} + +// ____________________________________________________________________________ +template +DeltaTriples::State& DeltaTriples::getState() { + if constexpr (isInternal) { + return internalState_; + } else { + return state_; + } +} + +// ____________________________________________________________________________ +template +auto& DeltaTriples::getLocatedTriple() { + return getState().locatedTriples_; } // ____________________________________________________________________________ -std::vector +template +std::vector::LocatedTripleHandles> DeltaTriples::locateAndAddTriples(CancellationHandle cancellationHandle, ql::span> triples, bool insertOrDelete, ad_utility::timer::TimeTracer& tracer) { - std::array, Permutation::ALL.size()> + constexpr const auto& allPermutations = Permutation::all(); + auto& lt = getLocatedTriple(); + std::array, allPermutations.size()> intermediateHandles; - for (auto permutation : Permutation::ALL) { + for (auto permutation : allPermutations) { tracer.beginTrace(std::string{Permutation::toString(permutation)}); tracer.beginTrace("locateTriples"); - auto& perm = index_.getPermutation(permutation); + auto& basePerm = index_.getPermutation(permutation); + auto& perm = isInternal ? basePerm.internalPermutation() : basePerm; auto locatedTriples = LocatedTriple::locateTriplesInPermutation( triples, perm.metaData().blockData(), perm.keyOrder(), insertOrDelete, cancellationHandle); @@ -53,15 +79,15 @@ DeltaTriples::locateAndAddTriples(CancellationHandle cancellationHandle, tracer.endTrace("locateTriples"); tracer.beginTrace("addToLocatedTriples"); intermediateHandles[static_cast(permutation)] = - this->locatedTriples()[static_cast(permutation)].add( - locatedTriples, tracer); + lt[static_cast(permutation)].add(locatedTriples, tracer); cancellationHandle->throwIfCancelled(); tracer.endTrace("addToLocatedTriples"); tracer.endTrace(Permutation::toString(permutation)); } tracer.beginTrace("transformHandles"); - std::vector handles{triples.size()}; - for (auto permutation : Permutation::ALL) { + std::vector::LocatedTripleHandles> handles{ + triples.size()}; + for (auto permutation : allPermutations) { for (size_t i = 0; i < triples.size(); i++) { handles[i].forPermutation(permutation) = intermediateHandles[static_cast(permutation)][i]; @@ -72,12 +98,14 @@ DeltaTriples::locateAndAddTriples(CancellationHandle cancellationHandle, } // ____________________________________________________________________________ -void DeltaTriples::eraseTripleInAllPermutations(LocatedTripleHandles& handles) { +template +void DeltaTriples::eraseTripleInAllPermutations( + typename State::LocatedTripleHandles& handles) { + auto& lt = getLocatedTriple(); // Erase for all permutations. - for (auto permutation : Permutation::ALL) { + for (auto permutation : Permutation::all()) { auto ltIter = handles.forPermutation(permutation); - locatedTriples()[static_cast(permutation)].erase(ltIter->blockIndex_, - ltIter); + lt[static_cast(permutation)].erase(ltIter->blockIndex_, ltIter); } } @@ -90,22 +118,32 @@ DeltaTriplesCount DeltaTriples::getCounts() const { void DeltaTriples::insertTriples(CancellationHandle cancellationHandle, Triples triples, ad_utility::timer::TimeTracer& tracer) { - AD_LOG_DEBUG << "Inserting" - << " " << triples.size() - << " triples (including idempotent triples)." << std::endl; - modifyTriplesImpl(std::move(cancellationHandle), std::move(triples), true, - triplesInserted_, triplesDeleted_, tracer); + modifyTriplesImpl(std::move(cancellationHandle), + std::move(triples), tracer); } // ____________________________________________________________________________ void DeltaTriples::deleteTriples(CancellationHandle cancellationHandle, Triples triples, ad_utility::timer::TimeTracer& tracer) { - AD_LOG_DEBUG << "Deleting" - << " " << triples.size() - << " triples (including idempotent triples)." << std::endl; - modifyTriplesImpl(std::move(cancellationHandle), std::move(triples), false, - triplesDeleted_, triplesInserted_, tracer); + modifyTriplesImpl(std::move(cancellationHandle), + std::move(triples), tracer); +} + +// ____________________________________________________________________________ +void DeltaTriples::insertInternalTriples( + CancellationHandle cancellationHandle, Triples triples, + ad_utility::timer::TimeTracer& tracer) { + modifyTriplesImpl(std::move(cancellationHandle), + std::move(triples), tracer); +} + +// ____________________________________________________________________________ +void DeltaTriples::deleteInternalTriples( + CancellationHandle cancellationHandle, Triples triples, + ad_utility::timer::TimeTracer& tracer) { + modifyTriplesImpl(std::move(cancellationHandle), + std::move(triples), tracer); } // ____________________________________________________________________________ @@ -164,11 +202,21 @@ void DeltaTriples::rewriteLocalVocabEntriesAndBlankNodes(Triples& triples) { } // ____________________________________________________________________________ +template void DeltaTriples::modifyTriplesImpl(CancellationHandle cancellationHandle, - Triples triples, bool insertOrDelete, - TriplesToHandlesMap& targetMap, - TriplesToHandlesMap& inverseMap, + Triples triples, ad_utility::timer::TimeTracer& tracer) { + AD_LOG_DEBUG << (insertOrDelete ? "Inserting" : "Deleting") << " " + << triples.size() << (isInternal ? " internal" : "") + << " triples (including idempotent triples)." << std::endl; + auto [targetMap, inverseMap] = [this]() { + auto& state = getState(); + if constexpr (insertOrDelete) { + return std::tie(state.triplesInserted_, state.triplesDeleted_); + } else { + return std::tie(state.triplesDeleted_, state.triplesInserted_); + } + }(); tracer.beginTrace("rewriteLocalVocabEntries"); rewriteLocalVocabEntriesAndBlankNodes(triples); tracer.endTrace("rewriteLocalVocabEntries"); @@ -188,14 +236,14 @@ void DeltaTriples::modifyTriplesImpl(CancellationHandle cancellationHandle, ql::ranges::for_each(triples, [this, &inverseMap](const IdTriple<0>& triple) { auto handle = inverseMap.find(triple); if (handle != inverseMap.end()) { - eraseTripleInAllPermutations(handle->second); + eraseTripleInAllPermutations(handle->second); inverseMap.erase(triple); } }); tracer.endTrace("removeInverseTriples"); tracer.beginTrace("locatedAndAdd"); - std::vector handles = locateAndAddTriples( + auto handles = locateAndAddTriples( std::move(cancellationHandle), triples, insertOrDelete, tracer); tracer.endTrace("locatedAndAdd"); tracer.beginTrace("markTriples"); @@ -215,6 +263,15 @@ LocatedTriplesSnapshot::getLocatedTriplesForPermutation( return locatedTriplesPerBlock_[static_cast(permutation)]; } +// ____________________________________________________________________________ +const LocatedTriplesPerBlock& +LocatedTriplesSnapshot::getInternalLocatedTriplesForPermutation( + Permutation::Enum permutation) const { + AD_CONTRACT_CHECK(permutation == Permutation::PSO || + permutation == Permutation::POS); + return internalLocatedTriplesPerBlock_[static_cast(permutation)]; +} + // ____________________________________________________________________________ SharedLocatedTriplesSnapshot DeltaTriples::getSnapshot() { // NOTE: Both members of the `LocatedTriplesSnapshot` are copied, but the @@ -224,7 +281,8 @@ SharedLocatedTriplesSnapshot DeltaTriples::getSnapshot() { ++nextSnapshotIndex_; return SharedLocatedTriplesSnapshot{ std::make_shared(LocatedTriplesSnapshot{ - locatedTriples(), localVocab_.getLifetimeExtender(), snapshotIndex})}; + state_.locatedTriples_, internalState_.locatedTriples_, + localVocab_.getLifetimeExtender(), snapshotIndex})}; } // ____________________________________________________________________________ @@ -328,16 +386,22 @@ SharedLocatedTriplesSnapshot DeltaTriplesManager::getCurrentSnapshot() const { // _____________________________________________________________________________ void DeltaTriples::setOriginalMetadata( Permutation::Enum permutation, - std::shared_ptr> metadata) { - locatedTriples() - .at(static_cast(permutation)) - .setOriginalMetadata(std::move(metadata)); + std::shared_ptr> metadata, + bool setInternalMetadata) { + auto& locatedTriplesPerBlock = + setInternalMetadata + ? internalState_.locatedTriples_.at(static_cast(permutation)) + : state_.locatedTriples_.at(static_cast(permutation)); + locatedTriplesPerBlock.setOriginalMetadata(std::move(metadata)); } // _____________________________________________________________________________ void DeltaTriples::updateAugmentedMetadata() { - ql::ranges::for_each(locatedTriples(), - &LocatedTriplesPerBlock::updateAugmentedMetadata); + auto update = [](auto& lt) { + ql::ranges::for_each(lt, &LocatedTriplesPerBlock::updateAugmentedMetadata); + }; + update(state_.locatedTriples_); + update(internalState_.locatedTriples_); } // _____________________________________________________________________________ @@ -345,7 +409,11 @@ void DeltaTriples::writeToDisk() const { if (!filenameForPersisting_.has_value()) { return; } - auto toRange = [](const TriplesToHandlesMap& map) { + // TODO Currently this only writes non-internal delta triples to + // disk. The internal triples will be regenerated when importing the rest + // again. In the future we might to also want to explicitly store the internal + // triples. + auto toRange = [](const State::TriplesToHandlesMap& map) { return map | ql::views::keys | ql::views::transform( [](const IdTriple<0>& triple) -> const std::array& { @@ -355,9 +423,9 @@ void DeltaTriples::writeToDisk() const { }; std::filesystem::path tempPath = filenameForPersisting_.value(); tempPath += ".tmp"; - ad_utility::serializeIds( - tempPath, localVocab_, - std::array{toRange(triplesDeleted_), toRange(triplesInserted_)}); + ad_utility::serializeIds(tempPath, localVocab_, + std::array{toRange(state_.triplesDeleted_), + toRange(state_.triplesInserted_)}); std::filesystem::rename(tempPath, filenameForPersisting_.value()); } diff --git a/src/index/DeltaTriples.h b/src/index/DeltaTriples.h index 983119fa01..65bd82f4bf 100644 --- a/src/index/DeltaTriples.h +++ b/src/index/DeltaTriples.h @@ -24,15 +24,17 @@ // Typedef for one `LocatedTriplesPerBlock` object for each of the six // permutations. +template using LocatedTriplesPerBlockAllPermutations = - std::array; + std::array().size()>; // The locations of a set of delta triples (triples that were inserted or // deleted since the index was built) in each of the six permutations, and a // local vocab. This is all the information that is required to perform a query // that correctly respects these delta triples, hence the name. struct LocatedTriplesSnapshot { - LocatedTriplesPerBlockAllPermutations locatedTriplesPerBlock_; + LocatedTriplesPerBlockAllPermutations locatedTriplesPerBlock_; + LocatedTriplesPerBlockAllPermutations internalLocatedTriplesPerBlock_; // Make sure to keep the local vocab alive as long as the snapshot is alive. // The `DeltaTriples` class may concurrently add new entries under the hood, // but this is safe because the `LifetimeExtender` prevents access entirely. @@ -42,6 +44,9 @@ struct LocatedTriplesSnapshot { // Get `TripleWithPosition` objects for given permutation. const LocatedTriplesPerBlock& getLocatedTriplesForPermutation( Permutation::Enum permutation) const; + // Get `TripleWithPosition` objects for given internal permutation. + const LocatedTriplesPerBlock& getInternalLocatedTriplesForPermutation( + Permutation::Enum permutation) const; }; // A shared pointer to a constant `LocatedTriplesSnapshot`, but as an explicit @@ -95,9 +100,6 @@ class DeltaTriples { const IndexImpl& index_; size_t nextSnapshotIndex_ = 0; - // The located triples for all the 6 permutations. - LocatedTriplesPerBlockAllPermutations locatedTriples_; - // The local vocabulary of the delta triples (they may have components, // which are not contained in the vocabulary of the original index). LocalVocab localVocab_; @@ -116,23 +118,32 @@ class DeltaTriples { static_assert(static_cast(Permutation::Enum::OSP) == 5); static_assert(Permutation::ALL.size() == 6); - // Each delta triple needs to know where it is stored in each of the six - // `LocatedTriplesPerBlock` above. - struct LocatedTripleHandles { - using It = LocatedTriples::iterator; - std::array handles_; - - LocatedTriples::iterator& forPermutation(Permutation::Enum permutation); + // Generic state wrapper to avoid code duplication for internal and regular + // triples. + template + struct State { + // The located triples for all the permutations. + LocatedTriplesPerBlockAllPermutations locatedTriples_; + // Each delta triple needs to know where it is stored in each of the six + // `LocatedTriplesPerBlock` above. + struct LocatedTripleHandles { + using It = LocatedTriples::iterator; + std::array().size()> handles_; + + LocatedTriples::iterator& forPermutation(Permutation::Enum permutation); + }; + using TriplesToHandlesMap = + ad_utility::HashMap, LocatedTripleHandles>; + // The sets of triples added to and subtracted from the original index. Any + // triple can be at most in one of the sets. The information whether a + // triple is in the index is missing. This means that a triple that is in + // the index may still be in the inserted set and vice versa. + TriplesToHandlesMap triplesInserted_; + TriplesToHandlesMap triplesDeleted_; }; - using TriplesToHandlesMap = - ad_utility::HashMap, LocatedTripleHandles>; - // The sets of triples added to and subtracted from the original index. Any - // triple can be at most in one of the sets. The information whether a triple - // is in the index is missing. This means that a triple that is in the index - // may still be in the inserted set and vice versa. - TriplesToHandlesMap triplesInserted_; - TriplesToHandlesMap triplesDeleted_; + State state_; + State internalState_; public: // Construct for given index. @@ -146,15 +157,13 @@ class DeltaTriples { // Get the common `LocalVocab` of the delta triples. private: LocalVocab& localVocab() { return localVocab_; } - auto& locatedTriples() { return locatedTriples_; } - const auto& locatedTriples() const { return locatedTriples_; } public: const LocalVocab& localVocab() const { return localVocab_; } const LocatedTriplesPerBlock& getLocatedTriplesForPermutation( Permutation::Enum permutation) const { - return locatedTriples_.at(static_cast(permutation)); + return state_.locatedTriples_.at(static_cast(permutation)); } // Clear `triplesAdded_` and `triplesSubtracted_` and all associated data @@ -163,13 +172,21 @@ class DeltaTriples { // The number of delta triples added and subtracted. int64_t numInserted() const { - return static_cast(triplesInserted_.size()); + return static_cast(state_.triplesInserted_.size()); } int64_t numDeleted() const { - return static_cast(triplesDeleted_.size()); + return static_cast(state_.triplesDeleted_.size()); } DeltaTriplesCount getCounts() const; + // The number of internal delta triples added and subtracted. + int64_t numInternalInserted() const { + return static_cast(internalState_.triplesInserted_.size()); + } + int64_t numInternalDeleted() const { + return static_cast(internalState_.triplesDeleted_.size()); + } + // Insert triples. void insertTriples(CancellationHandle cancellationHandle, Triples triples, ad_utility::timer::TimeTracer& tracer = @@ -180,6 +197,20 @@ class DeltaTriples { ad_utility::timer::TimeTracer& tracer = ad_utility::timer::DEFAULT_TIME_TRACER); + // Insert internal delta triples for efficient language filters and patterns. + // Currently only used by test code. + void insertInternalTriples(CancellationHandle cancellationHandle, + Triples triples, + ad_utility::timer::TimeTracer& tracer = + ad_utility::timer::DEFAULT_TIME_TRACER); + + // Delete triplesdelta triples for efficient language filters and patterns. + // Currently only used by test code. + void deleteInternalTriples(CancellationHandle cancellationHandle, + Triples triples, + ad_utility::timer::TimeTracer& tracer = + ad_utility::timer::DEFAULT_TIME_TRACER); + // If the `filename` is set, then `writeToDisk()` will write these // `DeltaTriples` to `filename.value()`. If `filename` is `nullopt`, then // `writeToDisk` will be a nullop. @@ -197,25 +228,38 @@ class DeltaTriples { SharedLocatedTriplesSnapshot getSnapshot(); // Register the original `metadata` for the given `permutation`. This has to - // be called before any updates are processed. + // be called before any updates are processed. If `setInternalMetadata` is + // true, this will set the metadata to the internal permutations instead. void setOriginalMetadata( Permutation::Enum permutation, - std::shared_ptr> metadata); + std::shared_ptr> metadata, + bool setInternalMetadata); // Update the block metadata. void updateAugmentedMetadata(); private: + // The the proper state according to the template parameter. This will either + // return a reference to `internalState_` or `state_`. + template + State& getState(); + + // Helper function to get the correct located triple (either internal or + // external), depending on the `internal` template parameter. + template + auto& getLocatedTriple(); + // Find the position of the given triple in the given permutation and add it // to each of the six `LocatedTriplesPerBlock` maps (one per permutation). // When `insertOrDelete` is `true`, the triples are inserted, otherwise // deleted. Return the iterators of where it was added (so that we can easily // delete it again from these maps later). - std::vector locateAndAddTriples( - CancellationHandle cancellationHandle, - ql::span> triples, bool insertOrDelete, - ad_utility::timer::TimeTracer& tracer = - ad_utility::timer::DEFAULT_TIME_TRACER); + template + std::vector::LocatedTripleHandles> + locateAndAddTriples(CancellationHandle cancellationHandle, + ql::span> triples, bool insertOrDelete, + ad_utility::timer::TimeTracer& tracer = + ad_utility::timer::DEFAULT_TIME_TRACER); // Common implementation for `insertTriples` and `deleteTriples`. When // `insertOrDelete` is `true`, the triples are inserted, `targetMap` contains @@ -223,9 +267,8 @@ class DeltaTriples { // triples. When `insertOrDelete` is `false`, the triples are deleted, and it // is the other way around:. This is used to resolve insertions or deletions // that are idempotent or cancel each other out. + template void modifyTriplesImpl(CancellationHandle cancellationHandle, Triples triples, - bool shouldExist, TriplesToHandlesMap& targetMap, - TriplesToHandlesMap& inverseMap, ad_utility::timer::TimeTracer& tracer = ad_utility::timer::DEFAULT_TIME_TRACER); @@ -246,7 +289,9 @@ class DeltaTriples { // NOTE: The iterators are invalid afterward. That is OK, as long as we also // delete the respective entry in `triplesInserted_` or `triplesDeleted_`, // which stores these iterators. - void eraseTripleInAllPermutations(LocatedTripleHandles& handles); + template + void eraseTripleInAllPermutations( + typename State::LocatedTripleHandles& handles); friend class DeltaTriplesManager; }; diff --git a/src/index/IndexImpl.cpp b/src/index/IndexImpl.cpp index 5a14880804..261ef6daab 100644 --- a/src/index/IndexImpl.cpp +++ b/src/index/IndexImpl.cpp @@ -946,17 +946,13 @@ void IndexImpl::createFromOnDiskIndex(const std::string& onDiskBase, // Load the permutations and register the original metadata for the delta // triples. - // TODO We could delegate the setting of the metadata to the - // `Permutation`class, but we first have to deal with The delta triples for - // the additional permutations. // The setting of the metadata doesn't affect the contents of the delta // triples, so we don't need to call `writeToDisk`, therefore the second // argument to `modify` is `false`. - auto setMetadata = [this](const Permutation& p) { + auto setMetadata = [this](const Permutation& permutation) { deltaTriplesManager().modify( - [&p](DeltaTriples& deltaTriples) { - deltaTriples.setOriginalMetadata(p.permutation(), - p.metaData().blockDataShared()); + [&permutation](DeltaTriples& deltaTriples) { + permutation.setOriginalMetadataForDeltaTriples(deltaTriples); }, false, false); }; diff --git a/src/index/Permutation.cpp b/src/index/Permutation.cpp index 4ad0b9220b..c8c8b40e91 100644 --- a/src/index/Permutation.cpp +++ b/src/index/Permutation.cpp @@ -64,6 +64,16 @@ void Permutation::loadFromDisk(const std::string& onDiskBase, isLoaded_ = true; } +// _____________________________________________________________________________ +void Permutation::setOriginalMetadataForDeltaTriples( + DeltaTriples& deltaTriples) const { + deltaTriples.setOriginalMetadata(permutation(), metaData().blockDataShared(), + isInternalPermutation_); + if (internalPermutation_ != nullptr) { + internalPermutation().setOriginalMetadataForDeltaTriples(deltaTriples); + } +} + // _____________________________________________________________________ IdTable Permutation::scan(const ScanSpecAndBlocks& scanSpecAndBlocks, ColumnIndicesRef additionalColumns, @@ -241,29 +251,27 @@ const Permutation& Permutation::getActualPermutation(Id id) const { ScanSpecification{id, std::nullopt, std::nullopt}); } -// TODO The following two functions always assume that there were no -// updates to the additional triples (which is technically true for now, because -// we never modify the additional triples with the delta triples, because there -// is some functionality missing for this. We have to fix this here and in the -// `DeltaTriples` class. - // ______________________________________________________________________ const LocatedTriplesPerBlock& Permutation::getLocatedTriplesForPermutation( const LocatedTriplesSnapshot& locatedTriplesSnapshot) const { - static const LocatedTriplesSnapshot emptySnapshot{ - {}, LocalVocab{}.getLifetimeExtender(), 0}; - const auto& actualSnapshot = - isInternalPermutation_ ? emptySnapshot : locatedTriplesSnapshot; - return actualSnapshot.getLocatedTriplesForPermutation(permutation_); + return isInternalPermutation_ + ? locatedTriplesSnapshot.getInternalLocatedTriplesForPermutation( + permutation_) + : locatedTriplesSnapshot.getLocatedTriplesForPermutation( + permutation_); } // ______________________________________________________________________ BlockMetadataRanges Permutation::getAugmentedMetadataForPermutation( const LocatedTriplesSnapshot& locatedTriplesSnapshot) const { BlockMetadataSpan blocks( - isInternalPermutation_ - ? meta_.blockData() - : getLocatedTriplesForPermutation(locatedTriplesSnapshot) - .getAugmentedMetadata()); + getLocatedTriplesForPermutation(locatedTriplesSnapshot) + .getAugmentedMetadata()); return {{blocks.begin(), blocks.end()}}; } + +// ______________________________________________________________________ +const Permutation& Permutation::internalPermutation() const { + AD_CONTRACT_CHECK(internalPermutation_ != nullptr); + return *internalPermutation_; +} diff --git a/src/index/Permutation.h b/src/index/Permutation.h index 2ec8c37f34..da2ff419af 100644 --- a/src/index/Permutation.h +++ b/src/index/Permutation.h @@ -23,6 +23,7 @@ class IdTable; class LocatedTriplesPerBlock; class SharedLocatedTriplesSnapshot; struct LocatedTriplesSnapshot; +class DeltaTriples; // Helper class to store static properties of the different permutations to // avoid code duplication. @@ -41,6 +42,16 @@ class Permutation { static constexpr auto OSP = Enum::OSP; static constexpr auto ALL = {Enum::PSO, Enum::POS, Enum::SPO, Enum::SOP, Enum::OPS, Enum::OSP}; + static constexpr auto INTERNAL = {Enum::PSO, Enum::POS}; + + template + static constexpr const auto& all() { + if constexpr (isInternal) { + return INTERNAL; + } else { + return ALL; + } + } using MetaData = IndexMetaDataMmapView; using Allocator = ad_utility::AllocatorWithLimit; @@ -64,6 +75,10 @@ class Permutation { std::function isInternalId, bool loadAdditional = false); + // Set the original metadata for the delta triples. This also sets the + // metadata for internal permutation if present. + void setOriginalMetadataForDeltaTriples(DeltaTriples& deltaTriples) const; + // For a given ID for the col0, retrieve all IDs of the col1 and col2. // If `col1Id` is specified, only the col2 is returned for triples that // additionally have the specified col1. .This is just a thin wrapper around @@ -188,6 +203,10 @@ class Permutation { Enum permutation() const { return permutation_; } + // Provide const access to a linked internal permutation. If no internal + // permutation is available, this function throws an exception. + const Permutation& internalPermutation() const; + private: // The base filename of the permutation without the suffix below std::string onDiskBase_; diff --git a/test/DeltaTriplesTest.cpp b/test/DeltaTriplesTest.cpp index bc0fd95813..e5cb36efab 100644 --- a/test/DeltaTriplesTest.cpp +++ b/test/DeltaTriplesTest.cpp @@ -104,26 +104,38 @@ TEST_F(DeltaTriplesTest, clear) { // Insert then clear. deltaTriples.insertTriples( cancellationHandle, makeIdTriples(vocab, localVocab, {" "})); + deltaTriples.insertInternalTriples( + cancellationHandle, + makeIdTriples(vocab, localVocab, + {" "})); - EXPECT_THAT(deltaTriples, NumTriples(1, 0, 1)); + EXPECT_THAT(deltaTriples, NumTriples(1, 0, 1, 1, 0)); deltaTriples.clear(); - EXPECT_THAT(deltaTriples, NumTriples(0, 0, 0)); + EXPECT_THAT(deltaTriples, NumTriples(0, 0, 0, 0, 0)); // Delete, insert and then clear. deltaTriples.deleteTriples( cancellationHandle, makeIdTriples(vocab, localVocab, {" "})); - EXPECT_THAT(deltaTriples, NumTriples(0, 1, 1)); + deltaTriples.deleteInternalTriples( + cancellationHandle, + makeIdTriples(vocab, localVocab, + {" "})); + EXPECT_THAT(deltaTriples, NumTriples(0, 1, 1, 0, 1)); deltaTriples.insertTriples( cancellationHandle, makeIdTriples(vocab, localVocab, {" "})); + deltaTriples.insertInternalTriples( + cancellationHandle, + makeIdTriples(vocab, localVocab, + {" "})); - EXPECT_THAT(deltaTriples, NumTriples(1, 1, 2)); + EXPECT_THAT(deltaTriples, NumTriples(1, 1, 2, 1, 1)); deltaTriples.clear(); - EXPECT_THAT(deltaTriples, NumTriples(0, 0, 0)); + EXPECT_THAT(deltaTriples, NumTriples(0, 0, 0, 0, 0)); } TEST_F(DeltaTriplesTest, insertTriplesAndDeleteTriples) { @@ -139,9 +151,12 @@ TEST_F(DeltaTriplesTest, insertTriplesAndDeleteTriples) { [](const auto& item) { return item.first; }); }; auto UnorderedTriplesAre = [&mapKeys, this, &vocab, &localVocab]( + [[maybe_unused]] auto isInternal, const std::vector& triples) -> testing::Matcher, DeltaTriples::LocatedTripleHandles>&> { + IdTriple<0>, + typename DeltaTriples::State< + decltype(isInternal)::value>::LocatedTripleHandles>&> { return testing::ResultOf( "mapKeys(...)", [&mapKeys](const auto map) { return mapKeys(map); }, testing::UnorderedElementsAreArray( @@ -154,23 +169,42 @@ TEST_F(DeltaTriplesTest, insertTriplesAndDeleteTriples) { auto StateIs = [&UnorderedTriplesAre]( size_t numInserted, size_t numDeleted, size_t numTriplesInAllPermutations, + size_t numInternalInserted, size_t numInternalDeleted, const std::vector& inserted, - const std::vector& deleted) + const std::vector& deleted, + const std::vector& internalInserted, + const std::vector& internalDeleted) -> testing::Matcher { + using ::testing::AllOf; + using ES = DeltaTriples::State; + using IS = DeltaTriples::State; return AllOf( - NumTriples(numInserted, numDeleted, numTriplesInAllPermutations), - AD_FIELD(DeltaTriples, triplesInserted_, UnorderedTriplesAre(inserted)), - AD_FIELD(DeltaTriples, triplesDeleted_, UnorderedTriplesAre(deleted))); + NumTriples(numInserted, numDeleted, numTriplesInAllPermutations, + numInternalInserted, numInternalDeleted), + AD_FIELD( + DeltaTriples, state_, + AllOf(AD_FIELD(ES, triplesInserted_, + UnorderedTriplesAre(std::false_type{}, inserted)), + AD_FIELD(ES, triplesDeleted_, + UnorderedTriplesAre(std::false_type{}, deleted)))), + AD_FIELD(DeltaTriples, internalState_, + AllOf(AD_FIELD(IS, triplesInserted_, + UnorderedTriplesAre(std::true_type{}, + internalInserted)), + AD_FIELD(IS, triplesDeleted_, + UnorderedTriplesAre(std::true_type{}, + internalDeleted))))); }; - EXPECT_THAT(deltaTriples, StateIs(0, 0, 0, {}, {})); + EXPECT_THAT(deltaTriples, StateIs(0, 0, 0, 0, 0, {}, {}, {}, {})); // Inserting triples. The triples being inserted must be sorted. deltaTriples.insertTriples( cancellationHandle, makeIdTriples(vocab, localVocab, {" ", " "})); - EXPECT_THAT(deltaTriples, - StateIs(2, 0, 2, {" ", " "}, {})); + EXPECT_THAT( + deltaTriples, + StateIs(2, 0, 2, 0, 0, {" ", " "}, {}, {}, {})); // We only locate triples in a Block but don't resolve whether they exist. // Inserting triples that exist in the index works normally. @@ -178,35 +212,36 @@ TEST_F(DeltaTriplesTest, insertTriplesAndDeleteTriples) { cancellationHandle, makeIdTriples(vocab, localVocab, {" "})); EXPECT_THAT( deltaTriples, - StateIs(3, 0, 3, {" ", " ", " "}, {})); + StateIs(3, 0, 3, 0, 0, {" ", " ", " "}, + {}, {}, {})); // Insert more triples. deltaTriples.insertTriples( cancellationHandle, makeIdTriples(vocab, localVocab, {" ", " "})); EXPECT_THAT(deltaTriples, - StateIs(5, 0, 5, + StateIs(5, 0, 5, 0, 0, {" ", " ", " ", " ", " "}, - {})); + {}, {}, {})); // Inserting already inserted triples has no effect. deltaTriples.insertTriples(cancellationHandle, makeIdTriples(vocab, localVocab, {" "})); EXPECT_THAT(deltaTriples, - StateIs(5, 0, 5, + StateIs(5, 0, 5, 0, 0, {" ", " ", " ", " ", " "}, - {})); + {}, {}, {})); // Deleting a previously inserted triple removes it from the inserted // triples and adds it to the deleted ones. deltaTriples.deleteTriples(cancellationHandle, makeIdTriples(vocab, localVocab, {" "})); - EXPECT_THAT(deltaTriples, StateIs(4, 1, 5, + EXPECT_THAT(deltaTriples, StateIs(4, 1, 5, 0, 0, {" ", " ", " ", " "}, - {" "})); + {" "}, {}, {})); // Deleting triples. deltaTriples.deleteTriples( @@ -214,9 +249,9 @@ TEST_F(DeltaTriplesTest, insertTriplesAndDeleteTriples) { makeIdTriples(vocab, localVocab, {" ", " "})); EXPECT_THAT( deltaTriples, - StateIs(4, 3, 7, + StateIs(4, 3, 7, 0, 0, {" ", " ", " ", " "}, - {" ", " ", " "})); + {" ", " ", " "}, {}, {})); // Deleting non-existent triples. deltaTriples.deleteTriples(cancellationHandle, @@ -224,9 +259,10 @@ TEST_F(DeltaTriplesTest, insertTriplesAndDeleteTriples) { EXPECT_THAT( deltaTriples, StateIs( - 4, 4, 8, + 4, 4, 8, 0, 0, {" ", " ", " ", " "}, - {" ", " ", " ", " "})); + {" ", " ", " ", " "}, + {}, {})); // Unsorted triples are not allowed, but the assertion that checks this is // 1. an `AD_EXPENSIVE_CHECK`. @@ -248,30 +284,104 @@ TEST_F(DeltaTriplesTest, insertTriplesAndDeleteTriples) { makeIdTriples(vocab, localVocab, {" ", " "})); EXPECT_THAT( deltaTriples, - StateIs(4, 6, 10, + StateIs(4, 6, 10, 0, 0, {" ", " ", " ", " "}, {" ", " ", " ", " ", - " ", " "})); + " ", " "}, + {}, {})); // Deleting previously deleted triples. deltaTriples.deleteTriples( cancellationHandle, makeIdTriples(vocab, localVocab, {" "})); EXPECT_THAT( deltaTriples, - StateIs(4, 6, 10, + StateIs(4, 6, 10, 0, 0, {" ", " ", " ", " "}, {" ", " ", " ", " ", - " ", " "})); + " ", " "}, + {}, {})); // Inserting previously deleted triple. deltaTriples.insertTriples(cancellationHandle, makeIdTriples(vocab, localVocab, {" "})); EXPECT_THAT(deltaTriples, - StateIs(5, 5, 10, + StateIs(5, 5, 10, 0, 0, + {" ", " ", " ", + " ", " "}, + {" ", " ", " ", + " ", " "}, + {}, {})); + + // Insert new internal triple. + deltaTriples.insertInternalTriples( + cancellationHandle, + makeIdTriples(vocab, localVocab, + {" "})); + EXPECT_THAT(deltaTriples, + StateIs(5, 5, 10, 1, 0, + {" ", " ", " ", + " ", " "}, + {" ", " ", " ", + " ", " "}, + {" "}, {})); + + // Remove "existing" internal triple. + deltaTriples.deleteInternalTriples( + cancellationHandle, + makeIdTriples(vocab, localVocab, + {" "})); + EXPECT_THAT(deltaTriples, + StateIs(5, 5, 10, 1, 1, + {" ", " ", " ", + " ", " "}, + {" ", " ", " ", + " ", " "}, + {" "}, + {" "})); + + // Remove previously inserted internal triple. + deltaTriples.deleteInternalTriples( + cancellationHandle, + makeIdTriples(vocab, localVocab, + {" "})); + EXPECT_THAT(deltaTriples, + StateIs(5, 5, 10, 0, 2, + {" ", " ", " ", + " ", " "}, + {" ", " ", " ", + " ", " "}, + {}, + {" ", + " "})); + + // Remove previously removes internal triple again. + deltaTriples.deleteInternalTriples( + cancellationHandle, + makeIdTriples(vocab, localVocab, + {" "})); + EXPECT_THAT(deltaTriples, + StateIs(5, 5, 10, 0, 2, + {" ", " ", " ", + " ", " "}, + {" ", " ", " ", + " ", " "}, + {}, + {" ", + " "})); + + // Inserting previously deleted internal triple. + deltaTriples.insertInternalTriples( + cancellationHandle, + makeIdTriples(vocab, localVocab, + {" "})); + EXPECT_THAT(deltaTriples, + StateIs(5, 5, 10, 1, 1, {" ", " ", " ", " ", " "}, {" ", " ", " ", - " ", " "})); + " ", " "}, + {" "}, + {" "})); } // Test the rewriting of local vocab entries and blank nodes. @@ -460,6 +570,8 @@ TEST_F(DeltaTriplesTest, restoreFromNonExistingFile) { EXPECT_NO_THROW(deltaTriples.readFromDisk()); EXPECT_EQ(deltaTriples.numDeleted(), 0); EXPECT_EQ(deltaTriples.numInserted(), 0); + EXPECT_EQ(deltaTriples.numInternalDeleted(), 0); + EXPECT_EQ(deltaTriples.numInternalInserted(), 0); } // _____________________________________________________________________________ @@ -582,6 +694,11 @@ TEST_F(DeltaTriplesTest, storeAndRestoreData) { EXPECT_EQ(deltaTriples.numDeleted(), 1); EXPECT_EQ(deltaTriples.numInserted(), 1); + // Currently we don't store internal delta triples in this format. In the + // future we might regenerate them from the regular delta triples, or change + // the format so they are also stored on disk. + EXPECT_EQ(deltaTriples.numInternalDeleted(), 0); + EXPECT_EQ(deltaTriples.numInternalInserted(), 0); EXPECT_THAT(deltaTriples.localVocab().getAllWordsForTesting(), ::testing::UnorderedElementsAre( @@ -591,7 +708,7 @@ TEST_F(DeltaTriplesTest, storeAndRestoreData) { ::testing::Eq("")))); std::vector> insertedTriples; - ql::ranges::copy(deltaTriples.triplesInserted_ | ql::views::keys, + ql::ranges::copy(deltaTriples.state_.triplesInserted_ | ql::views::keys, std::back_inserter(insertedTriples)); EXPECT_THAT( insertedTriples, @@ -604,7 +721,7 @@ TEST_F(DeltaTriplesTest, storeAndRestoreData) { .value()), Id::makeFromBool(true)}}))); std::vector> deletedTriples; - ql::ranges::copy(deltaTriples.triplesDeleted_ | ql::views::keys, + ql::ranges::copy(deltaTriples.state_.triplesDeleted_ | ql::views::keys, std::back_inserter(deletedTriples)); EXPECT_THAT( deletedTriples, diff --git a/test/DeltaTriplesTestHelpers.h b/test/DeltaTriplesTestHelpers.h index c5600b6b04..4ecba35610 100644 --- a/test/DeltaTriplesTestHelpers.h +++ b/test/DeltaTriplesTestHelpers.h @@ -46,11 +46,16 @@ inline auto NumTriplesInAllPermutations = // `getCounts()` of a `DeltaTriples` and `numTriples()` for all // `LocatedTriplesPerBlock` of the `DeltaTriples`. inline auto NumTriples = - [](int64_t inserted, int64_t deleted, - size_t inAllPermutations) -> testing::Matcher { + [](int64_t inserted, int64_t deleted, size_t inAllPermutations, + int64_t internalInserted = 0, + int64_t internalDeleted = 0) -> testing::Matcher { return testing::AllOf( AD_PROPERTY(DeltaTriples, numInserted, testing::Eq(inserted)), AD_PROPERTY(DeltaTriples, numDeleted, testing::Eq(deleted)), + AD_PROPERTY(DeltaTriples, numInternalInserted, + testing::Eq(internalInserted)), + AD_PROPERTY(DeltaTriples, numInternalDeleted, + testing::Eq(internalDeleted)), AD_PROPERTY(DeltaTriples, getCounts, testing::Eq(DeltaTriplesCount{inserted, deleted})), NumTriplesInAllPermutations(inAllPermutations));