mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-30 19:42:00 +00:00
"term dictionary" --> "dictionary"
This commit is contained in:
parent
be936b257c
commit
5c3cc5283f
@ -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;
|
||||
|
||||
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user