mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 02:21:59 +00:00
Addressed more review comments and ClangTidy errors
This commit is contained in:
parent
6d576cf157
commit
e7add8218f
@ -100,13 +100,13 @@ bool LabelsAsBitmap::hasLabel(char label) const
|
||||
return ((data & bit_label) != 0);
|
||||
}
|
||||
|
||||
Arc* State::getArc(char label)
|
||||
Arc* State::getArc(char label) const
|
||||
{
|
||||
auto it = arcs.find(label);
|
||||
if (it == arcs.cend())
|
||||
return nullptr;
|
||||
|
||||
return &it->second;
|
||||
return const_cast<Arc *>(&it->second);
|
||||
}
|
||||
|
||||
void State::addArc(char label, Output output, StatePtr target)
|
||||
|
@ -83,7 +83,7 @@ public:
|
||||
|
||||
UInt64 hash() const;
|
||||
|
||||
Arc * getArc(char label);
|
||||
Arc * getArc(char label) const;
|
||||
|
||||
void addArc(char label, Output output, StatePtr target);
|
||||
|
||||
@ -170,7 +170,7 @@ class FiniteStateTransducer
|
||||
{
|
||||
public:
|
||||
FiniteStateTransducer() = default;
|
||||
FiniteStateTransducer(std::vector<UInt8> data_);
|
||||
explicit FiniteStateTransducer(std::vector<UInt8> data_);
|
||||
std::pair<UInt64, bool> getOutput(const String & term);
|
||||
void clear();
|
||||
std::vector<UInt8> & getData() { return data; }
|
||||
|
@ -33,7 +33,7 @@ GinFilter::GinFilter(const GinFilterParameters & params_)
|
||||
{
|
||||
}
|
||||
|
||||
void GinFilter::add(const char* data, size_t len, UInt32 rowID, GinIndexStorePtr& store, UInt64 limit)
|
||||
void GinFilter::add(const char* data, size_t len, UInt32 rowID, GinIndexStorePtr& store, UInt64 limit) const
|
||||
{
|
||||
if (len > FST::MAX_TERM_LENGTH)
|
||||
return;
|
||||
@ -49,7 +49,7 @@ void GinFilter::add(const char* data, size_t len, UInt32 rowID, GinIndexStorePtr
|
||||
else
|
||||
{
|
||||
UInt64 threshold = std::lround(limit * params.density);
|
||||
GinIndexPostingsBuilderPtr builder = std::make_shared<GinIndexPostingsBuilder>(threshold);
|
||||
GinIndexStore::GinIndexPostingsBuilderPtr builder = std::make_shared<GinIndexPostingsBuilder>(threshold);
|
||||
builder->add(rowID);
|
||||
|
||||
store->setPostingsBuilder(term, builder);
|
||||
|
@ -37,7 +37,7 @@ public:
|
||||
|
||||
/// Add term(which length is 'len' and located at 'data') and its row ID to
|
||||
/// the postings list builder for building inverted index for the given store.
|
||||
void add(const char* data, size_t len, UInt32 rowID, GinIndexStorePtr& store, UInt64 limit);
|
||||
void add(const char* data, size_t len, UInt32 rowID, GinIndexStorePtr& store, UInt64 limit) const;
|
||||
|
||||
/// Accumulate (segmentID, RowIDStart, RowIDEnd) for building skipping index
|
||||
void addRowRangeToGinFilter(UInt32 segmentID, UInt32 rowIDStart, UInt32 rowIDEnd);
|
||||
@ -47,7 +47,7 @@ public:
|
||||
|
||||
/// Check if the filter(built from query string) contains any rows in given filter 'af' by using
|
||||
/// given postings list cache
|
||||
bool contains(const GinFilter& af, PostingsCacheForStore &store) const;
|
||||
bool contains(const GinFilter & filter, PostingsCacheForStore &cache_store) const;
|
||||
|
||||
/// Const getter for the row ID ranges
|
||||
const GinSegmentWithRowIDRanges& getFilter() const { return rowid_ranges; }
|
||||
|
@ -905,7 +905,7 @@ std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(Storage
|
||||
return result;
|
||||
}
|
||||
|
||||
bool AlterCommands::hasInvertedIndex(const StorageInMemoryMetadata & metadata, ContextPtr context) const
|
||||
bool AlterCommands::hasInvertedIndex(const StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
{
|
||||
for (const auto & index : metadata.secondary_indices)
|
||||
{
|
||||
|
@ -212,7 +212,7 @@ public:
|
||||
MutationCommands getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, ContextPtr context, bool with_alters=false) const;
|
||||
|
||||
/// Check if commands have any inverted index
|
||||
bool hasInvertedIndex(const StorageInMemoryMetadata & metadata, ContextPtr context) const;
|
||||
static bool hasInvertedIndex(const StorageInMemoryMetadata & metadata, ContextPtr context);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -18,7 +18,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using TokenPostingsBuilderPair = std::pair<std::string_view, GinIndexPostingsBuilderPtr>;
|
||||
using TokenPostingsBuilderPair = std::pair<std::string_view, GinIndexStore::GinIndexPostingsBuilderPtr>;
|
||||
using TokenPostingsBuilderPairs = std::vector<TokenPostingsBuilderPair>;
|
||||
|
||||
namespace ErrorCodes
|
||||
@ -280,7 +280,7 @@ void GinIndexStore::writeSegment()
|
||||
|
||||
for (const auto& [token, postings_list] : current_postings)
|
||||
{
|
||||
token_postings_list_pairs.push_back({std::string_view(token), postings_list});
|
||||
token_postings_list_pairs.push_back({token, postings_list});
|
||||
}
|
||||
|
||||
/// Sort token-postings list pairs since all tokens have to be added in FST in sorted order
|
||||
|
@ -112,10 +112,6 @@ struct GinIndexSegment
|
||||
|
||||
using GinIndexSegments = std::vector<GinIndexSegment>;
|
||||
|
||||
using GinIndexPostingsBuilderPtr = std::shared_ptr<GinIndexPostingsBuilder>;
|
||||
|
||||
/// Container for all term's Gin Index Postings List Builder
|
||||
using GinIndexPostingsBuilderContainer = std::unordered_map<std::string, GinIndexPostingsBuilderPtr>;
|
||||
struct SegmentTermDictionary
|
||||
{
|
||||
/// .gin_post file offset of this segment's postings lists
|
||||
@ -138,6 +134,10 @@ using SegmentTermDictionaries = std::unordered_map<UInt32, SegmentTermDictionary
|
||||
class GinIndexStore
|
||||
{
|
||||
public:
|
||||
using GinIndexPostingsBuilderPtr = std::shared_ptr<GinIndexPostingsBuilder>;
|
||||
/// Container for all term's Gin Index Postings List Builder
|
||||
using GinIndexPostingsBuilderContainer = std::unordered_map<std::string, GinIndexPostingsBuilderPtr>;
|
||||
|
||||
explicit GinIndexStore(const String & name_, DataPartStoragePtr storage_);
|
||||
|
||||
GinIndexStore(const String& name_, DataPartStoragePtr storage_, MutableDataPartStoragePtr data_part_storage_builder_, UInt64 max_digestion_size_);
|
||||
|
@ -770,7 +770,7 @@ void ginIndexValidator(const IndexDescription & index, bool /*attach*/)
|
||||
if (index.arguments.size() > 2)
|
||||
throw Exception("Inverted index must have less than two arguments.", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
if (index.arguments.size() >= 1 && index.arguments[0].getType() != Field::Types::UInt64)
|
||||
if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64)
|
||||
throw Exception("The first Inverted index argument must be positive integer.", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
if (index.arguments.size() == 2 && (index.arguments[1].getType() != Field::Types::Float64 || index.arguments[1].get<Float64>() <= 0 || index.arguments[1].get<Float64>() > 1))
|
||||
|
Loading…
Reference in New Issue
Block a user