"term dictionary" --> "dictionary"

This commit is contained in:
Robert Schulze 2023-01-20 10:53:41 +00:00
parent be936b257c
commit 5c3cc5283f
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
2 changed files with 50 additions and 50 deletions

View File

@ -250,11 +250,11 @@ void GinIndexStore::finalize()
void GinIndexStore::initFileStreams()
{
String segment_file_name = getName() + GIN_SEGMENT_FILE_TYPE;
String term_dict_file_name = getName() + GIN_DICTIONARY_FILE_TYPE;
String dict_file_name = getName() + GIN_DICTIONARY_FILE_TYPE;
String postings_file_name = getName() + GIN_POSTINGS_FILE_TYPE;
segment_file_stream = data_part_storage_builder->writeFile(segment_file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {});
term_dict_file_stream = data_part_storage_builder->writeFile(term_dict_file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {});
dict_file_stream = data_part_storage_builder->writeFile(dict_file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {});
postings_file_stream = data_part_storage_builder->writeFile(postings_file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {});
}
@ -306,19 +306,19 @@ void GinIndexStore::writeSegment()
write_buf.finalize();
/// Write FST size
writeVarUInt(buffer.size(), *term_dict_file_stream);
current_segment.term_dict_start_offset += getLengthOfVarUInt(buffer.size());
writeVarUInt(buffer.size(), *dict_file_stream);
current_segment.dict_start_offset += getLengthOfVarUInt(buffer.size());
/// Write FST content
term_dict_file_stream->write(reinterpret_cast<char *>(buffer.data()), buffer.size());
current_segment.term_dict_start_offset += buffer.size();
dict_file_stream->write(reinterpret_cast<char *>(buffer.data()), buffer.size());
current_segment.dict_start_offset += buffer.size();
current_size = 0;
current_postings.clear();
current_segment.segment_id = getNextSegmentID();
segment_file_stream->sync();
term_dict_file_stream->sync();
dict_file_stream->sync();
postings_file_stream->sync();
}
@ -331,11 +331,11 @@ GinIndexStoreDeserializer::GinIndexStoreDeserializer(const GinIndexStorePtr & st
void GinIndexStoreDeserializer::initFileStreams()
{
String segment_file_name = store->getName() + GinIndexStore::GIN_SEGMENT_FILE_TYPE;
String term_dict_file_name = store->getName() + GinIndexStore::GIN_DICTIONARY_FILE_TYPE;
String dict_file_name = store->getName() + GinIndexStore::GIN_DICTIONARY_FILE_TYPE;
String postings_file_name = store->getName() + GinIndexStore::GIN_POSTINGS_FILE_TYPE;
segment_file_stream = store->storage->readFile(segment_file_name, {}, std::nullopt, std::nullopt);
term_dict_file_stream = store->storage->readFile(term_dict_file_name, {}, std::nullopt, std::nullopt);
dict_file_stream = store->storage->readFile(dict_file_name, {}, std::nullopt, std::nullopt);
postings_file_stream = store->storage->readFile(postings_file_name, {}, std::nullopt, std::nullopt);
}
void GinIndexStoreDeserializer::readSegments()
@ -353,39 +353,39 @@ void GinIndexStoreDeserializer::readSegments()
for (size_t i = 0; i < num_segments; ++i)
{
auto seg_id = segments[i].segment_id;
auto term_dict = std::make_shared<SegmentTermDictionary>();
term_dict->postings_start_offset = segments[i].postings_start_offset;
term_dict->term_dict_start_offset = segments[i].term_dict_start_offset;
store->term_dicts[seg_id] = term_dict;
auto dict = std::make_shared<SegmentDictionary>();
dict->postings_start_offset = segments[i].postings_start_offset;
dict->dict_start_offset = segments[i].dict_start_offset;
store->dicts[seg_id] = dict;
}
}
void GinIndexStoreDeserializer::readSegmentTermDictionaries()
void GinIndexStoreDeserializer::readSegmentDictionaries()
{
for (UInt32 seg_index = 0; seg_index < store->getNumOfSegments(); ++seg_index)
readSegmentTermDictionary(seg_index);
readSegmentDictionary(seg_index);
}
void GinIndexStoreDeserializer::readSegmentTermDictionary(UInt32 segment_id)
void GinIndexStoreDeserializer::readSegmentDictionary(UInt32 segment_id)
{
/// Check validity of segment_id
auto it = store->term_dicts.find(segment_id);
if (it == store->term_dicts.end())
auto it = store->dicts.find(segment_id);
if (it == store->dicts.end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid segment id {}", segment_id);
assert(term_dict_file_stream != nullptr);
assert(dict_file_stream != nullptr);
/// Set file pointer of term dictionary file
term_dict_file_stream->seek(it->second->term_dict_start_offset, SEEK_SET);
/// Set file pointer of dictionary file
dict_file_stream->seek(it->second->dict_start_offset, SEEK_SET);
it->second->offsets.getData().clear();
/// Read FST size
size_t fst_size = 0;
readVarUInt(fst_size, *term_dict_file_stream);
readVarUInt(fst_size, *dict_file_stream);
/// Read FST content
it->second->offsets.getData().resize(fst_size);
term_dict_file_stream->readStrict(reinterpret_cast<char *>(it->second->offsets.getData().data()), fst_size);
dict_file_stream->readStrict(reinterpret_cast<char *>(it->second->offsets.getData().data()), fst_size);
}
SegmentedPostingsListContainer GinIndexStoreDeserializer::readSegmentedPostingsLists(const String & term)
@ -393,16 +393,16 @@ SegmentedPostingsListContainer GinIndexStoreDeserializer::readSegmentedPostingsL
assert(postings_file_stream != nullptr);
SegmentedPostingsListContainer container;
for (auto const & seg_term_dict : store->term_dicts)
for (auto const & seg_dict : store->dicts)
{
auto segment_id = seg_term_dict.first;
auto segment_id = seg_dict.first;
auto [offset, found] = seg_term_dict.second->offsets.getOutput(term);
auto [offset, found] = seg_dict.second->offsets.getOutput(term);
if (!found)
continue;
// Set postings file pointer for reading postings list
postings_file_stream->seek(seg_term_dict.second->postings_start_offset + offset, SEEK_SET);
postings_file_stream->seek(seg_dict.second->postings_start_offset + offset, SEEK_SET);
// Read posting list
auto postings_list = GinIndexPostingsBuilder::deserialize(*postings_file_stream);
@ -448,7 +448,7 @@ GinIndexStorePtr GinIndexStoreFactory::get(const String & name, DataPartStorageP
GinIndexStoreDeserializer deserializer(store);
deserializer.readSegments();
deserializer.readSegmentTermDictionaries();
deserializer.readSegmentDictionaries();
stores[key] = store;

View File

@ -20,14 +20,14 @@
/// 2. Segment Metadata file(.gin_seg): it contains index segment metadata.
/// - Its file format is an array of GinIndexSegment as defined in this file.
/// - postings_start_offset points to the file(.gin_post) starting position for the segment's postings list.
/// - term_dict_start_offset points to the file(.gin_dict) starting position for the segment's term dictionaries.
/// 3. Term Dictionary file(.gin_dict): it contains term dictionaries.
/// - dict_start_offset points to the file(.gin_dict) starting position for the segment's dictionaries.
/// 3. Dictionary file(.gin_dict): it contains dictionaries.
/// - It contains an array of (FST_size, FST_blob) which has size and actual data of FST.
/// 4. Postings Lists(.gin_post): it contains postings lists data.
/// - It contains an array of serialized postings lists.
///
/// During the searching in the segment, the segment's meta data can be found in .gin_seg file. From the meta data,
/// the starting position of its term dictionary is used to locate its FST. Then FST is read into memory.
/// the starting position of its dictionary is used to locate its FST. Then FST is read into memory.
/// By using the term and FST, the offset("output" in FST) of the postings list for the term
/// in FST is found. The offset plus the postings_start_offset is the file location in .gin_post file
/// for its postings list.
@ -106,27 +106,27 @@ struct GinIndexSegment
/// .gin_post file offset of this segment's postings lists
UInt64 postings_start_offset = 0;
/// .term_dict file offset of this segment's term dictionaries
UInt64 term_dict_start_offset = 0;
/// .dict file offset of this segment's dictionaries
UInt64 dict_start_offset = 0;
};
struct SegmentTermDictionary
struct SegmentDictionary
{
/// .gin_post file offset of this segment's postings lists
UInt64 postings_start_offset;
/// .gin_dict file offset of this segment's term dictionaries
UInt64 term_dict_start_offset;
/// .gin_dict file offset of this segment's dictionaries
UInt64 dict_start_offset;
/// (Minimized) Finite State Transducer, which can be viewed as a map of <term, offset>, where offset is the
/// offset to the term's posting list in postings list file
FST::FiniteStateTransducer offsets;
};
using SegmentTermDictionaryPtr = std::shared_ptr<SegmentTermDictionary>;
using SegmentDictionaryPtr = std::shared_ptr<SegmentDictionary>;
/// Term dictionaries indexed by segment ID
using SegmentTermDictionaries = std::unordered_map<UInt32, SegmentTermDictionaryPtr>;
/// Dictionaries indexed by segment ID
using SegmentDictionaries = std::unordered_map<UInt32, SegmentDictionaryPtr>;
/// Gin Index Store which has Gin Index meta data for the corresponding Data Part
class GinIndexStore
@ -190,8 +190,8 @@ private:
std::mutex mutex;
/// Terms dictionaries which are loaded from .gin_dict files
SegmentTermDictionaries term_dicts;
/// Dictionaries which are loaded from .gin_dict files
SegmentDictionaries dicts;
/// Container for building postings lists during index construction
GinIndexPostingsBuilderContainer current_postings;
@ -201,9 +201,9 @@ private:
UInt64 current_size = 0;
const UInt64 max_digestion_size = 0;
/// File streams for segment, term dictionaries and postings lists
/// File streams for segment, dictionaries and postings lists
std::unique_ptr<WriteBufferFromFileBase> segment_file_stream;
std::unique_ptr<WriteBufferFromFileBase> term_dict_file_stream;
std::unique_ptr<WriteBufferFromFileBase> dict_file_stream;
std::unique_ptr<WriteBufferFromFileBase> postings_file_stream;
static constexpr auto GIN_SEGMENT_ID_FILE_TYPE = ".gin_sid";
@ -265,9 +265,9 @@ private:
std::mutex mutex;
};
/// Term dictionary information, which contains:
/// Dictionary information, which contains:
/// Gin Index Store Reader which helps to read segments, term dictionaries and postings list
/// Gin Index Store Reader which helps to read segments, dictionaries and postings list
class GinIndexStoreDeserializer : private boost::noncopyable
{
public:
@ -276,11 +276,11 @@ public:
/// Read segment information from .gin_seg files
void readSegments();
/// Read all term dictionaries from .gin_dict files
void readSegmentTermDictionaries();
/// Read all dictionaries from .gin_dict files
void readSegmentDictionaries();
/// Read term dictionary for given segment id
void readSegmentTermDictionary(UInt32 segment_id);
/// Read dictionary for given segment id
void readSegmentDictionary(UInt32 segment_id);
/// Read postings lists for the term
SegmentedPostingsListContainer readSegmentedPostingsLists(const String & term);
@ -297,7 +297,7 @@ private:
/// File streams for reading Gin Index
std::unique_ptr<ReadBufferFromFileBase> segment_file_stream;
std::unique_ptr<ReadBufferFromFileBase> term_dict_file_stream;
std::unique_ptr<ReadBufferFromFileBase> dict_file_stream;
std::unique_ptr<ReadBufferFromFileBase> postings_file_stream;
/// Current segment, used in building index