clang-tidy, part 2

This commit is contained in:
Alexey Milovidov 2020-03-18 03:57:00 +03:00
parent a800fa7986
commit 1bd7e594b0
100 changed files with 633 additions and 607 deletions

View File

@ -1,2 +1,63 @@
Checks: '-*,google-readability-avoid-underscore-in-googletest-name,misc-throw-by-value-catch-by-reference,misc-misplaced-const,misc-unconventional-assign-operator,modernize-avoid-bind,modernize-loop-convert,modernize-make-shared,modernize-make-unique,modernize-raw-string-literal,modernize-redundant-void-arg,modernize-replace-auto-ptr,modernize-replace-random-shuffle,modernize-use-bool-literals,modernize-use-nullptr,modernize-use-using,performance-faster-string-find,performance-for-range-copy,readability-avoid-const-params-in-decls,readability-const-return-type,readability-container-size-empty,readability-convert-member-functions-to-static,readability-delete-null-pointer,readability-deleted-default,readability-make-member-function-const,readability-misplaced-array-index,readability-non-const-parameter,readability-qualified-auto,readability-redundant-access-specifiers,readability-redundant-control-flow,readability-redundant-function-ptr-dereference,readability-redundant-smartptr-get,readability-redundant-string-cstr,readability-redundant-string-init,readability-static-definition-in-anonymous-namespace,readability-string-compare,readability-uniqueptr-delete-release,modernize-use-equals-default,modernize-use-equals-delete,bugprone-undelegated-constructor,readability-redundant-member-init,readability-simplify-subscript-expr,readability-simplify-boolean-expr,readability-inconsistent-declaration-parameter-name'
Checks: '-*,
google-readability-avoid-underscore-in-googletest-name,
misc-throw-by-value-catch-by-reference,
misc-misplaced-const,
misc-unconventional-assign-operator,
modernize-avoid-bind,
modernize-loop-convert,
modernize-make-shared,
modernize-make-unique,
modernize-raw-string-literal,
modernize-redundant-void-arg,
modernize-replace-auto-ptr,
modernize-replace-random-shuffle,
modernize-use-bool-literals,
modernize-use-nullptr,
modernize-use-using,
modernize-use-equals-default,
modernize-use-equals-delete,
performance-faster-string-find,
performance-for-range-copy,
readability-avoid-const-params-in-decls,
readability-const-return-type,
readability-container-size-empty,
readability-convert-member-functions-to-static,
readability-delete-null-pointer,
readability-deleted-default,
readability-make-member-function-const,
readability-misplaced-array-index,
readability-non-const-parameter,
readability-qualified-auto,
readability-redundant-access-specifiers,
readability-redundant-control-flow,
readability-redundant-function-ptr-dereference,
readability-redundant-smartptr-get,
readability-redundant-string-cstr,
readability-redundant-string-init,
readability-static-definition-in-anonymous-namespace,
readability-string-compare,
readability-uniqueptr-delete-release,
readability-redundant-member-init,
readability-simplify-subscript-expr,
readability-simplify-boolean-expr,
readability-inconsistent-declaration-parameter-name,
bugprone-undelegated-constructor,
bugprone-argument-comment,
bugprone-bad-signal-to-kill-thread,
bugprone-bool-pointer-implicit-conversion,
bugprone-copy-constructor-init,
bugprone-dangling-handle,
bugprone-forward-declaration-namespace,
bugprone-fold-init-type,
bugprone-inaccurate-erase,
bugprone-incorrect-roundings,
bugprone-infinite-loop,
boost-use-to-string,
'
WarningsAsErrors: '*'

View File

@ -362,19 +362,8 @@ void BaseDaemon::reloadConfiguration()
}
BaseDaemon::BaseDaemon()
namespace
{
checkRequiredInstructions();
}
BaseDaemon::~BaseDaemon()
{
writeSignalIDtoSignalPipe(SignalListener::StopThread);
signal_listener_thread.join();
signal_pipe.close();
}
enum class InstructionFail
{
@ -388,7 +377,7 @@ enum class InstructionFail
AVX512 = 7
};
static std::string instructionFailToString(InstructionFail fail)
std::string instructionFailToString(InstructionFail fail)
{
switch (fail)
{
@ -413,16 +402,16 @@ static std::string instructionFailToString(InstructionFail fail)
}
static sigjmp_buf jmpbuf;
sigjmp_buf jmpbuf;
static void sigIllCheckHandler(int sig, siginfo_t * info, void * context)
void sigIllCheckHandler(int sig, siginfo_t * info, void * context)
{
siglongjmp(jmpbuf, 1);
}
/// Check if necessary sse extensions are available by trying to execute some sse instructions.
/// If instruction is unavailable, SIGILL will be sent by kernel.
static void checkRequiredInstructions(volatile InstructionFail & fail)
void checkRequiredInstructionsImpl(volatile InstructionFail & fail)
{
#if __SSE3__
fail = InstructionFail::SSE3;
@ -463,8 +452,9 @@ static void checkRequiredInstructions(volatile InstructionFail & fail)
fail = InstructionFail::NONE;
}
void BaseDaemon::checkRequiredInstructions()
/// Check SSE and others instructions availability
/// Calls exit on fail
void checkRequiredInstructions()
{
struct sigaction sa{};
struct sigaction sa_old{};
@ -487,7 +477,7 @@ void BaseDaemon::checkRequiredInstructions()
exit(1);
}
::checkRequiredInstructions(fail);
checkRequiredInstructionsImpl(fail);
if (sigaction(signal, &sa_old, nullptr))
{
@ -496,6 +486,22 @@ void BaseDaemon::checkRequiredInstructions()
}
}
}
BaseDaemon::BaseDaemon()
{
checkRequiredInstructions();
}
BaseDaemon::~BaseDaemon()
{
writeSignalIDtoSignalPipe(SignalListener::StopThread);
signal_listener_thread.join();
signal_pipe.close();
}
void BaseDaemon::terminate()
{

View File

@ -128,7 +128,7 @@ public:
/// close all process FDs except
/// 0-2 -- stdin, stdout, stderr
/// also doesn't close global internal pipes for signal handling
void closeFDs();
static void closeFDs();
protected:
/// Возвращает TaskManager приложения
@ -198,12 +198,6 @@ protected:
std::string config_path;
DB::ConfigProcessor::LoadedConfig loaded_config;
Poco::Util::AbstractConfiguration * last_configuration = nullptr;
private:
/// Check SSE and others instructions availability
/// Calls exit on fail
void checkRequiredInstructions();
};

View File

@ -281,7 +281,7 @@ private:
}
/// Should we celebrate a bit?
bool isNewYearMode()
static bool isNewYearMode()
{
time_t current_time = time(nullptr);
@ -294,7 +294,7 @@ private:
|| (now.month() == 1 && now.day() <= 5);
}
bool isChineseNewYearMode(const String & local_tz)
static bool isChineseNewYearMode(const String & local_tz)
{
/// Days of Dec. 20 in Chinese calendar starting from year 2019 to year 2105
static constexpr UInt16 chineseNewYearIndicators[]
@ -1594,7 +1594,7 @@ private:
std::cout << "Ok." << std::endl;
}
void showClientVersion()
static void showClientVersion()
{
std::cout << DBMS_NAME << " client version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl;
}

View File

@ -499,7 +499,10 @@ ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast
return new_query_ast;
}
std::shared_ptr<ASTCreateQuery> ClusterCopier::rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast)
/// Replaces ENGINE and table name in a create query
static std::shared_ptr<ASTCreateQuery> rewriteCreateQueryStorage(
const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast)
{
const auto & create = create_query_ast->as<ASTCreateQuery &>();
auto res = std::make_shared<ASTCreateQuery>(create);

View File

@ -88,11 +88,6 @@ protected:
/// Removes MATERIALIZED and ALIAS columns from create table query
static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr &query_ast);
/// Replaces ENGINE and table name in a create query
std::shared_ptr<ASTCreateQuery>
rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table,
const ASTPtr & new_storage_ast);
bool tryDropPartition(ShardPartition & task_partition,
const zkutil::ZooKeeperPtr & zookeeper,
const CleanStateClock & clean_state_clock);

View File

@ -118,6 +118,20 @@ void LocalServer::tryInitPath()
}
static void attachSystemTables()
{
DatabasePtr system_database = DatabaseCatalog::instance().tryGetDatabase(DatabaseCatalog::SYSTEM_DATABASE);
if (!system_database)
{
/// TODO: add attachTableDelayed into DatabaseMemory to speedup loading
system_database = std::make_shared<DatabaseMemory>(DatabaseCatalog::SYSTEM_DATABASE);
DatabaseCatalog::instance().attachDatabase(DatabaseCatalog::SYSTEM_DATABASE, system_database);
}
attachSystemTablesLocal(*system_database);
}
int LocalServer::main(const std::vector<std::string> & /*args*/)
try
{
@ -248,20 +262,6 @@ std::string LocalServer::getInitialCreateTableQuery()
}
void LocalServer::attachSystemTables()
{
DatabasePtr system_database = DatabaseCatalog::instance().tryGetDatabase(DatabaseCatalog::SYSTEM_DATABASE);
if (!system_database)
{
/// TODO: add attachTableDelayed into DatabaseMemory to speedup loading
system_database = std::make_shared<DatabaseMemory>(DatabaseCatalog::SYSTEM_DATABASE);
DatabaseCatalog::instance().attachDatabase(DatabaseCatalog::SYSTEM_DATABASE, system_database);
}
attachSystemTablesLocal(*system_database);
}
void LocalServer::processQueries()
{
String initial_create_query = getInitialCreateTableQuery();
@ -375,7 +375,7 @@ static void showClientVersion()
std::cout << DBMS_NAME << " client version " << VERSION_STRING << VERSION_OFFICIAL << "." << '\n';
}
std::string LocalServer::getHelpHeader() const
static std::string getHelpHeader()
{
return
"usage: clickhouse-local [initial table definition] [--query <query>]\n"
@ -390,7 +390,7 @@ std::string LocalServer::getHelpHeader() const
"Either through corresponding command line parameters --table --structure --input-format and --file.";
}
std::string LocalServer::getHelpFooter() const
static std::string getHelpFooter()
{
return
"Example printing memory used by each Unix user:\n"

View File

@ -37,13 +37,9 @@ private:
void tryInitPath();
void applyCmdOptions();
void applyCmdSettings();
void attachSystemTables();
void processQueries();
void setupUsers();
std::string getHelpHeader() const;
std::string getHelpFooter() const;
protected:
std::unique_ptr<Context> context;

View File

@ -529,13 +529,13 @@ private:
static constexpr CodePoint END = -2;
NGramHash hashContext(const CodePoint * begin, const CodePoint * end) const
static NGramHash hashContext(const CodePoint * begin, const CodePoint * end)
{
return CRC32Hash()(StringRef(reinterpret_cast<const char *>(begin), (end - begin) * sizeof(CodePoint)));
}
/// By the way, we don't have to use actual Unicode numbers. We use just arbitrary bijective mapping.
CodePoint readCodePoint(const char *& pos, const char * end)
static CodePoint readCodePoint(const char *& pos, const char * end)
{
size_t length = UTF8::seqLength(*pos);
@ -550,7 +550,7 @@ private:
return res;
}
bool writeCodePoint(CodePoint code, char *& pos, const char * end)
static bool writeCodePoint(CodePoint code, char *& pos, const char * end)
{
size_t length
= (code & 0xFF000000) ? 4

View File

@ -41,7 +41,7 @@ void ConfigPreprocessor::removeConfigurationsIf(
std::vector<XMLConfigurationPtr> & configs,
ConfigPreprocessor::FilterType filter_type,
const Strings & values,
bool leave) const
bool leave)
{
auto checker = [&filter_type, &values, &leave] (XMLConfigurationPtr & config)
{

View File

@ -39,11 +39,11 @@ private:
/// Removes configurations that has a given value.
/// If leave is true, the logic is reversed.
void removeConfigurationsIf(
static void removeConfigurationsIf(
std::vector<XMLConfigurationPtr> & configs,
FilterType filter_type,
const Strings & values,
bool leave = false) const;
bool leave = false);
const Strings paths;
};

View File

@ -40,7 +40,7 @@ ReportBuilder::ReportBuilder(const std::string & server_version_)
{
}
std::string ReportBuilder::getCurrentTime() const
static std::string getCurrentTime()
{
return DateLUT::instance().timeToString(time(nullptr));
}
@ -163,7 +163,7 @@ std::string ReportBuilder::buildFullReport(
std::string ReportBuilder::buildCompactReport(
const PerformanceTestInfo & test_info,
std::vector<TestStats> & stats,
const std::vector<std::size_t> & queries_to_run) const
const std::vector<std::size_t> & queries_to_run)
{
FormatSettings settings;
std::ostringstream output;

View File

@ -10,16 +10,16 @@ class ReportBuilder
{
public:
ReportBuilder(const std::string & server_version_);
std::string buildFullReport(
const PerformanceTestInfo & test_info,
std::vector<TestStats> & stats,
const std::vector<std::size_t> & queries_to_run) const;
std::string buildCompactReport(
static std::string buildCompactReport(
const PerformanceTestInfo & test_info,
std::vector<TestStats> & stats,
const std::vector<std::size_t> & queries_to_run) const;
const std::vector<std::size_t> & queries_to_run);
private:
std::string server_version;
@ -27,10 +27,6 @@ private:
size_t num_cores;
size_t num_threads;
size_t ram;
private:
std::string getCurrentTime() const;
};
}

View File

@ -77,7 +77,7 @@ private:
Poco::Net::HTTPServerResponse & response,
Output & used_output);
void pushDelayedResults(Output & used_output);
static void pushDelayedResults(Output & used_output);
};
}

View File

@ -158,7 +158,7 @@ public:
void revoke(const AccessFlags & access_to_revoke, const Helper & helper)
{
if constexpr (mode == NORMAL_REVOKE_MODE)
{
{ // NOLINT
if (level == TABLE_LEVEL)
removeExplicitGrantsRec(access_to_revoke);
else
@ -166,11 +166,12 @@ public:
}
else if constexpr (mode == PARTIAL_REVOKE_MODE)
{
AccessFlags new_partial_revokes = access_to_revoke - explicit_grants;
if (level == TABLE_LEVEL)
removeExplicitGrantsRec(access_to_revoke);
else
removeExplicitGrants(access_to_revoke);
AccessFlags new_partial_revokes = access_to_revoke - explicit_grants;
removePartialRevokesRec(new_partial_revokes);
partial_revokes |= new_partial_revokes;
}

View File

@ -379,7 +379,7 @@ void IAccessStorage::throwNotFound(std::type_index type, const String & name) co
}
void IAccessStorage::throwBadCast(const UUID & id, std::type_index type, const String & name, std::type_index required_type) const
void IAccessStorage::throwBadCast(const UUID & id, std::type_index type, const String & name, std::type_index required_type)
{
throw Exception(
"ID {" + toString(id) + "}: " + getTypeName(type) + backQuote(name) + " expected to be of type " + getTypeName(required_type),

View File

@ -151,8 +151,9 @@ protected:
static String getTypeName(std::type_index type) { return IAccessEntity::getTypeName(type); }
[[noreturn]] void throwNotFound(const UUID & id) const;
[[noreturn]] void throwNotFound(std::type_index type, const String & name) const;
[[noreturn]] void throwBadCast(const UUID & id, std::type_index type, const String & name, std::type_index required_type) const;
[[noreturn]] void throwIDCollisionCannotInsert(const UUID & id, std::type_index type, const String & name, std::type_index existing_type, const String & existing_name) const;
[[noreturn]] static void throwBadCast(const UUID & id, std::type_index type, const String & name, std::type_index required_type);
[[noreturn]] void throwIDCollisionCannotInsert(
const UUID & id, std::type_index type, const String & name, std::type_index existing_type, const String & existing_name) const;
[[noreturn]] void throwNameCollisionCannotInsert(std::type_index type, const String & name) const;
[[noreturn]] void throwNameCollisionCannotRename(std::type_index type, const String & old_name, const String & new_name) const;
[[noreturn]] void throwReadonlyCannotInsert(std::type_index type, const String & name) const;

View File

@ -576,8 +576,9 @@ ColumnAggregateFunction::MutablePtr ColumnAggregateFunction::createView() const
}
ColumnAggregateFunction::ColumnAggregateFunction(const ColumnAggregateFunction & src_)
: foreign_arenas(concatArenas(src_.foreign_arenas, src_.my_arena)),
func(src_.func), src(src_.getPtr()), data(src_.data.begin(), src_.data.end())
: COWHelper<IColumn, ColumnAggregateFunction>(src_),
foreign_arenas(concatArenas(src_.foreign_arenas, src_.my_arena)),
func(src_.func), src(src_.getPtr()), data(src_.data.begin(), src_.data.end())
{
}

View File

@ -16,6 +16,12 @@ namespace ErrorCodes
namespace
{
void checkColumn(const IColumn & column)
{
if (!dynamic_cast<const IColumnUnique *>(&column))
throw Exception("ColumnUnique expected as an argument of ColumnLowCardinality.", ErrorCodes::ILLEGAL_COLUMN);
}
template <typename T>
PaddedPODArray<T> * getIndexesData(IColumn & indexes)
{
@ -651,13 +657,6 @@ ColumnLowCardinality::Dictionary::Dictionary(ColumnPtr column_unique_, bool is_s
checkColumn(*column_unique);
}
void ColumnLowCardinality::Dictionary::checkColumn(const IColumn & column)
{
if (!dynamic_cast<const IColumnUnique *>(&column))
throw Exception("ColumnUnique expected as an argument of ColumnLowCardinality.", ErrorCodes::ILLEGAL_COLUMN);
}
void ColumnLowCardinality::Dictionary::setShared(const ColumnPtr & column_unique_)
{
checkColumn(*column_unique_);

View File

@ -275,8 +275,6 @@ private:
private:
WrappedPtr column_unique;
bool shared = false;
void checkColumn(const IColumn & column);
};
Dictionary dictionary;

View File

@ -219,7 +219,7 @@ void ConfigProcessor::merge(XMLDocumentPtr config, XMLDocumentPtr with)
mergeRecursive(config, config_root, with_root);
}
std::string ConfigProcessor::layerFromHost()
static std::string layerFromHost()
{
utsname buf;
if (uname(&buf))

View File

@ -97,8 +97,8 @@ public:
/// If preprocessed_dir is empty - calculate from loaded_config.path + /preprocessed_configs/
void savePreprocessedConfig(const LoadedConfig & loaded_config, std::string preprocessed_dir);
/// Set path of main config.xml . It will be cutted from all configs placed to preprocessed_configs/
void setConfigPath(const std::string & config_path);
/// Set path of main config.xml. It will be cutted from all configs placed to preprocessed_configs/
static void setConfigPath(const std::string & config_path);
public:
using Files = std::vector<std::string>;
@ -131,8 +131,6 @@ private:
void merge(XMLDocumentPtr config, XMLDocumentPtr with);
std::string layerFromHost();
void doIncludesRecursive(
XMLDocumentPtr config,
XMLDocumentPtr include_from,

View File

@ -75,15 +75,10 @@ static int wcwidth(wchar_t wc)
switch (width)
{
case widechar_nonprint:
[[fallthrough]];
case widechar_combining:
[[fallthrough]];
case widechar_unassigned:
return 0;
case widechar_ambiguous:
[[fallthrough]];
case widechar_private_use:
[[fallthrough]];
case widechar_widened_in_9:
return 1;
default:

View File

@ -519,7 +519,7 @@ public:
CODEC_WITHOUT_DATA_TYPE,
};
CompressionCodecPtr makeCodec(MakeCodecParam with_data_type)
static CompressionCodecPtr makeCodec(MakeCodecParam with_data_type)
{
const auto & codec_string = std::get<0>(GetParam()).codec_statement;
const auto & data_type = with_data_type == CODEC_WITH_DATA_TYPE ? std::get<1>(GetParam()).data_type : nullptr;
@ -527,7 +527,7 @@ public:
return ::makeCodec(codec_string, data_type);
}
void testTranscoding(ICompressionCodec & codec)
static void testTranscoding(ICompressionCodec & codec)
{
NoOpTimer timer;
::testTranscoding(timer, codec, std::get<1>(GetParam()), std::get<0>(GetParam()).expected_compression_ratio);

View File

@ -208,7 +208,7 @@ void SettingMaxThreads::setAuto()
is_auto = true;
}
UInt64 SettingMaxThreads::getAutoValue() const
UInt64 SettingMaxThreads::getAutoValue()
{
static auto res = getNumberOfPhysicalCPUCores();
return res;

View File

@ -91,7 +91,7 @@ struct SettingMaxThreads
void deserialize(ReadBuffer & buf, SettingsBinaryFormat format);
void setAuto();
UInt64 getAutoValue() const;
static UInt64 getAutoValue();
};

View File

@ -32,6 +32,101 @@ namespace ErrorCodes
}
static void checkCalculated(const ColumnWithTypeAndName & col_read,
const ColumnWithTypeAndName & col_defaults,
size_t defaults_needed)
{
size_t column_size = col_read.column->size();
if (column_size != col_defaults.column->size())
throw Exception("Mismatch column sizes while adding defaults", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
if (column_size < defaults_needed)
throw Exception("Unexpected defaults count", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
if (!col_read.type->equals(*col_defaults.type))
throw Exception("Mismach column types while adding defaults", ErrorCodes::TYPE_MISMATCH);
}
static void mixNumberColumns(
TypeIndex type_idx,
MutableColumnPtr & column_mixed,
const ColumnPtr & col_defaults,
const BlockMissingValues::RowsBitMask & defaults_mask)
{
auto call = [&](const auto & types) -> bool
{
using Types = std::decay_t<decltype(types)>;
using DataType = typename Types::LeftType;
if constexpr (!std::is_same_v<DataType, DataTypeString> && !std::is_same_v<DataType, DataTypeFixedString>)
{
using FieldType = typename DataType::FieldType;
using ColVecType = std::conditional_t<IsDecimalNumber<FieldType>, ColumnDecimal<FieldType>, ColumnVector<FieldType>>;
auto col_read = typeid_cast<ColVecType *>(column_mixed.get());
if (!col_read)
return false;
typename ColVecType::Container & dst = col_read->getData();
if (auto const_col_defs = checkAndGetColumnConst<ColVecType>(col_defaults.get()))
{
FieldType value = checkAndGetColumn<ColVecType>(const_col_defs->getDataColumnPtr().get())->getData()[0];
for (size_t i = 0; i < defaults_mask.size(); ++i)
if (defaults_mask[i])
dst[i] = value;
return true;
}
else if (auto col_defs = checkAndGetColumn<ColVecType>(col_defaults.get()))
{
auto & src = col_defs->getData();
for (size_t i = 0; i < defaults_mask.size(); ++i)
if (defaults_mask[i])
dst[i] = src[i];
return true;
}
}
return false;
};
if (!callOnIndexAndDataType<void>(type_idx, call))
throw Exception("Unexpected type on mixNumberColumns", ErrorCodes::LOGICAL_ERROR);
}
static MutableColumnPtr mixColumns(const ColumnWithTypeAndName & col_read,
const ColumnWithTypeAndName & col_defaults,
const BlockMissingValues::RowsBitMask & defaults_mask)
{
size_t column_size = col_read.column->size();
size_t defaults_needed = defaults_mask.size();
MutableColumnPtr column_mixed = col_read.column->cloneEmpty();
for (size_t i = 0; i < defaults_needed; ++i)
{
if (defaults_mask[i])
{
if (isColumnConst(*col_defaults.column))
column_mixed->insert((*col_defaults.column)[i]);
else
column_mixed->insertFrom(*col_defaults.column, i);
}
else
column_mixed->insertFrom(*col_read.column, i);
}
for (size_t i = defaults_needed; i < column_size; ++i)
column_mixed->insertFrom(*col_read.column, i);
return column_mixed;
}
AddingDefaultsBlockInputStream::AddingDefaultsBlockInputStream(const BlockInputStreamPtr & input,
const ColumnDefaults & column_defaults_,
const Context & context_)
@ -124,95 +219,4 @@ Block AddingDefaultsBlockInputStream::readImpl()
return res;
}
void AddingDefaultsBlockInputStream::checkCalculated(const ColumnWithTypeAndName & col_read,
const ColumnWithTypeAndName & col_defaults,
size_t defaults_needed) const
{
size_t column_size = col_read.column->size();
if (column_size != col_defaults.column->size())
throw Exception("Mismatch column sizes while adding defaults", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
if (column_size < defaults_needed)
throw Exception("Unexpected defaults count", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
if (!col_read.type->equals(*col_defaults.type))
throw Exception("Mismach column types while adding defaults", ErrorCodes::TYPE_MISMATCH);
}
void AddingDefaultsBlockInputStream::mixNumberColumns(TypeIndex type_idx, MutableColumnPtr & column_mixed, const ColumnPtr & col_defaults,
const BlockMissingValues::RowsBitMask & defaults_mask) const
{
auto call = [&](const auto & types) -> bool
{
using Types = std::decay_t<decltype(types)>;
using DataType = typename Types::LeftType;
if constexpr (!std::is_same_v<DataType, DataTypeString> && !std::is_same_v<DataType, DataTypeFixedString>)
{
using FieldType = typename DataType::FieldType;
using ColVecType = std::conditional_t<IsDecimalNumber<FieldType>, ColumnDecimal<FieldType>, ColumnVector<FieldType>>;
auto col_read = typeid_cast<ColVecType *>(column_mixed.get());
if (!col_read)
return false;
typename ColVecType::Container & dst = col_read->getData();
if (auto const_col_defs = checkAndGetColumnConst<ColVecType>(col_defaults.get()))
{
FieldType value = checkAndGetColumn<ColVecType>(const_col_defs->getDataColumnPtr().get())->getData()[0];
for (size_t i = 0; i < defaults_mask.size(); ++i)
if (defaults_mask[i])
dst[i] = value;
return true;
}
else if (auto col_defs = checkAndGetColumn<ColVecType>(col_defaults.get()))
{
auto & src = col_defs->getData();
for (size_t i = 0; i < defaults_mask.size(); ++i)
if (defaults_mask[i])
dst[i] = src[i];
return true;
}
}
return false;
};
if (!callOnIndexAndDataType<void>(type_idx, call))
throw Exception("Unexpected type on mixNumberColumns", ErrorCodes::LOGICAL_ERROR);
}
MutableColumnPtr AddingDefaultsBlockInputStream::mixColumns(const ColumnWithTypeAndName & col_read,
const ColumnWithTypeAndName & col_defaults,
const BlockMissingValues::RowsBitMask & defaults_mask) const
{
size_t column_size = col_read.column->size();
size_t defaults_needed = defaults_mask.size();
MutableColumnPtr column_mixed = col_read.column->cloneEmpty();
for (size_t i = 0; i < defaults_needed; ++i)
{
if (defaults_mask[i])
{
if (isColumnConst(*col_defaults.column))
column_mixed->insert((*col_defaults.column)[i]);
else
column_mixed->insertFrom(*col_defaults.column, i);
}
else
column_mixed->insertFrom(*col_read.column, i);
}
for (size_t i = defaults_needed; i < column_size; ++i)
column_mixed->insertFrom(*col_read.column, i);
return column_mixed;
}
}

View File

@ -27,12 +27,6 @@ private:
Block header;
const ColumnDefaults column_defaults;
const Context & context;
void checkCalculated(const ColumnWithTypeAndName & col_read, const ColumnWithTypeAndName & col_defaults, size_t needed) const;
MutableColumnPtr mixColumns(const ColumnWithTypeAndName & col_read, const ColumnWithTypeAndName & col_defaults,
const BlockMissingValues::RowsBitMask & defaults_mask) const;
void mixNumberColumns(TypeIndex type_idx, MutableColumnPtr & column_mixed, const ColumnPtr & col_defaults,
const BlockMissingValues::RowsBitMask & defaults_mask) const;
};
}

View File

@ -37,6 +37,89 @@ namespace
return false;
}
/// Returns true if merge result is not empty
bool mergeMap(const SummingSortedBlockInputStream::MapDescription & desc, Row & row, SortCursor & cursor)
{
/// Strongly non-optimal.
Row & left = row;
Row right(left.size());
for (size_t col_num : desc.key_col_nums)
right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get<Array>();
for (size_t col_num : desc.val_col_nums)
right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get<Array>();
auto at_ith_column_jth_row = [&](const Row & matrix, size_t i, size_t j) -> const Field &
{
return matrix[i].get<Array>()[j];
};
auto tuple_of_nth_columns_at_jth_row = [&](const Row & matrix, const ColumnNumbers & col_nums, size_t j) -> Array
{
size_t size = col_nums.size();
Array res(size);
for (size_t col_num_index = 0; col_num_index < size; ++col_num_index)
res[col_num_index] = at_ith_column_jth_row(matrix, col_nums[col_num_index], j);
return res;
};
std::map<Array, Array> merged;
auto accumulate = [](Array & dst, const Array & src)
{
bool has_non_zero = false;
size_t size = dst.size();
for (size_t i = 0; i < size; ++i)
if (applyVisitor(FieldVisitorSum(src[i]), dst[i]))
has_non_zero = true;
return has_non_zero;
};
auto merge = [&](const Row & matrix)
{
size_t rows = matrix[desc.key_col_nums[0]].get<Array>().size();
for (size_t j = 0; j < rows; ++j)
{
Array key = tuple_of_nth_columns_at_jth_row(matrix, desc.key_col_nums, j);
Array value = tuple_of_nth_columns_at_jth_row(matrix, desc.val_col_nums, j);
auto it = merged.find(key);
if (merged.end() == it)
merged.emplace(std::move(key), std::move(value));
else
{
if (!accumulate(it->second, value))
merged.erase(it);
}
}
};
merge(left);
merge(right);
for (size_t col_num : desc.key_col_nums)
row[col_num] = Array(merged.size());
for (size_t col_num : desc.val_col_nums)
row[col_num] = Array(merged.size());
size_t row_num = 0;
for (const auto & key_value : merged)
{
for (size_t col_num_index = 0, size = desc.key_col_nums.size(); col_num_index < size; ++col_num_index)
row[desc.key_col_nums[col_num_index]].get<Array>()[row_num] = key_value.first[col_num_index];
for (size_t col_num_index = 0, size = desc.val_col_nums.size(); col_num_index < size; ++col_num_index)
row[desc.val_col_nums[col_num_index]].get<Array>()[row_num] = key_value.second[col_num_index];
++row_num;
}
return row_num != 0;
}
}
@ -401,89 +484,6 @@ void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, Sorti
}
bool SummingSortedBlockInputStream::mergeMap(const MapDescription & desc, Row & row, SortCursor & cursor)
{
/// Strongly non-optimal.
Row & left = row;
Row right(left.size());
for (size_t col_num : desc.key_col_nums)
right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get<Array>();
for (size_t col_num : desc.val_col_nums)
right[col_num] = (*cursor->all_columns[col_num])[cursor->pos].template get<Array>();
auto at_ith_column_jth_row = [&](const Row & matrix, size_t i, size_t j) -> const Field &
{
return matrix[i].get<Array>()[j];
};
auto tuple_of_nth_columns_at_jth_row = [&](const Row & matrix, const ColumnNumbers & col_nums, size_t j) -> Array
{
size_t size = col_nums.size();
Array res(size);
for (size_t col_num_index = 0; col_num_index < size; ++col_num_index)
res[col_num_index] = at_ith_column_jth_row(matrix, col_nums[col_num_index], j);
return res;
};
std::map<Array, Array> merged;
auto accumulate = [](Array & dst, const Array & src)
{
bool has_non_zero = false;
size_t size = dst.size();
for (size_t i = 0; i < size; ++i)
if (applyVisitor(FieldVisitorSum(src[i]), dst[i]))
has_non_zero = true;
return has_non_zero;
};
auto merge = [&](const Row & matrix)
{
size_t rows = matrix[desc.key_col_nums[0]].get<Array>().size();
for (size_t j = 0; j < rows; ++j)
{
Array key = tuple_of_nth_columns_at_jth_row(matrix, desc.key_col_nums, j);
Array value = tuple_of_nth_columns_at_jth_row(matrix, desc.val_col_nums, j);
auto it = merged.find(key);
if (merged.end() == it)
merged.emplace(std::move(key), std::move(value));
else
{
if (!accumulate(it->second, value))
merged.erase(it);
}
}
};
merge(left);
merge(right);
for (size_t col_num : desc.key_col_nums)
row[col_num] = Array(merged.size());
for (size_t col_num : desc.val_col_nums)
row[col_num] = Array(merged.size());
size_t row_num = 0;
for (const auto & key_value : merged)
{
for (size_t col_num_index = 0, size = desc.key_col_nums.size(); col_num_index < size; ++col_num_index)
row[desc.key_col_nums[col_num_index]].get<Array>()[row_num] = key_value.first[col_num_index];
for (size_t col_num_index = 0, size = desc.val_col_nums.size(); col_num_index < size; ++col_num_index)
row[desc.val_col_nums[col_num_index]].get<Array>()[row_num] = key_value.second[col_num_index];
++row_num;
}
return row_num != 0;
}
void SummingSortedBlockInputStream::addRow(SortCursor & cursor)
{
for (auto & desc : columns_to_aggregate)

View File

@ -35,6 +35,13 @@ public:
String getName() const override { return "SummingSorted"; }
/// Stores numbers of key-columns and value-columns.
struct MapDescription
{
std::vector<size_t> key_col_nums;
std::vector<size_t> val_col_nums;
};
protected:
/// Can return 1 more records than max_block_size.
Block readImpl() override;
@ -120,13 +127,6 @@ private:
AggregateDescription(const AggregateDescription &) = delete;
};
/// Stores numbers of key-columns and value-columns.
struct MapDescription
{
std::vector<size_t> key_col_nums;
std::vector<size_t> val_col_nums;
};
std::vector<AggregateDescription> columns_to_aggregate;
std::vector<MapDescription> maps_to_sum;
@ -146,9 +146,6 @@ private:
/// Insert the summed row for the current group into the result and updates some of per-block flags if the row is not "zero".
void insertCurrentRowIfNeeded(MutableColumns & merged_columns);
/// Returns true if merge result is not empty
bool mergeMap(const MapDescription & desc, Row & row, SortCursor & cursor);
// Add the row under the cursor to the `row`.
void addRow(SortCursor & cursor);
};

View File

@ -10,7 +10,6 @@
class GeoDictionariesLoader
{
public:
std::unique_ptr<RegionsHierarchies> reloadRegionsHierarchies(const Poco::Util::AbstractConfiguration & config);
std::unique_ptr<RegionsNames> reloadRegionsNames(const Poco::Util::AbstractConfiguration & config);
static std::unique_ptr<RegionsHierarchies> reloadRegionsHierarchies(const Poco::Util::AbstractConfiguration & config);
static std::unique_ptr<RegionsNames> reloadRegionsNames(const Poco::Util::AbstractConfiguration & config);
};

View File

@ -34,6 +34,18 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
}
static void validateKeyTypes(const DataTypes & key_types)
{
if (key_types.size() != 1)
throw Exception{"Expected a single IP address", ErrorCodes::TYPE_MISMATCH};
const auto & actual_type = key_types[0]->getName();
if (actual_type != "UInt32" && actual_type != "FixedString(16)")
throw Exception{"Key does not match, expected either UInt32 or FixedString(16)", ErrorCodes::TYPE_MISMATCH};
}
TrieDictionary::TrieDictionary(
const std::string & database_,
const std::string & name_,
@ -416,17 +428,6 @@ void TrieDictionary::calculateBytesAllocated()
bytes_allocated += btrie_allocated(trie);
}
void TrieDictionary::validateKeyTypes(const DataTypes & key_types) const
{
if (key_types.size() != 1)
throw Exception{"Expected a single IP address", ErrorCodes::TYPE_MISMATCH};
const auto & actual_type = key_types[0]->getName();
if (actual_type != "UInt32" && actual_type != "FixedString(16)")
throw Exception{"Key does not match, expected either UInt32 or FixedString(16)", ErrorCodes::TYPE_MISMATCH};
}
template <typename T>
void TrieDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value)

View File

@ -207,8 +207,6 @@ private:
void calculateBytesAllocated();
void validateKeyTypes(const DataTypes & key_types) const;
template <typename T>
void createAttributeImpl(Attribute & attribute, const Field & null_value);

View File

@ -34,6 +34,15 @@ namespace ErrorCodes
namespace
{
String getRandomName()
{
std::uniform_int_distribution<int> distribution('a', 'z');
String res(32, ' '); /// The number of bits of entropy should be not less than 128.
for (auto & c : res)
c = distribution(thread_local_rng);
return res;
}
template <typename Result, typename Error>
void throwIfError(Aws::Utils::Outcome<Result, Error> && response)
{
@ -570,14 +579,6 @@ void DiskS3::removeRecursive(const String & path)
}
}
String DiskS3::getRandomName() const
{
std::uniform_int_distribution<int> distribution('a', 'z');
String res(32, ' '); /// The number of bits of entropy should be not less than 128.
for (auto & c : res)
c = distribution(thread_local_rng);
return res;
}
bool DiskS3::tryReserve(UInt64 bytes)
{

View File

@ -92,8 +92,6 @@ public:
Poco::Timestamp getLastModified(const String & path) override;
private:
String getRandomName() const;
bool tryReserve(UInt64 bytes);
private:

View File

@ -32,7 +32,7 @@ public:
void init(Block & /*block*/, const ColumnNumbers & /*arguments*/) {}
/// Returns the position of the argument that is the column of rows
size_t getStringsArgumentPosition()
static size_t getStringsArgumentPosition()
{
return 0;
}

View File

@ -31,7 +31,7 @@ public:
void init(Block & /*block*/, const ColumnNumbers & /*arguments*/) {}
/// Returns the position of the argument that is the column of rows
size_t getStringsArgumentPosition()
static size_t getStringsArgumentPosition()
{
return 0;
}

View File

@ -29,7 +29,7 @@ public:
}
/// Returns the position of the argument that is the column of rows
size_t getStringsArgumentPosition()
static size_t getStringsArgumentPosition()
{
return 0;
}

View File

@ -31,7 +31,7 @@ public:
void init(Block & /*block*/, const ColumnNumbers & /*arguments*/) {}
/// Returns the position of the argument that is the column of rows
size_t getStringsArgumentPosition()
static size_t getStringsArgumentPosition()
{
return 0;
}

View File

@ -62,21 +62,21 @@ private:
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
template <typename T>
bool executeNumber(
static bool executeNumber(
const IColumn & src_data,
const ColumnArray::Offsets & src_offsets,
IColumn & res_data_col,
ColumnArray::Offsets & res_offsets,
const ColumnNullable * nullable_col);
bool executeString(
static bool executeString(
const IColumn & src_data,
const ColumnArray::Offsets & src_offsets,
IColumn & res_data_col,
ColumnArray::Offsets & res_offsets,
const ColumnNullable * nullable_col);
void executeHashed(
static void executeHashed(
const IColumn & src_data,
const ColumnArray::Offsets & src_offsets,
IColumn & res_data_col,

View File

@ -53,29 +53,29 @@ private:
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count);
template <typename DataType>
bool executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
static bool executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
ArrayImpl::NullMapBuilder & builder);
template <typename IndexType, typename DataType>
bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
static bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
ArrayImpl::NullMapBuilder & builder);
bool executeStringConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
static bool executeStringConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
ArrayImpl::NullMapBuilder & builder);
template <typename IndexType>
bool executeString(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
static bool executeString(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
ArrayImpl::NullMapBuilder & builder);
bool executeGenericConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
static bool executeGenericConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
ArrayImpl::NullMapBuilder & builder);
template <typename IndexType>
bool executeGeneric(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
static bool executeGeneric(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
ArrayImpl::NullMapBuilder & builder);
template <typename IndexType>
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result,
static bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result,
const PaddedPODArray <IndexType> & indices, ArrayImpl::NullMapBuilder & builder,
size_t input_rows_count);

View File

@ -45,11 +45,11 @@ public:
private:
template <typename T>
bool executeNumber(const IColumn & src_data, const ColumnArray::Offsets & src_offsets, IColumn & res_data);
static bool executeNumber(const IColumn & src_data, const ColumnArray::Offsets & src_offsets, IColumn & res_data);
bool executeFixedString(const IColumn & src_data, const ColumnArray::Offsets & src_offsets, IColumn & res_data);
bool executeString(const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, IColumn & res_data);
bool executeGeneric(const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, IColumn & res_data);
static bool executeFixedString(const IColumn & src_data, const ColumnArray::Offsets & src_offsets, IColumn & res_data);
static bool executeString(const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, IColumn & res_data);
static bool executeGeneric(const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, IColumn & res_data);
};

View File

@ -39,7 +39,7 @@ public:
input_rows_count, size)->convertToFullColumnIfConst();
}
UInt64 blockSerializedSizeOne(const ColumnWithTypeAndName & elem) const
static UInt64 blockSerializedSizeOne(const ColumnWithTypeAndName & elem)
{
ColumnPtr full_column = elem.column->convertToFullColumnIfConst();

View File

@ -78,13 +78,13 @@ private:
/// Separate converter is created for each thread.
using Pool = ObjectPoolMap<Converter, String>;
Pool::Pointer getConverter(const String & charset)
static Pool::Pointer getConverter(const String & charset)
{
static Pool pool;
return pool.get(charset, [&charset] { return new Converter(charset); });
}
void convert(const String & from_charset, const String & to_charset,
static void convert(const String & from_charset, const String & to_charset,
const ColumnString::Chars & from_chars, const ColumnString::Offsets & from_offsets,
ColumnString::Chars & to_chars, ColumnString::Offsets & to_offsets)
{

View File

@ -416,7 +416,7 @@ private:
return true;
}
bool executeString(const ColumnUInt8 * cond_col, Block & block, const ColumnNumbers & arguments, size_t result)
static bool executeString(const ColumnUInt8 * cond_col, Block & block, const ColumnNumbers & arguments, size_t result)
{
const IColumn * col_then_untyped = block.getByPosition(arguments[1]).column.get();
const IColumn * col_else_untyped = block.getByPosition(arguments[2]).column.get();
@ -494,7 +494,7 @@ private:
return false;
}
bool executeGenericArray(const ColumnUInt8 * cond_col, Block & block, const ColumnNumbers & arguments, size_t result)
static bool executeGenericArray(const ColumnUInt8 * cond_col, Block & block, const ColumnNumbers & arguments, size_t result)
{
/// For generic implementation, arrays must be of same type.
if (!block.getByPosition(arguments[1]).type->equals(*block.getByPosition(arguments[2]).type))

View File

@ -39,7 +39,7 @@ public:
throw Exception("Cannot reinterpret " + type.getName() + " as FixedString because it is not fixed size and contiguous in memory", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
void NO_INLINE executeToFixedString(const IColumn & src, ColumnFixedString & dst, size_t n)
static void NO_INLINE executeToFixedString(const IColumn & src, ColumnFixedString & dst, size_t n)
{
size_t rows = src.size();
ColumnFixedString::Chars & data_to = dst.getChars();
@ -54,7 +54,7 @@ public:
}
}
void NO_INLINE executeContiguousToFixedString(const IColumn & src, ColumnFixedString & dst, size_t n)
static void NO_INLINE executeContiguousToFixedString(const IColumn & src, ColumnFixedString & dst, size_t n)
{
size_t rows = src.size();
ColumnFixedString::Chars & data_to = dst.getChars();

View File

@ -39,7 +39,7 @@ public:
throw Exception("Cannot reinterpret " + type.getName() + " as String because it is not contiguous in memory", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
void executeToString(const IColumn & src, ColumnString & dst)
static void executeToString(const IColumn & src, ColumnString & dst)
{
size_t rows = src.size();
ColumnString::Chars & data_to = dst.getChars();

View File

@ -188,7 +188,7 @@ public:
}
private:
void executeConst(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
static void executeConst(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
/// Materialize the input column and compute the function as usual.

View File

@ -84,7 +84,7 @@ namespace S3
return ret;
}
std::shared_ptr<Aws::S3::S3Client> ClientFactory::create(
std::shared_ptr<Aws::S3::S3Client> ClientFactory::create( // NOLINT
const String & endpoint,
const String & access_key_id,
const String & secret_access_key)

View File

@ -81,7 +81,7 @@ void WriteBufferValidUTF8::nextImpl()
size_t len = length_of_utf8_sequence[static_cast<unsigned char>(*p)];
if (len > 4)
{
{ // NOLINT
/// Invalid start of sequence. Skip one byte.
putValid(valid_start, p - valid_start);
putReplacement();

View File

@ -479,12 +479,12 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date
if (read_alpha_month(alpha))
{
}
else if (0 == strncasecmp(alpha, "UTC", 3)) has_time_zone_offset = true;
else if (0 == strncasecmp(alpha, "UTC", 3)) has_time_zone_offset = true; // NOLINT
else if (0 == strncasecmp(alpha, "GMT", 3)) has_time_zone_offset = true;
else if (0 == strncasecmp(alpha, "MSK", 3)) { has_time_zone_offset = true; time_zone_offset_hour = 3; }
else if (0 == strncasecmp(alpha, "MSD", 3)) { has_time_zone_offset = true; time_zone_offset_hour = 4; }
else if (0 == strncasecmp(alpha, "Mon", 3)) has_day_of_week = true;
else if (0 == strncasecmp(alpha, "Mon", 3)) has_day_of_week = true; // NOLINT
else if (0 == strncasecmp(alpha, "Tue", 3)) has_day_of_week = true;
else if (0 == strncasecmp(alpha, "Wed", 3)) has_day_of_week = true;
else if (0 == strncasecmp(alpha, "Thu", 3)) has_day_of_week = true;

View File

@ -510,7 +510,7 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl(
AggregatedDataWithoutKey & res,
size_t rows,
AggregateFunctionInstruction * aggregate_instructions,
Arena * arena) const
Arena * arena)
{
/// Adding values
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)

View File

@ -1082,11 +1082,11 @@ protected:
AggregateFunctionInstruction * aggregate_instructions) const;
/// For case when there are no keys (all aggregate into one row).
void executeWithoutKeyImpl(
static void executeWithoutKeyImpl(
AggregatedDataWithoutKey & res,
size_t rows,
AggregateFunctionInstruction * aggregate_instructions,
Arena * arena) const;
Arena * arena);
template <typename Method>
void writeToTemporaryFileImpl(

View File

@ -218,7 +218,7 @@ private:
/// Buffer should be allocated with features_count * column->size() elements.
/// Place string pointers in positions buffer[0], buffer[features_count], ... , buffer[size * features_count]
void placeStringColumn(const ColumnString & column, const char ** buffer, size_t features_count) const
static void placeStringColumn(const ColumnString & column, const char ** buffer, size_t features_count)
{
size_t size = column.size();
for (size_t i = 0; i < size; ++i)
@ -231,8 +231,8 @@ private:
/// Buffer should be allocated with features_count * column->size() elements.
/// Place string pointers in positions buffer[0], buffer[features_count], ... , buffer[size * features_count]
/// Returns PODArray which holds data (because ColumnFixedString doesn't store terminating zero).
PODArray<char> placeFixedStringColumn(
const ColumnFixedString & column, const char ** buffer, size_t features_count) const
static PODArray<char> placeFixedStringColumn(
const ColumnFixedString & column, const char ** buffer, size_t features_count)
{
size_t size = column.size();
size_t str_size = column.getN();
@ -281,8 +281,8 @@ private:
/// Place columns into buffer, returns data which was used for fixed string columns.
/// Buffer should contains column->size() values, each value contains size strings.
std::vector<PODArray<char>> placeStringColumns(
const ColumnRawPtrs & columns, size_t offset, size_t size, const char ** buffer) const
static std::vector<PODArray<char>> placeStringColumns(
const ColumnRawPtrs & columns, size_t offset, size_t size, const char ** buffer)
{
if (size == 0)
return {};

View File

@ -668,7 +668,7 @@ void DDLWorker::processTask(DDLTask & task, const ZooKeeperPtr & zookeeper)
}
bool DDLWorker::taskShouldBeExecutedOnLeader(const ASTPtr ast_ddl, const StoragePtr storage) const
bool DDLWorker::taskShouldBeExecutedOnLeader(const ASTPtr ast_ddl, const StoragePtr storage)
{
/// Pure DROP queries have to be executed on each node separately
if (auto query = ast_ddl->as<ASTDropQuery>(); query && query->kind != ASTDropQuery::Kind::Truncate)

View File

@ -59,7 +59,7 @@ private:
void processTask(DDLTask & task, const ZooKeeperPtr & zookeeper);
/// Check that query should be executed on leader replica only
bool taskShouldBeExecutedOnLeader(const ASTPtr ast_ddl, StoragePtr storage) const;
static bool taskShouldBeExecutedOnLeader(const ASTPtr ast_ddl, StoragePtr storage);
/// Check that shard has consistent config with table
void checkShardConfig(const String & table, const DDLTask & task, StoragePtr storage) const;
@ -84,7 +84,7 @@ private:
void cleanupQueue(Int64 current_time_seconds, const ZooKeeperPtr & zookeeper);
/// Init task node
void createStatusDirs(const std::string & node_path, const ZooKeeperPtr & zookeeper);
static void createStatusDirs(const std::string & node_path, const ZooKeeperPtr & zookeeper);
void runMainThread();

View File

@ -868,7 +868,7 @@ private:
}
}
void cancelLoading(Info & info)
static void cancelLoading(Info & info)
{
if (!info.is_loading())
return;

View File

@ -25,6 +25,12 @@ namespace ErrorCodes
}
static DatabasePtr tryGetDatabase(const String & database_name, bool if_exists)
{
return if_exists ? DatabaseCatalog::instance().tryGetDatabase(database_name) : DatabaseCatalog::instance().getDatabase(database_name);
}
InterpreterDropQuery::InterpreterDropQuery(const ASTPtr & query_ptr_, Context & context_) : query_ptr(query_ptr_), context(context_) {}
@ -227,6 +233,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(const String & table_name,
return {};
}
BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, ASTDropQuery::Kind kind, bool if_exists)
{
auto ddl_guard = DatabaseCatalog::instance().getDDLGuard(database_name, "");
@ -263,10 +270,6 @@ BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, AS
return {};
}
DatabasePtr InterpreterDropQuery::tryGetDatabase(const String & database_name, bool if_exists)
{
return if_exists ? DatabaseCatalog::instance().tryGetDatabase(database_name) : DatabaseCatalog::instance().getDatabase(database_name);
}
DatabaseAndTable InterpreterDropQuery::tryGetDatabaseAndTable(const String & database_name, const String & table_name, bool if_exists)
{

View File

@ -34,8 +34,6 @@ private:
BlockIO executeToDictionary(const String & database_name, const String & dictionary_name, ASTDropQuery::Kind kind, bool if_exists, bool is_temporary, bool no_ddl_lock);
DatabasePtr tryGetDatabase(const String & database_name, bool exists);
DatabaseAndTable tryGetDatabaseAndTable(const String & database_name, const String & table_name, bool if_exists);
BlockIO executeToTemporaryTable(const String & table_name, ASTDropQuery::Kind kind);

View File

@ -168,7 +168,7 @@ private:
void executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final);
void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final);
void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression);
void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression);
static void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression);
void executeOrder(Pipeline & pipeline, InputSortingInfoPtr sorting_info);
void executeWithFill(Pipeline & pipeline);
void executeMergeSorted(Pipeline & pipeline);
@ -176,7 +176,7 @@ private:
void executeUnion(Pipeline & pipeline, Block header);
void executeLimitBy(Pipeline & pipeline);
void executeLimit(Pipeline & pipeline);
void executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression);
static void executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression);
void executeDistinct(Pipeline & pipeline, bool before_order, Names columns);
void executeExtremes(Pipeline & pipeline);
void executeSubqueriesInSetsAndJoins(Pipeline & pipeline, const std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
@ -187,14 +187,14 @@ private:
void executeMergeAggregated(QueryPipeline & pipeline, bool overflow_row, bool final);
void executeTotalsAndHaving(QueryPipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final);
void executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression);
void executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression);
static void executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression);
void executeOrder(QueryPipeline & pipeline, InputSortingInfoPtr sorting_info);
void executeWithFill(QueryPipeline & pipeline);
void executeMergeSorted(QueryPipeline & pipeline);
void executePreLimit(QueryPipeline & pipeline, bool do_not_skip_offset);
void executeLimitBy(QueryPipeline & pipeline);
void executeLimit(QueryPipeline & pipeline);
void executeProjection(QueryPipeline & pipeline, const ExpressionActionsPtr & expression);
static void executeProjection(QueryPipeline & pipeline, const ExpressionActionsPtr & expression);
void executeDistinct(QueryPipeline & pipeline, bool before_order, Names columns);
void executeExtremes(QueryPipeline & pipeline);
void executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, const std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);

View File

@ -28,10 +28,6 @@ namespace ErrorCodes
extern const int SUPPORT_IS_DISABLED;
}
BlockInputStreamPtr InterpreterWatchQuery::executeImpl()
{
return std::make_shared<OneBlockInputStream>(Block());
}
BlockIO InterpreterWatchQuery::execute()
{

View File

@ -39,7 +39,6 @@ private:
ASTPtr query_ptr;
Context & context;
BlockInputStreamPtr executeImpl();
/// Table from where to read data, if not subquery.
StoragePtr storage;
/// Streams of read data

View File

@ -81,7 +81,7 @@ private:
bool mayOptimizeDisjunctiveEqualityChain(const DisjunctiveEqualityChain & chain) const;
/// Insert the IN expression into the OR chain.
void addInExpression(const DisjunctiveEqualityChain & chain);
static void addInExpression(const DisjunctiveEqualityChain & chain);
/// Delete the equalities that were replaced by the IN expressions.
void cleanupOrExpressions();

View File

@ -45,9 +45,9 @@ struct SelectQueryOptions
return out;
}
SelectQueryOptions & analyze(bool value = true)
SelectQueryOptions & analyze(bool dry_run = true)
{
only_analyze = value;
only_analyze = dry_run;
return *this;
}

View File

@ -887,7 +887,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(ASTPtr & query, const NamesAndTy
return std::make_shared<const SyntaxAnalyzerResult>(result);
}
void SyntaxAnalyzer::normalize(ASTPtr & query, Aliases & aliases, const Settings & settings) const
void SyntaxAnalyzer::normalize(ASTPtr & query, Aliases & aliases, const Settings & settings)
{
CustomizeFunctionsVisitor::Data data{settings.count_distinct_implementation};
CustomizeFunctionsVisitor(data).visit(query);

View File

@ -99,7 +99,7 @@ public:
private:
const Context & context;
void normalize(ASTPtr & query, Aliases & aliases, const Settings & settings) const;
static void normalize(ASTPtr & query, Aliases & aliases, const Settings & settings);
};
}

View File

@ -64,7 +64,7 @@ struct RestoreQualifiedNamesData
{
using TypeToVisit = ASTIdentifier;
void visit(ASTIdentifier & identifier, ASTPtr & ast);
static void visit(ASTIdentifier & identifier, ASTPtr & ast);
};
using RestoreQualifiedNamesMatcher = OneTypeMatcher<RestoreQualifiedNamesData>;

View File

@ -277,7 +277,7 @@ struct Grower : public HashTableGrower<>
}
/// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table.
[[noreturn]] void set(size_t /*num_elems*/)
[[noreturn]] static void set(size_t /*num_elems*/)
{
throw Poco::Exception(__PRETTY_FUNCTION__);
}

View File

@ -6,11 +6,11 @@
namespace DB
{
void ASTWithAlias::writeAlias(const String & name, const FormatSettings & settings) const
static void writeAlias(const String & name, const ASTWithAlias::FormatSettings & settings)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_alias : "");
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " AS " << (settings.hilite ? IAST::hilite_alias : "");
settings.writeIdentifier(name);
settings.ostr << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? IAST::hilite_none : "");
}

View File

@ -32,8 +32,6 @@ public:
protected:
virtual void appendColumnNameImpl(WriteBuffer & ostr) const = 0;
void writeAlias(const String & name, const FormatSettings & settings) const;
};
/// helper for setting aliases and chaining result to other functions

View File

@ -8,49 +8,10 @@
namespace DB
{
namespace ErrorCodes
namespace
{
}
bool ParserDropQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_drop("DROP");
ParserKeyword s_detach("DETACH");
ParserKeyword s_truncate("TRUNCATE");
if (s_drop.ignore(pos, expected))
return parseDropQuery(pos, node, expected);
else if (s_detach.ignore(pos, expected))
return parseDetachQuery(pos, node, expected);
else if (s_truncate.ignore(pos, expected))
return parseTruncateQuery(pos, node, expected);
else
return false;
}
bool ParserDropQuery::parseDetachQuery(Pos & pos, ASTPtr & node, Expected & expected)
{
if (parseDropQuery(pos, node, expected))
{
auto * drop_query = node->as<ASTDropQuery>();
drop_query->kind = ASTDropQuery::Kind::Detach;
return true;
}
return false;
}
bool ParserDropQuery::parseTruncateQuery(Pos & pos, ASTPtr & node, Expected & expected)
{
if (parseDropQuery(pos, node, expected))
{
auto * drop_query = node->as<ASTDropQuery>();
drop_query->kind = ASTDropQuery::Kind::Truncate;
return true;
}
return false;
}
bool ParserDropQuery::parseDropQuery(Pos & pos, ASTPtr & node, Expected & expected)
bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_temporary("TEMPORARY");
ParserKeyword s_table("TABLE");
@ -129,4 +90,44 @@ bool ParserDropQuery::parseDropQuery(Pos & pos, ASTPtr & node, Expected & expect
return true;
}
bool parseDetachQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
if (parseDropQuery(pos, node, expected))
{
auto * drop_query = node->as<ASTDropQuery>();
drop_query->kind = ASTDropQuery::Kind::Detach;
return true;
}
return false;
}
bool parseTruncateQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
if (parseDropQuery(pos, node, expected))
{
auto * drop_query = node->as<ASTDropQuery>();
drop_query->kind = ASTDropQuery::Kind::Truncate;
return true;
}
return false;
}
}
bool ParserDropQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_drop("DROP");
ParserKeyword s_detach("DETACH");
ParserKeyword s_truncate("TRUNCATE");
if (s_drop.ignore(pos, expected))
return parseDropQuery(pos, node, expected);
else if (s_detach.ignore(pos, expected))
return parseDetachQuery(pos, node, expected);
else if (s_truncate.ignore(pos, expected))
return parseTruncateQuery(pos, node, expected);
else
return false;
}
}

View File

@ -21,10 +21,6 @@ class ParserDropQuery : public IParserBase
protected:
const char * getName() const override{ return "DROP query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
bool parseDropQuery(Pos & pos, ASTPtr & node, Expected & expected);
bool parseDetachQuery(Pos & pos, ASTPtr & node, Expected & expected);
bool parseTruncateQuery(Pos & pos, ASTPtr & node, Expected & expected);
};
}

View File

@ -128,7 +128,7 @@ private:
return true;
}
void setDataType(LiteralInfo & info)
static void setDataType(LiteralInfo & info)
{
/// Type (Field::Types:Which) of literal in AST can be: String, UInt64, Int64, Float64, Null or Array of simple literals (not of Arrays).
/// Null and empty Array literals are considered as tokens, because template with Nullable(Nothing) or Array(Nothing) is useless.

View File

@ -12,12 +12,19 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
DB::RowInputFormatWithDiagnosticInfo::RowInputFormatWithDiagnosticInfo(const Block & header_, ReadBuffer & in_, const Params & params_)
static String alignedName(const String & name, size_t max_length)
{
size_t spaces_count = max_length >= name.size() ? max_length - name.size() : 0;
return name + ", " + std::string(spaces_count, ' ');
}
RowInputFormatWithDiagnosticInfo::RowInputFormatWithDiagnosticInfo(const Block & header_, ReadBuffer & in_, const Params & params_)
: IRowInputFormat(header_, in_, params_)
{
}
void DB::RowInputFormatWithDiagnosticInfo::updateDiagnosticInfo()
void RowInputFormatWithDiagnosticInfo::updateDiagnosticInfo()
{
++row_num;
@ -28,7 +35,7 @@ void DB::RowInputFormatWithDiagnosticInfo::updateDiagnosticInfo()
offset_of_current_row = in.offset();
}
String DB::RowInputFormatWithDiagnosticInfo::getDiagnosticInfo()
String RowInputFormatWithDiagnosticInfo::getDiagnosticInfo()
{
if (in.eof()) /// Buffer has gone, cannot extract information about what has been parsed.
return {};
@ -158,12 +165,6 @@ bool RowInputFormatWithDiagnosticInfo::deserializeFieldAndPrintDiagnosticInfo(co
return true;
}
String RowInputFormatWithDiagnosticInfo::alignedName(const String & name, size_t max_length) const
{
size_t spaces_count = max_length >= name.size() ? max_length - name.size() : 0;
return name + ", " + std::string(spaces_count, ' ');
}
void RowInputFormatWithDiagnosticInfo::resetParser()
{
IRowInputFormat::resetParser();

View File

@ -22,7 +22,6 @@ protected:
void updateDiagnosticInfo();
bool deserializeFieldAndPrintDiagnosticInfo(const String & col_name, const DataTypePtr & type, IColumn & column,
WriteBuffer & out, size_t file_column);
String alignedName(const String & name, size_t max_length) const;
virtual bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) = 0;
virtual void tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column,

View File

@ -195,7 +195,7 @@ private:
QueryStatus * process_list_element = nullptr;
void checkInitialized();
void checkSource(const ProcessorPtr & source, bool can_have_totals);
static void checkSource(const ProcessorPtr & source, bool can_have_totals);
template <typename TProcessorGetter>
void addSimpleTransformImpl(const TProcessorGetter & getter);

View File

@ -350,7 +350,7 @@ IProcessor::Status StrictResizeProcessor::prepare(const PortNumbers & updated_in
if (waiting_output.status != OutputStatus::NeedData)
throw Exception("Invalid status for associated output.", ErrorCodes::LOGICAL_ERROR);
waiting_output.port->pushData(input_with_data.port->pullData(/* set_not_deeded = */ true));
waiting_output.port->pushData(input_with_data.port->pullData(/* set_not_needed = */ true));
waiting_output.status = OutputStatus::NotActive;
if (input_with_data.port->isFinished())

View File

@ -382,7 +382,7 @@ private:
void traverseAST(const ASTPtr & node, const Context & context, Block & block_with_constants);
bool tryParseAtomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out);
bool tryParseLogicalOperatorFromAST(const ASTFunction * func, RPNElement & out);
static bool tryParseLogicalOperatorFromAST(const ASTFunction * func, RPNElement & out);
/** Is node the key column
* or expression in which column of key is wrapped by chain of functions,

View File

@ -2753,14 +2753,7 @@ MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const String & part_na
}
MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const DiskPtr & disk, const String & relative_path)
{
MutableDataPartPtr part = createPart(Poco::Path(relative_path).getFileName(), disk, relative_path);
loadPartAndFixMetadata(part);
return part;
}
void MergeTreeData::loadPartAndFixMetadata(MutableDataPartPtr part)
static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part)
{
String full_part_path = part->getFullPath();
@ -2786,6 +2779,13 @@ void MergeTreeData::loadPartAndFixMetadata(MutableDataPartPtr part)
}
}
MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const DiskPtr & disk, const String & relative_path)
{
MutableDataPartPtr part = createPart(Poco::Path(relative_path).getFileName(), disk, relative_path);
loadPartAndFixMetadataImpl(part);
return part;
}
void MergeTreeData::calculateColumnSizesImpl()
{
@ -3233,7 +3233,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const
{
LOG_DEBUG(log, "Checking part " << part_names.second);
MutableDataPartPtr part = createPart(part_names.first, name_to_disk[part_names.first], source_dir + part_names.second);
loadPartAndFixMetadata(part);
loadPartAndFixMetadataImpl(part);
loaded_parts.push_back(part);
}
@ -3257,25 +3257,20 @@ inline ReservationPtr checkAndReturnReservation(UInt64 expected_size, Reservatio
ReservationPtr MergeTreeData::reserveSpace(UInt64 expected_size) const
{
expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size);
auto reservation = getStoragePolicy()->reserve(expected_size);
return checkAndReturnReservation(expected_size, std::move(reservation));
}
ReservationPtr MergeTreeData::reserveSpace(UInt64 expected_size, SpacePtr space) const
ReservationPtr MergeTreeData::reserveSpace(UInt64 expected_size, SpacePtr space)
{
expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size);
auto reservation = tryReserveSpace(expected_size, space);
return checkAndReturnReservation(expected_size, std::move(reservation));
}
ReservationPtr MergeTreeData::tryReserveSpace(UInt64 expected_size, SpacePtr space) const
ReservationPtr MergeTreeData::tryReserveSpace(UInt64 expected_size, SpacePtr space)
{
expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size);
return space->reserve(expected_size);
}

View File

@ -601,7 +601,6 @@ public:
/// Check that the part is not broken and calculate the checksums for it if they are not present.
MutableDataPartPtr loadPartAndFixMetadata(const DiskPtr & disk, const String & relative_path);
void loadPartAndFixMetadata(MutableDataPartPtr part);
/** Create local backup (snapshot) for parts with specified prefix.
* Backup is created in directory clickhouse_dir/shadow/i/, where i - incremental number,
@ -689,8 +688,8 @@ public:
ReservationPtr reserveSpace(UInt64 expected_size) const;
/// Reserves space at least 1MB on specific disk or volume.
ReservationPtr reserveSpace(UInt64 expected_size, SpacePtr space) const;
ReservationPtr tryReserveSpace(UInt64 expected_size, SpacePtr space) const;
static ReservationPtr reserveSpace(UInt64 expected_size, SpacePtr space);
static ReservationPtr tryReserveSpace(UInt64 expected_size, SpacePtr space);
/// Reserves space at least 1MB preferring best destination according to `ttl_infos`.
ReservationPtr reserveSpacePreferringTTLRules(UInt64 expected_size,

View File

@ -1318,7 +1318,7 @@ void MergeTreeDataMergerMutator::splitMutationCommands(
MergeTreeData::DataPartPtr part,
const MutationCommands & commands,
MutationCommands & for_interpreter,
MutationCommands & for_file_renames) const
MutationCommands & for_file_renames)
{
NameSet removed_columns_from_compact_part;
NameSet already_changed_columns;
@ -1379,7 +1379,7 @@ void MergeTreeDataMergerMutator::splitMutationCommands(
NameSet MergeTreeDataMergerMutator::collectFilesToRemove(
MergeTreeData::DataPartPtr source_part, const MutationCommands & commands_for_removes, const String & mrk_extension) const
MergeTreeData::DataPartPtr source_part, const MutationCommands & commands_for_removes, const String & mrk_extension)
{
/// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes.
std::map<String, size_t> stream_counts;
@ -1393,7 +1393,6 @@ NameSet MergeTreeDataMergerMutator::collectFilesToRemove(
{});
}
NameSet remove_files;
/// Remove old indices
for (const auto & command : commands_for_removes)
@ -1422,11 +1421,12 @@ NameSet MergeTreeDataMergerMutator::collectFilesToRemove(
column->type->enumerateStreams(callback, stream_path);
}
}
return remove_files;
}
NameSet MergeTreeDataMergerMutator::collectFilesToSkip(
const Block & updated_header, const std::set<MergeTreeIndexPtr> & indices_to_recalc, const String & mrk_extension) const
const Block & updated_header, const std::set<MergeTreeIndexPtr> & indices_to_recalc, const String & mrk_extension)
{
NameSet files_to_skip = {"checksums.txt", "columns.txt"};
@ -1454,7 +1454,7 @@ NameSet MergeTreeDataMergerMutator::collectFilesToSkip(
NamesAndTypesList MergeTreeDataMergerMutator::getColumnsForNewDataPart(
MergeTreeData::DataPartPtr source_part, const Block & updated_header, NamesAndTypesList all_columns) const
MergeTreeData::DataPartPtr source_part, const Block & updated_header, NamesAndTypesList all_columns)
{
Names source_column_names = source_part->getColumns().getNames();
NameSet source_columns_name_set(source_column_names.begin(), source_column_names.end());

View File

@ -138,23 +138,23 @@ private:
* First part should be executed by mutations interpreter.
* Other is just simple drop/renames, so they can be executed without interpreter.
*/
void splitMutationCommands(
static void splitMutationCommands(
MergeTreeData::DataPartPtr part,
const MutationCommands & commands,
MutationCommands & for_interpreter,
MutationCommands & for_file_renames) const;
MutationCommands & for_file_renames);
/// Apply commands to source_part i.e. remove some columns in source_part
/// and return set of files, that have to be removed from filesystem and checksums
NameSet collectFilesToRemove(MergeTreeData::DataPartPtr source_part, const MutationCommands & commands_for_removes, const String & mrk_extension) const;
static NameSet collectFilesToRemove(MergeTreeData::DataPartPtr source_part, const MutationCommands & commands_for_removes, const String & mrk_extension);
/// Files, that we don't need to remove and don't need to hardlink, for example columns.txt and checksums.txt.
/// Because we will generate new versions of them after we perform mutation.
NameSet collectFilesToSkip(const Block & updated_header, const std::set<MergeTreeIndexPtr> & indices_to_recalc, const String & mrk_extension) const;
static NameSet collectFilesToSkip(const Block & updated_header, const std::set<MergeTreeIndexPtr> & indices_to_recalc, const String & mrk_extension);
/// Get the columns list of the resulting part in the same order as all_columns.
NamesAndTypesList getColumnsForNewDataPart(MergeTreeData::DataPartPtr source_part, const Block & updated_header, NamesAndTypesList all_columns) const;
static NamesAndTypesList getColumnsForNewDataPart(MergeTreeData::DataPartPtr source_part, const Block & updated_header, NamesAndTypesList all_columns);
bool shouldExecuteTTL(const Names & columns, const MutationCommands & commands) const;

View File

@ -17,6 +17,21 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
static void assertGranuleBlocksStructure(const Blocks & granule_index_blocks)
{
Block prev_block;
for (size_t index = 0; index < granule_index_blocks.size(); ++index)
{
Block granule_index_block = granule_index_blocks[index];
if (index != 0)
assertBlocksHaveEqualStructure(prev_block, granule_index_block, "Granule blocks of bloom filter has difference structure.");
prev_block = granule_index_block;
}
}
MergeTreeIndexGranuleBloomFilter::MergeTreeIndexGranuleBloomFilter(size_t bits_per_row_, size_t hash_functions_, size_t index_columns_)
: bits_per_row(bits_per_row_), hash_functions(hash_functions_)
{
@ -96,20 +111,6 @@ void MergeTreeIndexGranuleBloomFilter::serializeBinary(WriteBuffer & ostr) const
ostr.write(reinterpret_cast<const char *>(bloom_filter->getFilter().data()), bytes_size);
}
void MergeTreeIndexGranuleBloomFilter::assertGranuleBlocksStructure(const Blocks & granule_index_blocks) const
{
Block prev_block;
for (size_t index = 0; index < granule_index_blocks.size(); ++index)
{
Block granule_index_block = granule_index_blocks[index];
if (index != 0)
assertBlocksHaveEqualStructure(prev_block, granule_index_block, "Granule blocks of bloom filter has difference structure.");
prev_block = granule_index_block;
}
}
void MergeTreeIndexGranuleBloomFilter::fillingBloomFilter(BloomFilterPtr & bf, const Block & granule_index_block, size_t index_hash_column)
{
const auto & column = granule_index_block.getByPosition(index_hash_column);

View File

@ -27,8 +27,6 @@ private:
size_t hash_functions;
std::vector<BloomFilterPtr> bloom_filters;
void assertGranuleBlocksStructure(const Blocks & granule_index_blocks) const;
void fillingBloomFilter(BloomFilterPtr & bf, const Block & granule_index_block, size_t index_hash_column);
};

View File

@ -342,7 +342,7 @@ bool MergeTreeIndexConditionSet::atomFromAST(ASTPtr & node) const
return false;
}
bool MergeTreeIndexConditionSet::operatorFromAST(ASTPtr & node) const
bool MergeTreeIndexConditionSet::operatorFromAST(ASTPtr & node)
{
/// Functions AND, OR, NOT. Replace with bit*.
auto * func = node->as<ASTFunction>();

View File

@ -78,7 +78,7 @@ public:
private:
void traverseAST(ASTPtr & node) const;
bool atomFromAST(ASTPtr & node) const;
bool operatorFromAST(ASTPtr & node) const;
static bool operatorFromAST(ASTPtr & node);
bool checkASTUseless(const ASTPtr & node, bool atomic = false) const;

View File

@ -15,6 +15,25 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
static void filterColumns(Columns & columns, const IColumn::Filter & filter)
{
for (auto & column : columns)
{
if (column)
{
column = column->filter(filter, -1);
if (column->empty())
{
columns.clear();
return;
}
}
}
}
MergeTreeRangeReader::DelayedStream::DelayedStream(
size_t from_mark, IMergeTreeReader * merge_tree_reader_)
: current_mark(from_mark), current_offset(0), num_delayed_rows(0)
@ -682,22 +701,6 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
return read_result;
}
void MergeTreeRangeReader::filterColumns(Columns & columns, const IColumn::Filter & filter) const
{
for (auto & column : columns)
{
if (column)
{
column = column->filter(filter, -1);
if (column->empty())
{
columns.clear();
return;
}
}
}
}
MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t max_rows, MarkRanges & ranges)
{

View File

@ -213,7 +213,6 @@ private:
ReadResult startReadingChain(size_t max_rows, MarkRanges & ranges);
Columns continueReadingChain(ReadResult & result, size_t & num_rows);
void executePrewhereActionsAndFilterColumns(ReadResult & result);
void filterColumns(Columns & columns, const IColumn::Filter & filter) const;
IMergeTreeReader * merge_tree_reader = nullptr;
const MergeTreeIndexGranularity * index_granularity = nullptr;

View File

@ -71,6 +71,55 @@ static void collectIdentifiersNoSubqueries(const ASTPtr & ast, NameSet & set)
collectIdentifiersNoSubqueries(child, set);
}
static bool isConditionGood(const ASTPtr & condition)
{
const auto * function = condition->as<ASTFunction>();
if (!function)
return false;
/** we are only considering conditions of form `equals(one, another)` or `one = another`,
* especially if either `one` or `another` is ASTIdentifier */
if (function->name != "equals")
return false;
auto left_arg = function->arguments->children.front().get();
auto right_arg = function->arguments->children.back().get();
/// try to ensure left_arg points to ASTIdentifier
if (!left_arg->as<ASTIdentifier>() && right_arg->as<ASTIdentifier>())
std::swap(left_arg, right_arg);
if (left_arg->as<ASTIdentifier>())
{
/// condition may be "good" if only right_arg is a constant and its value is outside the threshold
if (const auto * literal = right_arg->as<ASTLiteral>())
{
const auto & field = literal->value;
const auto type = field.getType();
/// check the value with respect to threshold
if (type == Field::Types::UInt64)
{
const auto value = field.get<UInt64>();
return value > threshold;
}
else if (type == Field::Types::Int64)
{
const auto value = field.get<Int64>();
return value < -threshold || threshold < value;
}
else if (type == Field::Types::Float64)
{
const auto value = field.get<Float64>();
return value < threshold || threshold < value;
}
}
}
return false;
}
void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const ASTPtr & node) const
{
if (const auto * func_and = node->as<ASTFunction>(); func_and && func_and->name == "and")
@ -116,7 +165,7 @@ MergeTreeWhereOptimizer::Conditions MergeTreeWhereOptimizer::analyze(const ASTPt
}
/// Transform Conditions list to WHERE or PREWHERE expression.
ASTPtr MergeTreeWhereOptimizer::reconstruct(const Conditions & conditions) const
ASTPtr MergeTreeWhereOptimizer::reconstruct(const Conditions & conditions)
{
if (conditions.empty())
return {};
@ -205,55 +254,6 @@ UInt64 MergeTreeWhereOptimizer::getIdentifiersColumnSize(const NameSet & identif
}
bool MergeTreeWhereOptimizer::isConditionGood(const ASTPtr & condition) const
{
const auto * function = condition->as<ASTFunction>();
if (!function)
return false;
/** we are only considering conditions of form `equals(one, another)` or `one = another`,
* especially if either `one` or `another` is ASTIdentifier */
if (function->name != "equals")
return false;
auto left_arg = function->arguments->children.front().get();
auto right_arg = function->arguments->children.back().get();
/// try to ensure left_arg points to ASTIdentifier
if (!left_arg->as<ASTIdentifier>() && right_arg->as<ASTIdentifier>())
std::swap(left_arg, right_arg);
if (left_arg->as<ASTIdentifier>())
{
/// condition may be "good" if only right_arg is a constant and its value is outside the threshold
if (const auto * literal = right_arg->as<ASTLiteral>())
{
const auto & field = literal->value;
const auto type = field.getType();
/// check the value with respect to threshold
if (type == Field::Types::UInt64)
{
const auto value = field.get<UInt64>();
return value > threshold;
}
else if (type == Field::Types::Int64)
{
const auto value = field.get<Int64>();
return value < -threshold || threshold < value;
}
else if (type == Field::Types::Float64)
{
const auto value = field.get<Float64>();
return value < threshold || threshold < value;
}
}
}
return false;
}
bool MergeTreeWhereOptimizer::hasPrimaryKeyAtoms(const ASTPtr & ast) const
{
if (const auto * func = ast->as<ASTFunction>())

View File

@ -65,7 +65,7 @@ private:
Conditions analyze(const ASTPtr & expression) const;
/// Transform Conditions list to WHERE or PREWHERE expression.
ASTPtr reconstruct(const Conditions & conditions) const;
static ASTPtr reconstruct(const Conditions & conditions);
void calculateColumnSizes(const MergeTreeData & data, const Names & column_names);
@ -75,8 +75,6 @@ private:
UInt64 getIdentifiersColumnSize(const NameSet & identifiers) const;
bool isConditionGood(const ASTPtr & condition) const;
bool hasPrimaryKeyAtoms(const ASTPtr & ast) const;
bool isPrimaryKeyAtom(const ASTPtr & ast) const;

View File

@ -318,13 +318,8 @@ StoragePtr StorageDistributed::createWithOwnCluster(
return res;
}
QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context & context) const
{
auto cluster = getCluster();
return getQueryProcessingStage(context, cluster);
}
QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context & context, const ClusterPtr & cluster) const
static QueryProcessingStage::Enum getQueryProcessingStageImpl(const Context & context, const ClusterPtr & cluster)
{
const Settings & settings = context.getSettingsRef();
@ -339,6 +334,12 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Con
: QueryProcessingStage::WithMergeableState;
}
QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Context & context) const
{
auto cluster = getCluster();
return getQueryProcessingStageImpl(context, cluster);
}
Pipes StorageDistributed::read(
const Names & column_names,
const SelectQueryInfo & query_info,

View File

@ -67,7 +67,6 @@ public:
bool isRemote() const override { return true; }
QueryProcessingStage::Enum getQueryProcessingStage(const Context & context) const override;
QueryProcessingStage::Enum getQueryProcessingStage(const Context & context, const ClusterPtr & cluster) const;
Pipes read(
const Names & column_names,

View File

@ -175,7 +175,7 @@ public:
return splitted_blocks;
}
std::string dumpNamesWithBackQuote(const Block & block) const
static std::string dumpNamesWithBackQuote(const Block & block)
{
WriteBufferFromOwnString out;
for (auto it = block.begin(); it != block.end(); ++it)
@ -187,7 +187,6 @@ public:
return out.str();
}
private:
const StorageMySQL & storage;
std::string remote_database_name;

View File

@ -91,6 +91,28 @@ ASTPtr StorageView::getRuntimeViewQuery(const ASTSelectQuery & outer_query, cons
}
static void replaceTableNameWithSubquery(ASTSelectQuery * select_query, ASTPtr & subquery)
{
auto * select_element = select_query->tables()->children[0]->as<ASTTablesInSelectQueryElement>();
if (!select_element->table_expression)
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
auto * table_expression = select_element->table_expression->as<ASTTableExpression>();
if (!table_expression->database_and_table_name)
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
const auto alias = table_expression->database_and_table_name->tryGetAlias();
table_expression->database_and_table_name = {};
table_expression->subquery = std::make_shared<ASTSubquery>();
table_expression->subquery->children.push_back(subquery);
table_expression->children.push_back(table_expression->subquery);
if (!alias.empty())
table_expression->subquery->setAlias(alias);
}
ASTPtr StorageView::getRuntimeViewQuery(ASTSelectQuery * outer_query, const Context & context, bool normalize)
{
auto runtime_view_query = inner_query->clone();
@ -117,28 +139,6 @@ ASTPtr StorageView::getRuntimeViewQuery(ASTSelectQuery * outer_query, const Cont
return runtime_view_query;
}
void StorageView::replaceTableNameWithSubquery(ASTSelectQuery * select_query, ASTPtr & subquery)
{
auto * select_element = select_query->tables()->children[0]->as<ASTTablesInSelectQueryElement>();
if (!select_element->table_expression)
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
auto * table_expression = select_element->table_expression->as<ASTTableExpression>();
if (!table_expression->database_and_table_name)
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
const auto alias = table_expression->database_and_table_name->tryGetAlias();
table_expression->database_and_table_name = {};
table_expression->subquery = std::make_shared<ASTSubquery>();
table_expression->subquery->children.push_back(subquery);
table_expression->children.push_back(table_expression->subquery);
if (!alias.empty())
table_expression->subquery->setAlias(alias);
}
void registerStorageView(StorageFactory & factory)
{
factory.registerStorage("View", [](const StorageFactory::Arguments & args)

View File

@ -36,8 +36,6 @@ public:
private:
ASTPtr inner_query;
void replaceTableNameWithSubquery(ASTSelectQuery * select_query, ASTPtr & subquery);
protected:
StorageView(
const StorageID & table_id_,

View File

@ -25,10 +25,10 @@ NamesAndTypesList StorageSystemGraphite::getNamesAndTypes()
/*
* Looking for (Replicated)*GraphiteMergeTree and get all configuration parameters for them
*/
StorageSystemGraphite::Configs StorageSystemGraphite::getConfigs(const Context & context) const
static StorageSystemGraphite::Configs getConfigs(const Context & context)
{
const Databases databases = DatabaseCatalog::instance().getDatabases();
Configs graphite_configs;
StorageSystemGraphite::Configs graphite_configs;
for (const auto & db : databases)
{
@ -51,7 +51,7 @@ StorageSystemGraphite::Configs StorageSystemGraphite::getConfigs(const Context &
auto table_id = table_data->getStorageID();
if (!graphite_configs.count(config_name))
{
Config new_config =
StorageSystemGraphite::Config new_config =
{
table_data->merging_params.graphite_params,
{ table_id.database_name },
@ -73,7 +73,7 @@ StorageSystemGraphite::Configs StorageSystemGraphite::getConfigs(const Context &
void StorageSystemGraphite::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
{
Configs graphite_configs = StorageSystemGraphite::getConfigs(context);
Configs graphite_configs = getConfigs(context);
for (const auto & config : graphite_configs)
{

View File

@ -33,7 +33,6 @@ protected:
using IStorageSystemOneBlock::IStorageSystemOneBlock;
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override;
StorageSystemGraphite::Configs getConfigs(const Context & context) const;
};
}

View File

@ -106,7 +106,7 @@ private:
UInt64 block_size;
UInt64 max_counter;
Block createHeader() const
static Block createHeader()
{
return { ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "number") };
}

View File

@ -84,13 +84,23 @@ StoragePtr TableFunctionS3::getStorage(
const ColumnsDescription & columns,
Context & global_context,
const std::string & table_name,
const String & compression_method) const
const String & compression_method)
{
Poco::URI uri (source);
S3::URI s3_uri (uri);
UInt64 min_upload_part_size = global_context.getSettingsRef().s3_min_upload_part_size;
return StorageS3::create(s3_uri, access_key_id, secret_access_key, StorageID(getDatabaseName(), table_name), format, min_upload_part_size, columns, ConstraintsDescription{}, global_context, compression_method);
return StorageS3::create(
s3_uri,
access_key_id,
secret_access_key,
StorageID(getDatabaseName(), table_name),
format,
min_upload_part_size,
columns,
ConstraintsDescription{},
global_context,
compression_method);
}
void registerTableFunctionS3(TableFunctionFactory & factory)

View File

@ -29,7 +29,7 @@ private:
const Context & context,
const std::string & table_name) const override;
StoragePtr getStorage(
static StoragePtr getStorage(
const String & source,
const String & access_key_id,
const String & secret_access_key,
@ -37,7 +37,7 @@ private:
const ColumnsDescription & columns,
Context & global_context,
const std::string & table_name,
const String & compression_method) const;
const String & compression_method);
};
}