Addressed more review comments and ClangTidy errors

This commit is contained in:
HarryLeeIBM 2023-01-17 06:29:13 -08:00
parent 6d576cf157
commit e7add8218f
9 changed files with 17 additions and 17 deletions

View File

@ -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)

View File

@ -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; }

View File

@ -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);

View File

@ -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; }

View File

@ -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)
{

View File

@ -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);
};
}

View File

@ -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

View File

@ -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_);

View File

@ -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))