ClickHouse/src/Storages/MergeTree/GinIndexStore.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

300 lines
11 KiB
C++
Raw Normal View History

2022-06-24 01:56:15 +00:00
#pragma once
2023-01-20 09:32:36 +00:00
#include <Common/FST.h>
2022-06-24 01:56:15 +00:00
#include <Core/Block.h>
#include <Disks/IDisk.h>
#include <IO/ReadBufferFromFileBase.h>
#include <IO/WriteBufferFromFileBase.h>
2022-06-29 03:23:36 +00:00
#include <Storages/MergeTree/IDataPartStorage.h>
2023-01-20 09:32:36 +00:00
#include <roaring.hh>
#include <array>
#include <mutex>
#include <unordered_map>
#include <vector>
2023-01-11 21:40:20 +00:00
2023-01-20 09:32:36 +00:00
/// GinIndexStore manages the generalized inverted index ("gin") for a data part, and it is made up of one or more immutable
2023-01-11 21:40:20 +00:00
/// index segments.
///
/// There are 4 types of index files in a store:
/// 1. Segment ID file(.gin_sid): it contains one byte for version followed by the next available segment ID.
/// 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.
2023-01-20 10:53:41 +00:00
/// - dict_start_offset points to the file(.gin_dict) starting position for the segment's dictionaries.
/// 3. Dictionary file(.gin_dict): it contains dictionaries.
2023-01-11 21:40:20 +00:00
/// - 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,
2023-01-20 10:53:41 +00:00
/// the starting position of its dictionary is used to locate its FST. Then FST is read into memory.
2023-01-11 21:40:20 +00:00
/// 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.
2022-06-24 01:56:15 +00:00
namespace DB
{
/// GinIndexPostingsList which uses 32-bit Roaring
using GinIndexPostingsList = roaring::Roaring;
2023-01-10 16:26:27 +00:00
using GinIndexPostingsListPtr = std::shared_ptr<GinIndexPostingsList>;
2022-06-24 01:56:15 +00:00
2023-01-20 09:32:36 +00:00
/// Build a postings list for a term
class GinIndexPostingsBuilder
2022-06-24 01:56:15 +00:00
{
public:
2023-01-20 09:32:36 +00:00
explicit GinIndexPostingsBuilder(UInt64 limit);
2022-06-24 01:56:15 +00:00
/// Check whether a row_id is already added
bool contains(UInt32 row_id) const;
/// Add a row_id into the builder
void add(UInt32 row_id);
/// Serialize the content of builder to given WriteBuffer, returns the bytes of serialized data
2023-01-20 09:32:36 +00:00
UInt64 serialize(WriteBuffer & buffer) const;
2022-06-24 01:56:15 +00:00
/// Deserialize the postings list data from given ReadBuffer, return a pointer to the GinIndexPostingsList created by deserialization
2023-01-20 09:32:36 +00:00
static GinIndexPostingsListPtr deserialize(ReadBuffer & buffer);
private:
2023-01-20 09:32:36 +00:00
constexpr static int MIN_SIZE_FOR_ROARING_ENCODING = 16;
2023-01-10 16:26:27 +00:00
/// When the list length is no greater than MIN_SIZE_FOR_ROARING_ENCODING, array 'rowid_lst' is used
2023-01-20 09:32:36 +00:00
/// As a special case, rowid_lst[0] == CONTAINS_ALL encodes that all rowids are set.
2023-01-10 16:26:27 +00:00
std::array<UInt32, MIN_SIZE_FOR_ROARING_ENCODING> rowid_lst;
2023-01-20 09:32:36 +00:00
/// When the list length is greater than MIN_SIZE_FOR_ROARING_ENCODING, roaring bitmap 'rowid_bitmap' is used
2023-01-10 16:26:27 +00:00
roaring::Roaring rowid_bitmap;
/// rowid_lst_length stores the number of row IDs in 'rowid_lst' array, can also be a flag(0xFF) indicating that roaring bitmap is used
2023-01-20 09:32:36 +00:00
UInt8 rowid_lst_length = 0;
/// Indicates that all rowids are contained, see 'rowid_lst'
static constexpr UInt32 CONTAINS_ALL = std::numeric_limits<UInt32>::max();
2023-01-10 16:26:27 +00:00
2023-01-20 09:32:36 +00:00
/// Indicates that roaring bitmap is used, see 'rowid_lst_length'.
2023-01-20 10:58:28 +00:00
static constexpr UInt8 USES_BIT_MAP = 0xFF;
2023-01-20 09:32:36 +00:00
/// Clear the postings list and reset it with MATCHALL flags when the size of the postings list is beyond the limit
UInt64 size_limit;
2023-01-20 10:58:28 +00:00
/// Check whether the builder is using roaring bitmap
bool useRoaring() const { return rowid_lst_length == USES_BIT_MAP; }
/// Check whether the postings list has been flagged to contain all row ids
bool containsAllRows() const { return rowid_lst[0] == CONTAINS_ALL; }
2022-06-24 01:56:15 +00:00
};
2023-01-20 11:08:19 +00:00
using GinIndexPostingsBuilderPtr = std::shared_ptr<GinIndexPostingsBuilder>;
2022-06-24 01:56:15 +00:00
2023-01-20 11:08:19 +00:00
/// Gin index segment descriptor, which contains:
2023-01-10 16:26:27 +00:00
struct GinIndexSegment
{
/// Segment ID retrieved from next available ID from file .gin_sid
UInt32 segment_id = 0;
2023-01-20 09:32:36 +00:00
/// Start row ID for this segment
2023-01-10 16:26:27 +00:00
UInt32 next_row_id = 1;
/// .gin_post file offset of this segment's postings lists
UInt64 postings_start_offset = 0;
2023-01-20 11:08:19 +00:00
/// .gin_dict file offset of this segment's dictionaries
2023-01-20 10:53:41 +00:00
UInt64 dict_start_offset = 0;
2023-01-10 16:26:27 +00:00
};
struct GinSegmentDictionary
2022-06-24 01:56:15 +00:00
{
/// .gin_post file offset of this segment's postings lists
UInt64 postings_start_offset;
2023-01-20 10:53:41 +00:00
/// .gin_dict file offset of this segment's dictionaries
UInt64 dict_start_offset;
2022-06-24 01:56:15 +00:00
2023-01-20 09:32:36 +00:00
/// (Minimized) Finite State Transducer, which can be viewed as a map of <term, offset>, where offset is the
2023-01-10 16:26:27 +00:00
/// offset to the term's posting list in postings list file
FST::FiniteStateTransducer offsets;
2022-06-24 01:56:15 +00:00
};
using GinSegmentDictionaryPtr = std::shared_ptr<GinSegmentDictionary>;
2022-06-24 01:56:15 +00:00
2023-01-20 11:14:30 +00:00
/// Gin index store which has gin index meta data for the corresponding column data part
2022-06-24 01:56:15 +00:00
class GinIndexStore
{
public:
/// Container for all term's Gin Index Postings List Builder
using GinIndexPostingsBuilderContainer = std::unordered_map<std::string, GinIndexPostingsBuilderPtr>;
2023-01-20 09:32:36 +00:00
GinIndexStore(const String & name_, DataPartStoragePtr storage_);
GinIndexStore(const String & name_, DataPartStoragePtr storage_, MutableDataPartStoragePtr data_part_storage_builder_, UInt64 max_digestion_size_);
2022-06-24 01:56:15 +00:00
2023-01-10 16:26:27 +00:00
/// Check existence by checking the existence of file .gin_sid
2022-06-24 01:56:15 +00:00
bool exists() const;
2023-01-20 09:32:36 +00:00
/// Get a range of next 'numIDs'-many available row IDs
2023-01-10 16:26:27 +00:00
UInt32 getNextRowIDRange(size_t numIDs);
2022-06-24 01:56:15 +00:00
2023-01-10 16:26:27 +00:00
/// Get next available segment ID by updating file .gin_sid
2022-06-24 01:56:15 +00:00
UInt32 getNextSegmentID();
2023-01-10 16:26:27 +00:00
/// Get total number of segments in the store
UInt32 getNumOfSegments();
2022-06-24 01:56:15 +00:00
2023-01-10 16:26:27 +00:00
/// Get current postings list builder
2023-01-20 09:32:36 +00:00
const GinIndexPostingsBuilderContainer & getPostingsListBuilder() const { return current_postings; }
2022-06-24 01:56:15 +00:00
2023-01-10 16:26:27 +00:00
/// Set postings list builder for given term
void setPostingsBuilder(const String & term, GinIndexPostingsBuilderPtr builder) { current_postings[term] = builder; }
2023-01-20 09:32:36 +00:00
2022-06-24 01:56:15 +00:00
/// Check if we need to write segment to Gin index files
bool needToWrite() const;
/// Accumulate the size of text data which has been digested
2023-01-10 16:26:27 +00:00
void incrementCurrentSizeBy(UInt64 sz) { current_size += sz; }
2022-06-24 01:56:15 +00:00
2023-01-20 09:32:36 +00:00
UInt32 getCurrentSegmentID() const { return current_segment.segment_id; }
2022-06-24 01:56:15 +00:00
/// Do last segment writing
void finalize();
2023-01-20 09:32:36 +00:00
/// Method for writing segment data to Gin index files
2022-06-24 01:56:15 +00:00
void writeSegment();
2023-01-20 09:32:36 +00:00
const String & getName() const { return name; }
2022-06-29 03:23:36 +00:00
2022-06-24 01:56:15 +00:00
private:
friend class GinIndexStoreDeserializer;
2023-01-10 16:26:27 +00:00
/// Initialize all indexing files for this store
void initFileStreams();
2022-06-24 01:56:15 +00:00
2023-01-10 16:26:27 +00:00
/// Get a range of next available segment IDs by updating file .gin_sid
2023-01-20 09:32:36 +00:00
UInt32 getNextSegmentIDRange(const String & file_name, size_t n);
2023-01-10 16:26:27 +00:00
2022-06-24 01:56:15 +00:00
String name;
2023-01-05 04:08:28 +00:00
DataPartStoragePtr storage;
MutableDataPartStoragePtr data_part_storage_builder;
2022-06-24 01:56:15 +00:00
2022-09-29 19:30:02 +00:00
UInt32 cached_segment_num = 0;
2023-01-20 09:32:36 +00:00
std::mutex mutex;
2022-06-24 01:56:15 +00:00
/// Dictionaries indexed by segment ID
using GinSegmentDictionaries = std::unordered_map<UInt32, GinSegmentDictionaryPtr>;
2023-01-20 11:14:30 +00:00
/// Term's dictionaries which are loaded from .gin_dict files
GinSegmentDictionaries segment_dictionaries;
2022-06-24 01:56:15 +00:00
2023-01-20 09:32:36 +00:00
/// Container for building postings lists during index construction
2022-06-24 01:56:15 +00:00
GinIndexPostingsBuilderContainer current_postings;
2023-01-20 09:32:36 +00:00
/// For the segmentation of Gin indexes
GinIndexSegment current_segment;
2023-01-10 16:26:27 +00:00
UInt64 current_size = 0;
const UInt64 max_digestion_size = 0;
2022-06-24 01:56:15 +00:00
2023-01-20 10:53:41 +00:00
/// File streams for segment, dictionaries and postings lists
std::unique_ptr<WriteBufferFromFileBase> metadata_file_stream;
2023-01-20 10:53:41 +00:00
std::unique_ptr<WriteBufferFromFileBase> dict_file_stream;
2022-06-24 01:56:15 +00:00
std::unique_ptr<WriteBufferFromFileBase> postings_file_stream;
static constexpr auto GIN_SEGMENT_ID_FILE_TYPE = ".gin_sid";
static constexpr auto GIN_SEGMENT_METADATA_FILE_TYPE = ".gin_seg";
static constexpr auto GIN_DICTIONARY_FILE_TYPE = ".gin_dict";
static constexpr auto GIN_POSTINGS_FILE_TYPE = ".gin_post";
2023-01-20 10:47:42 +00:00
enum class Format : uint8_t
{
v0 = 0,
v1 = 1, /// Initial version
};
static constexpr auto CURRENT_GIN_FILE_FORMAT_VERSION = Format::v0;
2022-06-24 01:56:15 +00:00
};
using GinIndexStorePtr = std::shared_ptr<GinIndexStore>;
2023-01-10 16:26:27 +00:00
2023-01-20 11:08:19 +00:00
/// Container for postings lists for each segment
using GinSegmentedPostingsListContainer = std::unordered_map<UInt32, GinIndexPostingsListPtr>;
2023-01-20 11:08:19 +00:00
/// Postings lists and terms built from query string
using GinPostingsCache = std::unordered_map<std::string, GinSegmentedPostingsListContainer>;
using GinPostingsCachePtr = std::shared_ptr<GinPostingsCache>;
/// Gin index store reader which helps to read segments, dictionaries and postings list
class GinIndexStoreDeserializer : private boost::noncopyable
{
public:
explicit GinIndexStoreDeserializer(const GinIndexStorePtr & store_);
/// Read segment information from .gin_seg files
void readSegments();
/// Read all dictionaries from .gin_dict files
void readSegmentDictionaries();
/// Read dictionary for given segment id
void readSegmentDictionary(UInt32 segment_id);
/// Read postings lists for the term
GinSegmentedPostingsListContainer readSegmentedPostingsLists(const String & term);
/// Read postings lists for terms (which are created by tokenzing query string)
GinPostingsCachePtr createPostingsCacheFromTerms(const std::vector<String> & terms);
private:
/// Initialize gin index files
void initFileStreams();
/// The store for the reader
GinIndexStorePtr store;
/// File streams for reading Gin Index
std::unique_ptr<ReadBufferFromFileBase> metadata_file_stream;
std::unique_ptr<ReadBufferFromFileBase> dict_file_stream;
std::unique_ptr<ReadBufferFromFileBase> postings_file_stream;
/// Current segment, used in building index
GinIndexSegment current_segment;
};
2023-01-20 11:08:19 +00:00
2023-01-10 16:26:27 +00:00
/// PostingsCacheForStore contains postings lists from 'store' which are retrieved from Gin index files for the terms in query strings
/// GinPostingsCache is per query string (one query can have multiple query strings): when skipping index (row ID ranges) is used for the part during the
2023-01-10 16:26:27 +00:00
/// query, the postings cache is created and associated with the store where postings lists are read
/// for the tokenized query string. The postings caches are released automatically when the query is done.
2022-06-24 01:56:15 +00:00
struct PostingsCacheForStore
{
/// Which store to retrieve postings lists
GinIndexStorePtr store;
/// map of <query, postings lists>
std::unordered_map<String, GinPostingsCachePtr> cache;
2022-06-24 01:56:15 +00:00
/// Get postings lists for query string, return nullptr if not found
GinPostingsCachePtr getPostings(const String & query_string) const;
2022-06-24 01:56:15 +00:00
};
/// A singleton for storing GinIndexStores
2022-06-24 01:56:15 +00:00
class GinIndexStoreFactory : private boost::noncopyable
{
public:
/// Get singleton of GinIndexStoreFactory
2023-01-20 09:32:36 +00:00
static GinIndexStoreFactory & instance();
2022-06-24 01:56:15 +00:00
/// Get GinIndexStore by using index name, disk and part_path (which are combined to create key in stores)
2023-01-20 09:32:36 +00:00
GinIndexStorePtr get(const String & name, DataPartStoragePtr storage);
2022-06-24 01:56:15 +00:00
2023-01-10 16:26:27 +00:00
/// Remove all Gin index files which are under the same part_path
2023-01-20 09:32:36 +00:00
void remove(const String & part_path);
2022-06-24 01:56:15 +00:00
/// GinIndexStores indexed by part file path
using GinIndexStores = std::unordered_map<std::string, GinIndexStorePtr>;
2022-06-24 01:56:15 +00:00
private:
GinIndexStores stores;
2023-01-20 09:32:36 +00:00
std::mutex mutex;
2022-06-24 01:56:15 +00:00
};
}