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
2022-09-25 23:29:30 +00:00
class GinIndexPostingsBuilder
2022-06-24 01:56:15 +00:00
{
2022-09-25 23:29:30 +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 ) ;
2022-09-25 23:29:30 +00:00
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
2023-01-05 03:42:45 +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
} ;
2023-01-20 11:09:01 +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
2022-09-07 18:22:09 +00:00
FST : : FiniteStateTransducer offsets ;
2022-06-24 01:56:15 +00:00
} ;
2023-01-20 11:09:01 +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 :
2023-01-17 14:29:13 +00:00
/// 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 :
2022-09-25 23:29:30 +00:00
friend class GinIndexStoreDeserializer ;
2023-01-10 16:26:27 +00:00
/// Initialize all indexing files for this store
2022-09-25 23:29:30 +00:00
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 ;
2022-12-29 16:00:17 +00:00
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
2023-01-20 11:09:01 +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
2023-01-20 11:26:22 +00:00
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 ;
2022-09-25 23:29:30 +00:00
static constexpr auto GIN_SEGMENT_ID_FILE_TYPE = " .gin_sid " ;
2023-01-20 11:26:22 +00:00
static constexpr auto GIN_SEGMENT_METADATA_FILE_TYPE = " .gin_seg " ;
2022-09-25 23:29:30 +00:00
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
2023-01-20 11:18:40 +00:00
using GinSegmentedPostingsListContainer = std : : unordered_map < UInt32 , GinIndexPostingsListPtr > ;
2023-01-20 11:08:19 +00:00
/// Postings lists and terms built from query string
2023-01-20 11:18:40 +00:00
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
2023-01-20 11:26:22 +00:00
std : : unique_ptr < ReadBufferFromFileBase > metadata_file_stream ;
2023-01-20 11:18:40 +00:00
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
2023-01-20 11:24:23 +00:00
/// 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>
2023-01-20 11:18:40 +00:00
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
2023-01-20 11:24:23 +00:00
GinPostingsCachePtr getPostings ( const String & query_string ) const ;
2022-06-24 01:56:15 +00:00
} ;
2023-01-20 11:24:23 +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
2023-01-20 11:24:23 +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
} ;
}