diff --git a/src/index/CompressedRelation.cpp b/src/index/CompressedRelation.cpp index 4dee23172f..2f73d61059 100644 --- a/src/index/CompressedRelation.cpp +++ b/src/index/CompressedRelation.cpp @@ -21,12 +21,12 @@ void CompressedRelationReader::scan( ad_utility::SharedConcurrentTimeoutTimer timer) const { AD_CONTRACT_CHECK(result->numColumns() == NumColumns); - // get all the blocks where _col0FirstId <= col0Id <= _col0LastId + // get all the blocks where col0FirstId_ <= col0Id <= col0LastId_ struct KeyLhs { - Id _col0FirstId; - Id _col0LastId; + Id col0FirstId_; + Id col0LastId_; }; - Id col0Id = metadata._col0Id; + Id col0Id = metadata.col0Id_; // TODO Use a structured binding. Structured bindings are // currently not supported by clang when using OpenMP because clang internally // transforms the `#pragma`s into lambdas, and capturing structured bindings @@ -39,7 +39,7 @@ void CompressedRelationReader::scan( // we use clang 16. blockMetadata.begin(), blockMetadata.end(), KeyLhs{col0Id, col0Id}, [](const auto& a, const auto& b) { - return a._col0FirstId < b._col0FirstId && a._col0LastId < b._col0LastId; + return a.col0FirstId_ < b.col0FirstId_ && a.col0LastId_ < b.col0LastId_; }); // The total size of the result is now known. @@ -57,19 +57,19 @@ void CompressedRelationReader::scan( // actual scan result. bool firstBlockIsIncomplete = beginBlock < endBlock && - (beginBlock->_col0FirstId < col0Id || beginBlock->_col0LastId > col0Id); + (beginBlock->col0FirstId_ < col0Id || beginBlock->col0LastId_ > col0Id); auto lastBlock = endBlock - 1; bool lastBlockIsIncomplete = beginBlock < lastBlock && - (lastBlock->_col0FirstId < col0Id || lastBlock->_col0LastId > col0Id); + (lastBlock->col0FirstId_ < col0Id || lastBlock->col0LastId_ > col0Id); // Invariant: A relation spans multiple blocks exclusively or several // entities are stored completely in the same Block. AD_CORRECTNESS_CHECK(!firstBlockIsIncomplete || (beginBlock == lastBlock)); AD_CORRECTNESS_CHECK(!lastBlockIsIncomplete); if (firstBlockIsIncomplete) { - AD_CORRECTNESS_CHECK(metadata._offsetInBlock != + AD_CORRECTNESS_CHECK(metadata.offsetInBlock_ != std::numeric_limits::max()); } @@ -78,7 +78,7 @@ void CompressedRelationReader::scan( // the result. auto readIncompleteBlock = [&](const auto& block) { // A block is uniquely identified by its start position in the file. - auto cacheKey = block._offsetsAndCompressedSize.at(0)._offsetInFile; + auto cacheKey = block.offsetsAndCompressedSize_.at(0).offsetInFile_; auto uncompressedBuffer = blockCache_ .computeOnce(cacheKey, [&]() { @@ -88,12 +88,12 @@ void CompressedRelationReader::scan( ._resultPointer; // Extract the part of the block that actually belongs to the relation - auto numElements = metadata._numRows; + auto numElements = metadata.numRows_; AD_CORRECTNESS_CHECK(uncompressedBuffer->numColumns() == metadata.numColumns()); for (size_t i = 0; i < uncompressedBuffer->numColumns(); ++i) { const auto& inputCol = uncompressedBuffer->getColumn(i); - auto begin = inputCol.begin() + metadata._offsetInBlock; + auto begin = inputCol.begin() + metadata.offsetInBlock_; auto resultColumn = result->getColumn(i); AD_CORRECTNESS_CHECK(numElements <= spaceLeft); std::copy(begin, begin + numElements, resultColumn.begin()); @@ -130,7 +130,7 @@ void CompressedRelationReader::scan( std::move(compressedBuffer)]() { ad_utility::TimeBlockAndLog tbl{"Decompressing a block"}; - decompressBlockToExistingIdTable(compressedBuffer, block._numRows, + decompressBlockToExistingIdTable(compressedBuffer, block.numRows_, *result, rowIndexOfNextBlock); }; @@ -144,8 +144,8 @@ void CompressedRelationReader::scan( // this is again serial code, set up the correct pointers // for the next block; - spaceLeft -= block._numRows; - rowIndexOfNextBlock += block._numRows; + spaceLeft -= block.numRows_; + rowIndexOfNextBlock += block.numRows_; } AD_CORRECTNESS_CHECK(spaceLeft == 0); } // End of omp parallel region, all the decompression was handled now. @@ -162,20 +162,20 @@ void CompressedRelationReader::scan( // Get all the blocks that possibly might contain our pair of col0Id and // col1Id struct KeyLhs { - Id _col0FirstId; - Id _col0LastId; - Id _col1FirstId; - Id _col1LastId; + Id col0FirstId_; + Id col0LastId_; + Id col1FirstId_; + Id col1LastId_; }; auto comp = [](const auto& a, const auto& b) { - bool endBeforeBegin = a._col0LastId < b._col0FirstId; + bool endBeforeBegin = a.col0LastId_ < b.col0FirstId_; endBeforeBegin |= - (a._col0LastId == b._col0FirstId && a._col1LastId < b._col1FirstId); + (a.col0LastId_ == b.col0FirstId_ && a.col1LastId_ < b.col1FirstId_); return endBeforeBegin; }; - Id col0Id = metaData._col0Id; + Id col0Id = metaData.col0Id_; // Note: See the comment in the other overload for `scan` above for the // reason why we (currently) can't use a structured binding here. @@ -187,7 +187,7 @@ void CompressedRelationReader::scan( // Invariant: The col0Id is completely stored in a single block, or it is // contained in multiple blocks that only contain this col0Id, bool col0IdHasExclusiveBlocks = - metaData._offsetInBlock == std::numeric_limits::max(); + metaData.offsetInBlock_ == std::numeric_limits::max(); if (!col0IdHasExclusiveBlocks) { // This might also be zero if no block was found at all. AD_CORRECTNESS_CHECK(endBlock - beginBlock <= 1); @@ -207,13 +207,13 @@ void CompressedRelationReader::scan( // Find the range in the block, that belongs to the same relation `col0Id` bool containedInOnlyOneBlock = - metaData._offsetInBlock != std::numeric_limits::max(); + metaData.offsetInBlock_ != std::numeric_limits::max(); auto begin = col1Column.begin(); if (containedInOnlyOneBlock) { - begin += metaData._offsetInBlock; + begin += metaData.offsetInBlock_; } auto end = - containedInOnlyOneBlock ? begin + metaData._numRows : col1Column.end(); + containedInOnlyOneBlock ? begin + metaData.numRows_ : col1Column.end(); // Find the range in the block, where also the col1Id matches (the second // ID in the `std::array` does not matter). @@ -250,7 +250,7 @@ void CompressedRelationReader::scan( // First accumulate the complete blocks in the "middle" auto totalResultSize = std::accumulate( beginBlock, endBlock, 0ul, [](const auto& count, const auto& block) { - return count + block._numRows; + return count + block.numRows_; }); // Add the possibly incomplete blocks from the beginning and end; totalResultSize += firstBlockResult.size() + lastBlockResult.size(); @@ -270,7 +270,7 @@ void CompressedRelationReader::scan( const auto& block = *beginBlock; // Read the block serially, only read the second column. - AD_CORRECTNESS_CHECK(block._offsetsAndCompressedSize.size() == 2); + AD_CORRECTNESS_CHECK(block.offsetsAndCompressedSize_.size() == 2); CompressedBlock compressedBuffer = readCompressedBlockFromFile(block, file, std::vector{1ul}); @@ -281,7 +281,7 @@ void CompressedRelationReader::scan( std::move(compressedBuffer)]() mutable { ad_utility::TimeBlockAndLog tbl{"Decompression a block"}; - decompressBlockToExistingIdTable(compressedBuffer, block._numRows, + decompressBlockToExistingIdTable(compressedBuffer, block.numRows_, *result, rowIndexOfNextBlockStart); }; @@ -295,7 +295,7 @@ void CompressedRelationReader::scan( } // update the pointers - rowIndexOfNextBlockStart += block._numRows; + rowIndexOfNextBlockStart += block.numRows_; } // end of parallel region } // Add the last block. @@ -328,7 +328,7 @@ void CompressedRelationWriter::addRelation(Id col0Id, float multC1 = computeMultiplicity(col1And2Ids.numRows(), numDistinctCol1); // Dummy value that will be overwritten later float multC2 = 42.42; - // This sets everything except the _offsetInBlock, which will be set + // This sets everything except the offsetInBlock_, which will be set // explicitly below. CompressedRelationMetadata metaData{col0Id, col1And2Ids.numRows(), multC1, multC2}; @@ -345,10 +345,10 @@ void CompressedRelationWriter::addRelation(Id col0Id, // this relation are too large, we will write the buffered relations to file // and start a new block. bool relationHasExclusiveBlocks = - sizeInBytes(col1And2Ids) > 0.8 * static_cast(_numBytesPerBlock); + sizeInBytes(col1And2Ids) > 0.8 * static_cast(numBytesPerBlock_); if (relationHasExclusiveBlocks || - sizeInBytes(col1And2Ids) + sizeInBytes(_buffer) > - static_cast(_numBytesPerBlock) * 1.5) { + sizeInBytes(col1And2Ids) + sizeInBytes(buffer_) > + static_cast(numBytesPerBlock_) * 1.5) { writeBufferedRelationsToSingleBlock(); } @@ -356,32 +356,33 @@ void CompressedRelationWriter::addRelation(Id col0Id, // The relation is large, immediately write the relation to a set of // exclusive blocks. writeRelationToExclusiveBlocks(col0Id, col1And2Ids); - metaData._offsetInBlock = std::numeric_limits::max(); + metaData.offsetInBlock_ = std::numeric_limits::max(); } else { // Append to the current buffered block. - metaData._offsetInBlock = _buffer.numRows(); + metaData.offsetInBlock_ = buffer_.numRows(); static_assert(sizeof(col1And2Ids[0][0]) == sizeof(Id)); - if (_buffer.numRows() == 0) { - _currentBlockData._col0FirstId = col0Id; - _currentBlockData._col1FirstId = col1And2Ids(0, 0); + if (buffer_.numRows() == 0) { + currentBlockData_.col0FirstId_ = col0Id; + currentBlockData_.col1FirstId_ = col1And2Ids(0, 0); } - _currentBlockData._col0LastId = col0Id; - _currentBlockData._col1LastId = col1And2Ids(col1And2Ids.numRows() - 1, 0); - AD_CORRECTNESS_CHECK(_buffer.numColumns() == col1And2Ids.numColumns()); - auto bufferOldSize = _buffer.numRows(); - _buffer.resize(_buffer.numRows() + col1And2Ids.numRows()); + currentBlockData_.col0LastId_ = col0Id; + currentBlockData_.col1LastId_ = col1And2Ids(col1And2Ids.numRows() - 1, 0); + currentBlockData_.col2LastId_ = col1And2Ids(col1And2Ids.numRows() - 1, 1); + AD_CORRECTNESS_CHECK(buffer_.numColumns() == col1And2Ids.numColumns()); + auto bufferOldSize = buffer_.numRows(); + buffer_.resize(buffer_.numRows() + col1And2Ids.numRows()); for (size_t i = 0; i < col1And2Ids.numColumns(); ++i) { const auto& column = col1And2Ids.getColumn(i); - std::ranges::copy(column, _buffer.getColumn(i).begin() + bufferOldSize); + std::ranges::copy(column, buffer_.getColumn(i).begin() + bufferOldSize); } } - _metaDataBuffer.push_back(metaData); + metaDataBuffer_.push_back(metaData); } // _____________________________________________________________________________ void CompressedRelationWriter::writeRelationToExclusiveBlocks( Id col0Id, const BufferedIdTable& data) { - const size_t numRowsPerBlock = _numBytesPerBlock / (NumColumns * sizeof(Id)); + const size_t numRowsPerBlock = numBytesPerBlock_ / (NumColumns * sizeof(Id)); AD_CORRECTNESS_CHECK(numRowsPerBlock > 0); AD_CORRECTNESS_CHECK(data.numColumns() == NumColumns); const auto totalSize = data.numRows(); @@ -394,38 +395,39 @@ void CompressedRelationWriter::writeRelationToExclusiveBlocks( {column.begin() + i, column.begin() + i + actualNumRowsPerBlock})); } - _blockBuffer.push_back(CompressedBlockMetadata{ + blockBuffer_.push_back(CompressedBlockMetadata{ std::move(offsets), actualNumRowsPerBlock, col0Id, col0Id, data[i][0], - data[i + actualNumRowsPerBlock - 1][0]}); + data[i + actualNumRowsPerBlock - 1][0], + data[i + actualNumRowsPerBlock - 1][1]}); } } // ___________________________________________________________________________ void CompressedRelationWriter::writeBufferedRelationsToSingleBlock() { - if (_buffer.empty()) { + if (buffer_.empty()) { return; } - AD_CORRECTNESS_CHECK(_buffer.numColumns() == NumColumns); + AD_CORRECTNESS_CHECK(buffer_.numColumns() == NumColumns); // Convert from bytes to number of ID pairs. - size_t numRows = _buffer.numRows(); + size_t numRows = buffer_.numRows(); // TODO This is - // `ranges::to(ranges::transform_view(_buffer.getColumns(), + // `ranges::to(ranges::transform_view(buffer_.getColumns(), // compressAndWriteColumn))`; - std::ranges::for_each(_buffer.getColumns(), + std::ranges::for_each(buffer_.getColumns(), [this](const auto& column) mutable { - _currentBlockData._offsetsAndCompressedSize.push_back( + currentBlockData_.offsetsAndCompressedSize_.push_back( compressAndWriteColumn(column)); }); - _currentBlockData._numRows = numRows; - // The `firstId` and `lastId` of `_currentBlockData` were already set + currentBlockData_.numRows_ = numRows; + // The `firstId` and `lastId` of `currentBlockData_` were already set // correctly by `addRelation()`. - _blockBuffer.push_back(_currentBlockData); + blockBuffer_.push_back(currentBlockData_); // Reset the data of the current block. - _currentBlockData = CompressedBlockMetadata{}; - _buffer.clear(); + currentBlockData_ = CompressedBlockMetadata{}; + buffer_.clear(); } // _____________________________________________________________________________ @@ -448,10 +450,10 @@ CompressedBlock CompressedRelationReader::readCompressedBlockFromFile( // TODO Use `std::views::zip` for (size_t i = 0; i < compressedBuffer.size(); ++i) { const auto& offset = - blockMetaData._offsetsAndCompressedSize.at(columnIndices->at(i)); + blockMetaData.offsetsAndCompressedSize_.at(columnIndices->at(i)); auto& currentCol = compressedBuffer[i]; - currentCol.resize(offset._compressedSize); - file.read(currentCol.data(), offset._compressedSize, offset._offsetInFile); + currentCol.resize(offset.compressedSize_); + file.read(currentCol.data(), offset.compressedSize_, offset.offsetInFile_); } return compressedBuffer; } @@ -500,7 +502,7 @@ DecompressedBlock CompressedRelationReader::readAndDecompressBlock( std::optional> columnIndices) { CompressedBlock compressedColumns = readCompressedBlockFromFile( blockMetaData, file, std::move(columnIndices)); - const auto numRowsToRead = blockMetaData._numRows; + const auto numRowsToRead = blockMetaData.numRows_; return decompressBlock(compressedColumns, numRowsToRead); } @@ -509,8 +511,8 @@ CompressedBlockMetadata::OffsetAndCompressedSize CompressedRelationWriter::compressAndWriteColumn(std::span column) { std::vector compressedBlock = ZstdWrapper::compress( (void*)(column.data()), column.size() * sizeof(column[0])); - auto offsetInFile = _outfile.tell(); + auto offsetInFile = outfile_.tell(); auto compressedSize = compressedBlock.size(); - _outfile.write(compressedBlock.data(), compressedBlock.size()); + outfile_.write(compressedBlock.data(), compressedBlock.size()); return {offsetInFile, compressedSize}; }; diff --git a/src/index/CompressedRelation.h b/src/index/CompressedRelation.h index 86ba5931af..d912463589 100644 --- a/src/index/CompressedRelation.h +++ b/src/index/CompressedRelation.h @@ -51,19 +51,29 @@ struct CompressedBlockMetadata { // Since we have column-based indices, the two columns of each block are // stored separately (but adjacently). struct OffsetAndCompressedSize { - off_t _offsetInFile; - size_t _compressedSize; + off_t offsetInFile_; + size_t compressedSize_; bool operator==(const OffsetAndCompressedSize&) const = default; }; - std::vector _offsetsAndCompressedSize; - size_t _numRows; + std::vector offsetsAndCompressedSize_; + size_t numRows_; // For example, in the PSO permutation, col0 is the P and col1 is the S. The // col0 ID is not stored in the block. First and last are meant inclusively, // that is, they are both part of the block. - Id _col0FirstId; - Id _col0LastId; - Id _col1FirstId; - Id _col1LastId; + // + // NOTE: Strictly speaking, we don't need `col0FirstId_` and `col1FirstId_`. + // However, they are convenient to have and don't really harm with respect to + // space efficiency. For example, for Wikidata, we have only around 50K blocks + // with block size 8M and around 5M blocks with block size 80K; even the + // latter takes only half a GB in total. + Id col0FirstId_; + Id col0LastId_; + Id col1FirstId_; + Id col1LastId_; + + // For our `DeltaTriples` (https://github.com/ad-freiburg/qlever/pull/916), we + // need to know the least significant `Id` of the last triple as well. + Id col2LastId_; // Two of these are equal if all members are equal. bool operator==(const CompressedBlockMetadata&) const = default; @@ -71,37 +81,38 @@ struct CompressedBlockMetadata { // Serialization of the `OffsetAndcompressedSize` subclass. AD_SERIALIZE_FUNCTION(CompressedBlockMetadata::OffsetAndCompressedSize) { - serializer | arg._offsetInFile; - serializer | arg._compressedSize; + serializer | arg.offsetInFile_; + serializer | arg.compressedSize_; } // Serialization of the block metadata. AD_SERIALIZE_FUNCTION(CompressedBlockMetadata) { - serializer | arg._offsetsAndCompressedSize; - serializer | arg._numRows; - serializer | arg._col0FirstId; - serializer | arg._col0LastId; - serializer | arg._col1FirstId; - serializer | arg._col1LastId; + serializer | arg.offsetsAndCompressedSize_; + serializer | arg.numRows_; + serializer | arg.col0FirstId_; + serializer | arg.col0LastId_; + serializer | arg.col1FirstId_; + serializer | arg.col1LastId_; + serializer | arg.col2LastId_; } // The metadata of a whole compressed "relation", where relation refers to a // maximal sequence of triples with equal first component (e.g., P for the PSO // permutation). struct CompressedRelationMetadata { - Id _col0Id; - size_t _numRows; - float _multiplicityCol1; // E.g., in PSO this is the multiplicity of "S". - float _multiplicityCol2; // E.g., in PSO this is the multiplicity of "O". + Id col0Id_; + size_t numRows_; + float multiplicityCol1_; // E.g., in PSO this is the multiplicity of "S". + float multiplicityCol2_; // E.g., in PSO this is the multiplicity of "O". // If this "relation" is contained in a block together with other "relations", // then all of these relations are contained only in this block and - // `_offsetInBlock` stores the offset in this block (referring to the index in + // `offsetInBlock_` stores the offset in this block (referring to the index in // the uncompressed sequence of triples). Otherwise, this "relation" is - // stored in one or several blocks of its own, and we set `_offsetInBlock` to + // stored in one or several blocks of its own, and we set `offsetInBlock_` to // `Id(-1)`. - uint64_t _offsetInBlock = std::numeric_limits::max(); + uint64_t offsetInBlock_ = std::numeric_limits::max(); - size_t getNofElements() const { return _numRows; } + size_t getNofElements() const { return numRows_; } // We currently always store two columns (the second and third column of a // triple). This might change in the future when we might also store @@ -110,12 +121,12 @@ struct CompressedRelationMetadata { static constexpr size_t numColumns() { return NumColumns; } // Setters and getters for the multiplicities. - float getCol1Multiplicity() const { return _multiplicityCol1; } - float getCol2Multiplicity() const { return _multiplicityCol2; } - void setCol1Multiplicity(float mult) { _multiplicityCol1 = mult; } - void setCol2Multiplicity(float mult) { _multiplicityCol2 = mult; } + float getCol1Multiplicity() const { return multiplicityCol1_; } + float getCol2Multiplicity() const { return multiplicityCol2_; } + void setCol1Multiplicity(float mult) { multiplicityCol1_ = mult; } + void setCol2Multiplicity(float mult) { multiplicityCol2_ = mult; } - bool isFunctional() const { return _multiplicityCol1 == 1.0f; } + bool isFunctional() const { return multiplicityCol1_ == 1.0f; } // Two of these are equal if all members are equal. bool operator==(const CompressedRelationMetadata&) const = default; @@ -123,30 +134,30 @@ struct CompressedRelationMetadata { // Serialization of the compressed "relation" meta data. AD_SERIALIZE_FUNCTION(CompressedRelationMetadata) { - serializer | arg._col0Id; - serializer | arg._numRows; - serializer | arg._multiplicityCol1; - serializer | arg._multiplicityCol2; - serializer | arg._offsetInBlock; + serializer | arg.col0Id_; + serializer | arg.numRows_; + serializer | arg.multiplicityCol1_; + serializer | arg.multiplicityCol2_; + serializer | arg.offsetInBlock_; } /// Manage the compression and serialization of relations during the index /// build. class CompressedRelationWriter { private: - ad_utility::File _outfile; - std::vector _metaDataBuffer; - std::vector _blockBuffer; - CompressedBlockMetadata _currentBlockData; - SmallRelationsBuffer _buffer; - size_t _numBytesPerBlock; + ad_utility::File outfile_; + std::vector metaDataBuffer_; + std::vector blockBuffer_; + CompressedBlockMetadata currentBlockData_; + SmallRelationsBuffer buffer_; + size_t numBytesPerBlock_; public: /// Create using a filename, to which the relation data will be written. - CompressedRelationWriter( + explicit CompressedRelationWriter( ad_utility::File f, size_t numBytesPerBlock = BLOCKSIZE_COMPRESSED_METADATA) - : _outfile{std::move(f)}, _numBytesPerBlock{numBytesPerBlock} {} + : outfile_{std::move(f)}, numBytesPerBlock_{numBytesPerBlock} {} /** * Add a complete (single) relation. @@ -169,7 +180,7 @@ class CompressedRelationWriter { /// still be in some internal buffer. void finish() { writeBufferedRelationsToSingleBlock(); - _outfile.close(); + outfile_.close(); } /// Get the complete CompressedRelationMetaData created by the calls to @@ -177,8 +188,8 @@ class CompressedRelationWriter { /// CompressedRelationWriter. The typical workflow is: add all relations, /// then call `finish()` and then call this method. auto getFinishedMetaData() { - auto result = std::move(_metaDataBuffer); - _metaDataBuffer.clear(); + auto result = std::move(metaDataBuffer_); + metaDataBuffer_.clear(); return result; } @@ -187,8 +198,8 @@ class CompressedRelationWriter { /// CompressedRelationWriter. The typical workflow is: add all relations, /// then call `finish()` and then call this method. auto getFinishedBlocks() { - auto result = std::move(_blockBuffer); - _blockBuffer.clear(); + auto result = std::move(blockBuffer_); + blockBuffer_.clear(); return result; } @@ -202,18 +213,18 @@ class CompressedRelationWriter { size_t numDistinctElements); private: - // Compress the contents of `_buffer` into a single block and write it to - // _outfile. Update `_currentBlockData` with the meta data of the written - // block. Then clear `_buffer`. + // Compress the contents of `buffer_` into a single block and write it to + // outfile_. Update `currentBlockData_` with the meta data of the written + // block. Then clear `buffer_`. void writeBufferedRelationsToSingleBlock(); // Compress the relation from `data` into one or more blocks, depending on - // its size. Write the blocks to `_outfile` and append all the created - // block metadata to `_blockBuffer`. + // its size. Write the blocks to `outfile_` and append all the created + // block metadata to `blockBuffer_`. void writeRelationToExclusiveBlocks(Id col0Id, const BufferedIdTable& data); - // Compress the `column` and write it to the `_outfile`. Return the offset and - // size of the compressed column in the `_outfile`. + // Compress the `column` and write it to the `outfile_`. Return the offset and + // size of the compressed column in the `outfile_`. CompressedBlockMetadata::OffsetAndCompressedSize compressAndWriteColumn( std::span column); }; diff --git a/src/index/ConstantsIndexBuilding.h b/src/index/ConstantsIndexBuilding.h index 6d73100e89..3d03736cb9 100644 --- a/src/index/ConstantsIndexBuilding.h +++ b/src/index/ConstantsIndexBuilding.h @@ -79,6 +79,12 @@ constexpr size_t QUEUE_SIZE_BEFORE_PARALLEL_PARSING = 10; // time constexpr size_t QUEUE_SIZE_AFTER_PARALLEL_PARSING = 10; -// The uncompressed size in bytes of a block of the permutations. Currently 8MB -// is chosen which is well suited for zstd compression -constexpr size_t BLOCKSIZE_COMPRESSED_METADATA = 1ul << 23u; +// The uncompressed size in bytes of a block of the permutations. +// +// NOTE: This used to be `1 << 23` (over 8M), which is fairly large (we always +// need to decompress at least one whole block, even when reading only few +// triples). With 100K, the total space for all the `CompressedBlockMetadata` is +// still small compared to the rest of the index. However, with 100K, and single +// block is just 10K compresse, which might result in sub-optimal IO-efficiency +// when reading many blocks. We take 500K as a compromise. +constexpr size_t BLOCKSIZE_COMPRESSED_METADATA = 500'000; diff --git a/src/index/IndexMetaDataImpl.h b/src/index/IndexMetaDataImpl.h index 0ba77bfa13..e06bc5ecfe 100644 --- a/src/index/IndexMetaDataImpl.h +++ b/src/index/IndexMetaDataImpl.h @@ -19,7 +19,7 @@ void IndexMetaData::add(AddType addedValue) { // only add rmd to _data if it's not already present there if constexpr (!isPersistentMetaData) { _totalElements += addedValue.getNofElements(); - _data.set(addedValue._col0Id, addedValue); + _data.set(addedValue.col0Id_, addedValue); } } diff --git a/src/index/MetaDataHandler.h b/src/index/MetaDataHandler.h index bbf545e095..37e6404fe8 100644 --- a/src/index/MetaDataHandler.h +++ b/src/index/MetaDataHandler.h @@ -25,11 +25,11 @@ class MetaDataWrapperDense { AddGetIdIterator(BaseIterator base) : BaseIterator{base} {} [[nodiscard]] Id getId() const { return getIdFromElement(*(*this)); } static Id getIdFromElement(const typename BaseIterator::value_type& v) { - return v._col0Id; + return v.col0Id_; } static auto getNumRowsFromElement( const typename BaseIterator::value_type& v) { - return v._numRows; + return v.numRows_; } }; @@ -93,28 +93,28 @@ class MetaDataWrapperDense { // ____________________________________________________________ void set(Id id, const value_type& value) { // Assert that the ids are ascending. - AD_CONTRACT_CHECK(_vec.size() == 0 || _vec.back()._col0Id < id); + AD_CONTRACT_CHECK(_vec.size() == 0 || _vec.back().col0Id_ < id); _vec.push_back(value); } // __________________________________________________________ const value_type& getAsserted(Id id) const { auto it = lower_bound(id); - AD_CONTRACT_CHECK(it != _vec.end() && it->_col0Id == id); + AD_CONTRACT_CHECK(it != _vec.end() && it->col0Id_ == id); return *it; } // _________________________________________________________ value_type& operator[](Id id) { auto it = lower_bound(id); - AD_CONTRACT_CHECK(it != _vec.end() && it->_col0Id == id); + AD_CONTRACT_CHECK(it != _vec.end() && it->col0Id_ == id); return *it; } // ________________________________________________________ size_t count(Id id) const { auto it = lower_bound(id); - return it != _vec.end() && it->_col0Id == id; + return it != _vec.end() && it->col0Id_ == id; } // ___________________________________________________________ @@ -123,13 +123,13 @@ class MetaDataWrapperDense { private: ConstIterator lower_bound(Id id) const { auto cmp = [](const auto& metaData, Id id) { - return metaData._col0Id < id; + return metaData.col0Id_ < id; }; return std::lower_bound(_vec.begin(), _vec.end(), id, cmp); } Iterator lower_bound(Id id) { auto cmp = [](const auto& metaData, Id id) { - return metaData._col0Id < id; + return metaData.col0Id_ < id; }; return std::lower_bound(_vec.begin(), _vec.end(), id, cmp); } @@ -144,13 +144,13 @@ class MetaDataWrapperHashMap { struct AddGetIdIterator : public BaseIterator { using BaseIterator::BaseIterator; AddGetIdIterator(BaseIterator base) : BaseIterator{base} {} - [[nodiscard]] Id getId() const { return (*this)->second._col0Id; } + [[nodiscard]] Id getId() const { return (*this)->second.col0Id_; } static Id getIdFromElement(const typename BaseIterator::value_type& v) { - return v.second._col0Id; + return v.second.col0Id_; } static auto getNumRowsFromElement( const typename BaseIterator::value_type& v) { - return v.second._numRows; + return v.second.numRows_; } }; using Iterator = AddGetIdIterator; diff --git a/test/CompressedRelationsTest.cpp b/test/CompressedRelationsTest.cpp index d05b1d12c1..85b5f7b4ed 100644 --- a/test/CompressedRelationsTest.cpp +++ b/test/CompressedRelationsTest.cpp @@ -108,12 +108,12 @@ void testCompressedRelations(const std::vector& inputs, CompressedRelationReader reader; for (size_t i = 0; i < metaData.size(); ++i) { const auto& m = metaData[i]; - ASSERT_EQ(V(inputs[i].col0_), m._col0Id); - ASSERT_EQ(inputs[i].col1And2_.size(), m._numRows); + ASSERT_EQ(V(inputs[i].col0_), m.col0Id_); + ASSERT_EQ(inputs[i].col1And2_.size(), m.numRows_); // The number of distinct elements in `col1` was passed in as `i + 1` for // testing purposes, so this is the expected multiplicity. - ASSERT_FLOAT_EQ(m._numRows / static_cast(i + 1), - m._multiplicityCol1); + ASSERT_FLOAT_EQ(m.numRows_ / static_cast(i + 1), + m.multiplicityCol1_); // Scan for all distinct `col0` and check that we get the expected result. IdTable table{2, ad_utility::testing::makeAllocator()}; reader.scan(metaData[i], blocks, file, &table, timer); @@ -259,13 +259,13 @@ TEST(CompressedRelationMetadata, GettersAndSetters) { CompressedRelationMetadata m; m.setCol1Multiplicity(2.0f); ASSERT_FLOAT_EQ(2.0f, m.getCol1Multiplicity()); - ASSERT_FLOAT_EQ(2.0f, m._multiplicityCol1); + ASSERT_FLOAT_EQ(2.0f, m.multiplicityCol1_); m.setCol2Multiplicity(1.0f); - ASSERT_FLOAT_EQ(1.0f, m._multiplicityCol2); + ASSERT_FLOAT_EQ(1.0f, m.multiplicityCol2_); ASSERT_FLOAT_EQ(1.0f, m.getCol2Multiplicity()); ASSERT_FALSE(m.isFunctional()); m.setCol1Multiplicity(1.0f); ASSERT_TRUE(m.isFunctional()); - m._numRows = 43; - ASSERT_EQ(43, m._numRows); + m.numRows_ = 43; + ASSERT_EQ(43, m.numRows_); } diff --git a/test/IndexMetaDataTest.cpp b/test/IndexMetaDataTest.cpp index 5c5b5af654..f1af1e04e3 100644 --- a/test/IndexMetaDataTest.cpp +++ b/test/IndexMetaDataTest.cpp @@ -17,7 +17,7 @@ auto V = ad_utility::testing::VocabId; TEST(RelationMetaDataTest, writeReadTest) { CompressedBlockMetadata rmdB{ - {{12, 34}, {46, 11}}, 5, V(0), V(2), V(13), V(24)}; + {{12, 34}, {46, 11}}, 5, V(0), V(2), V(13), V(24), V(62)}; CompressedRelationMetadata rmdF{V(1), 3, 2.0, 42.0, 16}; ad_utility::serialization::FileWriteSerializer f("_testtmp.rmd"); @@ -39,9 +39,9 @@ TEST(RelationMetaDataTest, writeReadTest) { TEST(IndexMetaDataTest, writeReadTest2Hmap) { vector bs; bs.push_back(CompressedBlockMetadata{ - {{12, 34}, {42, 5}}, 5, V(0), V(2), V(13), V(24)}); + {{12, 34}, {42, 5}}, 5, V(0), V(2), V(13), V(24), V(62)}); bs.push_back(CompressedBlockMetadata{ - {{16, 34}, {165, 3}}, 5, V(0), V(2), V(13), V(24)}); + {{16, 34}, {165, 3}}, 5, V(0), V(2), V(13), V(24), V(62)}); CompressedRelationMetadata rmdF{V(1), 3, 2.0, 42.0, 16}; CompressedRelationMetadata rmdF2{V(2), 5, 3.0, 43.0, 10}; IndexMetaDataHmap imd; @@ -71,9 +71,9 @@ TEST(IndexMetaDataTest, writeReadTest2Mmap) { std::string mmapFilename = imdFilename + ".mmap"; vector bs; bs.push_back(CompressedBlockMetadata{ - {{12, 34}, {42, 17}}, 5, V(0), V(2), V(13), V(24)}); + {{12, 34}, {42, 17}}, 5, V(0), V(2), V(13), V(24), V(62)}); bs.push_back(CompressedBlockMetadata{ - {{12, 34}, {16, 12}}, 5, V(0), V(2), V(13), V(24)}); + {{12, 34}, {16, 12}}, 5, V(0), V(2), V(13), V(24), V(62)}); CompressedRelationMetadata rmdF{V(1), 3, 2.0, 42.0, 16}; CompressedRelationMetadata rmdF2{V(2), 5, 3.0, 43.0, 10}; // The index MetaData does not have an explicit clear, so we diff --git a/test/IndexTestHelpers.h b/test/IndexTestHelpers.h index 72a86d64ee..5ae4638861 100644 --- a/test/IndexTestHelpers.h +++ b/test/IndexTestHelpers.h @@ -34,7 +34,8 @@ inline Index makeIndexWithTestSettings() { // when the files were not deleted after the test). inline std::vector getAllIndexFilenames( const std::string indexBasename) { - return {indexBasename + ".index.pos", + return {indexBasename + ".ttl", + indexBasename + ".index.pos", indexBasename + ".index.pso", indexBasename + ".index.sop", indexBasename + ".index.sop.meta", @@ -64,7 +65,7 @@ inline Index makeTestIndex(const std::string& indexBasename, // these tests. static std::ostringstream ignoreLogStream; ad_utility::setGlobalLoggingStream(&ignoreLogStream); - std::string filename = "relationalExpressionTestIndex.ttl"; + std::string inputFilename = indexBasename + ".ttl"; if (turtleInput.empty()) { turtleInput = "