From 580146eedbb9da277677b6c27a5dab0d1b801371 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 10 Oct 2018 21:24:11 +0300 Subject: [PATCH 01/69] add primary key syntax [#CLICKHOUSE-3859] --- dbms/src/Interpreters/InterpreterCreateQuery.cpp | 2 +- dbms/src/Parsers/ASTCreateQuery.h | 8 ++++++++ dbms/src/Parsers/ParserCreateQuery.cpp | 11 +++++++++++ .../Storages/MergeTree/registerStorageMergeTree.cpp | 2 +- dbms/src/Storages/StorageFactory.cpp | 4 ++-- 5 files changed, 23 insertions(+), 4 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 69e7ae63a15..d2711e0d129 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -100,7 +100,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) const ASTStorage & storage = *create.storage; const ASTFunction & engine = *storage.engine; /// Currently, there are no database engines, that support any arguments. - if (engine.arguments || engine.parameters || storage.partition_by || storage.order_by || storage.sample_by || storage.settings) + if (engine.arguments || engine.parameters || storage.partition_by || storage.primary_key || storage.order_by || storage.sample_by || storage.settings) { std::stringstream ostr; formatAST(storage, ostr, false, false); diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index f257f6686d0..c341b6b40fe 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -16,6 +16,7 @@ class ASTStorage : public IAST public: ASTFunction * engine = nullptr; IAST * partition_by = nullptr; + IAST * primary_key = nullptr; IAST * order_by = nullptr; IAST * sample_by = nullptr; ASTSetQuery * settings = nullptr; @@ -31,6 +32,8 @@ public: res->set(res->engine, engine->clone()); if (partition_by) res->set(res->partition_by, partition_by->clone()); + if (primary_key) + res->set(res->primary_key, primary_key->clone()); if (order_by) res->set(res->order_by, order_by->clone()); if (sample_by) @@ -53,6 +56,11 @@ public: s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "PARTITION BY " << (s.hilite ? hilite_none : ""); partition_by->formatImpl(s, state, frame); } + if (primary_key) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "PRIMARY KEY " << (s.hilite ? hilite_none : ""); + primary_key->formatImpl(s, state, frame); + } if (order_by) { s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "ORDER BY " << (s.hilite ? hilite_none : ""); diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 6eaf5c4d5d8..290b3958457 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -96,6 +96,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_engine("ENGINE"); ParserToken s_eq(TokenType::Equals); ParserKeyword s_partition_by("PARTITION BY"); + ParserKeyword s_primary_key("PRIMARY KEY"); ParserKeyword s_order_by("ORDER BY"); ParserKeyword s_sample_by("SAMPLE BY"); ParserKeyword s_settings("SETTINGS"); @@ -106,6 +107,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr engine; ASTPtr partition_by; + ASTPtr primary_key; ASTPtr order_by; ASTPtr sample_by; ASTPtr settings; @@ -128,6 +130,14 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } + if (!primary_key && s_primary_key.ignore(pos, expected)) + { + if (expression_p.parse(pos, primary_key, expected)) + continue; + else + return false; + } + if (!order_by && s_order_by.ignore(pos, expected)) { if (expression_p.parse(pos, order_by, expected)) @@ -156,6 +166,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) auto storage = std::make_shared(); storage->set(storage->engine, engine); storage->set(storage->partition_by, partition_by); + storage->set(storage->primary_key, primary_key); storage->set(storage->order_by, order_by); storage->set(storage->sample_by, sample_by); storage->set(storage->settings, settings); diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index a875b743760..2b6715cf070 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -349,7 +349,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) */ bool is_extended_storage_def = - args.storage_def->partition_by || args.storage_def->order_by || args.storage_def->sample_by || args.storage_def->settings; + args.storage_def->partition_by || args.storage_def->primary_key || args.storage_def->order_by || args.storage_def->sample_by || args.storage_def->settings; String name_part = args.engine_name.substr(0, args.engine_name.size() - strlen("MergeTree")); diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index 05a61343108..3312ffaea80 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -95,11 +95,11 @@ StoragePtr StorageFactory::get( ErrorCodes::BAD_ARGUMENTS); } - if ((storage_def->partition_by || storage_def->order_by || storage_def->sample_by) + if ((storage_def->partition_by || storage_def->primary_key || storage_def->order_by || storage_def->sample_by) && !endsWith(name, "MergeTree")) { throw Exception( - "Engine " + name + " doesn't support PARTITION BY, ORDER BY or SAMPLE BY clauses. " + "Engine " + name + " doesn't support PARTITION BY, PRIMARY KEY, ORDER BY or SAMPLE BY clauses. " "Currently only the MergeTree family of engines supports them", ErrorCodes::BAD_ARGUMENTS); } From 9a92c0a55cab077027b5981139e2b64c4a55cc30 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 11 Oct 2018 17:53:23 +0300 Subject: [PATCH 02/69] sort key is distinct from primary key [#CLICKHOUSE-3859] --- .../src/Interpreters/MutationsInterpreter.cpp | 10 ++- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 63 +++++++++---------- dbms/src/Storages/MergeTree/MergeTreeData.h | 22 +++---- .../MergeTree/MergeTreeDataMergerMutator.cpp | 42 ++++++------- .../Storages/MergeTree/MergeTreeDataPart.cpp | 6 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 16 ++--- .../MergeTree/MergeTreeDataWriter.cpp | 11 +--- .../MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 36 +++++------ .../ReplicatedMergeTreeAlterThread.cpp | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 20 +++--- dbms/src/Storages/StorageMergeTree.cpp | 13 ++-- dbms/src/Storages/StorageMergeTree.h | 6 +- .../Storages/StorageReplicatedMergeTree.cpp | 12 ++-- .../src/Storages/StorageReplicatedMergeTree.h | 4 +- 15 files changed, 126 insertions(+), 139 deletions(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 16517532e38..37431ad72b8 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -102,20 +102,18 @@ static NameSet getKeyColumns(const StoragePtr & storage) for (const String & col : merge_tree_data->partition_expr->getRequiredColumns()) key_columns.insert(col); - auto primary_expr = merge_tree_data->getPrimaryExpression(); + auto primary_expr = merge_tree_data->getSortExpression(); if (primary_expr) for (const String & col : primary_expr->getRequiredColumns()) key_columns.insert(col); /// We don't process sampling_expression separately because it must be among the primary key columns. - auto secondary_sort_expr = merge_tree_data->getSecondarySortExpression(); - if (secondary_sort_expr) - for (const String & col : secondary_sort_expr->getRequiredColumns()) - key_columns.insert(col); - if (!merge_tree_data->merging_params.sign_column.empty()) key_columns.insert(merge_tree_data->merging_params.sign_column); + if (!merge_tree_data->merging_params.version_column.empty()) + key_columns.insert(merge_tree_data->merging_params.version_column); + return key_columns; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index a31d12d932f..f362387a423 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -86,8 +86,8 @@ MergeTreeData::MergeTreeData( const String & database_, const String & table_, const String & full_path_, const ColumnsDescription & columns_, Context & context_, - const ASTPtr & primary_expr_ast_, - const ASTPtr & secondary_sort_expr_ast_, + const ASTPtr & primary_key_expr_ast_, + const ASTPtr & sort_expr_ast_, const String & date_column_name, const ASTPtr & partition_expr_ast_, const ASTPtr & sampling_expression_, @@ -102,8 +102,8 @@ MergeTreeData::MergeTreeData( index_granularity(settings_.index_granularity), merging_params(merging_params_), settings(settings_), - primary_expr_ast(primary_expr_ast_), - secondary_sort_expr_ast(secondary_sort_expr_ast_), + primary_key_expr_ast(primary_key_expr_ast_), + sort_expr_ast(sort_expr_ast_), partition_expr_ast(partition_expr_ast_), require_part_metadata(require_part_metadata_), database_name(database_), table_name(table_), @@ -116,7 +116,7 @@ MergeTreeData::MergeTreeData( /// NOTE: using the same columns list as is read when performing actual merges. merging_params.check(getColumns().getAllPhysical()); - if (!primary_expr_ast) + if (!primary_key_expr_ast) throw Exception("Primary key cannot be empty", ErrorCodes::BAD_ARGUMENTS); initPrimaryKey(); @@ -216,44 +216,46 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam void MergeTreeData::initPrimaryKey() { - auto addSortColumns = [](Names & out, const ASTPtr & expr_ast) + auto add_columns = [](Names & out, const ASTPtr & expr_ast) { out.reserve(out.size() + expr_ast->children.size()); for (const ASTPtr & ast : expr_ast->children) out.emplace_back(ast->getColumnName()); }; - /// Initialize description of sorting for primary key. - primary_sort_columns.clear(); - addSortColumns(primary_sort_columns, primary_expr_ast); + primary_key_columns.clear(); + add_columns(primary_key_columns, primary_key_expr_ast); - primary_expr = ExpressionAnalyzer(primary_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(false); + primary_key_expr = ExpressionAnalyzer(primary_key_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(false); { ExpressionActionsPtr projected_expr = - ExpressionAnalyzer(primary_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(true); + ExpressionAnalyzer(primary_key_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(true); primary_key_sample = projected_expr->getSampleBlock(); } - checkKeyExpression(*primary_expr, primary_key_sample, "Primary"); + checkKeyExpression(*primary_key_expr, primary_key_sample, "Primary"); size_t primary_key_size = primary_key_sample.columns(); primary_key_data_types.resize(primary_key_size); for (size_t i = 0; i < primary_key_size; ++i) primary_key_data_types[i] = primary_key_sample.getByPosition(i).type; - sort_columns = primary_sort_columns; - if (secondary_sort_expr_ast) + sort_columns.clear(); + if (sort_expr_ast) { - addSortColumns(sort_columns, secondary_sort_expr_ast); - secondary_sort_expr = ExpressionAnalyzer(secondary_sort_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(false); + add_columns(sort_columns, sort_expr_ast); + sort_expr = ExpressionAnalyzer(sort_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(false); ExpressionActionsPtr projected_expr = - ExpressionAnalyzer(secondary_sort_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(true); - auto secondary_key_sample = projected_expr->getSampleBlock(); + ExpressionAnalyzer(sort_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(true); + auto sort_expr_sample = projected_expr->getSampleBlock(); - checkKeyExpression(*secondary_sort_expr, secondary_key_sample, "Secondary"); + checkKeyExpression(*sort_expr, sort_expr_sample, "Sorting"); } + + /// TODO: make more transactional + /// TODO: check that sort key is a prefix of the primary key } @@ -881,24 +883,19 @@ void MergeTreeData::checkAlter(const AlterCommands & commands) columns_alter_forbidden.insert(col); } - auto processSortingColumns = - [&columns_alter_forbidden, &columns_alter_metadata_only] (const ExpressionActionsPtr & expression) + if (sort_expr) { - for (const ExpressionAction & action : expression->getActions()) + for (const ExpressionAction & action : sort_expr->getActions()) { auto action_columns = action.getNeededColumns(); columns_alter_forbidden.insert(action_columns.begin(), action_columns.end()); } - for (const String & col : expression->getRequiredColumns()) + for (const String & col : sort_expr->getRequiredColumns()) columns_alter_metadata_only.insert(col); - }; - if (primary_expr) - processSortingColumns(primary_expr); - /// We don't process sampling_expression separately because it must be among the primary key columns. - - if (secondary_sort_expr) - processSortingColumns(secondary_sort_expr); + /// We don't process sampling_expression separately because it must be among the primary key columns + /// and we don't process primary_key_expr separately because it is a prefix of sort_expr. + } if (!merging_params.sign_column.empty()) columns_alter_forbidden.insert(merging_params.sign_column); @@ -1146,7 +1143,7 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( MergeTreeDataPartChecksum::uint128 new_primary_key_hash{}; /// TODO: Check the order of secondary sorting key columns. - if (new_primary_key.get() != primary_expr_ast.get()) + if (new_primary_key.get() != primary_key_expr_ast.get()) { ExpressionActionsPtr new_primary_expr = ExpressionAnalyzer(new_primary_key, context, nullptr, new_columns).getActions(true); Block new_primary_key_sample = new_primary_expr->getSampleBlock(); @@ -2342,7 +2339,7 @@ bool MergeTreeData::isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const A { const String column_name = node->getColumnName(); - for (const auto & name : primary_sort_columns) + for (const auto & name : primary_key_columns) if (column_name == name) return true; @@ -2399,7 +2396,7 @@ MergeTreeData * MergeTreeData::checkStructureAndGetMergeTreeData(const StoragePt return ast ? queryToString(ast) : ""; }; - if (query_to_string(secondary_sort_expr_ast) != query_to_string(src_data->secondary_sort_expr_ast)) + if (query_to_string(sort_expr_ast) != query_to_string(src_data->sort_expr_ast)) throw Exception("Tables have different ordering", ErrorCodes::BAD_ARGUMENTS); if (query_to_string(partition_expr_ast) != query_to_string(src_data->partition_expr_ast)) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 5ad413f21f8..f1a1e60d306 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -294,7 +294,7 @@ public: const ColumnsDescription & columns_, Context & context_, const ASTPtr & primary_expr_ast_, - const ASTPtr & secondary_sort_expr_ast_, + const ASTPtr & sort_expr_ast_, const String & date_column_name, const ASTPtr & partition_expr_ast_, const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported. @@ -486,10 +486,11 @@ public: broken_part_callback(name); } - bool hasPrimaryKey() const { return !primary_sort_columns.empty(); } - ExpressionActionsPtr getPrimaryExpression() const { return primary_expr; } - ExpressionActionsPtr getSecondarySortExpression() const { return secondary_sort_expr; } /// may return nullptr - Names getPrimarySortColumns() const { return primary_sort_columns; } + bool hasPrimaryKey() const { return !primary_key_columns.empty(); } + ExpressionActionsPtr getPrimaryKeyExpression() const { return primary_key_expr; } + bool hasSortExpression() const { return !sort_columns.empty(); } + ExpressionActionsPtr getSortExpression() const { return sort_expr; } /// may return nullptr + Names getPrimaryKeyColumns() const { return primary_key_columns; } Names getSortColumns() const { return sort_columns; } /// Check that the part is not broken and calculate the checksums for it if they are not present. @@ -548,8 +549,8 @@ public: const MergeTreeSettings settings; - ASTPtr primary_expr_ast; - ASTPtr secondary_sort_expr_ast; + ASTPtr primary_key_expr_ast; + ASTPtr sort_expr_ast; Block primary_key_sample; DataTypes primary_key_data_types; @@ -578,11 +579,10 @@ private: bool require_part_metadata; - ExpressionActionsPtr primary_expr; - /// Additional expression for sorting (of rows with the same primary keys). - ExpressionActionsPtr secondary_sort_expr; + ExpressionActionsPtr primary_key_expr; + ExpressionActionsPtr sort_expr; /// Names of columns for primary key. Is the prefix of sort_columns. - Names primary_sort_columns; + Names primary_key_columns; /// Names of columns for primary key + secondary sorting columns. Names sort_columns; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 2244fb28af2..50e38c4e08a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -333,20 +333,15 @@ MergeTreeData::DataPartsVector MergeTreeDataMergerMutator::selectAllPartsFromPar /// PK columns are sorted and merged, ordinary columns are gathered using info from merge step -static void extractMergingAndGatheringColumns(const NamesAndTypesList & all_columns, - const ExpressionActionsPtr & primary_key_expressions, const ExpressionActionsPtr & secondary_key_expressions, +static void extractMergingAndGatheringColumns( + const NamesAndTypesList & all_columns, + const ExpressionActionsPtr & sort_key_expressions, const MergeTreeData::MergingParams & merging_params, NamesAndTypesList & gathering_columns, Names & gathering_column_names, - NamesAndTypesList & merging_columns, Names & merging_column_names -) + NamesAndTypesList & merging_columns, Names & merging_column_names) { - Names primary_key_columns_vec = primary_key_expressions->getRequiredColumns(); - std::set key_columns(primary_key_columns_vec.cbegin(), primary_key_columns_vec.cend()); - if (secondary_key_expressions) - { - Names secondary_key_columns_vec = secondary_key_expressions->getRequiredColumns(); - key_columns.insert(secondary_key_columns_vec.begin(), secondary_key_columns_vec.end()); - } + Names sort_key_columns_vec = sort_key_expressions->getRequiredColumns(); + std::set key_columns(sort_key_columns_vec.cbegin(), sort_key_columns_vec.cend()); /// Force sign column for Collapsing mode if (merging_params.mode == MergeTreeData::MergingParams::Collapsing) @@ -366,20 +361,18 @@ static void extractMergingAndGatheringColumns(const NamesAndTypesList & all_colu /// TODO: also force "summing" and "aggregating" columns to make Horizontal merge only for such columns - for (auto & column : all_columns) + for (const auto & column : all_columns) { - auto it = std::find(key_columns.cbegin(), key_columns.cend(), column.name); - - if (key_columns.end() == it) - { - gathering_columns.emplace_back(column); - gathering_column_names.emplace_back(column.name); - } - else + if (key_columns.count(column.name)) { merging_columns.emplace_back(column); merging_column_names.emplace_back(column.name); } + else + { + gathering_columns.emplace_back(column); + gathering_column_names.emplace_back(column.name); + } } } @@ -555,8 +548,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor NamesAndTypesList gathering_columns, merging_columns; Names gathering_column_names, merging_column_names; - extractMergingAndGatheringColumns(all_columns, data.getPrimaryExpression(), data.getSecondarySortExpression() - , data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names); + extractMergingAndGatheringColumns( + all_columns, data.getSortExpression(), + data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names); MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared( data, future_part.name, future_part.part_info); @@ -638,7 +632,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (data.hasPrimaryKey()) src_streams.emplace_back(std::make_shared( - std::make_shared(BlockInputStreamPtr(std::move(input)), data.getPrimaryExpression()))); + std::make_shared(BlockInputStreamPtr(std::move(input)), data.getPrimaryKeyExpression()))); else src_streams.emplace_back(std::move(input)); } @@ -911,7 +905,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor if (data.hasPrimaryKey()) in = std::make_shared( - std::make_shared(in, data.getPrimaryExpression())); + std::make_shared(in, data.getPrimaryKeyExpression())); MergeTreeDataPart::MinMaxIndex minmax_idx; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp index 1c3d21d4653..459ed89e837 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp @@ -448,7 +448,7 @@ void MergeTreeDataPart::loadIndex() .getSize() / MERGE_TREE_MARK_SIZE; } - size_t key_size = storage.primary_sort_columns.size(); + size_t key_size = storage.primary_key_columns.size(); if (key_size) { @@ -631,7 +631,7 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata) if (!checksums.empty()) { - if (!storage.primary_sort_columns.empty() && !checksums.files.count("primary.idx")) + if (!storage.primary_key_columns.empty() && !checksums.files.count("primary.idx")) throw Exception("No checksum for primary.idx", ErrorCodes::NO_FILE_IN_DATA_PART); if (require_part_metadata) @@ -685,7 +685,7 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata) }; /// Check that the primary key index is not empty. - if (!storage.primary_sort_columns.empty()) + if (!storage.primary_key_columns.empty()) check_file_not_empty(path + "primary.idx"); if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index cbef565b7f0..c5dac809ee0 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -211,18 +211,18 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( data.check(real_column_names); const Settings & settings = context.getSettingsRef(); - Names primary_sort_columns = data.getPrimarySortColumns(); + Names primary_key_columns = data.getPrimaryKeyColumns(); KeyCondition key_condition( query_info, context, available_real_and_virtual_columns, - primary_sort_columns, data.getPrimaryExpression()); + primary_key_columns, data.getPrimaryKeyExpression()); if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue()) { std::stringstream exception_message; exception_message << "Primary key ("; - for (size_t i = 0, size = primary_sort_columns.size(); i < size; ++i) - exception_message << (i == 0 ? "" : ", ") << primary_sort_columns[i]; + for (size_t i = 0, size = primary_key_columns.size(); i < size; ++i) + exception_message << (i == 0 ? "" : ", ") << primary_key_columns[i]; exception_message << ") is not used and setting 'force_primary_key' is set."; throw Exception(exception_message.str(), ErrorCodes::INDEX_NOT_USED); @@ -379,7 +379,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); RelativeSize size_of_universum = 0; - DataTypePtr type = data.getPrimaryExpression()->getSampleBlock().getByName(data.sampling_expression->getColumnName()).type; + DataTypePtr type = data.primary_key_sample.getByName(data.sampling_expression->getColumnName()).type; if (typeid_cast(type.get())) size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); @@ -553,8 +553,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( if (select.final()) { - /// Add columns needed to calculate primary key and the sign. - std::vector add_columns = data.getPrimaryExpression()->getRequiredColumns(); + /// Add columns needed to calculate the sorting expression and the sign. + std::vector add_columns = data.getSortExpression()->getRequiredColumns(); column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); if (!data.merging_params.sign_column.empty()) @@ -782,7 +782,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal prewhere_info, true, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, true, virt_columns, part.part_index_in_query); - to_merge.emplace_back(std::make_shared(source_stream, data.getPrimaryExpression())); + to_merge.emplace_back(std::make_shared(source_stream, data.getSortExpression())); } Names sort_columns = data.getSortColumns(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 0446506e9d5..4a3e7af4576 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -175,13 +175,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa dir.createDirectories(); /// If we need to calculate some columns to sort. - if (data.hasPrimaryKey()) - { - data.getPrimaryExpression()->execute(block); - auto secondary_sort_expr = data.getSecondarySortExpression(); - if (secondary_sort_expr) - secondary_sort_expr->execute(block); - } + if (data.hasSortExpression()) + data.getSortExpression()->execute(block); Names sort_columns = data.getSortColumns(); SortDescription sort_description; @@ -196,7 +191,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa /// Sort. IColumn::Permutation * perm_ptr = nullptr; IColumn::Permutation perm; - if (data.hasPrimaryKey()) + if (!sort_description.empty()) { if (!isAlreadySorted(block, sort_description)) { diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 998084381cb..2c96bbf3781 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -40,7 +40,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( const MergeTreeData & data, const Names & column_names, Logger * log) - : primary_key_columns{ext::collection_cast(data.getPrimarySortColumns())}, + : primary_key_columns{ext::collection_cast(data.getPrimaryKeyColumns())}, table_columns{ext::map(data.getColumns().getAllPhysical(), [] (const NameAndTypePair & col) { return col.name; })}, block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, context, data.getColumns().getAllPhysical())}, diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index c12ddc51381..05098e1b82b 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -398,31 +398,31 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm /// The set of written offset columns so that you do not write shared offsets of nested structures columns several times WrittenOffsetColumns offset_columns; - auto sort_columns = storage.getPrimarySortColumns(); + auto primary_key_column_names = storage.getPrimaryKeyColumns(); /// Here we will add the columns related to the Primary Key, then write the index. - std::vector primary_columns(sort_columns.size()); - std::map primary_columns_name_to_position; + std::vector primary_key_columns(primary_key_column_names.size()); + std::map primary_key_column_name_to_position; - for (size_t i = 0, size = sort_columns.size(); i < size; ++i) + for (size_t i = 0, size = primary_key_column_names.size(); i < size; ++i) { - const auto & name = sort_columns[i]; + const auto & name = primary_key_column_names[i]; - if (!primary_columns_name_to_position.emplace(name, i).second) + if (!primary_key_column_name_to_position.emplace(name, i).second) throw Exception("Primary key contains duplicate columns", ErrorCodes::BAD_ARGUMENTS); - primary_columns[i] = block.getByName(name); + primary_key_columns[i] = block.getByName(name); - /// Reorder primary key columns in advance and add them to `primary_columns`. + /// Reorder primary key columns in advance and add them to `primary_key_columns`. if (permutation) - primary_columns[i].column = primary_columns[i].column->permute(*permutation, 0); + primary_key_columns[i].column = primary_key_columns[i].column->permute(*permutation, 0); } if (index_columns.empty()) { - index_columns.resize(sort_columns.size()); - for (size_t i = 0, size = sort_columns.size(); i < size; ++i) - index_columns[i] = primary_columns[i].column->cloneEmpty(); + index_columns.resize(primary_key_column_names.size()); + for (size_t i = 0, size = primary_key_column_names.size(); i < size; ++i) + index_columns[i] = primary_key_columns[i].column->cloneEmpty(); } if (serialization_states.empty()) @@ -447,10 +447,10 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm if (permutation) { - auto primary_column_it = primary_columns_name_to_position.find(it->name); - if (primary_columns_name_to_position.end() != primary_column_it) + auto primary_column_it = primary_key_column_name_to_position.find(it->name); + if (primary_key_column_name_to_position.end() != primary_column_it) { - auto & primary_column = *primary_columns[primary_column_it->second].column; + auto & primary_column = *primary_key_columns[primary_column_it->second].column; writeData(column.name, *column.type, primary_column, offset_columns, false, serialization_states[i]); } else @@ -482,11 +482,11 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm { if (storage.hasPrimaryKey()) { - for (size_t j = 0, size = primary_columns.size(); j < size; ++j) + for (size_t j = 0, size = primary_key_columns.size(); j < size; ++j) { - const IColumn & primary_column = *primary_columns[j].column.get(); + const IColumn & primary_column = *primary_key_columns[j].column.get(); index_columns[j]->insertFrom(primary_column, i); - primary_columns[j].type->serializeBinary(primary_column, i, *index_stream); + primary_key_columns[j].type->serializeBinary(primary_column, i, *index_stream); } } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp index a3e4dbb7bad..ccd7af7967f 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp @@ -131,7 +131,7 @@ void ReplicatedMergeTreeAlterThread::run() /// TODO: You can skip checking for too large changes if ZooKeeper has, for example, /// node /flags/force_alter. auto transaction = storage.data.alterDataPart( - part, columns_for_parts, storage.data.primary_expr_ast, false); + part, columns_for_parts, storage.data.primary_key_expr_ast, false); if (!transaction) continue; diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index 2b6715cf070..096dbde3b9f 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -492,8 +492,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) engine_args.erase(engine_args.begin(), engine_args.begin() + 2); } - ASTPtr secondary_sorting_expr_list; - if (merging_params.mode == MergeTreeData::MergingParams::Collapsing) { if (auto ast = typeid_cast(engine_args.back().get())) @@ -550,12 +548,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) } else if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) { - if (auto ast = typeid_cast(engine_args.back().get())) - { + if (auto ast = typeid_cast(engine_args.back().get())) merging_params.version_column = ast->name; - secondary_sorting_expr_list = std::make_shared(); - secondary_sorting_expr_list->children.push_back(engine_args.back()); - } else throw Exception( "Version column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), @@ -576,6 +570,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) String date_column_name; ASTPtr partition_expr_list; ASTPtr primary_expr_list; + ASTPtr sort_expr_list; ASTPtr sampling_expression; MergeTreeSettings storage_settings = args.context.getMergeTreeSettings(); @@ -585,7 +580,10 @@ static StoragePtr create(const StorageFactory::Arguments & args) partition_expr_list = extractKeyExpressionList(*args.storage_def->partition_by); if (args.storage_def->order_by) + { primary_expr_list = extractKeyExpressionList(*args.storage_def->order_by); + sort_expr_list = primary_expr_list->clone(); + } else throw Exception("You must provide an ORDER BY expression in the table definition. " "If you don't want this table to be sorted, use ORDER BY tuple()", @@ -615,6 +613,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) ErrorCodes::BAD_ARGUMENTS); primary_expr_list = extractKeyExpressionList(*engine_args[1]); + sort_expr_list = primary_expr_list->clone(); auto ast = typeid_cast(engine_args.back().get()); if (ast && ast->value.getType() == Field::Types::UInt64) @@ -625,17 +624,20 @@ static StoragePtr create(const StorageFactory::Arguments & args) ErrorCodes::BAD_ARGUMENTS); } + if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) + sort_expr_list->children.push_back(std::make_shared(merging_params.version_column)); + if (replicated) return StorageReplicatedMergeTree::create( zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name, args.columns, - args.context, primary_expr_list, secondary_sorting_expr_list, date_column_name, partition_expr_list, + args.context, primary_expr_list, sort_expr_list, date_column_name, partition_expr_list, sampling_expression, merging_params, storage_settings, args.has_force_restore_data_flag); else return StorageMergeTree::create( args.data_path, args.database_name, args.table_name, args.columns, args.attach, - args.context, primary_expr_list, secondary_sorting_expr_list, date_column_name, partition_expr_list, + args.context, primary_expr_list, sort_expr_list, date_column_name, partition_expr_list, sampling_expression, merging_params, storage_settings, args.has_force_restore_data_flag); } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 7f62f807456..9a098ca6cf5 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -46,8 +46,8 @@ StorageMergeTree::StorageMergeTree( const ColumnsDescription & columns_, bool attach, Context & context_, - const ASTPtr & primary_expr_ast_, - const ASTPtr & secondary_sorting_expr_list_, + const ASTPtr & primary_key_expr_ast_, + const ASTPtr & sort_expr_ast_, const String & date_column_name, const ASTPtr & partition_expr_ast_, const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported. @@ -58,7 +58,7 @@ StorageMergeTree::StorageMergeTree( context(context_), background_pool(context_.getBackgroundPool()), data(database_name, table_name, full_path, columns_, - context_, primary_expr_ast_, secondary_sorting_expr_list_, date_column_name, partition_expr_ast_, + context_, primary_key_expr_ast_, sort_expr_ast_, date_column_name, partition_expr_ast_, sampling_expression_, merging_params_, settings_, false, attach), reader(data), writer(data), merger_mutator(data, context.getBackgroundPool()), @@ -202,7 +202,8 @@ void StorageMergeTree::alter( bool primary_key_is_modified = false; - ASTPtr new_primary_key_ast = data.primary_expr_ast; + ASTPtr new_primary_key_ast = data.primary_key_expr_ast; + /// TODO: modify sort expression for (const AlterCommand & param : params) { @@ -248,7 +249,7 @@ void StorageMergeTree::alter( if (primary_key_is_modified) { - data.primary_expr_ast = new_primary_key_ast; + data.primary_key_expr_ast = new_primary_key_ast; } /// Reinitialize primary key because primary key column types might have changed. data.initPrimaryKey(); @@ -715,7 +716,7 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi if (part->info.partition_id != partition_id) throw Exception("Unexpected partition ID " + part->info.partition_id + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); - if (auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_expr_ast, false)) + if (auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_key_expr_ast, false)) transactions.push_back(std::move(transaction)); LOG_DEBUG(log, "Removing column " << get(column_name) << " from part " << part->name); diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 0182f31dc8e..dd348104282 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -96,7 +96,7 @@ public: ASTPtr getSamplingExpression() const override { return data.sampling_expression; } - ASTPtr getPrimaryExpression() const override { return data.primary_expr_ast; } + ASTPtr getPrimaryExpression() const override { return data.primary_key_expr_ast; } private: String path; @@ -168,8 +168,8 @@ protected: const ColumnsDescription & columns_, bool attach, Context & context_, - const ASTPtr & primary_expr_ast_, - const ASTPtr & secondary_sorting_expr_list_, + const ASTPtr & primary_key_expr_ast_, + const ASTPtr & sort_expr_ast_, const String & date_column_name, const ASTPtr & partition_expr_ast_, const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported. diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index bee9f627d5e..42a19e7d945 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -197,8 +197,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const String & path_, const String & database_name_, const String & name_, const ColumnsDescription & columns_, Context & context_, - const ASTPtr & primary_expr_ast_, - const ASTPtr & secondary_sorting_expr_list_, + const ASTPtr & primary_key_expr_ast_, + const ASTPtr & sort_expr_ast_, const String & date_column_name, const ASTPtr & partition_expr_ast_, const ASTPtr & sampling_expression_, @@ -212,7 +212,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( replica_name(context.getMacros()->expand(replica_name_, database_name, table_name)), data(database_name, table_name, full_path, columns_, - context_, primary_expr_ast_, secondary_sorting_expr_list_, date_column_name, partition_expr_ast_, + context_, primary_key_expr_ast_, sort_expr_ast_, date_column_name, partition_expr_ast_, sampling_expression_, merging_params_, settings_, true, attach, [this] (const std::string & name) { enqueuePartForCheck(name); }), @@ -353,7 +353,7 @@ namespace << "index granularity: " << data.index_granularity << "\n" << "mode: " << static_cast(data.merging_params.mode) << "\n" << "sign column: " << data.merging_params.sign_column << "\n" - << "primary key: " << formattedAST(data.primary_expr_ast) << "\n"; + << "primary key: " << formattedAST(data.primary_key_expr_ast) << "\n"; if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { @@ -433,7 +433,7 @@ namespace in >> "\nprimary key: "; String read_primary_key; - String local_primary_key = formattedAST(data.primary_expr_ast); + String local_primary_key = formattedAST(data.primary_key_expr_ast); in >> read_primary_key; /// NOTE: You can make a less strict check of match expressions so that tables do not break from small changes @@ -1610,7 +1610,7 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & LOG_DEBUG(log, "Clearing column " << entry.column_name << " in part " << part->name); - auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_expr_ast, false); + auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_key_expr_ast, false); if (!transaction) continue; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 491f30d93e5..79802a71636 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -195,7 +195,7 @@ public: ASTPtr getSamplingExpression() const override { return data.sampling_expression; } - ASTPtr getPrimaryExpression() const override { return data.primary_expr_ast; } + ASTPtr getPrimaryExpression() const override { return data.primary_key_expr_ast; } private: /// Delete old parts from disk and from ZooKeeper. @@ -523,7 +523,7 @@ protected: const ColumnsDescription & columns_, Context & context_, const ASTPtr & primary_expr_ast_, - const ASTPtr & secondary_sorting_expr_list_, + const ASTPtr & sort_expr_ast_, const String & date_column_name, const ASTPtr & partition_expr_ast_, const ASTPtr & sampling_expression_, From 0fa86162d93a446cf2823185af7b3ec7e4505c9c Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 11 Oct 2018 20:06:55 +0300 Subject: [PATCH 03/69] separate sorting key and primary key, allow setting primary key separately [#CLICKHOUSE-3859] --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 103 +++++++++++------- dbms/src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../ReplicatedMergeTreeAlterThread.cpp | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 12 +- dbms/src/Storages/StorageMergeTree.cpp | 9 +- 6 files changed, 75 insertions(+), 55 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index f362387a423..2cedfe4febf 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -102,8 +102,6 @@ MergeTreeData::MergeTreeData( index_granularity(settings_.index_granularity), merging_params(merging_params_), settings(settings_), - primary_key_expr_ast(primary_key_expr_ast_), - sort_expr_ast(sort_expr_ast_), partition_expr_ast(partition_expr_ast_), require_part_metadata(require_part_metadata_), database_name(database_), table_name(table_), @@ -116,10 +114,7 @@ MergeTreeData::MergeTreeData( /// NOTE: using the same columns list as is read when performing actual merges. merging_params.check(getColumns().getAllPhysical()); - if (!primary_key_expr_ast) - throw Exception("Primary key cannot be empty", ErrorCodes::BAD_ARGUMENTS); - - initPrimaryKey(); + setPrimaryKey(primary_key_expr_ast_, sort_expr_ast_); if (sampling_expression && (!primary_key_sample.has(sampling_expression->getColumnName())) && !attach && !settings.compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility. @@ -214,48 +209,78 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam } -void MergeTreeData::initPrimaryKey() +void MergeTreeData::setPrimaryKey(ASTPtr new_primary_key_expr_ast, const ASTPtr & new_sort_expr_ast) { - auto add_columns = [](Names & out, const ASTPtr & expr_ast) + if (!new_sort_expr_ast) + throw Exception("Sorting key cannot be empty", ErrorCodes::BAD_ARGUMENTS); + + if (!new_primary_key_expr_ast) + new_primary_key_expr_ast = new_sort_expr_ast->clone(); + + if (new_sort_expr_ast.get() != sort_expr_ast.get() + && merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) { - out.reserve(out.size() + expr_ast->children.size()); - for (const ASTPtr & ast : expr_ast->children) - out.emplace_back(ast->getColumnName()); - }; - - primary_key_columns.clear(); - add_columns(primary_key_columns, primary_key_expr_ast); - - primary_key_expr = ExpressionAnalyzer(primary_key_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(false); - - { - ExpressionActionsPtr projected_expr = - ExpressionAnalyzer(primary_key_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(true); - primary_key_sample = projected_expr->getSampleBlock(); + new_sort_expr_ast->children.push_back(std::make_shared(merging_params.version_column)); } - checkKeyExpression(*primary_key_expr, primary_key_sample, "Primary"); + size_t primary_key_size = new_primary_key_expr_ast->children.size(); + size_t sort_key_size = new_sort_expr_ast->children.size(); + if (primary_key_size > sort_key_size) + throw Exception("Primary key must be a prefix of the sorting key, but its length: " + + toString(primary_key_size) + " is greater than the sorting key length: " + toString(sort_key_size), + ErrorCodes::BAD_ARGUMENTS); - size_t primary_key_size = primary_key_sample.columns(); - primary_key_data_types.resize(primary_key_size); + Names new_primary_key_columns; + Names new_sort_key_columns; + + for (size_t i = 0; i < sort_key_size; ++i) + { + String sort_key_column = new_sort_expr_ast->children[i]->getColumnName(); + new_sort_key_columns.push_back(sort_key_column); + + if (i < primary_key_size) + { + String pk_column = new_primary_key_expr_ast->children[i]->getColumnName(); + if (pk_column != sort_key_column) + throw Exception("Primary key must be a prefix of the sorting key, but in position " + + toString(i) + " its column is " + pk_column + ", not " + sort_key_column, + ErrorCodes::BAD_ARGUMENTS); + + new_primary_key_columns.push_back(pk_column); + } + } + + auto all_columns = getColumns().getAllPhysical(); + + auto new_sort_expr = ExpressionAnalyzer(new_sort_expr_ast, context, nullptr, all_columns) + .getActions(false); + auto new_sort_expr_sample = + ExpressionAnalyzer(new_sort_expr_ast, context, nullptr, all_columns) + .getActions(true)->getSampleBlock(); + + checkKeyExpression(*new_sort_expr, new_sort_expr_sample, "Sorting"); + + auto new_primary_key_expr = ExpressionAnalyzer(new_primary_key_expr_ast, context, nullptr, all_columns) + .getActions(false); + + Block new_primary_key_sample; + DataTypes new_primary_key_data_types; for (size_t i = 0; i < primary_key_size; ++i) - primary_key_data_types[i] = primary_key_sample.getByPosition(i).type; - - sort_columns.clear(); - if (sort_expr_ast) { - add_columns(sort_columns, sort_expr_ast); - sort_expr = ExpressionAnalyzer(sort_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(false); - - ExpressionActionsPtr projected_expr = - ExpressionAnalyzer(sort_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(true); - auto sort_expr_sample = projected_expr->getSampleBlock(); - - checkKeyExpression(*sort_expr, sort_expr_sample, "Sorting"); + const auto & elem = new_sort_expr_sample.getByPosition(i); + new_primary_key_sample.insert(elem); + new_primary_key_data_types.push_back(elem.type); } - /// TODO: make more transactional - /// TODO: check that sort key is a prefix of the primary key + sort_expr_ast = new_sort_expr_ast; + sort_columns = std::move(new_sort_key_columns); + sort_expr = std::move(new_sort_expr); + + primary_key_expr_ast = new_primary_key_expr_ast; + primary_key_columns = std::move(new_primary_key_columns); + primary_key_expr = std::move(new_primary_key_expr); + primary_key_sample = std::move(new_primary_key_sample); + primary_key_data_types = std::move(new_primary_key_data_types); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index f1a1e60d306..83f3636c068 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -687,7 +687,7 @@ private: /// The same for clearOldTemporaryDirectories. std::mutex clear_old_temporary_directories_mutex; - void initPrimaryKey(); + void setPrimaryKey(ASTPtr new_primary_key_expr_ast, const ASTPtr & new_sort_expr_ast); void initPartitionKey(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 50e38c4e08a..bac88c5414f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -632,7 +632,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (data.hasPrimaryKey()) src_streams.emplace_back(std::make_shared( - std::make_shared(BlockInputStreamPtr(std::move(input)), data.getPrimaryKeyExpression()))); + std::make_shared(BlockInputStreamPtr(std::move(input)), data.getSortExpression()))); else src_streams.emplace_back(std::move(input)); } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp index ccd7af7967f..2b310e9bacb 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp @@ -95,7 +95,7 @@ void ReplicatedMergeTreeAlterThread::run() storage.setColumns(std::move(columns_in_zk)); /// Reinitialize primary key because primary key column types might have changed. - storage.data.initPrimaryKey(); + storage.data.setPrimaryKey(storage.data.primary_key_expr_ast, storage.data.sort_expr_ast); LOG_INFO(log, "Applied changes to table."); } diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index 096dbde3b9f..a4436545bde 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -579,10 +579,12 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.storage_def->partition_by) partition_expr_list = extractKeyExpressionList(*args.storage_def->partition_by); + if (args.storage_def->primary_key) + primary_expr_list = extractKeyExpressionList(*args.storage_def->primary_key); + if (args.storage_def->order_by) { - primary_expr_list = extractKeyExpressionList(*args.storage_def->order_by); - sort_expr_list = primary_expr_list->clone(); + sort_expr_list = extractKeyExpressionList(*args.storage_def->order_by); } else throw Exception("You must provide an ORDER BY expression in the table definition. " @@ -612,8 +614,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) "Date column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS); - primary_expr_list = extractKeyExpressionList(*engine_args[1]); - sort_expr_list = primary_expr_list->clone(); + sort_expr_list = extractKeyExpressionList(*engine_args[1]); auto ast = typeid_cast(engine_args.back().get()); if (ast && ast->value.getType() == Field::Types::UInt64) @@ -624,9 +625,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) ErrorCodes::BAD_ARGUMENTS); } - if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) - sort_expr_list->children.push_back(std::make_shared(merging_params.version_column)); - if (replicated) return StorageReplicatedMergeTree::create( zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name, diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 9a098ca6cf5..4b6f0dd4665 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -203,7 +203,7 @@ void StorageMergeTree::alter( bool primary_key_is_modified = false; ASTPtr new_primary_key_ast = data.primary_key_expr_ast; - /// TODO: modify sort expression + ASTPtr new_sort_expr_ast = data.sort_expr_ast; for (const AlterCommand & param : params) { @@ -211,6 +211,7 @@ void StorageMergeTree::alter( { primary_key_is_modified = true; new_primary_key_ast = param.primary_key; + new_sort_expr_ast = param.primary_key->clone(); } } @@ -247,12 +248,8 @@ void StorageMergeTree::alter( context.getDatabase(database_name)->alterTable(context, table_name, new_columns, storage_modifier); setColumns(std::move(new_columns)); - if (primary_key_is_modified) - { - data.primary_key_expr_ast = new_primary_key_ast; - } /// Reinitialize primary key because primary key column types might have changed. - data.initPrimaryKey(); + data.setPrimaryKey(new_primary_key_ast, new_sort_expr_ast); for (auto & transaction : transactions) transaction->commit(); From f4c26e78d27ae03253ede2e781312f3db580bbf2 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 12 Oct 2018 22:00:43 +0300 Subject: [PATCH 04/69] renames [#CLICKHOUSE-3859] --- .../src/Interpreters/MutationsInterpreter.cpp | 6 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 72 +++++++++---------- dbms/src/Storages/MergeTree/MergeTreeData.h | 27 +++---- .../MergeTree/MergeTreeDataMergerMutator.cpp | 10 +-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 +- .../MergeTree/MergeTreeDataWriter.cpp | 6 +- .../ReplicatedMergeTreeAlterThread.cpp | 4 +- dbms/src/Storages/StorageMergeTree.cpp | 16 ++--- dbms/src/Storages/StorageMergeTree.h | 8 +-- .../Storages/StorageReplicatedMergeTree.cpp | 12 ++-- .../src/Storages/StorageReplicatedMergeTree.h | 6 +- 11 files changed, 88 insertions(+), 85 deletions(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 37431ad72b8..13e4b4c25d3 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -102,9 +102,9 @@ static NameSet getKeyColumns(const StoragePtr & storage) for (const String & col : merge_tree_data->partition_expr->getRequiredColumns()) key_columns.insert(col); - auto primary_expr = merge_tree_data->getSortExpression(); - if (primary_expr) - for (const String & col : primary_expr->getRequiredColumns()) + auto sorting_key_expr = merge_tree_data->getSortingKeyExpression(); + if (sorting_key_expr) + for (const String & col : sorting_key_expr->getRequiredColumns()) key_columns.insert(col); /// We don't process sampling_expression separately because it must be among the primary key columns. diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 2cedfe4febf..58cd2b82dc3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -86,8 +86,8 @@ MergeTreeData::MergeTreeData( const String & database_, const String & table_, const String & full_path_, const ColumnsDescription & columns_, Context & context_, - const ASTPtr & primary_key_expr_ast_, - const ASTPtr & sort_expr_ast_, + const ASTPtr & primary_key_ast_, + const ASTPtr & sorting_key_ast_, const String & date_column_name, const ASTPtr & partition_expr_ast_, const ASTPtr & sampling_expression_, @@ -114,7 +114,7 @@ MergeTreeData::MergeTreeData( /// NOTE: using the same columns list as is read when performing actual merges. merging_params.check(getColumns().getAllPhysical()); - setPrimaryKey(primary_key_expr_ast_, sort_expr_ast_); + setPrimaryKey(primary_key_ast_, sorting_key_ast_); if (sampling_expression && (!primary_key_sample.has(sampling_expression->getColumnName())) && !attach && !settings.compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility. @@ -209,41 +209,41 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam } -void MergeTreeData::setPrimaryKey(ASTPtr new_primary_key_expr_ast, const ASTPtr & new_sort_expr_ast) +void MergeTreeData::setPrimaryKey(ASTPtr new_primary_key_ast, const ASTPtr & new_sorting_key_ast) { - if (!new_sort_expr_ast) + if (!new_sorting_key_ast) throw Exception("Sorting key cannot be empty", ErrorCodes::BAD_ARGUMENTS); - if (!new_primary_key_expr_ast) - new_primary_key_expr_ast = new_sort_expr_ast->clone(); + if (!new_primary_key_ast) + new_primary_key_ast = new_sorting_key_ast->clone(); - if (new_sort_expr_ast.get() != sort_expr_ast.get() + if (new_sorting_key_ast.get() != sorting_key_ast.get() && merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) { - new_sort_expr_ast->children.push_back(std::make_shared(merging_params.version_column)); + new_sorting_key_ast->children.push_back(std::make_shared(merging_params.version_column)); } - size_t primary_key_size = new_primary_key_expr_ast->children.size(); - size_t sort_key_size = new_sort_expr_ast->children.size(); - if (primary_key_size > sort_key_size) + size_t primary_key_size = new_primary_key_ast->children.size(); + size_t sorting_key_size = new_sorting_key_ast->children.size(); + if (primary_key_size > sorting_key_size) throw Exception("Primary key must be a prefix of the sorting key, but its length: " - + toString(primary_key_size) + " is greater than the sorting key length: " + toString(sort_key_size), + + toString(primary_key_size) + " is greater than the sorting key length: " + toString(sorting_key_size), ErrorCodes::BAD_ARGUMENTS); Names new_primary_key_columns; - Names new_sort_key_columns; + Names new_sorting_key_columns; - for (size_t i = 0; i < sort_key_size; ++i) + for (size_t i = 0; i < sorting_key_size; ++i) { - String sort_key_column = new_sort_expr_ast->children[i]->getColumnName(); - new_sort_key_columns.push_back(sort_key_column); + String sorting_key_column = new_sorting_key_ast->children[i]->getColumnName(); + new_sorting_key_columns.push_back(sorting_key_column); if (i < primary_key_size) { - String pk_column = new_primary_key_expr_ast->children[i]->getColumnName(); - if (pk_column != sort_key_column) + String pk_column = new_primary_key_ast->children[i]->getColumnName(); + if (pk_column != sorting_key_column) throw Exception("Primary key must be a prefix of the sorting key, but in position " - + toString(i) + " its column is " + pk_column + ", not " + sort_key_column, + + toString(i) + " its column is " + pk_column + ", not " + sorting_key_column, ErrorCodes::BAD_ARGUMENTS); new_primary_key_columns.push_back(pk_column); @@ -252,31 +252,31 @@ void MergeTreeData::setPrimaryKey(ASTPtr new_primary_key_expr_ast, const ASTPtr auto all_columns = getColumns().getAllPhysical(); - auto new_sort_expr = ExpressionAnalyzer(new_sort_expr_ast, context, nullptr, all_columns) + auto new_sorting_key_expr = ExpressionAnalyzer(new_sorting_key_ast, context, nullptr, all_columns) .getActions(false); - auto new_sort_expr_sample = - ExpressionAnalyzer(new_sort_expr_ast, context, nullptr, all_columns) + auto new_sorting_key_sample = + ExpressionAnalyzer(new_sorting_key_ast, context, nullptr, all_columns) .getActions(true)->getSampleBlock(); - checkKeyExpression(*new_sort_expr, new_sort_expr_sample, "Sorting"); + checkKeyExpression(*new_sorting_key_expr, new_sorting_key_sample, "Sorting"); - auto new_primary_key_expr = ExpressionAnalyzer(new_primary_key_expr_ast, context, nullptr, all_columns) + auto new_primary_key_expr = ExpressionAnalyzer(new_primary_key_ast, context, nullptr, all_columns) .getActions(false); Block new_primary_key_sample; DataTypes new_primary_key_data_types; for (size_t i = 0; i < primary_key_size; ++i) { - const auto & elem = new_sort_expr_sample.getByPosition(i); + const auto & elem = new_sorting_key_sample.getByPosition(i); new_primary_key_sample.insert(elem); new_primary_key_data_types.push_back(elem.type); } - sort_expr_ast = new_sort_expr_ast; - sort_columns = std::move(new_sort_key_columns); - sort_expr = std::move(new_sort_expr); + sorting_key_ast = new_sorting_key_ast; + sorting_key_columns = std::move(new_sorting_key_columns); + sorting_key_expr = std::move(new_sorting_key_expr); - primary_key_expr_ast = new_primary_key_expr_ast; + primary_key_ast = new_primary_key_ast; primary_key_columns = std::move(new_primary_key_columns); primary_key_expr = std::move(new_primary_key_expr); primary_key_sample = std::move(new_primary_key_sample); @@ -908,18 +908,18 @@ void MergeTreeData::checkAlter(const AlterCommands & commands) columns_alter_forbidden.insert(col); } - if (sort_expr) + if (sorting_key_expr) { - for (const ExpressionAction & action : sort_expr->getActions()) + for (const ExpressionAction & action : sorting_key_expr->getActions()) { auto action_columns = action.getNeededColumns(); columns_alter_forbidden.insert(action_columns.begin(), action_columns.end()); } - for (const String & col : sort_expr->getRequiredColumns()) + for (const String & col : sorting_key_expr->getRequiredColumns()) columns_alter_metadata_only.insert(col); /// We don't process sampling_expression separately because it must be among the primary key columns - /// and we don't process primary_key_expr separately because it is a prefix of sort_expr. + /// and we don't process primary_key_expr separately because it is a prefix of sorting_key_expr. } if (!merging_params.sign_column.empty()) @@ -1168,7 +1168,7 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( MergeTreeDataPartChecksum::uint128 new_primary_key_hash{}; /// TODO: Check the order of secondary sorting key columns. - if (new_primary_key.get() != primary_key_expr_ast.get()) + if (new_primary_key.get() != primary_key_ast.get()) { ExpressionActionsPtr new_primary_expr = ExpressionAnalyzer(new_primary_key, context, nullptr, new_columns).getActions(true); Block new_primary_key_sample = new_primary_expr->getSampleBlock(); @@ -2421,7 +2421,7 @@ MergeTreeData * MergeTreeData::checkStructureAndGetMergeTreeData(const StoragePt return ast ? queryToString(ast) : ""; }; - if (query_to_string(sort_expr_ast) != query_to_string(src_data->sort_expr_ast)) + if (query_to_string(sorting_key_ast) != query_to_string(src_data->sorting_key_ast)) throw Exception("Tables have different ordering", ErrorCodes::BAD_ARGUMENTS); if (query_to_string(partition_expr_ast) != query_to_string(src_data->partition_expr_ast)) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 83f3636c068..7390807c50d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -284,7 +284,7 @@ public: /// Attach the table corresponding to the directory in full_path (must end with /), with the given columns. /// Correctness of names and paths is not checked. /// - /// primary_expr_ast - expression used for sorting; + /// primary_key_ast - expression used for sorting; /// date_column_name - if not empty, the name of the Date column used for partitioning by month. /// Otherwise, partition_expr_ast is used for partitioning. /// require_part_metadata - should checksums.txt and columns.txt exist in the part directory. @@ -293,8 +293,8 @@ public: const String & full_path_, const ColumnsDescription & columns_, Context & context_, - const ASTPtr & primary_expr_ast_, - const ASTPtr & sort_expr_ast_, + const ASTPtr & primary_key_ast_, + const ASTPtr & sorting_key_ast_, const String & date_column_name, const ASTPtr & partition_expr_ast_, const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported. @@ -488,10 +488,11 @@ public: bool hasPrimaryKey() const { return !primary_key_columns.empty(); } ExpressionActionsPtr getPrimaryKeyExpression() const { return primary_key_expr; } - bool hasSortExpression() const { return !sort_columns.empty(); } - ExpressionActionsPtr getSortExpression() const { return sort_expr; } /// may return nullptr Names getPrimaryKeyColumns() const { return primary_key_columns; } - Names getSortColumns() const { return sort_columns; } + + bool hasSortingKey() const { return !sorting_key_columns.empty(); } + ExpressionActionsPtr getSortingKeyExpression() const { return sorting_key_expr; } + Names getSortingKeyColumns() const { return sorting_key_columns; } /// Check that the part is not broken and calculate the checksums for it if they are not present. MutableDataPartPtr loadPartAndFixMetadata(const String & relative_path); @@ -549,11 +550,12 @@ public: const MergeTreeSettings settings; - ASTPtr primary_key_expr_ast; - ASTPtr sort_expr_ast; + ASTPtr primary_key_ast; Block primary_key_sample; DataTypes primary_key_data_types; + ASTPtr sorting_key_ast; + ASTPtr partition_expr_ast; ExpressionActionsPtr partition_expr; Block partition_key_sample; @@ -580,11 +582,12 @@ private: bool require_part_metadata; ExpressionActionsPtr primary_key_expr; - ExpressionActionsPtr sort_expr; - /// Names of columns for primary key. Is the prefix of sort_columns. + /// Names of columns for primary key. Names primary_key_columns; + + ExpressionActionsPtr sorting_key_expr; /// Names of columns for primary key + secondary sorting columns. - Names sort_columns; + Names sorting_key_columns; String database_name; String table_name; @@ -687,7 +690,7 @@ private: /// The same for clearOldTemporaryDirectories. std::mutex clear_old_temporary_directories_mutex; - void setPrimaryKey(ASTPtr new_primary_key_expr_ast, const ASTPtr & new_sort_expr_ast); + void setPrimaryKey(ASTPtr new_primary_key_ast, const ASTPtr & new_sorting_key_ast); void initPartitionKey(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index bac88c5414f..9aec7aa42d1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -335,12 +335,12 @@ MergeTreeData::DataPartsVector MergeTreeDataMergerMutator::selectAllPartsFromPar /// PK columns are sorted and merged, ordinary columns are gathered using info from merge step static void extractMergingAndGatheringColumns( const NamesAndTypesList & all_columns, - const ExpressionActionsPtr & sort_key_expressions, + const ExpressionActionsPtr & sorting_key_expr, const MergeTreeData::MergingParams & merging_params, NamesAndTypesList & gathering_columns, Names & gathering_column_names, NamesAndTypesList & merging_columns, Names & merging_column_names) { - Names sort_key_columns_vec = sort_key_expressions->getRequiredColumns(); + Names sort_key_columns_vec = sorting_key_expr->getRequiredColumns(); std::set key_columns(sort_key_columns_vec.cbegin(), sort_key_columns_vec.cend()); /// Force sign column for Collapsing mode @@ -549,7 +549,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor NamesAndTypesList gathering_columns, merging_columns; Names gathering_column_names, merging_column_names; extractMergingAndGatheringColumns( - all_columns, data.getSortExpression(), + all_columns, data.getSortingKeyExpression(), data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names); MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared( @@ -632,12 +632,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (data.hasPrimaryKey()) src_streams.emplace_back(std::make_shared( - std::make_shared(BlockInputStreamPtr(std::move(input)), data.getSortExpression()))); + std::make_shared(BlockInputStreamPtr(std::move(input)), data.getSortingKeyExpression()))); else src_streams.emplace_back(std::move(input)); } - Names sort_columns = data.getSortColumns(); + Names sort_columns = data.getSortingKeyColumns(); SortDescription sort_description; size_t sort_columns_size = sort_columns.size(); sort_description.reserve(sort_columns_size); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c5dac809ee0..6fb9f445ef9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -554,7 +554,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( if (select.final()) { /// Add columns needed to calculate the sorting expression and the sign. - std::vector add_columns = data.getSortExpression()->getRequiredColumns(); + std::vector add_columns = data.getSortingKeyExpression()->getRequiredColumns(); column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); if (!data.merging_params.sign_column.empty()) @@ -782,10 +782,10 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal prewhere_info, true, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, true, virt_columns, part.part_index_in_query); - to_merge.emplace_back(std::make_shared(source_stream, data.getSortExpression())); + to_merge.emplace_back(std::make_shared(source_stream, data.getSortingKeyExpression())); } - Names sort_columns = data.getSortColumns(); + Names sort_columns = data.getSortingKeyColumns(); SortDescription sort_description; size_t sort_columns_size = sort_columns.size(); sort_description.reserve(sort_columns_size); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 4a3e7af4576..a10aff25b6b 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -175,10 +175,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa dir.createDirectories(); /// If we need to calculate some columns to sort. - if (data.hasSortExpression()) - data.getSortExpression()->execute(block); + if (data.hasSortingKey()) + data.getSortingKeyExpression()->execute(block); - Names sort_columns = data.getSortColumns(); + Names sort_columns = data.getSortingKeyColumns(); SortDescription sort_description; size_t sort_columns_size = sort_columns.size(); sort_description.reserve(sort_columns_size); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp index 2b310e9bacb..5436aab75a3 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp @@ -95,7 +95,7 @@ void ReplicatedMergeTreeAlterThread::run() storage.setColumns(std::move(columns_in_zk)); /// Reinitialize primary key because primary key column types might have changed. - storage.data.setPrimaryKey(storage.data.primary_key_expr_ast, storage.data.sort_expr_ast); + storage.data.setPrimaryKey(storage.data.primary_key_ast, storage.data.sorting_key_ast); LOG_INFO(log, "Applied changes to table."); } @@ -131,7 +131,7 @@ void ReplicatedMergeTreeAlterThread::run() /// TODO: You can skip checking for too large changes if ZooKeeper has, for example, /// node /flags/force_alter. auto transaction = storage.data.alterDataPart( - part, columns_for_parts, storage.data.primary_key_expr_ast, false); + part, columns_for_parts, storage.data.primary_key_ast, false); if (!transaction) continue; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 4b6f0dd4665..1041c5f977d 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -46,8 +46,8 @@ StorageMergeTree::StorageMergeTree( const ColumnsDescription & columns_, bool attach, Context & context_, - const ASTPtr & primary_key_expr_ast_, - const ASTPtr & sort_expr_ast_, + const ASTPtr & primary_key_ast_, + const ASTPtr & sorting_key_ast_, const String & date_column_name, const ASTPtr & partition_expr_ast_, const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported. @@ -58,7 +58,7 @@ StorageMergeTree::StorageMergeTree( context(context_), background_pool(context_.getBackgroundPool()), data(database_name, table_name, full_path, columns_, - context_, primary_key_expr_ast_, sort_expr_ast_, date_column_name, partition_expr_ast_, + context_, primary_key_ast_, sorting_key_ast_, date_column_name, partition_expr_ast_, sampling_expression_, merging_params_, settings_, false, attach), reader(data), writer(data), merger_mutator(data, context.getBackgroundPool()), @@ -202,8 +202,8 @@ void StorageMergeTree::alter( bool primary_key_is_modified = false; - ASTPtr new_primary_key_ast = data.primary_key_expr_ast; - ASTPtr new_sort_expr_ast = data.sort_expr_ast; + ASTPtr new_primary_key_ast = data.primary_key_ast; + ASTPtr new_sorting_key_ast = data.sorting_key_ast; for (const AlterCommand & param : params) { @@ -211,7 +211,7 @@ void StorageMergeTree::alter( { primary_key_is_modified = true; new_primary_key_ast = param.primary_key; - new_sort_expr_ast = param.primary_key->clone(); + new_sorting_key_ast = param.primary_key->clone(); } } @@ -249,7 +249,7 @@ void StorageMergeTree::alter( setColumns(std::move(new_columns)); /// Reinitialize primary key because primary key column types might have changed. - data.setPrimaryKey(new_primary_key_ast, new_sort_expr_ast); + data.setPrimaryKey(new_primary_key_ast, new_sorting_key_ast); for (auto & transaction : transactions) transaction->commit(); @@ -713,7 +713,7 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi if (part->info.partition_id != partition_id) throw Exception("Unexpected partition ID " + part->info.partition_id + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); - if (auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_key_expr_ast, false)) + if (auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_key_ast, false)) transactions.push_back(std::move(transaction)); LOG_DEBUG(log, "Removing column " << get(column_name) << " from part " << part->name); diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index dd348104282..a9b006bbb99 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -96,7 +96,7 @@ public: ASTPtr getSamplingExpression() const override { return data.sampling_expression; } - ASTPtr getPrimaryExpression() const override { return data.primary_key_expr_ast; } + ASTPtr getPrimaryExpression() const override { return data.primary_key_ast; } private: String path; @@ -157,7 +157,7 @@ protected: * (correctness of names and paths are not checked) * consisting of the specified columns. * - * primary_expr_ast - expression for sorting; + * primary_key_ast - expression for sorting; * date_column_name - if not empty, the name of the column with the date used for partitioning by month; otherwise, partition_expr_ast is used as the partitioning expression; */ @@ -168,8 +168,8 @@ protected: const ColumnsDescription & columns_, bool attach, Context & context_, - const ASTPtr & primary_key_expr_ast_, - const ASTPtr & sort_expr_ast_, + const ASTPtr & primary_key_ast_, + const ASTPtr & sorting_key_ast_, const String & date_column_name, const ASTPtr & partition_expr_ast_, const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported. diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 42a19e7d945..a3c2def0446 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -197,8 +197,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const String & path_, const String & database_name_, const String & name_, const ColumnsDescription & columns_, Context & context_, - const ASTPtr & primary_key_expr_ast_, - const ASTPtr & sort_expr_ast_, + const ASTPtr & primary_key_ast_, + const ASTPtr & sorting_key_ast_, const String & date_column_name, const ASTPtr & partition_expr_ast_, const ASTPtr & sampling_expression_, @@ -212,7 +212,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( replica_name(context.getMacros()->expand(replica_name_, database_name, table_name)), data(database_name, table_name, full_path, columns_, - context_, primary_key_expr_ast_, sort_expr_ast_, date_column_name, partition_expr_ast_, + context_, primary_key_ast_, sorting_key_ast_, date_column_name, partition_expr_ast_, sampling_expression_, merging_params_, settings_, true, attach, [this] (const std::string & name) { enqueuePartForCheck(name); }), @@ -353,7 +353,7 @@ namespace << "index granularity: " << data.index_granularity << "\n" << "mode: " << static_cast(data.merging_params.mode) << "\n" << "sign column: " << data.merging_params.sign_column << "\n" - << "primary key: " << formattedAST(data.primary_key_expr_ast) << "\n"; + << "primary key: " << formattedAST(data.primary_key_ast) << "\n"; if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { @@ -433,7 +433,7 @@ namespace in >> "\nprimary key: "; String read_primary_key; - String local_primary_key = formattedAST(data.primary_key_expr_ast); + String local_primary_key = formattedAST(data.primary_key_ast); in >> read_primary_key; /// NOTE: You can make a less strict check of match expressions so that tables do not break from small changes @@ -1610,7 +1610,7 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & LOG_DEBUG(log, "Clearing column " << entry.column_name << " in part " << part->name); - auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_key_expr_ast, false); + auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_key_ast, false); if (!transaction) continue; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 79802a71636..0ba0e401a82 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -195,7 +195,7 @@ public: ASTPtr getSamplingExpression() const override { return data.sampling_expression; } - ASTPtr getPrimaryExpression() const override { return data.primary_key_expr_ast; } + ASTPtr getPrimaryExpression() const override { return data.primary_key_ast; } private: /// Delete old parts from disk and from ZooKeeper. @@ -522,8 +522,8 @@ protected: const String & path_, const String & database_name_, const String & name_, const ColumnsDescription & columns_, Context & context_, - const ASTPtr & primary_expr_ast_, - const ASTPtr & sort_expr_ast_, + const ASTPtr & primary_key_ast_, + const ASTPtr & sorting_key_ast_, const String & date_column_name, const ASTPtr & partition_expr_ast_, const ASTPtr & sampling_expression_, From c4bf38ef0703b59b3117e1b1807ce4faf66f2229 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 15 Oct 2018 21:02:07 +0300 Subject: [PATCH 05/69] comments [#CLICKHOUSE-3859] --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 6 +++--- dbms/src/Storages/MergeTree/MergeTreeData.h | 6 +++++- .../MergeTree/registerStorageMergeTree.cpp | 19 +++++++++++-------- dbms/src/Storages/StorageMergeTree.h | 6 +++++- 4 files changed, 24 insertions(+), 13 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 58cd2b82dc3..3dbcee32d24 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -211,12 +211,12 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam void MergeTreeData::setPrimaryKey(ASTPtr new_primary_key_ast, const ASTPtr & new_sorting_key_ast) { + if (!new_primary_key_ast) + throw Exception("Primary key cannot be empty", ErrorCodes::BAD_ARGUMENTS); + if (!new_sorting_key_ast) throw Exception("Sorting key cannot be empty", ErrorCodes::BAD_ARGUMENTS); - if (!new_primary_key_ast) - new_primary_key_ast = new_sorting_key_ast->clone(); - if (new_sorting_key_ast.get() != sorting_key_ast.get() && merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) { diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 7390807c50d..3341c20a5de 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -284,7 +284,11 @@ public: /// Attach the table corresponding to the directory in full_path (must end with /), with the given columns. /// Correctness of names and paths is not checked. /// - /// primary_key_ast - expression used for sorting; + /// sorting_key_ast - expression used for sorting data in parts; + /// primary_key_ast - values of this expression for one row in every `index_granularity` rows + /// are written in the primary.idx to speed up range queries. + /// Primary key must be a prefix of the sorting key; + /// /// date_column_name - if not empty, the name of the Date column used for partitioning by month. /// Otherwise, partition_expr_ast is used for partitioning. /// require_part_metadata - should checksums.txt and columns.txt exist in the part directory. diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index a4436545bde..418795fcee7 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -570,7 +570,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) String date_column_name; ASTPtr partition_expr_list; ASTPtr primary_expr_list; - ASTPtr sort_expr_list; + ASTPtr sorting_expr_list; ASTPtr sampling_expression; MergeTreeSettings storage_settings = args.context.getMergeTreeSettings(); @@ -579,18 +579,20 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.storage_def->partition_by) partition_expr_list = extractKeyExpressionList(*args.storage_def->partition_by); - if (args.storage_def->primary_key) - primary_expr_list = extractKeyExpressionList(*args.storage_def->primary_key); - if (args.storage_def->order_by) { - sort_expr_list = extractKeyExpressionList(*args.storage_def->order_by); + sorting_expr_list = extractKeyExpressionList(*args.storage_def->order_by); } else throw Exception("You must provide an ORDER BY expression in the table definition. " "If you don't want this table to be sorted, use ORDER BY tuple()", ErrorCodes::BAD_ARGUMENTS); + if (args.storage_def->primary_key) + primary_expr_list = extractKeyExpressionList(*args.storage_def->primary_key); + else + primary_expr_list = sorting_expr_list->clone(); + if (args.storage_def->sample_by) sampling_expression = args.storage_def->sample_by->ptr(); @@ -614,7 +616,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) "Date column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS); - sort_expr_list = extractKeyExpressionList(*engine_args[1]); + sorting_expr_list = extractKeyExpressionList(*engine_args[1]); + primary_expr_list = sorting_expr_list->clone(); auto ast = typeid_cast(engine_args.back().get()); if (ast && ast->value.getType() == Field::Types::UInt64) @@ -629,13 +632,13 @@ static StoragePtr create(const StorageFactory::Arguments & args) return StorageReplicatedMergeTree::create( zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name, args.columns, - args.context, primary_expr_list, sort_expr_list, date_column_name, partition_expr_list, + args.context, primary_expr_list, sorting_expr_list, date_column_name, partition_expr_list, sampling_expression, merging_params, storage_settings, args.has_force_restore_data_flag); else return StorageMergeTree::create( args.data_path, args.database_name, args.table_name, args.columns, args.attach, - args.context, primary_expr_list, sort_expr_list, date_column_name, partition_expr_list, + args.context, primary_expr_list, sorting_expr_list, date_column_name, partition_expr_list, sampling_expression, merging_params, storage_settings, args.has_force_restore_data_flag); } diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index a9b006bbb99..dcb0ce56300 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -157,7 +157,11 @@ protected: * (correctness of names and paths are not checked) * consisting of the specified columns. * - * primary_key_ast - expression for sorting; + * sorting_key_ast - expression used for sorting data in parts; + * primary_key_ast - values of this expression for one row in every `index_granularity` rows + * are written in the primary.idx to speed up range queries. + * Primary key must be a prefix of the sorting key; + * * date_column_name - if not empty, the name of the column with the date used for partitioning by month; otherwise, partition_expr_ast is used as the partitioning expression; */ From 02337242b55d1f31c1e3b834dd79cd336ba1087a Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 15 Oct 2018 21:47:47 +0300 Subject: [PATCH 06/69] ALTER MODIFY ORDER BY syntax [#CLICKHOUSE-3859] --- dbms/src/Parsers/ASTAlterQuery.cpp | 12 ++++++++++++ dbms/src/Parsers/ASTAlterQuery.h | 5 +++++ dbms/src/Parsers/ParserAlterQuery.cpp | 16 ++++++++++++++++ dbms/src/Storages/AlterCommands.cpp | 9 ++++++++- dbms/src/Storages/AlterCommands.h | 4 ++++ 5 files changed, 45 insertions(+), 1 deletion(-) diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index 7081b512247..7bed86d7dcb 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -30,6 +30,11 @@ ASTPtr ASTAlterCommand::clone() const res->primary_key = primary_key->clone(); res->children.push_back(res->primary_key); } + if (sorting_key) + { + res->sorting_key = sorting_key->clone(); + res->children.push_back(res->sorting_key); + } if (partition) { res->partition = partition->clone(); @@ -84,6 +89,13 @@ void ASTAlterCommand::formatImpl( primary_key->formatImpl(settings, state, frame); settings.ostr << ")"; } + else if (type == ASTAlterCommand::MODIFY_ORDER_BY) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY ORDER BY " << (settings.hilite ? hilite_none : ""); + settings.ostr << "("; + sorting_key->formatImpl(settings, state, frame); + settings.ostr << ")"; + } else if (type == ASTAlterCommand::DROP_PARTITION) { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << (detach ? "DETACH" : "DROP") << " PARTITION " diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index c79f9ba8b2f..8aeee01b035 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -25,6 +25,7 @@ public: DROP_COLUMN, MODIFY_COLUMN, MODIFY_PRIMARY_KEY, + MODIFY_ORDER_BY, DROP_PARTITION, ATTACH_PARTITION, @@ -55,6 +56,10 @@ public: */ ASTPtr primary_key; + /** For MODIFY ORDER BY + */ + ASTPtr sorting_key; + /** Used in DROP PARTITION and ATTACH PARTITION FROM queries. * The value or ID of the partition is stored here. */ diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 56eaddb38ee..619755a7800 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -24,6 +24,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_clear_column("CLEAR COLUMN"); ParserKeyword s_modify_column("MODIFY COLUMN"); ParserKeyword s_modify_primary_key("MODIFY PRIMARY KEY"); + ParserKeyword s_modify_order_by("MODIFY ORDER BY"); ParserKeyword s_attach_partition("ATTACH PARTITION"); ParserKeyword s_detach_partition("DETACH PARTITION"); @@ -200,6 +201,21 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::MODIFY_PRIMARY_KEY; } + else if (s_modify_order_by.ignore(pos, expected)) + { + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + if (!ParserNotEmptyExpressionList(false).parse(pos, command->sorting_key, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + command->type = ASTAlterCommand::MODIFY_ORDER_BY; + } else if (s_delete_where.ignore(pos, expected)) { if (!parser_exp_elem.parse(pos, command->predicate, expected)) diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 9e6d525f685..8f8537968da 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -90,6 +90,13 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.primary_key = command_ast->primary_key; return command; } + else if (command_ast->type == ASTAlterCommand::MODIFY_ORDER_BY) + { + AlterCommand command; + command.type = AlterCommand::MODIFY_ORDER_BY; + command.sorting_key = command_ast->sorting_key; + return command; + } else return {}; } @@ -232,7 +239,7 @@ void AlterCommand::apply(ColumnsDescription & columns_description) const /// both old and new columns have default expression, update it columns_description.defaults[column_name].expression = default_expression; } - else if (type == MODIFY_PRIMARY_KEY) + else if (type == MODIFY_PRIMARY_KEY || type == MODIFY_ORDER_BY) { /// This have no relation to changing the list of columns. /// TODO Check that all columns exist, that only columns with constant defaults are added. diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index 888bd64f03e..d6806eb48af 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -21,6 +21,7 @@ struct AlterCommand DROP_COLUMN, MODIFY_COLUMN, MODIFY_PRIMARY_KEY, + MODIFY_ORDER_BY, }; Type type; @@ -42,6 +43,9 @@ struct AlterCommand /// For MODIFY_PRIMARY_KEY ASTPtr primary_key; + /// For MODIFY_ORDER_BY + ASTPtr sorting_key; + AlterCommand() = default; AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type, const ColumnDefaultKind default_kind, const ASTPtr & default_expression, From 07a22100cb4749c24e8caabf9b9e75428b1464bf Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Tue, 16 Oct 2018 17:32:36 +0300 Subject: [PATCH 07/69] ALTER ORDER BY for ordinary MergeTree [#CLICKHOUSE-3859] --- dbms/src/Storages/StorageMergeTree.cpp | 63 +++++++++++++++++++++----- 1 file changed, 51 insertions(+), 12 deletions(-) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 1041c5f977d..12fb3b2cbc9 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -201,8 +201,9 @@ void StorageMergeTree::alter( std::vector transactions; bool primary_key_is_modified = false; - ASTPtr new_primary_key_ast = data.primary_key_ast; + + bool sorting_key_is_modified = false; ASTPtr new_sorting_key_ast = data.sorting_key_ast; for (const AlterCommand & param : params) @@ -211,7 +212,11 @@ void StorageMergeTree::alter( { primary_key_is_modified = true; new_primary_key_ast = param.primary_key; - new_sorting_key_ast = param.primary_key->clone(); + } + else if (param.type == AlterCommand::MODIFY_ORDER_BY) + { + sorting_key_is_modified = true; + new_sorting_key_ast = param.sorting_key; } } @@ -229,19 +234,53 @@ void StorageMergeTree::alter( auto table_hard_lock = lockStructureForAlter(__PRETTY_FUNCTION__); IDatabase::ASTModifier storage_modifier; - if (primary_key_is_modified) + if (primary_key_is_modified || sorting_key_is_modified) { - storage_modifier = [&new_primary_key_ast] (IAST & ast) + storage_modifier = [&] (IAST & ast) { - auto tuple = std::make_shared(); - tuple->name = "tuple"; - tuple->arguments = new_primary_key_ast; - tuple->children.push_back(tuple->arguments); - - /// Primary key is in the second place in table engine description and can be represented as a tuple. - /// TODO: Not always in second place. If there is a sampling key, then the third one. Fix it. auto & storage_ast = typeid_cast(ast); - typeid_cast(*storage_ast.engine->arguments).children.at(1) = tuple; + + if (primary_key_is_modified) + { + auto tuple = std::make_shared(); + tuple->name = "tuple"; + tuple->arguments = new_primary_key_ast; + tuple->children.push_back(tuple->arguments); + + if (storage_ast.order_by) + { + if (storage_ast.primary_key) + storage_ast.set(storage_ast.primary_key, tuple); + else + { + storage_ast.set(storage_ast.order_by, tuple); + new_sorting_key_ast = new_primary_key_ast->clone(); + } + } + else + { + /// Primary key is in the second place in table engine description and can be represented as a tuple. + /// TODO: Not always in second place. If there is a sampling key, then the third one. Fix it. + storage_ast.engine->arguments->children.at(1) = tuple; + } + } + + if (sorting_key_is_modified) + { + /// TODO: helper for tuple creation + auto tuple = std::make_shared(); + tuple->name = "tuple"; + tuple->arguments = new_sorting_key_ast; + tuple->children.push_back(tuple->arguments); + + if (!storage_ast.order_by) + throw Exception("Not supported", ErrorCodes::LOGICAL_ERROR); /// TODO: better exception message + + if (!storage_ast.primary_key) + storage_ast.set(storage_ast.primary_key, storage_ast.order_by->clone()); + + storage_ast.set(storage_ast.order_by, tuple); + } }; } From 0c0c4fd28d59d612cd944f9d0681ad726adf4eff Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Tue, 23 Oct 2018 16:34:04 +0300 Subject: [PATCH 08/69] add sorting_and_primary_keys_independent [#CLICKHOUSE-3859] --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 11 ++++++++--- dbms/src/Storages/MergeTree/MergeTreeData.h | 3 +++ .../MergeTree/registerStorageMergeTree.cpp | 3 --- dbms/src/Storages/StorageMergeTree.cpp | 18 +++++++++++++----- dbms/src/Storages/StorageMergeTree.h | 1 + 5 files changed, 25 insertions(+), 11 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 3dbcee32d24..1f9ec8c6465 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -211,12 +211,16 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam void MergeTreeData::setPrimaryKey(ASTPtr new_primary_key_ast, const ASTPtr & new_sorting_key_ast) { - if (!new_primary_key_ast) - throw Exception("Primary key cannot be empty", ErrorCodes::BAD_ARGUMENTS); - if (!new_sorting_key_ast) throw Exception("Sorting key cannot be empty", ErrorCodes::BAD_ARGUMENTS); + bool new_sorting_and_primary_keys_independent = true; + if (!new_primary_key_ast) + { + new_primary_key_ast = new_sorting_key_ast->clone(); + new_sorting_and_primary_keys_independent = false; + } + if (new_sorting_key_ast.get() != sorting_key_ast.get() && merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) { @@ -275,6 +279,7 @@ void MergeTreeData::setPrimaryKey(ASTPtr new_primary_key_ast, const ASTPtr & new sorting_key_ast = new_sorting_key_ast; sorting_key_columns = std::move(new_sorting_key_columns); sorting_key_expr = std::move(new_sorting_key_expr); + sorting_and_primary_keys_independent = new_sorting_and_primary_keys_independent; primary_key_ast = new_primary_key_ast; primary_key_columns = std::move(new_primary_key_columns); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 3341c20a5de..4da0b0aa473 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -288,6 +288,7 @@ public: /// primary_key_ast - values of this expression for one row in every `index_granularity` rows /// are written in the primary.idx to speed up range queries. /// Primary key must be a prefix of the sorting key; + /// If it is nullptr, then it will be determined from sorting_key_ast. /// /// date_column_name - if not empty, the name of the Date column used for partitioning by month. /// Otherwise, partition_expr_ast is used for partitioning. @@ -559,6 +560,8 @@ public: DataTypes primary_key_data_types; ASTPtr sorting_key_ast; + /// If true, sorting and primary keys were set using separate clauses in the CREATE TABLE statement. + bool sorting_and_primary_keys_independent = false; ASTPtr partition_expr_ast; ExpressionActionsPtr partition_expr; diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index 418795fcee7..05d524ec61c 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -590,8 +590,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.storage_def->primary_key) primary_expr_list = extractKeyExpressionList(*args.storage_def->primary_key); - else - primary_expr_list = sorting_expr_list->clone(); if (args.storage_def->sample_by) sampling_expression = args.storage_def->sample_by->ptr(); @@ -617,7 +615,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) ErrorCodes::BAD_ARGUMENTS); sorting_expr_list = extractKeyExpressionList(*engine_args[1]); - primary_expr_list = sorting_expr_list->clone(); auto ast = typeid_cast(engine_args.back().get()); if (ast && ast->value.getType() == Field::Types::UInt64) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 12fb3b2cbc9..b849f9c85a1 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -206,6 +206,8 @@ void StorageMergeTree::alter( bool sorting_key_is_modified = false; ASTPtr new_sorting_key_ast = data.sorting_key_ast; + bool new_sorting_and_primary_keys_independent = data.sorting_and_primary_keys_independent; + for (const AlterCommand & param : params) { if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) @@ -216,6 +218,7 @@ void StorageMergeTree::alter( else if (param.type == AlterCommand::MODIFY_ORDER_BY) { sorting_key_is_modified = true; + new_sorting_and_primary_keys_independent = true; new_sorting_key_ast = param.sorting_key; } } @@ -249,13 +252,11 @@ void StorageMergeTree::alter( if (storage_ast.order_by) { - if (storage_ast.primary_key) + /// The table was created using the syntax with key expressions in separate clauses. + if (new_sorting_and_primary_keys_independent) storage_ast.set(storage_ast.primary_key, tuple); else - { storage_ast.set(storage_ast.order_by, tuple); - new_sorting_key_ast = new_primary_key_ast->clone(); - } } else { @@ -277,7 +278,11 @@ void StorageMergeTree::alter( throw Exception("Not supported", ErrorCodes::LOGICAL_ERROR); /// TODO: better exception message if (!storage_ast.primary_key) + { + /// Primary and sorting key become independent after this ALTER so we have to + /// save the old ORDER BY expression as the new primary key. storage_ast.set(storage_ast.primary_key, storage_ast.order_by->clone()); + } storage_ast.set(storage_ast.order_by, tuple); } @@ -288,7 +293,10 @@ void StorageMergeTree::alter( setColumns(std::move(new_columns)); /// Reinitialize primary key because primary key column types might have changed. - data.setPrimaryKey(new_primary_key_ast, new_sorting_key_ast); + if (new_sorting_and_primary_keys_independent) + data.setPrimaryKey(new_primary_key_ast, new_sorting_key_ast); + else + data.setPrimaryKey(nullptr, new_primary_key_ast); for (auto & transaction : transactions) transaction->commit(); diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index dcb0ce56300..8d31632d570 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -161,6 +161,7 @@ protected: * primary_key_ast - values of this expression for one row in every `index_granularity` rows * are written in the primary.idx to speed up range queries. * Primary key must be a prefix of the sorting key; + * If it is nullptr, then it will be determined from sorting_key_ast. * * date_column_name - if not empty, the name of the column with the date used for partitioning by month; otherwise, partition_expr_ast is used as the partitioning expression; From a0a9aad77784514a2a94848ff1a0aab553bdcaa8 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 17 Oct 2018 19:08:00 +0300 Subject: [PATCH 09/69] Stat data in ZooKeeperNodeCache [#CLICKHOUSE-3859] --- dbms/src/Common/Config/ConfigProcessor.cpp | 6 +-- .../Common/ZooKeeper/ZooKeeperNodeCache.cpp | 45 ++++++++----------- .../src/Common/ZooKeeper/ZooKeeperNodeCache.h | 12 +++-- 3 files changed, 30 insertions(+), 33 deletions(-) diff --git a/dbms/src/Common/Config/ConfigProcessor.cpp b/dbms/src/Common/Config/ConfigProcessor.cpp index 8cefc16500b..17507f3bedd 100644 --- a/dbms/src/Common/Config/ConfigProcessor.cpp +++ b/dbms/src/Common/Config/ConfigProcessor.cpp @@ -352,12 +352,12 @@ void ConfigProcessor::doIncludesRecursive( XMLDocumentPtr zk_document; auto get_zk_node = [&](const std::string & name) -> const Node * { - std::optional contents = zk_node_cache->get(name); - if (!contents) + zkutil::ZooKeeperNodeCache::GetResult result = zk_node_cache->get(name); + if (!result.exists) return nullptr; /// Enclose contents into a fake tag to allow pure text substitutions. - zk_document = dom_parser.parseString("" + *contents + ""); + zk_document = dom_parser.parseString("" + result.contents + ""); return getRootNode(zk_document.get()); }; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp index b4e6d857797..7b111e5c767 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp @@ -9,7 +9,7 @@ ZooKeeperNodeCache::ZooKeeperNodeCache(GetZooKeeper get_zookeeper_) { } -std::optional ZooKeeperNodeCache::get(const std::string & path) +ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path) { zkutil::ZooKeeperPtr zookeeper; std::unordered_set invalidated_paths; @@ -19,7 +19,6 @@ std::optional ZooKeeperNodeCache::get(const std::string & path) if (!context->zookeeper) { /// Possibly, there was a previous session and it has expired. Clear the cache. - nonexistent_nodes.clear(); node_cache.clear(); context->zookeeper = get_zookeeper(); @@ -33,13 +32,11 @@ std::optional ZooKeeperNodeCache::get(const std::string & path) throw DB::Exception("Could not get znode: `" + path + "'. ZooKeeper not configured.", DB::ErrorCodes::NO_ZOOKEEPER); for (const auto & invalidated_path : invalidated_paths) - { - nonexistent_nodes.erase(invalidated_path); node_cache.erase(invalidated_path); - } - if (nonexistent_nodes.count(path)) - return std::nullopt; + auto cache_it = node_cache.find(path); + if (cache_it != node_cache.end()) + return cache_it->second; auto watch_callback = [context=context](const Coordination::WatchResponse & response) { @@ -63,35 +60,29 @@ std::optional ZooKeeperNodeCache::get(const std::string & path) context->changed_event.set(); }; - std::string contents; + GetResult result; - auto cache_it = node_cache.find(path); - if (cache_it != node_cache.end()) + result.exists = zookeeper->tryGetWatch(path, result.contents, &result.stat, watch_callback); + if (result.exists) { - return cache_it->second; + node_cache.emplace(path, result); + return result; } - if (zookeeper->tryGetWatch(path, contents, /* stat = */nullptr, watch_callback)) + /// Node doesn't exist. We must set a watch on node creation (because it wasn't set by tryGetWatch). + + result.exists = zookeeper->existsWatch(path, &result.stat, watch_callback); + if (!result.exists) { - node_cache.emplace(path, contents); - return contents; + node_cache.emplace(path, result); + return result; } - /// Node doesn't exist. Create a watch on node creation. - nonexistent_nodes.insert(path); - - if (!zookeeper->existsWatch(path, /* stat = */nullptr, watch_callback)) - return std::nullopt; - /// Node was created between the two previous calls, try again. Watch is already set. - if (zookeeper->tryGet(path, contents)) - { - nonexistent_nodes.erase(path); - node_cache.emplace(path, contents); - return contents; - } - return std::nullopt; + result.exists = zookeeper->tryGet(path, result.contents, &result.stat); + node_cache.emplace(path, result); + return result; } } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h index 7a4cd6ce5bd..5ad9043ffbb 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h @@ -32,7 +32,14 @@ public: ZooKeeperNodeCache(const ZooKeeperNodeCache &) = delete; ZooKeeperNodeCache(ZooKeeperNodeCache &&) = default; - std::optional get(const std::string & path); + struct GetResult + { + bool exists = false; + std::string contents; + Coordination::Stat stat; + }; + + GetResult get(const std::string & path); Poco::Event & getChangedEvent() { return context->changed_event; } @@ -50,8 +57,7 @@ private: std::shared_ptr context; - std::unordered_set nonexistent_nodes; - std::unordered_map node_cache; + std::unordered_map node_cache; }; } From ce1ead5b8887af8e65ab8a69282d50a1015e1e20 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 17 Oct 2018 20:23:10 +0300 Subject: [PATCH 10/69] allow arbitrary callbacks in ZooKeeperNodeChange [#CLICKHOUSE-3859] --- dbms/programs/server/Server.cpp | 5 ++++- dbms/src/Common/Config/ConfigProcessor.cpp | 15 +++++++++------ dbms/src/Common/Config/ConfigProcessor.h | 6 +++++- dbms/src/Common/Config/ConfigReloader.cpp | 8 +++++--- dbms/src/Common/Config/ConfigReloader.h | 2 ++ .../src/Common/ZooKeeper/ZooKeeperNodeCache.cpp | 17 +++++++++++++---- dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h | 14 ++++++++------ 7 files changed, 46 insertions(+), 21 deletions(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index ba603114783..0e22d7dc814 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -114,12 +114,13 @@ int Server::main(const std::vector & /*args*/) bool has_zookeeper = config().has("zookeeper"); zkutil::ZooKeeperNodeCache main_config_zk_node_cache([&] { return global_context->getZooKeeper(); }); + zkutil::EventPtr main_config_zk_changed_event = std::make_shared(); if (loaded_config.has_zk_includes) { auto old_configuration = loaded_config.configuration; ConfigProcessor config_processor(config_path); loaded_config = config_processor.loadConfigWithZooKeeperIncludes( - main_config_zk_node_cache, /* fallback_to_preprocessed = */ true); + main_config_zk_node_cache, main_config_zk_changed_event, /* fallback_to_preprocessed = */ true); config_processor.savePreprocessedConfig(loaded_config); config().removeConfiguration(old_configuration.get()); config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false); @@ -267,6 +268,7 @@ int Server::main(const std::vector & /*args*/) auto main_config_reloader = std::make_unique(config_path, include_from_path, std::move(main_config_zk_node_cache), + main_config_zk_changed_event, [&](ConfigurationPtr config) { buildLoggers(*config); @@ -288,6 +290,7 @@ int Server::main(const std::vector & /*args*/) auto users_config_reloader = std::make_unique(users_config_path, include_from_path, zkutil::ZooKeeperNodeCache([&] { return global_context->getZooKeeper(); }), + std::make_shared(), [&](ConfigurationPtr config) { global_context->setUsersConfig(config); }, /* already_loaded = */ false); diff --git a/dbms/src/Common/Config/ConfigProcessor.cpp b/dbms/src/Common/Config/ConfigProcessor.cpp index 17507f3bedd..98790cf5a4f 100644 --- a/dbms/src/Common/Config/ConfigProcessor.cpp +++ b/dbms/src/Common/Config/ConfigProcessor.cpp @@ -234,6 +234,7 @@ void ConfigProcessor::doIncludesRecursive( XMLDocumentPtr include_from, Node * node, zkutil::ZooKeeperNodeCache * zk_node_cache, + const zkutil::EventPtr & zk_changed_event, std::unordered_set & contributing_zk_paths) { if (node->nodeType() == Node::TEXT_NODE) @@ -352,7 +353,7 @@ void ConfigProcessor::doIncludesRecursive( XMLDocumentPtr zk_document; auto get_zk_node = [&](const std::string & name) -> const Node * { - zkutil::ZooKeeperNodeCache::GetResult result = zk_node_cache->get(name); + zkutil::ZooKeeperNodeCache::GetResult result = zk_node_cache->get(name, zk_changed_event); if (!result.exists) return nullptr; @@ -383,13 +384,13 @@ void ConfigProcessor::doIncludesRecursive( } if (included_something) - doIncludesRecursive(config, include_from, node, zk_node_cache, contributing_zk_paths); + doIncludesRecursive(config, include_from, node, zk_node_cache, zk_changed_event, contributing_zk_paths); else { NodeListPtr children = node->childNodes(); Node * child = nullptr; for (size_t i = 0; (child = children->item(i)); ++i) - doIncludesRecursive(config, include_from, child, zk_node_cache, contributing_zk_paths); + doIncludesRecursive(config, include_from, child, zk_node_cache, zk_changed_event, contributing_zk_paths); } } @@ -433,7 +434,8 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string & XMLDocumentPtr ConfigProcessor::processConfig( bool * has_zk_includes, - zkutil::ZooKeeperNodeCache * zk_node_cache) + zkutil::ZooKeeperNodeCache * zk_node_cache, + const zkutil::EventPtr & zk_changed_event) { XMLDocumentPtr config = dom_parser.parse(path); @@ -476,7 +478,7 @@ XMLDocumentPtr ConfigProcessor::processConfig( include_from = dom_parser.parse(include_from_path); } - doIncludesRecursive(config, include_from, getRootNode(config.get()), zk_node_cache, contributing_zk_paths); + doIncludesRecursive(config, include_from, getRootNode(config.get()), zk_node_cache, zk_changed_event, contributing_zk_paths); } catch (Poco::Exception & e) { @@ -525,6 +527,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfig(bool allow_zk_includes ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( zkutil::ZooKeeperNodeCache & zk_node_cache, + const zkutil::EventPtr & zk_changed_event, bool fallback_to_preprocessed) { XMLDocumentPtr config_xml; @@ -532,7 +535,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( bool processed_successfully = false; try { - config_xml = processConfig(&has_zk_includes, &zk_node_cache); + config_xml = processConfig(&has_zk_includes, &zk_node_cache, zk_changed_event); processed_successfully = true; } catch (const Poco::Exception & ex) diff --git a/dbms/src/Common/Config/ConfigProcessor.h b/dbms/src/Common/Config/ConfigProcessor.h index 8663ecb682f..270ff5aef49 100644 --- a/dbms/src/Common/Config/ConfigProcessor.h +++ b/dbms/src/Common/Config/ConfigProcessor.h @@ -22,6 +22,7 @@ namespace zkutil { class ZooKeeperNodeCache; + using EventPtr = std::shared_ptr; } using ConfigurationPtr = Poco::AutoPtr; @@ -58,7 +59,8 @@ public: /// 5) (Yandex.Metrika-specific) Substitute "" with "layer number from the hostname". XMLDocumentPtr processConfig( bool * has_zk_includes = nullptr, - zkutil::ZooKeeperNodeCache * zk_node_cache = nullptr); + zkutil::ZooKeeperNodeCache * zk_node_cache = nullptr, + const zkutil::EventPtr & zk_changed_event = nullptr); /// loadConfig* functions apply processConfig and create Poco::Util::XMLConfiguration. @@ -83,6 +85,7 @@ public: /// processing, load the configuration from the preprocessed file. LoadedConfig loadConfigWithZooKeeperIncludes( zkutil::ZooKeeperNodeCache & zk_node_cache, + const zkutil::EventPtr & zk_changed_event, bool fallback_to_preprocessed = false); void savePreprocessedConfig(const LoadedConfig & loaded_config); @@ -125,5 +128,6 @@ private: XMLDocumentPtr include_from, Poco::XML::Node * node, zkutil::ZooKeeperNodeCache * zk_node_cache, + const zkutil::EventPtr & zk_changed_event, std::unordered_set & contributing_zk_paths); }; diff --git a/dbms/src/Common/Config/ConfigReloader.cpp b/dbms/src/Common/Config/ConfigReloader.cpp index f798569d0c0..2e793034f80 100644 --- a/dbms/src/Common/Config/ConfigReloader.cpp +++ b/dbms/src/Common/Config/ConfigReloader.cpp @@ -16,10 +16,12 @@ ConfigReloader::ConfigReloader( const std::string & path_, const std::string & include_from_path_, zkutil::ZooKeeperNodeCache && zk_node_cache_, + const zkutil::EventPtr & zk_changed_event_, Updater && updater_, bool already_loaded) : path(path_), include_from_path(include_from_path_) , zk_node_cache(std::move(zk_node_cache_)) + , zk_changed_event(zk_changed_event_) , updater(std::move(updater_)) { if (!already_loaded) @@ -38,7 +40,7 @@ ConfigReloader::~ConfigReloader() try { quit = true; - zk_node_cache.getChangedEvent().set(); + zk_changed_event->set(); if (thread.joinable()) thread.join(); @@ -58,7 +60,7 @@ void ConfigReloader::run() { try { - bool zk_changed = zk_node_cache.getChangedEvent().tryWait(std::chrono::milliseconds(reload_interval).count()); + bool zk_changed = zk_changed_event->tryWait(std::chrono::milliseconds(reload_interval).count()); if (quit) return; @@ -88,7 +90,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true); if (loaded_config.has_zk_includes) loaded_config = config_processor.loadConfigWithZooKeeperIncludes( - zk_node_cache, fallback_to_preprocessed); + zk_node_cache, zk_changed_event, fallback_to_preprocessed); } catch (...) { diff --git a/dbms/src/Common/Config/ConfigReloader.h b/dbms/src/Common/Config/ConfigReloader.h index 63ed18a9105..9cb400cf0f7 100644 --- a/dbms/src/Common/Config/ConfigReloader.h +++ b/dbms/src/Common/Config/ConfigReloader.h @@ -34,6 +34,7 @@ public: const std::string & path, const std::string & include_from_path, zkutil::ZooKeeperNodeCache && zk_node_cache, + const zkutil::EventPtr & zk_changed_event, Updater && updater, bool already_loaded); @@ -72,6 +73,7 @@ private: std::string include_from_path; FilesChangesTracker files; zkutil::ZooKeeperNodeCache zk_node_cache; + zkutil::EventPtr zk_changed_event = std::make_shared(); Updater updater; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp index 7b111e5c767..cb823d7cd29 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp @@ -9,7 +9,16 @@ ZooKeeperNodeCache::ZooKeeperNodeCache(GetZooKeeper get_zookeeper_) { } -ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path) +ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path, EventPtr watch_event) +{ + Coordination::WatchCallback watch_callback; + if (watch_event) + watch_callback = [watch_event](const Coordination::WatchResponse &) { watch_event->set(); }; + + return get(path, watch_callback); +} + +ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path, Coordination::WatchCallback caller_watch_callback) { zkutil::ZooKeeperPtr zookeeper; std::unordered_set invalidated_paths; @@ -38,7 +47,7 @@ ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path) if (cache_it != node_cache.end()) return cache_it->second; - auto watch_callback = [context=context](const Coordination::WatchResponse & response) + auto watch_callback = [=](const Coordination::WatchResponse & response) { if (!(response.type != Coordination::SESSION || response.state == Coordination::EXPIRED_SESSION)) return; @@ -56,8 +65,8 @@ ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path) changed = true; } } - if (changed) - context->changed_event.set(); + if (changed && caller_watch_callback) + caller_watch_callback(response); }; GetResult result; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h index 5ad9043ffbb..89edd485b88 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h @@ -22,8 +22,13 @@ namespace zkutil /// This class allows querying the contents of ZooKeeper nodes and caching the results. /// Watches are set for cached nodes and for nodes that were nonexistent at the time of query. -/// After a watch fires, a notification is generated for the change event. +/// After a watch fires, the callback or event that was passed by the user is notified. +/// /// NOTE: methods of this class are not thread-safe. +/// +/// Intended use case: if you need one thread to watch changes in several nodes. +/// If instead you use simple a watch event for this, watches will accumulate for nodes that do not change +/// or change rarely. class ZooKeeperNodeCache { public: @@ -39,17 +44,14 @@ public: Coordination::Stat stat; }; - GetResult get(const std::string & path); - - Poco::Event & getChangedEvent() { return context->changed_event; } + GetResult get(const std::string & path, EventPtr watch_event); + GetResult get(const std::string & path, Coordination::WatchCallback watch_callback); private: GetZooKeeper get_zookeeper; struct Context { - Poco::Event changed_event; - std::mutex mutex; zkutil::ZooKeeperPtr zookeeper; std::unordered_set invalidated_paths; From 4036ec83af32eb54bfee3b036ae73fa272837076 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 1 Nov 2018 16:30:38 +0300 Subject: [PATCH 11/69] write metadata znode in StorageReplicatedMergeTree::alter() [#CLICKHOUSE-3859] --- dbms/src/Parsers/formatAST.cpp | 7 + dbms/src/Parsers/formatAST.h | 1 + .../ReplicatedMergeTreeTableMetadata.cpp | 248 ++++++++++++ .../ReplicatedMergeTreeTableMetadata.h | 36 ++ .../Storages/StorageReplicatedMergeTree.cpp | 364 ++++++++---------- .../src/Storages/StorageReplicatedMergeTree.h | 2 + 6 files changed, 445 insertions(+), 213 deletions(-) create mode 100644 dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp create mode 100644 dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h diff --git a/dbms/src/Parsers/formatAST.cpp b/dbms/src/Parsers/formatAST.cpp index f083afcea34..fca5130cb89 100644 --- a/dbms/src/Parsers/formatAST.cpp +++ b/dbms/src/Parsers/formatAST.cpp @@ -13,4 +13,11 @@ void formatAST(const IAST & ast, std::ostream & s, bool hilite, bool one_line) ast.format(settings); } +String serializeAST(const IAST & ast, bool one_line) +{ + std::stringstream ss; + formatAST(ast, ss, false, one_line); + return ss.str(); +} + } diff --git a/dbms/src/Parsers/formatAST.h b/dbms/src/Parsers/formatAST.h index e5428ea7598..685c504514e 100644 --- a/dbms/src/Parsers/formatAST.h +++ b/dbms/src/Parsers/formatAST.h @@ -12,6 +12,7 @@ namespace DB */ void formatAST(const IAST & ast, std::ostream & s, bool hilite = true, bool one_line = false); +String serializeAST(const IAST & ast, bool one_line = true); inline std::ostream & operator<<(std::ostream & os, const IAST & ast) { diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp new file mode 100644 index 00000000000..285c62177d3 --- /dev/null +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -0,0 +1,248 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int METADATA_MISMATCH; +} + +ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTreeData & data_) + : data(data_) + , sorting_and_primary_keys_independent(data.sorting_and_primary_keys_independent) + , sorting_key_str(serializeAST(*data.sorting_key_ast)) +{} + +static String formattedAST(const ASTPtr & ast) +{ + if (!ast) + return ""; + std::stringstream ss; + formatAST(*ast, ss, false, true); + return ss.str(); +} + +void ReplicatedMergeTreeTableMetadata::write(WriteBuffer & out) const +{ + out << "metadata format version: 1" << "\n" + << "date column: "; + + if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + out << data.minmax_idx_columns[data.minmax_idx_date_column_pos] << "\n"; + else + out << "\n"; + + out << "sampling expression: " << formattedAST(data.sampling_expression) << "\n" + << "index granularity: " << data.index_granularity << "\n" + << "mode: " << static_cast(data.merging_params.mode) << "\n" + << "sign column: " << data.merging_params.sign_column << "\n" + << "primary key: " << formattedAST(data.primary_key_ast) << "\n"; + + if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + { + out << "data format version: " << data.format_version.toUnderType() << "\n"; + out << "partition key: " << formattedAST(data.partition_expr_ast) << "\n"; + } + + if (sorting_and_primary_keys_independent) + out << "sorting key: " << sorting_key_str << "\n"; +} + +String ReplicatedMergeTreeTableMetadata::toString() const +{ + WriteBufferFromOwnString out; + write(out); + return out.str(); +} + +void ReplicatedMergeTreeTableMetadata::read(ReadBuffer & in) +{ + in >> "metadata format version: 1\n"; + + in >> "date column: "; + String read_date_column; + in >> read_date_column >> "\n"; + + in >> "sampling expression: "; + String read_sample_expression; + in >> read_sample_expression >> "\n"; + + in >> "index granularity: "; + size_t read_index_granularity = 0; + in >> read_index_granularity >> "\n"; + + in >> "mode: "; + int read_mode = 0; + in >> read_mode >> "\n"; + + in >> "sign column: "; + String read_sign_column; + in >> read_sign_column >> "\n"; + + in >> "primary key: "; + String read_primary_key; + in >> read_primary_key >> "\n"; + + MergeTreeDataFormatVersion read_data_format_version; + if (in.eof()) + read_data_format_version = 0; + else + { + in >> "data format version: "; + in >> read_data_format_version.toUnderType() >> "\n"; + } + + if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + { + in >> "partition key: "; + String read_partition_key; + in >> read_partition_key >> "\n"; + } + + if (checkString("sorting key: ", in)) + { + String read_sorting_key; + in >> read_sorting_key >> "\n"; + sorting_key_str = read_sorting_key; + } + + assertEOF(in); +} + +ReplicatedMergeTreeTableMetadata ReplicatedMergeTreeTableMetadata::parse( + const MergeTreeData & data_, const String & s) +{ + ReplicatedMergeTreeTableMetadata metadata(data_); + ReadBufferFromString buf(s); + metadata.read(buf); + return metadata; +} + +void ReplicatedMergeTreeTableMetadata::check(ReadBuffer & in) const +{ + /// TODO Can be made less cumbersome. + + in >> "metadata format version: 1"; + + in >> "\ndate column: "; + String read_date_column; + in >> read_date_column; + + if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + { + const String & local_date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos]; + if (local_date_column != read_date_column) + throw Exception("Existing table metadata in ZooKeeper differs in date index column." + " Stored in ZooKeeper: " + read_date_column + ", local: " + local_date_column, + ErrorCodes::METADATA_MISMATCH); + } + else if (!read_date_column.empty()) + throw Exception( + "Existing table metadata in ZooKeeper differs in date index column." + " Stored in ZooKeeper: " + read_date_column + ", local is custom-partitioned.", + ErrorCodes::METADATA_MISMATCH); + + in >> "\nsampling expression: "; + String read_sample_expression; + String local_sample_expression = formattedAST(data.sampling_expression); + in >> read_sample_expression; + + if (read_sample_expression != local_sample_expression) + throw Exception("Existing table metadata in ZooKeeper differs in sample expression." + " Stored in ZooKeeper: " + read_sample_expression + ", local: " + local_sample_expression, + ErrorCodes::METADATA_MISMATCH); + + in >> "\nindex granularity: "; + size_t read_index_granularity = 0; + in >> read_index_granularity; + + if (read_index_granularity != data.index_granularity) + throw Exception("Existing table metadata in ZooKeeper differs in index granularity." + " Stored in ZooKeeper: " + DB::toString(read_index_granularity) + ", local: " + DB::toString(data.index_granularity), + ErrorCodes::METADATA_MISMATCH); + + in >> "\nmode: "; + int read_mode = 0; + in >> read_mode; + + if (read_mode != static_cast(data.merging_params.mode)) + throw Exception("Existing table metadata in ZooKeeper differs in mode of merge operation." + " Stored in ZooKeeper: " + DB::toString(read_mode) + ", local: " + + DB::toString(static_cast(data.merging_params.mode)), + ErrorCodes::METADATA_MISMATCH); + + in >> "\nsign column: "; + String read_sign_column; + in >> read_sign_column; + + if (read_sign_column != data.merging_params.sign_column) + throw Exception("Existing table metadata in ZooKeeper differs in sign column." + " Stored in ZooKeeper: " + read_sign_column + ", local: " + data.merging_params.sign_column, + ErrorCodes::METADATA_MISMATCH); + + in >> "\nprimary key: "; + String read_primary_key; + String local_primary_key = formattedAST(data.primary_key_ast); + in >> read_primary_key; + + /// NOTE: You can make a less strict check of match expressions so that tables do not break from small changes + /// in formatAST code. + if (read_primary_key != local_primary_key) + throw Exception("Existing table metadata in ZooKeeper differs in primary key." + " Stored in ZooKeeper: " + read_primary_key + ", local: " + local_primary_key, + ErrorCodes::METADATA_MISMATCH); + + in >> "\n"; + MergeTreeDataFormatVersion read_data_format_version; + if (in.eof()) + read_data_format_version = 0; + else + { + in >> "data format version: "; + in >> read_data_format_version.toUnderType(); + } + + if (read_data_format_version != data.format_version) + throw Exception("Existing table metadata in ZooKeeper differs in data format version." + " Stored in ZooKeeper: " + DB::toString(read_data_format_version.toUnderType()) + + ", local: " + DB::toString(data.format_version.toUnderType()), + ErrorCodes::METADATA_MISMATCH); + + if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + { + in >> "\npartition key: "; + String read_partition_key; + String local_partition_key = formattedAST(data.partition_expr_ast); + in >> read_partition_key; + + if (read_partition_key != local_partition_key) + throw Exception( + "Existing table metadata in ZooKeeper differs in partition key expression." + " Stored in ZooKeeper: " + read_partition_key + ", local: " + local_partition_key, + ErrorCodes::METADATA_MISMATCH); + + in >> "\n"; + } + + if (checkString("sorting key: ", in)) + { + String read_sorting_key; + in >> read_sorting_key >> "\n"; + } + + assertEOF(in); +} + +void ReplicatedMergeTreeTableMetadata::check(const String & s) const +{ + ReadBufferFromString in(s); + check(in); +} + +} diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h new file mode 100644 index 00000000000..edca8a2e607 --- /dev/null +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h @@ -0,0 +1,36 @@ +#pragma once + +#include + +namespace DB +{ + +class MergeTreeData; +class WriteBuffer; +class ReadBuffer; + +/** The basic parameters of ReplicatedMergeTree table engine for saving in ZooKeeper. + * Lets you verify that they match local ones. + */ +struct ReplicatedMergeTreeTableMetadata +{ + const MergeTreeData & data; + + bool sorting_and_primary_keys_independent = false; + String sorting_key_str; + + explicit ReplicatedMergeTreeTableMetadata(const MergeTreeData & data_); + + void write(WriteBuffer & out) const; + String toString() const; + + void read(ReadBuffer & in); + static ReplicatedMergeTreeTableMetadata parse(const MergeTreeData & data_, const String & s); + + void check(ReadBuffer & in) const; + void check(const String & s) const; + + /// TODO: checkAndFindDiff(other); +}; + +} diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index a3c2def0446..50784c1be4b 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -99,7 +100,6 @@ namespace ErrorCodes extern const int UNEXPECTED_FILE_IN_DATA_PART; extern const int NO_FILE_IN_DATA_PART; extern const int UNFINISHED; - extern const int METADATA_MISMATCH; extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS; extern const int TOO_MANY_FETCHES; extern const int BAD_DATA_PART_NAME; @@ -314,176 +314,9 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes() /// Mutations zookeeper->createIfNotExists(zookeeper_path + "/mutations", String()); zookeeper->createIfNotExists(replica_path + "/mutation_pointer", String()); -} - -static String formattedAST(const ASTPtr & ast) -{ - if (!ast) - return ""; - std::stringstream ss; - formatAST(*ast, ss, false, true); - return ss.str(); -} - - -namespace -{ - /** The basic parameters of table engine for saving in ZooKeeper. - * Lets you verify that they match local ones. - */ - struct TableMetadata - { - const MergeTreeData & data; - - explicit TableMetadata(const MergeTreeData & data_) - : data(data_) {} - - void write(WriteBuffer & out) const - { - out << "metadata format version: 1" << "\n" - << "date column: "; - - if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) - out << data.minmax_idx_columns[data.minmax_idx_date_column_pos] << "\n"; - else - out << "\n"; - - out << "sampling expression: " << formattedAST(data.sampling_expression) << "\n" - << "index granularity: " << data.index_granularity << "\n" - << "mode: " << static_cast(data.merging_params.mode) << "\n" - << "sign column: " << data.merging_params.sign_column << "\n" - << "primary key: " << formattedAST(data.primary_key_ast) << "\n"; - - if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) - { - out << "data format version: " << data.format_version.toUnderType() << "\n"; - out << "partition key: " << formattedAST(data.partition_expr_ast) << "\n"; - } - } - - String toString() const - { - WriteBufferFromOwnString out; - write(out); - return out.str(); - } - - void check(ReadBuffer & in) const - { - /// TODO Can be made less cumbersome. - - in >> "metadata format version: 1"; - - in >> "\ndate column: "; - String read_date_column; - in >> read_date_column; - - if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) - { - const String & local_date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos]; - if (local_date_column != read_date_column) - throw Exception("Existing table metadata in ZooKeeper differs in date index column." - " Stored in ZooKeeper: " + read_date_column + ", local: " + local_date_column, - ErrorCodes::METADATA_MISMATCH); - } - else if (!read_date_column.empty()) - throw Exception( - "Existing table metadata in ZooKeeper differs in date index column." - " Stored in ZooKeeper: " + read_date_column + ", local is custom-partitioned.", - ErrorCodes::METADATA_MISMATCH); - - in >> "\nsampling expression: "; - String read_sample_expression; - String local_sample_expression = formattedAST(data.sampling_expression); - in >> read_sample_expression; - - if (read_sample_expression != local_sample_expression) - throw Exception("Existing table metadata in ZooKeeper differs in sample expression." - " Stored in ZooKeeper: " + read_sample_expression + ", local: " + local_sample_expression, - ErrorCodes::METADATA_MISMATCH); - - in >> "\nindex granularity: "; - size_t read_index_granularity = 0; - in >> read_index_granularity; - - if (read_index_granularity != data.index_granularity) - throw Exception("Existing table metadata in ZooKeeper differs in index granularity." - " Stored in ZooKeeper: " + DB::toString(read_index_granularity) + ", local: " + DB::toString(data.index_granularity), - ErrorCodes::METADATA_MISMATCH); - - in >> "\nmode: "; - int read_mode = 0; - in >> read_mode; - - if (read_mode != static_cast(data.merging_params.mode)) - throw Exception("Existing table metadata in ZooKeeper differs in mode of merge operation." - " Stored in ZooKeeper: " + DB::toString(read_mode) + ", local: " - + DB::toString(static_cast(data.merging_params.mode)), - ErrorCodes::METADATA_MISMATCH); - - in >> "\nsign column: "; - String read_sign_column; - in >> read_sign_column; - - if (read_sign_column != data.merging_params.sign_column) - throw Exception("Existing table metadata in ZooKeeper differs in sign column." - " Stored in ZooKeeper: " + read_sign_column + ", local: " + data.merging_params.sign_column, - ErrorCodes::METADATA_MISMATCH); - - in >> "\nprimary key: "; - String read_primary_key; - String local_primary_key = formattedAST(data.primary_key_ast); - in >> read_primary_key; - - /// NOTE: You can make a less strict check of match expressions so that tables do not break from small changes - /// in formatAST code. - if (read_primary_key != local_primary_key) - throw Exception("Existing table metadata in ZooKeeper differs in primary key." - " Stored in ZooKeeper: " + read_primary_key + ", local: " + local_primary_key, - ErrorCodes::METADATA_MISMATCH); - - in >> "\n"; - MergeTreeDataFormatVersion read_data_format_version; - if (in.eof()) - read_data_format_version = 0; - else - { - in >> "data format version: "; - in >> read_data_format_version.toUnderType(); - } - - if (read_data_format_version != data.format_version) - throw Exception("Existing table metadata in ZooKeeper differs in data format version." - " Stored in ZooKeeper: " + DB::toString(read_data_format_version.toUnderType()) + - ", local: " + DB::toString(data.format_version.toUnderType()), - ErrorCodes::METADATA_MISMATCH); - - if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) - { - in >> "\npartition key: "; - String read_partition_key; - String local_partition_key = formattedAST(data.partition_expr_ast); - in >> read_partition_key; - - if (read_partition_key != local_partition_key) - throw Exception( - "Existing table metadata in ZooKeeper differs in partition key expression." - " Stored in ZooKeeper: " + read_partition_key + ", local: " + local_partition_key, - ErrorCodes::METADATA_MISMATCH); - - in >> "\n"; - } - - assertEOF(in); - } - - void check(const String & s) const - { - ReadBufferFromString in(s); - check(in); - } - }; + /// ALTERs of the metadata node. + zookeeper->createIfNotExists(replica_path + "/metadata", String()); } @@ -499,7 +332,7 @@ void StorageReplicatedMergeTree::createTableIfNotExists() zookeeper->createAncestors(zookeeper_path); /// We write metadata of table so that the replicas can check table parameters with them. - String metadata = TableMetadata(data).toString(); + String metadata = ReplicatedMergeTreeTableMetadata(data).toString(); Coordination::Requests ops; ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", @@ -537,12 +370,14 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo { auto zookeeper = getZooKeeper(); - String metadata_str = zookeeper->get(zookeeper_path + "/metadata"); - TableMetadata(data).check(metadata_str); + Coordination::Stat metadata_stat; + String metadata_str = zookeeper->get(zookeeper_path + "/metadata", &metadata_stat); + ReplicatedMergeTreeTableMetadata(data).check(metadata_str); + metadata_version = metadata_stat.version; - Coordination::Stat stat; - auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_path + "/columns", &stat)); - columns_version = stat.version; + Coordination::Stat columns_stat; + auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_path + "/columns", &columns_stat)); + columns_version = columns_stat.version; const ColumnsDescription & old_columns = getColumns(); if (columns_from_zk != old_columns) @@ -3090,9 +2925,33 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, LOG_DEBUG(log, "Doing ALTER"); - int new_columns_version = -1; /// Initialization is to suppress (useless) false positive warning found by cppcheck. - String new_columns_str; - Coordination::Stat stat; + /// Alter is done by modifying the metadata nodes in ZK that are shared between all replicas + /// (/columns, /metadata). We set contents of the shared nodes to the new values and wait while + /// replicas asynchronously apply changes (see ReplicatedMergeTreeAlterThread.cpp) and modify + /// their respective replica metadata nodes (/replicas//columns, /replicas//metadata). + + struct ChangedNode + { + ChangedNode(const String & table_path_, String name_, String new_value_) + : table_path(table_path_), name(std::move(name_)), shared_path(table_path + "/" + name) + , new_value(std::move(new_value_)) + {} + + const String & table_path; + String name; + + String shared_path; + + String getReplicaPath(const String & replica) const + { + return table_path + "/replicas/" + replica + "/" + name; + } + + String new_value; + int32_t new_version = -1; /// Initialization is to suppress (useless) false positive warning found by cppcheck. + }; + + std::vector changed_nodes; { /// Just to read current structure. Alter will be done in separate thread. @@ -3109,28 +2968,51 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, ColumnsDescription new_columns = data.getColumns(); params.apply(new_columns); + String new_columns_str = new_columns.toString(); + if (new_columns_str != data.getColumns().toString()) + changed_nodes.emplace_back(zookeeper_path, "columns", new_columns_str); - new_columns_str = new_columns.toString(); + ReplicatedMergeTreeTableMetadata new_metadata(data); + for (const AlterCommand & param : params) + { + if (param.type == AlterCommand::MODIFY_ORDER_BY) + { + new_metadata.sorting_and_primary_keys_independent = true; + new_metadata.sorting_key_str = serializeAST(*param.sorting_key); + } + } + String new_metadata_str = new_metadata.toString(); + if (new_metadata_str != ReplicatedMergeTreeTableMetadata(data).toString()) + changed_nodes.emplace_back(zookeeper_path, "metadata", new_metadata_str); - /// Do ALTER. - getZooKeeper()->set(zookeeper_path + "/columns", new_columns_str, -1, &stat); + /// Modify shared metadata nodes in ZooKeeper. + Coordination::Requests ops; + for (const auto & node : changed_nodes) + ops.emplace_back(zkutil::makeSetRequest(node.shared_path, node.new_value, -1)); - new_columns_version = stat.version; + Coordination::Responses results = getZooKeeper()->multi(ops); + + for (size_t i = 0; i < changed_nodes.size(); ++i) + changed_nodes[i].new_version = dynamic_cast(*results[i]).stat.version; } - LOG_DEBUG(log, "Updated columns in ZooKeeper. Waiting for replicas to apply changes."); + LOG_DEBUG(log, "Updated shared metadata nodes in ZooKeeper. Waiting for replicas to apply changes."); /// Wait until all replicas will apply ALTER. - /// Subscribe to change of columns, to finish waiting if someone will do another ALTER. - if (!getZooKeeper()->exists(zookeeper_path + "/columns", &stat, alter_query_event)) - throw Exception(zookeeper_path + "/columns doesn't exist", ErrorCodes::NOT_FOUND_NODE); - - if (stat.version != new_columns_version) + for (const auto & node : changed_nodes) { - LOG_WARNING(log, zookeeper_path + "/columns changed before this ALTER finished; " - "overlapping ALTER-s are fine but use caution with nontransitive changes"); - return; + Coordination::Stat stat; + /// Subscribe to change of shared ZK metadata nodes, to finish waiting if someone will do another ALTER. + if (!getZooKeeper()->exists(node.shared_path, &stat, alter_query_event)) + throw Exception(node.shared_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE); + + if (stat.version != node.new_version) + { + LOG_WARNING(log, node.shared_path + " changed before this ALTER finished; " + + "overlapping ALTER-s are fine but use caution with nontransitive changes"); + return; + } } Strings replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); @@ -3146,8 +3028,10 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, while (!partial_shutdown_called) { + auto zookeeper = getZooKeeper(); + /// Replica could be inactive. - if (!getZooKeeper()->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) { LOG_WARNING(log, "Replica " << replica << " is not active during ALTER query." " ALTER will be done asynchronously when replica becomes active."); @@ -3156,40 +3040,92 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, break; } - String replica_columns_str; - - /// Replica could has been removed. - if (!getZooKeeper()->tryGet(zookeeper_path + "/replicas/" + replica + "/columns", replica_columns_str, &stat)) + struct ReplicaNode { - LOG_WARNING(log, replica << " was removed"); - break; + explicit ReplicaNode(String path_) : path(std::move(path_)) {} + + String path; + String value; + int32_t version = -1; + }; + + std::vector replica_nodes; + for (const auto & node : changed_nodes) + replica_nodes.emplace_back(node.getReplicaPath(replica)); + + bool replica_was_removed = false; + for (auto & node : replica_nodes) + { + Coordination::Stat stat; + + /// Replica could has been removed. + if (!zookeeper->tryGet(node.path, node.value, &stat)) + { + LOG_WARNING(log, replica << " was removed"); + replica_was_removed = true; + break; + } + + node.version = stat.version; } - int replica_columns_version = stat.version; + if (replica_was_removed) + break; + + bool alter_was_applied = true; + for (size_t i = 0; i < replica_nodes.size(); ++i) + { + if (replica_nodes[i].value != changed_nodes[i].new_value) + { + alter_was_applied = false; + break; + } + } /// The ALTER has been successfully applied. - if (replica_columns_str == new_columns_str) + if (alter_was_applied) break; - if (!getZooKeeper()->exists(zookeeper_path + "/columns", &stat)) - throw Exception(zookeeper_path + "/columns doesn't exist", ErrorCodes::NOT_FOUND_NODE); - - if (stat.version != new_columns_version) + for (const auto & node : changed_nodes) { - LOG_WARNING(log, zookeeper_path + "/columns changed before ALTER finished; " - "overlapping ALTER-s are fine but use caution with nontransitive changes"); - return; + Coordination::Stat stat; + if (!zookeeper->exists(node.shared_path, &stat)) + throw Exception(node.shared_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE); + + if (stat.version != node.new_version) + { + LOG_WARNING(log, node.shared_path + " changed before this ALTER finished; " + "overlapping ALTER-s are fine but use caution with nontransitive changes"); + return; + } } - if (!getZooKeeper()->exists(zookeeper_path + "/replicas/" + replica + "/columns", &stat, alter_query_event)) + bool replica_nodes_changed_concurrently = false; + for (const auto & replica_node : replica_nodes) { - LOG_WARNING(log, replica << " was removed"); + Coordination::Stat stat; + if (!zookeeper->exists(replica_node.path, &stat, alter_query_event)) + { + LOG_WARNING(log, replica << " was removed"); + replica_was_removed = true; + break; + } + + if (stat.version != replica_node.version) + { + replica_nodes_changed_concurrently = true; + break; + } + } + + if (replica_was_removed) break; - } - if (stat.version != replica_columns_version) + if (replica_nodes_changed_concurrently) continue; + /// Now wait for replica nodes to change. + if (!replication_alter_columns_timeout) { alter_query_event->wait(); @@ -3844,7 +3780,9 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query leader_address.database, context.getClientInfo().current_user, context.getClientInfo().current_password, timeouts, "ClickHouse replica"); - RemoteBlockInputStream stream(connection, formattedAST(new_query), {}, context, &settings); + std::stringstream new_query_ss; + formatAST(*new_query, new_query_ss, false, true); + RemoteBlockInputStream stream(connection, new_query_ss.str(), {}, context, &settings); NullBlockOutputStream output({}); copyData(stream, output); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 0ba0e401a82..315ad8e3561 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -243,6 +243,8 @@ private: */ int columns_version = -1; + int metadata_version = -1; + /** Is this replica "leading". The leader replica selects the parts to merge. */ std::atomic is_leader {false}; From 106800b857947cca4cef574c2363f1e493264876 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 1 Nov 2018 16:37:36 +0300 Subject: [PATCH 12/69] apply changes in ReplicatedMergeTreeAlterThread [#CLICKHOUSE-3859] --- .../ReplicatedMergeTreeAlterThread.cpp | 264 +++++++++++------- .../ReplicatedMergeTreeAlterThread.h | 2 + 2 files changed, 163 insertions(+), 103 deletions(-) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp index 5436aab75a3..4fdec4b75cc 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp @@ -1,21 +1,29 @@ -#include +#include +#include +#include +#include #include #include #include -#include -#include -#include #include +#include + namespace DB { +namespace ErrorCodes +{ + extern const int NOT_FOUND_NODE; +} + static const auto ALTER_ERROR_SLEEP_MS = 10 * 1000; ReplicatedMergeTreeAlterThread::ReplicatedMergeTreeAlterThread(StorageReplicatedMergeTree & storage_) : storage(storage_) + , zk_node_cache([&] { return storage.getZooKeeper(); }) , log_name(storage.database_name + "." + storage.table_name + " (ReplicatedMergeTreeAlterThread)") , log(&Logger::get(log_name)) { @@ -53,136 +61,186 @@ void ReplicatedMergeTreeAlterThread::run() auto zookeeper = storage.getZooKeeper(); - Coordination::Stat stat; - const String columns_str = zookeeper->getWatch(storage.zookeeper_path + "/columns", &stat, task->getWatchCallback()); + String columns_path = storage.zookeeper_path + "/columns"; + auto columns_result = zk_node_cache.get(columns_path, task->getWatchCallback()); + if (!columns_result.exists) + throw Exception(columns_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE); + int32_t columns_version = columns_result.stat.version; + + String metadata_path = storage.zookeeper_path + "/metadata"; + auto metadata_result = zk_node_cache.get(metadata_path, task->getWatchCallback()); + if (!metadata_result.exists) + throw Exception(metadata_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE); + int32_t metadata_version = metadata_result.stat.version; + + const bool changed_columns_version = (columns_version != storage.columns_version); + const bool changed_metadata_version = (metadata_version != storage.metadata_version); + + if (!(changed_columns_version || changed_metadata_version || force_recheck_parts)) + return; + + const String & columns_str = columns_result.contents; auto columns_in_zk = ColumnsDescription::parse(columns_str); - bool changed_version = (stat.version != storage.columns_version); + const String & metadata_str = metadata_result.contents; + auto metadata_in_zk = ReplicatedMergeTreeTableMetadata::parse(storage.data, metadata_str); + /// If you need to lock table structure, then suspend merges. + ActionLock merge_blocker = storage.merger_mutator.actions_blocker.cancel(); + + MergeTreeData::DataParts parts; + + /// If metadata nodes have changed, we will update table structure locally. + if (changed_columns_version || changed_metadata_version) { - /// If you need to lock table structure, then suspend merges. - ActionLock merge_blocker; + /// Temporarily cancel part checks to avoid locking for long time. + auto temporarily_stop_part_checks = storage.part_check_thread.temporarilyStop(); - if (changed_version || force_recheck_parts) - merge_blocker = storage.merger_mutator.actions_blocker.cancel(); + /// Temporarily cancel parts sending + ActionLock data_parts_exchange_blocker; + if (storage.data_parts_exchange_endpoint_holder) + data_parts_exchange_blocker = storage.data_parts_exchange_endpoint_holder->getBlocker().cancel(); - MergeTreeData::DataParts parts; + /// Temporarily cancel part fetches + auto fetches_blocker = storage.fetcher.blocker.cancel(); - /// If columns description has changed, we will update table structure locally. - if (changed_version) + LOG_INFO(log, "Version of metadata nodes in ZooKeeper changed. Waiting for structure write lock."); + + auto table_lock = storage.lockStructureForAlter(__PRETTY_FUNCTION__); + + if (columns_in_zk == storage.getColumns() + && ReplicatedMergeTreeTableMetadata(storage.data).sorting_key_str == metadata_in_zk.sorting_key_str) { - /// Temporarily cancel part checks to avoid locking for long time. - auto temporarily_stop_part_checks = storage.part_check_thread.temporarilyStop(); + LOG_INFO(log, "Metadata nodes changed in ZooKeeper, but their contents didn't change. " + "Most probably it is a cyclic ALTER."); + } + else + { + LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally."); - /// Temporarily cancel parts sending - ActionLock data_parts_exchange_blocker; - if (storage.data_parts_exchange_endpoint_holder) - data_parts_exchange_blocker = storage.data_parts_exchange_endpoint_holder->getBlocker().cancel(); + /// Note: setting columns first so that the new sorting key can use new columns. + storage.setColumns(std::move(columns_in_zk)); - /// Temporarily cancel part fetches - auto fetches_blocker = storage.fetcher.blocker.cancel(); - - LOG_INFO(log, "Changed version of 'columns' node in ZooKeeper. Waiting for structure write lock."); - - auto table_lock = storage.lockStructureForAlter(__PRETTY_FUNCTION__); - - if (columns_in_zk != storage.getColumns()) + ASTPtr new_sorting_key_ast = storage.data.sorting_key_ast; + IDatabase::ASTModifier storage_modifier; + if (ReplicatedMergeTreeTableMetadata(storage.data).sorting_key_str != metadata_in_zk.sorting_key_str) { - LOG_INFO(log, "Columns list changed in ZooKeeper. Applying changes locally."); + ParserNotEmptyExpressionList parser(false); + new_sorting_key_ast = parseQuery(parser, metadata_in_zk.sorting_key_str, 0); - storage.context.getDatabase(storage.database_name)->alterTable( - storage.context, storage.table_name, columns_in_zk, {}); - storage.setColumns(std::move(columns_in_zk)); + storage_modifier = [&](IAST & ast) + { + auto & storage_ast = typeid_cast(ast); - /// Reinitialize primary key because primary key column types might have changed. - storage.data.setPrimaryKey(storage.data.primary_key_ast, storage.data.sorting_key_ast); + auto tuple = std::make_shared(); + tuple->name = "tuple"; + tuple->arguments = new_sorting_key_ast; + tuple->children.push_back(tuple->arguments); - LOG_INFO(log, "Applied changes to table."); - } - else - { - LOG_INFO(log, "Columns version changed in ZooKeeper, but data wasn't changed. It's like cyclic ALTERs."); + if (!storage_ast.order_by) + throw Exception("Not supported", ErrorCodes::LOGICAL_ERROR); /// TODO: better exception message + + if (!storage_ast.primary_key) + { + /// Primary and sorting key become independent after this ALTER so we have to + /// save the old ORDER BY expression as the new primary key. + storage_ast.set(storage_ast.primary_key, storage_ast.order_by->clone()); + } + + storage_ast.set(storage_ast.order_by, tuple); + }; } - /// You need to get a list of parts under table lock to avoid race condition with merge. + /// Even if the primary/sorting keys didn't change we must reinitialize it + /// because primary key column types might have changed. + storage.data.setPrimaryKey(storage.data.primary_key_ast, new_sorting_key_ast); + + storage.context.getDatabase(storage.database_name)->alterTable( + storage.context, storage.table_name, storage.getColumns(), storage_modifier); + + LOG_INFO(log, "Applied changes to the metadata of the table."); + } + + /// You need to get a list of parts under table lock to avoid race condition with merge. + parts = storage.data.getDataParts(); + + storage.columns_version = columns_version; + storage.metadata_version = metadata_version; + } + + /// Update parts. + if (changed_columns_version || force_recheck_parts) + { + auto table_lock = storage.lockStructure(false, __PRETTY_FUNCTION__); + + if (changed_columns_version) + LOG_INFO(log, "ALTER-ing parts"); + + int changed_parts = 0; + + if (!changed_columns_version) parts = storage.data.getDataParts(); - storage.columns_version = stat.version; - } + const auto columns_for_parts = storage.getColumns().getAllPhysical(); - /// Update parts. - if (changed_version || force_recheck_parts) + for (const MergeTreeData::DataPartPtr & part : parts) { - auto table_lock = storage.lockStructure(false, __PRETTY_FUNCTION__); + /// Update the part and write result to temporary files. + /// TODO: You can skip checking for too large changes if ZooKeeper has, for example, + /// node /flags/force_alter. + auto transaction = storage.data.alterDataPart( + part, columns_for_parts, storage.data.primary_key_ast, false); - if (changed_version) - LOG_INFO(log, "ALTER-ing parts"); + if (!transaction) + continue; - int changed_parts = 0; + ++changed_parts; - if (!changed_version) - parts = storage.data.getDataParts(); + /// Update part metadata in ZooKeeper. + Coordination::Requests ops; + ops.emplace_back(zkutil::makeSetRequest( + storage.replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1)); + ops.emplace_back(zkutil::makeSetRequest( + storage.replica_path + "/parts/" + part->name + "/checksums", + storage.getChecksumsForZooKeeper(transaction->getNewChecksums()), + -1)); - const auto columns_for_parts = storage.getColumns().getAllPhysical(); - - for (const MergeTreeData::DataPartPtr & part : parts) + try { - /// Update the part and write result to temporary files. - /// TODO: You can skip checking for too large changes if ZooKeeper has, for example, - /// node /flags/force_alter. - auto transaction = storage.data.alterDataPart( - part, columns_for_parts, storage.data.primary_key_ast, false); + zookeeper->multi(ops); + } + catch (const Coordination::Exception & e) + { + /// The part does not exist in ZK. We will add to queue for verification - maybe the part is superfluous, and it must be removed locally. + if (e.code == Coordination::ZNONODE) + storage.enqueuePartForCheck(part->name); - if (!transaction) - continue; - - ++changed_parts; - - /// Update part metadata in ZooKeeper. - Coordination::Requests ops; - ops.emplace_back(zkutil::makeSetRequest( - storage.replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1)); - ops.emplace_back(zkutil::makeSetRequest( - storage.replica_path + "/parts/" + part->name + "/checksums", - storage.getChecksumsForZooKeeper(transaction->getNewChecksums()), - -1)); - - try - { - zookeeper->multi(ops); - } - catch (const Coordination::Exception & e) - { - /// The part does not exist in ZK. We will add to queue for verification - maybe the part is superfluous, and it must be removed locally. - if (e.code == Coordination::ZNONODE) - storage.enqueuePartForCheck(part->name); - - throw; - } - - /// Apply file changes. - transaction->commit(); + throw; } - /// Columns sizes could be quietly changed in case of MODIFY/ADD COLUMN - storage.data.recalculateColumnSizes(); - - /// List of columns for a specific replica. - zookeeper->set(storage.replica_path + "/columns", columns_str); - - if (changed_version) - { - if (changed_parts != 0) - LOG_INFO(log, "ALTER-ed " << changed_parts << " parts"); - else - LOG_INFO(log, "No parts ALTER-ed"); - } - - force_recheck_parts = false; + /// Apply file changes. + transaction->commit(); } - /// It's important that parts and merge_blocker are destroyed before the wait. + /// Columns sizes could be quietly changed in case of MODIFY/ADD COLUMN + storage.data.recalculateColumnSizes(); + + if (changed_columns_version) + { + if (changed_parts != 0) + LOG_INFO(log, "ALTER-ed " << changed_parts << " parts"); + else + LOG_INFO(log, "No parts ALTER-ed"); + } } + + /// Update metadata ZK nodes for a specific replica. + if (changed_columns_version || force_recheck_parts) + zookeeper->set(storage.replica_path + "/columns", columns_str); + if (changed_metadata_version || force_recheck_parts) + zookeeper->set(storage.replica_path + "/metadata", metadata_str); + + force_recheck_parts = false; } catch (const Coordination::Exception & e) { diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.h index 44659ea03ac..bb93d06bf6a 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -31,6 +32,7 @@ private: void run(); StorageReplicatedMergeTree & storage; + zkutil::ZooKeeperNodeCache zk_node_cache; String log_name; Logger * log; BackgroundSchedulePool::TaskHolder task; From 96d7eee785bbee7a363c2a4d2ba62e65d2f13886 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 1 Nov 2018 18:15:19 +0300 Subject: [PATCH 13/69] fix use-after-free in ZooKeeperNodeCache [#CLICKHOUSE-3859] --- dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp index cb823d7cd29..f44e897cad9 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp @@ -47,21 +47,26 @@ ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path, if (cache_it != node_cache.end()) return cache_it->second; - auto watch_callback = [=](const Coordination::WatchResponse & response) + std::weak_ptr weak_context(context); + auto watch_callback = [weak_context, caller_watch_callback](const Coordination::WatchResponse & response) { if (!(response.type != Coordination::SESSION || response.state == Coordination::EXPIRED_SESSION)) return; + auto owned_context = weak_context.lock(); + if (!owned_context) + return; + bool changed = false; { - std::lock_guard lock(context->mutex); + std::lock_guard lock(owned_context->mutex); if (response.type != Coordination::SESSION) - changed = context->invalidated_paths.emplace(response.path).second; + changed = owned_context->invalidated_paths.emplace(response.path).second; else if (response.state == Coordination::EXPIRED_SESSION) { - context->zookeeper = nullptr; - context->invalidated_paths.clear(); + owned_context->zookeeper = nullptr; + owned_context->invalidated_paths.clear(); changed = true; } } From 74c48ed226127dfdfc1194fb889f24e540b2cb41 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 2 Nov 2018 14:53:05 +0300 Subject: [PATCH 14/69] apply metadata changes on startup [#CLICKHOUSE-3859] --- .../ReplicatedMergeTreeAlterThread.cpp | 40 +------------- .../Storages/StorageReplicatedMergeTree.cpp | 55 +++++++++++++++++-- .../src/Storages/StorageReplicatedMergeTree.h | 6 ++ 3 files changed, 57 insertions(+), 44 deletions(-) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp index 4fdec4b75cc..923935bcffe 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp @@ -118,45 +118,7 @@ void ReplicatedMergeTreeAlterThread::run() { LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally."); - /// Note: setting columns first so that the new sorting key can use new columns. - storage.setColumns(std::move(columns_in_zk)); - - ASTPtr new_sorting_key_ast = storage.data.sorting_key_ast; - IDatabase::ASTModifier storage_modifier; - if (ReplicatedMergeTreeTableMetadata(storage.data).sorting_key_str != metadata_in_zk.sorting_key_str) - { - ParserNotEmptyExpressionList parser(false); - new_sorting_key_ast = parseQuery(parser, metadata_in_zk.sorting_key_str, 0); - - storage_modifier = [&](IAST & ast) - { - auto & storage_ast = typeid_cast(ast); - - auto tuple = std::make_shared(); - tuple->name = "tuple"; - tuple->arguments = new_sorting_key_ast; - tuple->children.push_back(tuple->arguments); - - if (!storage_ast.order_by) - throw Exception("Not supported", ErrorCodes::LOGICAL_ERROR); /// TODO: better exception message - - if (!storage_ast.primary_key) - { - /// Primary and sorting key become independent after this ALTER so we have to - /// save the old ORDER BY expression as the new primary key. - storage_ast.set(storage_ast.primary_key, storage_ast.order_by->clone()); - } - - storage_ast.set(storage_ast.order_by, tuple); - }; - } - - /// Even if the primary/sorting keys didn't change we must reinitialize it - /// because primary key column types might have changed. - storage.data.setPrimaryKey(storage.data.primary_key_ast, new_sorting_key_ast); - - storage.context.getDatabase(storage.database_name)->alterTable( - storage.context, storage.table_name, storage.getColumns(), storage_modifier); + storage.setTableStructure(std::move(columns_in_zk), metadata_in_zk); LOG_INFO(log, "Applied changes to the metadata of the table."); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 50784c1be4b..9f5734887f7 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -370,9 +370,13 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo { auto zookeeper = getZooKeeper(); + ReplicatedMergeTreeTableMetadata old_metadata(data); + Coordination::Stat metadata_stat; String metadata_str = zookeeper->get(zookeeper_path + "/metadata", &metadata_stat); - ReplicatedMergeTreeTableMetadata(data).check(metadata_str); + auto metadata_from_zk = ReplicatedMergeTreeTableMetadata::parse(data, metadata_str); + old_metadata.check(metadata_str); + const bool metadata_changed = metadata_str != old_metadata.toString(); metadata_version = metadata_stat.version; Coordination::Stat columns_stat; @@ -380,7 +384,7 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo columns_version = columns_stat.version; const ColumnsDescription & old_columns = getColumns(); - if (columns_from_zk != old_columns) + if (columns_from_zk != old_columns || metadata_changed) { if (allow_alter && (skip_sanity_checks || @@ -390,9 +394,7 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo LOG_WARNING(log, "Table structure in ZooKeeper is a little different from local table structure. Assuming ALTER."); /// Without any locks, because table has not been created yet. - context.getDatabase(database_name)->alterTable(context, table_name, columns_from_zk, {}); - - setColumns(std::move(columns_from_zk)); + setTableStructure(std::move(columns_from_zk), metadata_from_zk); } else { @@ -403,6 +405,49 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo } +void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata & new_metadata) +{ + /// Note: setting columns first so that the new sorting key can use new columns. + setColumns(std::move(new_columns)); + + ASTPtr new_sorting_key_ast = data.sorting_key_ast; + IDatabase::ASTModifier storage_modifier; + if (ReplicatedMergeTreeTableMetadata(data).sorting_key_str != new_metadata.sorting_key_str) + { + ParserNotEmptyExpressionList parser(false); + new_sorting_key_ast = parseQuery(parser, new_metadata.sorting_key_str, 0); + + storage_modifier = [&](IAST & ast) + { + auto & storage_ast = typeid_cast(ast); + + auto tuple = std::make_shared(); + tuple->name = "tuple"; + tuple->arguments = new_sorting_key_ast; + tuple->children.push_back(tuple->arguments); + + if (!storage_ast.order_by) + throw Exception("Not supported", ErrorCodes::LOGICAL_ERROR); /// TODO: better exception message + + if (!storage_ast.primary_key) + { + /// Primary and sorting key become independent after this ALTER so we have to + /// save the old ORDER BY expression as the new primary key. + storage_ast.set(storage_ast.primary_key, storage_ast.order_by->clone()); + } + + storage_ast.set(storage_ast.order_by, tuple); + }; + } + + /// Even if the primary/sorting keys didn't change we must reinitialize it + /// because primary key column types might have changed. + data.setPrimaryKey(data.primary_key_ast, new_sorting_key_ast); + + context.getDatabase(database_name)->alterTable(context, table_name, getColumns(), storage_modifier); +} + + /** If necessary, restore a part, replica itself adds a record for its receipt. * What time should I put for this entry in the queue? Time is taken into account when calculating lag of replica. * For these purposes, it makes sense to use creation time of missing part diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 315ad8e3561..27e5599d89f 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -29,6 +29,8 @@ namespace DB { +class ReplicatedMergeTreeTableMetadata; + /** The engine that uses the merge tree (see MergeTreeData) and replicated through ZooKeeper. * * ZooKeeper is used for the following things: @@ -332,6 +334,10 @@ private: */ void checkTableStructure(bool skip_sanity_checks, bool allow_alter); + /// A part of ALTER: apply metadata changes only (data parts are altered separately). + /// Must be called under IStorage::lockStructureForAlter() lock. + void setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata & new_metadata); + /** Check that the set of parts corresponds to that in ZK (/replicas/me/parts/). * If any parts described in ZK are not locally, throw an exception. * If any local parts are not mentioned in ZK, remove them. From 31a6a6b4fdd3a246b1ba1e5e1fccc3eb3f84ac06 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 2 Nov 2018 18:39:19 +0300 Subject: [PATCH 15/69] calculate ReplicatedMergeTreeTableMetadata diff [#CLICKHOUSE-3859] --- .../ReplicatedMergeTreeAlterThread.cpp | 8 +- .../ReplicatedMergeTreeTableMetadata.cpp | 234 ++++++------------ .../ReplicatedMergeTreeTableMetadata.h | 31 ++- .../Storages/StorageReplicatedMergeTree.cpp | 27 +- .../src/Storages/StorageReplicatedMergeTree.h | 5 +- 5 files changed, 123 insertions(+), 182 deletions(-) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp index 923935bcffe..7886965232e 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp @@ -83,7 +83,8 @@ void ReplicatedMergeTreeAlterThread::run() auto columns_in_zk = ColumnsDescription::parse(columns_str); const String & metadata_str = metadata_result.contents; - auto metadata_in_zk = ReplicatedMergeTreeTableMetadata::parse(storage.data, metadata_str); + auto metadata_in_zk = ReplicatedMergeTreeTableMetadata::parse(metadata_str); + auto metadata_diff = ReplicatedMergeTreeTableMetadata(storage.data).checkAndFindDiff(metadata_in_zk, /* allow_alter = */ true); /// If you need to lock table structure, then suspend merges. ActionLock merge_blocker = storage.merger_mutator.actions_blocker.cancel(); @@ -108,8 +109,7 @@ void ReplicatedMergeTreeAlterThread::run() auto table_lock = storage.lockStructureForAlter(__PRETTY_FUNCTION__); - if (columns_in_zk == storage.getColumns() - && ReplicatedMergeTreeTableMetadata(storage.data).sorting_key_str == metadata_in_zk.sorting_key_str) + if (columns_in_zk == storage.getColumns() && metadata_diff.empty()) { LOG_INFO(log, "Metadata nodes changed in ZooKeeper, but their contents didn't change. " "Most probably it is a cyclic ALTER."); @@ -118,7 +118,7 @@ void ReplicatedMergeTreeAlterThread::run() { LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally."); - storage.setTableStructure(std::move(columns_in_zk), metadata_in_zk); + storage.setTableStructure(std::move(columns_in_zk), metadata_diff); LOG_INFO(log, "Applied changes to the metadata of the table."); } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 285c62177d3..268fa53ea57 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -13,12 +13,6 @@ namespace ErrorCodes extern const int METADATA_MISMATCH; } -ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTreeData & data_) - : data(data_) - , sorting_and_primary_keys_independent(data.sorting_and_primary_keys_independent) - , sorting_key_str(serializeAST(*data.sorting_key_ast)) -{} - static String formattedAST(const ASTPtr & ast) { if (!ast) @@ -28,30 +22,43 @@ static String formattedAST(const ASTPtr & ast) return ss.str(); } +ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTreeData & data) +{ + if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos]; + + sampling_expression = formattedAST(data.sampling_expression); + index_granularity = data.index_granularity; + merging_params_mode = static_cast(data.merging_params.mode); + sign_column = data.merging_params.sign_column; + primary_key = formattedAST(data.primary_key_ast); + data_format_version = data.format_version; + + if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + partition_key = formattedAST(data.partition_expr_ast); + + if (data.sorting_and_primary_keys_independent) + sorting_key = formattedAST(data.sorting_key_ast); +} + void ReplicatedMergeTreeTableMetadata::write(WriteBuffer & out) const { out << "metadata format version: 1" << "\n" - << "date column: "; + << "date column: " << date_column << "\n" + << "sampling expression: " << sampling_expression << "\n" + << "index granularity: " << index_granularity << "\n" + << "mode: " << merging_params_mode << "\n" + << "sign column: " << sign_column << "\n" + << "primary key: " << primary_key << "\n"; - if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) - out << data.minmax_idx_columns[data.minmax_idx_date_column_pos] << "\n"; - else - out << "\n"; - - out << "sampling expression: " << formattedAST(data.sampling_expression) << "\n" - << "index granularity: " << data.index_granularity << "\n" - << "mode: " << static_cast(data.merging_params.mode) << "\n" - << "sign column: " << data.merging_params.sign_column << "\n" - << "primary key: " << formattedAST(data.primary_key_ast) << "\n"; - - if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + if (data_format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { - out << "data format version: " << data.format_version.toUnderType() << "\n"; - out << "partition key: " << formattedAST(data.partition_expr_ast) << "\n"; + out << "data format version: " << data_format_version.toUnderType() << "\n" + << "partition key: " << partition_key << "\n"; } - if (sorting_and_primary_keys_independent) - out << "sorting key: " << sorting_key_str << "\n"; + if (!sorting_key.empty()) + out << "sorting key: " << sorting_key << "\n"; } String ReplicatedMergeTreeTableMetadata::toString() const @@ -64,185 +71,106 @@ String ReplicatedMergeTreeTableMetadata::toString() const void ReplicatedMergeTreeTableMetadata::read(ReadBuffer & in) { in >> "metadata format version: 1\n"; + in >> "date column: " >> date_column >> "\n"; + in >> "sampling expression: " >> sampling_expression >> "\n"; + in >> "index granularity: " >> index_granularity >> "\n"; + in >> "mode: " >> merging_params_mode >> "\n"; + in >> "sign column: " >> sign_column >> "\n"; + in >> "primary key: " >> primary_key >> "\n"; - in >> "date column: "; - String read_date_column; - in >> read_date_column >> "\n"; - - in >> "sampling expression: "; - String read_sample_expression; - in >> read_sample_expression >> "\n"; - - in >> "index granularity: "; - size_t read_index_granularity = 0; - in >> read_index_granularity >> "\n"; - - in >> "mode: "; - int read_mode = 0; - in >> read_mode >> "\n"; - - in >> "sign column: "; - String read_sign_column; - in >> read_sign_column >> "\n"; - - in >> "primary key: "; - String read_primary_key; - in >> read_primary_key >> "\n"; - - MergeTreeDataFormatVersion read_data_format_version; if (in.eof()) - read_data_format_version = 0; + data_format_version = 0; else - { - in >> "data format version: "; - in >> read_data_format_version.toUnderType() >> "\n"; - } + in >> "data format version: " >> data_format_version.toUnderType() >> "\n"; - if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) - { - in >> "partition key: "; - String read_partition_key; - in >> read_partition_key >> "\n"; - } + if (data_format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + in >> "partition key: " >> partition_key >> "\n"; if (checkString("sorting key: ", in)) - { - String read_sorting_key; - in >> read_sorting_key >> "\n"; - sorting_key_str = read_sorting_key; - } - - assertEOF(in); + in >> sorting_key >> "\n"; } -ReplicatedMergeTreeTableMetadata ReplicatedMergeTreeTableMetadata::parse( - const MergeTreeData & data_, const String & s) +ReplicatedMergeTreeTableMetadata ReplicatedMergeTreeTableMetadata::parse(const String & s) { - ReplicatedMergeTreeTableMetadata metadata(data_); + ReplicatedMergeTreeTableMetadata metadata; ReadBufferFromString buf(s); metadata.read(buf); return metadata; } -void ReplicatedMergeTreeTableMetadata::check(ReadBuffer & in) const +ReplicatedMergeTreeTableMetadata::Diff +ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTableMetadata & from_zk, bool allow_alter) const { - /// TODO Can be made less cumbersome. + Diff diff; - in >> "metadata format version: 1"; - - in >> "\ndate column: "; - String read_date_column; - in >> read_date_column; - - if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + if (data_format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { - const String & local_date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos]; - if (local_date_column != read_date_column) + if (date_column != from_zk.date_column) throw Exception("Existing table metadata in ZooKeeper differs in date index column." - " Stored in ZooKeeper: " + read_date_column + ", local: " + local_date_column, + " Stored in ZooKeeper: " + from_zk.date_column + ", local: " + date_column, ErrorCodes::METADATA_MISMATCH); } - else if (!read_date_column.empty()) + else if (!from_zk.date_column.empty()) throw Exception( "Existing table metadata in ZooKeeper differs in date index column." - " Stored in ZooKeeper: " + read_date_column + ", local is custom-partitioned.", + " Stored in ZooKeeper: " + from_zk.date_column + ", local is custom-partitioned.", ErrorCodes::METADATA_MISMATCH); - in >> "\nsampling expression: "; - String read_sample_expression; - String local_sample_expression = formattedAST(data.sampling_expression); - in >> read_sample_expression; - - if (read_sample_expression != local_sample_expression) + if (sampling_expression != from_zk.sampling_expression) throw Exception("Existing table metadata in ZooKeeper differs in sample expression." - " Stored in ZooKeeper: " + read_sample_expression + ", local: " + local_sample_expression, + " Stored in ZooKeeper: " + from_zk.sampling_expression + ", local: " + sampling_expression, ErrorCodes::METADATA_MISMATCH); - in >> "\nindex granularity: "; - size_t read_index_granularity = 0; - in >> read_index_granularity; - - if (read_index_granularity != data.index_granularity) + if (index_granularity != from_zk.index_granularity) throw Exception("Existing table metadata in ZooKeeper differs in index granularity." - " Stored in ZooKeeper: " + DB::toString(read_index_granularity) + ", local: " + DB::toString(data.index_granularity), + " Stored in ZooKeeper: " + DB::toString(from_zk.index_granularity) + ", local: " + DB::toString(index_granularity), ErrorCodes::METADATA_MISMATCH); - in >> "\nmode: "; - int read_mode = 0; - in >> read_mode; - - if (read_mode != static_cast(data.merging_params.mode)) + if (merging_params_mode != from_zk.merging_params_mode) throw Exception("Existing table metadata in ZooKeeper differs in mode of merge operation." - " Stored in ZooKeeper: " + DB::toString(read_mode) + ", local: " - + DB::toString(static_cast(data.merging_params.mode)), + " Stored in ZooKeeper: " + DB::toString(from_zk.merging_params_mode) + ", local: " + + DB::toString(merging_params_mode), ErrorCodes::METADATA_MISMATCH); - in >> "\nsign column: "; - String read_sign_column; - in >> read_sign_column; - - if (read_sign_column != data.merging_params.sign_column) + if (sign_column != from_zk.sign_column) throw Exception("Existing table metadata in ZooKeeper differs in sign column." - " Stored in ZooKeeper: " + read_sign_column + ", local: " + data.merging_params.sign_column, + " Stored in ZooKeeper: " + from_zk.sign_column + ", local: " + sign_column, ErrorCodes::METADATA_MISMATCH); - in >> "\nprimary key: "; - String read_primary_key; - String local_primary_key = formattedAST(data.primary_key_ast); - in >> read_primary_key; - /// NOTE: You can make a less strict check of match expressions so that tables do not break from small changes /// in formatAST code. - if (read_primary_key != local_primary_key) + if (primary_key != from_zk.primary_key) throw Exception("Existing table metadata in ZooKeeper differs in primary key." - " Stored in ZooKeeper: " + read_primary_key + ", local: " + local_primary_key, + " Stored in ZooKeeper: " + from_zk.primary_key + ", local: " + primary_key, ErrorCodes::METADATA_MISMATCH); - in >> "\n"; - MergeTreeDataFormatVersion read_data_format_version; - if (in.eof()) - read_data_format_version = 0; - else - { - in >> "data format version: "; - in >> read_data_format_version.toUnderType(); - } - - if (read_data_format_version != data.format_version) + if (data_format_version != from_zk.data_format_version) throw Exception("Existing table metadata in ZooKeeper differs in data format version." - " Stored in ZooKeeper: " + DB::toString(read_data_format_version.toUnderType()) + - ", local: " + DB::toString(data.format_version.toUnderType()), + " Stored in ZooKeeper: " + DB::toString(from_zk.data_format_version.toUnderType()) + + ", local: " + DB::toString(data_format_version.toUnderType()), ErrorCodes::METADATA_MISMATCH); - if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) - { - in >> "\npartition key: "; - String read_partition_key; - String local_partition_key = formattedAST(data.partition_expr_ast); - in >> read_partition_key; + if (partition_key != from_zk.partition_key) + throw Exception( + "Existing table metadata in ZooKeeper differs in partition key expression." + " Stored in ZooKeeper: " + from_zk.partition_key + ", local: " + partition_key, + ErrorCodes::METADATA_MISMATCH); - if (read_partition_key != local_partition_key) + if (sorting_key != from_zk.sorting_key) + { + if (allow_alter) + { + diff.sorting_key_changed = true; + diff.new_sorting_key = from_zk.sorting_key; + } + else throw Exception( - "Existing table metadata in ZooKeeper differs in partition key expression." - " Stored in ZooKeeper: " + read_partition_key + ", local: " + local_partition_key, + "Existing table metadata in ZooKeeper differs in sorting key expression." + " Stored in ZooKeeper: " + from_zk.sorting_key + ", local: " + sorting_key, ErrorCodes::METADATA_MISMATCH); - - in >> "\n"; } - if (checkString("sorting key: ", in)) - { - String read_sorting_key; - in >> read_sorting_key >> "\n"; - } - - assertEOF(in); -} - -void ReplicatedMergeTreeTableMetadata::check(const String & s) const -{ - ReadBufferFromString in(s); - check(in); + return diff; } } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h index edca8a2e607..b063e226348 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h @@ -1,6 +1,8 @@ #pragma once #include +#include +#include namespace DB { @@ -14,23 +16,34 @@ class ReadBuffer; */ struct ReplicatedMergeTreeTableMetadata { - const MergeTreeData & data; + String date_column; + String sampling_expression; + UInt64 index_granularity; + int merging_params_mode; + String sign_column; + String primary_key; + MergeTreeDataFormatVersion data_format_version; + String partition_key; + String sorting_key; - bool sorting_and_primary_keys_independent = false; - String sorting_key_str; + ReplicatedMergeTreeTableMetadata() = default; + explicit ReplicatedMergeTreeTableMetadata(const MergeTreeData & data); - explicit ReplicatedMergeTreeTableMetadata(const MergeTreeData & data_); + void read(ReadBuffer & in); + static ReplicatedMergeTreeTableMetadata parse(const String & s); void write(WriteBuffer & out) const; String toString() const; - void read(ReadBuffer & in); - static ReplicatedMergeTreeTableMetadata parse(const MergeTreeData & data_, const String & s); + struct Diff + { + bool sorting_key_changed = false; + String new_sorting_key; - void check(ReadBuffer & in) const; - void check(const String & s) const; + bool empty() const { return !sorting_key_changed; } + }; - /// TODO: checkAndFindDiff(other); + Diff checkAndFindDiff(const ReplicatedMergeTreeTableMetadata & from_zk, bool allow_alter) const; }; } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 9f5734887f7..36494092371 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -374,9 +374,8 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo Coordination::Stat metadata_stat; String metadata_str = zookeeper->get(zookeeper_path + "/metadata", &metadata_stat); - auto metadata_from_zk = ReplicatedMergeTreeTableMetadata::parse(data, metadata_str); - old_metadata.check(metadata_str); - const bool metadata_changed = metadata_str != old_metadata.toString(); + auto metadata_from_zk = ReplicatedMergeTreeTableMetadata::parse(metadata_str); + auto metadata_diff = old_metadata.checkAndFindDiff(metadata_from_zk, allow_alter); metadata_version = metadata_stat.version; Coordination::Stat columns_stat; @@ -384,7 +383,7 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo columns_version = columns_stat.version; const ColumnsDescription & old_columns = getColumns(); - if (columns_from_zk != old_columns || metadata_changed) + if (columns_from_zk != old_columns || !metadata_diff.empty()) { if (allow_alter && (skip_sanity_checks || @@ -394,7 +393,7 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo LOG_WARNING(log, "Table structure in ZooKeeper is a little different from local table structure. Assuming ALTER."); /// Without any locks, because table has not been created yet. - setTableStructure(std::move(columns_from_zk), metadata_from_zk); + setTableStructure(std::move(columns_from_zk), metadata_diff); } else { @@ -405,17 +404,19 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo } -void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata & new_metadata) +void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff) { /// Note: setting columns first so that the new sorting key can use new columns. setColumns(std::move(new_columns)); ASTPtr new_sorting_key_ast = data.sorting_key_ast; + bool new_sorting_and_primary_keys_independent = data.sorting_and_primary_keys_independent; IDatabase::ASTModifier storage_modifier; - if (ReplicatedMergeTreeTableMetadata(data).sorting_key_str != new_metadata.sorting_key_str) + if (!metadata_diff.empty()) { + new_sorting_and_primary_keys_independent = true; ParserNotEmptyExpressionList parser(false); - new_sorting_key_ast = parseQuery(parser, new_metadata.sorting_key_str, 0); + new_sorting_key_ast = parseQuery(parser, metadata_diff.new_sorting_key, 0); storage_modifier = [&](IAST & ast) { @@ -442,7 +443,10 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column /// Even if the primary/sorting keys didn't change we must reinitialize it /// because primary key column types might have changed. - data.setPrimaryKey(data.primary_key_ast, new_sorting_key_ast); + if (new_sorting_and_primary_keys_independent) + data.setPrimaryKey(data.primary_key_ast, new_sorting_key_ast); + else + data.setPrimaryKey(nullptr, new_sorting_key_ast); context.getDatabase(database_name)->alterTable(context, table_name, getColumns(), storage_modifier); } @@ -3021,10 +3025,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, for (const AlterCommand & param : params) { if (param.type == AlterCommand::MODIFY_ORDER_BY) - { - new_metadata.sorting_and_primary_keys_independent = true; - new_metadata.sorting_key_str = serializeAST(*param.sorting_key); - } + new_metadata.sorting_key = serializeAST(*param.sorting_key); } String new_metadata_str = new_metadata.toString(); if (new_metadata_str != ReplicatedMergeTreeTableMetadata(data).toString()) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 27e5599d89f..f64d793be20 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -29,8 +30,6 @@ namespace DB { -class ReplicatedMergeTreeTableMetadata; - /** The engine that uses the merge tree (see MergeTreeData) and replicated through ZooKeeper. * * ZooKeeper is used for the following things: @@ -336,7 +335,7 @@ private: /// A part of ALTER: apply metadata changes only (data parts are altered separately). /// Must be called under IStorage::lockStructureForAlter() lock. - void setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata & new_metadata); + void setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff); /** Check that the set of parts corresponds to that in ZK (/replicas/me/parts/). * If any parts described in ZK are not locally, throw an exception. From 8b858221925c06927b0f2b97c1b4791c72c07603 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Tue, 6 Nov 2018 21:25:36 +0300 Subject: [PATCH 16/69] store original key ASTs, and not extracted ASTExpressionLists [#CLICKHOUSE-3859] --- .../src/Interpreters/MutationsInterpreter.cpp | 4 +- dbms/src/Parsers/ASTAlterQuery.cpp | 12 +- dbms/src/Parsers/ASTAlterQuery.h | 2 +- dbms/src/Parsers/ASTFunction.h | 2 +- dbms/src/Parsers/ParserAlterQuery.cpp | 20 +-- dbms/src/Storages/AlterCommands.cpp | 2 +- dbms/src/Storages/AlterCommands.h | 2 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 107 ++++++++-------- dbms/src/Storages/MergeTree/MergeTreeData.h | 52 ++++---- .../Storages/MergeTree/MergeTreeDataPart.cpp | 4 +- .../MergeTree/MergeTreeDataWriter.cpp | 4 +- .../Storages/MergeTree/MergeTreePartition.cpp | 4 +- .../ReplicatedMergeTreeAlterThread.cpp | 3 +- .../ReplicatedMergeTreeTableMetadata.cpp | 15 ++- .../MergeTree/registerStorageMergeTree.cpp | 46 ++----- dbms/src/Storages/StorageMergeTree.cpp | 114 +++++++----------- dbms/src/Storages/StorageMergeTree.h | 15 +-- .../Storages/StorageReplicatedMergeTree.cpp | 52 ++++---- .../src/Storages/StorageReplicatedMergeTree.h | 6 +- 19 files changed, 205 insertions(+), 261 deletions(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 13e4b4c25d3..bf0c856da73 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -98,8 +98,8 @@ static NameSet getKeyColumns(const StoragePtr & storage) NameSet key_columns; - if (merge_tree_data->partition_expr) - for (const String & col : merge_tree_data->partition_expr->getRequiredColumns()) + if (merge_tree_data->partition_key_expr) + for (const String & col : merge_tree_data->partition_key_expr->getRequiredColumns()) key_columns.insert(col); auto sorting_key_expr = merge_tree_data->getSortingKeyExpression(); diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index 7bed86d7dcb..c3b7d2a1841 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -30,10 +30,10 @@ ASTPtr ASTAlterCommand::clone() const res->primary_key = primary_key->clone(); res->children.push_back(res->primary_key); } - if (sorting_key) + if (order_by) { - res->sorting_key = sorting_key->clone(); - res->children.push_back(res->sorting_key); + res->order_by = order_by->clone(); + res->children.push_back(res->order_by); } if (partition) { @@ -85,16 +85,12 @@ void ASTAlterCommand::formatImpl( else if (type == ASTAlterCommand::MODIFY_PRIMARY_KEY) { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY PRIMARY KEY " << (settings.hilite ? hilite_none : ""); - settings.ostr << "("; primary_key->formatImpl(settings, state, frame); - settings.ostr << ")"; } else if (type == ASTAlterCommand::MODIFY_ORDER_BY) { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY ORDER BY " << (settings.hilite ? hilite_none : ""); - settings.ostr << "("; - sorting_key->formatImpl(settings, state, frame); - settings.ostr << ")"; + order_by->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::DROP_PARTITION) { diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index 8aeee01b035..cdd1c28e329 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -58,7 +58,7 @@ public: /** For MODIFY ORDER BY */ - ASTPtr sorting_key; + ASTPtr order_by; /** Used in DROP PARTITION and ATTACH PARTITION FROM queries. * The value or ID of the partition is stored here. diff --git a/dbms/src/Parsers/ASTFunction.h b/dbms/src/Parsers/ASTFunction.h index e580b8c38b1..3bed72d4305 100644 --- a/dbms/src/Parsers/ASTFunction.h +++ b/dbms/src/Parsers/ASTFunction.h @@ -30,7 +30,7 @@ protected: template -ASTPtr makeASTFunction(const String & name, Args &&... args) +std::shared_ptr makeASTFunction(const String & name, Args &&... args) { const auto function = std::make_shared(); diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 619755a7800..a3b6dd635df 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -188,31 +188,15 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (s_modify_primary_key.ignore(pos, expected)) { - if (pos->type != TokenType::OpeningRoundBracket) + if (!parser_exp_elem.parse(pos, command->primary_key, expected)) return false; - ++pos; - - if (!ParserNotEmptyExpressionList(false).parse(pos, command->primary_key, expected)) - return false; - - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; command->type = ASTAlterCommand::MODIFY_PRIMARY_KEY; } else if (s_modify_order_by.ignore(pos, expected)) { - if (pos->type != TokenType::OpeningRoundBracket) + if (!parser_exp_elem.parse(pos, command->order_by, expected)) return false; - ++pos; - - if (!ParserNotEmptyExpressionList(false).parse(pos, command->sorting_key, expected)) - return false; - - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; command->type = ASTAlterCommand::MODIFY_ORDER_BY; } diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 8f8537968da..612e7479fb5 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -94,7 +94,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ { AlterCommand command; command.type = AlterCommand::MODIFY_ORDER_BY; - command.sorting_key = command_ast->sorting_key; + command.order_by = command_ast->order_by; return command; } else diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index d6806eb48af..e0ec754bf7a 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -44,7 +44,7 @@ struct AlterCommand ASTPtr primary_key; /// For MODIFY_ORDER_BY - ASTPtr sorting_key; + ASTPtr order_by; AlterCommand() = default; AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type, diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 1f9ec8c6465..c3b7bb332e4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -86,10 +86,10 @@ MergeTreeData::MergeTreeData( const String & database_, const String & table_, const String & full_path_, const ColumnsDescription & columns_, Context & context_, - const ASTPtr & primary_key_ast_, - const ASTPtr & sorting_key_ast_, const String & date_column_name, - const ASTPtr & partition_expr_ast_, + const ASTPtr & partition_by_ast_, + const ASTPtr & order_by_ast_, + const ASTPtr & primary_key_ast_, const ASTPtr & sampling_expression_, const MergingParams & merging_params_, const MergeTreeSettings & settings_, @@ -102,7 +102,7 @@ MergeTreeData::MergeTreeData( index_granularity(settings_.index_granularity), merging_params(merging_params_), settings(settings_), - partition_expr_ast(partition_expr_ast_), + partition_by_ast(partition_by_ast_), require_part_metadata(require_part_metadata_), database_name(database_), table_name(table_), full_path(full_path_), @@ -114,7 +114,7 @@ MergeTreeData::MergeTreeData( /// NOTE: using the same columns list as is read when performing actual merges. merging_params.check(getColumns().getAllPhysical()); - setPrimaryKey(primary_key_ast_, sorting_key_ast_); + setPrimaryKey(order_by_ast_, primary_key_ast_); if (sampling_expression && (!primary_key_sample.has(sampling_expression->getColumnName())) && !attach && !settings.compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility. @@ -125,11 +125,7 @@ MergeTreeData::MergeTreeData( { try { - String partition_expr_str = "toYYYYMM(" + backQuoteIfNeed(date_column_name) + ")"; - ParserNotEmptyExpressionList parser(/* allow_alias_without_as_keyword = */ false); - partition_expr_ast = parseQuery( - parser, partition_expr_str.data(), partition_expr_str.data() + partition_expr_str.length(), "partition expression", 0); - + partition_by_ast = makeASTFunction("toYYYYMM", std::make_shared(date_column_name)); initPartitionKey(); if (minmax_idx_date_column_pos == -1) @@ -209,26 +205,20 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam } -void MergeTreeData::setPrimaryKey(ASTPtr new_primary_key_ast, const ASTPtr & new_sorting_key_ast) +void MergeTreeData::setPrimaryKey(const ASTPtr & new_order_by_ast, ASTPtr new_primary_key_ast) { - if (!new_sorting_key_ast) - throw Exception("Sorting key cannot be empty", ErrorCodes::BAD_ARGUMENTS); + if (!new_order_by_ast) + throw Exception("ORDER BY cannot be empty", ErrorCodes::BAD_ARGUMENTS); - bool new_sorting_and_primary_keys_independent = true; - if (!new_primary_key_ast) - { - new_primary_key_ast = new_sorting_key_ast->clone(); - new_sorting_and_primary_keys_independent = false; - } + ASTPtr new_sorting_key_expr_list = extractKeyExpressionList(new_order_by_ast); + ASTPtr new_primary_key_expr_list = new_primary_key_ast + ? extractKeyExpressionList(new_primary_key_ast) : new_sorting_key_expr_list->clone(); - if (new_sorting_key_ast.get() != sorting_key_ast.get() - && merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) - { - new_sorting_key_ast->children.push_back(std::make_shared(merging_params.version_column)); - } + if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) + new_sorting_key_expr_list->children.push_back(std::make_shared(merging_params.version_column)); - size_t primary_key_size = new_primary_key_ast->children.size(); - size_t sorting_key_size = new_sorting_key_ast->children.size(); + size_t primary_key_size = new_primary_key_expr_list->children.size(); + size_t sorting_key_size = new_sorting_key_expr_list->children.size(); if (primary_key_size > sorting_key_size) throw Exception("Primary key must be a prefix of the sorting key, but its length: " + toString(primary_key_size) + " is greater than the sorting key length: " + toString(sorting_key_size), @@ -239,12 +229,12 @@ void MergeTreeData::setPrimaryKey(ASTPtr new_primary_key_ast, const ASTPtr & new for (size_t i = 0; i < sorting_key_size; ++i) { - String sorting_key_column = new_sorting_key_ast->children[i]->getColumnName(); + String sorting_key_column = new_sorting_key_expr_list->children[i]->getColumnName(); new_sorting_key_columns.push_back(sorting_key_column); if (i < primary_key_size) { - String pk_column = new_primary_key_ast->children[i]->getColumnName(); + String pk_column = new_primary_key_expr_list->children[i]->getColumnName(); if (pk_column != sorting_key_column) throw Exception("Primary key must be a prefix of the sorting key, but in position " + toString(i) + " its column is " + pk_column + ", not " + sorting_key_column, @@ -256,15 +246,15 @@ void MergeTreeData::setPrimaryKey(ASTPtr new_primary_key_ast, const ASTPtr & new auto all_columns = getColumns().getAllPhysical(); - auto new_sorting_key_expr = ExpressionAnalyzer(new_sorting_key_ast, context, nullptr, all_columns) + auto new_sorting_key_expr = ExpressionAnalyzer(new_sorting_key_expr_list, context, nullptr, all_columns) .getActions(false); auto new_sorting_key_sample = - ExpressionAnalyzer(new_sorting_key_ast, context, nullptr, all_columns) + ExpressionAnalyzer(new_sorting_key_expr_list, context, nullptr, all_columns) .getActions(true)->getSampleBlock(); checkKeyExpression(*new_sorting_key_expr, new_sorting_key_sample, "Sorting"); - auto new_primary_key_expr = ExpressionAnalyzer(new_primary_key_ast, context, nullptr, all_columns) + auto new_primary_key_expr = ExpressionAnalyzer(new_primary_key_expr_list, context, nullptr, all_columns) .getActions(false); Block new_primary_key_sample; @@ -276,10 +266,9 @@ void MergeTreeData::setPrimaryKey(ASTPtr new_primary_key_ast, const ASTPtr & new new_primary_key_data_types.push_back(elem.type); } - sorting_key_ast = new_sorting_key_ast; + order_by_ast = new_order_by_ast; sorting_key_columns = std::move(new_sorting_key_columns); sorting_key_expr = std::move(new_sorting_key_expr); - sorting_and_primary_keys_independent = new_sorting_and_primary_keys_independent; primary_key_ast = new_primary_key_ast; primary_key_columns = std::move(new_primary_key_columns); @@ -289,22 +278,46 @@ void MergeTreeData::setPrimaryKey(ASTPtr new_primary_key_ast, const ASTPtr & new } +ASTPtr MergeTreeData::extractKeyExpressionList(const ASTPtr & node) +{ + if (!node) + return std::make_shared(); + + const ASTFunction * expr_func = typeid_cast(node.get()); + + if (expr_func && expr_func->name == "tuple") + { + /// Primary key is specified in tuple. + return expr_func->children.at(0); + } + else + { + /// Primary key consists of one column. + auto res = std::make_shared(); + res->children.push_back(node); + return res; + } +} + + void MergeTreeData::initPartitionKey() { - if (!partition_expr_ast || partition_expr_ast->children.empty()) + ASTPtr partition_key_expr_list = extractKeyExpressionList(partition_by_ast); + + if (partition_key_expr_list->children.empty()) return; - partition_expr = ExpressionAnalyzer(partition_expr_ast, context, nullptr, getColumns().getAllPhysical()).getActions(false); - for (const ASTPtr & ast : partition_expr_ast->children) + partition_key_expr = ExpressionAnalyzer(partition_key_expr_list, context, nullptr, getColumns().getAllPhysical()).getActions(false); + for (const ASTPtr & ast : partition_key_expr_list->children) { String col_name = ast->getColumnName(); - partition_key_sample.insert(partition_expr->getSampleBlock().getByName(col_name)); + partition_key_sample.insert(partition_key_expr->getSampleBlock().getByName(col_name)); } - checkKeyExpression(*partition_expr, partition_key_sample, "Partition"); + checkKeyExpression(*partition_key_expr, partition_key_sample, "Partition"); /// Add all columns used in the partition key to the min-max index. - const NamesAndTypesList & minmax_idx_columns_with_types = partition_expr->getRequiredColumnsWithTypes(); + const NamesAndTypesList & minmax_idx_columns_with_types = partition_key_expr->getRequiredColumnsWithTypes(); minmax_idx_expr = std::make_shared(minmax_idx_columns_with_types, context); for (const NameAndTypePair & column : minmax_idx_columns_with_types) { @@ -904,12 +917,12 @@ void MergeTreeData::checkAlter(const AlterCommands & commands) /// (and not as a part of some expression) and if the ALTER only affects column metadata. NameSet columns_alter_metadata_only; - if (partition_expr) + if (partition_key_expr) { /// Forbid altering partition key columns because it can change partition ID format. /// TODO: in some cases (e.g. adding an Enum value) a partition key column can still be ALTERed. /// We should allow it. - for (const String & col : partition_expr->getRequiredColumns()) + for (const String & col : partition_key_expr->getRequiredColumns()) columns_alter_forbidden.insert(col); } @@ -1107,7 +1120,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( const DataPartPtr & part, const NamesAndTypesList & new_columns, - const ASTPtr & new_primary_key, + const ASTPtr & new_primary_key_expr_list, bool skip_sanity_checks) { ExpressionActionsPtr expression; @@ -1172,10 +1185,10 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( size_t new_primary_key_file_size{}; MergeTreeDataPartChecksum::uint128 new_primary_key_hash{}; - /// TODO: Check the order of secondary sorting key columns. - if (new_primary_key.get() != primary_key_ast.get()) + if (new_primary_key_expr_list) { - ExpressionActionsPtr new_primary_expr = ExpressionAnalyzer(new_primary_key, context, nullptr, new_columns).getActions(true); + ExpressionActionsPtr new_primary_expr = ExpressionAnalyzer( + new_primary_key_expr_list, context, nullptr, new_columns).getActions(true); Block new_primary_key_sample = new_primary_expr->getSampleBlock(); size_t new_key_size = new_primary_key_sample.columns(); @@ -2426,10 +2439,10 @@ MergeTreeData * MergeTreeData::checkStructureAndGetMergeTreeData(const StoragePt return ast ? queryToString(ast) : ""; }; - if (query_to_string(sorting_key_ast) != query_to_string(src_data->sorting_key_ast)) + if (query_to_string(order_by_ast) != query_to_string(src_data->order_by_ast)) throw Exception("Tables have different ordering", ErrorCodes::BAD_ARGUMENTS); - if (query_to_string(partition_expr_ast) != query_to_string(src_data->partition_expr_ast)) + if (query_to_string(partition_by_ast) != query_to_string(src_data->partition_by_ast)) throw Exception("Tables have different partition key", ErrorCodes::BAD_ARGUMENTS); if (format_version != src_data->format_version) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 4da0b0aa473..afcd18caac6 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -284,24 +284,27 @@ public: /// Attach the table corresponding to the directory in full_path (must end with /), with the given columns. /// Correctness of names and paths is not checked. /// - /// sorting_key_ast - expression used for sorting data in parts; - /// primary_key_ast - values of this expression for one row in every `index_granularity` rows - /// are written in the primary.idx to speed up range queries. - /// Primary key must be a prefix of the sorting key; - /// If it is nullptr, then it will be determined from sorting_key_ast. - /// /// date_column_name - if not empty, the name of the Date column used for partitioning by month. - /// Otherwise, partition_expr_ast is used for partitioning. + /// Otherwise, partition_by_ast is used for partitioning. + /// + /// order_by_ast - a single expression or a tuple. It is used as a sorting key + /// (an ASTExpressionList used for sorting data in parts); + /// primary_key_ast - can be nullptr, an expression, or a tuple. + /// Used to determine an ASTExpressionList values of which are written in the primary.idx file + /// for one row in every `index_granularity` rows to speed up range queries. + /// Primary key must be a prefix of the sorting key; + /// If it is nullptr, then it will be determined from order_by_ast. + /// /// require_part_metadata - should checksums.txt and columns.txt exist in the part directory. /// attach - whether the existing table is attached or the new table is created. MergeTreeData(const String & database_, const String & table_, const String & full_path_, const ColumnsDescription & columns_, Context & context_, - const ASTPtr & primary_key_ast_, - const ASTPtr & sorting_key_ast_, const String & date_column_name, - const ASTPtr & partition_expr_ast_, + const ASTPtr & partition_by_ast_, + const ASTPtr & order_by_ast_, + const ASTPtr & primary_key_ast_, const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported. const MergingParams & merging_params_, const MergeTreeSettings & settings_, @@ -477,12 +480,13 @@ public: /// Performs ALTER of the data part, writes the result to temporary files. /// Returns an object allowing to rename temporary files to permanent files. + /// If new_primary_key_expr_list is not nullptr, will prepare the new primary.idx file. /// If the number of affected columns is suspiciously high and skip_sanity_checks is false, throws an exception. /// If no data transformations are necessary, returns nullptr. AlterDataPartTransactionPtr alterDataPart( const DataPartPtr & part, const NamesAndTypesList & new_columns, - const ASTPtr & new_primary_key, + const ASTPtr & new_primary_key_expr_list, bool skip_sanity_checks); /// Should be called if part data is suspected to be corrupted. @@ -491,6 +495,12 @@ public: broken_part_callback(name); } + /** Get the key expression AST as an ASTExpressionList. + * It can be specified in the tuple: (CounterID, Date), + * or as one column: CounterID. + */ + static ASTPtr extractKeyExpressionList(const ASTPtr & node); + bool hasPrimaryKey() const { return !primary_key_columns.empty(); } ExpressionActionsPtr getPrimaryKeyExpression() const { return primary_key_expr; } Names getPrimaryKeyColumns() const { return primary_key_columns; } @@ -555,16 +565,14 @@ public: const MergeTreeSettings settings; + ASTPtr order_by_ast; ASTPtr primary_key_ast; + Block primary_key_sample; DataTypes primary_key_data_types; - ASTPtr sorting_key_ast; - /// If true, sorting and primary keys were set using separate clauses in the CREATE TABLE statement. - bool sorting_and_primary_keys_independent = false; - - ASTPtr partition_expr_ast; - ExpressionActionsPtr partition_expr; + ASTPtr partition_by_ast; + ExpressionActionsPtr partition_key_expr; Block partition_key_sample; ExpressionActionsPtr minmax_idx_expr; @@ -588,14 +596,14 @@ private: bool require_part_metadata; - ExpressionActionsPtr primary_key_expr; - /// Names of columns for primary key. - Names primary_key_columns; - ExpressionActionsPtr sorting_key_expr; /// Names of columns for primary key + secondary sorting columns. Names sorting_key_columns; + ExpressionActionsPtr primary_key_expr; + /// Names of columns for primary key. + Names primary_key_columns; + String database_name; String table_name; String full_path; @@ -697,7 +705,7 @@ private: /// The same for clearOldTemporaryDirectories. std::mutex clear_old_temporary_directories_mutex; - void setPrimaryKey(ASTPtr new_primary_key_ast, const ASTPtr & new_sorting_key_ast); + void setPrimaryKey(const ASTPtr & new_order_by_ast, ASTPtr new_primary_key_ast); void initPartitionKey(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp index 459ed89e837..4d72b3e8632 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp @@ -659,7 +659,7 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata) if (!checksums.files.count("count.txt")) throw Exception("No checksum for count.txt", ErrorCodes::NO_FILE_IN_DATA_PART); - if (storage.partition_expr && !checksums.files.count("partition.dat")) + if (storage.partition_key_expr && !checksums.files.count("partition.dat")) throw Exception("No checksum for partition.dat", ErrorCodes::NO_FILE_IN_DATA_PART); if (!isEmpty()) @@ -692,7 +692,7 @@ void MergeTreeDataPart::checkConsistency(bool require_part_metadata) { check_file_not_empty(path + "count.txt"); - if (storage.partition_expr) + if (storage.partition_key_expr) check_file_not_empty(path + "partition.dat"); for (const String & col_name : storage.minmax_idx_columns) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index a10aff25b6b..b88a2f13960 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -71,14 +71,14 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block data.check(block, true); block.checkNumberOfRows(); - if (!data.partition_expr) /// Table is not partitioned. + if (!data.partition_key_expr) /// Table is not partitioned. { result.emplace_back(Block(block), Row()); return result; } Block block_copy = block; - data.partition_expr->execute(block_copy); + data.partition_key_expr->execute(block_copy); ColumnRawPtrs partition_columns; partition_columns.reserve(data.partition_key_sample.columns()); diff --git a/dbms/src/Storages/MergeTree/MergeTreePartition.cpp b/dbms/src/Storages/MergeTree/MergeTreePartition.cpp index 6064625d49e..8160536da0b 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreePartition.cpp @@ -107,7 +107,7 @@ void MergeTreePartition::serializeTextQuoted(const MergeTreeData & storage, Writ void MergeTreePartition::load(const MergeTreeData & storage, const String & part_path) { - if (!storage.partition_expr) + if (!storage.partition_key_expr) return; ReadBufferFromFile file = openForReading(part_path + "partition.dat"); @@ -118,7 +118,7 @@ void MergeTreePartition::load(const MergeTreeData & storage, const String & part void MergeTreePartition::store(const MergeTreeData & storage, const String & part_path, MergeTreeDataPartChecksums & checksums) const { - if (!storage.partition_expr) + if (!storage.partition_key_expr) return; WriteBufferFromFile out(part_path + "partition.dat"); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp index 7886965232e..1b5630f32ed 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp @@ -150,8 +150,7 @@ void ReplicatedMergeTreeAlterThread::run() /// Update the part and write result to temporary files. /// TODO: You can skip checking for too large changes if ZooKeeper has, for example, /// node /flags/force_alter. - auto transaction = storage.data.alterDataPart( - part, columns_for_parts, storage.data.primary_key_ast, false); + auto transaction = storage.data.alterDataPart(part, columns_for_parts, nullptr, false); if (!transaction) continue; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 268fa53ea57..8f9ab03cd55 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -31,14 +31,19 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr index_granularity = data.index_granularity; merging_params_mode = static_cast(data.merging_params.mode); sign_column = data.merging_params.sign_column; - primary_key = formattedAST(data.primary_key_ast); + + if (!data.primary_key_ast) + primary_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.order_by_ast)); + else + { + primary_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.primary_key_ast)); + sorting_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.order_by_ast)); + } + data_format_version = data.format_version; if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) - partition_key = formattedAST(data.partition_expr_ast); - - if (data.sorting_and_primary_keys_independent) - sorting_key = formattedAST(data.sorting_key_ast); + partition_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.partition_by_ast)); } void ReplicatedMergeTreeTableMetadata::write(WriteBuffer & out) const diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index 05d524ec61c..69e6053eedf 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -28,28 +28,6 @@ namespace ErrorCodes } -/** Get the key expression AST as an ASTExpressionList. - * It can be specified in the tuple: (CounterID, Date), - * or as one column: CounterID. - */ -static ASTPtr extractKeyExpressionList(IAST & node) -{ - const ASTFunction * expr_func = typeid_cast(&node); - - if (expr_func && expr_func->name == "tuple") - { - /// Primary key is specified in tuple. - return expr_func->children.at(0); - } - else - { - /// Primary key consists of one column. - auto res = std::make_shared(); - res->children.push_back(node.ptr()); - return res; - } -} - /** Get the list of column names. * It can be specified in the tuple: (Clicks, Cost), * or as one column: Clicks. @@ -568,28 +546,26 @@ static StoragePtr create(const StorageFactory::Arguments & args) } String date_column_name; - ASTPtr partition_expr_list; - ASTPtr primary_expr_list; - ASTPtr sorting_expr_list; + ASTPtr partition_by_ast; + ASTPtr order_by_ast; + ASTPtr primary_key_ast; ASTPtr sampling_expression; MergeTreeSettings storage_settings = args.context.getMergeTreeSettings(); if (is_extended_storage_def) { if (args.storage_def->partition_by) - partition_expr_list = extractKeyExpressionList(*args.storage_def->partition_by); + partition_by_ast = args.storage_def->partition_by->ptr(); - if (args.storage_def->order_by) - { - sorting_expr_list = extractKeyExpressionList(*args.storage_def->order_by); - } - else + if (!args.storage_def->order_by) throw Exception("You must provide an ORDER BY expression in the table definition. " "If you don't want this table to be sorted, use ORDER BY tuple()", ErrorCodes::BAD_ARGUMENTS); + order_by_ast = args.storage_def->order_by->ptr(); + if (args.storage_def->primary_key) - primary_expr_list = extractKeyExpressionList(*args.storage_def->primary_key); + primary_key_ast = args.storage_def->primary_key->ptr(); if (args.storage_def->sample_by) sampling_expression = args.storage_def->sample_by->ptr(); @@ -614,7 +590,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) "Date column name must be an unquoted string" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS); - sorting_expr_list = extractKeyExpressionList(*engine_args[1]); + order_by_ast = engine_args[1]; auto ast = typeid_cast(engine_args.back().get()); if (ast && ast->value.getType() == Field::Types::UInt64) @@ -629,13 +605,13 @@ static StoragePtr create(const StorageFactory::Arguments & args) return StorageReplicatedMergeTree::create( zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name, args.columns, - args.context, primary_expr_list, sorting_expr_list, date_column_name, partition_expr_list, + args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast, sampling_expression, merging_params, storage_settings, args.has_force_restore_data_flag); else return StorageMergeTree::create( args.data_path, args.database_name, args.table_name, args.columns, args.attach, - args.context, primary_expr_list, sorting_expr_list, date_column_name, partition_expr_list, + args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast, sampling_expression, merging_params, storage_settings, args.has_force_restore_data_flag); } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index b849f9c85a1..6af8cfdcace 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -46,10 +46,10 @@ StorageMergeTree::StorageMergeTree( const ColumnsDescription & columns_, bool attach, Context & context_, - const ASTPtr & primary_key_ast_, - const ASTPtr & sorting_key_ast_, const String & date_column_name, - const ASTPtr & partition_expr_ast_, + const ASTPtr & partition_by_ast_, + const ASTPtr & order_by_ast_, + const ASTPtr & primary_key_ast_, const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported. const MergeTreeData::MergingParams & merging_params_, const MergeTreeSettings & settings_, @@ -58,7 +58,7 @@ StorageMergeTree::StorageMergeTree( context(context_), background_pool(context_.getBackgroundPool()), data(database_name, table_name, full_path, columns_, - context_, primary_key_ast_, sorting_key_ast_, date_column_name, partition_expr_ast_, + context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_, sampling_expression_, merging_params_, settings_, false, attach), reader(data), writer(data), merger_mutator(data, context.getBackgroundPool()), @@ -200,103 +200,75 @@ void StorageMergeTree::alter( std::vector transactions; - bool primary_key_is_modified = false; + ASTPtr new_order_by_ast = data.order_by_ast; + ASTPtr new_primary_key_ast = data.primary_key_ast; - - bool sorting_key_is_modified = false; - ASTPtr new_sorting_key_ast = data.sorting_key_ast; - - bool new_sorting_and_primary_keys_independent = data.sorting_and_primary_keys_independent; + ASTPtr primary_expr_list_for_altering_parts; for (const AlterCommand & param : params) { - if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) + if (param.type == AlterCommand::MODIFY_ORDER_BY) { - primary_key_is_modified = true; - new_primary_key_ast = param.primary_key; + if (!data.primary_key_ast) + { + /// Primary and sorting key become independent after this ALTER so we have to + /// save the old ORDER BY expression as the new primary key. + new_primary_key_ast = data.order_by_ast->clone(); + } + + new_order_by_ast = param.order_by; } - else if (param.type == AlterCommand::MODIFY_ORDER_BY) + else if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) { - sorting_key_is_modified = true; - new_sorting_and_primary_keys_independent = true; - new_sorting_key_ast = param.sorting_key; + primary_expr_list_for_altering_parts = MergeTreeData::extractKeyExpressionList(param.primary_key); + if (!data.primary_key_ast) + new_order_by_ast = param.primary_key; + else + new_primary_key_ast = param.primary_key; } } - if (primary_key_is_modified && supportsSampling()) + if (primary_expr_list_for_altering_parts && supportsSampling()) throw Exception("MODIFY PRIMARY KEY only supported for tables without sampling key", ErrorCodes::BAD_ARGUMENTS); auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}); auto columns_for_parts = new_columns.getAllPhysical(); for (const MergeTreeData::DataPartPtr & part : parts) { - if (auto transaction = data.alterDataPart(part, columns_for_parts, new_primary_key_ast, false)) + if (auto transaction = data.alterDataPart(part, columns_for_parts, primary_expr_list_for_altering_parts, false)) transactions.push_back(std::move(transaction)); } auto table_hard_lock = lockStructureForAlter(__PRETTY_FUNCTION__); - IDatabase::ASTModifier storage_modifier; - if (primary_key_is_modified || sorting_key_is_modified) + IDatabase::ASTModifier storage_modifier = [&] (IAST & ast) { - storage_modifier = [&] (IAST & ast) + auto & storage_ast = typeid_cast(ast); + + if (new_order_by_ast.get() != data.order_by_ast.get()) { - auto & storage_ast = typeid_cast(ast); - - if (primary_key_is_modified) + if (storage_ast.order_by) { - auto tuple = std::make_shared(); - tuple->name = "tuple"; - tuple->arguments = new_primary_key_ast; - tuple->children.push_back(tuple->arguments); - - if (storage_ast.order_by) - { - /// The table was created using the syntax with key expressions in separate clauses. - if (new_sorting_and_primary_keys_independent) - storage_ast.set(storage_ast.primary_key, tuple); - else - storage_ast.set(storage_ast.order_by, tuple); - } - else - { - /// Primary key is in the second place in table engine description and can be represented as a tuple. - /// TODO: Not always in second place. If there is a sampling key, then the third one. Fix it. - storage_ast.engine->arguments->children.at(1) = tuple; - } + /// The table was created using the "new" syntax (with key expressions in separate clauses). + storage_ast.set(storage_ast.order_by, new_order_by_ast); } - - if (sorting_key_is_modified) + else { - /// TODO: helper for tuple creation - auto tuple = std::make_shared(); - tuple->name = "tuple"; - tuple->arguments = new_sorting_key_ast; - tuple->children.push_back(tuple->arguments); - - if (!storage_ast.order_by) - throw Exception("Not supported", ErrorCodes::LOGICAL_ERROR); /// TODO: better exception message - - if (!storage_ast.primary_key) - { - /// Primary and sorting key become independent after this ALTER so we have to - /// save the old ORDER BY expression as the new primary key. - storage_ast.set(storage_ast.primary_key, storage_ast.order_by->clone()); - } - - storage_ast.set(storage_ast.order_by, tuple); + /// Primary key is in the second place in table engine description and can be represented as a tuple. + /// TODO: Not always in second place. If there is a sampling key, then the third one. Fix it. + storage_ast.engine->arguments->children.at(1) = new_order_by_ast; } - }; - } + } + + if (new_primary_key_ast.get() != data.primary_key_ast.get()) + storage_ast.set(storage_ast.primary_key, new_primary_key_ast); + }; context.getDatabase(database_name)->alterTable(context, table_name, new_columns, storage_modifier); setColumns(std::move(new_columns)); /// Reinitialize primary key because primary key column types might have changed. - if (new_sorting_and_primary_keys_independent) - data.setPrimaryKey(new_primary_key_ast, new_sorting_key_ast); - else - data.setPrimaryKey(nullptr, new_primary_key_ast); + data.setPrimaryKey(new_order_by_ast, new_primary_key_ast); for (auto & transaction : transactions) transaction->commit(); @@ -304,7 +276,7 @@ void StorageMergeTree::alter( /// Columns sizes could be changed data.recalculateColumnSizes(); - if (primary_key_is_modified) + if (primary_expr_list_for_altering_parts) data.loadDataParts(false); } @@ -760,7 +732,7 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi if (part->info.partition_id != partition_id) throw Exception("Unexpected partition ID " + part->info.partition_id + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); - if (auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_key_ast, false)) + if (auto transaction = data.alterDataPart(part, columns_for_parts, nullptr, false)) transactions.push_back(std::move(transaction)); LOG_DEBUG(log, "Removing column " << get(column_name) << " from part " << part->name); diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 8d31632d570..fac02d7528a 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -157,14 +157,7 @@ protected: * (correctness of names and paths are not checked) * consisting of the specified columns. * - * sorting_key_ast - expression used for sorting data in parts; - * primary_key_ast - values of this expression for one row in every `index_granularity` rows - * are written in the primary.idx to speed up range queries. - * Primary key must be a prefix of the sorting key; - * If it is nullptr, then it will be determined from sorting_key_ast. - * - * date_column_name - if not empty, the name of the column with the date used for partitioning by month; - otherwise, partition_expr_ast is used as the partitioning expression; + * See MergeTreeData constructor for comments on parameters. */ StorageMergeTree( const String & path_, @@ -173,10 +166,10 @@ protected: const ColumnsDescription & columns_, bool attach, Context & context_, - const ASTPtr & primary_key_ast_, - const ASTPtr & sorting_key_ast_, const String & date_column_name, - const ASTPtr & partition_expr_ast_, + const ASTPtr & partition_by_ast_, + const ASTPtr & order_by_ast_, + const ASTPtr & primary_key_ast_, const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported. const MergeTreeData::MergingParams & merging_params_, const MergeTreeSettings & settings_, diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 36494092371..e032d16ae0d 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -197,10 +197,10 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const String & path_, const String & database_name_, const String & name_, const ColumnsDescription & columns_, Context & context_, - const ASTPtr & primary_key_ast_, - const ASTPtr & sorting_key_ast_, const String & date_column_name, - const ASTPtr & partition_expr_ast_, + const ASTPtr & partition_by_ast_, + const ASTPtr & order_by_ast_, + const ASTPtr & primary_key_ast_, const ASTPtr & sampling_expression_, const MergeTreeData::MergingParams & merging_params_, const MergeTreeSettings & settings_, @@ -212,7 +212,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( replica_name(context.getMacros()->expand(replica_name_, database_name, table_name)), data(database_name, table_name, full_path, columns_, - context_, primary_key_ast_, sorting_key_ast_, date_column_name, partition_expr_ast_, + context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_, sampling_expression_, merging_params_, settings_, true, attach, [this] (const std::string & name) { enqueuePartForCheck(name); }), @@ -409,46 +409,44 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column /// Note: setting columns first so that the new sorting key can use new columns. setColumns(std::move(new_columns)); - ASTPtr new_sorting_key_ast = data.sorting_key_ast; - bool new_sorting_and_primary_keys_independent = data.sorting_and_primary_keys_independent; + ASTPtr new_primary_key_ast = data.primary_key_ast; + ASTPtr new_order_by_ast = data.order_by_ast; IDatabase::ASTModifier storage_modifier; if (!metadata_diff.empty()) { - new_sorting_and_primary_keys_independent = true; ParserNotEmptyExpressionList parser(false); - new_sorting_key_ast = parseQuery(parser, metadata_diff.new_sorting_key, 0); + auto new_sorting_key_expr_list = parseQuery(parser, metadata_diff.new_sorting_key, 0); + + auto tuple = makeASTFunction("tuple"); + tuple->arguments->children = new_sorting_key_expr_list->children; + new_order_by_ast = tuple; + + if (!data.primary_key_ast) + { + /// Primary and sorting key become independent after this ALTER so we have to + /// save the old ORDER BY expression as the new primary key. + new_primary_key_ast = data.order_by_ast->clone(); + } storage_modifier = [&](IAST & ast) { auto & storage_ast = typeid_cast(ast); - auto tuple = std::make_shared(); - tuple->name = "tuple"; - tuple->arguments = new_sorting_key_ast; - tuple->children.push_back(tuple->arguments); - if (!storage_ast.order_by) throw Exception("Not supported", ErrorCodes::LOGICAL_ERROR); /// TODO: better exception message - if (!storage_ast.primary_key) - { - /// Primary and sorting key become independent after this ALTER so we have to - /// save the old ORDER BY expression as the new primary key. - storage_ast.set(storage_ast.primary_key, storage_ast.order_by->clone()); - } + if (new_primary_key_ast.get() != data.primary_key_ast.get()) + storage_ast.set(storage_ast.primary_key, new_primary_key_ast); storage_ast.set(storage_ast.order_by, tuple); }; } + context.getDatabase(database_name)->alterTable(context, table_name, getColumns(), storage_modifier); + /// Even if the primary/sorting keys didn't change we must reinitialize it /// because primary key column types might have changed. - if (new_sorting_and_primary_keys_independent) - data.setPrimaryKey(data.primary_key_ast, new_sorting_key_ast); - else - data.setPrimaryKey(nullptr, new_sorting_key_ast); - - context.getDatabase(database_name)->alterTable(context, table_name, getColumns(), storage_modifier); + data.setPrimaryKey(new_order_by_ast, new_primary_key_ast); } @@ -1494,7 +1492,7 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & LOG_DEBUG(log, "Clearing column " << entry.column_name << " in part " << part->name); - auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_key_ast, false); + auto transaction = data.alterDataPart(part, columns_for_parts, nullptr, false); if (!transaction) continue; @@ -3025,7 +3023,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, for (const AlterCommand & param : params) { if (param.type == AlterCommand::MODIFY_ORDER_BY) - new_metadata.sorting_key = serializeAST(*param.sorting_key); + new_metadata.sorting_key = serializeAST(*MergeTreeData::extractKeyExpressionList(param.order_by)); } String new_metadata_str = new_metadata.toString(); if (new_metadata_str != ReplicatedMergeTreeTableMetadata(data).toString()) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index f64d793be20..7a7e81f8949 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -529,10 +529,10 @@ protected: const String & path_, const String & database_name_, const String & name_, const ColumnsDescription & columns_, Context & context_, - const ASTPtr & primary_key_ast_, - const ASTPtr & sorting_key_ast_, const String & date_column_name, - const ASTPtr & partition_expr_ast_, + const ASTPtr & partition_by_ast_, + const ASTPtr & order_by_ast_, + const ASTPtr & primary_key_ast_, const ASTPtr & sampling_expression_, const MergeTreeData::MergingParams & merging_params_, const MergeTreeSettings & settings_, From 7167bfd7b365538f7a91c4307ad77e552ab4e8c1 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 8 Nov 2018 19:39:43 +0300 Subject: [PATCH 17/69] fix prewhere with FINAL for VersionedCollapsingMergeTree [#CLICKHOUSE-3859] --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 16 +++------------ dbms/src/Interpreters/ExpressionAnalyzer.h | 5 ++--- .../Interpreters/InterpreterSelectQuery.cpp | 20 ++++++++++++++++--- dbms/src/Storages/IStorage.h | 8 ++++---- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 13 +++++++++--- dbms/src/Storages/MergeTree/MergeTreeData.h | 5 +++++ dbms/src/Storages/StorageMergeTree.h | 4 ++-- .../src/Storages/StorageReplicatedMergeTree.h | 4 ++-- .../00712_prewhere_with_final.reference | 1 + .../0_stateless/00712_prewhere_with_final.sql | 15 +++++++------- 10 files changed, 53 insertions(+), 38 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 69ab1d04302..63d6b6c87d1 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -1425,24 +1425,14 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty return true; } -bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool only_types, - const ASTPtr & sampling_expression, const ASTPtr & primary_expression) +bool ExpressionAnalyzer::appendPrewhere( + ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns) { assertSelect(); if (!select_query->prewhere_expression) return false; - Names additional_required_mergetree_columns; - if (sampling_expression) - additional_required_mergetree_columns = ExpressionAnalyzer(sampling_expression, context, storage).getRequiredSourceColumns(); - if (primary_expression) - { - auto required_primary_columns = ExpressionAnalyzer(primary_expression, context, storage).getRequiredSourceColumns(); - additional_required_mergetree_columns.insert(additional_required_mergetree_columns.end(), - required_primary_columns.begin(), required_primary_columns.end()); - } - initChain(chain, source_columns); auto & step = chain.getLastStep(); getRootActions(select_query->prewhere_expression, only_types, step.actions); @@ -1460,7 +1450,7 @@ bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool onl /// Add required columns to required output in order not to remove them after prewhere execution. /// TODO: add sampling and final execution to common chain. - for (const auto & column : additional_required_mergetree_columns) + for (const auto & column : additional_required_columns) { if (required_source_columns.count(column)) { diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index b0a3948def2..47c53d691cd 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -86,9 +86,8 @@ public: bool appendArrayJoin(ExpressionActionsChain & chain, bool only_types); bool appendJoin(ExpressionActionsChain & chain, bool only_types); /// remove_filter is set in ExpressionActionsChain::finalize(); - /// sampling_expression and primary_expression are needed in order to not remove columns are used in it. - bool appendPrewhere(ExpressionActionsChain & chain, bool only_types, - const ASTPtr & sampling_expression, const ASTPtr & primary_expression); + /// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier). + bool appendPrewhere(ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns); bool appendWhere(ExpressionActionsChain & chain, bool only_types); bool appendGroupBy(ExpressionActionsChain & chain, bool only_types); void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index ddac4a07611..01157907365 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -342,9 +342,23 @@ InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpression { ExpressionActionsChain chain(context); - ASTPtr sampling_expression = (storage && query.sample_size()) ? storage->getSamplingExpression() : nullptr; - ASTPtr primary_expression = (storage && query.final()) ? storage->getPrimaryExpression() : nullptr; - if (query_analyzer->appendPrewhere(chain, !res.first_stage, sampling_expression, primary_expression)) + Names additional_required_columns_after_prewhere; + + if (storage && query.sample_size()) + { + Names columns_for_sampling = storage->getColumnsRequiredForSampling(); + additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), + columns_for_sampling.begin(), columns_for_sampling.end()); + } + + if (storage && query.final()) + { + Names columns_for_final = storage->getColumnsRequiredForFinal(); + additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), + columns_for_final.begin(), columns_for_final.end()); + } + + if (query_analyzer->appendPrewhere(chain, !res.first_stage, additional_required_columns_after_prewhere)) { has_prewhere = true; diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index a0a61035580..8eeb24c4bf4 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -343,11 +343,11 @@ public: /// Returns data path if storage supports it, empty string otherwise. virtual String getDataPath() const { return {}; } - /// Returns sampling expression for storage or nullptr if there is no. - virtual ASTPtr getSamplingExpression() const { return nullptr; } + /// Returns additional columns that need to be read for sampling to work. + virtual Names getColumnsRequiredForSampling() const { return {}; } - /// Returns primary expression for storage or nullptr if there is no. - virtual ASTPtr getPrimaryExpression() const { return nullptr; } + /// Returns additional columns that need to be read for FINAL to work. + virtual Names getColumnsRequiredForFinal() const { return {}; } using ITableDeclaration::ITableDeclaration; using std::enable_shared_from_this::shared_from_this; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index c3b7bb332e4..819e9c73166 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -116,9 +116,16 @@ MergeTreeData::MergeTreeData( setPrimaryKey(order_by_ast_, primary_key_ast_); - if (sampling_expression && (!primary_key_sample.has(sampling_expression->getColumnName())) - && !attach && !settings.compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility. - throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS); + if (sampling_expression) + { + if (!primary_key_sample.has(sampling_expression->getColumnName()) + && !attach && !settings.compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility. + throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS); + + columns_required_for_sampling = ExpressionAnalyzer( + sampling_expression, context, nullptr, getColumns().getAllPhysical()) + .getRequiredSourceColumns(); + } MergeTreeDataFormatVersion min_format_version(0); if (!date_column_name.empty()) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index afcd18caac6..b15993cb3cc 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -509,6 +509,9 @@ public: ExpressionActionsPtr getSortingKeyExpression() const { return sorting_key_expr; } Names getSortingKeyColumns() const { return sorting_key_columns; } + Names getColumnsRequiredForSampling() const { return columns_required_for_sampling; } + Names getColumnsRequiredForFinal() const { return sorting_key_expr->getRequiredColumns(); } + /// Check that the part is not broken and calculate the checksums for it if they are not present. MutableDataPartPtr loadPartAndFixMetadata(const String & relative_path); @@ -604,6 +607,8 @@ private: /// Names of columns for primary key. Names primary_key_columns; + Names columns_required_for_sampling; + String database_name; String table_name; String full_path; diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index fac02d7528a..8b5dad77081 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -94,9 +94,9 @@ public: String getDataPath() const override { return full_path; } - ASTPtr getSamplingExpression() const override { return data.sampling_expression; } + Names getColumnsRequiredForSampling() const override { return data.getColumnsRequiredForSampling(); } - ASTPtr getPrimaryExpression() const override { return data.primary_key_ast; } + Names getColumnsRequiredForFinal() const override { return data.getColumnsRequiredForFinal(); } private: String path; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 7a7e81f8949..8bb3448b4ae 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -194,9 +194,9 @@ public: String getDataPath() const override { return full_path; } - ASTPtr getSamplingExpression() const override { return data.sampling_expression; } + Names getColumnsRequiredForSampling() const override { return data.getColumnsRequiredForSampling(); } - ASTPtr getPrimaryExpression() const override { return data.primary_key_ast; } + Names getColumnsRequiredForFinal() const override { return data.getColumnsRequiredForFinal(); } private: /// Delete old parts from disk and from ZooKeeper. diff --git a/dbms/tests/queries/0_stateless/00712_prewhere_with_final.reference b/dbms/tests/queries/0_stateless/00712_prewhere_with_final.reference index d00491fd7e5..a247b96d92d 100644 --- a/dbms/tests/queries/0_stateless/00712_prewhere_with_final.reference +++ b/dbms/tests/queries/0_stateless/00712_prewhere_with_final.reference @@ -1 +1,2 @@ 1 +123 diff --git a/dbms/tests/queries/0_stateless/00712_prewhere_with_final.sql b/dbms/tests/queries/0_stateless/00712_prewhere_with_final.sql index 6379cae8a34..88c0ad22b26 100644 --- a/dbms/tests/queries/0_stateless/00712_prewhere_with_final.sql +++ b/dbms/tests/queries/0_stateless/00712_prewhere_with_final.sql @@ -1,13 +1,12 @@ drop table if exists test.trepl; -create table test.trepl -( -d Date, -a Int32, -b Int32 -) engine = ReplacingMergeTree(d, (a,b), 8192); - - +create table test.trepl(d Date,a Int32, b Int32) engine = ReplacingMergeTree(d, (a,b), 8192); insert into test.trepl values ('2018-09-19', 1, 1); select b from test.trepl FINAL prewhere a < 1000; drop table test.trepl; + +drop table if exists test.versioned_collapsing; +create table test.versioned_collapsing(d Date, x UInt32, sign Int8, version UInt32) engine = VersionedCollapsingMergeTree(d, x, 8192, sign, version); +insert into test.versioned_collapsing values ('2018-09-19', 123, 1, 0); +select x from test.versioned_collapsing FINAL prewhere version < 1000; +drop table test.versioned_collapsing; From 2cc0aae82c356cc95af3f459c8048bb73a490ad4 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 9 Nov 2018 22:01:39 +0300 Subject: [PATCH 18/69] rearrange AST members in MergeTreeData [#CLICKHOUSE-3859] --- .../src/Interpreters/MutationsInterpreter.cpp | 4 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 16 ++--- dbms/src/Storages/MergeTree/MergeTreeData.h | 61 +++++++++---------- .../MergeTree/MergeTreeDataMergerMutator.cpp | 8 +-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 24 ++++---- .../MergeTree/MergeTreeDataWriter.cpp | 4 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 2 +- .../ReplicatedMergeTreeTableMetadata.cpp | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 10 +-- dbms/src/Storages/StorageMergeTree.cpp | 4 +- dbms/src/Storages/StorageMergeTree.h | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 4 +- .../src/Storages/StorageReplicatedMergeTree.h | 2 +- 14 files changed, 72 insertions(+), 73 deletions(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index bf0c856da73..d41c6b38faf 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -102,11 +102,11 @@ static NameSet getKeyColumns(const StoragePtr & storage) for (const String & col : merge_tree_data->partition_key_expr->getRequiredColumns()) key_columns.insert(col); - auto sorting_key_expr = merge_tree_data->getSortingKeyExpression(); + auto sorting_key_expr = merge_tree_data->sorting_key_expr; if (sorting_key_expr) for (const String & col : sorting_key_expr->getRequiredColumns()) key_columns.insert(col); - /// We don't process sampling_expression separately because it must be among the primary key columns. + /// We don't process sample_by_ast separately because it must be among the primary key columns. if (!merge_tree_data->merging_params.sign_column.empty()) key_columns.insert(merge_tree_data->merging_params.sign_column); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 819e9c73166..e12cc72c6fc 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -90,7 +90,7 @@ MergeTreeData::MergeTreeData( const ASTPtr & partition_by_ast_, const ASTPtr & order_by_ast_, const ASTPtr & primary_key_ast_, - const ASTPtr & sampling_expression_, + const ASTPtr & sample_by_ast_, const MergingParams & merging_params_, const MergeTreeSettings & settings_, bool require_part_metadata_, @@ -98,11 +98,11 @@ MergeTreeData::MergeTreeData( BrokenPartCallback broken_part_callback_) : ITableDeclaration{columns_}, context(context_), - sampling_expression(sampling_expression_), - index_granularity(settings_.index_granularity), merging_params(merging_params_), + index_granularity(settings_.index_granularity), settings(settings_), partition_by_ast(partition_by_ast_), + sample_by_ast(sample_by_ast_), require_part_metadata(require_part_metadata_), database_name(database_), table_name(table_), full_path(full_path_), @@ -116,14 +116,16 @@ MergeTreeData::MergeTreeData( setPrimaryKey(order_by_ast_, primary_key_ast_); - if (sampling_expression) + if (sample_by_ast) { - if (!primary_key_sample.has(sampling_expression->getColumnName()) + sampling_expr_column_name = sample_by_ast->getColumnName(); + + if (!primary_key_sample.has(sampling_expr_column_name) && !attach && !settings.compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility. throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS); columns_required_for_sampling = ExpressionAnalyzer( - sampling_expression, context, nullptr, getColumns().getAllPhysical()) + sample_by_ast, context, nullptr, getColumns().getAllPhysical()) .getRequiredSourceColumns(); } @@ -943,7 +945,7 @@ void MergeTreeData::checkAlter(const AlterCommands & commands) for (const String & col : sorting_key_expr->getRequiredColumns()) columns_alter_metadata_only.insert(col); - /// We don't process sampling_expression separately because it must be among the primary key columns + /// We don't process sample_by_ast separately because it must be among the primary key columns /// and we don't process primary_key_expr separately because it is a prefix of sorting_key_expr. } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index b15993cb3cc..baab7396b77 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -305,7 +305,7 @@ public: const ASTPtr & partition_by_ast_, const ASTPtr & order_by_ast_, const ASTPtr & primary_key_ast_, - const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported. + const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. const MergingParams & merging_params_, const MergeTreeSettings & settings_, bool require_part_metadata_, @@ -315,7 +315,6 @@ public: /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks); - bool supportsSampling() const { return sampling_expression != nullptr; } bool supportsPrewhere() const { return true; } bool supportsFinal() const @@ -502,16 +501,14 @@ public: static ASTPtr extractKeyExpressionList(const ASTPtr & node); bool hasPrimaryKey() const { return !primary_key_columns.empty(); } - ExpressionActionsPtr getPrimaryKeyExpression() const { return primary_key_expr; } - Names getPrimaryKeyColumns() const { return primary_key_columns; } - bool hasSortingKey() const { return !sorting_key_columns.empty(); } - ExpressionActionsPtr getSortingKeyExpression() const { return sorting_key_expr; } - Names getSortingKeyColumns() const { return sorting_key_columns; } - Names getColumnsRequiredForSampling() const { return columns_required_for_sampling; } Names getColumnsRequiredForFinal() const { return sorting_key_expr->getRequiredColumns(); } + bool supportsSampling() const { return sample_by_ast != nullptr; } + ASTPtr getSamplingExpression() const { return sample_by_ast; } + Names getColumnsRequiredForSampling() const { return columns_required_for_sampling; } + /// Check that the part is not broken and calculate the checksums for it if they are not present. MutableDataPartPtr loadPartAndFixMetadata(const String & relative_path); @@ -560,21 +557,10 @@ public: MergeTreeDataFormatVersion format_version; Context & context; - const ASTPtr sampling_expression; - const size_t index_granularity; /// Merging params - what additional actions to perform during merge. const MergingParams merging_params; - const MergeTreeSettings settings; - - ASTPtr order_by_ast; - ASTPtr primary_key_ast; - - Block primary_key_sample; - DataTypes primary_key_data_types; - - ASTPtr partition_by_ast; ExpressionActionsPtr partition_key_expr; Block partition_key_sample; @@ -583,6 +569,22 @@ public: DataTypes minmax_idx_column_types; Int64 minmax_idx_date_column_pos = -1; /// In a common case minmax index includes a date column. + /// Names of columns for primary key + secondary sorting columns. + Names sorting_key_columns; + ExpressionActionsPtr sorting_key_expr; + + /// Names of columns for primary key. + Names primary_key_columns; + ExpressionActionsPtr primary_key_expr; + Block primary_key_sample; + DataTypes primary_key_data_types; + + String sampling_expr_column_name; + Names columns_required_for_sampling; + + const size_t index_granularity; + const MergeTreeSettings settings; + /// Limiting parallel sends per one table, used in DataPartsExchange std::atomic_uint current_table_sends {0}; @@ -592,23 +594,18 @@ public: private: friend struct MergeTreeDataPart; friend class StorageMergeTree; - friend class ReplicatedMergeTreeAlterThread; - friend class MergeTreeDataMergerMutator; - friend class StorageMergeTree; friend class StorageReplicatedMergeTree; + friend class MergeTreeDataMergerMutator; + friend class ReplicatedMergeTreeAlterThread; + friend struct ReplicatedMergeTreeTableMetadata; + + ASTPtr partition_by_ast; + ASTPtr order_by_ast; + ASTPtr primary_key_ast; + const ASTPtr sample_by_ast; bool require_part_metadata; - ExpressionActionsPtr sorting_key_expr; - /// Names of columns for primary key + secondary sorting columns. - Names sorting_key_columns; - - ExpressionActionsPtr primary_key_expr; - /// Names of columns for primary key. - Names primary_key_columns; - - Names columns_required_for_sampling; - String database_name; String table_name; String full_path; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 9aec7aa42d1..d2eeba125fa 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -549,7 +549,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor NamesAndTypesList gathering_columns, merging_columns; Names gathering_column_names, merging_column_names; extractMergingAndGatheringColumns( - all_columns, data.getSortingKeyExpression(), + all_columns, data.sorting_key_expr, data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names); MergeTreeData::MutableDataPartPtr new_data_part = std::make_shared( @@ -632,12 +632,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (data.hasPrimaryKey()) src_streams.emplace_back(std::make_shared( - std::make_shared(BlockInputStreamPtr(std::move(input)), data.getSortingKeyExpression()))); + std::make_shared(BlockInputStreamPtr(std::move(input)), data.sorting_key_expr))); else src_streams.emplace_back(std::move(input)); } - Names sort_columns = data.getSortingKeyColumns(); + Names sort_columns = data.sorting_key_columns; SortDescription sort_description; size_t sort_columns_size = sort_columns.size(); sort_description.reserve(sort_columns_size); @@ -905,7 +905,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor if (data.hasPrimaryKey()) in = std::make_shared( - std::make_shared(in, data.getPrimaryKeyExpression())); + std::make_shared(in, data.primary_key_expr)); MergeTreeDataPart::MinMaxIndex minmax_idx; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 6fb9f445ef9..4e31d10fa40 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -211,11 +211,11 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( data.check(real_column_names); const Settings & settings = context.getSettingsRef(); - Names primary_key_columns = data.getPrimaryKeyColumns(); + Names primary_key_columns = data.primary_key_columns; KeyCondition key_condition( query_info, context, available_real_and_virtual_columns, - primary_key_columns, data.getPrimaryKeyExpression()); + primary_key_columns, data.primary_key_expr); if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue()) { @@ -372,14 +372,14 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( if (use_sampling) { - if (!data.sampling_expression) + if (!data.supportsSampling()) throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED); if (sample_factor_column_queried && relative_sample_size != RelativeSize(0)) used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); RelativeSize size_of_universum = 0; - DataTypePtr type = data.primary_key_sample.getByName(data.sampling_expression->getColumnName()).type; + DataTypePtr type = data.primary_key_sample.getByName(data.sampling_expr_column_name).type; if (typeid_cast(type.get())) size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); @@ -446,11 +446,11 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( if (has_lower_limit) { - if (!key_condition.addCondition(data.sampling_expression->getColumnName(), Range::createLeftBounded(lower, true))) + if (!key_condition.addCondition(data.sampling_expr_column_name, Range::createLeftBounded(lower, true))) throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); ASTPtr args = std::make_shared(); - args->children.push_back(data.sampling_expression); + args->children.push_back(data.getSamplingExpression()); args->children.push_back(std::make_shared(lower)); lower_function = std::make_shared(); @@ -463,11 +463,11 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( if (has_upper_limit) { - if (!key_condition.addCondition(data.sampling_expression->getColumnName(), Range::createRightBounded(upper, false))) + if (!key_condition.addCondition(data.sampling_expr_column_name, Range::createRightBounded(upper, false))) throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); ASTPtr args = std::make_shared(); - args->children.push_back(data.sampling_expression); + args->children.push_back(data.getSamplingExpression()); args->children.push_back(std::make_shared(upper)); upper_function = std::make_shared(); @@ -492,7 +492,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( filter_expression = ExpressionAnalyzer(filter_function, context, nullptr, available_real_columns).getActions(false); - /// Add columns needed for `sampling_expression` to `column_names_to_read`. + /// Add columns needed for `sample_by_ast` to `column_names_to_read`. std::vector add_columns = filter_expression->getRequiredColumns(); column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); std::sort(column_names_to_read.begin(), column_names_to_read.end()); @@ -554,7 +554,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts( if (select.final()) { /// Add columns needed to calculate the sorting expression and the sign. - std::vector add_columns = data.getSortingKeyExpression()->getRequiredColumns(); + std::vector add_columns = data.sorting_key_expr->getRequiredColumns(); column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); if (!data.merging_params.sign_column.empty()) @@ -782,10 +782,10 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal prewhere_info, true, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, true, virt_columns, part.part_index_in_query); - to_merge.emplace_back(std::make_shared(source_stream, data.getSortingKeyExpression())); + to_merge.emplace_back(std::make_shared(source_stream, data.sorting_key_expr)); } - Names sort_columns = data.getSortingKeyColumns(); + Names sort_columns = data.sorting_key_columns; SortDescription sort_description; size_t sort_columns_size = sort_columns.size(); sort_description.reserve(sort_columns_size); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index b88a2f13960..345e4bd8192 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -176,9 +176,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa /// If we need to calculate some columns to sort. if (data.hasSortingKey()) - data.getSortingKeyExpression()->execute(block); + data.sorting_key_expr->execute(block); - Names sort_columns = data.getSortingKeyColumns(); + Names sort_columns = data.sorting_key_columns; SortDescription sort_description; size_t sort_columns_size = sort_columns.size(); sort_description.reserve(sort_columns_size); diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 2c96bbf3781..3fc622ea6c4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -40,7 +40,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( const MergeTreeData & data, const Names & column_names, Logger * log) - : primary_key_columns{ext::collection_cast(data.getPrimaryKeyColumns())}, + : primary_key_columns{ext::collection_cast(data.primary_key_columns)}, table_columns{ext::map(data.getColumns().getAllPhysical(), [] (const NameAndTypePair & col) { return col.name; })}, block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, context, data.getColumns().getAllPhysical())}, diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 05098e1b82b..e81336a3910 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -398,7 +398,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm /// The set of written offset columns so that you do not write shared offsets of nested structures columns several times WrittenOffsetColumns offset_columns; - auto primary_key_column_names = storage.getPrimaryKeyColumns(); + auto primary_key_column_names = storage.primary_key_columns; /// Here we will add the columns related to the Primary Key, then write the index. std::vector primary_key_columns(primary_key_column_names.size()); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 8f9ab03cd55..ae5249d3d16 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -27,7 +27,7 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos]; - sampling_expression = formattedAST(data.sampling_expression); + sampling_expression = formattedAST(data.sample_by_ast); index_granularity = data.index_granularity; merging_params_mode = static_cast(data.merging_params.mode); sign_column = data.merging_params.sign_column; diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index 69e6053eedf..72ca32f492b 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -549,7 +549,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) ASTPtr partition_by_ast; ASTPtr order_by_ast; ASTPtr primary_key_ast; - ASTPtr sampling_expression; + ASTPtr sample_by_ast; MergeTreeSettings storage_settings = args.context.getMergeTreeSettings(); if (is_extended_storage_def) @@ -568,7 +568,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) primary_key_ast = args.storage_def->primary_key->ptr(); if (args.storage_def->sample_by) - sampling_expression = args.storage_def->sample_by->ptr(); + sample_by_ast = args.storage_def->sample_by->ptr(); storage_settings.loadFromQuery(*args.storage_def); } @@ -577,7 +577,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// If there is an expression for sampling. MergeTree(date, [sample_key], primary_key, index_granularity) if (engine_args.size() == 4) { - sampling_expression = engine_args[1]; + sample_by_ast = engine_args[1]; engine_args.erase(engine_args.begin() + 1); } @@ -606,13 +606,13 @@ static StoragePtr create(const StorageFactory::Arguments & args) zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name, args.columns, args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast, - sampling_expression, merging_params, storage_settings, + sample_by_ast, merging_params, storage_settings, args.has_force_restore_data_flag); else return StorageMergeTree::create( args.data_path, args.database_name, args.table_name, args.columns, args.attach, args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast, - sampling_expression, merging_params, storage_settings, + sample_by_ast, merging_params, storage_settings, args.has_force_restore_data_flag); } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 6af8cfdcace..7cb624e216a 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -50,7 +50,7 @@ StorageMergeTree::StorageMergeTree( const ASTPtr & partition_by_ast_, const ASTPtr & order_by_ast_, const ASTPtr & primary_key_ast_, - const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported. + const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. const MergeTreeData::MergingParams & merging_params_, const MergeTreeSettings & settings_, bool has_force_restore_data_flag) @@ -59,7 +59,7 @@ StorageMergeTree::StorageMergeTree( data(database_name, table_name, full_path, columns_, context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_, - sampling_expression_, merging_params_, + sample_by_ast_, merging_params_, settings_, false, attach), reader(data), writer(data), merger_mutator(data, context.getBackgroundPool()), log(&Logger::get(database_name_ + "." + table_name + " (StorageMergeTree)")) diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 8b5dad77081..8217ad8c754 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -170,7 +170,7 @@ protected: const ASTPtr & partition_by_ast_, const ASTPtr & order_by_ast_, const ASTPtr & primary_key_ast_, - const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported. + const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported. const MergeTreeData::MergingParams & merging_params_, const MergeTreeSettings & settings_, bool has_force_restore_data_flag); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index e032d16ae0d..caaf5246ffa 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -201,7 +201,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const ASTPtr & partition_by_ast_, const ASTPtr & order_by_ast_, const ASTPtr & primary_key_ast_, - const ASTPtr & sampling_expression_, + const ASTPtr & sample_by_ast_, const MergeTreeData::MergingParams & merging_params_, const MergeTreeSettings & settings_, bool has_force_restore_data_flag) @@ -213,7 +213,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( data(database_name, table_name, full_path, columns_, context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_, - sampling_expression_, merging_params_, + sample_by_ast_, merging_params_, settings_, true, attach, [this] (const std::string & name) { enqueuePartForCheck(name); }), reader(data), writer(data), merger_mutator(data, context.getBackgroundPool()), queue(*this), diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 8bb3448b4ae..9c95e92f8c4 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -533,7 +533,7 @@ protected: const ASTPtr & partition_by_ast_, const ASTPtr & order_by_ast_, const ASTPtr & primary_key_ast_, - const ASTPtr & sampling_expression_, + const ASTPtr & sample_by_ast_, const MergeTreeData::MergingParams & merging_params_, const MergeTreeSettings & settings_, bool has_force_restore_data_flag); From b071547b2de7fd910a44c8ad534562cf3cf53d55 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Tue, 13 Nov 2018 15:51:55 +0300 Subject: [PATCH 19/69] add checks for ALTER MODIFY ORDER BY [#CLICKHOUSE-3859] --- dbms/src/Core/NamesAndTypes.cpp | 13 +++ dbms/src/Core/NamesAndTypes.h | 3 + dbms/src/Storages/AlterCommands.cpp | 34 +++++-- dbms/src/Storages/AlterCommands.h | 4 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 93 ++++++++++++++++--- dbms/src/Storages/MergeTree/MergeTreeData.h | 3 +- dbms/src/Storages/StorageMergeTree.cpp | 33 ++----- .../Storages/StorageReplicatedMergeTree.cpp | 39 ++++---- 8 files changed, 156 insertions(+), 66 deletions(-) diff --git a/dbms/src/Core/NamesAndTypes.cpp b/dbms/src/Core/NamesAndTypes.cpp index cbc4f4b159f..09985d97463 100644 --- a/dbms/src/Core/NamesAndTypes.cpp +++ b/dbms/src/Core/NamesAndTypes.cpp @@ -84,6 +84,19 @@ size_t NamesAndTypesList::sizeOfDifference(const NamesAndTypesList & rhs) const return (std::unique(vector.begin(), vector.end()) - vector.begin()) * 2 - size() - rhs.size(); } +void NamesAndTypesList::getDifference(const NamesAndTypesList & rhs, NamesAndTypesList & deleted, NamesAndTypesList & added) const +{ + NamesAndTypes lhs_vector(begin(), end()); + std::sort(lhs_vector.begin(), lhs_vector.end()); + NamesAndTypes rhs_vector(rhs.begin(), rhs.end()); + std::sort(rhs_vector.begin(), rhs_vector.end()); + + std::set_difference(lhs_vector.begin(), lhs_vector.end(), rhs_vector.begin(), rhs_vector.end(), + std::back_inserter(deleted)); + std::set_difference(rhs_vector.begin(), rhs_vector.end(), lhs_vector.begin(), lhs_vector.end(), + std::back_inserter(added)); +} + Names NamesAndTypesList::getNames() const { Names res; diff --git a/dbms/src/Core/NamesAndTypes.h b/dbms/src/Core/NamesAndTypes.h index 849141e43f7..1b7d2d1f175 100644 --- a/dbms/src/Core/NamesAndTypes.h +++ b/dbms/src/Core/NamesAndTypes.h @@ -58,6 +58,9 @@ public: /// (in other words, the added and deleted columns are counted once, the columns that changed the type - twice). size_t sizeOfDifference(const NamesAndTypesList & rhs) const; + /// If an element changes type, it is present both in deleted (with the old type) and in added (with the new type). + void getDifference(const NamesAndTypesList & rhs, NamesAndTypesList & deleted, NamesAndTypesList & added) const; + Names getNames() const; DataTypes getTypes() const; diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 0bb4691ea97..94f4a59bf4d 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -22,6 +22,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; } @@ -110,7 +111,7 @@ static bool namesEqual(const String & name_without_dot, const DB::NameAndTypePai return (name_with_dot == name_type.name.substr(0, name_without_dot.length() + 1) || name_without_dot == name_type.name); } -void AlterCommand::apply(ColumnsDescription & columns_description) const +void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr * order_by_ast, ASTPtr * primary_key_ast) const { if (type == ADD_COLUMN) { @@ -240,22 +241,43 @@ void AlterCommand::apply(ColumnsDescription & columns_description) const /// both old and new columns have default expression, update it columns_description.defaults[column_name].expression = default_expression; } - else if (type == MODIFY_PRIMARY_KEY || type == MODIFY_ORDER_BY) + else if (type == MODIFY_PRIMARY_KEY) { - /// This have no relation to changing the list of columns. - /// TODO Check that all columns exist, that only columns with constant defaults are added. + if (!primary_key_ast || !order_by_ast) + throw Exception("ALTER MODIFY PRIMARY KEY is not supported for this type of tables", + ErrorCodes::BAD_ARGUMENTS); + + if (!(*primary_key_ast)) + *order_by_ast = primary_key; + else + *primary_key_ast = primary_key; + } + else if (type == MODIFY_ORDER_BY) + { + if (!primary_key_ast || !order_by_ast) + throw Exception("ALTER MODIFY PRIMARY KEY is not supported for this type of tables", + ErrorCodes::BAD_ARGUMENTS); + + if (!(*primary_key_ast)) + { + /// Primary and sorting key become independent after this ALTER so we have to + /// save the old ORDER BY expression as the new primary key. + *primary_key_ast = (*order_by_ast)->clone(); + } + + *order_by_ast = order_by; } else throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR); } -void AlterCommands::apply(ColumnsDescription & columns_description) const +void AlterCommands::apply(ColumnsDescription & columns_description, ASTPtr * order_by_ast, ASTPtr * primary_key_ast) const { auto new_columns_description = columns_description; for (const AlterCommand & command : *this) - command.apply(new_columns_description); + command.apply(new_columns_description, order_by_ast, primary_key_ast); columns_description = std::move(new_columns_description); } diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index e0ec754bf7a..8397873d073 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -56,7 +56,7 @@ struct AlterCommand static std::optional parse(const ASTAlterCommand * command); - void apply(ColumnsDescription & columns_description) const; + void apply(ColumnsDescription & columns_description, ASTPtr * order_by_ast = nullptr, ASTPtr * primary_key_ast = nullptr) const; }; @@ -66,7 +66,7 @@ class Context; class AlterCommands : public std::vector { public: - void apply(ColumnsDescription & columns_description) const; + void apply(ColumnsDescription & columns_description, ASTPtr * order_by_ast = nullptr, ASTPtr * primary_key_ast = nullptr) const; void validate(const IStorage & table, const Context & context); }; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 21931dac339..3209ea03883 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -96,8 +96,7 @@ MergeTreeData::MergeTreeData( bool require_part_metadata_, bool attach, BrokenPartCallback broken_part_callback_) - : ITableDeclaration{columns_}, - context(context_), + : context(context_), merging_params(merging_params_), index_granularity(settings_.index_granularity), settings(settings_), @@ -111,11 +110,11 @@ MergeTreeData::MergeTreeData( data_parts_by_info(data_parts_indexes.get()), data_parts_by_state_and_info(data_parts_indexes.get()) { + setPrimaryKeyAndColumns(order_by_ast_, primary_key_ast_, columns_); + /// NOTE: using the same columns list as is read when performing actual merges. merging_params.check(getColumns().getAllPhysical()); - setPrimaryKey(order_by_ast_, primary_key_ast_); - if (sample_by_ast) { sampling_expr_column_name = sample_by_ast->getColumnName(); @@ -149,6 +148,7 @@ MergeTreeData::MergeTreeData( } else { + is_custom_partitioned = true; initPartitionKey(); min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING; } @@ -214,7 +214,8 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam } -void MergeTreeData::setPrimaryKey(const ASTPtr & new_order_by_ast, ASTPtr new_primary_key_ast) +void MergeTreeData::setPrimaryKeyAndColumns( + const ASTPtr & new_order_by_ast, ASTPtr new_primary_key_ast, const ColumnsDescription & new_columns, bool only_check) { if (!new_order_by_ast) throw Exception("ORDER BY cannot be empty", ErrorCodes::BAD_ARGUMENTS); @@ -253,7 +254,51 @@ void MergeTreeData::setPrimaryKey(const ASTPtr & new_order_by_ast, ASTPtr new_pr } } - auto all_columns = getColumns().getAllPhysical(); + auto all_columns = new_columns.getAllPhysical(); + + if (order_by_ast && only_check) + { + /// This is ALTER, not CREATE/ATTACH TABLE. Let us check that all new columns used in the sorting key + /// expression have just been added (so that the sorting order is guaranteed to be valid with the new key). + + ASTPtr added_key_column_expr_list = std::make_shared(); + for (size_t new_i = 0, old_i = 0; new_i < sorting_key_size; ++new_i) + { + if (old_i < sorting_key_columns.size()) + { + if (new_sorting_key_columns[new_i] != sorting_key_columns[old_i]) + added_key_column_expr_list->children.push_back(new_sorting_key_expr_list->children[new_i]); + else + ++old_i; + } + else + added_key_column_expr_list->children.push_back(new_sorting_key_expr_list->children[new_i]); + } + + if (!added_key_column_expr_list->children.empty()) + { + auto syntax = SyntaxAnalyzer(context, {}).analyze(added_key_column_expr_list, all_columns); + Names used_columns = ExpressionAnalyzer(added_key_column_expr_list, syntax, context) + .getRequiredSourceColumns(); + + NamesAndTypesList deleted_columns; + NamesAndTypesList added_columns; + getColumns().getAllPhysical().getDifference(all_columns, deleted_columns, added_columns); + + for (const String & col : used_columns) + { + if (!added_columns.contains(col) || deleted_columns.contains(col)) + throw Exception("Existing column " + col + " is used in the expression that was " + "added to the sorting key. You can add expressions that use only the newly added columns", + ErrorCodes::BAD_ARGUMENTS); + + if (new_columns.defaults.count(col)) + throw Exception("Newly added column " + col + " has a default expression, so adding " + "expressions that use it to the sorting key is forbidden", + ErrorCodes::BAD_ARGUMENTS); + } + } + } auto new_sorting_key_syntax = SyntaxAnalyzer(context, {}).analyze(new_sorting_key_expr_list, all_columns); auto new_sorting_key_expr = ExpressionAnalyzer(new_sorting_key_expr_list, new_sorting_key_syntax, context) @@ -277,17 +322,23 @@ void MergeTreeData::setPrimaryKey(const ASTPtr & new_order_by_ast, ASTPtr new_pr new_primary_key_data_types.push_back(elem.type); } - order_by_ast = new_order_by_ast; - sorting_key_columns = std::move(new_sorting_key_columns); - sorting_key_expr = std::move(new_sorting_key_expr); + if (!only_check) + { + setColumns(new_columns); - primary_key_ast = new_primary_key_ast; - primary_key_columns = std::move(new_primary_key_columns); - primary_key_expr = std::move(new_primary_key_expr); - primary_key_sample = std::move(new_primary_key_sample); - primary_key_data_types = std::move(new_primary_key_data_types); + order_by_ast = new_order_by_ast; + sorting_key_columns = std::move(new_sorting_key_columns); + sorting_key_expr = std::move(new_sorting_key_expr); + + primary_key_ast = new_primary_key_ast; + primary_key_columns = std::move(new_primary_key_columns); + primary_key_expr = std::move(new_primary_key_expr); + primary_key_sample = std::move(new_primary_key_sample); + primary_key_data_types = std::move(new_primary_key_data_types); + } } + ASTPtr MergeTreeData::extractKeyExpressionList(const ASTPtr & node) { if (!node) @@ -922,7 +973,9 @@ void MergeTreeData::checkAlter(const AlterCommands & commands) { /// Check that needed transformations can be applied to the list of columns without considering type conversions. auto new_columns = getColumns(); - commands.apply(new_columns); + ASTPtr new_order_by_ast = order_by_ast; + ASTPtr new_primary_key_ast = primary_key_ast; + commands.apply(new_columns, &new_order_by_ast, &new_primary_key_ast); /// Set of columns that shouldn't be altered. NameSet columns_alter_forbidden; @@ -979,8 +1032,18 @@ void MergeTreeData::checkAlter(const AlterCommands & commands) "ALTER of key column " + command.column_name + " must be metadata-only", ErrorCodes::ILLEGAL_COLUMN); } + + if (command.type == AlterCommand::MODIFY_ORDER_BY) + { + if (!is_custom_partitioned) + throw Exception( + "ALTER MODIFY ORDER BY is not supported for default-partitioned tables created with the old syntax", + ErrorCodes::BAD_ARGUMENTS); + } } + setPrimaryKeyAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, /* only_check = */ true); + /// Check that type conversions are possible. ExpressionActionsPtr unused_expression; NameToNameMap unused_map; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index bf590a0b5a4..d6434fdf7cb 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -561,6 +561,7 @@ public: /// Merging params - what additional actions to perform during merge. const MergingParams merging_params; + bool is_custom_partitioned = false; ExpressionActionsPtr partition_key_expr; Block partition_key_sample; @@ -707,7 +708,7 @@ private: /// The same for clearOldTemporaryDirectories. std::mutex clear_old_temporary_directories_mutex; - void setPrimaryKey(const ASTPtr & new_order_by_ast, ASTPtr new_primary_key_ast); + void setPrimaryKeyAndColumns(const ASTPtr & new_order_by_ast, ASTPtr new_primary_key_ast, const ColumnsDescription & new_columns, bool only_check = false); void initPartitionKey(); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 662ae10c6c9..e4be8330f34 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -196,43 +196,25 @@ void StorageMergeTree::alter( data.checkAlter(params); auto new_columns = data.getColumns(); - params.apply(new_columns); - - std::vector transactions; - ASTPtr new_order_by_ast = data.order_by_ast; - ASTPtr new_primary_key_ast = data.primary_key_ast; - ASTPtr primary_expr_list_for_altering_parts; + params.apply(new_columns, &new_order_by_ast, &new_primary_key_ast); + ASTPtr primary_expr_list_for_altering_parts; for (const AlterCommand & param : params) { - if (param.type == AlterCommand::MODIFY_ORDER_BY) + if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) { - if (!data.primary_key_ast) - { - /// Primary and sorting key become independent after this ALTER so we have to - /// save the old ORDER BY expression as the new primary key. - new_primary_key_ast = data.order_by_ast->clone(); - } + if (supportsSampling()) + throw Exception("MODIFY PRIMARY KEY only supported for tables without sampling key", ErrorCodes::BAD_ARGUMENTS); - new_order_by_ast = param.order_by; - } - else if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) - { primary_expr_list_for_altering_parts = MergeTreeData::extractKeyExpressionList(param.primary_key); - if (!data.primary_key_ast) - new_order_by_ast = param.primary_key; - else - new_primary_key_ast = param.primary_key; } } - if (primary_expr_list_for_altering_parts && supportsSampling()) - throw Exception("MODIFY PRIMARY KEY only supported for tables without sampling key", ErrorCodes::BAD_ARGUMENTS); - auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}); auto columns_for_parts = new_columns.getAllPhysical(); + std::vector transactions; for (const MergeTreeData::DataPartPtr & part : parts) { if (auto transaction = data.alterDataPart(part, columns_for_parts, primary_expr_list_for_altering_parts, false)) @@ -265,10 +247,9 @@ void StorageMergeTree::alter( }; context.getDatabase(database_name)->alterTable(context, table_name, new_columns, storage_modifier); - setColumns(std::move(new_columns)); /// Reinitialize primary key because primary key column types might have changed. - data.setPrimaryKey(new_order_by_ast, new_primary_key_ast); + data.setPrimaryKeyAndColumns(new_order_by_ast, new_primary_key_ast, new_columns); for (auto & transaction : transactions) transaction->commit(); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 0379838ec57..fdd228f3fc1 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -407,9 +407,6 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff) { - /// Note: setting columns first so that the new sorting key can use new columns. - setColumns(std::move(new_columns)); - ASTPtr new_primary_key_ast = data.primary_key_ast; ASTPtr new_order_by_ast = data.order_by_ast; IDatabase::ASTModifier storage_modifier; @@ -418,9 +415,14 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column ParserNotEmptyExpressionList parser(false); auto new_sorting_key_expr_list = parseQuery(parser, metadata_diff.new_sorting_key, 0); - auto tuple = makeASTFunction("tuple"); - tuple->arguments->children = new_sorting_key_expr_list->children; - new_order_by_ast = tuple; + if (new_sorting_key_expr_list->children.size() == 1) + new_order_by_ast = new_sorting_key_expr_list->children[0]; + else + { + auto tuple = makeASTFunction("tuple"); + tuple->arguments->children = new_sorting_key_expr_list->children; + new_order_by_ast = tuple; + } if (!data.primary_key_ast) { @@ -434,20 +436,22 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column auto & storage_ast = typeid_cast(ast); if (!storage_ast.order_by) - throw Exception("Not supported", ErrorCodes::LOGICAL_ERROR); /// TODO: better exception message + throw Exception( + "ALTER MODIFY ORDER BY of default-partitioned tables is not supported", + ErrorCodes::LOGICAL_ERROR); if (new_primary_key_ast.get() != data.primary_key_ast.get()) storage_ast.set(storage_ast.primary_key, new_primary_key_ast); - storage_ast.set(storage_ast.order_by, tuple); + storage_ast.set(storage_ast.order_by, new_order_by_ast); }; } - context.getDatabase(database_name)->alterTable(context, table_name, getColumns(), storage_modifier); + context.getDatabase(database_name)->alterTable(context, table_name, new_columns, storage_modifier); /// Even if the primary/sorting keys didn't change we must reinitialize it /// because primary key column types might have changed. - data.setPrimaryKey(new_order_by_ast, new_primary_key_ast); + data.setPrimaryKeyAndColumns(new_order_by_ast, new_primary_key_ast, new_columns); } @@ -3044,21 +3048,24 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, data.checkAlter(params); for (const AlterCommand & param : params) + { if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) throw Exception("Modification of primary key is not supported for replicated tables", ErrorCodes::NOT_IMPLEMENTED); + } ColumnsDescription new_columns = data.getColumns(); - params.apply(new_columns); + ASTPtr new_order_by_ast = data.order_by_ast; + ASTPtr new_primary_key_ast = data.primary_key_ast; + params.apply(new_columns, &new_order_by_ast, &new_primary_key_ast); + String new_columns_str = new_columns.toString(); if (new_columns_str != data.getColumns().toString()) changed_nodes.emplace_back(zookeeper_path, "columns", new_columns_str); ReplicatedMergeTreeTableMetadata new_metadata(data); - for (const AlterCommand & param : params) - { - if (param.type == AlterCommand::MODIFY_ORDER_BY) - new_metadata.sorting_key = serializeAST(*MergeTreeData::extractKeyExpressionList(param.order_by)); - } + if (new_order_by_ast.get() != data.order_by_ast.get()) + new_metadata.sorting_key = serializeAST(*MergeTreeData::extractKeyExpressionList(new_order_by_ast)); + String new_metadata_str = new_metadata.toString(); if (new_metadata_str != ReplicatedMergeTreeTableMetadata(data).toString()) changed_nodes.emplace_back(zookeeper_path, "metadata", new_metadata_str); From ffb54509dbcee35ffe1db9ac9bff55848c192fcd Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 14 Nov 2018 14:51:18 +0300 Subject: [PATCH 20/69] add tests [#CLICKHOUSE-3859] --- ...olumn_in_partition_concurrent_zookeeper.sh | 4 +- .../00754_alter_modify_order_by.reference | 12 ++++++ .../00754_alter_modify_order_by.sql | 38 ++++++++++++++++ ...fy_order_by_replicated_zookeeper.reference | 12 ++++++ ...r_modify_order_by_replicated_zookeeper.sql | 43 +++++++++++++++++++ 5 files changed, 107 insertions(+), 2 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00754_alter_modify_order_by.reference create mode 100644 dbms/tests/queries/0_stateless/00754_alter_modify_order_by.sql create mode 100644 dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference create mode 100644 dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql diff --git a/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_concurrent_zookeeper.sh b/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_concurrent_zookeeper.sh index 79164982d7d..cf3c1fdadaa 100755 --- a/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_concurrent_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_concurrent_zookeeper.sh @@ -34,5 +34,5 @@ wait $ch "SELECT DISTINCT * FROM test.clear_column1 WHERE d != toDate('2000-03-01') ORDER BY d, i, s" $ch "SELECT DISTINCT * FROM test.clear_column2 WHERE d != toDate('2000-03-01') ORDER BY d, i, s" -#$ch "DROP TABLE IF EXISTS test.clear_column1" -#$ch "DROP TABLE IF EXISTS test.clear_column2" +$ch "DROP TABLE IF EXISTS test.clear_column1" +$ch "DROP TABLE IF EXISTS test.clear_column2" diff --git a/dbms/tests/queries/0_stateless/00754_alter_modify_order_by.reference b/dbms/tests/queries/0_stateless/00754_alter_modify_order_by.reference new file mode 100644 index 00000000000..b63ec7d0c42 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00754_alter_modify_order_by.reference @@ -0,0 +1,12 @@ +*** Check that the parts are sorted according to the new key. *** +1 2 0 10 +1 2 0 20 +1 2 2 40 +1 2 2 50 +1 2 1 30 +*** Check that the rows are collapsed according to the new key. *** +1 2 0 30 +1 2 1 30 +1 2 4 90 +*** Check SHOW CREATE TABLE *** +CREATE TABLE test.summing ( x UInt32, y UInt32, z UInt32, val UInt32) ENGINE = SummingMergeTree PRIMARY KEY (x, y) ORDER BY (x, y, -z) SETTINGS index_granularity = 8192 diff --git a/dbms/tests/queries/0_stateless/00754_alter_modify_order_by.sql b/dbms/tests/queries/0_stateless/00754_alter_modify_order_by.sql new file mode 100644 index 00000000000..77b7a531034 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00754_alter_modify_order_by.sql @@ -0,0 +1,38 @@ +SET send_logs_level = 'none'; + +DROP TABLE IF EXISTS test.old_style; +CREATE TABLE test.old_style(d Date, x UInt32) ENGINE MergeTree(d, x, 8192); +ALTER TABLE test.old_style ADD COLUMN y UInt32, MODIFY ORDER BY (x, y); -- { serverError 36} +DROP TABLE test.old_style; + +DROP TABLE IF EXISTS test.summing; +CREATE TABLE test.summing(x UInt32, y UInt32, val UInt32) ENGINE SummingMergeTree ORDER BY (x, y); + +/* Can't add an expression with existing column to ORDER BY. */ +ALTER TABLE test.summing MODIFY ORDER BY (x, y, -val); -- { serverError 36} + +/* Can't add an expression with existing column to ORDER BY. */ +ALTER TABLE test.summing ADD COLUMN z UInt32 DEFAULT x + 1, MODIFY ORDER BY (x, y, -z); -- { serverError 36} + +/* Can't add nonexistent column to ORDER BY. */ +ALTER TABLE test.summing MODIFY ORDER BY (x, y, nonexistent); -- { serverError 47} + +/* Can't modyfy ORDER BY so that it is no longer a prefix of the PRIMARY KEY. */ +ALTER TABLE test.summing MODIFY ORDER BY x; -- { serverError 36} + +INSERT INTO test.summing(x, y, val) VALUES (1, 2, 10), (1, 2, 20); + +ALTER TABLE test.summing ADD COLUMN z UInt32 AFTER y, MODIFY ORDER BY (x, y, -z); + +INSERT INTO test.summing(x, y, z, val) values (1, 2, 1, 30), (1, 2, 2, 40), (1, 2, 2, 50); + +SELECT '*** Check that the parts are sorted according to the new key. ***'; +SELECT * FROM test.summing ORDER BY _part; + +SELECT '*** Check that the rows are collapsed according to the new key. ***'; +SELECT * FROM test.summing FINAL ORDER BY x, y, z; + +SELECT '*** Check SHOW CREATE TABLE ***'; +SHOW CREATE TABLE test.summing; + +DROP TABLE test.summing; diff --git a/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference b/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference new file mode 100644 index 00000000000..cf2141d2f85 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference @@ -0,0 +1,12 @@ +*** Check that the parts are sorted according to the new key. *** +1 2 0 10 +1 2 0 20 +1 2 2 40 +1 2 2 50 +1 2 1 30 +*** Check that the rows are collapsed according to the new key. *** +1 2 0 30 +1 2 1 30 +1 2 4 90 +*** Check SHOW CREATE TABLE *** +CREATE TABLE test.summing_r2 ( x UInt32, y UInt32, z UInt32, val UInt32) ENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test/summing\', \'r2\') PRIMARY KEY (x, y) ORDER BY (x, y, -z) SETTINGS index_granularity = 8192 diff --git a/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql b/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql new file mode 100644 index 00000000000..c9358648d82 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql @@ -0,0 +1,43 @@ +SET send_logs_level = 'none'; + +DROP TABLE IF EXISTS test.old_style; +CREATE TABLE test.old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/old_style', 'r1', d, x, 8192); +ALTER TABLE test.old_style ADD COLUMN y UInt32, MODIFY ORDER BY (x, y); -- { serverError 36 } +DROP TABLE test.old_style; + +DROP TABLE IF EXISTS test.summing_r1; +DROP TABLE IF EXISTS test.summing_r2; +CREATE TABLE test.summing_r1(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/test/summing', 'r1') ORDER BY (x, y); +CREATE TABLE test.summing_r2(x UInt32, y UInt32, val UInt32) ENGINE ReplicatedSummingMergeTree('/clickhouse/tables/test/summing', 'r2') ORDER BY (x, y); + +/* Can't add an expression with existing column to ORDER BY. */ +ALTER TABLE test.summing_r1 MODIFY ORDER BY (x, y, -val); -- { serverError 36 } + +/* Can't add an expression with existing column to ORDER BY. */ +ALTER TABLE test.summing_r1 ADD COLUMN z UInt32 DEFAULT x + 1, MODIFY ORDER BY (x, y, -z); -- { serverError 36 } + +/* Can't add nonexistent column to ORDER BY. */ +ALTER TABLE test.summing_r1 MODIFY ORDER BY (x, y, nonexistent); -- { serverError 47 } + +/* Can't modyfy ORDER BY so that it is no longer a prefix of the PRIMARY KEY. */ +ALTER TABLE test.summing_r1 MODIFY ORDER BY x; -- { serverError 36 } + +INSERT INTO test.summing_r1(x, y, val) VALUES (1, 2, 10), (1, 2, 20); +SYSTEM SYNC REPLICA test.summing_r2; + +ALTER TABLE test.summing_r1 ADD COLUMN z UInt32 AFTER y, MODIFY ORDER BY (x, y, -z); + +INSERT INTO test.summing_r1(x, y, z, val) values (1, 2, 1, 30), (1, 2, 2, 40), (1, 2, 2, 50); +SYSTEM SYNC REPLICA test.summing_r2; + +SELECT '*** Check that the parts are sorted according to the new key. ***'; +SELECT * FROM test.summing_r2 ORDER BY _part; + +SELECT '*** Check that the rows are collapsed according to the new key. ***'; +SELECT * FROM test.summing_r2 FINAL ORDER BY x, y, z; + +SELECT '*** Check SHOW CREATE TABLE ***'; +SHOW CREATE TABLE test.summing_r2; + +DROP TABLE test.summing_r1; +DROP TABLE test.summing_r2; From 470f96ce191dd0b75218595a1d9b260ac7765a77 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Tue, 27 Nov 2018 19:45:45 +0300 Subject: [PATCH 21/69] Minor refactoring --- .../src/Common/{RWLockFIFO.cpp => RWLock.cpp} | 37 +- dbms/src/Common/{RWLockFIFO.h => RWLock.h} | 29 +- .../Common/tests/gtest_rw_lock_fifo.cpp.cpp | 30 +- dbms/src/Storages/IStorage.cpp | 4 +- dbms/src/Storages/IStorage.h | 21 +- suppress_base.json | 19832 ++++++++-------- 6 files changed, 9960 insertions(+), 9993 deletions(-) rename dbms/src/Common/{RWLockFIFO.cpp => RWLock.cpp} (84%) rename dbms/src/Common/{RWLockFIFO.h => RWLock.h} (73%) diff --git a/dbms/src/Common/RWLockFIFO.cpp b/dbms/src/Common/RWLock.cpp similarity index 84% rename from dbms/src/Common/RWLockFIFO.cpp rename to dbms/src/Common/RWLock.cpp index d511685111d..6890da72812 100644 --- a/dbms/src/Common/RWLockFIFO.cpp +++ b/dbms/src/Common/RWLock.cpp @@ -1,4 +1,4 @@ -#include "RWLockFIFO.h" +#include "RWLock.h" #include #include #include @@ -33,15 +33,15 @@ namespace ErrorCodes } -class RWLockFIFO::LockHandlerImpl +class RWLockImpl::LockHandlerImpl { - RWLockFIFOPtr parent; + RWLock parent; GroupsContainer::iterator it_group; ClientsContainer::iterator it_client; ThreadToHandler::iterator it_handler; CurrentMetrics::Increment active_client_increment; - LockHandlerImpl(RWLockFIFOPtr && parent, GroupsContainer::iterator it_group, ClientsContainer::iterator it_client); + LockHandlerImpl(RWLock && parent, GroupsContainer::iterator it_group, ClientsContainer::iterator it_client); public: @@ -49,11 +49,11 @@ public: ~LockHandlerImpl(); - friend class RWLockFIFO; + friend class RWLockImpl; }; -RWLockFIFO::LockHandler RWLockFIFO::getLock(RWLockFIFO::Type type, RWLockFIFO::Client client) +RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type, RWLockImpl::Client client) { Stopwatch watch(CLOCK_MONOTONIC_COARSE); CurrentMetrics::Increment waiting_client_increment((type == Read) ? CurrentMetrics::RWLockWaitingReaders @@ -142,24 +142,7 @@ RWLockFIFO::LockHandler RWLockFIFO::getLock(RWLockFIFO::Type type, RWLockFIFO::C } -RWLockFIFO::Clients RWLockFIFO::getClientsInTheQueue() const -{ - std::unique_lock lock(mutex); - - Clients res; - for (const auto & group : queue) - { - for (const auto & client : group.clients) - { - res.emplace_back(client); - } - } - - return res; -} - - -RWLockFIFO::LockHandlerImpl::~LockHandlerImpl() +RWLockImpl::LockHandlerImpl::~LockHandlerImpl() { std::unique_lock lock(parent->mutex); @@ -183,10 +166,10 @@ RWLockFIFO::LockHandlerImpl::~LockHandlerImpl() } -RWLockFIFO::LockHandlerImpl::LockHandlerImpl(RWLockFIFOPtr && parent, RWLockFIFO::GroupsContainer::iterator it_group, - RWLockFIFO::ClientsContainer::iterator it_client) +RWLockImpl::LockHandlerImpl::LockHandlerImpl(RWLock && parent, RWLockImpl::GroupsContainer::iterator it_group, + RWLockImpl::ClientsContainer::iterator it_client) : parent{std::move(parent)}, it_group{it_group}, it_client{it_client}, - active_client_increment{(it_client->type == RWLockFIFO::Read) ? CurrentMetrics::RWLockActiveReaders + active_client_increment{(it_client->type == RWLockImpl::Read) ? CurrentMetrics::RWLockActiveReaders : CurrentMetrics::RWLockActiveWriters} {} diff --git a/dbms/src/Common/RWLockFIFO.h b/dbms/src/Common/RWLock.h similarity index 73% rename from dbms/src/Common/RWLockFIFO.h rename to dbms/src/Common/RWLock.h index 29c37ff78b6..d9937054727 100644 --- a/dbms/src/Common/RWLockFIFO.h +++ b/dbms/src/Common/RWLock.h @@ -12,19 +12,19 @@ namespace DB { -class RWLockFIFO; -using RWLockFIFOPtr = std::shared_ptr; +class RWLockImpl; +using RWLock = std::shared_ptr; /// Implements shared lock with FIFO service -/// You could call it recursively (several calls from the same thread) in Read mode -class RWLockFIFO : public std::enable_shared_from_this +/// Can be acquired recursively (several calls from the same thread) in Read mode +class RWLockImpl : public std::enable_shared_from_this { public: enum Type { Read, - Write + Write, }; private: @@ -44,11 +44,7 @@ private: }; public: - static RWLockFIFOPtr create() - { - return RWLockFIFOPtr(new RWLockFIFO); - } - + static RWLock create() { return RWLock(new RWLockImpl); } /// Just use LockHandler::reset() to release the lock class LockHandlerImpl; @@ -58,19 +54,10 @@ public: /// Waits in the queue and returns appropriate lock LockHandler getLock(Type type, Client client = Client{}); - - LockHandler getLock(Type type, const std::string & who) - { - return getLock(type, Client(who)); - } - - using Clients = std::vector; - - /// Returns list of executing and waiting clients - Clients getClientsInTheQueue() const; + LockHandler getLock(Type type, const std::string & who) { return getLock(type, Client(who)); } private: - RWLockFIFO() = default; + RWLockImpl() = default; struct Group; using GroupsContainer = std::list; diff --git a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp b/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp index 638a8576f3f..44e053b61a4 100644 --- a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp +++ b/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp @@ -4,7 +4,7 @@ #endif #include -#include +#include #include #include #include @@ -18,7 +18,7 @@ using namespace DB; -TEST(Common, RWLockFIFO_1) +TEST(Common, RWLock_1) { constexpr int cycles = 1000; const std::vector pool_sizes{1, 2, 4, 8}; @@ -26,7 +26,7 @@ TEST(Common, RWLockFIFO_1) static std::atomic readers{0}; static std::atomic writers{0}; - static auto fifo_lock = RWLockFIFO::create(); + static auto fifo_lock = RWLockImpl::create(); static thread_local std::random_device rd; static thread_local pcg64 gen(rd()); @@ -35,12 +35,12 @@ TEST(Common, RWLockFIFO_1) { for (int i = 0; i < cycles; ++i) { - auto type = (std::uniform_int_distribution<>(0, 9)(gen) >= round) ? RWLockFIFO::Read : RWLockFIFO::Write; + auto type = (std::uniform_int_distribution<>(0, 9)(gen) >= round) ? RWLockImpl::Read : RWLockImpl::Write; auto sleep_for = std::chrono::duration(std::uniform_int_distribution<>(1, 100)(gen)); auto lock = fifo_lock->getLock(type, "RW"); - if (type == RWLockFIFO::Write) + if (type == RWLockImpl::Write) { ++writers; @@ -85,11 +85,11 @@ TEST(Common, RWLockFIFO_1) } } -TEST(Common, RWLockFIFO_Recursive) +TEST(Common, RWLock_Recursive) { constexpr auto cycles = 10000; - static auto fifo_lock = RWLockFIFO::create(); + static auto fifo_lock = RWLockImpl::create(); static thread_local std::random_device rd; static thread_local pcg64 gen(rd()); @@ -98,7 +98,7 @@ TEST(Common, RWLockFIFO_Recursive) { for (int i = 0; i < 2 * cycles; ++i) { - auto lock = fifo_lock->getLock(RWLockFIFO::Write); + auto lock = fifo_lock->getLock(RWLockImpl::Write); auto sleep_for = std::chrono::duration(std::uniform_int_distribution<>(1, 100)(gen)); std::this_thread::sleep_for(sleep_for); @@ -109,17 +109,17 @@ TEST(Common, RWLockFIFO_Recursive) { for (int i = 0; i < cycles; ++i) { - auto lock1 = fifo_lock->getLock(RWLockFIFO::Read); + auto lock1 = fifo_lock->getLock(RWLockImpl::Read); auto sleep_for = std::chrono::duration(std::uniform_int_distribution<>(1, 100)(gen)); std::this_thread::sleep_for(sleep_for); - auto lock2 = fifo_lock->getLock(RWLockFIFO::Read); + auto lock2 = fifo_lock->getLock(RWLockImpl::Read); - EXPECT_ANY_THROW({fifo_lock->getLock(RWLockFIFO::Write);}); + EXPECT_ANY_THROW({fifo_lock->getLock(RWLockImpl::Write);}); } - fifo_lock->getLock(RWLockFIFO::Write); + fifo_lock->getLock(RWLockImpl::Write); }); t1.join(); @@ -127,12 +127,12 @@ TEST(Common, RWLockFIFO_Recursive) } -TEST(Common, RWLockFIFO_PerfTest_Readers) +TEST(Common, RWLock_PerfTest_Readers) { constexpr int cycles = 100000; // 100k const std::vector pool_sizes{1, 2, 4, 8}; - static auto fifo_lock = RWLockFIFO::create(); + static auto fifo_lock = RWLockImpl::create(); for (auto pool_size : pool_sizes) { @@ -142,7 +142,7 @@ TEST(Common, RWLockFIFO_PerfTest_Readers) { for (auto i = 0; i < cycles; ++i) { - auto lock = fifo_lock->getLock(RWLockFIFO::Read); + auto lock = fifo_lock->getLock(RWLockImpl::Read); } }; diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 70d016566d4..498d486b7a1 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -8,9 +8,9 @@ TableStructureReadLock::TableStructureReadLock(StoragePtr storage_, bool lock_st : storage(storage_) { if (lock_data) - data_lock = storage->data_lock->getLock(RWLockFIFO::Read, who); + data_lock = storage->data_lock->getLock(RWLockImpl::Read, who); if (lock_structure) - structure_lock = storage->structure_lock->getLock(RWLockFIFO::Read, who); + structure_lock = storage->structure_lock->getLock(RWLockImpl::Read, who); } } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 81249446746..446f2269f6f 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -28,9 +28,6 @@ class Context; class IBlockInputStream; class IBlockOutputStream; -class RWLockFIFO; -using RWLockFIFOPtr = std::shared_ptr; - using StorageActionBlockType = size_t; using BlockOutputStreamPtr = std::shared_ptr; @@ -64,8 +61,8 @@ private: StoragePtr storage; /// Order is important. - RWLockFIFO::LockHandler data_lock; - RWLockFIFO::LockHandler structure_lock; + RWLockImpl::LockHandler data_lock; + RWLockImpl::LockHandler structure_lock; public: TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data, const std::string & who); @@ -75,8 +72,8 @@ public: using TableStructureReadLockPtr = std::shared_ptr; using TableStructureReadLocks = std::vector; -using TableStructureWriteLock = RWLockFIFO::LockHandler; -using TableDataWriteLock = RWLockFIFO::LockHandler; +using TableStructureWriteLock = RWLockImpl::LockHandler; +using TableDataWriteLock = RWLockImpl::LockHandler; using TableFullWriteLock = std::pair; @@ -148,7 +145,7 @@ public: */ TableDataWriteLock lockDataForAlter(const std::string & who = "Alter") { - auto res = data_lock->getLock(RWLockFIFO::Write, who); + auto res = data_lock->getLock(RWLockImpl::Write, who); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); return res; @@ -156,7 +153,7 @@ public: TableStructureWriteLock lockStructureForAlter(const std::string & who = "Alter") { - auto res = structure_lock->getLock(RWLockFIFO::Write, who); + auto res = structure_lock->getLock(RWLockImpl::Write, who); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); return res; @@ -367,7 +364,7 @@ private: * 2) all changes to the data after releasing the lock will be based on the structure of the table at the time after the lock was released. * You need to take for read for the entire time of the operation that changes the data. */ - mutable RWLockFIFOPtr data_lock = RWLockFIFO::create(); + mutable RWLock data_lock = RWLockImpl::create(); /** Lock for multiple columns and path to table. It is taken for write at RENAME, ALTER (for ALTER MODIFY for a while) and DROP. * It is taken for read for the whole time of SELECT, INSERT and merge parts (for MergeTree). @@ -376,7 +373,7 @@ private: * That is, if this lock is taken for write, you should not worry about `parts_writing_lock`. * parts_writing_lock is only needed for cases when you do not want to take `table_structure_lock` for long operations (ALTER MODIFY). */ - mutable RWLockFIFOPtr structure_lock = RWLockFIFO::create(); + mutable RWLock structure_lock = RWLockImpl::create(); }; /// table name -> table diff --git a/suppress_base.json b/suppress_base.json index 9627a389de5..cc0ba5cbcb5 100644 --- a/suppress_base.json +++ b/suppress_base.json @@ -1,9917 +1,9917 @@ { - "version": 1, - "warnings": [ - { - "CodeCurrent": 1403077733, - "CodeNext": 951341, - "CodePrev": 1270846060, - "ErrorCode": "V550", - "FileName": "preciseExp10.c", - "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(y) < Epsilon." - }, - { - "CodeCurrent": 1403077733, - "CodeNext": 3854685, - "CodePrev": 1391368717, - "ErrorCode": "V550", - "FileName": "preciseExp10.c", - "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(y) < Epsilon." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "shift10.cpp", - "Message": "A code fragment from 'shift_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1360858873, - "CodeNext": 0, - "CodePrev": 864691173, - "ErrorCode": "V524", - "FileName": "DateLUTImpl.h", - "Message": "It is odd that the body of 'getValues' function is fully equivalent to the body of 'find' function." - }, - { - "CodeCurrent": 3167014141, - "CodeNext": 1528350630, - "CodePrev": 0, - "ErrorCode": "V524", - "FileName": "DateLUTImpl.h", - "Message": "It is odd that the body of 'toDate' function is fully equivalent to the body of 'fromDayNum' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DateLUTImpl.cpp", - "Message": "A code fragment from 'DateLUTImpl.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "demangle.cpp", - "Message": "A code fragment from 'demangle.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ThreadPool.cpp", - "Message": "A code fragment from 'ThreadPool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 18396, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "JSON.h", - "Message": "The 'JSON' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 1301811264, - "CodeNext": 123, - "CodePrev": 44642, - "ErrorCode": "V524", - "FileName": "JSON.cpp", - "Message": "It is odd that the body of 'get' function is fully equivalent to the body of 'getName' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "JSON.cpp", - "Message": "A code fragment from 'JSON.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DateLUT.cpp", - "Message": "A code fragment from 'DateLUT.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SetTerminalEcho.cpp", - "Message": "A code fragment from 'SetTerminalEcho.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "date_lut2.cpp", - "Message": "A code fragment from 'date_lut_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "date_lut3.cpp", - "Message": "A code fragment from 'date_lut_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "gtest_strong_typedef.cpp", - "Message": "A code fragment from 'gtest_strong_typedef.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "date_lut4.cpp", - "Message": "A code fragment from 'date_lut_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "dump_variable.cpp", - "Message": "A code fragment from 'dump_variable.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "date_lut_init.cpp", - "Message": "A code fragment from 'date_lut_init.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "gtest_json_test.cpp", - "Message": "A code fragment from 'gtest_json_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "multi_version.cpp", - "Message": "A code fragment from 'multi_version.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "date_lut_default_timezone.cpp", - "Message": "A code fragment from 'date_lut_default_timezone.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "LevelFilterChannel.cpp", - "Message": "A code fragment from 'LevelFilterChannel.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SessionPoolHelpers.cpp", - "Message": "A code fragment from 'SessionPoolHelpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 123, - "CodeNext": 2077384594, - "CodePrev": 2854941240, - "ErrorCode": "V565", - "FileName": "BaseDaemon.h", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "GraphiteWriter.cpp", - "Message": "A code fragment from 'GraphiteWriter.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3676031242, - "CodeNext": 3056897128, - "CodePrev": 1102711319, - "ErrorCode": "V707", - "FileName": "lgamma.c", - "Message": "Giving short names to global variables is considered to be bad practice. It is suggested to rename 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'tc', 'tf', 'tt', 't_', 't_', 't_', 't_', 't_', 't_', 't_', 't_', 't_', 't_', 'u_', 'u_', 'u_', 'u_', 'u_', 'u_', 'v_', 'v_', 'v_', 'v_', 'v_', 's_', 's_', 's_', 's_', 's_', 's_', 's_', 'r_', 'r_', 'r_', 'r_', 'r_', 'r_', 'w_', 'w_', 'w_', 'w_', 'w_', 'w_', 'w_' variables." - }, - { - "CodeCurrent": 196849995, - "CodeNext": 2821665, - "CodePrev": 257609, - "ErrorCode": "V550", - "FileName": "lgamma.c", - "Message": "An odd precise comparison: t == _._. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 2367177615, - "CodeNext": 1620191745, - "CodePrev": 47045, - "ErrorCode": "V605", - "FileName": "posix_spawn.c", - "Message": "Consider verifying the expression. An unsigned value is compared to the number belonging to the range [-_.._]." - }, - { - "CodeCurrent": 2676964558, - "CodeNext": 17733, - "CodePrev": 17733, - "ErrorCode": "V526", - "FileName": "glob.c", - "Message": "The 'strcmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "exp2f.c", - "Message": "A code fragment from 'exp_f.c' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "OwnPatternFormatter.cpp", - "Message": "A code fragment from 'OwnPatternFormatter.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1586910037, - "CodeNext": 1159405304, - "CodePrev": 123, - "ErrorCode": "V769", - "FileName": "BaseDaemon.cpp", - "Message": "The 'name_start' pointer in the 'name_start - symbols[i]' expression could be nullptr. In such case, resulting value will be senseless and it should not be used." - }, - { - "CodeCurrent": 123, - "CodeNext": 125, - "CodePrev": 35837, - "ErrorCode": "V565", - "FileName": "BaseDaemon.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BaseDaemon.cpp", - "Message": "A code fragment from 'BaseDaemon.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Exception.cpp", - "Message": "A code fragment from 'Exception.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2045, - "CodeNext": 123, - "CodePrev": 2452368582, - "ErrorCode": "V730", - "FileName": "Row.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: lengths." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Connection.cpp", - "Message": "A code fragment from 'Connection.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 123, - "CodeNext": 2202583, - "CodePrev": 1128383580, - "ErrorCode": "V794", - "FileName": "Query.cpp", - "Message": "The assignment operator should be protected from the case of 'this == &other'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Query.cpp", - "Message": "A code fragment from 'Query.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ResultBase.cpp", - "Message": "A code fragment from 'ResultBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2377956688, - "CodeNext": 703533, - "CodePrev": 2042816737, - "ErrorCode": "V688", - "FileName": "StoreQueryResult.cpp", - "Message": "The 'fields' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StoreQueryResult.cpp", - "Message": "A code fragment from 'StoreQueryResult.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Row.cpp", - "Message": "A code fragment from 'Row.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Value.cpp", - "Message": "A code fragment from 'Value.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "UseQueryResult.cpp", - "Message": "A code fragment from 'UseQueryResult.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "consistent_hashing.cpp", - "Message": "A code fragment from 'consistent_hashing.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Pool.cpp", - "Message": "A code fragment from 'Pool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 188688957, - "CodeNext": 125, - "CodePrev": 769, - "ErrorCode": "V658", - "FileName": "sumbur.cpp", - "Message": "A value is being subtracted from the unsigned variable. This can result in an overflow. In such a case, the '<' comparison operation can potentially behave unexpectedly. Consider inspecting the '_xFFFFFFFF / _ - h < part' expression." - }, - { - "CodeCurrent": 450631639, - "CodeNext": 22890, - "CodePrev": 0, - "ErrorCode": "V1003", - "FileName": "sumbur.cpp", - "Message": "The macro 'curslice' is a dangerous expression. The parameter 'i' must be surrounded by parentheses." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "sumbur.cpp", - "Message": "A code fragment from 'sumbur.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PoolWithFailover.cpp", - "Message": "A code fragment from 'PoolWithFailover.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "mysqlxx_test.cpp", - "Message": "A code fragment from 'mysqlxx_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zstd_test.cpp", - "Message": "A code fragment from 'zstd_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "iotest.cpp", - "Message": "A code fragment from 'iotest.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "iotest_nonblock.cpp", - "Message": "A code fragment from 'iotest_nonblock.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "iotest_aio.cpp", - "Message": "A code fragment from 'iotest_aio.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "corrector_utf8.cpp", - "Message": "A code fragment from 'corrector_utf_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "config-processor.cpp", - "Message": "A code fragment from 'config-processor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2919138394, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "ZooKeeperImpl.h", - "Message": "The 'Exception' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zookeeper-cli.cpp", - "Message": "A code fragment from 'zookeeper-cli.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'main.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "main.cpp", - "Message": "A code fragment from 'main.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1244410449, - "CodeNext": 123, - "CodePrev": 182455598, - "ErrorCode": "V730", - "FileName": "HashTable.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: zero_value_storage." - }, - { - "CodeCurrent": 3824136199, - "CodeNext": 2663, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "HashTable.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: size." - }, - { - "CodeCurrent": 1712271, - "CodeNext": 0, - "CodePrev": 3363495029, - "ErrorCode": "V730", - "FileName": "MarkovModel.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: n." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MarkovModel.h", - "Message": "A code fragment from 'main.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 259986933, - "CodeNext": 0, - "CodePrev": 123, - "ErrorCode": "V730", - "FileName": "RWLockFIFO.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: type." - }, - { - "CodeCurrent": 6265, - "CodeNext": 5125961, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "Field.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: storage." - }, - { - "CodeCurrent": 101009567, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V730", - "FileName": "Field.h", - "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: storage, which." - }, - { - "CodeCurrent": 6341727, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "Field.h", - "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: storage, which." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AnalyzeColumns.cpp", - "Message": "A code fragment from 'AnalyzeColumns.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'markov-model.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AnalyzeLambdas.cpp", - "Message": "A code fragment from 'AnalyzeLambdas.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CollectAliases.cpp", - "Message": "A code fragment from 'CollectAliases.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3859608575, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 3859608575, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 728751127, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: f == static_cast< DB::Float_ > (u). It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 1801682850, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: static_cast< DB::Float_ > (u) == f. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 134227181, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: f == static_cast< DB::Float_ > (u). It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 940474457, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: static_cast< DB::Float_ > (u) == f. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 728751127, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: f == static_cast< DB::Float_ > (u). It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 1801682850, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: static_cast< DB::Float_ > (u) == f. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 134227181, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: f == static_cast< DB::Float_ > (u). It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 940474457, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: static_cast< DB::Float_ > (u) == f. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 3765134313, - "CodeNext": 3685848331, - "CodePrev": 1405257529, - "ErrorCode": "V550", - "FileName": "FieldVisitors.h", - "Message": "An odd precise comparison: l == r. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 51079373, - "CodeNext": 0, - "CodePrev": 836015037, - "ErrorCode": "V550", - "FileName": "FieldVisitors.h", - "Message": "An odd precise comparison: x != _. It's probably better to use a comparison with defined precision: fabs(A - B) > Epsilon." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AnalyzeResultOfQuery.cpp", - "Message": "A code fragment from 'AnalyzeResultOfQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CollectTables.cpp", - "Message": "A code fragment from 'CollectTables.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ExecuteTableFunctions.cpp", - "Message": "A code fragment from 'ExecuteTableFunctions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "OptimizeGroupOrderLimitBy.cpp", - "Message": "A code fragment from 'OptimizeGroupOrderLimitBy.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TranslatePositionalArguments.cpp", - "Message": "A code fragment from 'TranslatePositionalArguments.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BlockInfo.cpp", - "Message": "A code fragment from 'BlockInfo.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 4174113710, - "CodeNext": 3531930912, - "CodePrev": 181464845, - "ErrorCode": "V659", - "FileName": "ColumnConst.h", - "Message": "Declarations of functions with 'getDataColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Block.cpp", - "Message": "A code fragment from 'Block.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 199355300, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V730", - "FileName": "ExpressionActions.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TypeAndConstantInference.cpp", - "Message": "A code fragment from 'TypeAndConstantInference.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnWithTypeAndName.cpp", - "Message": "A code fragment from 'ColumnWithTypeAndName.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Field.cpp", - "Message": "A code fragment from 'Field.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SortDescription.cpp", - "Message": "A code fragment from 'SortDescription.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NamesAndTypes.cpp", - "Message": "A code fragment from 'NamesAndTypes.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "iostream_debug_helpers.cpp", - "Message": "A code fragment from 'iostream_debug_helpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3826643199, - "CodeNext": 123, - "CodePrev": 22807034, - "ErrorCode": "V690", - "FileName": "ColumnVector.h", - "Message": "Copy constructor is declared as private in the 'ColumnVector' class, but the default '=' operator will still be generated by compiler. It is dangerous to use such a class." - }, - { - "CodeCurrent": 1132122309, - "CodeNext": 0, - "CodePrev": 931267045, - "ErrorCode": "V659", - "FileName": "ColumnArray.h", - "Message": "Declarations of functions with 'getData' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 1116407654, - "CodeNext": 0, - "CodePrev": 1356382257, - "ErrorCode": "V659", - "FileName": "ColumnArray.h", - "Message": "Declarations of functions with 'getOffsetsColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 2390648255, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V659", - "FileName": "ColumnArray.h", - "Message": "Declarations of functions with 'getOffsets' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AddingDefaultBlockOutputStream.cpp", - "Message": "A code fragment from 'AddingDefaultBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3168042121, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V690", - "FileName": "ColumnString.h", - "Message": "Copy constructor is declared as private in the 'ColumnString' class, but the default '=' operator will still be generated by compiler. It is dangerous to use such a class." - }, - { - "CodeCurrent": 4054773003, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V690", - "FileName": "ColumnAggregateFunction.h", - "Message": "Copy constructor is declared as private in the 'ColumnAggregateFunction' class, but the default '=' operator will still be generated by compiler. It is dangerous to use such a class." - }, - { - "CodeCurrent": 3194944389, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "AggregatingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 14743151, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "SortCursor.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: order." - }, - { - "CodeCurrent": 195436, - "CodeNext": 123, - "CodePrev": 45270, - "ErrorCode": "V730", - "FileName": "MergingSortedBlockInputStream.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: row_num." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregatingSortedBlockInputStream.cpp", - "Message": "A code fragment from 'AggregatingSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 4136571990, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V690", - "FileName": "ColumnFixedString.h", - "Message": "Copy constructor is declared as private in the 'ColumnFixedString' class, but the default '=' operator will still be generated by compiler. It is dangerous to use such a class." - }, - { - "CodeCurrent": 1054674186, - "CodeNext": 0, - "CodePrev": 925160603, - "ErrorCode": "V659", - "FileName": "ColumnNullable.h", - "Message": "Declarations of functions with 'getNestedColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 3317648788, - "CodeNext": 0, - "CodePrev": 3497477841, - "ErrorCode": "V659", - "FileName": "ColumnNullable.h", - "Message": "Declarations of functions with 'getNullMapColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 146511793, - "CodeNext": 1296505, - "CodePrev": 196592208, - "ErrorCode": "V522", - "FileName": "Aggregator.h", - "Message": "There might be dereferencing of a potential null pointer 'null_map'." - }, - { - "CodeCurrent": 132655, - "CodeNext": 0, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "TwoLevelHashTable.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: container, bucket, current_it." - }, - { - "CodeCurrent": 9156143, - "CodeNext": 1642228070, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "TwoLevelHashTable.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: container, bucket, current_it." - }, - { - "CodeCurrent": 3310514761, - "CodeNext": 2820625488, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "Aggregator.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: keys_size." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregatingBlockInputStream.cpp", - "Message": "A code fragment from 'AggregatingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BinaryRowInputStream.cpp", - "Message": "A code fragment from 'BinaryRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BinaryRowOutputStream.cpp", - "Message": "A code fragment from 'BinaryRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ApplyingMutationsBlockInputStream.cpp", - "Message": "A code fragment from 'ApplyingMutationsBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 389785, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "BlockIO.h", - "Message": "The 'BlockIO' class implements the '=' operator, but lacks a copy constructor. It is dangerous to use such a class." - }, - { - "CodeCurrent": 253306735, - "CodeNext": 4054391213, - "CodePrev": 0, - "ErrorCode": "V550", - "FileName": "ProcessList.h", - "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(memory_tracker_fault_probability) > Epsilon." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BlockIO.cpp", - "Message": "A code fragment from 'BlockIO.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BlockOutputStreamFromRowOutputStream.cpp", - "Message": "A code fragment from 'BlockOutputStreamFromRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2065830648, - "CodeNext": 2809, - "CodePrev": 0, - "ErrorCode": "V550", - "FileName": "BlockInputStreamFromRowInputStream.cpp", - "Message": "An odd precise comparison: allow_errors_ratio == _. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 123, - "CodeNext": 485783650, - "CodePrev": 35837, - "ErrorCode": "V565", - "FileName": "BlockInputStreamFromRowInputStream.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BlockInputStreamFromRowInputStream.cpp", - "Message": "A code fragment from 'BlockInputStreamFromRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BlockStreamProfileInfo.cpp", - "Message": "A code fragment from 'BlockStreamProfileInfo.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2244466177, - "CodeNext": 3040549265, - "CodePrev": 1036506627, - "ErrorCode": "V656", - "FileName": "CSVRowInputStream.cpp", - "Message": "Variables 'prev_position', 'curr_position' are initialized through the call to the same function. It's probably an error or un-optimized code. Consider inspecting the 'istr.position()' expression. Check lines: _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CSVRowInputStream.cpp", - "Message": "A code fragment from 'CSVRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CSVRowOutputStream.cpp", - "Message": "A code fragment from 'CSVRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 56687, - "CodeNext": 4160558847, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "CollapsingFinalBlockInputStream.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: pos." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CollapsingFinalBlockInputStream.cpp", - "Message": "A code fragment from 'CollapsingFinalBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3189192997, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "CollapsingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3474608216, - "CodeNext": 3529831664, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "CollapsingSortedBlockInputStream.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: current_key, next_key, first_negative, last_positive, last_negative." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CollapsingSortedBlockInputStream.cpp", - "Message": "A code fragment from 'CollapsingSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 194852, - "CodeNext": 123, - "CodePrev": 2043716261, - "ErrorCode": "V730", - "FileName": "CapnProtoRowInputStream.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'CapnProtoRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 92287, - "CodeNext": 231331854, - "CodePrev": 360716783, - "ErrorCode": "V550", - "FileName": "ColumnGathererStream.cpp", - "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(seconds) > Epsilon." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnGathererStream.cpp", - "Message": "A code fragment from 'ColumnGathererStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CountingBlockOutputStream.cpp", - "Message": "A code fragment from 'CountingBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ConvertingBlockInputStream.cpp", - "Message": "A code fragment from 'ConvertingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 578660774, - "CodeNext": 2441941198, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "HashSet.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: saved_hash." - }, - { - "CodeCurrent": 2441941198, - "CodeNext": 0, - "CodePrev": 578660774, - "ErrorCode": "V730", - "FileName": "HashSet.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: saved_hash." - }, - { - "CodeCurrent": 3398701039, - "CodeNext": 413159266, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "ClearableHashSet.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: version." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DistinctBlockInputStream.cpp", - "Message": "A code fragment from 'DistinctBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 233229516, - "CodeNext": 17813717, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "CreatingSetsBlockInputStream.cpp", - "Message": "There might be dereferencing of a potential null pointer 'subquery.set'." - }, - { - "CodeCurrent": 14128946, - "CodeNext": 35621461, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "CreatingSetsBlockInputStream.cpp", - "Message": "There might be dereferencing of a potential null pointer 'subquery.join'." - }, - { - "CodeCurrent": 207231, - "CodeNext": 150934975, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "BoolMask.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: can_be_true, can_be_false." - }, - { - "CodeCurrent": 61359, - "CodeNext": 1727213102, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "Join.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: block, row_num." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CreatingSetsBlockInputStream.cpp", - "Message": "A code fragment from 'CreatingSetsBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DistinctSortedBlockInputStream.cpp", - "Message": "A code fragment from 'DistinctSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ExpressionBlockInputStream.cpp", - "Message": "A code fragment from 'ExpressionBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FilterBlockInputStream.cpp", - "Message": "A code fragment from 'FilterBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FilterColumnsBlockInputStream.cpp", - "Message": "A code fragment from 'FilterColumnsBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3416694064, - "CodeNext": 1249324771, - "CodePrev": 2609586663, - "ErrorCode": "V526", - "FileName": "OptimizedRegularExpression.inl.h", - "Message": "The 'strncmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." - }, - { - "CodeCurrent": 1249324771, - "CodeNext": 611713731, - "CodePrev": 3416694064, - "ErrorCode": "V526", - "FileName": "OptimizedRegularExpression.inl.h", - "Message": "The 'strncmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." - }, - { - "CodeCurrent": 611713731, - "CodeNext": 3436934615, - "CodePrev": 1249324771, - "ErrorCode": "V526", - "FileName": "OptimizedRegularExpression.inl.h", - "Message": "The 'strncmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." - }, - { - "CodeCurrent": 3436934615, - "CodeNext": 123, - "CodePrev": 611713731, - "ErrorCode": "V526", - "FileName": "OptimizedRegularExpression.inl.h", - "Message": "The 'strncmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." - }, - { - "CodeCurrent": 3165904209, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "GraphiteRollupSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "GraphiteRollupSortedBlockInputStream.cpp", - "Message": "A code fragment from 'GraphiteRollupSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IBlockInputStream.cpp", - "Message": "A code fragment from 'IBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 505357, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "Quota.h", - "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: queries, errors, result_rows, result_bytes, read_rows, read_bytes, ..." - }, - { - "CodeCurrent": 1713353540, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "Quota.h", - "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: queries, errors, result_rows, result_bytes, read_rows, read_bytes, ..." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IProfilingBlockInputStream.cpp", - "Message": "A code fragment from 'IProfilingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'FormatFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IRowInputStream.cpp", - "Message": "A code fragment from 'IRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IRowOutputStream.cpp", - "Message": "A code fragment from 'IRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "JSONCompactRowOutputStream.cpp", - "Message": "A code fragment from 'JSONCompactRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InputStreamFromASTInsertQuery.cpp", - "Message": "A code fragment from 'InputStreamFromASTInsertQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "JSONEachRowRowInputStream.cpp", - "Message": "A code fragment from 'JSONEachRowRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "JSONEachRowRowOutputStream.cpp", - "Message": "A code fragment from 'JSONEachRowRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "JSONRowOutputStream.cpp", - "Message": "A code fragment from 'JSONRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "LimitBlockInputStream.cpp", - "Message": "A code fragment from 'LimitBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "LimitByBlockInputStream.cpp", - "Message": "A code fragment from 'LimitByBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MaterializingBlockInputStream.cpp", - "Message": "A code fragment from 'MaterializingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3161195889, - "CodeNext": 123, - "CodePrev": 1879495099, - "ErrorCode": "V688", - "FileName": "MergeSortingBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeSortingBlockInputStream.cpp", - "Message": "A code fragment from 'MergeSortingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergingAggregatedBlockInputStream.cpp", - "Message": "A code fragment from 'MergingAggregatedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1919288073, - "CodeNext": 123, - "CodePrev": 1879495099, - "ErrorCode": "V688", - "FileName": "MergingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3223507410, - "CodeNext": 123, - "CodePrev": 1879495099, - "ErrorCode": "V688", - "FileName": "MergingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2080638586, - "CodeNext": 123, - "CodePrev": 1879495099, - "ErrorCode": "V688", - "FileName": "MergingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergingSortedBlockInputStream.cpp", - "Message": "A code fragment from 'MergingSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergingAggregatedMemoryEfficientBlockInputStream.cpp", - "Message": "A code fragment from 'MergingAggregatedMemoryEfficientBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NativeBlockInputStream.cpp", - "Message": "A code fragment from 'NativeBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ODBCDriverBlockOutputStream.cpp", - "Message": "A code fragment from 'ODBCDriverBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NativeBlockOutputStream.cpp", - "Message": "A code fragment from 'NativeBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 429503, - "CodeNext": 3001760339, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "ParallelInputsProcessor.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: i." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParallelAggregatingBlockInputStream.cpp", - "Message": "A code fragment from 'ParallelAggregatingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PartialSortingBlockInputStream.cpp", - "Message": "A code fragment from 'PartialSortingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PrettyBlockOutputStream.cpp", - "Message": "A code fragment from 'PrettyBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PrettyCompactBlockOutputStream.cpp", - "Message": "A code fragment from 'PrettyCompactBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PrettySpaceBlockOutputStream.cpp", - "Message": "A code fragment from 'PrettySpaceBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "RemoteBlockOutputStream.cpp", - "Message": "A code fragment from 'RemoteBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 12514670, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "PoolWithFailoverBase.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: pool, state, index." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "RemoteBlockInputStream.cpp", - "Message": "A code fragment from 'RemoteBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SizeLimits.cpp", - "Message": "A code fragment from 'SizeLimits.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2251640247, - "CodeNext": 350966580, - "CodePrev": 576202780, - "ErrorCode": "V688", - "FileName": "PushingToViewsBlockOutputStream.cpp", - "Message": "The 'storage' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PushingToViewsBlockOutputStream.cpp", - "Message": "A code fragment from 'PushingToViewsBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3190533093, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ReplacingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2535084036, - "CodeNext": 1504104628, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "ReplacingSortedBlockInputStream.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: current_key, next_key, selected_row." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplacingSortedBlockInputStream.cpp", - "Message": "A code fragment from 'ReplacingSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SquashingBlockOutputStream.cpp", - "Message": "A code fragment from 'SquashingBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SquashingBlockInputStream.cpp", - "Message": "A code fragment from 'SquashingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SquashingTransform.cpp", - "Message": "A code fragment from 'SquashingTransform.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3967747240, - "CodeNext": 0, - "CodePrev": 893858483, - "ErrorCode": "V659", - "FileName": "ColumnTuple.h", - "Message": "Declarations of functions with 'getColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 3191513253, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "SummingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SummingSortedBlockInputStream.cpp", - "Message": "A code fragment from 'SummingSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TSKVRowInputStream.cpp", - "Message": "A code fragment from 'TSKVRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TabSeparatedRowInputStream.cpp", - "Message": "A code fragment from 'TabSeparatedRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TSKVRowOutputStream.cpp", - "Message": "A code fragment from 'TSKVRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TabSeparatedRowOutputStream.cpp", - "Message": "A code fragment from 'TabSeparatedRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ValuesRowOutputStream.cpp", - "Message": "A code fragment from 'ValuesRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TotalsHavingBlockInputStream.cpp", - "Message": "A code fragment from 'TotalsHavingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ValuesRowInputStream.cpp", - "Message": "A code fragment from 'ValuesRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "VerticalRowOutputStream.cpp", - "Message": "A code fragment from 'VerticalRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3998578981, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "VersionedCollapsingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "VersionedCollapsingSortedBlockInputStream.cpp", - "Message": "A code fragment from 'VersionedCollapsingSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "XMLRowOutputStream.cpp", - "Message": "A code fragment from 'XMLRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "copyData.cpp", - "Message": "A code fragment from 'copyData.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "materializeBlock.cpp", - "Message": "A code fragment from 'materializeBlock.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "verbosePrintString.cpp", - "Message": "A code fragment from 'verbosePrintString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "narrowBlockInputStreams.cpp", - "Message": "A code fragment from 'narrowBlockInputStreams.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeAggregateFunction.cpp", - "Message": "A code fragment from 'DataTypeAggregateFunction.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeArray.cpp", - "Message": "A code fragment from 'DataTypeArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeDate.cpp", - "Message": "A code fragment from 'DataTypeDate.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeDateTime.cpp", - "Message": "A code fragment from 'DataTypeDateTime.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeFactory.cpp", - "Message": "A code fragment from 'DataTypeFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeFunction.cpp", - "Message": "A code fragment from 'DataTypeFunction.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 566420924, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "DataTypeEnum.h", - "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2818381, - "CodeNext": 690132628, - "CodePrev": 3160185338, - "ErrorCode": "V688", - "FileName": "DataTypeEnum.cpp", - "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2818381, - "CodeNext": 368631651, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "DataTypeEnum.cpp", - "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2818381, - "CodeNext": 2806683281, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "DataTypeEnum.cpp", - "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2818381, - "CodeNext": 710643812, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "DataTypeEnum.cpp", - "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeEnum.cpp", - "Message": "A code fragment from 'DataTypeEnum.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeInterval.cpp", - "Message": "A code fragment from 'DataTypeInterval.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeFixedString.cpp", - "Message": "A code fragment from 'DataTypeFixedString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeNothing.cpp", - "Message": "A code fragment from 'DataTypeNothing.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeString.cpp", - "Message": "A code fragment from 'DataTypeString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeNumberBase.cpp", - "Message": "A code fragment from 'DataTypeNumberBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeNullable.cpp", - "Message": "A code fragment from 'DataTypeNullable.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypesNumber.cpp", - "Message": "A code fragment from 'DataTypesNumber.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FieldToDataType.cpp", - "Message": "A code fragment from 'FieldToDataType.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeTuple.cpp", - "Message": "A code fragment from 'DataTypeTuple.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeUUID.cpp", - "Message": "A code fragment from 'DataTypeUUID.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IDataType.cpp", - "Message": "A code fragment from 'IDataType.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NestedUtils.cpp", - "Message": "A code fragment from 'NestedUtils.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getLeastSupertype.cpp", - "Message": "A code fragment from 'getLeastSupertype.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getMostSubtype.cpp", - "Message": "A code fragment from 'getMostSubtype.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DatabaseFactory.cpp", - "Message": "A code fragment from 'DatabaseFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DatabaseMemory.cpp", - "Message": "A code fragment from 'DatabaseMemory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2421950747, - "CodeNext": 281663836, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "DatabaseDictionary.cpp", - "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'DatabaseDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 4574138, - "CodeNext": 2250845352, - "CodePrev": 72410316, - "ErrorCode": "V688", - "FileName": "DatabaseOrdinary.cpp", - "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DatabaseOrdinary.cpp", - "Message": "A code fragment from 'DatabaseOrdinary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DatabasesCommon.cpp", - "Message": "A code fragment from 'DatabasesCommon.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3735361261, - "CodeNext": 123, - "CodePrev": 1623468225, - "ErrorCode": "V688", - "FileName": "CatBoostModel.cpp", - "Message": "The 'cat_features_count' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2354497829, - "CodeNext": 2736089741, - "CodePrev": 2412400287, - "ErrorCode": "V688", - "FileName": "CatBoostModel.cpp", - "Message": "The 'float_features_count' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2354497829, - "CodeNext": 2736089741, - "CodePrev": 2412400287, - "ErrorCode": "V688", - "FileName": "CatBoostModel.cpp", - "Message": "The 'cat_features_count' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2575466056, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "CatBoostModel.cpp", - "Message": "The 'lib_path' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'CatBoostModel.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 207025177, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "CacheDictionary.h", - "Message": "The 'CacheDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 518403501, - "CodeNext": 610328694, - "CodePrev": 1500196055, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 411059078, - "CodeNext": 123, - "CodePrev": 610328694, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 518630893, - "CodeNext": 610328694, - "CodePrev": 1500196055, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 411059078, - "CodeNext": 123, - "CodePrev": 610328694, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 411059078, - "CodeNext": 123, - "CodePrev": 1382482652, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 411059078, - "CodeNext": 123, - "CodePrev": 1382482652, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3731206478, - "CodeNext": 123, - "CodePrev": 11900062, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3673035245, - "CodeNext": 1382482652, - "CodePrev": 3006984332, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2414448007, - "CodeNext": 123, - "CodePrev": 1382482652, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3673035245, - "CodeNext": 1382482652, - "CodePrev": 3467903158, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2414448007, - "CodeNext": 123, - "CodePrev": 1382482652, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 921248968, - "CodeNext": 123, - "CodePrev": 1342750387, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3690553619, - "CodeNext": 37308961, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "CacheDictionary.cpp", - "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 173283489, - "CodeNext": 1039775048, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "CacheDictionary.cpp", - "Message": "The 'size' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 549768760, - "CodeNext": 1152076000, - "CodePrev": 1919584191, - "ErrorCode": "V688", - "FileName": "CacheDictionary.cpp", - "Message": "The 'size' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 579497379, - "CodeNext": 93115, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "CacheDictionary.cpp", - "Message": "The 'size' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'CacheDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1825423383, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V690", - "FileName": "ClickHouseDictionarySource.h", - "Message": "The 'ClickHouseDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ClickHouseDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2185656967, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V690", - "FileName": "ComplexKeyCacheDictionary.h", - "Message": "The 'ComplexKeyCacheDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 3504260784, - "CodeNext": 37308961, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ComplexKeyCacheDictionary.cpp", - "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2142244675, - "CodeNext": 2508128848, - "CodePrev": 41371507, - "ErrorCode": "V688", - "FileName": "ComplexKeyCacheDictionary.cpp", - "Message": "The 'key_size' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyCacheDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyCacheDictionary_createAttributeWithType.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyCacheDictionary_generate_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyCacheDictionary_generate_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyCacheDictionary_generate_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 579497379, - "CodeNext": 93115, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ComplexKeyCacheDictionary_setAttributeValue.cpp", - "Message": "The 'size' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyCacheDictionary_setAttributeValue.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1173598882, - "CodeNext": 78734017, - "CodePrev": 38981872, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStreamBase.cpp", - "Message": "The 'block' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DictionaryBlockInputStreamBase.cpp", - "Message": "A code fragment from 'DictionaryBlockInputStreamBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1259294311, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "ComplexKeyHashedDictionary.h", - "Message": "The 'ComplexKeyHashedDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyHashedDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'DictionarySourceHelpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2179468235, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "DictionaryStructure.cpp", - "Message": "The 'attributes' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'DictionaryStructure.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1562050581, - "CodeNext": 123, - "CodePrev": 2247801432, - "ErrorCode": "V690", - "FileName": "FileDictionarySource.h", - "Message": "The 'FileDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 1863286807, - "CodeNext": 123, - "CodePrev": 3087958036, - "ErrorCode": "V690", - "FileName": "ExecutableDictionarySource.h", - "Message": "The 'ExecutableDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 1560485909, - "CodeNext": 123, - "CodePrev": 3228890078, - "ErrorCode": "V690", - "FileName": "HTTPDictionarySource.h", - "Message": "The 'HTTPDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 314376213, - "CodeNext": 123, - "CodePrev": 4033243237, - "ErrorCode": "V690", - "FileName": "LibraryDictionarySource.h", - "Message": "The 'LibraryDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 313384981, - "CodeNext": 123, - "CodePrev": 247020956, - "ErrorCode": "V690", - "FileName": "MongoDBDictionarySource.h", - "Message": "The 'MongoDBDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 1561051157, - "CodeNext": 123, - "CodePrev": 1880965760, - "ErrorCode": "V690", - "FileName": "ODBCDictionarySource.h", - "Message": "The 'ODBCDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 1143955477, - "CodeNext": 123, - "CodePrev": 2162917980, - "ErrorCode": "V690", - "FileName": "MySQLDictionarySource.h", - "Message": "The 'MySQLDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'DictionarySourceFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3300655365, - "CodeNext": 2232737895, - "CodePrev": 3529503794, - "ErrorCode": "V669", - "FileName": "VarInt.h", - "Message": "The 'ostr' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 2232737895, - "CodeNext": 3413908603, - "CodePrev": 3300655365, - "ErrorCode": "V669", - "FileName": "VarInt.h", - "Message": "The 'ostr' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 2696744022, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "SettingsCommon.h", - "Message": "The 'buf' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 2696744022, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "SettingsCommon.h", - "Message": "The 'buf' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 2696744022, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "SettingsCommon.h", - "Message": "The 'buf' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 3674152177, - "CodeNext": 516173517, - "CodePrev": 5816, - "ErrorCode": "V688", - "FileName": "OwningBlockInputStream.h", - "Message": "The 'stream' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ExecutableDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ExternalResultDescription.cpp", - "Message": "A code fragment from 'ExternalResultDescription.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3266193319, - "CodeNext": 193, - "CodePrev": 123, - "ErrorCode": "V516", - "FileName": "IColumn.h", - "Message": "Consider inspecting an odd expression. Non-null function pointer is compared to null." - }, - { - "CodeCurrent": 3266193319, - "CodeNext": 193, - "CodePrev": 123, - "ErrorCode": "V516", - "FileName": "IColumn.h", - "Message": "Consider inspecting an odd expression. Non-null function pointer is compared to null: 'IsMutableColumns < Args >::value'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ExternalQueryBuilder.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FileDictionarySource.cpp", - "Message": "A code fragment from 'FileDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 207054359, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "FlatDictionary.h", - "Message": "The 'FlatDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'FlatDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'HTTPDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 206951383, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "HashedDictionary.h", - "Message": "The 'HashedDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'HashedDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 605484233, - "CodeNext": 26586574, - "CodePrev": 3065615242, - "ErrorCode": "V688", - "FileName": "LibraryDictionarySource.cpp", - "Message": "The 'config_prefix' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 26586574, - "CodeNext": 72410313, - "CodePrev": 605484233, - "ErrorCode": "V688", - "FileName": "LibraryDictionarySource.cpp", - "Message": "The 'sample_block' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'LibraryDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'MongoDBBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'MongoDBDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MySQLBlockInputStream.cpp", - "Message": "A code fragment from 'MySQLBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3362020784, - "CodeNext": 0, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "MySQLDictionarySource.cpp", - "Message": "The 'update_time' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 13569497, - "CodeNext": 985361697, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "MySQLDictionarySource.cpp", - "Message": "The 'sample_block' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'MySQLDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ODBCBlockInputStream.cpp", - "Message": "A code fragment from 'ODBCBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3238522471, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "RangeHashedDictionary.h", - "Message": "The 'RangeHashedDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 3406274085, - "CodeNext": 4182595664, - "CodePrev": 3042861068, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 4182595664, - "CodeNext": 123, - "CodePrev": 3406274085, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 456188231, - "CodeNext": 2414448007, - "CodePrev": 1469584987, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2414448007, - "CodeNext": 123, - "CodePrev": 456188231, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2178677200, - "CodeNext": 2078138180, - "CodePrev": 2641309169, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'column_names' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2418126002, - "CodeNext": 123, - "CodePrev": 2901253088, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'start_dates' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2418126002, - "CodeNext": 123, - "CodePrev": 2901253088, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'end_dates' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1525601398, - "CodeNext": 2418126002, - "CodePrev": 682989491, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2418126002, - "CodeNext": 123, - "CodePrev": 1525601398, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'start_dates' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2418126002, - "CodeNext": 123, - "CodePrev": 1525601398, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'end_dates' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 127934457, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "RangeHashedDictionary.cpp", - "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'RangeHashedDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ODBCDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "GeoDictionariesLoader.cpp", - "Message": "A code fragment from 'GeoDictionariesLoader.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "readInvalidateQuery.cpp", - "Message": "A code fragment from 'readInvalidateQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "writeParenthesisedString.cpp", - "Message": "A code fragment from 'writeParenthesisedString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3312877383, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "TrieDictionary.h", - "Message": "The 'TrieDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'TrieDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "RegionsHierarchies.cpp", - "Message": "A code fragment from 'RegionsHierarchies.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "RegionsHierarchy.cpp", - "Message": "A code fragment from 'RegionsHierarchy.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "RegionsNames.cpp", - "Message": "A code fragment from 'RegionsNames.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TechDataHierarchy.cpp", - "Message": "A code fragment from 'TechDataHierarchy.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "HierarchiesProvider.cpp", - "Message": "A code fragment from 'HierarchiesProvider.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NamesProvider.cpp", - "Message": "A code fragment from 'NamesProvider.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "HierarchyFormatReader.cpp", - "Message": "A code fragment from 'HierarchyFormatReader.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NamesFormatReader.cpp", - "Message": "A code fragment from 'NamesFormatReader.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ActionLocksManager.cpp", - "Message": "A code fragment from 'ActionLocksManager.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ClientInfo.cpp", - "Message": "A code fragment from 'ClientInfo.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Compiler.cpp", - "Message": "A code fragment from 'Compiler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 871352, - "CodeNext": 223912592, - "CodePrev": 766487020, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 73089598, - "CodeNext": 4084406, - "CodePrev": 2724040669, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 3873811996, - "CodePrev": 3673856596, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 73089598, - "CodeNext": 4084406, - "CodePrev": 2724040669, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 36816, - "CodePrev": 3126615886, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 511803757, - "CodePrev": 3126615886, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 122870, - "CodePrev": 1474597293, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 73089598, - "CodeNext": 1179330, - "CodePrev": 2685314853, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 122870, - "CodePrev": 4240594574, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2338865922, - "CodeNext": 123, - "CodePrev": 2685314853, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 122870, - "CodePrev": 3859987971, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2338865922, - "CodeNext": 123, - "CodePrev": 4042434526, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 36816, - "CodePrev": 3126615886, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 73089598, - "CodeNext": 223912592, - "CodePrev": 207246, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 122870, - "CodePrev": 223912592, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 73089598, - "CodeNext": 223912592, - "CodePrev": 207246, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 122870, - "CodePrev": 223912592, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 109104, - "CodePrev": 3874875103, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 73089598, - "CodeNext": 4084406, - "CodePrev": 3397567370, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Aggregator.cpp", - "Message": "A code fragment from 'Aggregator.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 304867129, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "Cluster.cpp", - "Message": "The 'host_name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 304867129, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "Cluster.cpp", - "Message": "The 'port' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3548849765, - "CodeNext": 2541401097, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "HexWriteBuffer.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: buf." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Cluster.cpp", - "Message": "A code fragment from 'Cluster.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AsynchronousMetrics.cpp", - "Message": "A code fragment from 'AsynchronousMetrics.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 123, - "CodeNext": 129791, - "CodePrev": 35837, - "ErrorCode": "V565", - "FileName": "DNSCacheUpdater.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 1922414329, - "CodeNext": 123, - "CodePrev": 125, - "ErrorCode": "V759", - "FileName": "DNSCacheUpdater.cpp", - "Message": "Incorrect order of exception handling blocks. Exception of 'DNSException' type will be caught in a catch block for 'Exception' base type." - }, - { - "CodeCurrent": 958195402, - "CodeNext": 123, - "CodePrev": 125, - "ErrorCode": "V759", - "FileName": "DNSCacheUpdater.cpp", - "Message": "Incorrect order of exception handling blocks. Exception of 'TimeoutException' type will be caught in a catch block for 'Exception' base type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DNSCacheUpdater.cpp", - "Message": "A code fragment from 'DNSCacheUpdater.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 390633, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "DDLWorker.cpp", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: host_shard_num, host_replica_num." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DDLWorker.cpp", - "Message": "A code fragment from 'DDLWorker.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'DictionaryFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3538236801, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V524", - "FileName": "RuntimeComponentsFactory.h", - "Message": "It is odd that the body of 'createExternalModelsConfigRepository' function is fully equivalent to the body of 'createExternalDictionariesConfigRepository' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'Context.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "EmbeddedDictionaries.cpp", - "Message": "A code fragment from 'EmbeddedDictionaries.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1141335469, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionActions.cpp", - "Message": "The 'sample_block' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ExpressionActions.cpp", - "Message": "A code fragment from 'ExpressionActions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ExternalDictionaries.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3392699298, - "CodeNext": 0, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "ExternalLoader.cpp", - "Message": "The 'config' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ExternalLoader.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ExternalLoaderConfigRepository.cpp", - "Message": "A code fragment from 'ExternalLoaderConfigRepository.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3405548629, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2136590205, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'aliases' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1478353069, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3783766598, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3829470883, - "CodeNext": 0, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "There might be dereferencing of a potential null pointer 'subquery'." - }, - { - "CodeCurrent": 3237744880, - "CodeNext": 123, - "CodePrev": 3248671758, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1459789653, - "CodeNext": 123, - "CodePrev": 4129363794, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3479198451, - "CodeNext": 4055667619, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'aliases' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 4216604761, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 427259459, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3061641111, - "CodeNext": 123, - "CodePrev": 137989537, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1240966709, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3626124927, - "CodeNext": 2885543787, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2661019153, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1362320973, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3178398591, - "CodeNext": 1853405735, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 238898558, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3409673381, - "CodeNext": 72789921, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ExpressionAnalyzer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ExternalModels.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InJoinSubqueriesPreprocessor.cpp", - "Message": "A code fragment from 'InJoinSubqueriesPreprocessor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 200100772, - "CodeNext": 123, - "CodePrev": 11496, - "ErrorCode": "V730", - "FileName": "InterpreterAlterQuery.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterAlterQuery.cpp", - "Message": "A code fragment from 'InterpreterAlterQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterCheckQuery.cpp", - "Message": "A code fragment from 'InterpreterCheckQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterDescribeQuery.cpp", - "Message": "A code fragment from 'InterpreterDescribeQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3494840295, - "CodeNext": 193, - "CodePrev": 0, - "ErrorCode": "V524", - "FileName": "StorageLog.h", - "Message": "It is odd that the body of 'getFullPath' function is fully equivalent to the body of 'full_path' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterCreateQuery.cpp", - "Message": "A code fragment from 'InterpreterCreateQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterExistsQuery.cpp", - "Message": "A code fragment from 'InterpreterExistsQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2270843740, - "CodeNext": 125, - "CodePrev": 4111337664, - "ErrorCode": "V522", - "FileName": "InterpreterDropQuery.cpp", - "Message": "There might be dereferencing of a potential null pointer 'database'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterDropQuery.cpp", - "Message": "A code fragment from 'InterpreterDropQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterFactory.cpp", - "Message": "A code fragment from 'InterpreterFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterKillQueryQuery.cpp", - "Message": "A code fragment from 'InterpreterKillQueryQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterInsertQuery.cpp", - "Message": "A code fragment from 'InterpreterInsertQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterOptimizeQuery.cpp", - "Message": "A code fragment from 'InterpreterOptimizeQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterRenameQuery.cpp", - "Message": "A code fragment from 'InterpreterRenameQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3481785836, - "CodeNext": 0, - "CodePrev": 263098193, - "ErrorCode": "V678", - "FileName": "InterpreterSelectWithUnionQuery.cpp", - "Message": "An object is used as an argument to its own method. Consider checking the first actual argument of the 'setQueryContext' function." - }, - { - "CodeCurrent": 146241864, - "CodeNext": 72410313, - "CodePrev": 607166188, - "ErrorCode": "V688", - "FileName": "InterpreterSelectWithUnionQuery.cpp", - "Message": "The 'query_ptr' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 72410313, - "CodeNext": 123, - "CodePrev": 146241864, - "ErrorCode": "V688", - "FileName": "InterpreterSelectWithUnionQuery.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterSelectWithUnionQuery.cpp", - "Message": "A code fragment from 'InterpreterSelectWithUnionQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterSetQuery.cpp", - "Message": "A code fragment from 'InterpreterSetQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterShowTablesQuery.cpp", - "Message": "A code fragment from 'InterpreterShowTablesQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3481785836, - "CodeNext": 0, - "CodePrev": 263098193, - "ErrorCode": "V678", - "FileName": "InterpreterSelectQuery.cpp", - "Message": "An object is used as an argument to its own method. Consider checking the first actual argument of the 'setQueryContext' function." - }, - { - "CodeCurrent": 1460847663, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "InterpreterSelectQuery.cpp", - "Message": "The 'input' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2449370409, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V524", - "FileName": "InterpreterSelectQuery.cpp", - "Message": "It is odd that the body of 'executeProjection' function is fully equivalent to the body of 'executeExpression' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterSelectQuery.cpp", - "Message": "A code fragment from 'InterpreterSelectQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterShowProcesslistQuery.cpp", - "Message": "A code fragment from 'InterpreterShowProcesslistQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1627430578, - "CodeNext": 520103033, - "CodePrev": 721433261, - "ErrorCode": "V522", - "FileName": "InterpreterShowCreateQuery.cpp", - "Message": "There might be dereferencing of a potential null pointer 'create_query'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterShowCreateQuery.cpp", - "Message": "A code fragment from 'InterpreterShowCreateQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterUseQuery.cpp", - "Message": "A code fragment from 'InterpreterUseQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "LogicalExpressionsOptimizer.cpp", - "Message": "A code fragment from 'LogicalExpressionsOptimizer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1421843502, - "CodeNext": 2932320854, - "CodePrev": 3193279124, - "ErrorCode": "V688", - "FileName": "Join.cpp", - "Message": "The 'columns_left' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Join.cpp", - "Message": "A code fragment from 'Join.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NullableUtils.cpp", - "Message": "A code fragment from 'NullableUtils.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2628352566, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V659", - "FileName": "MergeTreeData.h", - "Message": "Declarations of functions with 'modifyPartState' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 3583851774, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "InterpreterSystemQuery.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3000727018, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "InterpreterSystemQuery.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'InterpreterSystemQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3170877678, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "ProcessList.cpp", - "Message": "The 'it' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ProcessList.cpp", - "Message": "A code fragment from 'ProcessList.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ProjectionManipulation.cpp", - "Message": "A code fragment from 'ProjectionManipulation.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PartLog.cpp", - "Message": "A code fragment from 'PartLog.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "QueryLog.cpp", - "Message": "A code fragment from 'QueryLog.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Quota.cpp", - "Message": "A code fragment from 'Quota.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SecurityManager.cpp", - "Message": "A code fragment from 'SecurityManager.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Set.cpp", - "Message": "A code fragment from 'Set.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SetVariants.cpp", - "Message": "A code fragment from 'SetVariants.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SystemLog.cpp", - "Message": "A code fragment from 'SystemLog.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Settings.cpp", - "Message": "A code fragment from 'Settings.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TablesStatus.cpp", - "Message": "A code fragment from 'TablesStatus.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Users.cpp", - "Message": "A code fragment from 'Users.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "createBlockSelector.cpp", - "Message": "A code fragment from 'createBlockSelector.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "castColumn.cpp", - "Message": "A code fragment from 'castColumn.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "convertFieldToType.cpp", - "Message": "A code fragment from 'convertFieldToType.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "evaluateConstantExpression.cpp", - "Message": "A code fragment from 'evaluateConstantExpression.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getClusterName.cpp", - "Message": "A code fragment from 'getClusterName.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "evaluateMissingDefaults.cpp", - "Message": "A code fragment from 'evaluateMissingDefaults.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 143231, - "CodeNext": 125, - "CodePrev": 125, - "ErrorCode": "V565", - "FileName": "executeQuery.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 3481785836, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V678", - "FileName": "executeQuery.cpp", - "Message": "An object is used as an argument to its own method. Consider checking the first actual argument of the 'setQueryContext' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "executeQuery.cpp", - "Message": "A code fragment from 'executeQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "sortBlock.cpp", - "Message": "A code fragment from 'sortBlock.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "loadMetadata.cpp", - "Message": "A code fragment from 'loadMetadata.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DescribeStreamFactory.cpp", - "Message": "A code fragment from 'DescribeStreamFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Collator.cpp", - "Message": "A code fragment from 'Collator.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "executeQuery.cpp", - "Message": "A code fragment from 'executeQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 39392417, - "CodeNext": 1261068192, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ColumnAggregateFunction.cpp", - "Message": "The 'data' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 699364582, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ColumnAggregateFunction.cpp", - "Message": "The 'src' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 4081716836, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ColumnAggregateFunction.cpp", - "Message": "The 'src' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnAggregateFunction.cpp", - "Message": "A code fragment from 'ColumnAggregateFunction.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SelectStreamFactory.cpp", - "Message": "A code fragment from 'SelectStreamFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2533242737, - "CodeNext": 3575089120, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "ColumnArray.cpp", - "Message": "The 'offsets' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnArray.cpp", - "Message": "A code fragment from 'ColumnArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnConst.cpp", - "Message": "A code fragment from 'ColumnConst.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnFixedString.cpp", - "Message": "A code fragment from 'ColumnFixedString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3418757250, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ColumnNullable.cpp", - "Message": "The 'null_map' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnNullable.cpp", - "Message": "A code fragment from 'ColumnNullable.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnFunction.cpp", - "Message": "A code fragment from 'ColumnFunction.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnString.cpp", - "Message": "A code fragment from 'ColumnString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FilterDescription.cpp", - "Message": "A code fragment from 'FilterDescription.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3124110550, - "CodeNext": 1303704304, - "CodePrev": 6015, - "ErrorCode": "V792", - "FileName": "ColumnsCommon.cpp", - "Message": "The '_mm_movemask_ps' function located to the right of the operator '|' will be called regardless of the value of the left operand. Perhaps, it is better to use '||'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnsCommon.cpp", - "Message": "A code fragment from 'ColumnsCommon.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnTuple.cpp", - "Message": "A code fragment from 'ColumnTuple.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 951091257, - "CodeNext": 0, - "CodePrev": 951484473, - "ErrorCode": "V656", - "FileName": "ColumnVector.cpp", - "Message": "Variables 'cur_min', 'cur_max' are initialized through the call to the same function. It's probably an error or un-optimized code. Consider inspecting the 'NaNOrZero < T > ()' expression. Check lines: _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnVector.cpp", - "Message": "A code fragment from 'ColumnVector.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnDefault.cpp", - "Message": "A code fragment from 'ColumnDefault.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IColumn.cpp", - "Message": "A code fragment from 'IColumn.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AlterCommands.cpp", - "Message": "A code fragment from 'AlterCommands.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IStorage.cpp", - "Message": "A code fragment from 'IStorage.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnsDescription.cpp", - "Message": "A code fragment from 'ColumnsDescription.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ITableDeclaration.cpp", - "Message": "A code fragment from 'ITableDeclaration.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MutationCommands.cpp", - "Message": "A code fragment from 'MutationCommands.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2317070405, - "CodeNext": 813477413, - "CodePrev": 608950573, - "ErrorCode": "V688", - "FileName": "StorageCatBoostPool.cpp", - "Message": "The 'sample_block' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageCatBoostPool.cpp", - "Message": "A code fragment from 'StorageCatBoostPool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 72410316, - "CodeNext": 2690972308, - "CodePrev": 3645137672, - "ErrorCode": "V688", - "FileName": "StorageBuffer.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2232984829, - "CodeNext": 345608494, - "CodePrev": 0, - "ErrorCode": "V547", - "FileName": "StorageBuffer.cpp", - "Message": "Expression '!buffer.first_write_time' is always true." - }, - { - "CodeCurrent": 2935677976, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageBuffer.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageBuffer.cpp", - "Message": "A code fragment from 'StorageBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'StorageDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageFile.cpp", - "Message": "A code fragment from 'StorageFile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 72410316, - "CodeNext": 2690972308, - "CodePrev": 3645137672, - "ErrorCode": "V688", - "FileName": "StorageDistributed.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3889328658, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageDistributed.cpp", - "Message": "The 'table_name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3889328658, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageDistributed.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2637534258, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageDistributed.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageDistributed.cpp", - "Message": "A code fragment from 'StorageDistributed.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageJoin.cpp", - "Message": "A code fragment from 'StorageJoin.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageFactory.cpp", - "Message": "A code fragment from 'StorageFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3969971195, - "CodeNext": 0, - "CodePrev": 2997480514, - "ErrorCode": "V656", - "FileName": "StorageMemory.cpp", - "Message": "Variables 'begin', 'end' are initialized through the call to the same function. It's probably an error or un-optimized code. Consider inspecting the 'data.begin()' expression. Check lines: _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageMemory.cpp", - "Message": "A code fragment from 'StorageMemory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 581555750, - "CodeNext": 2488995567, - "CodePrev": 2696082656, - "ErrorCode": "V688", - "FileName": "StorageLog.cpp", - "Message": "The 'column_names' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageLog.cpp", - "Message": "A code fragment from 'StorageLog.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 11714530, - "CodeNext": 0, - "CodePrev": 2931558369, - "ErrorCode": "V522", - "FileName": "StorageMaterializedView.cpp", - "Message": "There might be dereferencing of a potential null pointer 'query.storage'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageMaterializedView.cpp", - "Message": "A code fragment from 'StorageMaterializedView.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 72410316, - "CodeNext": 2690972308, - "CodePrev": 2488995567, - "ErrorCode": "V688", - "FileName": "StorageKafka.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'StorageKafka.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 72410316, - "CodeNext": 2690972308, - "CodePrev": 3645137672, - "ErrorCode": "V688", - "FileName": "StorageMySQL.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageMySQL.cpp", - "Message": "A code fragment from 'StorageMySQL.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 72410316, - "CodeNext": 2690972308, - "CodePrev": 3645137672, - "ErrorCode": "V688", - "FileName": "StorageMerge.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3595788056, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageMerge.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageMerge.cpp", - "Message": "A code fragment from 'StorageMerge.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3926377496, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageNull.cpp", - "Message": "The 'table_name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageNull.cpp", - "Message": "A code fragment from 'StorageNull.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 72410316, - "CodeNext": 2690972308, - "CodePrev": 3645137672, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2341879450, - "CodeNext": 292720666, - "CodePrev": 2322283894, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'database_name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 292720666, - "CodeNext": 72410313, - "CodePrev": 2341879450, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'table_name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 72410313, - "CodeNext": 123, - "CodePrev": 292720666, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2033765224, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3317773483, - "CodeNext": 123, - "CodePrev": 329476003, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2082196268, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 694167244, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 391214422, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3490868120, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageMergeTree.cpp", - "Message": "A code fragment from 'StorageMergeTree.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageODBC.cpp", - "Message": "A code fragment from 'StorageODBC.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSet.cpp", - "Message": "A code fragment from 'StorageSet.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageStripeLog.cpp", - "Message": "A code fragment from 'StorageStripeLog.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "VirtualColumnFactory.cpp", - "Message": "A code fragment from 'VirtualColumnFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageTinyLog.cpp", - "Message": "A code fragment from 'StorageTinyLog.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 630197398, - "CodeNext": 1056331065, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'replica_path' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 438297239, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'replica_path' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 72410316, - "CodeNext": 2690972308, - "CodePrev": 3645137672, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1038581435, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 4265130885, - "CodeNext": 123, - "CodePrev": 2362549484, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 423923019, - "CodeNext": 123, - "CodePrev": 3425255468, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1795911157, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1257284900, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 29091704, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2265033744, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 72410313, - "CodeNext": 123, - "CodePrev": 2533115030, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1394960724, - "CodeNext": 2903347041, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "ReplicatedMergeTreeQuorumEntry.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: required_number_of_replicas." - }, - { - "CodeCurrent": 2903347041, - "CodeNext": 123, - "CodePrev": 1394960724, - "ErrorCode": "V730", - "FileName": "ReplicatedMergeTreeQuorumEntry.h", - "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: required_number_of_replicas." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "A code fragment from 'StorageReplicatedMergeTree.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageView.cpp", - "Message": "A code fragment from 'StorageView.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerStorages.cpp", - "Message": "A code fragment from 'registerStorages.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "VirtualColumnUtils.cpp", - "Message": "A code fragment from 'VirtualColumnUtils.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getStructureOfRemoteTable.cpp", - "Message": "A code fragment from 'getStructureOfRemoteTable.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "transformQueryForExternalDatabase.cpp", - "Message": "A code fragment from 'transformQueryForExternalDatabase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ActiveDataPartSet.cpp", - "Message": "A code fragment from 'ActiveDataPartSet.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AllMergeSelector.cpp", - "Message": "A code fragment from 'AllMergeSelector.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3361369910, - "CodeNext": 123, - "CodePrev": 588179213, - "ErrorCode": "V658", - "FileName": "AbandonableLockInZooKeeper.cpp", - "Message": "A value is being subtracted from the unsigned variable. This can result in an overflow. In such a case, the '<' comparison operation can potentially behave unexpectedly. Consider inspecting the 'i < partitions.size() - holders.size()' expression." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AbandonableLockInZooKeeper.cpp", - "Message": "A code fragment from 'AbandonableLockInZooKeeper.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2664754306, - "CodeNext": 0, - "CodePrev": 1296085, - "ErrorCode": "V688", - "FileName": "DistributedBlockOutputStream.cpp", - "Message": "The 'query_string' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DistributedBlockOutputStream.cpp", - "Message": "A code fragment from 'DistributedBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1912588196, - "CodeNext": 979922832, - "CodePrev": 1045335318, - "ErrorCode": "V688", - "FileName": "DirectoryMonitor.cpp", - "Message": "The 'path' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'DirectoryMonitor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BackgroundProcessingPool.cpp", - "Message": "A code fragment from 'BackgroundProcessingPool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DiskSpaceMonitor.cpp", - "Message": "A code fragment from 'DiskSpaceMonitor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "LevelMergeSelector.cpp", - "Message": "A code fragment from 'LevelMergeSelector.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataPartsExchange.cpp", - "Message": "A code fragment from 'DataPartsExchange.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3216131445, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V601", - "FileName": "KeyCondition.cpp", - "Message": "The 'true' value is implicitly cast to the integer type. Inspect the first argument." - }, - { - "CodeCurrent": 2137295876, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V601", - "FileName": "KeyCondition.cpp", - "Message": "The 'false' value is implicitly cast to the integer type. Inspect the first argument." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "KeyCondition.cpp", - "Message": "A code fragment from 'KeyCondition.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeList.cpp", - "Message": "A code fragment from 'MergeList.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1098322110, - "CodeNext": 279045268, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeBaseBlockInputStream.cpp", - "Message": "The 'max_block_size_rows' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 279045268, - "CodeNext": 2329935579, - "CodePrev": 1098322110, - "ErrorCode": "V688", - "FileName": "MergeTreeBaseBlockInputStream.cpp", - "Message": "The 'preferred_block_size_bytes' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2329935579, - "CodeNext": 3894021058, - "CodePrev": 279045268, - "ErrorCode": "V688", - "FileName": "MergeTreeBaseBlockInputStream.cpp", - "Message": "The 'preferred_max_column_in_block_size_bytes' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeBaseBlockInputStream.cpp", - "Message": "A code fragment from 'MergeTreeBaseBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeBlockInputStream.cpp", - "Message": "A code fragment from 'MergeTreeBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeBlockOutputStream.cpp", - "Message": "A code fragment from 'MergeTreeBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeBlockReadUtils.cpp", - "Message": "A code fragment from 'MergeTreeBlockReadUtils.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1550988213, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeDataPartChecksum.cpp", - "Message": "The 'full_checksums' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3214271742, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeDataPartChecksum.cpp", - "Message": "The 'full_checksums' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeDataPartChecksum.cpp", - "Message": "A code fragment from 'MergeTreeDataPartChecksum.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 123, - "CodeNext": 3380448882, - "CodePrev": 1441276206, - "ErrorCode": "V565", - "FileName": "MergeTreeData.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 2654040905, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeData.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2240934897, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeData.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeData.cpp", - "Message": "A code fragment from 'MergeTreeData.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 4013367774, - "CodeNext": 3698074834, - "CodePrev": 1857246075, - "ErrorCode": "V688", - "FileName": "MergeTreeDataMergerMutator.cpp", - "Message": "The 'data' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 896405418, - "CodeNext": 2464759539, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "MergeTreeDataMergerMutator.cpp", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: average_elem_progress." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeDataMergerMutator.cpp", - "Message": "A code fragment from 'MergeTreeDataMergerMutator.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3572738747, - "CodeNext": 123, - "CodePrev": 3750922986, - "ErrorCode": "V688", - "FileName": "MergeTreeDataPart.cpp", - "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1616975535, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeDataPart.cpp", - "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2800774877, - "CodeNext": 403175306, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "MergeTreeDataPart.cpp", - "Message": "The 'columns' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeDataPart.cpp", - "Message": "A code fragment from 'MergeTreeDataPart.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreePartInfo.cpp", - "Message": "A code fragment from 'MergeTreePartInfo.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeDataWriter.cpp", - "Message": "A code fragment from 'MergeTreeDataWriter.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeDataSelectExecutor.cpp", - "Message": "A code fragment from 'MergeTreeDataSelectExecutor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreePartition.cpp", - "Message": "A code fragment from 'MergeTreePartition.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2300449615, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeRangeReader.cpp", - "Message": "The 'filter' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3630487317, - "CodeNext": 580147943, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeRangeReader.cpp", - "Message": "The 'filter' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeRangeReader.cpp", - "Message": "A code fragment from 'MergeTreeRangeReader.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeSettings.cpp", - "Message": "A code fragment from 'MergeTreeSettings.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 800849723, - "CodeNext": 355939354, - "CodePrev": 998672422, - "ErrorCode": "V688", - "FileName": "MergeTreeReadPool.cpp", - "Message": "The 'parts' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2641370323, - "CodeNext": 285699983, - "CodePrev": 807170999, - "ErrorCode": "V688", - "FileName": "MergeTreeReadPool.cpp", - "Message": "The 'parts' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1997550293, - "CodeNext": 0, - "CodePrev": 539, - "ErrorCode": "V601", - "FileName": "MergeTreeReadPool.cpp", - "Message": "The 'false' value is implicitly cast to the integer type. Inspect the first argument." - }, - { - "CodeCurrent": 2679702858, - "CodeNext": 123, - "CodePrev": 4119509742, - "ErrorCode": "V688", - "FileName": "MergeTreeReadPool.cpp", - "Message": "The 'parts' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeReadPool.cpp", - "Message": "A code fragment from 'MergeTreeReadPool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1269723304, - "CodeNext": 4205841567, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeReader.cpp", - "Message": "The 'all_mark_ranges' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeReader.cpp", - "Message": "A code fragment from 'MergeTreeReader.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1175987107, - "CodeNext": 123, - "CodePrev": 2568774401, - "ErrorCode": "V730", - "FileName": "ReplicatedMergeTreeAddress.h", - "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: replication_port, queries_port." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeAddress.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeAddress.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 763384319, - "CodeNext": 2422251580, - "CodePrev": 2529539154, - "ErrorCode": "V688", - "FileName": "MergeTreeThreadBlockInputStream.cpp", - "Message": "The 'max_block_size_rows' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 222867506, - "CodeNext": 163883724, - "CodePrev": 2381055092, - "ErrorCode": "V688", - "FileName": "MergeTreeThreadBlockInputStream.cpp", - "Message": "The 'storage' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeThreadBlockInputStream.cpp", - "Message": "A code fragment from 'MergeTreeThreadBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeWhereOptimizer.cpp", - "Message": "A code fragment from 'MergeTreeWhereOptimizer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergedBlockOutputStream.cpp", - "Message": "A code fragment from 'MergedBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeLogEntry.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeLogEntry.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeAlterThread.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeAlterThread.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeBlockOutputStream.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeCleanupThread.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeCleanupThread.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeMutationEntry.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeMutationEntry.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SimpleMergeSelector.cpp", - "Message": "A code fragment from 'SimpleMergeSelector.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreePartCheckThread.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreePartCheckThread.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeQueue.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeQueue.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeRestartingThread.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeRestartingThread.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "checkDataPart.cpp", - "Message": "A code fragment from 'checkDataPart.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Connection.cpp", - "Message": "A code fragment from 'Connection.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 404950407, - "CodeNext": 125, - "CodePrev": 3626175879, - "ErrorCode": "V656", - "FileName": "ConnectionPoolWithFailover.cpp", - "Message": "Variables 'min_entries', 'max_entries' are initialized through the call to the same function. It's probably an error or un-optimized code. Consider inspecting the 'nested_pools.size()' expression. Check lines: _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ConnectionPoolWithFailover.cpp", - "Message": "A code fragment from 'ConnectionPoolWithFailover.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MultiplexedConnections.cpp", - "Message": "A code fragment from 'MultiplexedConnections.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerStorageMergeTree.cpp", - "Message": "A code fragment from 'registerStorageMergeTree.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionFactory.cpp", - "Message": "A code fragment from 'FunctionFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IFunction.cpp", - "Message": "A code fragment from 'IFunction.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionHelpers.cpp", - "Message": "A code fragment from 'FunctionHelpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionFactory.cpp", - "Message": "A code fragment from 'AggregateFunctionFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionCombinatorFactory.cpp", - "Message": "A code fragment from 'AggregateFunctionCombinatorFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FactoryHelpers.cpp", - "Message": "A code fragment from 'FactoryHelpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ITableFunction.cpp", - "Message": "A code fragment from 'ITableFunction.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parseAggregateFunctionParameters.cpp", - "Message": "A code fragment from 'parseAggregateFunctionParameters.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionState.cpp", - "Message": "A code fragment from 'AggregateFunctionState.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ActionLock.cpp", - "Message": "A code fragment from 'ActionLock.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CurrentMetrics.cpp", - "Message": "A code fragment from 'CurrentMetrics.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BackgroundSchedulePool.cpp", - "Message": "A code fragment from 'BackgroundSchedulePool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Allocator.cpp", - "Message": "A code fragment from 'Allocator.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DNSResolver.cpp", - "Message": "A code fragment from 'DNSResolver.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionFactory.cpp", - "Message": "A code fragment from 'TableFunctionFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 123, - "CodeNext": 125, - "CodePrev": 35837, - "ErrorCode": "V565", - "FileName": "Exception.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 143231, - "CodeNext": 125, - "CodePrev": 125, - "ErrorCode": "V565", - "FileName": "Exception.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 143231, - "CodeNext": 125, - "CodePrev": 125, - "ErrorCode": "V565", - "FileName": "Exception.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 143231, - "CodeNext": 125, - "CodePrev": 125, - "ErrorCode": "V565", - "FileName": "Exception.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 143231, - "CodeNext": 0, - "CodePrev": 125, - "ErrorCode": "V565", - "FileName": "Exception.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Exception.cpp", - "Message": "A code fragment from 'Exception.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Macros.cpp", - "Message": "A code fragment from 'Macros.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3496318343, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "FileChecker.cpp", - "Message": "The 'map' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FileChecker.cpp", - "Message": "A code fragment from 'FileChecker.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FieldVisitors.cpp", - "Message": "A code fragment from 'FieldVisitors.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1249266459, - "CodeNext": 125, - "CodePrev": 82483, - "ErrorCode": "V769", - "FileName": "SharedLibrary.cpp", - "Message": "The 'dlerror()' pointer in the expression could be nullptr. In such case, resulting value of arithmetic operations on this pointer will be senseless and it should not be used." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SharedLibrary.cpp", - "Message": "A code fragment from 'SharedLibrary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "RWLockFIFO.cpp", - "Message": "A code fragment from 'RWLockFIFO.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "config_build.cpp", - "Message": "A code fragment from 'config_build.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StackTrace.cpp", - "Message": "A code fragment from 'StackTrace.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "escapeForFileName.cpp", - "Message": "A code fragment from 'escapeForFileName.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MemoryTracker.cpp", - "Message": "A code fragment from 'MemoryTracker.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 348362777, - "CodeNext": 348528025, - "CodePrev": 897852770, - "ErrorCode": "V525", - "FileName": "ShellCommand.cpp", - "Message": "The code contains the collection of similar blocks. Check items 'write_fd', 'read_fd', 'read_fd' in lines _, _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ShellCommand.cpp", - "Message": "A code fragment from 'ShellCommand.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "formatIPv6.cpp", - "Message": "A code fragment from 'formatIPv_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getFQDNOrHostName.cpp", - "Message": "A code fragment from 'getFQDNOrHostName.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getMultipleKeysFromConfig.cpp", - "Message": "A code fragment from 'getMultipleKeysFromConfig.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getNumberOfPhysicalCPUCores.cpp", - "Message": "A code fragment from 'getNumberOfPhysicalCPUCores.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "formatReadable.cpp", - "Message": "A code fragment from 'formatReadable.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hex.cpp", - "Message": "A code fragment from 'hex.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "localBackup.cpp", - "Message": "A code fragment from 'localBackup.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getPerformanceMessage.cpp", - "Message": "A code fragment from 'getPerformanceMessage.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "isLocalAddress.cpp", - "Message": "A code fragment from 'isLocalAddress.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "randomSeed.cpp", - "Message": "A code fragment from 'randomSeed.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "setThreadName.cpp", - "Message": "A code fragment from 'setThreadName.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parseAddress.cpp", - "Message": "A code fragment from 'parseAddress.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CascadeWriteBuffer.cpp", - "Message": "A code fragment from 'CascadeWriteBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CompressedReadBuffer.cpp", - "Message": "A code fragment from 'CompressedReadBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CachedCompressedReadBuffer.cpp", - "Message": "A code fragment from 'CachedCompressedReadBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CompressedReadBufferBase.cpp", - "Message": "A code fragment from 'CompressedReadBufferBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CompressedWriteBuffer.cpp", - "Message": "A code fragment from 'CompressedWriteBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CompressedReadBufferFromFile.cpp", - "Message": "A code fragment from 'CompressedReadBufferFromFile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CompressionSettings.cpp", - "Message": "A code fragment from 'CompressionSettings.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "HexWriteBuffer.cpp", - "Message": "A code fragment from 'HexWriteBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "HTTPCommon.cpp", - "Message": "A code fragment from 'HTTPCommon.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "HashingWriteBuffer.cpp", - "Message": "A code fragment from 'HashingWriteBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "LimitReadBuffer.cpp", - "Message": "A code fragment from 'LimitReadBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MemoryReadWriteBuffer.cpp", - "Message": "A code fragment from 'MemoryReadWriteBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterserverWriteBuffer.cpp", - "Message": "A code fragment from 'InterserverWriteBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Progress.cpp", - "Message": "A code fragment from 'Progress.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReadBufferFromFile.cpp", - "Message": "A code fragment from 'ReadBufferFromFile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 286621345, - "CodeNext": 0, - "CodePrev": 3895740, - "ErrorCode": "V688", - "FileName": "AIO.h", - "Message": "The 'id' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3733418444, - "CodeNext": 380034981, - "CodePrev": 729278174, - "ErrorCode": "V1007", - "FileName": "ReadBufferAIO.cpp", - "Message": "The value from the potentially uninitialized optional 'watch' is used. Probably it is a mistake." - }, - { - "CodeCurrent": 3788532949, - "CodeNext": 0, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "ReadBufferAIO.cpp", - "Message": "The 'metric_increment' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3788532949, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ReadBufferAIO.cpp", - "Message": "The 'metric_increment' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReadBufferAIO.cpp", - "Message": "A code fragment from 'ReadBufferAIO.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 743595643, - "CodeNext": 2465481192, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "ReadBufferFromFileBase.cpp", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: clock_type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReadBufferFromFileBase.cpp", - "Message": "A code fragment from 'ReadBufferFromFileBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReadBufferFromPocoSocket.cpp", - "Message": "A code fragment from 'ReadBufferFromPocoSocket.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3733418444, - "CodeNext": 380034981, - "CodePrev": 39251785, - "ErrorCode": "V1007", - "FileName": "ReadBufferFromFileDescriptor.cpp", - "Message": "The value from the potentially uninitialized optional 'watch' is used. Probably it is a mistake." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReadBufferFromFileDescriptor.cpp", - "Message": "A code fragment from 'ReadBufferFromFileDescriptor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3116768807, - "CodeNext": 3116768807, - "CodePrev": 127832661, - "ErrorCode": "V525", - "FileName": "ReadHelpers.cpp", - "Message": "The code contains the collection of similar blocks. Check items '_', '_', '_' in lines _, _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReadHelpers.cpp", - "Message": "A code fragment from 'ReadHelpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3282098908, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "WriteBufferAIO.cpp", - "Message": "The 'metric_increment' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferAIO.cpp", - "Message": "A code fragment from 'WriteBufferAIO.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2604535988, - "CodeNext": 1150311866, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ReadWriteBufferFromHTTP.cpp", - "Message": "The 'uri' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 376832871, - "CodeNext": 21716451, - "CodePrev": 3114652973, - "ErrorCode": "V688", - "FileName": "ReadWriteBufferFromHTTP.cpp", - "Message": "The 'timeouts' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReadWriteBufferFromHTTP.cpp", - "Message": "A code fragment from 'ReadWriteBufferFromHTTP.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferFromFile.cpp", - "Message": "A code fragment from 'WriteBufferFromFile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferFromFileBase.cpp", - "Message": "A code fragment from 'WriteBufferFromFileBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferFromPocoSocket.cpp", - "Message": "A code fragment from 'WriteBufferFromPocoSocket.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferFromHTTPServerResponse.cpp", - "Message": "A code fragment from 'WriteBufferFromHTTPServerResponse.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferFromTemporaryFile.cpp", - "Message": "A code fragment from 'WriteBufferFromTemporaryFile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferFromFileDescriptor.cpp", - "Message": "A code fragment from 'WriteBufferFromFileDescriptor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferValidUTF8.cpp", - "Message": "A code fragment from 'WriteBufferValidUTF_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ZlibDeflatingWriteBuffer.cpp", - "Message": "A code fragment from 'ZlibDeflatingWriteBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ZlibInflatingReadBuffer.cpp", - "Message": "A code fragment from 'ZlibInflatingReadBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 622470420, - "CodeNext": 157897, - "CodePrev": 123, - "ErrorCode": "V525", - "FileName": "WriteHelpers.cpp", - "Message": "The code contains the collection of similar blocks. Check items '_', '_', '_', '_' in lines _, _, _, _." - }, - { - "CodeCurrent": 1208444161, - "CodeNext": 157897, - "CodePrev": 123, - "ErrorCode": "V525", - "FileName": "WriteHelpers.cpp", - "Message": "The code contains the collection of similar blocks. Check items '_', '_', '_' in lines _, _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteHelpers.cpp", - "Message": "A code fragment from 'WriteHelpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "copyData.cpp", - "Message": "A code fragment from 'copyData.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "createWriteBufferFromFileBase.cpp", - "Message": "A code fragment from 'createWriteBufferFromFileBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "createReadBufferFromFileBase.cpp", - "Message": "A code fragment from 'createReadBufferFromFileBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parseDateTimeBestEffort.cpp", - "Message": "A code fragment from 'parseDateTimeBestEffort.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "readFloatText.cpp", - "Message": "A code fragment from 'readFloatText.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "gtest_unescapeForFileName.cpp", - "Message": "A code fragment from 'gtest_unescapeForFileName.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "gtest_rw_lock_fifo.cpp.cpp", - "Message": "A code fragment from 'gtest_rw_lock_fifo.cpp.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "gtest_zkutil_test_multi_exception.cpp", - "Message": "A code fragment from 'gtest_zkutil_test_multi_exception.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StringUtils.cpp", - "Message": "A code fragment from 'StringUtils.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3791607228, - "CodeNext": 60247417, - "CodePrev": 0, - "ErrorCode": "V522", - "FileName": "gtest_cascade_and_memory_write_buffer.cpp", - "Message": "There might be dereferencing of a potential null pointer 'wbuf_readable'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "gtest_cascade_and_memory_write_buffer.cpp", - "Message": "A code fragment from 'gtest_cascade_and_memory_write_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "stopwatch.cpp", - "Message": "A code fragment from 'stopwatch.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "allocator.cpp", - "Message": "A code fragment from 'allocator.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "thread_pool.cpp", - "Message": "A code fragment from 'thread_pool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "gtest_row_source_bits_test.cpp", - "Message": "A code fragment from 'gtest_row_source_bits_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "integer_hash_tables_and_hashes.cpp", - "Message": "A code fragment from 'integer_hash_tables_and_hashes.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3255795021, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "ArrayCache.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: ptr, size, chunk." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "array_cache.cpp", - "Message": "A code fragment from 'array_cache.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "thread_creation_latency.cpp", - "Message": "A code fragment from 'thread_creation_latency.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "lru_cache.cpp", - "Message": "A code fragment from 'lru_cache.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "sip_hash_perf.cpp", - "Message": "A code fragment from 'sip_hash_perf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "arena_with_free_lists.cpp", - "Message": "A code fragment from 'arena_with_free_lists.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "auto_array.cpp", - "Message": "A code fragment from 'auto_array.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "sip_hash.cpp", - "Message": "A code fragment from 'sip_hash.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hashes_test.cpp", - "Message": "A code fragment from 'hashes_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 178542355, - "CodeNext": 1913064305, - "CodePrev": 0, - "ErrorCode": "V760", - "FileName": "small_table.cpp", - "Message": "Two identical blocks of text were found. The second block begins from line _." - }, - { - "CodeCurrent": 3824136199, - "CodeNext": 2663, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "SmallTable.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: size." - }, - { - "CodeCurrent": 132655, - "CodeNext": 1795381173, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "SmallTable.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: container, ptr." - }, - { - "CodeCurrent": 9156143, - "CodeNext": 377744136, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "SmallTable.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: container, ptr." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "small_table.cpp", - "Message": "A code fragment from 'small_table.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parallel_aggregation.cpp", - "Message": "A code fragment from 'parallel_aggregation.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AvalancheTest.cpp", - "Message": "A code fragment from 'AvalancheTest.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parallel_aggregation2.cpp", - "Message": "A code fragment from 'parallel_aggregation_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 100335, - "CodeNext": 0, - "CodePrev": 123, - "ErrorCode": "V730", - "FileName": "SpaceSaving.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: slot, hash, count, error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "space_saving.cpp", - "Message": "A code fragment from 'space_saving.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "int_hashes_perf.cpp", - "Message": "A code fragment from 'int_hashes_perf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "simple_cache.cpp", - "Message": "A code fragment from 'simple_cache.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "cow_columns.cpp", - "Message": "A code fragment from 'cow_columns.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "pod_array.cpp", - "Message": "A code fragment from 'pod_array.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 178542355, - "CodeNext": 1913064305, - "CodePrev": 0, - "ErrorCode": "V760", - "FileName": "hash_table.cpp", - "Message": "Two identical blocks of text were found. The second block begins from line _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hash_table.cpp", - "Message": "A code fragment from 'hash_table.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 238596103, - "CodeNext": 2663, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "CompactArray.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: value_l, value_r, fits_in_byte." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "compact_array.cpp", - "Message": "A code fragment from 'compact_array.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1524853010, - "CodeNext": 193, - "CodePrev": 182716498, - "ErrorCode": "V524", - "FileName": "RadixSort.h", - "Message": "It is odd that the body of 'backward' function is fully equivalent to the body of 'forward' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "radix_sort.cpp", - "Message": "A code fragment from 'radix_sort.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "shell_command_test.cpp", - "Message": "A code fragment from 'shell_command_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "rvo_test.cpp", - "Message": "A code fragment from 'rvo_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "exception.cpp", - "Message": "A code fragment from 'exception.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "move_field.cpp", - "Message": "A code fragment from 'move_field.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "string_pool.cpp", - "Message": "A code fragment from 'string_pool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "string_ref_hash.cpp", - "Message": "A code fragment from 'string_ref_hash.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "field.cpp", - "Message": "A code fragment from 'field.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "filter_stream_hitlog.cpp", - "Message": "A code fragment from 'filter_stream_hitlog.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "filter_stream.cpp", - "Message": "A code fragment from 'filter_stream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "block_row_transforms.cpp", - "Message": "A code fragment from 'block_row_transforms.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "native_streams.cpp", - "Message": "A code fragment from 'native_streams.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "expression_stream.cpp", - "Message": "A code fragment from 'expression_stream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "tab_separated_streams.cpp", - "Message": "A code fragment from 'tab_separated_streams.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "sorting_stream.cpp", - "Message": "A code fragment from 'sorting_stream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "data_type_string.cpp", - "Message": "A code fragment from 'data_type_string.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "union_stream2.cpp", - "Message": "A code fragment from 'union_stream_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "collapsing_sorted_stream.cpp", - "Message": "A code fragment from 'collapsing_sorted_stream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3646287644, - "CodeNext": 3487437108, - "CodePrev": 3633985876, - "ErrorCode": "V560", - "FileName": "data_type_get_common_type.cpp", - "Message": "A part of conditional expression is always false: !print_stack_trace." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "data_type_get_common_type.cpp", - "Message": "A code fragment from 'data_type_get_common_type.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "data_types_number_fixed.cpp", - "Message": "A code fragment from 'data_types_number_fixed.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemAsynchronousMetrics.cpp", - "Message": "A code fragment from 'StorageSystemAsynchronousMetrics.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemBuildOptions.cpp", - "Message": "A code fragment from 'StorageSystemBuildOptions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemClusters.cpp", - "Message": "A code fragment from 'StorageSystemClusters.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemDatabases.cpp", - "Message": "A code fragment from 'StorageSystemDatabases.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'StorageSystemDictionaries.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemColumns.cpp", - "Message": "A code fragment from 'StorageSystemColumns.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemEvents.cpp", - "Message": "A code fragment from 'StorageSystemEvents.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemFunctions.cpp", - "Message": "A code fragment from 'StorageSystemFunctions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1466671053, - "CodeNext": 1467990035, - "CodePrev": 1938442408, - "ErrorCode": "V601", - "FileName": "StorageSystemGraphite.cpp", - "Message": "The 'false' value is implicitly cast to the integer type." - }, - { - "CodeCurrent": 733335125, - "CodeNext": 2791142600, - "CodePrev": 1938442408, - "ErrorCode": "V601", - "FileName": "StorageSystemGraphite.cpp", - "Message": "The 'true' value is implicitly cast to the integer type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemGraphite.cpp", - "Message": "A code fragment from 'StorageSystemGraphite.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemMacros.cpp", - "Message": "A code fragment from 'StorageSystemMacros.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemMerges.cpp", - "Message": "A code fragment from 'StorageSystemMerges.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemMetrics.cpp", - "Message": "A code fragment from 'StorageSystemMetrics.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'StorageSystemModels.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemNumbers.cpp", - "Message": "A code fragment from 'StorageSystemNumbers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemOne.cpp", - "Message": "A code fragment from 'StorageSystemOne.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemParts.cpp", - "Message": "A code fragment from 'StorageSystemParts.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemProcesses.cpp", - "Message": "A code fragment from 'StorageSystemProcesses.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3754734131, - "CodeNext": 0, - "CodePrev": 3910557249, - "ErrorCode": "V688", - "FileName": "StorageSystemPartsBase.cpp", - "Message": "The 'database_column' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3833782239, - "CodeNext": 3076741407, - "CodePrev": 0, - "ErrorCode": "V525", - "FileName": "StorageSystemPartsBase.cpp", - "Message": "The code contains the collection of similar blocks. Check items 'ColumnString', 'ColumnString', 'ColumnUInt_' in lines _, _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemPartsBase.cpp", - "Message": "A code fragment from 'StorageSystemPartsBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemPartsColumns.cpp", - "Message": "A code fragment from 'StorageSystemPartsColumns.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemTables.cpp", - "Message": "A code fragment from 'StorageSystemTables.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemSettings.cpp", - "Message": "A code fragment from 'StorageSystemSettings.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemReplicas.cpp", - "Message": "A code fragment from 'StorageSystemReplicas.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemReplicationQueue.cpp", - "Message": "A code fragment from 'StorageSystemReplicationQueue.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "seek_speed_test.cpp", - "Message": "A code fragment from 'seek_speed_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemZooKeeper.cpp", - "Message": "A code fragment from 'StorageSystemZooKeeper.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "part_name.cpp", - "Message": "A code fragment from 'part_name.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hit_log.cpp", - "Message": "A code fragment from 'hit_log.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "attachSystemTables.cpp", - "Message": "A code fragment from 'attachSystemTables.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "remove_symlink_directory.cpp", - "Message": "A code fragment from 'remove_symlink_directory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "merge_selector2.cpp", - "Message": "A code fragment from 'merge_selector_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "storage_log.cpp", - "Message": "A code fragment from 'storage_log.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "part_checker.cpp", - "Message": "A code fragment from 'part_checker.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "get_current_inserts_in_replicated.cpp", - "Message": "A code fragment from 'get_current_inserts_in_replicated.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "system_numbers.cpp", - "Message": "A code fragment from 'system_numbers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTAlterQuery.cpp", - "Message": "A code fragment from 'ASTAlterQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTExpressionList.cpp", - "Message": "A code fragment from 'ASTExpressionList.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "merge_selector.cpp", - "Message": "A code fragment from 'merge_selector.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "get_abandonable_lock_in_all_partitions.cpp", - "Message": "A code fragment from 'get_abandonable_lock_in_all_partitions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTInsertQuery.cpp", - "Message": "A code fragment from 'ASTInsertQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTIdentifier.cpp", - "Message": "A code fragment from 'ASTIdentifier.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTFunction.cpp", - "Message": "A code fragment from 'ASTFunction.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTKillQueryQuery.cpp", - "Message": "A code fragment from 'ASTKillQueryQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTOrderByElement.cpp", - "Message": "A code fragment from 'ASTOrderByElement.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTLiteral.cpp", - "Message": "A code fragment from 'ASTLiteral.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTQualifiedAsterisk.cpp", - "Message": "A code fragment from 'ASTQualifiedAsterisk.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTPartition.cpp", - "Message": "A code fragment from 'ASTPartition.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTQueryWithOutput.cpp", - "Message": "A code fragment from 'ASTQueryWithOutput.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTSampleRatio.cpp", - "Message": "A code fragment from 'ASTSampleRatio.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTQueryWithOnCluster.cpp", - "Message": "A code fragment from 'ASTQueryWithOnCluster.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTSelectWithUnionQuery.cpp", - "Message": "A code fragment from 'ASTSelectWithUnionQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTSystemQuery.cpp", - "Message": "A code fragment from 'ASTSystemQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTSelectQuery.cpp", - "Message": "A code fragment from 'ASTSelectQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTTablesInSelectQuery.cpp", - "Message": "A code fragment from 'ASTTablesInSelectQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTSubquery.cpp", - "Message": "A code fragment from 'ASTSubquery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1818347258, - "CodeNext": 176909, - "CodePrev": 0, - "ErrorCode": "V526", - "FileName": "CommonParsers.cpp", - "Message": "The 'strncasecmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CommonParsers.cpp", - "Message": "A code fragment from 'CommonParsers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTWithAlias.cpp", - "Message": "A code fragment from 'ASTWithAlias.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ExpressionListParsers.cpp", - "Message": "A code fragment from 'ExpressionListParsers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Lexer.cpp", - "Message": "A code fragment from 'Lexer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ExpressionElementParsers.cpp", - "Message": "A code fragment from 'ExpressionElementParsers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IParserBase.cpp", - "Message": "A code fragment from 'IParserBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IAST.cpp", - "Message": "A code fragment from 'IAST.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserCheckQuery.cpp", - "Message": "A code fragment from 'ParserCheckQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserAlterQuery.cpp", - "Message": "A code fragment from 'ParserAlterQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserCase.cpp", - "Message": "A code fragment from 'ParserCase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserDescribeTableQuery.cpp", - "Message": "A code fragment from 'ParserDescribeTableQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserCreateQuery.cpp", - "Message": "A code fragment from 'ParserCreateQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserDropQuery.cpp", - "Message": "A code fragment from 'ParserDropQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserKillQueryQuery.cpp", - "Message": "A code fragment from 'ParserKillQueryQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserInsertQuery.cpp", - "Message": "A code fragment from 'ParserInsertQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserOptimizeQuery.cpp", - "Message": "A code fragment from 'ParserOptimizeQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserQuery.cpp", - "Message": "A code fragment from 'ParserQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserPartition.cpp", - "Message": "A code fragment from 'ParserPartition.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserQueryWithOutput.cpp", - "Message": "A code fragment from 'ParserQueryWithOutput.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserRenameQuery.cpp", - "Message": "A code fragment from 'ParserRenameQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserSelectQuery.cpp", - "Message": "A code fragment from 'ParserSelectQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserSelectWithUnionQuery.cpp", - "Message": "A code fragment from 'ParserSelectWithUnionQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserSampleRatio.cpp", - "Message": "A code fragment from 'ParserSampleRatio.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserSetQuery.cpp", - "Message": "A code fragment from 'ParserSetQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserTablePropertiesQuery.cpp", - "Message": "A code fragment from 'ParserTablePropertiesQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserShowTablesQuery.cpp", - "Message": "A code fragment from 'ParserShowTablesQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserSystemQuery.cpp", - "Message": "A code fragment from 'ParserSystemQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TokenIterator.cpp", - "Message": "A code fragment from 'TokenIterator.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserTablesInSelectQuery.cpp", - "Message": "A code fragment from 'ParserTablesInSelectQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserUnionQueryElement.cpp", - "Message": "A code fragment from 'ParserUnionQueryElement.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserUseQuery.cpp", - "Message": "A code fragment from 'ParserUseQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "formatAST.cpp", - "Message": "A code fragment from 'formatAST.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parseDatabaseAndTableName.cpp", - "Message": "A code fragment from 'parseDatabaseAndTableName.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parseIdentifierOrStringLiteral.cpp", - "Message": "A code fragment from 'parseIdentifierOrStringLiteral.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "queryToString.cpp", - "Message": "A code fragment from 'queryToString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "iostream_debug_helpers.cpp", - "Message": "A code fragment from 'iostream_debug_helpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parseQuery.cpp", - "Message": "A code fragment from 'parseQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "create_parser.cpp", - "Message": "A code fragment from 'create_parser.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "select_parser.cpp", - "Message": "A code fragment from 'select_parser.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "lexer.cpp", - "Message": "A code fragment from 'lexer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parse_date_time_best_effort.cpp", - "Message": "A code fragment from 'parse_date_time_best_effort.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "limit_read_buffer2.cpp", - "Message": "A code fragment from 'limit_read_buffer_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zlib_buffers.cpp", - "Message": "A code fragment from 'zlib_buffers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "operators.cpp", - "Message": "A code fragment from 'operators.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "read_buffer_aio.cpp", - "Message": "A code fragment from 'read_buffer_aio.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hashing_read_buffer.cpp", - "Message": "A code fragment from 'hashing_read_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hashing_write_buffer.cpp", - "Message": "A code fragment from 'hashing_write_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "valid_utf8_perf.cpp", - "Message": "A code fragment from 'valid_utf__perf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "o_direct_and_dirty_pages.cpp", - "Message": "A code fragment from 'o_direct_and_dirty_pages.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "read_buffer.cpp", - "Message": "A code fragment from 'read_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "read_float_perf.cpp", - "Message": "A code fragment from 'read_float_perf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "compressed_buffer.cpp", - "Message": "A code fragment from 'compressed_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "write_buffer_perf.cpp", - "Message": "A code fragment from 'write_buffer_perf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "write_buffer.cpp", - "Message": "A code fragment from 'write_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "read_escaped_string.cpp", - "Message": "A code fragment from 'read_escaped_string.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "read_buffer_perf.cpp", - "Message": "A code fragment from 'read_buffer_perf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "limit_read_buffer.cpp", - "Message": "A code fragment from 'limit_read_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "cached_compressed_read_buffer.cpp", - "Message": "A code fragment from 'cached_compressed_read_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "valid_utf8.cpp", - "Message": "A code fragment from 'valid_utf_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "mempbrk.cpp", - "Message": "A code fragment from 'mempbrk.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "var_uint.cpp", - "Message": "A code fragment from 'var_uint.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parse_int_perf2.cpp", - "Message": "A code fragment from 'parse_int_perf_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "write_buffer_aio.cpp", - "Message": "A code fragment from 'write_buffer_aio.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "async_write.cpp", - "Message": "A code fragment from 'async_write.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parse_int_perf.cpp", - "Message": "A code fragment from 'parse_int_perf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "read_write_int.cpp", - "Message": "A code fragment from 'read_write_int.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsCharset.cpp", - "Message": "A code fragment from 'FunctionsCharset.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsCoding.cpp", - "Message": "A code fragment from 'FunctionsCoding.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2701979474, - "CodeNext": 125, - "CodePrev": 539, - "ErrorCode": "V522", - "FileName": "FunctionsArray.h", - "Message": "There might be dereferencing of a potential null pointer 'null_map_data'." - }, - { - "CodeCurrent": 2701979474, - "CodeNext": 125, - "CodePrev": 539, - "ErrorCode": "V522", - "FileName": "FunctionsArray.h", - "Message": "There might be dereferencing of a potential null pointer 'null_map_item'." - }, - { - "CodeCurrent": 2701979474, - "CodeNext": 125, - "CodePrev": 539, - "ErrorCode": "V522", - "FileName": "FunctionsArray.h", - "Message": "There might be dereferencing of a potential null pointer 'null_map_data'." - }, - { - "CodeCurrent": 2701979474, - "CodeNext": 125, - "CodePrev": 539, - "ErrorCode": "V522", - "FileName": "FunctionsArray.h", - "Message": "There might be dereferencing of a potential null pointer 'null_map_item'." - }, - { - "CodeCurrent": 1933433026, - "CodeNext": 352513873, - "CodePrev": 1066296242, - "ErrorCode": "V522", - "FileName": "FunctionsArray.cpp", - "Message": "There might be dereferencing of a potential null pointer 'col_const_array'." - }, - { - "CodeCurrent": 3479588408, - "CodeNext": 360719, - "CodePrev": 0, - "ErrorCode": "V519", - "FileName": "FunctionsArray.cpp", - "Message": "The 'col_array' variable is assigned values twice successively. Perhaps this is a mistake. Check lines: _, _." - }, - { - "CodeCurrent": 2226276340, - "CodeNext": 2372992004, - "CodePrev": 914871094, - "ErrorCode": "V601", - "FileName": "FunctionsArray.cpp", - "Message": "The 'false' value is implicitly cast to the integer type. Inspect the second argument." - }, - { - "CodeCurrent": 642252437, - "CodeNext": 1179013152, - "CodePrev": 123, - "ErrorCode": "V601", - "FileName": "FunctionsArray.cpp", - "Message": "The 'true' value is implicitly cast to the integer type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ValueSourceVisitor.h", - "Message": "A code fragment from 'FunctionsArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'FunctionsComparison.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsConsistentHashing.cpp", - "Message": "A code fragment from 'FunctionsConsistentHashing.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 780446959, - "CodeNext": 0, - "CodePrev": 1833977828, - "ErrorCode": "V728", - "FileName": "Algorithms.h", - "Message": "An excessive check can be simplified. The '||' operator is surrounded by opposite expressions '!size_null_map' and 'size_null_map'." - }, - { - "CodeCurrent": 2874700209, - "CodeNext": 0, - "CodePrev": 539, - "ErrorCode": "V522", - "FileName": "FunctionsConditional.h", - "Message": "There might be dereferencing of a potential null pointer 'col_right_const'." - }, - { - "CodeCurrent": 4053347441, - "CodeNext": 0, - "CodePrev": 539, - "ErrorCode": "V522", - "FileName": "FunctionsConditional.h", - "Message": "There might be dereferencing of a potential null pointer 'col_right_const'." - }, - { - "CodeCurrent": 1316020648, - "CodeNext": 67675834, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "FunctionsConditional.h", - "Message": "There might be dereferencing of a potential null pointer 'col_right_const_array'." - }, - { - "CodeCurrent": 1316020648, - "CodeNext": 67675834, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "FunctionsConditional.h", - "Message": "There might be dereferencing of a potential null pointer 'col_right_const_array'." - }, - { - "CodeCurrent": 3479214347, - "CodeNext": 328401, - "CodePrev": 0, - "ErrorCode": "V519", - "FileName": "FunctionsConditional.h", - "Message": "The 'col_left' variable is assigned values twice successively. Perhaps this is a mistake. Check lines: _, _." - }, - { - "CodeCurrent": 3244155324, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V524", - "FileName": "Sources.h", - "Message": "It is odd that the body of 'getColumnSize' function is fully equivalent to the body of 'getSizeForReserve' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ValueSourceVisitor.h", - "Message": "A code fragment from 'FunctionsConditional.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 71291086, - "CodeNext": 285164697, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "FunctionsConversion.h", - "Message": "There might be dereferencing of a potential null pointer 'col_from_string'." - }, - { - "CodeCurrent": 401577488, - "CodeNext": 3512469952, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "FunctionsConversion.h", - "Message": "There might be dereferencing of a potential null pointer 'col_from_fixed_string'." - }, - { - "CodeCurrent": 2536180031, - "CodeNext": 4123240192, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "FunctionsConversion.h", - "Message": "There might be dereferencing of a potential null pointer 'offsets'." - }, - { - "CodeCurrent": 2581192971, - "CodeNext": 675433, - "CodePrev": 41145, - "ErrorCode": "V522", - "FileName": "FunctionsConversion.h", - "Message": "There might be dereferencing of a potential null pointer 'local_time_zone'." - }, - { - "CodeCurrent": 2581192971, - "CodeNext": 675433, - "CodePrev": 41145, - "ErrorCode": "V522", - "FileName": "FunctionsConversion.h", - "Message": "There might be dereferencing of a potential null pointer 'utc_time_zone'." - }, - { - "CodeCurrent": 1069551226, - "CodeNext": 125, - "CodePrev": 3401161010, - "ErrorCode": "V522", - "FileName": "FunctionsConversion.h", - "Message": "There might be dereferencing of a potential null pointer 'vec_null_map_to'." - }, - { - "CodeCurrent": 3903117136, - "CodeNext": 3272131897, - "CodePrev": 0, - "ErrorCode": "V788", - "FileName": "FunctionsConversion.h", - "Message": "Uninitialized variable 'from_nested_type' will be used in the lambda expression, as it was captured by value." - }, - { - "CodeCurrent": 3903117136, - "CodeNext": 3272131897, - "CodePrev": 0, - "ErrorCode": "V788", - "FileName": "FunctionsConversion.h", - "Message": "Uninitialized variable 'to_nested_type' will be used in the lambda expression, as it was captured by value." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsConversion.cpp", - "Message": "A code fragment from 'FunctionsConversion.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsDateTime.cpp", - "Message": "A code fragment from 'FunctionsDateTime.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsEmbeddedDictionaries.cpp", - "Message": "A code fragment from 'FunctionsEmbeddedDictionaries.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsFindCluster.cpp", - "Message": "A code fragment from 'FunctionsFindCluster.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'FunctionsExternalModels.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'FunctionsExternalDictionaries.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsFormatting.cpp", - "Message": "A code fragment from 'FunctionsFormatting.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsHashing.cpp", - "Message": "A code fragment from 'FunctionsHashing.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsHigherOrder.cpp", - "Message": "A code fragment from 'FunctionsHigherOrder.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'FunctionsLogical.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsMath.cpp", - "Message": "A code fragment from 'FunctionsMath.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1915387808, - "CodeNext": 123, - "CodePrev": 388146916, - "ErrorCode": "V688", - "FileName": "GeoUtils.h", - "Message": "The 'polygon' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'FunctionsGeo.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'FunctionsNull.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsMiscellaneous.cpp", - "Message": "A code fragment from 'FunctionsMiscellaneous.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsProjection.cpp", - "Message": "A code fragment from 'FunctionsProjection.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsRandom.cpp", - "Message": "A code fragment from 'FunctionsRandom.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsReinterpret.cpp", - "Message": "A code fragment from 'FunctionsReinterpret.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ValueSourceVisitor.h", - "Message": "A code fragment from 'FunctionsString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsStringArray.cpp", - "Message": "A code fragment from 'FunctionsStringArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3108965974, - "CodeNext": 4119542900, - "CodePrev": 5816, - "ErrorCode": "V688", - "FileName": "StringSearcher.h", - "Message": "The 'needle_size' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3108965974, - "CodeNext": 4119542900, - "CodePrev": 5816, - "ErrorCode": "V688", - "FileName": "StringSearcher.h", - "Message": "The 'needle_size' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3108965974, - "CodeNext": 4119542900, - "CodePrev": 5816, - "ErrorCode": "V688", - "FileName": "StringSearcher.h", - "Message": "The 'needle_size' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 4259932362, - "CodeNext": 4031099091, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "Volnitsky.h", - "Message": "The 'needle_size' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2578531992, - "CodeNext": 1724192963, - "CodePrev": 3112257027, - "ErrorCode": "V519", - "FileName": "Volnitsky.h", - "Message": "The 'chars.c_' variable is assigned values twice successively. Perhaps this is a mistake. Check lines: _, _." - }, - { - "CodeCurrent": 1724192963, - "CodeNext": 371674977, - "CodePrev": 2578531992, - "ErrorCode": "V519", - "FileName": "Volnitsky.h", - "Message": "The 'chars.c_' variable is assigned values twice successively. Perhaps this is a mistake. Check lines: _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'FunctionsStringSearch.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsTransform.cpp", - "Message": "A code fragment from 'FunctionsTransform.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsTuple.cpp", - "Message": "A code fragment from 'FunctionsTuple.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'FunctionsURL.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 88117, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V601", - "FileName": "FunctionsVisitParam.h", - "Message": "The 'true' value is implicitly cast to the integer type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'FunctionsVisitParam.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionAbs.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArray.cpp", - "Message": "A code fragment from 'registerFunctionArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayConcat.cpp", - "Message": "A code fragment from 'registerFunctionArrayConcat.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayElement.cpp", - "Message": "A code fragment from 'registerFunctionArrayElement.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayEnumerate.cpp", - "Message": "A code fragment from 'registerFunctionArrayEnumerate.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayHasAll.cpp", - "Message": "A code fragment from 'registerFunctionArrayHasAll.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayEnumerateUniq.cpp", - "Message": "A code fragment from 'registerFunctionArrayEnumerateUniq.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayHasAny.cpp", - "Message": "A code fragment from 'registerFunctionArrayHasAny.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayIntersect.cpp", - "Message": "A code fragment from 'registerFunctionArrayIntersect.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayPopBack.cpp", - "Message": "A code fragment from 'registerFunctionArrayPopBack.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayPopFront.cpp", - "Message": "A code fragment from 'registerFunctionArrayPopFront.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayPushFront.cpp", - "Message": "A code fragment from 'registerFunctionArrayPushFront.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayPushBack.cpp", - "Message": "A code fragment from 'registerFunctionArrayPushBack.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayReduce.cpp", - "Message": "A code fragment from 'registerFunctionArrayReduce.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayReverse.cpp", - "Message": "A code fragment from 'registerFunctionArrayReverse.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArraySlice.cpp", - "Message": "A code fragment from 'registerFunctionArraySlice.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayResize.cpp", - "Message": "A code fragment from 'registerFunctionArrayResize.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayUniq.cpp", - "Message": "A code fragment from 'registerFunctionArrayUniq.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitAnd.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitNot.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitRotateRight.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitOr.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitRotateLeft.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitShiftLeft.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitTest.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitShiftRight.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitTestAny.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitXor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitTestAll.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'FunctionsRound.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionCountEqual.cpp", - "Message": "A code fragment from 'registerFunctionCountEqual.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionDivideFloating.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionBuildProjectionComposition.cpp", - "Message": "A code fragment from 'registerFunctionBuildProjectionComposition.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionDivideIntegral.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionDivideIntegralOrZero.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayDate.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayDate.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayDateTime.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayDateTime.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayFloat32.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayFloat_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayInt16.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayFloat64.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayFloat_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayInt64.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayInt32.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayInt8.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayString.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayToSingle.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayToSingle.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayUInt16.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayUInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayUInt32.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayUInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayUInt64.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayUInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayUInt8.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayUInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionGCD.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionGreatest.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionHas.cpp", - "Message": "A code fragment from 'registerFunctionHas.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionIndexOf.cpp", - "Message": "A code fragment from 'registerFunctionIndexOf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionIntExp_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionIntExp_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionLCM.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionLeast.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionMinus.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionModulo.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionMultiply.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionNegate.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionOneOrZero.cpp", - "Message": "A code fragment from 'registerFunctionOneOrZero.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionPlus.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionRange.cpp", - "Message": "A code fragment from 'registerFunctionRange.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionProject.cpp", - "Message": "A code fragment from 'registerFunctionProject.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctions.cpp", - "Message": "A code fragment from 'registerFunctions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionRestoreProjection.cpp", - "Message": "A code fragment from 'registerFunctionRestoreProjection.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionsArray.cpp", - "Message": "A code fragment from 'registerFunctionsArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionsArithmetic.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1835696802, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "concat.cpp", - "Message": "The 'sink' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'concat.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionsProjection.cpp", - "Message": "A code fragment from 'registerFunctionsProjection.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySinkVisitor.h", - "Message": "A code fragment from 'createArraySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySinkVisitor.h", - "Message": "A code fragment from 'createArraySink.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySinkVisitor.h", - "Message": "A code fragment from 'createValueSource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 412830845, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "has.cpp", - "Message": "The 'first', 'second', 'result' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'has.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3069352387, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "resizeConstantSize.cpp", - "Message": "The 'array_source', 'value_source', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'resizeConstantSize.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 652442793, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "push.cpp", - "Message": "The 'array_source', 'value_source', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'push.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3240218462, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "sliceDynamicOffsetBounded.cpp", - "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'sliceDynamicOffsetBounded.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1330432147, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "resizeDynamicSize.cpp", - "Message": "The 'array_source', 'value_source', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'resizeDynamicSize.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3966500669, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "sliceDynamicOffsetUnbounded.cpp", - "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'sliceDynamicOffsetUnbounded.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2727169871, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "sliceFromLeftConstantOffsetBounded.cpp", - "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'sliceFromLeftConstantOffsetBounded.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "number_traits.cpp", - "Message": "A code fragment from 'number_traits.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 566054038, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "sliceFromRightConstantOffsetBounded.cpp", - "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'sliceFromRightConstantOffsetBounded.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1515238783, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "sliceFromLeftConstantOffsetUnbounded.cpp", - "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'sliceFromLeftConstantOffsetUnbounded.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1741194879, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "sliceFromRightConstantOffsetUnbounded.cpp", - "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'sliceFromRightConstantOffsetUnbounded.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hash_map.cpp", - "Message": "A code fragment from 'hash_map.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1419470395, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V524", - "FileName": "hash_map_string_2.cpp", - "Message": "It is odd that the body of 'compare_byIntSSE' function is fully equivalent to the body of 'compareSSE_' function (StringRef.h, line _)." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hash_map_string_2.cpp", - "Message": "A code fragment from 'hash_map_string__.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hash_map3.cpp", - "Message": "A code fragment from 'hash_map_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "select_query.cpp", - "Message": "A code fragment from 'select_query.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "aggregate.cpp", - "Message": "A code fragment from 'aggregate.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hash_map_string_3.cpp", - "Message": "A code fragment from 'hash_map_string__.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 15386799, - "CodeNext": 0, - "CodePrev": 193061534, - "ErrorCode": "V730", - "FileName": "hash_map_string_small.cpp", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: size." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hash_map_string_small.cpp", - "Message": "A code fragment from 'hash_map_string_small.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "two_level_hash_map.cpp", - "Message": "A code fragment from 'two_level_hash_map.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "expression.cpp", - "Message": "A code fragment from 'expression.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "compiler_test.cpp", - "Message": "A code fragment from 'compiler_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "create_query.cpp", - "Message": "A code fragment from 'create_query.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "logical_expressions_optimizer.cpp", - "Message": "A code fragment from 'logical_expressions_optimizer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "in_join_subqueries_preprocessor.cpp", - "Message": "A code fragment from 'in_join_subqueries_preprocessor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hash_map_string.cpp", - "Message": "A code fragment from 'hash_map_string.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "users.cpp", - "Message": "A code fragment from 'users.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionArray.cpp", - "Message": "A code fragment from 'AggregateFunctionArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionAvg.cpp", - "Message": "A code fragment from 'AggregateFunctionAvg.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionBitwise.cpp", - "Message": "A code fragment from 'AggregateFunctionBitwise.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionCount.cpp", - "Message": "A code fragment from 'AggregateFunctionCount.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionForEach.cpp", - "Message": "A code fragment from 'AggregateFunctionForEach.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionGroupArray.cpp", - "Message": "A code fragment from 'AggregateFunctionGroupArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionGroupArrayInsertAt.cpp", - "Message": "A code fragment from 'AggregateFunctionGroupArrayInsertAt.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionGroupUniqArray.cpp", - "Message": "A code fragment from 'AggregateFunctionGroupUniqArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionIf.cpp", - "Message": "A code fragment from 'AggregateFunctionIf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionMerge.cpp", - "Message": "A code fragment from 'AggregateFunctionMerge.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionMaxIntersections.cpp", - "Message": "A code fragment from 'AggregateFunctionMaxIntersections.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2360632089, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V522", - "FileName": "AggregateFunctionNull.cpp", - "Message": "There might be dereferencing of a potential null pointer 'nested_function'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionNull.cpp", - "Message": "A code fragment from 'AggregateFunctionNull.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2106398958, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V730", - "FileName": "AggregateFunctionMinMaxAny.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: large_data, small_data." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionMinMaxAny.cpp", - "Message": "A code fragment from 'AggregateFunctionMinMaxAny.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionStatistics.cpp", - "Message": "A code fragment from 'AggregateFunctionStatistics.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionSequenceMatch.cpp", - "Message": "A code fragment from 'AggregateFunctionSequenceMatch.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 707089, - "CodeNext": 3039935502, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "QuantileTiming.h", - "Message": "An odd precise comparison: level != _. It's probably better to use a comparison with defined precision: fabs(A - B) > Epsilon." - }, - { - "CodeCurrent": 407249, - "CodeNext": 172219641, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "QuantileTiming.h", - "Message": "The 'count' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionQuantile.cpp", - "Message": "A code fragment from 'AggregateFunctionQuantile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionSum.cpp", - "Message": "A code fragment from 'AggregateFunctionSum.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionSumMap.cpp", - "Message": "A code fragment from 'AggregateFunctionSumMap.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionTopK.cpp", - "Message": "A code fragment from 'AggregateFunctionTopK.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionUniq.cpp", - "Message": "A code fragment from 'AggregateFunctionUniq.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionUniqUpTo.cpp", - "Message": "A code fragment from 'AggregateFunctionUniqUpTo.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "UniqCombinedBiasData.cpp", - "Message": "A code fragment from 'UniqCombinedBiasData.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerAggregateFunctions.cpp", - "Message": "A code fragment from 'registerAggregateFunctions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionWindowFunnel.cpp", - "Message": "A code fragment from 'AggregateFunctionWindowFunnel.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionsStatisticsSimple.cpp", - "Message": "A code fragment from 'AggregateFunctionsStatisticsSimple.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'ExtractFromConfig.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'Benchmark.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1973016635, - "CodeNext": 0, - "CodePrev": 1667937151, - "ErrorCode": "V688", - "FileName": "ClusterCopier.cpp", - "Message": "The 'pid' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 158502902, - "CodeNext": 73185914, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "ClusterCopier.cpp", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: task_descprtion_current_stat." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'ClusterCopier.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 254105, - "CodeNext": 8134297, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "Client.cpp", - "Message": "The 'query' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 254105, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "Client.cpp", - "Message": "The 'query' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 862425183, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "Client.cpp", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: port, security, compression." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'Client.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'HTTPHandler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NotFoundHandler.cpp", - "Message": "A code fragment from 'NotFoundHandler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterserverIOHTTPHandler.cpp", - "Message": "A code fragment from 'InterserverIOHTTPHandler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MetricsTransmitter.cpp", - "Message": "A code fragment from 'MetricsTransmitter.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PingRequestHandler.cpp", - "Message": "A code fragment from 'PingRequestHandler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "RootRequestHandler.cpp", - "Message": "A code fragment from 'RootRequestHandler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StatusFile.cpp", - "Message": "A code fragment from 'StatusFile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicasStatusHandler.cpp", - "Message": "A code fragment from 'ReplicasStatusHandler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Server.cpp", - "Message": "A code fragment from 'Server.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1810641279, - "CodeNext": 2826637927, - "CodePrev": 123, - "ErrorCode": "V1002", - "FileName": "TCPHandler.cpp", - "Message": "The 'Context' class, containing pointers, constructor and destructor, is copied by the automatically generated operator=." - }, - { - "CodeCurrent": 2826637927, - "CodeNext": 0, - "CodePrev": 1810641279, - "ErrorCode": "V678", - "FileName": "TCPHandler.cpp", - "Message": "An object is used as an argument to its own method. Consider checking the first actual argument of the 'setSessionContext' function." - }, - { - "CodeCurrent": 143231, - "CodeNext": 0, - "CodePrev": 125, - "ErrorCode": "V565", - "FileName": "TCPHandler.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 4294129027, - "CodeNext": 0, - "CodePrev": 4133442322, - "ErrorCode": "V1002", - "FileName": "TCPHandler.cpp", - "Message": "The 'Context' class, containing pointers, constructor and destructor, is copied by the automatically generated operator=." - }, - { - "CodeCurrent": 1787830936, - "CodeNext": 123, - "CodePrev": 125, - "ErrorCode": "V759", - "FileName": "TCPHandler.cpp", - "Message": "Incorrect order of exception handling blocks. Exception of 'NetException' type will be caught in a catch block for 'Exception' base type." - }, - { - "CodeCurrent": 123, - "CodeNext": 2434086205, - "CodePrev": 35837, - "ErrorCode": "V565", - "FileName": "TCPHandler.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'TCPHandler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3472545402, - "CodeNext": 123, - "CodePrev": 125, - "ErrorCode": "V688", - "FileName": "PerformanceTest.cpp", - "Message": "The 'padding' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 188108971, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V550", - "FileName": "PerformanceTest.cpp", - "Message": "An odd precise comparison: avg_speed_first == _. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 876333961, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "PerformanceTest.cpp", - "Message": "The 'input_files' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2173845642, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "PerformanceTest.cpp", - "Message": "The 'substitutions' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 596635228, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "PerformanceTest.cpp", - "Message": "The 'substitutions' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1500859456, - "CodeNext": 0, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "PerformanceTest.cpp", - "Message": "The 'queries' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3001731668, - "CodeNext": 3796428353, - "CodePrev": 388618255, - "ErrorCode": "V688", - "FileName": "PerformanceTest.cpp", - "Message": "The 'queries' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2531644379, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "PerformanceTest.cpp", - "Message": "The 'main_metric' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'PerformanceTest.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'Compressor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 35857945, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "LocalServer.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 201131425, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "LocalServer.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'LocalServer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'Format.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionCatBoostPool.cpp", - "Message": "A code fragment from 'TableFunctionCatBoostPool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3790144045, - "CodeNext": 100, - "CodePrev": 123, - "ErrorCode": "V601", - "FileName": "main.cpp", - "Message": "The 'false' value is implicitly cast to the integer type. Inspect the second argument." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "main.cpp", - "Message": "A code fragment from 'main.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'TableFunctionFile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionMerge.cpp", - "Message": "A code fragment from 'TableFunctionMerge.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionNumbers.cpp", - "Message": "A code fragment from 'TableFunctionNumbers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionMySQL.cpp", - "Message": "A code fragment from 'TableFunctionMySQL.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerTableFunctions.cpp", - "Message": "A code fragment from 'registerTableFunctions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionODBC.cpp", - "Message": "A code fragment from 'TableFunctionODBC.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "translate_positional_arguments.cpp", - "Message": "A code fragment from 'translate_positional_arguments.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionRemote.cpp", - "Message": "A code fragment from 'TableFunctionRemote.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionShardByHash.cpp", - "Message": "A code fragment from 'TableFunctionShardByHash.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "collect_aliases.cpp", - "Message": "A code fragment from 'collect_aliases.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "analyze_result_of_query.cpp", - "Message": "A code fragment from 'analyze_result_of_query.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "type_and_constant_inference.cpp", - "Message": "A code fragment from 'type_and_constant_inference.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "analyze_columns.cpp", - "Message": "A code fragment from 'analyze_columns.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "collect_tables.cpp", - "Message": "A code fragment from 'collect_tables.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Lock.cpp", - "Message": "A code fragment from 'Lock.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "analyze_lambdas.cpp", - "Message": "A code fragment from 'analyze_lambdas.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "optimize_group_order_limit_by.cpp", - "Message": "A code fragment from 'optimize_group_order_limit_by.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ZooKeeperHolder.cpp", - "Message": "A code fragment from 'ZooKeeperHolder.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ZooKeeperNodeCache.cpp", - "Message": "A code fragment from 'ZooKeeperNodeCache.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 4277102732, - "CodeNext": 3079148690, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "ZooKeeperImpl.cpp", - "Message": "The 'out' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 20953377, - "CodeNext": 586135571, - "CodePrev": 4018551741, - "ErrorCode": "V688", - "FileName": "ZooKeeperImpl.cpp", - "Message": "The 'session_id' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 6647, - "CodeNext": 81911, - "CodePrev": 325699, - "ErrorCode": "V688", - "FileName": "ZooKeeperImpl.cpp", - "Message": "The 'xid' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 6647, - "CodeNext": 81911, - "CodePrev": 325699, - "ErrorCode": "V688", - "FileName": "ZooKeeperImpl.cpp", - "Message": "The 'xid' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 163059, - "CodeNext": 0, - "CodePrev": 18353, - "ErrorCode": "V688", - "FileName": "ZooKeeperImpl.cpp", - "Message": "The 'error' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ZooKeeperImpl.cpp", - "Message": "A code fragment from 'ZooKeeperImpl.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zookeeper_impl.cpp", - "Message": "A code fragment from 'zookeeper_impl.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3719570876, - "CodeNext": 67617425, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ZooKeeper.cpp", - "Message": "The 'hosts' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3719570876, - "CodeNext": 67617425, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ZooKeeper.cpp", - "Message": "The 'identity' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 67617425, - "CodeNext": 123, - "CodePrev": 3719570876, - "ErrorCode": "V688", - "FileName": "ZooKeeper.cpp", - "Message": "The 'session_timeout_ms' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 67617425, - "CodeNext": 123, - "CodePrev": 3719570876, - "ErrorCode": "V688", - "FileName": "ZooKeeper.cpp", - "Message": "The 'chroot' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1028915392, - "CodeNext": 2727151059, - "CodePrev": 0, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 1028915392, - "CodeNext": 2727151059, - "CodePrev": 0, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 1530511028, - "CodeNext": 3013342653, - "CodePrev": 1169372895, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 3013342653, - "CodeNext": 2123395508, - "CodePrev": 1530511028, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 2123395508, - "CodeNext": 2727151059, - "CodePrev": 3013342653, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 3807645422, - "CodeNext": 5607, - "CodePrev": 0, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 1530511028, - "CodeNext": 3013328497, - "CodePrev": 1169372895, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 3013328497, - "CodeNext": 1827075214, - "CodePrev": 1530511028, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 1827075214, - "CodeNext": 2727151059, - "CodePrev": 3013328497, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 1028915392, - "CodeNext": 2727151059, - "CodePrev": 0, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 1028915392, - "CodeNext": 2727151059, - "CodePrev": 0, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 1530511028, - "CodeNext": 3013328526, - "CodePrev": 1169372895, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 3013328526, - "CodeNext": 2727151059, - "CodePrev": 1530511028, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 3365467161, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ZooKeeper.cpp", - "Message": "The 'code' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3365467161, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ZooKeeper.cpp", - "Message": "The 'responses' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'ZooKeeper.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zkutil_expiration_test.cpp", - "Message": "A code fragment from 'zkutil_expiration_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zkutil_zookeeper_holder.cpp", - "Message": "A code fragment from 'zkutil_zookeeper_holder.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zkutil_test_lock.cpp", - "Message": "A code fragment from 'zkutil_test_lock.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zkutil_test_commands.cpp", - "Message": "A code fragment from 'zkutil_test_commands.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zk_many_watches_reconnect.cpp", - "Message": "A code fragment from 'zk_many_watches_reconnect.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ConfigProcessor.cpp", - "Message": "A code fragment from 'ConfigProcessor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zkutil_test_async.cpp", - "Message": "A code fragment from 'zkutil_test_async.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'zkutil_test_commands_new_lib.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "dictionary_library.cpp", - "Message": "A code fragment from 'dictionary_library.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ConfigReloader.cpp", - "Message": "A code fragment from 'ConfigReloader.cpp' cannot be analyzed." - } - ] -} \ No newline at end of file + "version": 1, + "warnings": [ + { + "CodeCurrent": 1403077733, + "CodeNext": 951341, + "CodePrev": 1270846060, + "ErrorCode": "V550", + "FileName": "preciseExp10.c", + "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(y) < Epsilon." + }, + { + "CodeCurrent": 1403077733, + "CodeNext": 3854685, + "CodePrev": 1391368717, + "ErrorCode": "V550", + "FileName": "preciseExp10.c", + "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(y) < Epsilon." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "shift10.cpp", + "Message": "A code fragment from 'shift_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1360858873, + "CodeNext": 0, + "CodePrev": 864691173, + "ErrorCode": "V524", + "FileName": "DateLUTImpl.h", + "Message": "It is odd that the body of 'getValues' function is fully equivalent to the body of 'find' function." + }, + { + "CodeCurrent": 3167014141, + "CodeNext": 1528350630, + "CodePrev": 0, + "ErrorCode": "V524", + "FileName": "DateLUTImpl.h", + "Message": "It is odd that the body of 'toDate' function is fully equivalent to the body of 'fromDayNum' function." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DateLUTImpl.cpp", + "Message": "A code fragment from 'DateLUTImpl.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "demangle.cpp", + "Message": "A code fragment from 'demangle.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ThreadPool.cpp", + "Message": "A code fragment from 'ThreadPool.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 18396, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V690", + "FileName": "JSON.h", + "Message": "The 'JSON' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 1301811264, + "CodeNext": 123, + "CodePrev": 44642, + "ErrorCode": "V524", + "FileName": "JSON.cpp", + "Message": "It is odd that the body of 'get' function is fully equivalent to the body of 'getName' function." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "JSON.cpp", + "Message": "A code fragment from 'JSON.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DateLUT.cpp", + "Message": "A code fragment from 'DateLUT.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "SetTerminalEcho.cpp", + "Message": "A code fragment from 'SetTerminalEcho.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "date_lut2.cpp", + "Message": "A code fragment from 'date_lut_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "date_lut3.cpp", + "Message": "A code fragment from 'date_lut_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "gtest_strong_typedef.cpp", + "Message": "A code fragment from 'gtest_strong_typedef.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "date_lut4.cpp", + "Message": "A code fragment from 'date_lut_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "dump_variable.cpp", + "Message": "A code fragment from 'dump_variable.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "date_lut_init.cpp", + "Message": "A code fragment from 'date_lut_init.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "gtest_json_test.cpp", + "Message": "A code fragment from 'gtest_json_test.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "multi_version.cpp", + "Message": "A code fragment from 'multi_version.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "date_lut_default_timezone.cpp", + "Message": "A code fragment from 'date_lut_default_timezone.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "LevelFilterChannel.cpp", + "Message": "A code fragment from 'LevelFilterChannel.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "SessionPoolHelpers.cpp", + "Message": "A code fragment from 'SessionPoolHelpers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 123, + "CodeNext": 2077384594, + "CodePrev": 2854941240, + "ErrorCode": "V565", + "FileName": "BaseDaemon.h", + "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "GraphiteWriter.cpp", + "Message": "A code fragment from 'GraphiteWriter.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3676031242, + "CodeNext": 3056897128, + "CodePrev": 1102711319, + "ErrorCode": "V707", + "FileName": "lgamma.c", + "Message": "Giving short names to global variables is considered to be bad practice. It is suggested to rename 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'tc', 'tf', 'tt', 't_', 't_', 't_', 't_', 't_', 't_', 't_', 't_', 't_', 't_', 'u_', 'u_', 'u_', 'u_', 'u_', 'u_', 'v_', 'v_', 'v_', 'v_', 'v_', 's_', 's_', 's_', 's_', 's_', 's_', 's_', 'r_', 'r_', 'r_', 'r_', 'r_', 'r_', 'w_', 'w_', 'w_', 'w_', 'w_', 'w_', 'w_' variables." + }, + { + "CodeCurrent": 196849995, + "CodeNext": 2821665, + "CodePrev": 257609, + "ErrorCode": "V550", + "FileName": "lgamma.c", + "Message": "An odd precise comparison: t == _._. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." + }, + { + "CodeCurrent": 2367177615, + "CodeNext": 1620191745, + "CodePrev": 47045, + "ErrorCode": "V605", + "FileName": "posix_spawn.c", + "Message": "Consider verifying the expression. An unsigned value is compared to the number belonging to the range [-_.._]." + }, + { + "CodeCurrent": 2676964558, + "CodeNext": 17733, + "CodePrev": 17733, + "ErrorCode": "V526", + "FileName": "glob.c", + "Message": "The 'strcmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "exp2f.c", + "Message": "A code fragment from 'exp_f.c' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "OwnPatternFormatter.cpp", + "Message": "A code fragment from 'OwnPatternFormatter.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1586910037, + "CodeNext": 1159405304, + "CodePrev": 123, + "ErrorCode": "V769", + "FileName": "BaseDaemon.cpp", + "Message": "The 'name_start' pointer in the 'name_start - symbols[i]' expression could be nullptr. In such case, resulting value will be senseless and it should not be used." + }, + { + "CodeCurrent": 123, + "CodeNext": 125, + "CodePrev": 35837, + "ErrorCode": "V565", + "FileName": "BaseDaemon.cpp", + "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "BaseDaemon.cpp", + "Message": "A code fragment from 'BaseDaemon.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Exception.cpp", + "Message": "A code fragment from 'Exception.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2045, + "CodeNext": 123, + "CodePrev": 2452368582, + "ErrorCode": "V730", + "FileName": "Row.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: lengths." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Connection.cpp", + "Message": "A code fragment from 'Connection.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 123, + "CodeNext": 2202583, + "CodePrev": 1128383580, + "ErrorCode": "V794", + "FileName": "Query.cpp", + "Message": "The assignment operator should be protected from the case of 'this == &other'." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Query.cpp", + "Message": "A code fragment from 'Query.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ResultBase.cpp", + "Message": "A code fragment from 'ResultBase.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2377956688, + "CodeNext": 703533, + "CodePrev": 2042816737, + "ErrorCode": "V688", + "FileName": "StoreQueryResult.cpp", + "Message": "The 'fields' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StoreQueryResult.cpp", + "Message": "A code fragment from 'StoreQueryResult.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Row.cpp", + "Message": "A code fragment from 'Row.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Value.cpp", + "Message": "A code fragment from 'Value.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "UseQueryResult.cpp", + "Message": "A code fragment from 'UseQueryResult.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "consistent_hashing.cpp", + "Message": "A code fragment from 'consistent_hashing.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Pool.cpp", + "Message": "A code fragment from 'Pool.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 188688957, + "CodeNext": 125, + "CodePrev": 769, + "ErrorCode": "V658", + "FileName": "sumbur.cpp", + "Message": "A value is being subtracted from the unsigned variable. This can result in an overflow. In such a case, the '<' comparison operation can potentially behave unexpectedly. Consider inspecting the '_xFFFFFFFF / _ - h < part' expression." + }, + { + "CodeCurrent": 450631639, + "CodeNext": 22890, + "CodePrev": 0, + "ErrorCode": "V1003", + "FileName": "sumbur.cpp", + "Message": "The macro 'curslice' is a dangerous expression. The parameter 'i' must be surrounded by parentheses." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "sumbur.cpp", + "Message": "A code fragment from 'sumbur.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "PoolWithFailover.cpp", + "Message": "A code fragment from 'PoolWithFailover.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "mysqlxx_test.cpp", + "Message": "A code fragment from 'mysqlxx_test.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "zstd_test.cpp", + "Message": "A code fragment from 'zstd_test.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "iotest.cpp", + "Message": "A code fragment from 'iotest.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "iotest_nonblock.cpp", + "Message": "A code fragment from 'iotest_nonblock.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "iotest_aio.cpp", + "Message": "A code fragment from 'iotest_aio.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "corrector_utf8.cpp", + "Message": "A code fragment from 'corrector_utf_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "config-processor.cpp", + "Message": "A code fragment from 'config-processor.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2919138394, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V690", + "FileName": "ZooKeeperImpl.h", + "Message": "The 'Exception' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "zookeeper-cli.cpp", + "Message": "A code fragment from 'zookeeper-cli.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'main.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "main.cpp", + "Message": "A code fragment from 'main.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1244410449, + "CodeNext": 123, + "CodePrev": 182455598, + "ErrorCode": "V730", + "FileName": "HashTable.h", + "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: zero_value_storage." + }, + { + "CodeCurrent": 3824136199, + "CodeNext": 2663, + "CodePrev": 5816, + "ErrorCode": "V730", + "FileName": "HashTable.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: size." + }, + { + "CodeCurrent": 1712271, + "CodeNext": 0, + "CodePrev": 3363495029, + "ErrorCode": "V730", + "FileName": "MarkovModel.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: n." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MarkovModel.h", + "Message": "A code fragment from 'main.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 259986933, + "CodeNext": 0, + "CodePrev": 123, + "ErrorCode": "V730", + "FileName": "RWLock.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: type." + }, + { + "CodeCurrent": 6265, + "CodeNext": 5125961, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "Field.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: storage." + }, + { + "CodeCurrent": 101009567, + "CodeNext": 123, + "CodePrev": 123, + "ErrorCode": "V730", + "FileName": "Field.h", + "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: storage, which." + }, + { + "CodeCurrent": 6341727, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "Field.h", + "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: storage, which." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AnalyzeColumns.cpp", + "Message": "A code fragment from 'AnalyzeColumns.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'markov-model.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AnalyzeLambdas.cpp", + "Message": "A code fragment from 'AnalyzeLambdas.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CollectAliases.cpp", + "Message": "A code fragment from 'CollectAliases.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3859608575, + "CodeNext": 125, + "CodePrev": 123, + "ErrorCode": "V550", + "FileName": "AccurateComparison.h", + "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." + }, + { + "CodeCurrent": 3859608575, + "CodeNext": 125, + "CodePrev": 123, + "ErrorCode": "V550", + "FileName": "AccurateComparison.h", + "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." + }, + { + "CodeCurrent": 728751127, + "CodeNext": 125, + "CodePrev": 123, + "ErrorCode": "V550", + "FileName": "AccurateComparison.h", + "Message": "An odd precise comparison: f == static_cast< DB::Float_ > (u). It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." + }, + { + "CodeCurrent": 1801682850, + "CodeNext": 125, + "CodePrev": 123, + "ErrorCode": "V550", + "FileName": "AccurateComparison.h", + "Message": "An odd precise comparison: static_cast< DB::Float_ > (u) == f. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." + }, + { + "CodeCurrent": 134227181, + "CodeNext": 125, + "CodePrev": 123, + "ErrorCode": "V550", + "FileName": "AccurateComparison.h", + "Message": "An odd precise comparison: f == static_cast< DB::Float_ > (u). It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." + }, + { + "CodeCurrent": 940474457, + "CodeNext": 125, + "CodePrev": 123, + "ErrorCode": "V550", + "FileName": "AccurateComparison.h", + "Message": "An odd precise comparison: static_cast< DB::Float_ > (u) == f. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." + }, + { + "CodeCurrent": 728751127, + "CodeNext": 125, + "CodePrev": 123, + "ErrorCode": "V550", + "FileName": "AccurateComparison.h", + "Message": "An odd precise comparison: f == static_cast< DB::Float_ > (u). It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." + }, + { + "CodeCurrent": 1801682850, + "CodeNext": 125, + "CodePrev": 123, + "ErrorCode": "V550", + "FileName": "AccurateComparison.h", + "Message": "An odd precise comparison: static_cast< DB::Float_ > (u) == f. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." + }, + { + "CodeCurrent": 134227181, + "CodeNext": 125, + "CodePrev": 123, + "ErrorCode": "V550", + "FileName": "AccurateComparison.h", + "Message": "An odd precise comparison: f == static_cast< DB::Float_ > (u). It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." + }, + { + "CodeCurrent": 940474457, + "CodeNext": 125, + "CodePrev": 123, + "ErrorCode": "V550", + "FileName": "AccurateComparison.h", + "Message": "An odd precise comparison: static_cast< DB::Float_ > (u) == f. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." + }, + { + "CodeCurrent": 3765134313, + "CodeNext": 3685848331, + "CodePrev": 1405257529, + "ErrorCode": "V550", + "FileName": "FieldVisitors.h", + "Message": "An odd precise comparison: l == r. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." + }, + { + "CodeCurrent": 51079373, + "CodeNext": 0, + "CodePrev": 836015037, + "ErrorCode": "V550", + "FileName": "FieldVisitors.h", + "Message": "An odd precise comparison: x != _. It's probably better to use a comparison with defined precision: fabs(A - B) > Epsilon." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AnalyzeResultOfQuery.cpp", + "Message": "A code fragment from 'AnalyzeResultOfQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CollectTables.cpp", + "Message": "A code fragment from 'CollectTables.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ExecuteTableFunctions.cpp", + "Message": "A code fragment from 'ExecuteTableFunctions.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "OptimizeGroupOrderLimitBy.cpp", + "Message": "A code fragment from 'OptimizeGroupOrderLimitBy.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TranslatePositionalArguments.cpp", + "Message": "A code fragment from 'TranslatePositionalArguments.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "BlockInfo.cpp", + "Message": "A code fragment from 'BlockInfo.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 4174113710, + "CodeNext": 3531930912, + "CodePrev": 181464845, + "ErrorCode": "V659", + "FileName": "ColumnConst.h", + "Message": "Declarations of functions with 'getDataColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Block.cpp", + "Message": "A code fragment from 'Block.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 199355300, + "CodeNext": 123, + "CodePrev": 123, + "ErrorCode": "V730", + "FileName": "ExpressionActions.h", + "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: type." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TypeAndConstantInference.cpp", + "Message": "A code fragment from 'TypeAndConstantInference.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ColumnWithTypeAndName.cpp", + "Message": "A code fragment from 'ColumnWithTypeAndName.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Field.cpp", + "Message": "A code fragment from 'Field.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "SortDescription.cpp", + "Message": "A code fragment from 'SortDescription.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "NamesAndTypes.cpp", + "Message": "A code fragment from 'NamesAndTypes.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "iostream_debug_helpers.cpp", + "Message": "A code fragment from 'iostream_debug_helpers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3826643199, + "CodeNext": 123, + "CodePrev": 22807034, + "ErrorCode": "V690", + "FileName": "ColumnVector.h", + "Message": "Copy constructor is declared as private in the 'ColumnVector' class, but the default '=' operator will still be generated by compiler. It is dangerous to use such a class." + }, + { + "CodeCurrent": 1132122309, + "CodeNext": 0, + "CodePrev": 931267045, + "ErrorCode": "V659", + "FileName": "ColumnArray.h", + "Message": "Declarations of functions with 'getData' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." + }, + { + "CodeCurrent": 1116407654, + "CodeNext": 0, + "CodePrev": 1356382257, + "ErrorCode": "V659", + "FileName": "ColumnArray.h", + "Message": "Declarations of functions with 'getOffsetsColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." + }, + { + "CodeCurrent": 2390648255, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V659", + "FileName": "ColumnArray.h", + "Message": "Declarations of functions with 'getOffsets' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AddingDefaultBlockOutputStream.cpp", + "Message": "A code fragment from 'AddingDefaultBlockOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3168042121, + "CodeNext": 123, + "CodePrev": 123, + "ErrorCode": "V690", + "FileName": "ColumnString.h", + "Message": "Copy constructor is declared as private in the 'ColumnString' class, but the default '=' operator will still be generated by compiler. It is dangerous to use such a class." + }, + { + "CodeCurrent": 4054773003, + "CodeNext": 123, + "CodePrev": 123, + "ErrorCode": "V690", + "FileName": "ColumnAggregateFunction.h", + "Message": "Copy constructor is declared as private in the 'ColumnAggregateFunction' class, but the default '=' operator will still be generated by compiler. It is dangerous to use such a class." + }, + { + "CodeCurrent": 3194944389, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "AggregatingSortedBlockInputStream.cpp", + "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 14743151, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "SortCursor.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: order." + }, + { + "CodeCurrent": 195436, + "CodeNext": 123, + "CodePrev": 45270, + "ErrorCode": "V730", + "FileName": "MergingSortedBlockInputStream.h", + "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: row_num." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregatingSortedBlockInputStream.cpp", + "Message": "A code fragment from 'AggregatingSortedBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 4136571990, + "CodeNext": 123, + "CodePrev": 123, + "ErrorCode": "V690", + "FileName": "ColumnFixedString.h", + "Message": "Copy constructor is declared as private in the 'ColumnFixedString' class, but the default '=' operator will still be generated by compiler. It is dangerous to use such a class." + }, + { + "CodeCurrent": 1054674186, + "CodeNext": 0, + "CodePrev": 925160603, + "ErrorCode": "V659", + "FileName": "ColumnNullable.h", + "Message": "Declarations of functions with 'getNestedColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." + }, + { + "CodeCurrent": 3317648788, + "CodeNext": 0, + "CodePrev": 3497477841, + "ErrorCode": "V659", + "FileName": "ColumnNullable.h", + "Message": "Declarations of functions with 'getNullMapColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." + }, + { + "CodeCurrent": 146511793, + "CodeNext": 1296505, + "CodePrev": 196592208, + "ErrorCode": "V522", + "FileName": "Aggregator.h", + "Message": "There might be dereferencing of a potential null pointer 'null_map'." + }, + { + "CodeCurrent": 132655, + "CodeNext": 0, + "CodePrev": 5816, + "ErrorCode": "V730", + "FileName": "TwoLevelHashTable.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: container, bucket, current_it." + }, + { + "CodeCurrent": 9156143, + "CodeNext": 1642228070, + "CodePrev": 5816, + "ErrorCode": "V730", + "FileName": "TwoLevelHashTable.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: container, bucket, current_it." + }, + { + "CodeCurrent": 3310514761, + "CodeNext": 2820625488, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "Aggregator.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: keys_size." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregatingBlockInputStream.cpp", + "Message": "A code fragment from 'AggregatingBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "BinaryRowInputStream.cpp", + "Message": "A code fragment from 'BinaryRowInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "BinaryRowOutputStream.cpp", + "Message": "A code fragment from 'BinaryRowOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ApplyingMutationsBlockInputStream.cpp", + "Message": "A code fragment from 'ApplyingMutationsBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 389785, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V690", + "FileName": "BlockIO.h", + "Message": "The 'BlockIO' class implements the '=' operator, but lacks a copy constructor. It is dangerous to use such a class." + }, + { + "CodeCurrent": 253306735, + "CodeNext": 4054391213, + "CodePrev": 0, + "ErrorCode": "V550", + "FileName": "ProcessList.h", + "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(memory_tracker_fault_probability) > Epsilon." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "BlockIO.cpp", + "Message": "A code fragment from 'BlockIO.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "BlockOutputStreamFromRowOutputStream.cpp", + "Message": "A code fragment from 'BlockOutputStreamFromRowOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2065830648, + "CodeNext": 2809, + "CodePrev": 0, + "ErrorCode": "V550", + "FileName": "BlockInputStreamFromRowInputStream.cpp", + "Message": "An odd precise comparison: allow_errors_ratio == _. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." + }, + { + "CodeCurrent": 123, + "CodeNext": 485783650, + "CodePrev": 35837, + "ErrorCode": "V565", + "FileName": "BlockInputStreamFromRowInputStream.cpp", + "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "BlockInputStreamFromRowInputStream.cpp", + "Message": "A code fragment from 'BlockInputStreamFromRowInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "BlockStreamProfileInfo.cpp", + "Message": "A code fragment from 'BlockStreamProfileInfo.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2244466177, + "CodeNext": 3040549265, + "CodePrev": 1036506627, + "ErrorCode": "V656", + "FileName": "CSVRowInputStream.cpp", + "Message": "Variables 'prev_position', 'curr_position' are initialized through the call to the same function. It's probably an error or un-optimized code. Consider inspecting the 'istr.position()' expression. Check lines: _, _." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CSVRowInputStream.cpp", + "Message": "A code fragment from 'CSVRowInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CSVRowOutputStream.cpp", + "Message": "A code fragment from 'CSVRowOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 56687, + "CodeNext": 4160558847, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "CollapsingFinalBlockInputStream.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: pos." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CollapsingFinalBlockInputStream.cpp", + "Message": "A code fragment from 'CollapsingFinalBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3189192997, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "CollapsingSortedBlockInputStream.cpp", + "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3474608216, + "CodeNext": 3529831664, + "CodePrev": 5816, + "ErrorCode": "V730", + "FileName": "CollapsingSortedBlockInputStream.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: current_key, next_key, first_negative, last_positive, last_negative." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CollapsingSortedBlockInputStream.cpp", + "Message": "A code fragment from 'CollapsingSortedBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 194852, + "CodeNext": 123, + "CodePrev": 2043716261, + "ErrorCode": "V730", + "FileName": "CapnProtoRowInputStream.h", + "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: type." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'CapnProtoRowInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 92287, + "CodeNext": 231331854, + "CodePrev": 360716783, + "ErrorCode": "V550", + "FileName": "ColumnGathererStream.cpp", + "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(seconds) > Epsilon." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ColumnGathererStream.cpp", + "Message": "A code fragment from 'ColumnGathererStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CountingBlockOutputStream.cpp", + "Message": "A code fragment from 'CountingBlockOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ConvertingBlockInputStream.cpp", + "Message": "A code fragment from 'ConvertingBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 578660774, + "CodeNext": 2441941198, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "HashSet.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: saved_hash." + }, + { + "CodeCurrent": 2441941198, + "CodeNext": 0, + "CodePrev": 578660774, + "ErrorCode": "V730", + "FileName": "HashSet.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: saved_hash." + }, + { + "CodeCurrent": 3398701039, + "CodeNext": 413159266, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "ClearableHashSet.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: version." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DistinctBlockInputStream.cpp", + "Message": "A code fragment from 'DistinctBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 233229516, + "CodeNext": 17813717, + "CodePrev": 123, + "ErrorCode": "V522", + "FileName": "CreatingSetsBlockInputStream.cpp", + "Message": "There might be dereferencing of a potential null pointer 'subquery.set'." + }, + { + "CodeCurrent": 14128946, + "CodeNext": 35621461, + "CodePrev": 123, + "ErrorCode": "V522", + "FileName": "CreatingSetsBlockInputStream.cpp", + "Message": "There might be dereferencing of a potential null pointer 'subquery.join'." + }, + { + "CodeCurrent": 207231, + "CodeNext": 150934975, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "BoolMask.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: can_be_true, can_be_false." + }, + { + "CodeCurrent": 61359, + "CodeNext": 1727213102, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "Join.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: block, row_num." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CreatingSetsBlockInputStream.cpp", + "Message": "A code fragment from 'CreatingSetsBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DistinctSortedBlockInputStream.cpp", + "Message": "A code fragment from 'DistinctSortedBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ExpressionBlockInputStream.cpp", + "Message": "A code fragment from 'ExpressionBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FilterBlockInputStream.cpp", + "Message": "A code fragment from 'FilterBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FilterColumnsBlockInputStream.cpp", + "Message": "A code fragment from 'FilterColumnsBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3416694064, + "CodeNext": 1249324771, + "CodePrev": 2609586663, + "ErrorCode": "V526", + "FileName": "OptimizedRegularExpression.inl.h", + "Message": "The 'strncmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." + }, + { + "CodeCurrent": 1249324771, + "CodeNext": 611713731, + "CodePrev": 3416694064, + "ErrorCode": "V526", + "FileName": "OptimizedRegularExpression.inl.h", + "Message": "The 'strncmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." + }, + { + "CodeCurrent": 611713731, + "CodeNext": 3436934615, + "CodePrev": 1249324771, + "ErrorCode": "V526", + "FileName": "OptimizedRegularExpression.inl.h", + "Message": "The 'strncmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." + }, + { + "CodeCurrent": 3436934615, + "CodeNext": 123, + "CodePrev": 611713731, + "ErrorCode": "V526", + "FileName": "OptimizedRegularExpression.inl.h", + "Message": "The 'strncmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." + }, + { + "CodeCurrent": 3165904209, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "GraphiteRollupSortedBlockInputStream.cpp", + "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "GraphiteRollupSortedBlockInputStream.cpp", + "Message": "A code fragment from 'GraphiteRollupSortedBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IBlockInputStream.cpp", + "Message": "A code fragment from 'IBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 505357, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "Quota.h", + "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: queries, errors, result_rows, result_bytes, read_rows, read_bytes, ..." + }, + { + "CodeCurrent": 1713353540, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "Quota.h", + "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: queries, errors, result_rows, result_bytes, read_rows, read_bytes, ..." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IProfilingBlockInputStream.cpp", + "Message": "A code fragment from 'IProfilingBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'FormatFactory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IRowInputStream.cpp", + "Message": "A code fragment from 'IRowInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IRowOutputStream.cpp", + "Message": "A code fragment from 'IRowOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "JSONCompactRowOutputStream.cpp", + "Message": "A code fragment from 'JSONCompactRowOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InputStreamFromASTInsertQuery.cpp", + "Message": "A code fragment from 'InputStreamFromASTInsertQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "JSONEachRowRowInputStream.cpp", + "Message": "A code fragment from 'JSONEachRowRowInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "JSONEachRowRowOutputStream.cpp", + "Message": "A code fragment from 'JSONEachRowRowOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "JSONRowOutputStream.cpp", + "Message": "A code fragment from 'JSONRowOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "LimitBlockInputStream.cpp", + "Message": "A code fragment from 'LimitBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "LimitByBlockInputStream.cpp", + "Message": "A code fragment from 'LimitByBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MaterializingBlockInputStream.cpp", + "Message": "A code fragment from 'MaterializingBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3161195889, + "CodeNext": 123, + "CodePrev": 1879495099, + "ErrorCode": "V688", + "FileName": "MergeSortingBlockInputStream.cpp", + "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeSortingBlockInputStream.cpp", + "Message": "A code fragment from 'MergeSortingBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergingAggregatedBlockInputStream.cpp", + "Message": "A code fragment from 'MergingAggregatedBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1919288073, + "CodeNext": 123, + "CodePrev": 1879495099, + "ErrorCode": "V688", + "FileName": "MergingSortedBlockInputStream.cpp", + "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3223507410, + "CodeNext": 123, + "CodePrev": 1879495099, + "ErrorCode": "V688", + "FileName": "MergingSortedBlockInputStream.cpp", + "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2080638586, + "CodeNext": 123, + "CodePrev": 1879495099, + "ErrorCode": "V688", + "FileName": "MergingSortedBlockInputStream.cpp", + "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergingSortedBlockInputStream.cpp", + "Message": "A code fragment from 'MergingSortedBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergingAggregatedMemoryEfficientBlockInputStream.cpp", + "Message": "A code fragment from 'MergingAggregatedMemoryEfficientBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "NativeBlockInputStream.cpp", + "Message": "A code fragment from 'NativeBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ODBCDriverBlockOutputStream.cpp", + "Message": "A code fragment from 'ODBCDriverBlockOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "NativeBlockOutputStream.cpp", + "Message": "A code fragment from 'NativeBlockOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 429503, + "CodeNext": 3001760339, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "ParallelInputsProcessor.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: i." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParallelAggregatingBlockInputStream.cpp", + "Message": "A code fragment from 'ParallelAggregatingBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "PartialSortingBlockInputStream.cpp", + "Message": "A code fragment from 'PartialSortingBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "PrettyBlockOutputStream.cpp", + "Message": "A code fragment from 'PrettyBlockOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "PrettyCompactBlockOutputStream.cpp", + "Message": "A code fragment from 'PrettyCompactBlockOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "PrettySpaceBlockOutputStream.cpp", + "Message": "A code fragment from 'PrettySpaceBlockOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "RemoteBlockOutputStream.cpp", + "Message": "A code fragment from 'RemoteBlockOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 12514670, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "PoolWithFailoverBase.h", + "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: pool, state, index." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "RemoteBlockInputStream.cpp", + "Message": "A code fragment from 'RemoteBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "SizeLimits.cpp", + "Message": "A code fragment from 'SizeLimits.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2251640247, + "CodeNext": 350966580, + "CodePrev": 576202780, + "ErrorCode": "V688", + "FileName": "PushingToViewsBlockOutputStream.cpp", + "Message": "The 'storage' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "PushingToViewsBlockOutputStream.cpp", + "Message": "A code fragment from 'PushingToViewsBlockOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3190533093, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ReplacingSortedBlockInputStream.cpp", + "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2535084036, + "CodeNext": 1504104628, + "CodePrev": 5816, + "ErrorCode": "V730", + "FileName": "ReplacingSortedBlockInputStream.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: current_key, next_key, selected_row." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReplacingSortedBlockInputStream.cpp", + "Message": "A code fragment from 'ReplacingSortedBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "SquashingBlockOutputStream.cpp", + "Message": "A code fragment from 'SquashingBlockOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "SquashingBlockInputStream.cpp", + "Message": "A code fragment from 'SquashingBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "SquashingTransform.cpp", + "Message": "A code fragment from 'SquashingTransform.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3967747240, + "CodeNext": 0, + "CodePrev": 893858483, + "ErrorCode": "V659", + "FileName": "ColumnTuple.h", + "Message": "Declarations of functions with 'getColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." + }, + { + "CodeCurrent": 3191513253, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "SummingSortedBlockInputStream.cpp", + "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "SummingSortedBlockInputStream.cpp", + "Message": "A code fragment from 'SummingSortedBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TSKVRowInputStream.cpp", + "Message": "A code fragment from 'TSKVRowInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TabSeparatedRowInputStream.cpp", + "Message": "A code fragment from 'TabSeparatedRowInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TSKVRowOutputStream.cpp", + "Message": "A code fragment from 'TSKVRowOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TabSeparatedRowOutputStream.cpp", + "Message": "A code fragment from 'TabSeparatedRowOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ValuesRowOutputStream.cpp", + "Message": "A code fragment from 'ValuesRowOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TotalsHavingBlockInputStream.cpp", + "Message": "A code fragment from 'TotalsHavingBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ValuesRowInputStream.cpp", + "Message": "A code fragment from 'ValuesRowInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "VerticalRowOutputStream.cpp", + "Message": "A code fragment from 'VerticalRowOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3998578981, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "VersionedCollapsingSortedBlockInputStream.cpp", + "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "VersionedCollapsingSortedBlockInputStream.cpp", + "Message": "A code fragment from 'VersionedCollapsingSortedBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "XMLRowOutputStream.cpp", + "Message": "A code fragment from 'XMLRowOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "copyData.cpp", + "Message": "A code fragment from 'copyData.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "materializeBlock.cpp", + "Message": "A code fragment from 'materializeBlock.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "verbosePrintString.cpp", + "Message": "A code fragment from 'verbosePrintString.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "narrowBlockInputStreams.cpp", + "Message": "A code fragment from 'narrowBlockInputStreams.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeAggregateFunction.cpp", + "Message": "A code fragment from 'DataTypeAggregateFunction.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeArray.cpp", + "Message": "A code fragment from 'DataTypeArray.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeDate.cpp", + "Message": "A code fragment from 'DataTypeDate.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeDateTime.cpp", + "Message": "A code fragment from 'DataTypeDateTime.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeFactory.cpp", + "Message": "A code fragment from 'DataTypeFactory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeFunction.cpp", + "Message": "A code fragment from 'DataTypeFunction.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 566420924, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "DataTypeEnum.h", + "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2818381, + "CodeNext": 690132628, + "CodePrev": 3160185338, + "ErrorCode": "V688", + "FileName": "DataTypeEnum.cpp", + "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2818381, + "CodeNext": 368631651, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "DataTypeEnum.cpp", + "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2818381, + "CodeNext": 2806683281, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "DataTypeEnum.cpp", + "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2818381, + "CodeNext": 710643812, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "DataTypeEnum.cpp", + "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeEnum.cpp", + "Message": "A code fragment from 'DataTypeEnum.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeInterval.cpp", + "Message": "A code fragment from 'DataTypeInterval.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeFixedString.cpp", + "Message": "A code fragment from 'DataTypeFixedString.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeNothing.cpp", + "Message": "A code fragment from 'DataTypeNothing.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeString.cpp", + "Message": "A code fragment from 'DataTypeString.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeNumberBase.cpp", + "Message": "A code fragment from 'DataTypeNumberBase.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeNullable.cpp", + "Message": "A code fragment from 'DataTypeNullable.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypesNumber.cpp", + "Message": "A code fragment from 'DataTypesNumber.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FieldToDataType.cpp", + "Message": "A code fragment from 'FieldToDataType.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeTuple.cpp", + "Message": "A code fragment from 'DataTypeTuple.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataTypeUUID.cpp", + "Message": "A code fragment from 'DataTypeUUID.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IDataType.cpp", + "Message": "A code fragment from 'IDataType.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "NestedUtils.cpp", + "Message": "A code fragment from 'NestedUtils.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "getLeastSupertype.cpp", + "Message": "A code fragment from 'getLeastSupertype.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "getMostSubtype.cpp", + "Message": "A code fragment from 'getMostSubtype.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DatabaseFactory.cpp", + "Message": "A code fragment from 'DatabaseFactory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DatabaseMemory.cpp", + "Message": "A code fragment from 'DatabaseMemory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2421950747, + "CodeNext": 281663836, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "DatabaseDictionary.cpp", + "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'DatabaseDictionary.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 4574138, + "CodeNext": 2250845352, + "CodePrev": 72410316, + "ErrorCode": "V688", + "FileName": "DatabaseOrdinary.cpp", + "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DatabaseOrdinary.cpp", + "Message": "A code fragment from 'DatabaseOrdinary.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DatabasesCommon.cpp", + "Message": "A code fragment from 'DatabasesCommon.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3735361261, + "CodeNext": 123, + "CodePrev": 1623468225, + "ErrorCode": "V688", + "FileName": "CatBoostModel.cpp", + "Message": "The 'cat_features_count' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2354497829, + "CodeNext": 2736089741, + "CodePrev": 2412400287, + "ErrorCode": "V688", + "FileName": "CatBoostModel.cpp", + "Message": "The 'float_features_count' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2354497829, + "CodeNext": 2736089741, + "CodePrev": 2412400287, + "ErrorCode": "V688", + "FileName": "CatBoostModel.cpp", + "Message": "The 'cat_features_count' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2575466056, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "CatBoostModel.cpp", + "Message": "The 'lib_path' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'CatBoostModel.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 207025177, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V690", + "FileName": "CacheDictionary.h", + "Message": "The 'CacheDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 518403501, + "CodeNext": 610328694, + "CodePrev": 1500196055, + "ErrorCode": "V688", + "FileName": "DictionaryBlockInputStream.h", + "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 411059078, + "CodeNext": 123, + "CodePrev": 610328694, + "ErrorCode": "V688", + "FileName": "DictionaryBlockInputStream.h", + "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 518630893, + "CodeNext": 610328694, + "CodePrev": 1500196055, + "ErrorCode": "V688", + "FileName": "DictionaryBlockInputStream.h", + "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 411059078, + "CodeNext": 123, + "CodePrev": 610328694, + "ErrorCode": "V688", + "FileName": "DictionaryBlockInputStream.h", + "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 411059078, + "CodeNext": 123, + "CodePrev": 1382482652, + "ErrorCode": "V688", + "FileName": "DictionaryBlockInputStream.h", + "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 411059078, + "CodeNext": 123, + "CodePrev": 1382482652, + "ErrorCode": "V688", + "FileName": "DictionaryBlockInputStream.h", + "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3731206478, + "CodeNext": 123, + "CodePrev": 11900062, + "ErrorCode": "V688", + "FileName": "DictionaryBlockInputStream.h", + "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3673035245, + "CodeNext": 1382482652, + "CodePrev": 3006984332, + "ErrorCode": "V688", + "FileName": "DictionaryBlockInputStream.h", + "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2414448007, + "CodeNext": 123, + "CodePrev": 1382482652, + "ErrorCode": "V688", + "FileName": "DictionaryBlockInputStream.h", + "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3673035245, + "CodeNext": 1382482652, + "CodePrev": 3467903158, + "ErrorCode": "V688", + "FileName": "DictionaryBlockInputStream.h", + "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2414448007, + "CodeNext": 123, + "CodePrev": 1382482652, + "ErrorCode": "V688", + "FileName": "DictionaryBlockInputStream.h", + "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 921248968, + "CodeNext": 123, + "CodePrev": 1342750387, + "ErrorCode": "V688", + "FileName": "DictionaryBlockInputStream.h", + "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3690553619, + "CodeNext": 37308961, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "CacheDictionary.cpp", + "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 173283489, + "CodeNext": 1039775048, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "CacheDictionary.cpp", + "Message": "The 'size' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 549768760, + "CodeNext": 1152076000, + "CodePrev": 1919584191, + "ErrorCode": "V688", + "FileName": "CacheDictionary.cpp", + "Message": "The 'size' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 579497379, + "CodeNext": 93115, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "CacheDictionary.cpp", + "Message": "The 'size' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'CacheDictionary.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1825423383, + "CodeNext": 123, + "CodePrev": 123, + "ErrorCode": "V690", + "FileName": "ClickHouseDictionarySource.h", + "Message": "The 'ClickHouseDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ClickHouseDictionarySource.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2185656967, + "CodeNext": 123, + "CodePrev": 123, + "ErrorCode": "V690", + "FileName": "ComplexKeyCacheDictionary.h", + "Message": "The 'ComplexKeyCacheDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 3504260784, + "CodeNext": 37308961, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ComplexKeyCacheDictionary.cpp", + "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2142244675, + "CodeNext": 2508128848, + "CodePrev": 41371507, + "ErrorCode": "V688", + "FileName": "ComplexKeyCacheDictionary.cpp", + "Message": "The 'key_size' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ComplexKeyCacheDictionary.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ComplexKeyCacheDictionary_createAttributeWithType.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ComplexKeyCacheDictionary_generate_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ComplexKeyCacheDictionary_generate_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ComplexKeyCacheDictionary_generate_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 579497379, + "CodeNext": 93115, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ComplexKeyCacheDictionary_setAttributeValue.cpp", + "Message": "The 'size' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ComplexKeyCacheDictionary_setAttributeValue.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1173598882, + "CodeNext": 78734017, + "CodePrev": 38981872, + "ErrorCode": "V688", + "FileName": "DictionaryBlockInputStreamBase.cpp", + "Message": "The 'block' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DictionaryBlockInputStreamBase.cpp", + "Message": "A code fragment from 'DictionaryBlockInputStreamBase.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1259294311, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V690", + "FileName": "ComplexKeyHashedDictionary.h", + "Message": "The 'ComplexKeyHashedDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ComplexKeyHashedDictionary.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'DictionarySourceHelpers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2179468235, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "DictionaryStructure.cpp", + "Message": "The 'attributes' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'DictionaryStructure.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1562050581, + "CodeNext": 123, + "CodePrev": 2247801432, + "ErrorCode": "V690", + "FileName": "FileDictionarySource.h", + "Message": "The 'FileDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 1863286807, + "CodeNext": 123, + "CodePrev": 3087958036, + "ErrorCode": "V690", + "FileName": "ExecutableDictionarySource.h", + "Message": "The 'ExecutableDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 1560485909, + "CodeNext": 123, + "CodePrev": 3228890078, + "ErrorCode": "V690", + "FileName": "HTTPDictionarySource.h", + "Message": "The 'HTTPDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 314376213, + "CodeNext": 123, + "CodePrev": 4033243237, + "ErrorCode": "V690", + "FileName": "LibraryDictionarySource.h", + "Message": "The 'LibraryDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 313384981, + "CodeNext": 123, + "CodePrev": 247020956, + "ErrorCode": "V690", + "FileName": "MongoDBDictionarySource.h", + "Message": "The 'MongoDBDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 1561051157, + "CodeNext": 123, + "CodePrev": 1880965760, + "ErrorCode": "V690", + "FileName": "ODBCDictionarySource.h", + "Message": "The 'ODBCDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 1143955477, + "CodeNext": 123, + "CodePrev": 2162917980, + "ErrorCode": "V690", + "FileName": "MySQLDictionarySource.h", + "Message": "The 'MySQLDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'DictionarySourceFactory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3300655365, + "CodeNext": 2232737895, + "CodePrev": 3529503794, + "ErrorCode": "V669", + "FileName": "VarInt.h", + "Message": "The 'ostr' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 2232737895, + "CodeNext": 3413908603, + "CodePrev": 3300655365, + "ErrorCode": "V669", + "FileName": "VarInt.h", + "Message": "The 'ostr' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 2696744022, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V669", + "FileName": "SettingsCommon.h", + "Message": "The 'buf' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 2696744022, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V669", + "FileName": "SettingsCommon.h", + "Message": "The 'buf' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 2696744022, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V669", + "FileName": "SettingsCommon.h", + "Message": "The 'buf' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 3674152177, + "CodeNext": 516173517, + "CodePrev": 5816, + "ErrorCode": "V688", + "FileName": "OwningBlockInputStream.h", + "Message": "The 'stream' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ExecutableDictionarySource.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ExternalResultDescription.cpp", + "Message": "A code fragment from 'ExternalResultDescription.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3266193319, + "CodeNext": 193, + "CodePrev": 123, + "ErrorCode": "V516", + "FileName": "IColumn.h", + "Message": "Consider inspecting an odd expression. Non-null function pointer is compared to null." + }, + { + "CodeCurrent": 3266193319, + "CodeNext": 193, + "CodePrev": 123, + "ErrorCode": "V516", + "FileName": "IColumn.h", + "Message": "Consider inspecting an odd expression. Non-null function pointer is compared to null: 'IsMutableColumns < Args >::value'." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ExternalQueryBuilder.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FileDictionarySource.cpp", + "Message": "A code fragment from 'FileDictionarySource.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 207054359, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V690", + "FileName": "FlatDictionary.h", + "Message": "The 'FlatDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'FlatDictionary.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'HTTPDictionarySource.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 206951383, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V690", + "FileName": "HashedDictionary.h", + "Message": "The 'HashedDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'HashedDictionary.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 605484233, + "CodeNext": 26586574, + "CodePrev": 3065615242, + "ErrorCode": "V688", + "FileName": "LibraryDictionarySource.cpp", + "Message": "The 'config_prefix' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 26586574, + "CodeNext": 72410313, + "CodePrev": 605484233, + "ErrorCode": "V688", + "FileName": "LibraryDictionarySource.cpp", + "Message": "The 'sample_block' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'LibraryDictionarySource.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'MongoDBBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'MongoDBDictionarySource.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MySQLBlockInputStream.cpp", + "Message": "A code fragment from 'MySQLBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3362020784, + "CodeNext": 0, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "MySQLDictionarySource.cpp", + "Message": "The 'update_time' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 13569497, + "CodeNext": 985361697, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "MySQLDictionarySource.cpp", + "Message": "The 'sample_block' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'MySQLDictionarySource.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ODBCBlockInputStream.cpp", + "Message": "A code fragment from 'ODBCBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3238522471, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V690", + "FileName": "RangeHashedDictionary.h", + "Message": "The 'RangeHashedDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 3406274085, + "CodeNext": 4182595664, + "CodePrev": 3042861068, + "ErrorCode": "V688", + "FileName": "RangeDictionaryBlockInputStream.h", + "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 4182595664, + "CodeNext": 123, + "CodePrev": 3406274085, + "ErrorCode": "V688", + "FileName": "RangeDictionaryBlockInputStream.h", + "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 456188231, + "CodeNext": 2414448007, + "CodePrev": 1469584987, + "ErrorCode": "V688", + "FileName": "RangeDictionaryBlockInputStream.h", + "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2414448007, + "CodeNext": 123, + "CodePrev": 456188231, + "ErrorCode": "V688", + "FileName": "RangeDictionaryBlockInputStream.h", + "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2178677200, + "CodeNext": 2078138180, + "CodePrev": 2641309169, + "ErrorCode": "V688", + "FileName": "RangeDictionaryBlockInputStream.h", + "Message": "The 'column_names' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2418126002, + "CodeNext": 123, + "CodePrev": 2901253088, + "ErrorCode": "V688", + "FileName": "RangeDictionaryBlockInputStream.h", + "Message": "The 'start_dates' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2418126002, + "CodeNext": 123, + "CodePrev": 2901253088, + "ErrorCode": "V688", + "FileName": "RangeDictionaryBlockInputStream.h", + "Message": "The 'end_dates' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 1525601398, + "CodeNext": 2418126002, + "CodePrev": 682989491, + "ErrorCode": "V688", + "FileName": "RangeDictionaryBlockInputStream.h", + "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2418126002, + "CodeNext": 123, + "CodePrev": 1525601398, + "ErrorCode": "V688", + "FileName": "RangeDictionaryBlockInputStream.h", + "Message": "The 'start_dates' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2418126002, + "CodeNext": 123, + "CodePrev": 1525601398, + "ErrorCode": "V688", + "FileName": "RangeDictionaryBlockInputStream.h", + "Message": "The 'end_dates' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 127934457, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "RangeHashedDictionary.cpp", + "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'RangeHashedDictionary.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ODBCDictionarySource.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "GeoDictionariesLoader.cpp", + "Message": "A code fragment from 'GeoDictionariesLoader.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "readInvalidateQuery.cpp", + "Message": "A code fragment from 'readInvalidateQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "writeParenthesisedString.cpp", + "Message": "A code fragment from 'writeParenthesisedString.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3312877383, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V690", + "FileName": "TrieDictionary.h", + "Message": "The 'TrieDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'TrieDictionary.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "RegionsHierarchies.cpp", + "Message": "A code fragment from 'RegionsHierarchies.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "RegionsHierarchy.cpp", + "Message": "A code fragment from 'RegionsHierarchy.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "RegionsNames.cpp", + "Message": "A code fragment from 'RegionsNames.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TechDataHierarchy.cpp", + "Message": "A code fragment from 'TechDataHierarchy.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "HierarchiesProvider.cpp", + "Message": "A code fragment from 'HierarchiesProvider.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "NamesProvider.cpp", + "Message": "A code fragment from 'NamesProvider.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "HierarchyFormatReader.cpp", + "Message": "A code fragment from 'HierarchyFormatReader.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "NamesFormatReader.cpp", + "Message": "A code fragment from 'NamesFormatReader.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ActionLocksManager.cpp", + "Message": "A code fragment from 'ActionLocksManager.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ClientInfo.cpp", + "Message": "A code fragment from 'ClientInfo.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Compiler.cpp", + "Message": "A code fragment from 'Compiler.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 871352, + "CodeNext": 223912592, + "CodePrev": 766487020, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 73089598, + "CodeNext": 4084406, + "CodePrev": 2724040669, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 871352, + "CodeNext": 3873811996, + "CodePrev": 3673856596, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 73089598, + "CodeNext": 4084406, + "CodePrev": 2724040669, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 871352, + "CodeNext": 36816, + "CodePrev": 3126615886, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 871352, + "CodeNext": 511803757, + "CodePrev": 3126615886, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 871352, + "CodeNext": 122870, + "CodePrev": 1474597293, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 73089598, + "CodeNext": 1179330, + "CodePrev": 2685314853, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 871352, + "CodeNext": 122870, + "CodePrev": 4240594574, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2338865922, + "CodeNext": 123, + "CodePrev": 2685314853, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 871352, + "CodeNext": 122870, + "CodePrev": 3859987971, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2338865922, + "CodeNext": 123, + "CodePrev": 4042434526, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 871352, + "CodeNext": 36816, + "CodePrev": 3126615886, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 73089598, + "CodeNext": 223912592, + "CodePrev": 207246, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 871352, + "CodeNext": 122870, + "CodePrev": 223912592, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 73089598, + "CodeNext": 223912592, + "CodePrev": 207246, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 871352, + "CodeNext": 122870, + "CodePrev": 223912592, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 871352, + "CodeNext": 109104, + "CodePrev": 3874875103, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 73089598, + "CodeNext": 4084406, + "CodePrev": 3397567370, + "ErrorCode": "V688", + "FileName": "Aggregator.cpp", + "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Aggregator.cpp", + "Message": "A code fragment from 'Aggregator.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 304867129, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "Cluster.cpp", + "Message": "The 'host_name' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 304867129, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "Cluster.cpp", + "Message": "The 'port' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3548849765, + "CodeNext": 2541401097, + "CodePrev": 5816, + "ErrorCode": "V730", + "FileName": "HexWriteBuffer.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: buf." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Cluster.cpp", + "Message": "A code fragment from 'Cluster.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AsynchronousMetrics.cpp", + "Message": "A code fragment from 'AsynchronousMetrics.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 123, + "CodeNext": 129791, + "CodePrev": 35837, + "ErrorCode": "V565", + "FileName": "DNSCacheUpdater.cpp", + "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." + }, + { + "CodeCurrent": 1922414329, + "CodeNext": 123, + "CodePrev": 125, + "ErrorCode": "V759", + "FileName": "DNSCacheUpdater.cpp", + "Message": "Incorrect order of exception handling blocks. Exception of 'DNSException' type will be caught in a catch block for 'Exception' base type." + }, + { + "CodeCurrent": 958195402, + "CodeNext": 123, + "CodePrev": 125, + "ErrorCode": "V759", + "FileName": "DNSCacheUpdater.cpp", + "Message": "Incorrect order of exception handling blocks. Exception of 'TimeoutException' type will be caught in a catch block for 'Exception' base type." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DNSCacheUpdater.cpp", + "Message": "A code fragment from 'DNSCacheUpdater.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 390633, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "DDLWorker.cpp", + "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: host_shard_num, host_replica_num." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DDLWorker.cpp", + "Message": "A code fragment from 'DDLWorker.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'DictionaryFactory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3538236801, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V524", + "FileName": "RuntimeComponentsFactory.h", + "Message": "It is odd that the body of 'createExternalModelsConfigRepository' function is fully equivalent to the body of 'createExternalDictionariesConfigRepository' function." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'Context.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "EmbeddedDictionaries.cpp", + "Message": "A code fragment from 'EmbeddedDictionaries.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1141335469, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ExpressionActions.cpp", + "Message": "The 'sample_block' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ExpressionActions.cpp", + "Message": "A code fragment from 'ExpressionActions.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ExternalDictionaries.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3392699298, + "CodeNext": 0, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "ExternalLoader.cpp", + "Message": "The 'config' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ExternalLoader.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ExternalLoaderConfigRepository.cpp", + "Message": "A code fragment from 'ExternalLoaderConfigRepository.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3405548629, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2136590205, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'aliases' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 1478353069, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3783766598, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3829470883, + "CodeNext": 0, + "CodePrev": 123, + "ErrorCode": "V522", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "There might be dereferencing of a potential null pointer 'subquery'." + }, + { + "CodeCurrent": 3237744880, + "CodeNext": 123, + "CodePrev": 3248671758, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 1459789653, + "CodeNext": 123, + "CodePrev": 4129363794, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3479198451, + "CodeNext": 4055667619, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'aliases' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 4216604761, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 427259459, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3061641111, + "CodeNext": 123, + "CodePrev": 137989537, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 1240966709, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3626124927, + "CodeNext": 2885543787, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2661019153, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 1362320973, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3178398591, + "CodeNext": 1853405735, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 238898558, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3409673381, + "CodeNext": 72789921, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ExpressionAnalyzer.cpp", + "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ExpressionAnalyzer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'ExternalModels.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InJoinSubqueriesPreprocessor.cpp", + "Message": "A code fragment from 'InJoinSubqueriesPreprocessor.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 200100772, + "CodeNext": 123, + "CodePrev": 11496, + "ErrorCode": "V730", + "FileName": "InterpreterAlterQuery.h", + "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: type." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterAlterQuery.cpp", + "Message": "A code fragment from 'InterpreterAlterQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterCheckQuery.cpp", + "Message": "A code fragment from 'InterpreterCheckQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterDescribeQuery.cpp", + "Message": "A code fragment from 'InterpreterDescribeQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3494840295, + "CodeNext": 193, + "CodePrev": 0, + "ErrorCode": "V524", + "FileName": "StorageLog.h", + "Message": "It is odd that the body of 'getFullPath' function is fully equivalent to the body of 'full_path' function." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterCreateQuery.cpp", + "Message": "A code fragment from 'InterpreterCreateQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterExistsQuery.cpp", + "Message": "A code fragment from 'InterpreterExistsQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2270843740, + "CodeNext": 125, + "CodePrev": 4111337664, + "ErrorCode": "V522", + "FileName": "InterpreterDropQuery.cpp", + "Message": "There might be dereferencing of a potential null pointer 'database'." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterDropQuery.cpp", + "Message": "A code fragment from 'InterpreterDropQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterFactory.cpp", + "Message": "A code fragment from 'InterpreterFactory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterKillQueryQuery.cpp", + "Message": "A code fragment from 'InterpreterKillQueryQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterInsertQuery.cpp", + "Message": "A code fragment from 'InterpreterInsertQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterOptimizeQuery.cpp", + "Message": "A code fragment from 'InterpreterOptimizeQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterRenameQuery.cpp", + "Message": "A code fragment from 'InterpreterRenameQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3481785836, + "CodeNext": 0, + "CodePrev": 263098193, + "ErrorCode": "V678", + "FileName": "InterpreterSelectWithUnionQuery.cpp", + "Message": "An object is used as an argument to its own method. Consider checking the first actual argument of the 'setQueryContext' function." + }, + { + "CodeCurrent": 146241864, + "CodeNext": 72410313, + "CodePrev": 607166188, + "ErrorCode": "V688", + "FileName": "InterpreterSelectWithUnionQuery.cpp", + "Message": "The 'query_ptr' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 72410313, + "CodeNext": 123, + "CodePrev": 146241864, + "ErrorCode": "V688", + "FileName": "InterpreterSelectWithUnionQuery.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterSelectWithUnionQuery.cpp", + "Message": "A code fragment from 'InterpreterSelectWithUnionQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterSetQuery.cpp", + "Message": "A code fragment from 'InterpreterSetQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterShowTablesQuery.cpp", + "Message": "A code fragment from 'InterpreterShowTablesQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3481785836, + "CodeNext": 0, + "CodePrev": 263098193, + "ErrorCode": "V678", + "FileName": "InterpreterSelectQuery.cpp", + "Message": "An object is used as an argument to its own method. Consider checking the first actual argument of the 'setQueryContext' function." + }, + { + "CodeCurrent": 1460847663, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "InterpreterSelectQuery.cpp", + "Message": "The 'input' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2449370409, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V524", + "FileName": "InterpreterSelectQuery.cpp", + "Message": "It is odd that the body of 'executeProjection' function is fully equivalent to the body of 'executeExpression' function." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterSelectQuery.cpp", + "Message": "A code fragment from 'InterpreterSelectQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterShowProcesslistQuery.cpp", + "Message": "A code fragment from 'InterpreterShowProcesslistQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1627430578, + "CodeNext": 520103033, + "CodePrev": 721433261, + "ErrorCode": "V522", + "FileName": "InterpreterShowCreateQuery.cpp", + "Message": "There might be dereferencing of a potential null pointer 'create_query'." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterShowCreateQuery.cpp", + "Message": "A code fragment from 'InterpreterShowCreateQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterpreterUseQuery.cpp", + "Message": "A code fragment from 'InterpreterUseQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "LogicalExpressionsOptimizer.cpp", + "Message": "A code fragment from 'LogicalExpressionsOptimizer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1421843502, + "CodeNext": 2932320854, + "CodePrev": 3193279124, + "ErrorCode": "V688", + "FileName": "Join.cpp", + "Message": "The 'columns_left' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Join.cpp", + "Message": "A code fragment from 'Join.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "NullableUtils.cpp", + "Message": "A code fragment from 'NullableUtils.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2628352566, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V659", + "FileName": "MergeTreeData.h", + "Message": "Declarations of functions with 'modifyPartState' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." + }, + { + "CodeCurrent": 3583851774, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "InterpreterSystemQuery.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3000727018, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "InterpreterSystemQuery.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'InterpreterSystemQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3170877678, + "CodeNext": 123, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "ProcessList.cpp", + "Message": "The 'it' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ProcessList.cpp", + "Message": "A code fragment from 'ProcessList.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ProjectionManipulation.cpp", + "Message": "A code fragment from 'ProjectionManipulation.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "PartLog.cpp", + "Message": "A code fragment from 'PartLog.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "QueryLog.cpp", + "Message": "A code fragment from 'QueryLog.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Quota.cpp", + "Message": "A code fragment from 'Quota.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "SecurityManager.cpp", + "Message": "A code fragment from 'SecurityManager.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Set.cpp", + "Message": "A code fragment from 'Set.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "SetVariants.cpp", + "Message": "A code fragment from 'SetVariants.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "SystemLog.cpp", + "Message": "A code fragment from 'SystemLog.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Settings.cpp", + "Message": "A code fragment from 'Settings.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TablesStatus.cpp", + "Message": "A code fragment from 'TablesStatus.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Users.cpp", + "Message": "A code fragment from 'Users.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "createBlockSelector.cpp", + "Message": "A code fragment from 'createBlockSelector.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "castColumn.cpp", + "Message": "A code fragment from 'castColumn.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "convertFieldToType.cpp", + "Message": "A code fragment from 'convertFieldToType.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "evaluateConstantExpression.cpp", + "Message": "A code fragment from 'evaluateConstantExpression.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "getClusterName.cpp", + "Message": "A code fragment from 'getClusterName.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "evaluateMissingDefaults.cpp", + "Message": "A code fragment from 'evaluateMissingDefaults.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 143231, + "CodeNext": 125, + "CodePrev": 125, + "ErrorCode": "V565", + "FileName": "executeQuery.cpp", + "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." + }, + { + "CodeCurrent": 3481785836, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V678", + "FileName": "executeQuery.cpp", + "Message": "An object is used as an argument to its own method. Consider checking the first actual argument of the 'setQueryContext' function." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "executeQuery.cpp", + "Message": "A code fragment from 'executeQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "sortBlock.cpp", + "Message": "A code fragment from 'sortBlock.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "loadMetadata.cpp", + "Message": "A code fragment from 'loadMetadata.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DescribeStreamFactory.cpp", + "Message": "A code fragment from 'DescribeStreamFactory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Collator.cpp", + "Message": "A code fragment from 'Collator.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "executeQuery.cpp", + "Message": "A code fragment from 'executeQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 39392417, + "CodeNext": 1261068192, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ColumnAggregateFunction.cpp", + "Message": "The 'data' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 699364582, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ColumnAggregateFunction.cpp", + "Message": "The 'src' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 4081716836, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ColumnAggregateFunction.cpp", + "Message": "The 'src' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ColumnAggregateFunction.cpp", + "Message": "A code fragment from 'ColumnAggregateFunction.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "SelectStreamFactory.cpp", + "Message": "A code fragment from 'SelectStreamFactory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2533242737, + "CodeNext": 3575089120, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "ColumnArray.cpp", + "Message": "The 'offsets' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ColumnArray.cpp", + "Message": "A code fragment from 'ColumnArray.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ColumnConst.cpp", + "Message": "A code fragment from 'ColumnConst.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ColumnFixedString.cpp", + "Message": "A code fragment from 'ColumnFixedString.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3418757250, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ColumnNullable.cpp", + "Message": "The 'null_map' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ColumnNullable.cpp", + "Message": "A code fragment from 'ColumnNullable.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ColumnFunction.cpp", + "Message": "A code fragment from 'ColumnFunction.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ColumnString.cpp", + "Message": "A code fragment from 'ColumnString.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FilterDescription.cpp", + "Message": "A code fragment from 'FilterDescription.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3124110550, + "CodeNext": 1303704304, + "CodePrev": 6015, + "ErrorCode": "V792", + "FileName": "ColumnsCommon.cpp", + "Message": "The '_mm_movemask_ps' function located to the right of the operator '|' will be called regardless of the value of the left operand. Perhaps, it is better to use '||'." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ColumnsCommon.cpp", + "Message": "A code fragment from 'ColumnsCommon.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ColumnTuple.cpp", + "Message": "A code fragment from 'ColumnTuple.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 951091257, + "CodeNext": 0, + "CodePrev": 951484473, + "ErrorCode": "V656", + "FileName": "ColumnVector.cpp", + "Message": "Variables 'cur_min', 'cur_max' are initialized through the call to the same function. It's probably an error or un-optimized code. Consider inspecting the 'NaNOrZero < T > ()' expression. Check lines: _, _." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ColumnVector.cpp", + "Message": "A code fragment from 'ColumnVector.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ColumnDefault.cpp", + "Message": "A code fragment from 'ColumnDefault.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IColumn.cpp", + "Message": "A code fragment from 'IColumn.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AlterCommands.cpp", + "Message": "A code fragment from 'AlterCommands.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IStorage.cpp", + "Message": "A code fragment from 'IStorage.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ColumnsDescription.cpp", + "Message": "A code fragment from 'ColumnsDescription.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ITableDeclaration.cpp", + "Message": "A code fragment from 'ITableDeclaration.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MutationCommands.cpp", + "Message": "A code fragment from 'MutationCommands.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2317070405, + "CodeNext": 813477413, + "CodePrev": 608950573, + "ErrorCode": "V688", + "FileName": "StorageCatBoostPool.cpp", + "Message": "The 'sample_block' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageCatBoostPool.cpp", + "Message": "A code fragment from 'StorageCatBoostPool.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 72410316, + "CodeNext": 2690972308, + "CodePrev": 3645137672, + "ErrorCode": "V688", + "FileName": "StorageBuffer.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2232984829, + "CodeNext": 345608494, + "CodePrev": 0, + "ErrorCode": "V547", + "FileName": "StorageBuffer.cpp", + "Message": "Expression '!buffer.first_write_time' is always true." + }, + { + "CodeCurrent": 2935677976, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageBuffer.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageBuffer.cpp", + "Message": "A code fragment from 'StorageBuffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'StorageDictionary.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageFile.cpp", + "Message": "A code fragment from 'StorageFile.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 72410316, + "CodeNext": 2690972308, + "CodePrev": 3645137672, + "ErrorCode": "V688", + "FileName": "StorageDistributed.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3889328658, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageDistributed.cpp", + "Message": "The 'table_name' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3889328658, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageDistributed.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2637534258, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageDistributed.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageDistributed.cpp", + "Message": "A code fragment from 'StorageDistributed.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageJoin.cpp", + "Message": "A code fragment from 'StorageJoin.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageFactory.cpp", + "Message": "A code fragment from 'StorageFactory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3969971195, + "CodeNext": 0, + "CodePrev": 2997480514, + "ErrorCode": "V656", + "FileName": "StorageMemory.cpp", + "Message": "Variables 'begin', 'end' are initialized through the call to the same function. It's probably an error or un-optimized code. Consider inspecting the 'data.begin()' expression. Check lines: _, _." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageMemory.cpp", + "Message": "A code fragment from 'StorageMemory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 581555750, + "CodeNext": 2488995567, + "CodePrev": 2696082656, + "ErrorCode": "V688", + "FileName": "StorageLog.cpp", + "Message": "The 'column_names' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageLog.cpp", + "Message": "A code fragment from 'StorageLog.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 11714530, + "CodeNext": 0, + "CodePrev": 2931558369, + "ErrorCode": "V522", + "FileName": "StorageMaterializedView.cpp", + "Message": "There might be dereferencing of a potential null pointer 'query.storage'." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageMaterializedView.cpp", + "Message": "A code fragment from 'StorageMaterializedView.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 72410316, + "CodeNext": 2690972308, + "CodePrev": 2488995567, + "ErrorCode": "V688", + "FileName": "StorageKafka.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'StorageKafka.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 72410316, + "CodeNext": 2690972308, + "CodePrev": 3645137672, + "ErrorCode": "V688", + "FileName": "StorageMySQL.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageMySQL.cpp", + "Message": "A code fragment from 'StorageMySQL.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 72410316, + "CodeNext": 2690972308, + "CodePrev": 3645137672, + "ErrorCode": "V688", + "FileName": "StorageMerge.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3595788056, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageMerge.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageMerge.cpp", + "Message": "A code fragment from 'StorageMerge.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3926377496, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageNull.cpp", + "Message": "The 'table_name' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageNull.cpp", + "Message": "A code fragment from 'StorageNull.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 72410316, + "CodeNext": 2690972308, + "CodePrev": 3645137672, + "ErrorCode": "V688", + "FileName": "StorageMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2341879450, + "CodeNext": 292720666, + "CodePrev": 2322283894, + "ErrorCode": "V688", + "FileName": "StorageMergeTree.cpp", + "Message": "The 'database_name' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 292720666, + "CodeNext": 72410313, + "CodePrev": 2341879450, + "ErrorCode": "V688", + "FileName": "StorageMergeTree.cpp", + "Message": "The 'table_name' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 72410313, + "CodeNext": 123, + "CodePrev": 292720666, + "ErrorCode": "V688", + "FileName": "StorageMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2033765224, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3317773483, + "CodeNext": 123, + "CodePrev": 329476003, + "ErrorCode": "V688", + "FileName": "StorageMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2082196268, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 694167244, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 391214422, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3490868120, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageMergeTree.cpp", + "Message": "A code fragment from 'StorageMergeTree.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageODBC.cpp", + "Message": "A code fragment from 'StorageODBC.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSet.cpp", + "Message": "A code fragment from 'StorageSet.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageStripeLog.cpp", + "Message": "A code fragment from 'StorageStripeLog.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "VirtualColumnFactory.cpp", + "Message": "A code fragment from 'VirtualColumnFactory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageTinyLog.cpp", + "Message": "A code fragment from 'StorageTinyLog.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 630197398, + "CodeNext": 1056331065, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "StorageReplicatedMergeTree.cpp", + "Message": "The 'replica_path' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 438297239, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageReplicatedMergeTree.cpp", + "Message": "The 'replica_path' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 72410316, + "CodeNext": 2690972308, + "CodePrev": 3645137672, + "ErrorCode": "V688", + "FileName": "StorageReplicatedMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 1038581435, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageReplicatedMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 4265130885, + "CodeNext": 123, + "CodePrev": 2362549484, + "ErrorCode": "V688", + "FileName": "StorageReplicatedMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 423923019, + "CodeNext": 123, + "CodePrev": 3425255468, + "ErrorCode": "V688", + "FileName": "StorageReplicatedMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 1795911157, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageReplicatedMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 1257284900, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageReplicatedMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 29091704, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageReplicatedMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2265033744, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "StorageReplicatedMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 72410313, + "CodeNext": 123, + "CodePrev": 2533115030, + "ErrorCode": "V688", + "FileName": "StorageReplicatedMergeTree.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 1394960724, + "CodeNext": 2903347041, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "ReplicatedMergeTreeQuorumEntry.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: required_number_of_replicas." + }, + { + "CodeCurrent": 2903347041, + "CodeNext": 123, + "CodePrev": 1394960724, + "ErrorCode": "V730", + "FileName": "ReplicatedMergeTreeQuorumEntry.h", + "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: required_number_of_replicas." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageReplicatedMergeTree.cpp", + "Message": "A code fragment from 'StorageReplicatedMergeTree.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageView.cpp", + "Message": "A code fragment from 'StorageView.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerStorages.cpp", + "Message": "A code fragment from 'registerStorages.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "VirtualColumnUtils.cpp", + "Message": "A code fragment from 'VirtualColumnUtils.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "getStructureOfRemoteTable.cpp", + "Message": "A code fragment from 'getStructureOfRemoteTable.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "transformQueryForExternalDatabase.cpp", + "Message": "A code fragment from 'transformQueryForExternalDatabase.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ActiveDataPartSet.cpp", + "Message": "A code fragment from 'ActiveDataPartSet.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AllMergeSelector.cpp", + "Message": "A code fragment from 'AllMergeSelector.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3361369910, + "CodeNext": 123, + "CodePrev": 588179213, + "ErrorCode": "V658", + "FileName": "AbandonableLockInZooKeeper.cpp", + "Message": "A value is being subtracted from the unsigned variable. This can result in an overflow. In such a case, the '<' comparison operation can potentially behave unexpectedly. Consider inspecting the 'i < partitions.size() - holders.size()' expression." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AbandonableLockInZooKeeper.cpp", + "Message": "A code fragment from 'AbandonableLockInZooKeeper.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2664754306, + "CodeNext": 0, + "CodePrev": 1296085, + "ErrorCode": "V688", + "FileName": "DistributedBlockOutputStream.cpp", + "Message": "The 'query_string' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DistributedBlockOutputStream.cpp", + "Message": "A code fragment from 'DistributedBlockOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1912588196, + "CodeNext": 979922832, + "CodePrev": 1045335318, + "ErrorCode": "V688", + "FileName": "DirectoryMonitor.cpp", + "Message": "The 'path' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'DirectoryMonitor.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "BackgroundProcessingPool.cpp", + "Message": "A code fragment from 'BackgroundProcessingPool.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DiskSpaceMonitor.cpp", + "Message": "A code fragment from 'DiskSpaceMonitor.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "LevelMergeSelector.cpp", + "Message": "A code fragment from 'LevelMergeSelector.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DataPartsExchange.cpp", + "Message": "A code fragment from 'DataPartsExchange.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3216131445, + "CodeNext": 125, + "CodePrev": 123, + "ErrorCode": "V601", + "FileName": "KeyCondition.cpp", + "Message": "The 'true' value is implicitly cast to the integer type. Inspect the first argument." + }, + { + "CodeCurrent": 2137295876, + "CodeNext": 125, + "CodePrev": 123, + "ErrorCode": "V601", + "FileName": "KeyCondition.cpp", + "Message": "The 'false' value is implicitly cast to the integer type. Inspect the first argument." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "KeyCondition.cpp", + "Message": "A code fragment from 'KeyCondition.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeList.cpp", + "Message": "A code fragment from 'MergeList.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1098322110, + "CodeNext": 279045268, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "MergeTreeBaseBlockInputStream.cpp", + "Message": "The 'max_block_size_rows' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 279045268, + "CodeNext": 2329935579, + "CodePrev": 1098322110, + "ErrorCode": "V688", + "FileName": "MergeTreeBaseBlockInputStream.cpp", + "Message": "The 'preferred_block_size_bytes' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2329935579, + "CodeNext": 3894021058, + "CodePrev": 279045268, + "ErrorCode": "V688", + "FileName": "MergeTreeBaseBlockInputStream.cpp", + "Message": "The 'preferred_max_column_in_block_size_bytes' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeBaseBlockInputStream.cpp", + "Message": "A code fragment from 'MergeTreeBaseBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeBlockInputStream.cpp", + "Message": "A code fragment from 'MergeTreeBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeBlockOutputStream.cpp", + "Message": "A code fragment from 'MergeTreeBlockOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeBlockReadUtils.cpp", + "Message": "A code fragment from 'MergeTreeBlockReadUtils.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1550988213, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "MergeTreeDataPartChecksum.cpp", + "Message": "The 'full_checksums' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3214271742, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "MergeTreeDataPartChecksum.cpp", + "Message": "The 'full_checksums' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeDataPartChecksum.cpp", + "Message": "A code fragment from 'MergeTreeDataPartChecksum.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 123, + "CodeNext": 3380448882, + "CodePrev": 1441276206, + "ErrorCode": "V565", + "FileName": "MergeTreeData.cpp", + "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." + }, + { + "CodeCurrent": 2654040905, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "MergeTreeData.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2240934897, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "MergeTreeData.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeData.cpp", + "Message": "A code fragment from 'MergeTreeData.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 4013367774, + "CodeNext": 3698074834, + "CodePrev": 1857246075, + "ErrorCode": "V688", + "FileName": "MergeTreeDataMergerMutator.cpp", + "Message": "The 'data' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 896405418, + "CodeNext": 2464759539, + "CodePrev": 5816, + "ErrorCode": "V730", + "FileName": "MergeTreeDataMergerMutator.cpp", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: average_elem_progress." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeDataMergerMutator.cpp", + "Message": "A code fragment from 'MergeTreeDataMergerMutator.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3572738747, + "CodeNext": 123, + "CodePrev": 3750922986, + "ErrorCode": "V688", + "FileName": "MergeTreeDataPart.cpp", + "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 1616975535, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "MergeTreeDataPart.cpp", + "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2800774877, + "CodeNext": 403175306, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "MergeTreeDataPart.cpp", + "Message": "The 'columns' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeDataPart.cpp", + "Message": "A code fragment from 'MergeTreeDataPart.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreePartInfo.cpp", + "Message": "A code fragment from 'MergeTreePartInfo.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeDataWriter.cpp", + "Message": "A code fragment from 'MergeTreeDataWriter.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeDataSelectExecutor.cpp", + "Message": "A code fragment from 'MergeTreeDataSelectExecutor.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreePartition.cpp", + "Message": "A code fragment from 'MergeTreePartition.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2300449615, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "MergeTreeRangeReader.cpp", + "Message": "The 'filter' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3630487317, + "CodeNext": 580147943, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "MergeTreeRangeReader.cpp", + "Message": "The 'filter' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeRangeReader.cpp", + "Message": "A code fragment from 'MergeTreeRangeReader.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeSettings.cpp", + "Message": "A code fragment from 'MergeTreeSettings.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 800849723, + "CodeNext": 355939354, + "CodePrev": 998672422, + "ErrorCode": "V688", + "FileName": "MergeTreeReadPool.cpp", + "Message": "The 'parts' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2641370323, + "CodeNext": 285699983, + "CodePrev": 807170999, + "ErrorCode": "V688", + "FileName": "MergeTreeReadPool.cpp", + "Message": "The 'parts' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 1997550293, + "CodeNext": 0, + "CodePrev": 539, + "ErrorCode": "V601", + "FileName": "MergeTreeReadPool.cpp", + "Message": "The 'false' value is implicitly cast to the integer type. Inspect the first argument." + }, + { + "CodeCurrent": 2679702858, + "CodeNext": 123, + "CodePrev": 4119509742, + "ErrorCode": "V688", + "FileName": "MergeTreeReadPool.cpp", + "Message": "The 'parts' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeReadPool.cpp", + "Message": "A code fragment from 'MergeTreeReadPool.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1269723304, + "CodeNext": 4205841567, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "MergeTreeReader.cpp", + "Message": "The 'all_mark_ranges' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeReader.cpp", + "Message": "A code fragment from 'MergeTreeReader.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1175987107, + "CodeNext": 123, + "CodePrev": 2568774401, + "ErrorCode": "V730", + "FileName": "ReplicatedMergeTreeAddress.h", + "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: replication_port, queries_port." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReplicatedMergeTreeAddress.cpp", + "Message": "A code fragment from 'ReplicatedMergeTreeAddress.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 763384319, + "CodeNext": 2422251580, + "CodePrev": 2529539154, + "ErrorCode": "V688", + "FileName": "MergeTreeThreadBlockInputStream.cpp", + "Message": "The 'max_block_size_rows' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 222867506, + "CodeNext": 163883724, + "CodePrev": 2381055092, + "ErrorCode": "V688", + "FileName": "MergeTreeThreadBlockInputStream.cpp", + "Message": "The 'storage' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeThreadBlockInputStream.cpp", + "Message": "A code fragment from 'MergeTreeThreadBlockInputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergeTreeWhereOptimizer.cpp", + "Message": "A code fragment from 'MergeTreeWhereOptimizer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MergedBlockOutputStream.cpp", + "Message": "A code fragment from 'MergedBlockOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReplicatedMergeTreeLogEntry.cpp", + "Message": "A code fragment from 'ReplicatedMergeTreeLogEntry.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReplicatedMergeTreeAlterThread.cpp", + "Message": "A code fragment from 'ReplicatedMergeTreeAlterThread.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReplicatedMergeTreeBlockOutputStream.cpp", + "Message": "A code fragment from 'ReplicatedMergeTreeBlockOutputStream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReplicatedMergeTreeCleanupThread.cpp", + "Message": "A code fragment from 'ReplicatedMergeTreeCleanupThread.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReplicatedMergeTreeMutationEntry.cpp", + "Message": "A code fragment from 'ReplicatedMergeTreeMutationEntry.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "SimpleMergeSelector.cpp", + "Message": "A code fragment from 'SimpleMergeSelector.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReplicatedMergeTreePartCheckThread.cpp", + "Message": "A code fragment from 'ReplicatedMergeTreePartCheckThread.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReplicatedMergeTreeQueue.cpp", + "Message": "A code fragment from 'ReplicatedMergeTreeQueue.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReplicatedMergeTreeRestartingThread.cpp", + "Message": "A code fragment from 'ReplicatedMergeTreeRestartingThread.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "checkDataPart.cpp", + "Message": "A code fragment from 'checkDataPart.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Connection.cpp", + "Message": "A code fragment from 'Connection.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 404950407, + "CodeNext": 125, + "CodePrev": 3626175879, + "ErrorCode": "V656", + "FileName": "ConnectionPoolWithFailover.cpp", + "Message": "Variables 'min_entries', 'max_entries' are initialized through the call to the same function. It's probably an error or un-optimized code. Consider inspecting the 'nested_pools.size()' expression. Check lines: _, _." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ConnectionPoolWithFailover.cpp", + "Message": "A code fragment from 'ConnectionPoolWithFailover.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MultiplexedConnections.cpp", + "Message": "A code fragment from 'MultiplexedConnections.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerStorageMergeTree.cpp", + "Message": "A code fragment from 'registerStorageMergeTree.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionFactory.cpp", + "Message": "A code fragment from 'FunctionFactory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IFunction.cpp", + "Message": "A code fragment from 'IFunction.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionHelpers.cpp", + "Message": "A code fragment from 'FunctionHelpers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionFactory.cpp", + "Message": "A code fragment from 'AggregateFunctionFactory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionCombinatorFactory.cpp", + "Message": "A code fragment from 'AggregateFunctionCombinatorFactory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FactoryHelpers.cpp", + "Message": "A code fragment from 'FactoryHelpers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ITableFunction.cpp", + "Message": "A code fragment from 'ITableFunction.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "parseAggregateFunctionParameters.cpp", + "Message": "A code fragment from 'parseAggregateFunctionParameters.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionState.cpp", + "Message": "A code fragment from 'AggregateFunctionState.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ActionLock.cpp", + "Message": "A code fragment from 'ActionLock.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CurrentMetrics.cpp", + "Message": "A code fragment from 'CurrentMetrics.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "BackgroundSchedulePool.cpp", + "Message": "A code fragment from 'BackgroundSchedulePool.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Allocator.cpp", + "Message": "A code fragment from 'Allocator.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "DNSResolver.cpp", + "Message": "A code fragment from 'DNSResolver.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TableFunctionFactory.cpp", + "Message": "A code fragment from 'TableFunctionFactory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 123, + "CodeNext": 125, + "CodePrev": 35837, + "ErrorCode": "V565", + "FileName": "Exception.cpp", + "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." + }, + { + "CodeCurrent": 143231, + "CodeNext": 125, + "CodePrev": 125, + "ErrorCode": "V565", + "FileName": "Exception.cpp", + "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." + }, + { + "CodeCurrent": 143231, + "CodeNext": 125, + "CodePrev": 125, + "ErrorCode": "V565", + "FileName": "Exception.cpp", + "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." + }, + { + "CodeCurrent": 143231, + "CodeNext": 125, + "CodePrev": 125, + "ErrorCode": "V565", + "FileName": "Exception.cpp", + "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." + }, + { + "CodeCurrent": 143231, + "CodeNext": 0, + "CodePrev": 125, + "ErrorCode": "V565", + "FileName": "Exception.cpp", + "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Exception.cpp", + "Message": "A code fragment from 'Exception.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Macros.cpp", + "Message": "A code fragment from 'Macros.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3496318343, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "FileChecker.cpp", + "Message": "The 'map' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FileChecker.cpp", + "Message": "A code fragment from 'FileChecker.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FieldVisitors.cpp", + "Message": "A code fragment from 'FieldVisitors.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1249266459, + "CodeNext": 125, + "CodePrev": 82483, + "ErrorCode": "V769", + "FileName": "SharedLibrary.cpp", + "Message": "The 'dlerror()' pointer in the expression could be nullptr. In such case, resulting value of arithmetic operations on this pointer will be senseless and it should not be used." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "SharedLibrary.cpp", + "Message": "A code fragment from 'SharedLibrary.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "RWLock.cpp", + "Message": "A code fragment from 'RWLock.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "config_build.cpp", + "Message": "A code fragment from 'config_build.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StackTrace.cpp", + "Message": "A code fragment from 'StackTrace.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "escapeForFileName.cpp", + "Message": "A code fragment from 'escapeForFileName.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MemoryTracker.cpp", + "Message": "A code fragment from 'MemoryTracker.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 348362777, + "CodeNext": 348528025, + "CodePrev": 897852770, + "ErrorCode": "V525", + "FileName": "ShellCommand.cpp", + "Message": "The code contains the collection of similar blocks. Check items 'write_fd', 'read_fd', 'read_fd' in lines _, _, _." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ShellCommand.cpp", + "Message": "A code fragment from 'ShellCommand.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "formatIPv6.cpp", + "Message": "A code fragment from 'formatIPv_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "getFQDNOrHostName.cpp", + "Message": "A code fragment from 'getFQDNOrHostName.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "getMultipleKeysFromConfig.cpp", + "Message": "A code fragment from 'getMultipleKeysFromConfig.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "getNumberOfPhysicalCPUCores.cpp", + "Message": "A code fragment from 'getNumberOfPhysicalCPUCores.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "formatReadable.cpp", + "Message": "A code fragment from 'formatReadable.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "hex.cpp", + "Message": "A code fragment from 'hex.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "localBackup.cpp", + "Message": "A code fragment from 'localBackup.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "getPerformanceMessage.cpp", + "Message": "A code fragment from 'getPerformanceMessage.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "isLocalAddress.cpp", + "Message": "A code fragment from 'isLocalAddress.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "randomSeed.cpp", + "Message": "A code fragment from 'randomSeed.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "setThreadName.cpp", + "Message": "A code fragment from 'setThreadName.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "parseAddress.cpp", + "Message": "A code fragment from 'parseAddress.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CascadeWriteBuffer.cpp", + "Message": "A code fragment from 'CascadeWriteBuffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CompressedReadBuffer.cpp", + "Message": "A code fragment from 'CompressedReadBuffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CachedCompressedReadBuffer.cpp", + "Message": "A code fragment from 'CachedCompressedReadBuffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CompressedReadBufferBase.cpp", + "Message": "A code fragment from 'CompressedReadBufferBase.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CompressedWriteBuffer.cpp", + "Message": "A code fragment from 'CompressedWriteBuffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CompressedReadBufferFromFile.cpp", + "Message": "A code fragment from 'CompressedReadBufferFromFile.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CompressionSettings.cpp", + "Message": "A code fragment from 'CompressionSettings.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "HexWriteBuffer.cpp", + "Message": "A code fragment from 'HexWriteBuffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "HTTPCommon.cpp", + "Message": "A code fragment from 'HTTPCommon.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "HashingWriteBuffer.cpp", + "Message": "A code fragment from 'HashingWriteBuffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "LimitReadBuffer.cpp", + "Message": "A code fragment from 'LimitReadBuffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MemoryReadWriteBuffer.cpp", + "Message": "A code fragment from 'MemoryReadWriteBuffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterserverWriteBuffer.cpp", + "Message": "A code fragment from 'InterserverWriteBuffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Progress.cpp", + "Message": "A code fragment from 'Progress.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReadBufferFromFile.cpp", + "Message": "A code fragment from 'ReadBufferFromFile.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 286621345, + "CodeNext": 0, + "CodePrev": 3895740, + "ErrorCode": "V688", + "FileName": "AIO.h", + "Message": "The 'id' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3733418444, + "CodeNext": 380034981, + "CodePrev": 729278174, + "ErrorCode": "V1007", + "FileName": "ReadBufferAIO.cpp", + "Message": "The value from the potentially uninitialized optional 'watch' is used. Probably it is a mistake." + }, + { + "CodeCurrent": 3788532949, + "CodeNext": 0, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "ReadBufferAIO.cpp", + "Message": "The 'metric_increment' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3788532949, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ReadBufferAIO.cpp", + "Message": "The 'metric_increment' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReadBufferAIO.cpp", + "Message": "A code fragment from 'ReadBufferAIO.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 743595643, + "CodeNext": 2465481192, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "ReadBufferFromFileBase.cpp", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: clock_type." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReadBufferFromFileBase.cpp", + "Message": "A code fragment from 'ReadBufferFromFileBase.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReadBufferFromPocoSocket.cpp", + "Message": "A code fragment from 'ReadBufferFromPocoSocket.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3733418444, + "CodeNext": 380034981, + "CodePrev": 39251785, + "ErrorCode": "V1007", + "FileName": "ReadBufferFromFileDescriptor.cpp", + "Message": "The value from the potentially uninitialized optional 'watch' is used. Probably it is a mistake." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReadBufferFromFileDescriptor.cpp", + "Message": "A code fragment from 'ReadBufferFromFileDescriptor.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3116768807, + "CodeNext": 3116768807, + "CodePrev": 127832661, + "ErrorCode": "V525", + "FileName": "ReadHelpers.cpp", + "Message": "The code contains the collection of similar blocks. Check items '_', '_', '_' in lines _, _, _." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReadHelpers.cpp", + "Message": "A code fragment from 'ReadHelpers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3282098908, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "WriteBufferAIO.cpp", + "Message": "The 'metric_increment' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "WriteBufferAIO.cpp", + "Message": "A code fragment from 'WriteBufferAIO.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2604535988, + "CodeNext": 1150311866, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ReadWriteBufferFromHTTP.cpp", + "Message": "The 'uri' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 376832871, + "CodeNext": 21716451, + "CodePrev": 3114652973, + "ErrorCode": "V688", + "FileName": "ReadWriteBufferFromHTTP.cpp", + "Message": "The 'timeouts' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReadWriteBufferFromHTTP.cpp", + "Message": "A code fragment from 'ReadWriteBufferFromHTTP.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "WriteBufferFromFile.cpp", + "Message": "A code fragment from 'WriteBufferFromFile.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "WriteBufferFromFileBase.cpp", + "Message": "A code fragment from 'WriteBufferFromFileBase.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "WriteBufferFromPocoSocket.cpp", + "Message": "A code fragment from 'WriteBufferFromPocoSocket.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "WriteBufferFromHTTPServerResponse.cpp", + "Message": "A code fragment from 'WriteBufferFromHTTPServerResponse.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "WriteBufferFromTemporaryFile.cpp", + "Message": "A code fragment from 'WriteBufferFromTemporaryFile.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "WriteBufferFromFileDescriptor.cpp", + "Message": "A code fragment from 'WriteBufferFromFileDescriptor.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "WriteBufferValidUTF8.cpp", + "Message": "A code fragment from 'WriteBufferValidUTF_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ZlibDeflatingWriteBuffer.cpp", + "Message": "A code fragment from 'ZlibDeflatingWriteBuffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ZlibInflatingReadBuffer.cpp", + "Message": "A code fragment from 'ZlibInflatingReadBuffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 622470420, + "CodeNext": 157897, + "CodePrev": 123, + "ErrorCode": "V525", + "FileName": "WriteHelpers.cpp", + "Message": "The code contains the collection of similar blocks. Check items '_', '_', '_', '_' in lines _, _, _, _." + }, + { + "CodeCurrent": 1208444161, + "CodeNext": 157897, + "CodePrev": 123, + "ErrorCode": "V525", + "FileName": "WriteHelpers.cpp", + "Message": "The code contains the collection of similar blocks. Check items '_', '_', '_' in lines _, _, _." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "WriteHelpers.cpp", + "Message": "A code fragment from 'WriteHelpers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "copyData.cpp", + "Message": "A code fragment from 'copyData.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "createWriteBufferFromFileBase.cpp", + "Message": "A code fragment from 'createWriteBufferFromFileBase.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "createReadBufferFromFileBase.cpp", + "Message": "A code fragment from 'createReadBufferFromFileBase.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "parseDateTimeBestEffort.cpp", + "Message": "A code fragment from 'parseDateTimeBestEffort.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "readFloatText.cpp", + "Message": "A code fragment from 'readFloatText.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "gtest_unescapeForFileName.cpp", + "Message": "A code fragment from 'gtest_unescapeForFileName.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "gtest_rw_lock_fifo.cpp.cpp", + "Message": "A code fragment from 'gtest_rw_lock_fifo.cpp.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "gtest_zkutil_test_multi_exception.cpp", + "Message": "A code fragment from 'gtest_zkutil_test_multi_exception.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StringUtils.cpp", + "Message": "A code fragment from 'StringUtils.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3791607228, + "CodeNext": 60247417, + "CodePrev": 0, + "ErrorCode": "V522", + "FileName": "gtest_cascade_and_memory_write_buffer.cpp", + "Message": "There might be dereferencing of a potential null pointer 'wbuf_readable'." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "gtest_cascade_and_memory_write_buffer.cpp", + "Message": "A code fragment from 'gtest_cascade_and_memory_write_buffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "stopwatch.cpp", + "Message": "A code fragment from 'stopwatch.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "allocator.cpp", + "Message": "A code fragment from 'allocator.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "thread_pool.cpp", + "Message": "A code fragment from 'thread_pool.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "gtest_row_source_bits_test.cpp", + "Message": "A code fragment from 'gtest_row_source_bits_test.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "integer_hash_tables_and_hashes.cpp", + "Message": "A code fragment from 'integer_hash_tables_and_hashes.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3255795021, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "ArrayCache.h", + "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: ptr, size, chunk." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "array_cache.cpp", + "Message": "A code fragment from 'array_cache.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "thread_creation_latency.cpp", + "Message": "A code fragment from 'thread_creation_latency.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "lru_cache.cpp", + "Message": "A code fragment from 'lru_cache.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "sip_hash_perf.cpp", + "Message": "A code fragment from 'sip_hash_perf.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "arena_with_free_lists.cpp", + "Message": "A code fragment from 'arena_with_free_lists.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "auto_array.cpp", + "Message": "A code fragment from 'auto_array.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "sip_hash.cpp", + "Message": "A code fragment from 'sip_hash.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "hashes_test.cpp", + "Message": "A code fragment from 'hashes_test.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 178542355, + "CodeNext": 1913064305, + "CodePrev": 0, + "ErrorCode": "V760", + "FileName": "small_table.cpp", + "Message": "Two identical blocks of text were found. The second block begins from line _." + }, + { + "CodeCurrent": 3824136199, + "CodeNext": 2663, + "CodePrev": 5816, + "ErrorCode": "V730", + "FileName": "SmallTable.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: size." + }, + { + "CodeCurrent": 132655, + "CodeNext": 1795381173, + "CodePrev": 5816, + "ErrorCode": "V730", + "FileName": "SmallTable.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: container, ptr." + }, + { + "CodeCurrent": 9156143, + "CodeNext": 377744136, + "CodePrev": 5816, + "ErrorCode": "V730", + "FileName": "SmallTable.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: container, ptr." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "small_table.cpp", + "Message": "A code fragment from 'small_table.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "parallel_aggregation.cpp", + "Message": "A code fragment from 'parallel_aggregation.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AvalancheTest.cpp", + "Message": "A code fragment from 'AvalancheTest.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "parallel_aggregation2.cpp", + "Message": "A code fragment from 'parallel_aggregation_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 100335, + "CodeNext": 0, + "CodePrev": 123, + "ErrorCode": "V730", + "FileName": "SpaceSaving.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: slot, hash, count, error." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "space_saving.cpp", + "Message": "A code fragment from 'space_saving.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "int_hashes_perf.cpp", + "Message": "A code fragment from 'int_hashes_perf.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "simple_cache.cpp", + "Message": "A code fragment from 'simple_cache.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "cow_columns.cpp", + "Message": "A code fragment from 'cow_columns.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "pod_array.cpp", + "Message": "A code fragment from 'pod_array.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 178542355, + "CodeNext": 1913064305, + "CodePrev": 0, + "ErrorCode": "V760", + "FileName": "hash_table.cpp", + "Message": "Two identical blocks of text were found. The second block begins from line _." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "hash_table.cpp", + "Message": "A code fragment from 'hash_table.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 238596103, + "CodeNext": 2663, + "CodePrev": 5816, + "ErrorCode": "V730", + "FileName": "CompactArray.h", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: value_l, value_r, fits_in_byte." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "compact_array.cpp", + "Message": "A code fragment from 'compact_array.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1524853010, + "CodeNext": 193, + "CodePrev": 182716498, + "ErrorCode": "V524", + "FileName": "RadixSort.h", + "Message": "It is odd that the body of 'backward' function is fully equivalent to the body of 'forward' function." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "radix_sort.cpp", + "Message": "A code fragment from 'radix_sort.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "shell_command_test.cpp", + "Message": "A code fragment from 'shell_command_test.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "rvo_test.cpp", + "Message": "A code fragment from 'rvo_test.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "exception.cpp", + "Message": "A code fragment from 'exception.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "move_field.cpp", + "Message": "A code fragment from 'move_field.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "string_pool.cpp", + "Message": "A code fragment from 'string_pool.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "string_ref_hash.cpp", + "Message": "A code fragment from 'string_ref_hash.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "field.cpp", + "Message": "A code fragment from 'field.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "filter_stream_hitlog.cpp", + "Message": "A code fragment from 'filter_stream_hitlog.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "filter_stream.cpp", + "Message": "A code fragment from 'filter_stream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "block_row_transforms.cpp", + "Message": "A code fragment from 'block_row_transforms.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "native_streams.cpp", + "Message": "A code fragment from 'native_streams.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "expression_stream.cpp", + "Message": "A code fragment from 'expression_stream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "tab_separated_streams.cpp", + "Message": "A code fragment from 'tab_separated_streams.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "sorting_stream.cpp", + "Message": "A code fragment from 'sorting_stream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "data_type_string.cpp", + "Message": "A code fragment from 'data_type_string.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "union_stream2.cpp", + "Message": "A code fragment from 'union_stream_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "collapsing_sorted_stream.cpp", + "Message": "A code fragment from 'collapsing_sorted_stream.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3646287644, + "CodeNext": 3487437108, + "CodePrev": 3633985876, + "ErrorCode": "V560", + "FileName": "data_type_get_common_type.cpp", + "Message": "A part of conditional expression is always false: !print_stack_trace." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "data_type_get_common_type.cpp", + "Message": "A code fragment from 'data_type_get_common_type.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "data_types_number_fixed.cpp", + "Message": "A code fragment from 'data_types_number_fixed.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemAsynchronousMetrics.cpp", + "Message": "A code fragment from 'StorageSystemAsynchronousMetrics.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemBuildOptions.cpp", + "Message": "A code fragment from 'StorageSystemBuildOptions.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemClusters.cpp", + "Message": "A code fragment from 'StorageSystemClusters.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemDatabases.cpp", + "Message": "A code fragment from 'StorageSystemDatabases.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'StorageSystemDictionaries.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemColumns.cpp", + "Message": "A code fragment from 'StorageSystemColumns.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemEvents.cpp", + "Message": "A code fragment from 'StorageSystemEvents.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemFunctions.cpp", + "Message": "A code fragment from 'StorageSystemFunctions.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1466671053, + "CodeNext": 1467990035, + "CodePrev": 1938442408, + "ErrorCode": "V601", + "FileName": "StorageSystemGraphite.cpp", + "Message": "The 'false' value is implicitly cast to the integer type." + }, + { + "CodeCurrent": 733335125, + "CodeNext": 2791142600, + "CodePrev": 1938442408, + "ErrorCode": "V601", + "FileName": "StorageSystemGraphite.cpp", + "Message": "The 'true' value is implicitly cast to the integer type." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemGraphite.cpp", + "Message": "A code fragment from 'StorageSystemGraphite.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemMacros.cpp", + "Message": "A code fragment from 'StorageSystemMacros.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemMerges.cpp", + "Message": "A code fragment from 'StorageSystemMerges.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemMetrics.cpp", + "Message": "A code fragment from 'StorageSystemMetrics.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'StorageSystemModels.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemNumbers.cpp", + "Message": "A code fragment from 'StorageSystemNumbers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemOne.cpp", + "Message": "A code fragment from 'StorageSystemOne.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemParts.cpp", + "Message": "A code fragment from 'StorageSystemParts.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemProcesses.cpp", + "Message": "A code fragment from 'StorageSystemProcesses.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3754734131, + "CodeNext": 0, + "CodePrev": 3910557249, + "ErrorCode": "V688", + "FileName": "StorageSystemPartsBase.cpp", + "Message": "The 'database_column' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3833782239, + "CodeNext": 3076741407, + "CodePrev": 0, + "ErrorCode": "V525", + "FileName": "StorageSystemPartsBase.cpp", + "Message": "The code contains the collection of similar blocks. Check items 'ColumnString', 'ColumnString', 'ColumnUInt_' in lines _, _, _." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemPartsBase.cpp", + "Message": "A code fragment from 'StorageSystemPartsBase.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemPartsColumns.cpp", + "Message": "A code fragment from 'StorageSystemPartsColumns.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemTables.cpp", + "Message": "A code fragment from 'StorageSystemTables.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemSettings.cpp", + "Message": "A code fragment from 'StorageSystemSettings.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemReplicas.cpp", + "Message": "A code fragment from 'StorageSystemReplicas.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemReplicationQueue.cpp", + "Message": "A code fragment from 'StorageSystemReplicationQueue.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "seek_speed_test.cpp", + "Message": "A code fragment from 'seek_speed_test.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StorageSystemZooKeeper.cpp", + "Message": "A code fragment from 'StorageSystemZooKeeper.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "part_name.cpp", + "Message": "A code fragment from 'part_name.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "hit_log.cpp", + "Message": "A code fragment from 'hit_log.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "attachSystemTables.cpp", + "Message": "A code fragment from 'attachSystemTables.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "remove_symlink_directory.cpp", + "Message": "A code fragment from 'remove_symlink_directory.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "merge_selector2.cpp", + "Message": "A code fragment from 'merge_selector_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "storage_log.cpp", + "Message": "A code fragment from 'storage_log.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "part_checker.cpp", + "Message": "A code fragment from 'part_checker.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "get_current_inserts_in_replicated.cpp", + "Message": "A code fragment from 'get_current_inserts_in_replicated.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "system_numbers.cpp", + "Message": "A code fragment from 'system_numbers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTAlterQuery.cpp", + "Message": "A code fragment from 'ASTAlterQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTExpressionList.cpp", + "Message": "A code fragment from 'ASTExpressionList.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "merge_selector.cpp", + "Message": "A code fragment from 'merge_selector.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "get_abandonable_lock_in_all_partitions.cpp", + "Message": "A code fragment from 'get_abandonable_lock_in_all_partitions.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTInsertQuery.cpp", + "Message": "A code fragment from 'ASTInsertQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTIdentifier.cpp", + "Message": "A code fragment from 'ASTIdentifier.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTFunction.cpp", + "Message": "A code fragment from 'ASTFunction.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTKillQueryQuery.cpp", + "Message": "A code fragment from 'ASTKillQueryQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTOrderByElement.cpp", + "Message": "A code fragment from 'ASTOrderByElement.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTLiteral.cpp", + "Message": "A code fragment from 'ASTLiteral.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTQualifiedAsterisk.cpp", + "Message": "A code fragment from 'ASTQualifiedAsterisk.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTPartition.cpp", + "Message": "A code fragment from 'ASTPartition.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTQueryWithOutput.cpp", + "Message": "A code fragment from 'ASTQueryWithOutput.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTSampleRatio.cpp", + "Message": "A code fragment from 'ASTSampleRatio.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTQueryWithOnCluster.cpp", + "Message": "A code fragment from 'ASTQueryWithOnCluster.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTSelectWithUnionQuery.cpp", + "Message": "A code fragment from 'ASTSelectWithUnionQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTSystemQuery.cpp", + "Message": "A code fragment from 'ASTSystemQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTSelectQuery.cpp", + "Message": "A code fragment from 'ASTSelectQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTTablesInSelectQuery.cpp", + "Message": "A code fragment from 'ASTTablesInSelectQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTSubquery.cpp", + "Message": "A code fragment from 'ASTSubquery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1818347258, + "CodeNext": 176909, + "CodePrev": 0, + "ErrorCode": "V526", + "FileName": "CommonParsers.cpp", + "Message": "The 'strncasecmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "CommonParsers.cpp", + "Message": "A code fragment from 'CommonParsers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ASTWithAlias.cpp", + "Message": "A code fragment from 'ASTWithAlias.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ExpressionListParsers.cpp", + "Message": "A code fragment from 'ExpressionListParsers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Lexer.cpp", + "Message": "A code fragment from 'Lexer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ExpressionElementParsers.cpp", + "Message": "A code fragment from 'ExpressionElementParsers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IParserBase.cpp", + "Message": "A code fragment from 'IParserBase.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IAST.cpp", + "Message": "A code fragment from 'IAST.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserCheckQuery.cpp", + "Message": "A code fragment from 'ParserCheckQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserAlterQuery.cpp", + "Message": "A code fragment from 'ParserAlterQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserCase.cpp", + "Message": "A code fragment from 'ParserCase.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserDescribeTableQuery.cpp", + "Message": "A code fragment from 'ParserDescribeTableQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserCreateQuery.cpp", + "Message": "A code fragment from 'ParserCreateQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserDropQuery.cpp", + "Message": "A code fragment from 'ParserDropQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserKillQueryQuery.cpp", + "Message": "A code fragment from 'ParserKillQueryQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserInsertQuery.cpp", + "Message": "A code fragment from 'ParserInsertQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserOptimizeQuery.cpp", + "Message": "A code fragment from 'ParserOptimizeQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserQuery.cpp", + "Message": "A code fragment from 'ParserQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserPartition.cpp", + "Message": "A code fragment from 'ParserPartition.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserQueryWithOutput.cpp", + "Message": "A code fragment from 'ParserQueryWithOutput.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserRenameQuery.cpp", + "Message": "A code fragment from 'ParserRenameQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserSelectQuery.cpp", + "Message": "A code fragment from 'ParserSelectQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserSelectWithUnionQuery.cpp", + "Message": "A code fragment from 'ParserSelectWithUnionQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserSampleRatio.cpp", + "Message": "A code fragment from 'ParserSampleRatio.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserSetQuery.cpp", + "Message": "A code fragment from 'ParserSetQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserTablePropertiesQuery.cpp", + "Message": "A code fragment from 'ParserTablePropertiesQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserShowTablesQuery.cpp", + "Message": "A code fragment from 'ParserShowTablesQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserSystemQuery.cpp", + "Message": "A code fragment from 'ParserSystemQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TokenIterator.cpp", + "Message": "A code fragment from 'TokenIterator.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserTablesInSelectQuery.cpp", + "Message": "A code fragment from 'ParserTablesInSelectQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserUnionQueryElement.cpp", + "Message": "A code fragment from 'ParserUnionQueryElement.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ParserUseQuery.cpp", + "Message": "A code fragment from 'ParserUseQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "formatAST.cpp", + "Message": "A code fragment from 'formatAST.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "parseDatabaseAndTableName.cpp", + "Message": "A code fragment from 'parseDatabaseAndTableName.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "parseIdentifierOrStringLiteral.cpp", + "Message": "A code fragment from 'parseIdentifierOrStringLiteral.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "queryToString.cpp", + "Message": "A code fragment from 'queryToString.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "iostream_debug_helpers.cpp", + "Message": "A code fragment from 'iostream_debug_helpers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "parseQuery.cpp", + "Message": "A code fragment from 'parseQuery.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "create_parser.cpp", + "Message": "A code fragment from 'create_parser.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "select_parser.cpp", + "Message": "A code fragment from 'select_parser.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "lexer.cpp", + "Message": "A code fragment from 'lexer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "parse_date_time_best_effort.cpp", + "Message": "A code fragment from 'parse_date_time_best_effort.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "limit_read_buffer2.cpp", + "Message": "A code fragment from 'limit_read_buffer_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "zlib_buffers.cpp", + "Message": "A code fragment from 'zlib_buffers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "operators.cpp", + "Message": "A code fragment from 'operators.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "read_buffer_aio.cpp", + "Message": "A code fragment from 'read_buffer_aio.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "hashing_read_buffer.cpp", + "Message": "A code fragment from 'hashing_read_buffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "hashing_write_buffer.cpp", + "Message": "A code fragment from 'hashing_write_buffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "valid_utf8_perf.cpp", + "Message": "A code fragment from 'valid_utf__perf.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "o_direct_and_dirty_pages.cpp", + "Message": "A code fragment from 'o_direct_and_dirty_pages.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "read_buffer.cpp", + "Message": "A code fragment from 'read_buffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "read_float_perf.cpp", + "Message": "A code fragment from 'read_float_perf.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "compressed_buffer.cpp", + "Message": "A code fragment from 'compressed_buffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "write_buffer_perf.cpp", + "Message": "A code fragment from 'write_buffer_perf.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "write_buffer.cpp", + "Message": "A code fragment from 'write_buffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "read_escaped_string.cpp", + "Message": "A code fragment from 'read_escaped_string.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "read_buffer_perf.cpp", + "Message": "A code fragment from 'read_buffer_perf.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "limit_read_buffer.cpp", + "Message": "A code fragment from 'limit_read_buffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "cached_compressed_read_buffer.cpp", + "Message": "A code fragment from 'cached_compressed_read_buffer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "valid_utf8.cpp", + "Message": "A code fragment from 'valid_utf_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "mempbrk.cpp", + "Message": "A code fragment from 'mempbrk.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "var_uint.cpp", + "Message": "A code fragment from 'var_uint.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "parse_int_perf2.cpp", + "Message": "A code fragment from 'parse_int_perf_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "write_buffer_aio.cpp", + "Message": "A code fragment from 'write_buffer_aio.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "async_write.cpp", + "Message": "A code fragment from 'async_write.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "parse_int_perf.cpp", + "Message": "A code fragment from 'parse_int_perf.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "read_write_int.cpp", + "Message": "A code fragment from 'read_write_int.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsCharset.cpp", + "Message": "A code fragment from 'FunctionsCharset.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsCoding.cpp", + "Message": "A code fragment from 'FunctionsCoding.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2701979474, + "CodeNext": 125, + "CodePrev": 539, + "ErrorCode": "V522", + "FileName": "FunctionsArray.h", + "Message": "There might be dereferencing of a potential null pointer 'null_map_data'." + }, + { + "CodeCurrent": 2701979474, + "CodeNext": 125, + "CodePrev": 539, + "ErrorCode": "V522", + "FileName": "FunctionsArray.h", + "Message": "There might be dereferencing of a potential null pointer 'null_map_item'." + }, + { + "CodeCurrent": 2701979474, + "CodeNext": 125, + "CodePrev": 539, + "ErrorCode": "V522", + "FileName": "FunctionsArray.h", + "Message": "There might be dereferencing of a potential null pointer 'null_map_data'." + }, + { + "CodeCurrent": 2701979474, + "CodeNext": 125, + "CodePrev": 539, + "ErrorCode": "V522", + "FileName": "FunctionsArray.h", + "Message": "There might be dereferencing of a potential null pointer 'null_map_item'." + }, + { + "CodeCurrent": 1933433026, + "CodeNext": 352513873, + "CodePrev": 1066296242, + "ErrorCode": "V522", + "FileName": "FunctionsArray.cpp", + "Message": "There might be dereferencing of a potential null pointer 'col_const_array'." + }, + { + "CodeCurrent": 3479588408, + "CodeNext": 360719, + "CodePrev": 0, + "ErrorCode": "V519", + "FileName": "FunctionsArray.cpp", + "Message": "The 'col_array' variable is assigned values twice successively. Perhaps this is a mistake. Check lines: _, _." + }, + { + "CodeCurrent": 2226276340, + "CodeNext": 2372992004, + "CodePrev": 914871094, + "ErrorCode": "V601", + "FileName": "FunctionsArray.cpp", + "Message": "The 'false' value is implicitly cast to the integer type. Inspect the second argument." + }, + { + "CodeCurrent": 642252437, + "CodeNext": 1179013152, + "CodePrev": 123, + "ErrorCode": "V601", + "FileName": "FunctionsArray.cpp", + "Message": "The 'true' value is implicitly cast to the integer type." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ValueSourceVisitor.h", + "Message": "A code fragment from 'FunctionsArray.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'FunctionsComparison.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsConsistentHashing.cpp", + "Message": "A code fragment from 'FunctionsConsistentHashing.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 780446959, + "CodeNext": 0, + "CodePrev": 1833977828, + "ErrorCode": "V728", + "FileName": "Algorithms.h", + "Message": "An excessive check can be simplified. The '||' operator is surrounded by opposite expressions '!size_null_map' and 'size_null_map'." + }, + { + "CodeCurrent": 2874700209, + "CodeNext": 0, + "CodePrev": 539, + "ErrorCode": "V522", + "FileName": "FunctionsConditional.h", + "Message": "There might be dereferencing of a potential null pointer 'col_right_const'." + }, + { + "CodeCurrent": 4053347441, + "CodeNext": 0, + "CodePrev": 539, + "ErrorCode": "V522", + "FileName": "FunctionsConditional.h", + "Message": "There might be dereferencing of a potential null pointer 'col_right_const'." + }, + { + "CodeCurrent": 1316020648, + "CodeNext": 67675834, + "CodePrev": 123, + "ErrorCode": "V522", + "FileName": "FunctionsConditional.h", + "Message": "There might be dereferencing of a potential null pointer 'col_right_const_array'." + }, + { + "CodeCurrent": 1316020648, + "CodeNext": 67675834, + "CodePrev": 123, + "ErrorCode": "V522", + "FileName": "FunctionsConditional.h", + "Message": "There might be dereferencing of a potential null pointer 'col_right_const_array'." + }, + { + "CodeCurrent": 3479214347, + "CodeNext": 328401, + "CodePrev": 0, + "ErrorCode": "V519", + "FileName": "FunctionsConditional.h", + "Message": "The 'col_left' variable is assigned values twice successively. Perhaps this is a mistake. Check lines: _, _." + }, + { + "CodeCurrent": 3244155324, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V524", + "FileName": "Sources.h", + "Message": "It is odd that the body of 'getColumnSize' function is fully equivalent to the body of 'getSizeForReserve' function." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ValueSourceVisitor.h", + "Message": "A code fragment from 'FunctionsConditional.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 71291086, + "CodeNext": 285164697, + "CodePrev": 123, + "ErrorCode": "V522", + "FileName": "FunctionsConversion.h", + "Message": "There might be dereferencing of a potential null pointer 'col_from_string'." + }, + { + "CodeCurrent": 401577488, + "CodeNext": 3512469952, + "CodePrev": 123, + "ErrorCode": "V522", + "FileName": "FunctionsConversion.h", + "Message": "There might be dereferencing of a potential null pointer 'col_from_fixed_string'." + }, + { + "CodeCurrent": 2536180031, + "CodeNext": 4123240192, + "CodePrev": 123, + "ErrorCode": "V522", + "FileName": "FunctionsConversion.h", + "Message": "There might be dereferencing of a potential null pointer 'offsets'." + }, + { + "CodeCurrent": 2581192971, + "CodeNext": 675433, + "CodePrev": 41145, + "ErrorCode": "V522", + "FileName": "FunctionsConversion.h", + "Message": "There might be dereferencing of a potential null pointer 'local_time_zone'." + }, + { + "CodeCurrent": 2581192971, + "CodeNext": 675433, + "CodePrev": 41145, + "ErrorCode": "V522", + "FileName": "FunctionsConversion.h", + "Message": "There might be dereferencing of a potential null pointer 'utc_time_zone'." + }, + { + "CodeCurrent": 1069551226, + "CodeNext": 125, + "CodePrev": 3401161010, + "ErrorCode": "V522", + "FileName": "FunctionsConversion.h", + "Message": "There might be dereferencing of a potential null pointer 'vec_null_map_to'." + }, + { + "CodeCurrent": 3903117136, + "CodeNext": 3272131897, + "CodePrev": 0, + "ErrorCode": "V788", + "FileName": "FunctionsConversion.h", + "Message": "Uninitialized variable 'from_nested_type' will be used in the lambda expression, as it was captured by value." + }, + { + "CodeCurrent": 3903117136, + "CodeNext": 3272131897, + "CodePrev": 0, + "ErrorCode": "V788", + "FileName": "FunctionsConversion.h", + "Message": "Uninitialized variable 'to_nested_type' will be used in the lambda expression, as it was captured by value." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsConversion.cpp", + "Message": "A code fragment from 'FunctionsConversion.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsDateTime.cpp", + "Message": "A code fragment from 'FunctionsDateTime.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsEmbeddedDictionaries.cpp", + "Message": "A code fragment from 'FunctionsEmbeddedDictionaries.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsFindCluster.cpp", + "Message": "A code fragment from 'FunctionsFindCluster.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'FunctionsExternalModels.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "IExternalLoadable.h", + "Message": "A code fragment from 'FunctionsExternalDictionaries.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsFormatting.cpp", + "Message": "A code fragment from 'FunctionsFormatting.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsHashing.cpp", + "Message": "A code fragment from 'FunctionsHashing.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsHigherOrder.cpp", + "Message": "A code fragment from 'FunctionsHigherOrder.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'FunctionsLogical.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsMath.cpp", + "Message": "A code fragment from 'FunctionsMath.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1915387808, + "CodeNext": 123, + "CodePrev": 388146916, + "ErrorCode": "V688", + "FileName": "GeoUtils.h", + "Message": "The 'polygon' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'FunctionsGeo.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'FunctionsNull.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsMiscellaneous.cpp", + "Message": "A code fragment from 'FunctionsMiscellaneous.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsProjection.cpp", + "Message": "A code fragment from 'FunctionsProjection.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsRandom.cpp", + "Message": "A code fragment from 'FunctionsRandom.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsReinterpret.cpp", + "Message": "A code fragment from 'FunctionsReinterpret.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ValueSourceVisitor.h", + "Message": "A code fragment from 'FunctionsString.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsStringArray.cpp", + "Message": "A code fragment from 'FunctionsStringArray.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3108965974, + "CodeNext": 4119542900, + "CodePrev": 5816, + "ErrorCode": "V688", + "FileName": "StringSearcher.h", + "Message": "The 'needle_size' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3108965974, + "CodeNext": 4119542900, + "CodePrev": 5816, + "ErrorCode": "V688", + "FileName": "StringSearcher.h", + "Message": "The 'needle_size' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3108965974, + "CodeNext": 4119542900, + "CodePrev": 5816, + "ErrorCode": "V688", + "FileName": "StringSearcher.h", + "Message": "The 'needle_size' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 4259932362, + "CodeNext": 4031099091, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "Volnitsky.h", + "Message": "The 'needle_size' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2578531992, + "CodeNext": 1724192963, + "CodePrev": 3112257027, + "ErrorCode": "V519", + "FileName": "Volnitsky.h", + "Message": "The 'chars.c_' variable is assigned values twice successively. Perhaps this is a mistake. Check lines: _, _." + }, + { + "CodeCurrent": 1724192963, + "CodeNext": 371674977, + "CodePrev": 2578531992, + "ErrorCode": "V519", + "FileName": "Volnitsky.h", + "Message": "The 'chars.c_' variable is assigned values twice successively. Perhaps this is a mistake. Check lines: _, _." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'FunctionsStringSearch.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsTransform.cpp", + "Message": "A code fragment from 'FunctionsTransform.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsTuple.cpp", + "Message": "A code fragment from 'FunctionsTuple.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'FunctionsURL.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 88117, + "CodeNext": 125, + "CodePrev": 123, + "ErrorCode": "V601", + "FileName": "FunctionsVisitParam.h", + "Message": "The 'true' value is implicitly cast to the integer type." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'FunctionsVisitParam.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionAbs.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArray.cpp", + "Message": "A code fragment from 'registerFunctionArray.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayConcat.cpp", + "Message": "A code fragment from 'registerFunctionArrayConcat.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayElement.cpp", + "Message": "A code fragment from 'registerFunctionArrayElement.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayEnumerate.cpp", + "Message": "A code fragment from 'registerFunctionArrayEnumerate.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayHasAll.cpp", + "Message": "A code fragment from 'registerFunctionArrayHasAll.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayEnumerateUniq.cpp", + "Message": "A code fragment from 'registerFunctionArrayEnumerateUniq.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayHasAny.cpp", + "Message": "A code fragment from 'registerFunctionArrayHasAny.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayIntersect.cpp", + "Message": "A code fragment from 'registerFunctionArrayIntersect.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayPopBack.cpp", + "Message": "A code fragment from 'registerFunctionArrayPopBack.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayPopFront.cpp", + "Message": "A code fragment from 'registerFunctionArrayPopFront.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayPushFront.cpp", + "Message": "A code fragment from 'registerFunctionArrayPushFront.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayPushBack.cpp", + "Message": "A code fragment from 'registerFunctionArrayPushBack.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayReduce.cpp", + "Message": "A code fragment from 'registerFunctionArrayReduce.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayReverse.cpp", + "Message": "A code fragment from 'registerFunctionArrayReverse.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArraySlice.cpp", + "Message": "A code fragment from 'registerFunctionArraySlice.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayResize.cpp", + "Message": "A code fragment from 'registerFunctionArrayResize.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionArrayUniq.cpp", + "Message": "A code fragment from 'registerFunctionArrayUniq.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionBitAnd.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionBitNot.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionBitRotateRight.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionBitOr.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionBitRotateLeft.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionBitShiftLeft.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionBitTest.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionBitShiftRight.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionBitTestAny.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionBitXor.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionBitTestAll.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'FunctionsRound.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionCountEqual.cpp", + "Message": "A code fragment from 'registerFunctionCountEqual.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionDivideFloating.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionBuildProjectionComposition.cpp", + "Message": "A code fragment from 'registerFunctionBuildProjectionComposition.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionDivideIntegral.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionDivideIntegralOrZero.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionEmptyArrayDate.cpp", + "Message": "A code fragment from 'registerFunctionEmptyArrayDate.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionEmptyArrayDateTime.cpp", + "Message": "A code fragment from 'registerFunctionEmptyArrayDateTime.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionEmptyArrayFloat32.cpp", + "Message": "A code fragment from 'registerFunctionEmptyArrayFloat_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionEmptyArrayInt16.cpp", + "Message": "A code fragment from 'registerFunctionEmptyArrayInt_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionEmptyArrayFloat64.cpp", + "Message": "A code fragment from 'registerFunctionEmptyArrayFloat_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionEmptyArrayInt64.cpp", + "Message": "A code fragment from 'registerFunctionEmptyArrayInt_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionEmptyArrayInt32.cpp", + "Message": "A code fragment from 'registerFunctionEmptyArrayInt_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionEmptyArrayInt8.cpp", + "Message": "A code fragment from 'registerFunctionEmptyArrayInt_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionEmptyArrayString.cpp", + "Message": "A code fragment from 'registerFunctionEmptyArrayString.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionEmptyArrayToSingle.cpp", + "Message": "A code fragment from 'registerFunctionEmptyArrayToSingle.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionEmptyArrayUInt16.cpp", + "Message": "A code fragment from 'registerFunctionEmptyArrayUInt_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionEmptyArrayUInt32.cpp", + "Message": "A code fragment from 'registerFunctionEmptyArrayUInt_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionEmptyArrayUInt64.cpp", + "Message": "A code fragment from 'registerFunctionEmptyArrayUInt_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionEmptyArrayUInt8.cpp", + "Message": "A code fragment from 'registerFunctionEmptyArrayUInt_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionGCD.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionGreatest.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionHas.cpp", + "Message": "A code fragment from 'registerFunctionHas.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionIndexOf.cpp", + "Message": "A code fragment from 'registerFunctionIndexOf.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionIntExp_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionIntExp_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionLCM.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionLeast.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionMinus.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionModulo.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionMultiply.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionNegate.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionOneOrZero.cpp", + "Message": "A code fragment from 'registerFunctionOneOrZero.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionPlus.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionRange.cpp", + "Message": "A code fragment from 'registerFunctionRange.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionProject.cpp", + "Message": "A code fragment from 'registerFunctionProject.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctions.cpp", + "Message": "A code fragment from 'registerFunctions.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionRestoreProjection.cpp", + "Message": "A code fragment from 'registerFunctionRestoreProjection.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionsArray.cpp", + "Message": "A code fragment from 'registerFunctionsArray.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "FunctionsArithmetic.h", + "Message": "A code fragment from 'registerFunctionsArithmetic.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1835696802, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V669", + "FileName": "concat.cpp", + "Message": "The 'sink' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ArraySourceVisitor.h", + "Message": "A code fragment from 'concat.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerFunctionsProjection.cpp", + "Message": "A code fragment from 'registerFunctionsProjection.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ArraySinkVisitor.h", + "Message": "A code fragment from 'createArraySource.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ArraySinkVisitor.h", + "Message": "A code fragment from 'createArraySink.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ArraySinkVisitor.h", + "Message": "A code fragment from 'createValueSource.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 412830845, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V669", + "FileName": "has.cpp", + "Message": "The 'first', 'second', 'result' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ArraySourceVisitor.h", + "Message": "A code fragment from 'has.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3069352387, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V669", + "FileName": "resizeConstantSize.cpp", + "Message": "The 'array_source', 'value_source', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ArraySourceVisitor.h", + "Message": "A code fragment from 'resizeConstantSize.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 652442793, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V669", + "FileName": "push.cpp", + "Message": "The 'array_source', 'value_source', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ArraySourceVisitor.h", + "Message": "A code fragment from 'push.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3240218462, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V669", + "FileName": "sliceDynamicOffsetBounded.cpp", + "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ArraySourceVisitor.h", + "Message": "A code fragment from 'sliceDynamicOffsetBounded.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1330432147, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V669", + "FileName": "resizeDynamicSize.cpp", + "Message": "The 'array_source', 'value_source', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ArraySourceVisitor.h", + "Message": "A code fragment from 'resizeDynamicSize.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3966500669, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V669", + "FileName": "sliceDynamicOffsetUnbounded.cpp", + "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ArraySourceVisitor.h", + "Message": "A code fragment from 'sliceDynamicOffsetUnbounded.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2727169871, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V669", + "FileName": "sliceFromLeftConstantOffsetBounded.cpp", + "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ArraySourceVisitor.h", + "Message": "A code fragment from 'sliceFromLeftConstantOffsetBounded.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "number_traits.cpp", + "Message": "A code fragment from 'number_traits.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 566054038, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V669", + "FileName": "sliceFromRightConstantOffsetBounded.cpp", + "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ArraySourceVisitor.h", + "Message": "A code fragment from 'sliceFromRightConstantOffsetBounded.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1515238783, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V669", + "FileName": "sliceFromLeftConstantOffsetUnbounded.cpp", + "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ArraySourceVisitor.h", + "Message": "A code fragment from 'sliceFromLeftConstantOffsetUnbounded.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1741194879, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V669", + "FileName": "sliceFromRightConstantOffsetUnbounded.cpp", + "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ArraySourceVisitor.h", + "Message": "A code fragment from 'sliceFromRightConstantOffsetUnbounded.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "hash_map.cpp", + "Message": "A code fragment from 'hash_map.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1419470395, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V524", + "FileName": "hash_map_string_2.cpp", + "Message": "It is odd that the body of 'compare_byIntSSE' function is fully equivalent to the body of 'compareSSE_' function (StringRef.h, line _)." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "hash_map_string_2.cpp", + "Message": "A code fragment from 'hash_map_string__.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "hash_map3.cpp", + "Message": "A code fragment from 'hash_map_.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "select_query.cpp", + "Message": "A code fragment from 'select_query.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "aggregate.cpp", + "Message": "A code fragment from 'aggregate.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "hash_map_string_3.cpp", + "Message": "A code fragment from 'hash_map_string__.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 15386799, + "CodeNext": 0, + "CodePrev": 193061534, + "ErrorCode": "V730", + "FileName": "hash_map_string_small.cpp", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: size." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "hash_map_string_small.cpp", + "Message": "A code fragment from 'hash_map_string_small.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "two_level_hash_map.cpp", + "Message": "A code fragment from 'two_level_hash_map.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "expression.cpp", + "Message": "A code fragment from 'expression.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "compiler_test.cpp", + "Message": "A code fragment from 'compiler_test.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "create_query.cpp", + "Message": "A code fragment from 'create_query.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "logical_expressions_optimizer.cpp", + "Message": "A code fragment from 'logical_expressions_optimizer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "in_join_subqueries_preprocessor.cpp", + "Message": "A code fragment from 'in_join_subqueries_preprocessor.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "hash_map_string.cpp", + "Message": "A code fragment from 'hash_map_string.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "users.cpp", + "Message": "A code fragment from 'users.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionArray.cpp", + "Message": "A code fragment from 'AggregateFunctionArray.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionAvg.cpp", + "Message": "A code fragment from 'AggregateFunctionAvg.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionBitwise.cpp", + "Message": "A code fragment from 'AggregateFunctionBitwise.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionCount.cpp", + "Message": "A code fragment from 'AggregateFunctionCount.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionForEach.cpp", + "Message": "A code fragment from 'AggregateFunctionForEach.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionGroupArray.cpp", + "Message": "A code fragment from 'AggregateFunctionGroupArray.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionGroupArrayInsertAt.cpp", + "Message": "A code fragment from 'AggregateFunctionGroupArrayInsertAt.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionGroupUniqArray.cpp", + "Message": "A code fragment from 'AggregateFunctionGroupUniqArray.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionIf.cpp", + "Message": "A code fragment from 'AggregateFunctionIf.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionMerge.cpp", + "Message": "A code fragment from 'AggregateFunctionMerge.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionMaxIntersections.cpp", + "Message": "A code fragment from 'AggregateFunctionMaxIntersections.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2360632089, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V522", + "FileName": "AggregateFunctionNull.cpp", + "Message": "There might be dereferencing of a potential null pointer 'nested_function'." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionNull.cpp", + "Message": "A code fragment from 'AggregateFunctionNull.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 2106398958, + "CodeNext": 123, + "CodePrev": 123, + "ErrorCode": "V730", + "FileName": "AggregateFunctionMinMaxAny.h", + "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: large_data, small_data." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionMinMaxAny.cpp", + "Message": "A code fragment from 'AggregateFunctionMinMaxAny.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionStatistics.cpp", + "Message": "A code fragment from 'AggregateFunctionStatistics.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionSequenceMatch.cpp", + "Message": "A code fragment from 'AggregateFunctionSequenceMatch.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 707089, + "CodeNext": 3039935502, + "CodePrev": 123, + "ErrorCode": "V550", + "FileName": "QuantileTiming.h", + "Message": "An odd precise comparison: level != _. It's probably better to use a comparison with defined precision: fabs(A - B) > Epsilon." + }, + { + "CodeCurrent": 407249, + "CodeNext": 172219641, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "QuantileTiming.h", + "Message": "The 'count' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionQuantile.cpp", + "Message": "A code fragment from 'AggregateFunctionQuantile.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionSum.cpp", + "Message": "A code fragment from 'AggregateFunctionSum.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionSumMap.cpp", + "Message": "A code fragment from 'AggregateFunctionSumMap.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionTopK.cpp", + "Message": "A code fragment from 'AggregateFunctionTopK.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionUniq.cpp", + "Message": "A code fragment from 'AggregateFunctionUniq.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionUniqUpTo.cpp", + "Message": "A code fragment from 'AggregateFunctionUniqUpTo.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "UniqCombinedBiasData.cpp", + "Message": "A code fragment from 'UniqCombinedBiasData.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerAggregateFunctions.cpp", + "Message": "A code fragment from 'registerAggregateFunctions.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionWindowFunnel.cpp", + "Message": "A code fragment from 'AggregateFunctionWindowFunnel.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "AggregateFunctionsStatisticsSimple.cpp", + "Message": "A code fragment from 'AggregateFunctionsStatisticsSimple.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'ExtractFromConfig.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'Benchmark.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1973016635, + "CodeNext": 0, + "CodePrev": 1667937151, + "ErrorCode": "V688", + "FileName": "ClusterCopier.cpp", + "Message": "The 'pid' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 158502902, + "CodeNext": 73185914, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "ClusterCopier.cpp", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: task_descprtion_current_stat." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'ClusterCopier.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 254105, + "CodeNext": 8134297, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "Client.cpp", + "Message": "The 'query' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 254105, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "Client.cpp", + "Message": "The 'query' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 862425183, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V730", + "FileName": "Client.cpp", + "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: port, security, compression." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'Client.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'HTTPHandler.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "NotFoundHandler.cpp", + "Message": "A code fragment from 'NotFoundHandler.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "InterserverIOHTTPHandler.cpp", + "Message": "A code fragment from 'InterserverIOHTTPHandler.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "MetricsTransmitter.cpp", + "Message": "A code fragment from 'MetricsTransmitter.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "PingRequestHandler.cpp", + "Message": "A code fragment from 'PingRequestHandler.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "RootRequestHandler.cpp", + "Message": "A code fragment from 'RootRequestHandler.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "StatusFile.cpp", + "Message": "A code fragment from 'StatusFile.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ReplicasStatusHandler.cpp", + "Message": "A code fragment from 'ReplicasStatusHandler.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Server.cpp", + "Message": "A code fragment from 'Server.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 1810641279, + "CodeNext": 2826637927, + "CodePrev": 123, + "ErrorCode": "V1002", + "FileName": "TCPHandler.cpp", + "Message": "The 'Context' class, containing pointers, constructor and destructor, is copied by the automatically generated operator=." + }, + { + "CodeCurrent": 2826637927, + "CodeNext": 0, + "CodePrev": 1810641279, + "ErrorCode": "V678", + "FileName": "TCPHandler.cpp", + "Message": "An object is used as an argument to its own method. Consider checking the first actual argument of the 'setSessionContext' function." + }, + { + "CodeCurrent": 143231, + "CodeNext": 0, + "CodePrev": 125, + "ErrorCode": "V565", + "FileName": "TCPHandler.cpp", + "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." + }, + { + "CodeCurrent": 4294129027, + "CodeNext": 0, + "CodePrev": 4133442322, + "ErrorCode": "V1002", + "FileName": "TCPHandler.cpp", + "Message": "The 'Context' class, containing pointers, constructor and destructor, is copied by the automatically generated operator=." + }, + { + "CodeCurrent": 1787830936, + "CodeNext": 123, + "CodePrev": 125, + "ErrorCode": "V759", + "FileName": "TCPHandler.cpp", + "Message": "Incorrect order of exception handling blocks. Exception of 'NetException' type will be caught in a catch block for 'Exception' base type." + }, + { + "CodeCurrent": 123, + "CodeNext": 2434086205, + "CodePrev": 35837, + "ErrorCode": "V565", + "FileName": "TCPHandler.cpp", + "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'TCPHandler.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3472545402, + "CodeNext": 123, + "CodePrev": 125, + "ErrorCode": "V688", + "FileName": "PerformanceTest.cpp", + "Message": "The 'padding' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 188108971, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V550", + "FileName": "PerformanceTest.cpp", + "Message": "An odd precise comparison: avg_speed_first == _. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." + }, + { + "CodeCurrent": 876333961, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "PerformanceTest.cpp", + "Message": "The 'input_files' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2173845642, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "PerformanceTest.cpp", + "Message": "The 'substitutions' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 596635228, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "PerformanceTest.cpp", + "Message": "The 'substitutions' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 1500859456, + "CodeNext": 0, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "PerformanceTest.cpp", + "Message": "The 'queries' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3001731668, + "CodeNext": 3796428353, + "CodePrev": 388618255, + "ErrorCode": "V688", + "FileName": "PerformanceTest.cpp", + "Message": "The 'queries' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 2531644379, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "PerformanceTest.cpp", + "Message": "The 'main_metric' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'PerformanceTest.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'Compressor.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 35857945, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "LocalServer.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 201131425, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "LocalServer.cpp", + "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'LocalServer.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'Format.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TableFunctionCatBoostPool.cpp", + "Message": "A code fragment from 'TableFunctionCatBoostPool.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3790144045, + "CodeNext": 100, + "CodePrev": 123, + "ErrorCode": "V601", + "FileName": "main.cpp", + "Message": "The 'false' value is implicitly cast to the integer type. Inspect the second argument." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "main.cpp", + "Message": "A code fragment from 'main.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'TableFunctionFile.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TableFunctionMerge.cpp", + "Message": "A code fragment from 'TableFunctionMerge.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TableFunctionNumbers.cpp", + "Message": "A code fragment from 'TableFunctionNumbers.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TableFunctionMySQL.cpp", + "Message": "A code fragment from 'TableFunctionMySQL.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "registerTableFunctions.cpp", + "Message": "A code fragment from 'registerTableFunctions.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TableFunctionODBC.cpp", + "Message": "A code fragment from 'TableFunctionODBC.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "translate_positional_arguments.cpp", + "Message": "A code fragment from 'translate_positional_arguments.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TableFunctionRemote.cpp", + "Message": "A code fragment from 'TableFunctionRemote.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "TableFunctionShardByHash.cpp", + "Message": "A code fragment from 'TableFunctionShardByHash.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "collect_aliases.cpp", + "Message": "A code fragment from 'collect_aliases.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "analyze_result_of_query.cpp", + "Message": "A code fragment from 'analyze_result_of_query.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "type_and_constant_inference.cpp", + "Message": "A code fragment from 'type_and_constant_inference.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "analyze_columns.cpp", + "Message": "A code fragment from 'analyze_columns.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "collect_tables.cpp", + "Message": "A code fragment from 'collect_tables.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "Lock.cpp", + "Message": "A code fragment from 'Lock.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "analyze_lambdas.cpp", + "Message": "A code fragment from 'analyze_lambdas.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "optimize_group_order_limit_by.cpp", + "Message": "A code fragment from 'optimize_group_order_limit_by.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ZooKeeperHolder.cpp", + "Message": "A code fragment from 'ZooKeeperHolder.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ZooKeeperNodeCache.cpp", + "Message": "A code fragment from 'ZooKeeperNodeCache.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 4277102732, + "CodeNext": 3079148690, + "CodePrev": 123, + "ErrorCode": "V688", + "FileName": "ZooKeeperImpl.cpp", + "Message": "The 'out' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 20953377, + "CodeNext": 586135571, + "CodePrev": 4018551741, + "ErrorCode": "V688", + "FileName": "ZooKeeperImpl.cpp", + "Message": "The 'session_id' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 6647, + "CodeNext": 81911, + "CodePrev": 325699, + "ErrorCode": "V688", + "FileName": "ZooKeeperImpl.cpp", + "Message": "The 'xid' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 6647, + "CodeNext": 81911, + "CodePrev": 325699, + "ErrorCode": "V688", + "FileName": "ZooKeeperImpl.cpp", + "Message": "The 'xid' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 163059, + "CodeNext": 0, + "CodePrev": 18353, + "ErrorCode": "V688", + "FileName": "ZooKeeperImpl.cpp", + "Message": "The 'error' local variable possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ZooKeeperImpl.cpp", + "Message": "A code fragment from 'ZooKeeperImpl.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "zookeeper_impl.cpp", + "Message": "A code fragment from 'zookeeper_impl.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 3719570876, + "CodeNext": 67617425, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ZooKeeper.cpp", + "Message": "The 'hosts' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3719570876, + "CodeNext": 67617425, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ZooKeeper.cpp", + "Message": "The 'identity' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 67617425, + "CodeNext": 123, + "CodePrev": 3719570876, + "ErrorCode": "V688", + "FileName": "ZooKeeper.cpp", + "Message": "The 'session_timeout_ms' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 67617425, + "CodeNext": 123, + "CodePrev": 3719570876, + "ErrorCode": "V688", + "FileName": "ZooKeeper.cpp", + "Message": "The 'chroot' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 1028915392, + "CodeNext": 2727151059, + "CodePrev": 0, + "ErrorCode": "V560", + "FileName": "ZooKeeper.cpp", + "Message": "A part of conditional expression is always false." + }, + { + "CodeCurrent": 1028915392, + "CodeNext": 2727151059, + "CodePrev": 0, + "ErrorCode": "V560", + "FileName": "ZooKeeper.cpp", + "Message": "A part of conditional expression is always false." + }, + { + "CodeCurrent": 1530511028, + "CodeNext": 3013342653, + "CodePrev": 1169372895, + "ErrorCode": "V560", + "FileName": "ZooKeeper.cpp", + "Message": "A part of conditional expression is always false." + }, + { + "CodeCurrent": 3013342653, + "CodeNext": 2123395508, + "CodePrev": 1530511028, + "ErrorCode": "V560", + "FileName": "ZooKeeper.cpp", + "Message": "A part of conditional expression is always false." + }, + { + "CodeCurrent": 2123395508, + "CodeNext": 2727151059, + "CodePrev": 3013342653, + "ErrorCode": "V560", + "FileName": "ZooKeeper.cpp", + "Message": "A part of conditional expression is always false." + }, + { + "CodeCurrent": 3807645422, + "CodeNext": 5607, + "CodePrev": 0, + "ErrorCode": "V560", + "FileName": "ZooKeeper.cpp", + "Message": "A part of conditional expression is always false." + }, + { + "CodeCurrent": 1530511028, + "CodeNext": 3013328497, + "CodePrev": 1169372895, + "ErrorCode": "V560", + "FileName": "ZooKeeper.cpp", + "Message": "A part of conditional expression is always false." + }, + { + "CodeCurrent": 3013328497, + "CodeNext": 1827075214, + "CodePrev": 1530511028, + "ErrorCode": "V560", + "FileName": "ZooKeeper.cpp", + "Message": "A part of conditional expression is always false." + }, + { + "CodeCurrent": 1827075214, + "CodeNext": 2727151059, + "CodePrev": 3013328497, + "ErrorCode": "V560", + "FileName": "ZooKeeper.cpp", + "Message": "A part of conditional expression is always false." + }, + { + "CodeCurrent": 1028915392, + "CodeNext": 2727151059, + "CodePrev": 0, + "ErrorCode": "V560", + "FileName": "ZooKeeper.cpp", + "Message": "A part of conditional expression is always false." + }, + { + "CodeCurrent": 1028915392, + "CodeNext": 2727151059, + "CodePrev": 0, + "ErrorCode": "V560", + "FileName": "ZooKeeper.cpp", + "Message": "A part of conditional expression is always false." + }, + { + "CodeCurrent": 1530511028, + "CodeNext": 3013328526, + "CodePrev": 1169372895, + "ErrorCode": "V560", + "FileName": "ZooKeeper.cpp", + "Message": "A part of conditional expression is always false." + }, + { + "CodeCurrent": 3013328526, + "CodeNext": 2727151059, + "CodePrev": 1530511028, + "ErrorCode": "V560", + "FileName": "ZooKeeper.cpp", + "Message": "A part of conditional expression is always false." + }, + { + "CodeCurrent": 3365467161, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ZooKeeper.cpp", + "Message": "The 'code' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 3365467161, + "CodeNext": 123, + "CodePrev": 0, + "ErrorCode": "V688", + "FileName": "ZooKeeper.cpp", + "Message": "The 'responses' function argument possesses the same name as one of the class members, which can result in a confusion." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'ZooKeeper.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "zkutil_expiration_test.cpp", + "Message": "A code fragment from 'zkutil_expiration_test.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "zkutil_zookeeper_holder.cpp", + "Message": "A code fragment from 'zkutil_zookeeper_holder.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "zkutil_test_lock.cpp", + "Message": "A code fragment from 'zkutil_test_lock.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "zkutil_test_commands.cpp", + "Message": "A code fragment from 'zkutil_test_commands.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "zk_many_watches_reconnect.cpp", + "Message": "A code fragment from 'zk_many_watches_reconnect.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ConfigProcessor.cpp", + "Message": "A code fragment from 'ConfigProcessor.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "zkutil_test_async.cpp", + "Message": "A code fragment from 'zkutil_test_async.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "variant", + "Message": "A code fragment from 'zkutil_test_commands_new_lib.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "dictionary_library.cpp", + "Message": "A code fragment from 'dictionary_library.cpp' cannot be analyzed." + }, + { + "CodeCurrent": 0, + "CodeNext": 0, + "CodePrev": 0, + "ErrorCode": "V001", + "FileName": "ConfigReloader.cpp", + "Message": "A code fragment from 'ConfigReloader.cpp' cannot be analyzed." + } + ] +} From e085bc981691e42a168b31722eab447f963fcd4b Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Tue, 27 Nov 2018 20:07:10 +0300 Subject: [PATCH 22/69] adapt code of #3609 [#CLICKHOUSE-3859] --- dbms/src/Storages/IStorage.h | 41 ++++++++------- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 + dbms/src/Storages/MergeTree/MergeTreeData.h | 14 ++++-- dbms/src/Storages/StorageMergeTree.cpp | 30 ----------- dbms/src/Storages/StorageMergeTree.h | 22 ++++---- .../Storages/StorageReplicatedMergeTree.cpp | 30 ----------- .../src/Storages/StorageReplicatedMergeTree.h | 19 ++++--- .../Storages/System/StorageSystemColumns.cpp | 30 +++++------ .../Storages/System/StorageSystemTables.cpp | 14 +++--- ...system_columns_and_system_tables.reference | 50 +++++++++---------- ...00753_system_columns_and_system_tables.sql | 12 ++--- 11 files changed, 106 insertions(+), 158 deletions(-) diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index dec7757ebbf..429c6e66914 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -326,30 +326,33 @@ public: /// Returns data path if storage supports it, empty string otherwise. virtual String getDataPath() const { return {}; } - /// Returns additional columns that need to be read for sampling to work. + /// Returns ASTExpressionList of partition key expression for storage or nullptr if there is none. + virtual ASTPtr getPartitionKeyAST() const { return nullptr; } + + /// Returns ASTExpressionList of sorting key expression for storage or nullptr if there is none. + virtual ASTPtr getSortingKeyAST() const { return nullptr; } + + /// Returns ASTExpressionList of primary key expression for storage or nullptr if there is none. + virtual ASTPtr getPrimaryKeyAST() const { return nullptr; } + + /// Returns sampling expression AST for storage or nullptr if there is none. + virtual ASTPtr getSamplingKeyAST() const { return nullptr; } + + /// Returns additional columns that need to be read to calculate partition key. + virtual Names getColumnsRequiredForPartitionKey() const { return {}; } + + /// Returns additional columns that need to be read to calculate sorting key. + virtual Names getColumnsRequiredForSortingKey() const { return {}; } + + /// Returns additional columns that need to be read to calculate primary key. + virtual Names getColumnsRequiredForPrimaryKey() const { return {}; } + + /// Returns additional columns that need to be read to calculate sampling key. virtual Names getColumnsRequiredForSampling() const { return {}; } /// Returns additional columns that need to be read for FINAL to work. virtual Names getColumnsRequiredForFinal() const { return {}; } - /// Returns partition expression for storage or nullptr if there is no. - virtual ASTPtr getPartitionExpression() const { return nullptr; } - - /// Returns secondary expression for storage or nullptr if there is no. - virtual ASTPtr getOrderExpression() const { return nullptr; } - - /// Returns sampling key names for storage or empty vector if there is no. - virtual Names getSamplingExpressionNames() const { return {}; } - - /// Returns primary key names for storage or empty vector if there is no. - virtual Names getPrimaryExpressionNames() const { return {}; } - - /// Returns partition key names for storage or empty vector if there is no. - virtual Names getPartitionExpressionNames() const { return {}; } - - /// Returns order key names for storage or empty vector if there is no. - virtual Names getOrderExpressionNames() const { return {}; } - using ITableDeclaration::ITableDeclaration; using std::enable_shared_from_this::shared_from_this; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 8f46e4a8682..938611f0bbb 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -331,10 +331,12 @@ void MergeTreeData::setPrimaryKeyAndColumns( order_by_ast = new_order_by_ast; sorting_key_columns = std::move(new_sorting_key_columns); + sorting_key_expr_ast = std::move(new_sorting_key_expr_list); sorting_key_expr = std::move(new_sorting_key_expr); primary_key_ast = new_primary_key_ast; primary_key_columns = std::move(new_primary_key_columns); + primary_key_expr_ast = std::move(new_primary_key_expr_list); primary_key_expr = std::move(new_primary_key_expr); primary_key_sample = std::move(new_primary_key_sample); primary_key_data_types = std::move(new_primary_key_data_types); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 14abec14b6c..47c97ba3527 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -503,10 +503,16 @@ public: */ static ASTPtr extractKeyExpressionList(const ASTPtr & node); - bool hasPrimaryKey() const { return !primary_key_columns.empty(); } - bool hasSortingKey() const { return !sorting_key_columns.empty(); } + Names getColumnsRequiredForPartitionKey() const { return (partition_key_expr ? partition_key_expr->getRequiredColumns() : Names{}); } - Names getColumnsRequiredForFinal() const { return sorting_key_expr->getRequiredColumns(); } + bool hasSortingKey() const { return !sorting_key_columns.empty(); } + bool hasPrimaryKey() const { return !primary_key_columns.empty(); } + + ASTPtr getSortingKeyAST() const { return sorting_key_expr_ast; } + ASTPtr getPrimaryKeyAST() const { return primary_key_expr_ast; } + + Names getColumnsRequiredForSortingKey() const { return sorting_key_expr->getRequiredColumns(); } + Names getColumnsRequiredForPrimaryKey() const { return primary_key_expr->getRequiredColumns(); } bool supportsSampling() const { return sample_by_ast != nullptr; } ASTPtr getSamplingExpression() const { return sample_by_ast; } @@ -575,10 +581,12 @@ public: /// Names of columns for primary key + secondary sorting columns. Names sorting_key_columns; + ASTPtr sorting_key_expr_ast; ExpressionActionsPtr sorting_key_expr; /// Names of columns for primary key. Names primary_key_columns; + ASTPtr primary_key_expr_ast; ExpressionActionsPtr primary_key_expr; Block primary_key_sample; DataTypes primary_key_data_types; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 7b154d8f0e8..62b142ab413 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -990,34 +990,4 @@ ActionLock StorageMergeTree::getActionLock(StorageActionBlockType action_type) return {}; } -Names StorageMergeTree::getSamplingExpressionNames() const -{ - NameOrderedSet names; - const auto & expr = data.sampling_expression; - if (expr) - expr->collectIdentifierNames(names); - - return Names(names.begin(), names.end()); -} - -Names StorageMergeTree::getPrimaryExpressionNames() const -{ - return data.getPrimarySortColumns(); -} - -Names StorageMergeTree::getPartitionExpressionNames() const -{ - NameOrderedSet names; - const auto & expr = data.partition_expr_ast; - if (expr) - expr->collectIdentifierNames(names); - - return Names(names.cbegin(), names.cend()); -} - -Names StorageMergeTree::getOrderExpressionNames() const -{ - return data.getSortColumns(); -} - } diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 4bb17b3e3b0..8c325505954 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -84,21 +84,17 @@ public: String getDataPath() const override { return full_path; } + ASTPtr getPartitionKeyAST() const override { return data.partition_by_ast; } + ASTPtr getSortingKeyAST() const override { return data.getSortingKeyAST(); } + ASTPtr getPrimaryKeyAST() const override { return data.getPrimaryKeyAST(); } + ASTPtr getSamplingKeyAST() const override { return data.getSamplingExpression(); } + + Names getColumnsRequiredForPartitionKey() const override { return data.getColumnsRequiredForPartitionKey(); } + Names getColumnsRequiredForSortingKey() const override { return data.getColumnsRequiredForSortingKey(); } + Names getColumnsRequiredForPrimaryKey() const override { return data.getColumnsRequiredForPrimaryKey(); } Names getColumnsRequiredForSampling() const override { return data.getColumnsRequiredForSampling(); } + Names getColumnsRequiredForFinal() const override { return data.getColumnsRequiredForSortingKey(); } - Names getColumnsRequiredForFinal() const override { return data.getColumnsRequiredForFinal(); } - - ASTPtr getPartitionExpression() const override { return data.partition_expr_ast; } - - ASTPtr getOrderExpression() const override { return data.secondary_sort_expr_ast; } - - Names getSamplingExpressionNames() const override; - - Names getPrimaryExpressionNames() const override; - - Names getPartitionExpressionNames() const override; - - Names getOrderExpressionNames() const override; private: String path; String database_name; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 0a64663686a..62a20bcfb74 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4895,34 +4895,4 @@ bool StorageReplicatedMergeTree::dropPartsInPartition( return true; } -Names StorageReplicatedMergeTree::getSamplingExpressionNames() const -{ - NameOrderedSet names; - const auto & expr = data.sampling_expression; - if (expr) - expr->collectIdentifierNames(names); - - return Names(names.begin(), names.end()); -} - -Names StorageReplicatedMergeTree::getPrimaryExpressionNames() const -{ - return data.getPrimarySortColumns(); -} - -Names StorageReplicatedMergeTree::getOrderExpressionNames() const -{ - return data.getSortColumns(); -} - -Names StorageReplicatedMergeTree::getPartitionExpressionNames() const -{ - NameOrderedSet names; - const auto & expr = data.partition_expr_ast; - if (expr) - expr->collectIdentifierNames(names); - - return Names(names.cbegin(), names.cend()); -} - } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 6b3889e3e9b..6614c7c1b44 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -189,17 +189,16 @@ public: String getDataPath() const override { return full_path; } + ASTPtr getPartitionKeyAST() const override { return data.partition_by_ast; } + ASTPtr getSortingKeyAST() const override { return data.getSortingKeyAST(); } + ASTPtr getPrimaryKeyAST() const override { return data.getPrimaryKeyAST(); } + ASTPtr getSamplingKeyAST() const override { return data.getSamplingExpression(); } + + Names getColumnsRequiredForPartitionKey() const override { return data.getColumnsRequiredForPartitionKey(); } + Names getColumnsRequiredForSortingKey() const override { return data.getColumnsRequiredForSortingKey(); } + Names getColumnsRequiredForPrimaryKey() const override { return data.getColumnsRequiredForPrimaryKey(); } Names getColumnsRequiredForSampling() const override { return data.getColumnsRequiredForSampling(); } - - Names getColumnsRequiredForFinal() const override { return data.getColumnsRequiredForFinal(); } - - Names getSamplingExpressionNames() const override; - - Names getPrimaryExpressionNames() const override; - - Names getOrderExpressionNames() const override; - - Names getPartitionExpressionNames() const override; + Names getColumnsRequiredForFinal() const override { return data.getColumnsRequiredForSortingKey(); } private: /// Delete old parts from disk and from ZooKeeper. diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index 91c514dba4b..98d322b1f70 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -37,10 +37,10 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_) { "data_uncompressed_bytes", std::make_shared() }, { "marks_bytes", std::make_shared() }, { "comment", std::make_shared() }, - { "is_in_primary_key", std::make_shared() }, - { "is_in_order_key", std::make_shared() }, { "is_in_partition_key", std::make_shared() }, - { "is_in_sample_key", std::make_shared() }, + { "is_in_sorting_key", std::make_shared() }, + { "is_in_primary_key", std::make_shared() }, + { "is_in_sampling_key", std::make_shared() }, })); } @@ -86,10 +86,10 @@ protected: NamesAndTypesList columns; ColumnDefaults column_defaults; ColumnComments column_comments; - Names partition_key_names; - Names order_key_names; - Names primary_key_names; - Names sampling_key_names; + Names cols_required_for_partition_key; + Names cols_required_for_sorting_key; + Names cols_required_for_primary_key; + Names cols_required_for_sampling; MergeTreeData::ColumnSizeByName column_sizes; { @@ -117,10 +117,10 @@ protected: column_defaults = storage->getColumns().defaults; column_comments = storage->getColumns().comments; - partition_key_names = storage->getPartitionExpressionNames(); - order_key_names = storage->getOrderExpressionNames(); - primary_key_names = storage->getPrimaryExpressionNames(); - sampling_key_names = storage->getSamplingExpressionNames(); + cols_required_for_partition_key = storage->getColumnsRequiredForPartitionKey(); + cols_required_for_sorting_key = storage->getColumnsRequiredForSortingKey(); + cols_required_for_primary_key = storage->getColumnsRequiredForPrimaryKey(); + cols_required_for_sampling = storage->getColumnsRequiredForSampling(); /** Info about sizes of columns for tables of MergeTree family. * NOTE: It is possible to add getter for this info to IStorage interface. @@ -210,13 +210,13 @@ protected: }; if (columns_mask[src_index++]) - res_columns[res_index++]->insert(find_in_vector(primary_key_names)); + res_columns[res_index++]->insert(find_in_vector(cols_required_for_partition_key)); if (columns_mask[src_index++]) - res_columns[res_index++]->insert(find_in_vector(order_key_names)); + res_columns[res_index++]->insert(find_in_vector(cols_required_for_sorting_key)); if (columns_mask[src_index++]) - res_columns[res_index++]->insert(find_in_vector(partition_key_names)); + res_columns[res_index++]->insert(find_in_vector(cols_required_for_primary_key)); if (columns_mask[src_index++]) - res_columns[res_index++]->insert(find_in_vector(sampling_key_names)); + res_columns[res_index++]->insert(find_in_vector(cols_required_for_sampling)); } ++rows_count; diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index 7a09f0aa30c..3fb22f8db66 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -40,10 +40,10 @@ StorageSystemTables::StorageSystemTables(const std::string & name_) {"dependencies_table", std::make_shared(std::make_shared())}, {"create_table_query", std::make_shared()}, {"engine_full", std::make_shared()}, - {"primary_key", std::make_shared()}, - {"order_key", std::make_shared()}, {"partition_key", std::make_shared()}, - {"sample_key", std::make_shared()}, + {"sorting_key", std::make_shared()}, + {"primary_key", std::make_shared()}, + {"sampling_key", std::make_shared()}, })); } @@ -257,7 +257,7 @@ protected: ASTPtr expression_ptr; if (columns_mask[src_index++]) { - if ((expression_ptr = table_it->getPrimaryExpression())) + if ((expression_ptr = table_it->getPartitionKeyAST())) res_columns[res_index++]->insert(queryToString(expression_ptr)); else res_columns[res_index++]->insertDefault(); @@ -265,7 +265,7 @@ protected: if (columns_mask[src_index++]) { - if ((expression_ptr = table_it->getOrderExpression())) + if ((expression_ptr = table_it->getSortingKeyAST())) res_columns[res_index++]->insert(queryToString(expression_ptr)); else res_columns[res_index++]->insertDefault(); @@ -273,7 +273,7 @@ protected: if (columns_mask[src_index++]) { - if ((expression_ptr = table_it->getPartitionExpression())) + if ((expression_ptr = table_it->getPrimaryKeyAST())) res_columns[res_index++]->insert(queryToString(expression_ptr)); else res_columns[res_index++]->insertDefault(); @@ -281,7 +281,7 @@ protected: if (columns_mask[src_index++]) { - if ((expression_ptr = table_it->getSamplingExpression())) + if ((expression_ptr = table_it->getSamplingKeyAST())) res_columns[res_index++]->insert(queryToString(expression_ptr)); else res_columns[res_index++]->insertDefault(); diff --git a/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference b/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference index 934c7ada99a..6868492f7b0 100644 --- a/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference +++ b/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference @@ -1,25 +1,25 @@ -┌─name────────────────┬─primary_key─┬─order_key─┬─partition_key─┬─sample_key─┐ -│ check_system_tables │ name1 │ │ name2 │ name1 │ -└─────────────────────┴─────────────┴───────────┴───────────────┴────────────┘ -┌─name──┬─is_in_primary_key─┬─is_in_order_key─┬─is_in_partition_key─┬─is_in_sample_key─┐ -│ name1 │ 1 │ 1 │ 0 │ 1 │ -│ name2 │ 0 │ 0 │ 1 │ 0 │ -│ name3 │ 0 │ 0 │ 0 │ 0 │ -└───────┴───────────────────┴─────────────────┴─────────────────────┴──────────────────┘ -┌─name────────────────┬─primary_key─┬─order_key─┬─partition_key─┬─sample_key─┐ -│ check_system_tables │ date │ version │ date │ │ -└─────────────────────┴─────────────┴───────────┴───────────────┴────────────┘ -┌─name────┬─is_in_primary_key─┬─is_in_order_key─┬─is_in_partition_key─┬─is_in_sample_key─┐ -│ date │ 1 │ 1 │ 1 │ 0 │ -│ value │ 0 │ 0 │ 0 │ 0 │ -│ version │ 0 │ 1 │ 0 │ 0 │ -│ sign │ 0 │ 0 │ 0 │ 0 │ -└─────────┴───────────────────┴─────────────────┴─────────────────────┴──────────────────┘ -┌─name────────────────┬─primary_key───────────────────────┬─order_key─┬─partition_key───┬─sample_key────────┐ -│ check_system_tables │ Counter, Event, intHash32(UserId) │ │ toYYYYMM(Event) │ intHash32(UserId) │ -└─────────────────────┴───────────────────────────────────┴───────────┴─────────────────┴───────────────────┘ -┌─name────┬─is_in_primary_key─┬─is_in_order_key─┬─is_in_partition_key─┬─is_in_sample_key─┐ -│ Event │ 1 │ 1 │ 1 │ 0 │ -│ UserId │ 0 │ 0 │ 0 │ 1 │ -│ Counter │ 1 │ 1 │ 0 │ 0 │ -└─────────┴───────────────────┴─────────────────┴─────────────────────┴──────────────────┘ +┌─name────────────────┬─partition_key─┬─sorting_key─┬─primary_key─┬─sampling_key─┐ +│ check_system_tables │ name2 │ name1 │ name1 │ name1 │ +└─────────────────────┴───────────────┴─────────────┴─────────────┴──────────────┘ +┌─name──┬─is_in_partition_key─┬─is_in_sorting_key─┬─is_in_primary_key─┬─is_in_sampling_key─┐ +│ name1 │ 0 │ 1 │ 1 │ 1 │ +│ name2 │ 1 │ 0 │ 0 │ 0 │ +│ name3 │ 0 │ 0 │ 0 │ 0 │ +└───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ +┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┐ +│ check_system_tables │ date │ date, version │ date │ │ +└─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘ +┌─name────┬─is_in_partition_key─┬─is_in_sorting_key─┬─is_in_primary_key─┬─is_in_sampling_key─┐ +│ date │ 1 │ 1 │ 1 │ 0 │ +│ value │ 0 │ 0 │ 0 │ 0 │ +│ version │ 0 │ 1 │ 0 │ 0 │ +│ sign │ 0 │ 0 │ 0 │ 0 │ +└─────────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ +┌─name────────────────┬─partition_key───┬─sorting_key───────────────────────┬─primary_key───────────────────────┬─sampling_key──────┐ +│ check_system_tables │ toYYYYMM(Event) │ Counter, Event, intHash32(UserId) │ Counter, Event, intHash32(UserId) │ intHash32(UserId) │ +└─────────────────────┴─────────────────┴───────────────────────────────────┴───────────────────────────────────┴───────────────────┘ +┌─name────┬─is_in_partition_key─┬─is_in_sorting_key─┬─is_in_primary_key─┬─is_in_sampling_key─┐ +│ Event │ 1 │ 1 │ 1 │ 0 │ +│ UserId │ 0 │ 1 │ 1 │ 1 │ +│ Counter │ 0 │ 1 │ 1 │ 0 │ +└─────────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ diff --git a/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql b/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql index 8ceb5f881ee..04b4ce28edf 100644 --- a/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql +++ b/dbms/tests/queries/0_stateless/00753_system_columns_and_system_tables.sql @@ -12,12 +12,12 @@ CREATE TABLE test.check_system_tables PARTITION BY name2 SAMPLE BY name1; -SELECT name, primary_key, order_key, partition_key, sample_key +SELECT name, partition_key, sorting_key, primary_key, sampling_key FROM system.tables WHERE name = 'check_system_tables' FORMAT PrettyCompactNoEscapes; -SELECT name, is_in_primary_key, is_in_order_key, is_in_partition_key, is_in_sample_key +SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key FROM system.columns WHERE table = 'check_system_tables' FORMAT PrettyCompactNoEscapes; @@ -35,12 +35,12 @@ CREATE TABLE test.check_system_tables PARTITION BY date ORDER BY date; -SELECT name, primary_key, order_key, partition_key, sample_key +SELECT name, partition_key, sorting_key, primary_key, sampling_key FROM system.tables WHERE name = 'check_system_tables' FORMAT PrettyCompactNoEscapes; -SELECT name, is_in_primary_key, is_in_order_key, is_in_partition_key, is_in_sample_key +SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key FROM system.columns WHERE table = 'check_system_tables' FORMAT PrettyCompactNoEscapes; @@ -55,12 +55,12 @@ CREATE TABLE test.check_system_tables Counter UInt32 ) ENGINE = MergeTree(Event, intHash32(UserId), (Counter, Event, intHash32(UserId)), 8192); -SELECT name, primary_key, order_key, partition_key, sample_key +SELECT name, partition_key, sorting_key, primary_key, sampling_key FROM system.tables WHERE name = 'check_system_tables' FORMAT PrettyCompactNoEscapes; -SELECT name, is_in_primary_key, is_in_order_key, is_in_partition_key, is_in_sample_key +SELECT name, is_in_partition_key, is_in_sorting_key, is_in_primary_key, is_in_sampling_key FROM system.columns WHERE table = 'check_system_tables' FORMAT PrettyCompactNoEscapes; From ceea7bde2efa79bd43046892ead1b99354882183 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 28 Nov 2018 14:40:59 +0300 Subject: [PATCH 23/69] better names in ZooKeeperNodeCache [#CLICKHOUSE-3859] --- dbms/src/Common/Config/ConfigProcessor.cpp | 6 +++--- .../Common/ZooKeeper/ZooKeeperNodeCache.cpp | 20 +++++++++---------- .../src/Common/ZooKeeper/ZooKeeperNodeCache.h | 8 ++++---- .../ReplicatedMergeTreeAlterThread.cpp | 16 +++++++-------- 4 files changed, 25 insertions(+), 25 deletions(-) diff --git a/dbms/src/Common/Config/ConfigProcessor.cpp b/dbms/src/Common/Config/ConfigProcessor.cpp index b06f6df7022..f48d517466f 100644 --- a/dbms/src/Common/Config/ConfigProcessor.cpp +++ b/dbms/src/Common/Config/ConfigProcessor.cpp @@ -353,12 +353,12 @@ void ConfigProcessor::doIncludesRecursive( XMLDocumentPtr zk_document; auto get_zk_node = [&](const std::string & name) -> const Node * { - zkutil::ZooKeeperNodeCache::GetResult result = zk_node_cache->get(name, zk_changed_event); - if (!result.exists) + zkutil::ZooKeeperNodeCache::ZNode znode = zk_node_cache->get(name, zk_changed_event); + if (!znode.exists) return nullptr; /// Enclose contents into a fake tag to allow pure text substitutions. - zk_document = dom_parser.parseString("" + result.contents + ""); + zk_document = dom_parser.parseString("" + znode.contents + ""); return getRootNode(zk_document.get()); }; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp index f44e897cad9..ff1e4fb5e53 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp @@ -9,7 +9,7 @@ ZooKeeperNodeCache::ZooKeeperNodeCache(GetZooKeeper get_zookeeper_) { } -ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path, EventPtr watch_event) +ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, EventPtr watch_event) { Coordination::WatchCallback watch_callback; if (watch_event) @@ -18,7 +18,7 @@ ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path, return get(path, watch_callback); } -ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path, Coordination::WatchCallback caller_watch_callback) +ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, Coordination::WatchCallback caller_watch_callback) { zkutil::ZooKeeperPtr zookeeper; std::unordered_set invalidated_paths; @@ -28,7 +28,7 @@ ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path, if (!context->zookeeper) { /// Possibly, there was a previous session and it has expired. Clear the cache. - node_cache.clear(); + path_to_cached_znode.clear(); context->zookeeper = get_zookeeper(); } @@ -41,10 +41,10 @@ ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path, throw DB::Exception("Could not get znode: `" + path + "'. ZooKeeper not configured.", DB::ErrorCodes::NO_ZOOKEEPER); for (const auto & invalidated_path : invalidated_paths) - node_cache.erase(invalidated_path); + path_to_cached_znode.erase(invalidated_path); - auto cache_it = node_cache.find(path); - if (cache_it != node_cache.end()) + auto cache_it = path_to_cached_znode.find(path); + if (cache_it != path_to_cached_znode.end()) return cache_it->second; std::weak_ptr weak_context(context); @@ -74,12 +74,12 @@ ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path, caller_watch_callback(response); }; - GetResult result; + ZNode result; result.exists = zookeeper->tryGetWatch(path, result.contents, &result.stat, watch_callback); if (result.exists) { - node_cache.emplace(path, result); + path_to_cached_znode.emplace(path, result); return result; } @@ -88,14 +88,14 @@ ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path, result.exists = zookeeper->existsWatch(path, &result.stat, watch_callback); if (!result.exists) { - node_cache.emplace(path, result); + path_to_cached_znode.emplace(path, result); return result; } /// Node was created between the two previous calls, try again. Watch is already set. result.exists = zookeeper->tryGet(path, result.contents, &result.stat); - node_cache.emplace(path, result); + path_to_cached_znode.emplace(path, result); return result; } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h index 89edd485b88..d47cbcb36fe 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h @@ -37,15 +37,15 @@ public: ZooKeeperNodeCache(const ZooKeeperNodeCache &) = delete; ZooKeeperNodeCache(ZooKeeperNodeCache &&) = default; - struct GetResult + struct ZNode { bool exists = false; std::string contents; Coordination::Stat stat; }; - GetResult get(const std::string & path, EventPtr watch_event); - GetResult get(const std::string & path, Coordination::WatchCallback watch_callback); + ZNode get(const std::string & path, EventPtr watch_event); + ZNode get(const std::string & path, Coordination::WatchCallback watch_callback); private: GetZooKeeper get_zookeeper; @@ -59,7 +59,7 @@ private: std::shared_ptr context; - std::unordered_map node_cache; + std::unordered_map path_to_cached_znode; }; } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp index 1b5630f32ed..3b741fd2bc2 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp @@ -62,16 +62,16 @@ void ReplicatedMergeTreeAlterThread::run() auto zookeeper = storage.getZooKeeper(); String columns_path = storage.zookeeper_path + "/columns"; - auto columns_result = zk_node_cache.get(columns_path, task->getWatchCallback()); - if (!columns_result.exists) + auto columns_znode = zk_node_cache.get(columns_path, task->getWatchCallback()); + if (!columns_znode.exists) throw Exception(columns_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE); - int32_t columns_version = columns_result.stat.version; + int32_t columns_version = columns_znode.stat.version; String metadata_path = storage.zookeeper_path + "/metadata"; - auto metadata_result = zk_node_cache.get(metadata_path, task->getWatchCallback()); - if (!metadata_result.exists) + auto metadata_znode = zk_node_cache.get(metadata_path, task->getWatchCallback()); + if (!metadata_znode.exists) throw Exception(metadata_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE); - int32_t metadata_version = metadata_result.stat.version; + int32_t metadata_version = metadata_znode.stat.version; const bool changed_columns_version = (columns_version != storage.columns_version); const bool changed_metadata_version = (metadata_version != storage.metadata_version); @@ -79,10 +79,10 @@ void ReplicatedMergeTreeAlterThread::run() if (!(changed_columns_version || changed_metadata_version || force_recheck_parts)) return; - const String & columns_str = columns_result.contents; + const String & columns_str = columns_znode.contents; auto columns_in_zk = ColumnsDescription::parse(columns_str); - const String & metadata_str = metadata_result.contents; + const String & metadata_str = metadata_znode.contents; auto metadata_in_zk = ReplicatedMergeTreeTableMetadata::parse(metadata_str); auto metadata_diff = ReplicatedMergeTreeTableMetadata(storage.data).checkAndFindDiff(metadata_in_zk, /* allow_alter = */ true); From 9ae79c60e81a3b8123e17c669bac25c9e581cab9 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 28 Nov 2018 15:06:52 +0300 Subject: [PATCH 24/69] get rid of (ASTPtr *) in AlterCommands::apply [#CLICKHOUSE-3859] --- dbms/src/Storages/AlterCommands.cpp | 46 +++++++++++-------- dbms/src/Storages/AlterCommands.h | 8 ++-- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- dbms/src/Storages/StorageBuffer.cpp | 4 -- dbms/src/Storages/StorageDistributed.cpp | 4 -- dbms/src/Storages/StorageMerge.cpp | 4 -- dbms/src/Storages/StorageMergeTree.cpp | 6 ++- .../Storages/StorageReplicatedMergeTree.cpp | 6 ++- 8 files changed, 42 insertions(+), 38 deletions(-) diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index a06ff876ac9..37ac0ed64a6 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -127,7 +127,7 @@ static bool namesEqual(const String & name_without_dot, const DB::NameAndTypePai return (name_with_dot == name_type.name.substr(0, name_without_dot.length() + 1) || name_without_dot == name_type.name); } -void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr * order_by_ast, ASTPtr * primary_key_ast) const +void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr & order_by_ast, ASTPtr & primary_key_ast) const { if (type == ADD_COLUMN) { @@ -273,33 +273,24 @@ void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr * orde } else if (type == MODIFY_PRIMARY_KEY) { - if (!primary_key_ast || !order_by_ast) - throw Exception("ALTER MODIFY PRIMARY KEY is not supported for this type of tables", - ErrorCodes::BAD_ARGUMENTS); - - if (!(*primary_key_ast)) - *order_by_ast = primary_key; + if (!primary_key_ast) + order_by_ast = primary_key; else - *primary_key_ast = primary_key; + primary_key_ast = primary_key; } else if (type == MODIFY_ORDER_BY) { - if (!primary_key_ast || !order_by_ast) - throw Exception("ALTER MODIFY PRIMARY KEY is not supported for this type of tables", - ErrorCodes::BAD_ARGUMENTS); - - if (!(*primary_key_ast)) + if (!primary_key_ast) { /// Primary and sorting key become independent after this ALTER so we have to /// save the old ORDER BY expression as the new primary key. - *primary_key_ast = (*order_by_ast)->clone(); + primary_key_ast = order_by_ast->clone(); } - *order_by_ast = order_by; + order_by_ast = order_by; } else if (type == COMMENT_COLUMN) { - columns_description.comments[column_name] = comment; } else @@ -316,14 +307,18 @@ bool AlterCommand::is_mutable() const return true; } -void AlterCommands::apply(ColumnsDescription & columns_description, ASTPtr * order_by_ast, ASTPtr * primary_key_ast) const +void AlterCommands::apply(ColumnsDescription & columns_description, ASTPtr & order_by_ast, ASTPtr & primary_key_ast) const { auto new_columns_description = columns_description; + auto new_order_by_ast = order_by_ast; + auto new_primary_key_ast = primary_key_ast; for (const AlterCommand & command : *this) - command.apply(new_columns_description, order_by_ast, primary_key_ast); + command.apply(new_columns_description, new_order_by_ast, new_primary_key_ast); columns_description = std::move(new_columns_description); + order_by_ast = std::move(new_order_by_ast); + primary_key_ast = std::move(new_primary_key_ast); } void AlterCommands::validate(const IStorage & table, const Context & context) @@ -511,6 +506,21 @@ void AlterCommands::validate(const IStorage & table, const Context & context) } } +void AlterCommands::apply(ColumnsDescription & columns_description) const +{ + auto out_columns_description = columns_description; + ASTPtr out_order_by; + ASTPtr out_primary_key; + apply(out_columns_description, out_order_by, out_primary_key); + + if (out_order_by) + throw Exception("Storage doesn't support modifying ORDER BY expression", ErrorCodes::NOT_IMPLEMENTED); + if (out_primary_key) + throw Exception("Storage doesn't support modifying PRIMARY KEY expression", ErrorCodes::NOT_IMPLEMENTED); + + columns_description = std::move(out_columns_description); +} + bool AlterCommands::is_mutable() const { for (const auto & param : *this) diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index f923ace103f..af606aa84ef 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -60,8 +60,7 @@ struct AlterCommand static std::optional parse(const ASTAlterCommand * command); - void apply(ColumnsDescription & columns_description, ASTPtr * order_by_ast = nullptr, ASTPtr * primary_key_ast = nullptr) const; - + void apply(ColumnsDescription & columns_description, ASTPtr & order_by_ast, ASTPtr & primary_key_ast) const; /// Checks that not only metadata touched by that command bool is_mutable() const; }; @@ -72,7 +71,10 @@ class Context; class AlterCommands : public std::vector { public: - void apply(ColumnsDescription & columns_description, ASTPtr * order_by_ast = nullptr, ASTPtr * primary_key_ast = nullptr) const; + void apply(ColumnsDescription & columns_description, ASTPtr & order_by_ast, ASTPtr & primary_key_ast) const; + + /// For storages that don't support MODIFY_PRIMARY_KEY or MODIFY_ORDER_BY. + void apply(ColumnsDescription & columns_description) const; void validate(const IStorage & table, const Context & context); bool is_mutable() const; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 938611f0bbb..283ac71c887 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -983,7 +983,7 @@ void MergeTreeData::checkAlter(const AlterCommands & commands) auto new_columns = getColumns(); ASTPtr new_order_by_ast = order_by_ast; ASTPtr new_primary_key_ast = primary_key_ast; - commands.apply(new_columns, &new_order_by_ast, &new_primary_key_ast); + commands.apply(new_columns, new_order_by_ast, new_primary_key_ast); /// Set of columns that shouldn't be altered. NameSet columns_alter_forbidden; diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 4ec32883e6b..0bf9cee355a 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -630,10 +630,6 @@ void StorageBuffer::flushThread() void StorageBuffer::alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) { - for (const auto & param : params) - if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) - throw Exception("Storage engine " + getName() + " doesn't support primary key.", ErrorCodes::NOT_IMPLEMENTED); - auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); /// So that no blocks of the old structure remain. diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 5d3860f449d..dfd45e43232 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -304,10 +304,6 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr & query, const Setti void StorageDistributed::alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) { - for (const auto & param : params) - if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) - throw Exception("Storage engine " + getName() + " doesn't support primary key.", ErrorCodes::NOT_IMPLEMENTED); - auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); ColumnsDescription new_columns = getColumns(); diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 6fcdb86e861..858812895fb 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -396,10 +396,6 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr void StorageMerge::alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) { - for (const auto & param : params) - if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) - throw Exception("Storage engine " + getName() + " doesn't support primary key.", ErrorCodes::NOT_IMPLEMENTED); - auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); ColumnsDescription new_columns = getColumns(); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 62b142ab413..428454db210 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -208,7 +208,7 @@ void StorageMergeTree::alter( auto new_columns = data.getColumns(); ASTPtr new_order_by_ast = data.order_by_ast; ASTPtr new_primary_key_ast = data.primary_key_ast; - params.apply(new_columns, &new_order_by_ast, &new_primary_key_ast); + params.apply(new_columns, new_order_by_ast, new_primary_key_ast); ASTPtr primary_expr_list_for_altering_parts; for (const AlterCommand & param : params) @@ -715,7 +715,9 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi alter_command.column_name = get(column_name); auto new_columns = getColumns(); - alter_command.apply(new_columns); + ASTPtr ignored_order_by_ast; + ASTPtr ignored_primary_key_ast; + alter_command.apply(new_columns, ignored_order_by_ast, ignored_primary_key_ast); auto columns_for_parts = new_columns.getAllPhysical(); for (const auto & part : parts) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 62a20bcfb74..255be0b7c81 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -1477,7 +1477,9 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & alter_command.column_name = entry.column_name; auto new_columns = getColumns(); - alter_command.apply(new_columns); + ASTPtr ignored_order_by_ast; + ASTPtr ignored_primary_key_ast; + alter_command.apply(new_columns, ignored_order_by_ast, ignored_primary_key_ast); size_t modified_parts = 0; auto parts = data.getDataParts(); @@ -3056,7 +3058,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, ColumnsDescription new_columns = data.getColumns(); ASTPtr new_order_by_ast = data.order_by_ast; ASTPtr new_primary_key_ast = data.primary_key_ast; - params.apply(new_columns, &new_order_by_ast, &new_primary_key_ast); + params.apply(new_columns, new_order_by_ast, new_primary_key_ast); String new_columns_str = new_columns.toString(); if (new_columns_str != data.getColumns().toString()) From f3519bb29781afbfb0a9e00cb2a57218cab37ffe Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 28 Nov 2018 16:31:19 +0300 Subject: [PATCH 25/69] add test for mixing MODIFY PRIMARY KEY and MODIFY ORDER BY statements [#CLICKHOUSE-3859] --- .../00329_alter_primary_key.reference | 7 +++++++ .../0_stateless/00329_alter_primary_key.sql | 17 +++++++++++++++++ 2 files changed, 24 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00329_alter_primary_key.reference b/dbms/tests/queries/0_stateless/00329_alter_primary_key.reference index 015f2ae4a74..2b32543ce55 100644 --- a/dbms/tests/queries/0_stateless/00329_alter_primary_key.reference +++ b/dbms/tests/queries/0_stateless/00329_alter_primary_key.reference @@ -121,3 +121,10 @@ 1 2 3 +*** Check table creation statement *** +CREATE TABLE test.pk2 ( x UInt32, y UInt32, z UInt32) ENGINE = MergeTree PRIMARY KEY (x, y) ORDER BY (x, y, z) SETTINGS index_granularity = 8192 +*** Check that the inserted values were correctly sorted *** +100 20 1 +100 20 2 +100 30 1 +100 30 2 diff --git a/dbms/tests/queries/0_stateless/00329_alter_primary_key.sql b/dbms/tests/queries/0_stateless/00329_alter_primary_key.sql index dbecac04d55..0d0ad6d2f96 100644 --- a/dbms/tests/queries/0_stateless/00329_alter_primary_key.sql +++ b/dbms/tests/queries/0_stateless/00329_alter_primary_key.sql @@ -1,3 +1,5 @@ +SET send_logs_level = 'none'; + DROP TABLE IF EXISTS test.pk; CREATE TABLE test.pk (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 1); @@ -64,3 +66,18 @@ SELECT x, y FROM test.pk WHERE x > 2 AND y > 'z' ORDER BY x, y; SELECT x, y FROM test.pk WHERE y < 'A' ORDER BY x, y; DROP TABLE test.pk; + +DROP TABLE IF EXISTS test.pk2; +CREATE TABLE test.pk2 (x UInt32) ENGINE MergeTree ORDER BY x; + +ALTER TABLE test.pk2 ADD COLUMN y UInt32, ADD COLUMN z UInt32, MODIFY ORDER BY (x, y, z); +ALTER TABLE test.pk2 MODIFY PRIMARY KEY (y); -- { serverError 36 } +ALTER TABLE test.pk2 MODIFY PRIMARY KEY (x, y); +SELECT '*** Check table creation statement ***'; +SHOW CREATE TABLE test.pk2; + +INSERT INTO test.pk2 VALUES (100, 30, 2), (100, 30, 1), (100, 20, 2), (100, 20, 1); +SELECT '*** Check that the inserted values were correctly sorted ***'; +SELECT * FROM test.pk2; + +DROP TABLE test.pk2; From b7cdd8fe7f3b6e9ce4f2889858d8526112493229 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 28 Nov 2018 18:05:28 +0300 Subject: [PATCH 26/69] Trying to simplify merge reading logic --- .../MergeTreeBaseBlockInputStream.cpp | 4 ++-- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 8 +++---- .../MergeTree/MergeTreeDataMergerMutator.cpp | 23 ++++++++----------- dbms/src/Storages/MergeTree/MergeTreeReader.h | 7 +++--- .../MergeTree/MergedBlockOutputStream.cpp | 14 +++++++---- 5 files changed, 27 insertions(+), 29 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp index 158cb64083f..23c14440bbd 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp @@ -183,12 +183,12 @@ Block MergeTreeBaseBlockInputStream::readFromPart() void MergeTreeBaseBlockInputStream::injectVirtualColumns(Block & block) const { - const auto rows = block.rows(); - /// add virtual columns /// Except _sample_factor, which is added from the outside. if (!virt_column_names.empty()) { + const auto rows = block.rows(); + for (const auto & virt_column_name : virt_column_names) { if (virt_column_name == "_part") diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index af8b78c7a01..e326ffb96bb 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -1223,10 +1223,8 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( /// Apply the expression and write the result to temporary files. if (expression) { - MarkRanges ranges{MarkRange(0, part->marks_count)}; - BlockInputStreamPtr part_in = std::make_shared( - *this, part, DEFAULT_MERGE_BLOCK_SIZE, 0, 0, expression->getRequiredColumns(), ranges, - false, nullptr, false, 0, DBMS_DEFAULT_BUFFER_SIZE, false); + BlockInputStreamPtr part_in = std::make_shared( + *this, part, expression->getRequiredColumns(), false); auto compression_settings = this->context.chooseCompressionSettings( part->bytes_on_disk, diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index e413cf25e70..7f8c5de946a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -606,11 +606,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor BlockInputStreams src_streams; UInt64 watch_prev_elapsed = 0; - /// Note: this is dirty hack. MergeTreeBlockInputStream expects minimal amount of bytes after which it will - /// use DIRECT_IO for every peace of data it reads. - /// When we send `min_bytes_when_use_direct_io = 1 (byte)`, it will use O_DIRECT in any case - /// because stream can't read less then single byte - size_t min_bytes_when_use_direct_io = 0; + bool read_with_direct_io = false; if (data.settings.min_merge_bytes_to_use_direct_io != 0) { size_t total_size = 0; @@ -620,7 +616,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (total_size >= data.settings.min_merge_bytes_to_use_direct_io) { LOG_DEBUG(log, "Will merge parts reading files in O_DIRECT"); - min_bytes_when_use_direct_io = 1; + read_with_direct_io = true; + break; } } @@ -629,9 +626,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor for (const auto & part : parts) { - auto input = std::make_unique( - data, part, DEFAULT_MERGE_BLOCK_SIZE, 0, 0, merging_column_names, MarkRanges(1, MarkRange(0, part->marks_count)), - false, nullptr, true, min_bytes_when_use_direct_io, DBMS_DEFAULT_BUFFER_SIZE, false); + auto input = std::make_unique( + data, part, merging_column_names, read_with_direct_io); input->setProgressCallback(MergeProgressCallback( merge_entry, sum_input_rows_upper_bound, column_sizes, watch_prev_elapsed, merge_alg)); @@ -703,7 +699,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor merged_stream = std::make_shared(merged_stream, SizeLimits(), 0 /*limit_hint*/, Names()); MergedBlockOutputStream to{ - data, new_part_tmp_path, merging_columns, compression_settings, merged_column_to_size, min_bytes_when_use_direct_io}; + data, new_part_tmp_path, merging_columns, compression_settings, merged_column_to_size, data.settings.min_merge_bytes_to_use_direct_io}; merged_stream->readPrefix(); to.writePrefix(); @@ -776,9 +772,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor for (size_t part_num = 0; part_num < parts.size(); ++part_num) { - auto column_part_stream = std::make_shared( - data, parts[part_num], DEFAULT_MERGE_BLOCK_SIZE, 0, 0, column_names, MarkRanges{MarkRange(0, parts[part_num]->marks_count)}, - false, nullptr, true, min_bytes_when_use_direct_io, DBMS_DEFAULT_BUFFER_SIZE, false, Names{}, 0, true); + auto column_part_stream = std::make_shared( + data, parts[part_num], column_names, read_with_direct_io); column_part_stream->setProgressCallback(MergeProgressCallbackVerticalStep( merge_entry, sum_input_rows_exact, column_sizes, column_name, watch_prev_elapsed)); diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index 8b165e607b1..dff0dbf2eae 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -53,6 +53,10 @@ public: const NamesAndTypesList & getColumns() const { return columns; } + /// Return the number of rows has been read or zero if there is no columns to read. + /// If continue_reading is true, continue reading from last state, otherwise seek to from_mark + size_t readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Block & res); + private: class Stream { @@ -125,9 +129,6 @@ private: size_t from_mark, bool continue_reading, size_t max_rows_to_read, bool read_offsets = true); - /// Return the number of rows has been read or zero if there is no columns to read. - /// If continue_reading is true, continue reading from last state, otherwise seek to from_mark - size_t readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Block & res); friend class MergeTreeRangeReader::DelayedStream; }; diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index c12ddc51381..36ef333d921 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -256,17 +256,21 @@ MergedBlockOutputStream::MergedBlockOutputStream( columns_list(columns_list_), part_path(part_path_) { init(); - for (const auto & it : columns_list) + + /// If summary size is more than threshold than we will use AIO + size_t total_size = 0; + if (aio_threshold > 0) { - size_t estimated_size = 0; - if (aio_threshold > 0) + for (const auto & it : columns_list) { auto it2 = merged_column_to_size_.find(it.name); if (it2 != merged_column_to_size_.end()) - estimated_size = it2->second; + total_size += it2->second; } - addStreams(part_path, it.name, *it.type, estimated_size, false); } + + for (const auto & it : columns_list) + addStreams(part_path, it.name, *it.type, total_size, false); } std::string MergedBlockOutputStream::getPartPath() const From 23aa6595cdb2efdf2e7255ad51f7c5e66a396cf6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 28 Nov 2018 18:05:53 +0300 Subject: [PATCH 27/69] Missed file --- .../MergeTreeSequentialBlockInputStream.cpp | 122 ++++++++++++++++++ .../MergeTreeSequentialBlockInputStream.h | 70 ++++++++++ 2 files changed, 192 insertions(+) create mode 100644 dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp create mode 100644 dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h diff --git a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp new file mode 100644 index 00000000000..53bbd8de676 --- /dev/null +++ b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp @@ -0,0 +1,122 @@ +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int MEMORY_LIMIT_EXCEEDED; +} + +MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( + const MergeTreeData & storage_, + const MergeTreeData::DataPartPtr & data_part_, + Names columns_to_read_, + bool read_with_direct_io_, + bool quiet) + : storage(storage_) + , data_part(data_part_) + , part_columns_lock(data_part->columns_lock) + , columns_to_read(columns_to_read_) + , read_with_direct_io(read_with_direct_io_) + , mark_cache(storage.context.getMarkCache()) +{ + if (!quiet) + LOG_TRACE(log, "Reading " << data_part->marks_count << " marks from part " << data_part->name + << ", totaly " << data_part->rows_count + << " rows starting from the begging of the part"); + + addTotalRowsApprox(data_part->rows_count); + + header = storage.getSampleBlockForColumns(columns_to_read); + LOG_INFO(log, "Reading columns:" << header.dumpNames()); + + fixHeader(header); + + LOG_INFO(log, "Reading columns(after fix):" << header.dumpNames()); + + reader = std::make_unique( + data_part->getFullPath(), data_part, header.getNamesAndTypesList(), /* uncompressed_cache = */ nullptr, + mark_cache.get(), /* save_marks_in_cache = */ false, storage, + MarkRanges{MarkRange(0, data_part->marks_count)}, + /* bytes to use AIO */ read_with_direct_io ? 1UL : std::numeric_limits::max(), + DBMS_DEFAULT_BUFFER_SIZE); +} + + +void MergeTreeSequentialBlockInputStream::fixHeader(Block & header_block) const +{ + /// Types may be different during ALTER (when this stream is used to perform an ALTER). + /// NOTE: We may use similar code to implement non blocking ALTERs. + for (const auto & name_type : data_part->columns) + { + if (header_block.has(name_type.name)) + { + auto & elem = header_block.getByName(name_type.name); + if (!elem.type->equals(*name_type.type)) + { + elem.type = name_type.type; + elem.column = elem.type->createColumn(); + } + } + } +} + +Block MergeTreeSequentialBlockInputStream::getHeader() const +{ + return header; +} + +Block MergeTreeSequentialBlockInputStream::readImpl() +try +{ + Block res; + if (!isCancelled() && current_row < data_part->rows_count) + { + bool continue_reading = (current_mark != 0); + size_t rows_readed = reader->readRows(current_mark, continue_reading, storage.index_granularity, res); + + res.checkNumberOfRows(); + + current_row += rows_readed; + current_mark += (rows_readed / storage.index_granularity); + bool should_reorder = false, should_evaluate_missing_defaults = false; + LOG_INFO(log, "Block before filling: " << res.dumpStructure()); + reader->fillMissingColumns(res, should_reorder, should_evaluate_missing_defaults, res.rows()); + + if (res && should_evaluate_missing_defaults) + reader->evaluateMissingDefaults(res); + + if (res && should_reorder) + reader->reorderColumns(res, header.getNames(), nullptr); + } + else + { + finish(); + } + + return res; +} +catch (...) +{ + /// Suspicion of the broken part. A part is added to the queue for verification. + if (getCurrentExceptionCode() != ErrorCodes::MEMORY_LIMIT_EXCEEDED) + storage.reportBrokenPart(data_part->name); + throw; +} + + +void MergeTreeSequentialBlockInputStream::finish() +{ + /** Close the files (before destroying the object). + * When many sources are created, but simultaneously reading only a few of them, + * buffers don't waste memory. + */ + reader.reset(); + part_columns_lock.unlock(); + data_part.reset(); +} + + +MergeTreeSequentialBlockInputStream::~MergeTreeSequentialBlockInputStream() = default; + +} diff --git a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h new file mode 100644 index 00000000000..8ebf58d0ea1 --- /dev/null +++ b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h @@ -0,0 +1,70 @@ +#pragma once +#include +#include +#include +#include +#include + +namespace DB +{ + +/// Lightweight (in terms of logic) stream for reading single part in merge process +class MergeTreeSequentialBlockInputStream : public IProfilingBlockInputStream +{ +public: + MergeTreeSequentialBlockInputStream( + const MergeTreeData & storage_, + const MergeTreeData::DataPartPtr & data_part_, + Names columns_to_read_, + bool read_with_direct_io_, + bool quiet = false + ); + + ~MergeTreeSequentialBlockInputStream() override; + + String getName() const override { return "MergeTreeSequentialBlockInputStream"; } + + Block getHeader() const override; + + /// Closes readers and unlock part locks + void finish(); + +protected: + Block readImpl() override; + +private: + + const MergeTreeData & storage; + + Block header; + + /// Data part will not be removed if the pointer owns it + MergeTreeData::DataPartPtr data_part; + + /// Forbids to change columns list of the part during reading + std::shared_lock part_columns_lock; + + /// Columns we have to read (each Block from read will contain them) + Names columns_to_read; + + /// Should read using direct IO + bool read_with_direct_io; + + Logger * log = &Logger::get("MergeTreeSequentialBlockInputStream"); + + std::shared_ptr mark_cache; + using MergeTreeReaderPtr = std::unique_ptr; + MergeTreeReaderPtr reader; + + /// current mark at which we stop reading + size_t current_mark = 0; + + /// current row at which we stop reading + size_t current_row = 0; + +private: + void fixHeader(Block & header_block) const; + +}; + +} From 57f25e978aee339cff78e595cba120b09e9f2a55 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 28 Nov 2018 18:12:44 +0300 Subject: [PATCH 28/69] update comments and docs [#CLICKHOUSE-3859] --- dbms/src/Parsers/ParserCreateQuery.h | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 15 +++++++++++---- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 59a8540b84f..1084f86d1eb 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -199,7 +199,7 @@ protected: }; -/** ENGINE = name [PARTITION BY expr] [ORDER BY expr] [SAMPLE BY expr] [SETTINGS name = value, ...] */ +/** ENGINE = name [PARTITION BY expr] [ORDER BY expr] [PRIMARY KEY expr] [SAMPLE BY expr] [SETTINGS name = value, ...] */ class ParserStorage : public IParserBase { protected: diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index b9203702afe..54b092fdb62 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -245,9 +245,13 @@ A common partitioning expression is some function of the event date column e.g. Rows with different partition expression values are never merged together. That allows manipulating partitions with ALTER commands. Also it acts as a kind of index. -Primary key is specified in the ORDER BY clause. It is mandatory for all MergeTree types. -It is like (CounterID, EventDate, intHash64(UserID)) - a list of column names or functional expressions in round brackets. -If your primary key has just one element, you may omit round brackets. +Sorting key is specified in the ORDER BY clause. It is mandatory for all MergeTree types. +It is like (CounterID, EventDate, intHash64(UserID)) - a list of column names or functional expressions +in round brackets. +If your sorting key has just one element, you may omit round brackets. + +By default primary key is equal to the sorting key. You can specify a primary key that is a prefix of the +sorting key in the PRIMARY KEY clause. Careful choice of the primary key is extremely important for processing short-time queries. @@ -263,6 +267,8 @@ MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate) SETTI MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) SAMPLE BY intHash32(UserID) +MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) PRIMARY KEY (CounterID, EventDate) SAMPLE BY intHash32(UserID) + CollapsingMergeTree(Sign) PARTITION BY StartDate SAMPLE BY intHash32(UserID) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SummingMergeTree PARTITION BY toMonday(EventDate) ORDER BY (OrderID, EventDate, BannerID, PhraseID, ContextType, RegionID, PageID, IsFlat, TypeID, ResourceNo) @@ -323,7 +329,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) * * Alternatively, you can specify: * - Partitioning expression in the PARTITION BY clause; - * - Primary key in the ORDER BY clause; + * - Sorting key in the ORDER BY clause; + * - Primary key (if it is different from the sorting key) in the PRIMARY KEY clause; * - Sampling expression in the SAMPLE BY clause; * - Additional MergeTreeSettings in the SETTINGS clause; */ From 8b5a05947c227f68843aacc82e07890af7958bb1 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Wed, 28 Nov 2018 18:50:52 +0300 Subject: [PATCH 29/69] Remove redundant information from RWLock. And fix broken exception in some race conditions --- dbms/src/Common/RWLock.cpp | 29 ++++++++----------- dbms/src/Common/RWLock.h | 23 ++------------- ...rw_lock_fifo.cpp.cpp => gtest_rw_lock.cpp} | 2 +- .../PushingToViewsBlockOutputStream.cpp | 4 +-- .../Interpreters/InterpreterCreateQuery.cpp | 2 +- .../Interpreters/InterpreterDescribeQuery.cpp | 2 +- .../src/Interpreters/InterpreterDropQuery.cpp | 10 +++---- .../Interpreters/InterpreterInsertQuery.cpp | 2 +- .../Interpreters/InterpreterOptimizeQuery.cpp | 2 +- .../Interpreters/InterpreterRenameQuery.cpp | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 2 +- .../Interpreters/InterpreterSystemQuery.cpp | 2 +- dbms/src/Storages/IStorage.cpp | 6 ++-- dbms/src/Storages/IStorage.h | 23 +++++++-------- .../Storages/MergeTree/DataPartsExchange.cpp | 2 +- .../ReplicatedMergeTreeAlterThread.cpp | 4 +-- .../ReplicatedMergeTreePartCheckThread.cpp | 2 +- dbms/src/Storages/StorageBuffer.cpp | 2 +- dbms/src/Storages/StorageDistributed.cpp | 2 +- dbms/src/Storages/StorageMerge.cpp | 6 ++-- dbms/src/Storages/StorageMergeTree.cpp | 22 +++++++------- dbms/src/Storages/StorageNull.cpp | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 24 +++++++-------- .../Storages/System/StorageSystemColumns.cpp | 2 +- .../System/StorageSystemPartsBase.cpp | 2 +- suppress_base.json | 4 +-- 26 files changed, 81 insertions(+), 104 deletions(-) rename dbms/src/Common/tests/{gtest_rw_lock_fifo.cpp.cpp => gtest_rw_lock.cpp} (98%) diff --git a/dbms/src/Common/RWLock.cpp b/dbms/src/Common/RWLock.cpp index 6890da72812..10a15cd7027 100644 --- a/dbms/src/Common/RWLock.cpp +++ b/dbms/src/Common/RWLock.cpp @@ -53,7 +53,7 @@ public: }; -RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type, RWLockImpl::Client client) +RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type) { Stopwatch watch(CLOCK_MONOTONIC_COARSE); CurrentMetrics::Increment waiting_client_increment((type == Read) ? CurrentMetrics::RWLockWaitingReaders @@ -78,15 +78,16 @@ RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type, RWLockImpl::C { auto handler_ptr = it_handler->second.lock(); - if (!handler_ptr) - throw Exception("Lock handler cannot be nullptr. This is a bug", ErrorCodes::LOGICAL_ERROR); + /// Lock may be released in another thread, but not yet deleted inside |~LogHandlerImpl()| - if (type != Read || handler_ptr->it_group->type != Read) - throw Exception("Attempt to acquire exclusive lock recursively", ErrorCodes::LOGICAL_ERROR); + if (handler_ptr) + { + /// XXX: it means we can't upgrade lock from read to write - with proper waiting! + if (type != Read || handler_ptr->it_group->type != Read) + throw Exception("Attempt to acquire exclusive lock recursively", ErrorCodes::LOGICAL_ERROR); - handler_ptr->it_client->info += "; " + client.info; - - return handler_ptr; + return handler_ptr; + } } if (type == Type::Write || queue.empty() || queue.back().type == Type::Write) @@ -104,7 +105,7 @@ RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type, RWLockImpl::C auto & clients = it_group->clients; try { - it_client = clients.emplace(clients.end(), std::move(client)); + it_client = clients.emplace(clients.end(), type); } catch (...) { @@ -114,10 +115,6 @@ RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type, RWLockImpl::C throw; } - it_client->thread_number = Poco::ThreadNumber::get(); - it_client->enqueue_time = time(nullptr); - it_client->type = type; - LockHandler res(new LockHandlerImpl(shared_from_this(), it_group, it_client)); /// Insert myself (weak_ptr to the handler) to threads set to implement recursive lock @@ -128,7 +125,6 @@ RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type, RWLockImpl::C /// If we are not the first client in the group, a notification could be already sent if (it_group == queue.begin()) { - it_client->start_time = it_client->enqueue_time; finalize_metrics(); return res; } @@ -136,7 +132,6 @@ RWLockImpl::LockHandler RWLockImpl::getLock(RWLockImpl::Type type, RWLockImpl::C /// Wait a notification it_group->cv.wait(lock, [&] () { return it_group == queue.begin(); }); - it_client->start_time = time(nullptr); finalize_metrics(); return res; } @@ -169,8 +164,8 @@ RWLockImpl::LockHandlerImpl::~LockHandlerImpl() RWLockImpl::LockHandlerImpl::LockHandlerImpl(RWLock && parent, RWLockImpl::GroupsContainer::iterator it_group, RWLockImpl::ClientsContainer::iterator it_client) : parent{std::move(parent)}, it_group{it_group}, it_client{it_client}, - active_client_increment{(it_client->type == RWLockImpl::Read) ? CurrentMetrics::RWLockActiveReaders - : CurrentMetrics::RWLockActiveWriters} + active_client_increment{(*it_client == RWLockImpl::Read) ? CurrentMetrics::RWLockActiveReaders + : CurrentMetrics::RWLockActiveWriters} {} } diff --git a/dbms/src/Common/RWLock.h b/dbms/src/Common/RWLock.h index d9937054727..fd95ed48e27 100644 --- a/dbms/src/Common/RWLock.h +++ b/dbms/src/Common/RWLock.h @@ -27,23 +27,6 @@ public: Write, }; -private: - /// Client is that who wants to acquire the lock. - struct Client - { - explicit Client(const std::string & info = {}) : info{info} {} - - bool isStarted() { return start_time != 0; } - - /// TODO: delete extra info below if there is no need fot it already. - std::string info; - int thread_number = 0; - std::time_t enqueue_time = 0; - std::time_t start_time = 0; - Type type = Read; - }; - -public: static RWLock create() { return RWLock(new RWLockImpl); } /// Just use LockHandler::reset() to release the lock @@ -53,21 +36,21 @@ public: /// Waits in the queue and returns appropriate lock - LockHandler getLock(Type type, Client client = Client{}); - LockHandler getLock(Type type, const std::string & who) { return getLock(type, Client(who)); } + LockHandler getLock(Type type); private: RWLockImpl() = default; struct Group; using GroupsContainer = std::list; - using ClientsContainer = std::list; + using ClientsContainer = std::list; using ThreadToHandler = std::map>; /// Group of clients that should be executed concurrently /// i.e. a group could contain several readers, but only one writer struct Group { + // FIXME: there is only redundant |type| information inside |clients|. const Type type; ClientsContainer clients; diff --git a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp b/dbms/src/Common/tests/gtest_rw_lock.cpp similarity index 98% rename from dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp rename to dbms/src/Common/tests/gtest_rw_lock.cpp index 44e053b61a4..5a10c4a0a48 100644 --- a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp +++ b/dbms/src/Common/tests/gtest_rw_lock.cpp @@ -38,7 +38,7 @@ TEST(Common, RWLock_1) auto type = (std::uniform_int_distribution<>(0, 9)(gen) >= round) ? RWLockImpl::Read : RWLockImpl::Write; auto sleep_for = std::chrono::duration(std::uniform_int_distribution<>(1, 100)(gen)); - auto lock = fifo_lock->getLock(type, "RW"); + auto lock = fifo_lock->getLock(type); if (type == RWLockImpl::Write) { diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index f54c62b3579..d3e7eaeb9a2 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -20,7 +20,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( * Although now any insertion into the table is done via PushingToViewsBlockOutputStream, * but it's clear that here is not the best place for this functionality. */ - addTableLock(storage->lockStructure(true, __PRETTY_FUNCTION__)); + addTableLock(storage->lockStructure(true)); /// If the "root" table deduplactes blocks, there are no need to make deduplication for children /// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks @@ -45,7 +45,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( auto & materialized_view = dynamic_cast(*dependent_table); if (StoragePtr inner_table = materialized_view.tryGetTargetTable()) - addTableLock(inner_table->lockStructure(true, __PRETTY_FUNCTION__)); + addTableLock(inner_table->lockStructure(true)); auto query = materialized_view.getInnerQuery(); BlockOutputStreamPtr out = std::make_shared( diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index eac6d2d776b..4dbe0bdfffa 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -547,7 +547,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!as_table_name.empty()) { as_storage = context.getTable(as_database_name, as_table_name); - as_storage_lock = as_storage->lockStructure(false, __PRETTY_FUNCTION__); + as_storage_lock = as_storage->lockStructure(false); } /// Set and retrieve list of columns. diff --git a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp index db33383b76d..b5e62b9b92d 100644 --- a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp @@ -102,7 +102,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() table = context.getTable(database_name, table_name); } - auto table_lock = table->lockStructure(false, __PRETTY_FUNCTION__); + auto table_lock = table->lockStructure(false); columns = table->getColumns().getAll(); column_defaults = table->getColumns().defaults; column_comments = table->getColumns().comments; diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index f2a3f45b55c..2107df35795 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -69,7 +69,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t { database_and_table.second->shutdown(); /// If table was already dropped by anyone, an exception will be thrown - auto table_lock = database_and_table.second->lockForAlter(__PRETTY_FUNCTION__); + auto table_lock = database_and_table.second->lockForAlter(); /// Drop table from memory, don't touch data and metadata database_and_table.first->detachTable(database_and_table.second->getTableName()); } @@ -78,7 +78,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t database_and_table.second->checkTableCanBeDropped(); /// If table was already dropped by anyone, an exception will be thrown - auto table_lock = database_and_table.second->lockForAlter(__PRETTY_FUNCTION__); + auto table_lock = database_and_table.second->lockForAlter(); /// Drop table data, don't touch metadata database_and_table.second->truncate(query_ptr); } @@ -88,7 +88,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t database_and_table.second->shutdown(); /// If table was already dropped by anyone, an exception will be thrown - auto table_lock = database_and_table.second->lockForAlter(__PRETTY_FUNCTION__); + auto table_lock = database_and_table.second->lockForAlter(); /// Delete table metdata and table itself from memory database_and_table.first->removeTable(context, database_and_table.second->getTableName()); /// Delete table data @@ -124,7 +124,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(String & table_name, ASTDr if (kind == ASTDropQuery::Kind::Truncate) { /// If table was already dropped by anyone, an exception will be thrown - auto table_lock = table->lockForAlter(__PRETTY_FUNCTION__); + auto table_lock = table->lockForAlter(); /// Drop table data, don't touch metadata table->truncate(query_ptr); } @@ -133,7 +133,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(String & table_name, ASTDr context_handle.tryRemoveExternalTable(table_name); table->shutdown(); /// If table was already dropped by anyone, an exception will be thrown - auto table_lock = table->lockForAlter(__PRETTY_FUNCTION__); + auto table_lock = table->lockForAlter(); /// Delete table data table->drop(); table->is_dropped = true; diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index f058550a441..444e4eed7aa 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -92,7 +92,7 @@ BlockIO InterpreterInsertQuery::execute() checkAccess(query); StoragePtr table = getTable(query); - auto table_lock = table->lockStructure(true, __PRETTY_FUNCTION__); + auto table_lock = table->lockStructure(true); /// We create a pipeline of several streams, into which we will write data. BlockOutputStreamPtr out; diff --git a/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp b/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp index 80a64d83f90..7d46881539c 100644 --- a/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -23,7 +23,7 @@ BlockIO InterpreterOptimizeQuery::execute() return executeDDLQueryOnCluster(query_ptr, context, {ast.database}); StoragePtr table = context.getTable(ast.database, ast.table); - auto table_lock = table->lockStructure(true, __PRETTY_FUNCTION__); + auto table_lock = table->lockStructure(true); table->optimize(query_ptr, ast.partition, ast.final, ast.deduplicate, context); return {}; } diff --git a/dbms/src/Interpreters/InterpreterRenameQuery.cpp b/dbms/src/Interpreters/InterpreterRenameQuery.cpp index abeb7ff5c1f..74dfe7adecf 100644 --- a/dbms/src/Interpreters/InterpreterRenameQuery.cpp +++ b/dbms/src/Interpreters/InterpreterRenameQuery.cpp @@ -101,7 +101,7 @@ BlockIO InterpreterRenameQuery::execute() for (const auto & names : unique_tables_from) if (auto table = context.tryGetTable(names.database_name, names.table_name)) - locks.emplace_back(table->lockForAlter(__PRETTY_FUNCTION__)); + locks.emplace_back(table->lockForAlter()); /** All tables are locked. If there are more than one rename in chain, * we need to hold global lock while doing all renames. Order matters to avoid deadlocks. diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 2a139edce9f..fe3801149d1 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -182,7 +182,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( } if (storage) - table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__); + table_lock = storage->lockStructure(false); syntax_analyzer_result = SyntaxAnalyzer(context, storage) .analyze(query_ptr, source_header.getNamesAndTypesList(), required_result_column_names, subquery_depth); diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 749827896aa..f91025b11de 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -238,7 +238,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam table->shutdown(); /// If table was already dropped by anyone, an exception will be thrown - auto table_lock = table->lockForAlter(__PRETTY_FUNCTION__); + auto table_lock = table->lockForAlter(); create_ast = system_context.getCreateTableQuery(database_name, table_name); database->detachTable(table_name); diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index 498d486b7a1..c0487b5ef02 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -4,13 +4,13 @@ namespace DB { -TableStructureReadLock::TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data, const std::string & who) +TableStructureReadLock::TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data) : storage(storage_) { if (lock_data) - data_lock = storage->data_lock->getLock(RWLockImpl::Read, who); + data_lock = storage->data_lock->getLock(RWLockImpl::Read); if (lock_structure) - structure_lock = storage->structure_lock->getLock(RWLockImpl::Read, who); + structure_lock = storage->structure_lock->getLock(RWLockImpl::Read); } } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 446f2269f6f..a68e036032e 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -65,7 +65,7 @@ private: RWLockImpl::LockHandler structure_lock; public: - TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data, const std::string & who); + TableStructureReadLock(StoragePtr storage_, bool lock_structure, bool lock_data); }; @@ -115,14 +115,13 @@ public: /** Does not allow you to change the structure or name of the table. * If you change the data in the table, you will need to specify will_modify_data = true. * This will take an extra lock that does not allow starting ALTER MODIFY. - * Parameter 'who' identifies a client of the lock (ALTER query, merge process, etc), used for diagnostic purposes. * * WARNING: You need to call methods from ITableDeclaration under such a lock. Without it, they are not thread safe. * WARNING: To avoid deadlocks, this method must not be called under lock of Context. */ - TableStructureReadLockPtr lockStructure(bool will_modify_data, const std::string & who) + TableStructureReadLockPtr lockStructure(bool will_modify_data) { - TableStructureReadLockPtr res = std::make_shared(shared_from_this(), true, will_modify_data, who); + TableStructureReadLockPtr res = std::make_shared(shared_from_this(), true, will_modify_data); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); return res; @@ -130,11 +129,11 @@ public: /** Does not allow reading the table structure. It is taken for ALTER, RENAME and DROP, TRUNCATE. */ - TableFullWriteLock lockForAlter(const std::string & who = "Alter") + TableFullWriteLock lockForAlter() { /// The calculation order is important. - auto res_data_lock = lockDataForAlter(who); - auto res_structure_lock = lockStructureForAlter(who); + auto res_data_lock = lockDataForAlter(); + auto res_structure_lock = lockStructureForAlter(); return {std::move(res_data_lock), std::move(res_structure_lock)}; } @@ -143,17 +142,17 @@ public: * It is taken during write temporary data in ALTER MODIFY. * Under this lock, you can take lockStructureForAlter() to change the structure of the table. */ - TableDataWriteLock lockDataForAlter(const std::string & who = "Alter") + TableDataWriteLock lockDataForAlter() { - auto res = data_lock->getLock(RWLockImpl::Write, who); + auto res = data_lock->getLock(RWLockImpl::Write); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); return res; } - TableStructureWriteLock lockStructureForAlter(const std::string & who = "Alter") + TableStructureWriteLock lockStructureForAlter() { - auto res = structure_lock->getLock(RWLockImpl::Write, who); + auto res = structure_lock->getLock(RWLockImpl::Write); if (is_dropped) throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED); return res; @@ -243,7 +242,7 @@ public: throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto lock = lockStructureForAlter(); auto new_columns = getColumns(); params.apply(new_columns); context.getDatabase(database_name)->alterTable(context, table_name, new_columns, {}); diff --git a/dbms/src/Storages/MergeTree/DataPartsExchange.cpp b/dbms/src/Storages/MergeTree/DataPartsExchange.cpp index ab1d28ee656..a1ffd45c629 100644 --- a/dbms/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/dbms/src/Storages/MergeTree/DataPartsExchange.cpp @@ -79,7 +79,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo try { - auto storage_lock = owned_storage->lockStructure(false, __PRETTY_FUNCTION__); + auto storage_lock = owned_storage->lockStructure(false); MergeTreeData::DataPartPtr part = findPart(part_name); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp index a3e4dbb7bad..8444b27505e 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp @@ -84,7 +84,7 @@ void ReplicatedMergeTreeAlterThread::run() LOG_INFO(log, "Changed version of 'columns' node in ZooKeeper. Waiting for structure write lock."); - auto table_lock = storage.lockStructureForAlter(__PRETTY_FUNCTION__); + auto table_lock = storage.lockStructureForAlter(); if (columns_in_zk != storage.getColumns()) { @@ -113,7 +113,7 @@ void ReplicatedMergeTreeAlterThread::run() /// Update parts. if (changed_version || force_recheck_parts) { - auto table_lock = storage.lockStructure(false, __PRETTY_FUNCTION__); + auto table_lock = storage.lockStructure(false); if (changed_version) LOG_INFO(log, "ALTER-ing parts"); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index fb7a2ad0730..c115264d393 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -202,7 +202,7 @@ void ReplicatedMergeTreePartCheckThread::checkPart(const String & part_name) else if (part->name == part_name) { auto zookeeper = storage.getZooKeeper(); - auto table_lock = storage.lockStructure(false, __PRETTY_FUNCTION__); + auto table_lock = storage.lockStructure(false); /// If the part is in ZooKeeper, check its data with its checksums, and them with ZooKeeper. if (zookeeper->exists(storage.replica_path + "/parts/" + part_name)) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 4ec32883e6b..2a81e5d1d1e 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -634,7 +634,7 @@ void StorageBuffer::alter(const AlterCommands & params, const String & database_ if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) throw Exception("Storage engine " + getName() + " doesn't support primary key.", ErrorCodes::NOT_IMPLEMENTED); - auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto lock = lockStructureForAlter(); /// So that no blocks of the old structure remain. optimize({} /*query*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context); diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 5d3860f449d..3961c530c53 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -308,7 +308,7 @@ void StorageDistributed::alter(const AlterCommands & params, const String & data if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) throw Exception("Storage engine " + getName() + " doesn't support primary key.", ErrorCodes::NOT_IMPLEMENTED); - auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto lock = lockStructureForAlter(); ColumnsDescription new_columns = getColumns(); params.apply(new_columns); diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 6fcdb86e861..11f7e1deb61 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -344,7 +344,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables() const { auto & table = iterator->table(); if (table.get() != this) - selected_tables.emplace_back(table, table->lockStructure(false, __PRETTY_FUNCTION__)); + selected_tables.emplace_back(table, table->lockStructure(false)); } iterator->next(); @@ -374,7 +374,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr if (storage.get() != this) { virtual_column->insert(storage->getTableName()); - selected_tables.emplace_back(storage, get_lock ? storage->lockStructure(false, __PRETTY_FUNCTION__) : TableStructureReadLockPtr{}); + selected_tables.emplace_back(storage, get_lock ? storage->lockStructure(false) : TableStructureReadLockPtr{}); } } @@ -400,7 +400,7 @@ void StorageMerge::alter(const AlterCommands & params, const String & database_n if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) throw Exception("Storage engine " + getName() + " doesn't support primary key.", ErrorCodes::NOT_IMPLEMENTED); - auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto lock = lockStructureForAlter(); ColumnsDescription new_columns = getColumns(); params.apply(new_columns); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 50d7af076d2..30473705108 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -190,7 +190,7 @@ void StorageMergeTree::alter( { if (!params.is_mutable()) { - auto table_soft_lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto table_soft_lock = lockStructureForAlter(); auto new_columns = getColumns(); params.apply(new_columns); context.getDatabase(database_name)->alterTable(context, table_name, new_columns, {}); @@ -201,7 +201,7 @@ void StorageMergeTree::alter( /// NOTE: Here, as in ReplicatedMergeTree, you can do ALTER which does not block the writing of data for a long time. auto merge_blocker = merger_mutator.actions_blocker.cancel(); - auto table_soft_lock = lockDataForAlter(__PRETTY_FUNCTION__); + auto table_soft_lock = lockDataForAlter(); data.checkAlter(params); @@ -234,7 +234,7 @@ void StorageMergeTree::alter( transactions.push_back(std::move(transaction)); } - auto table_hard_lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto table_hard_lock = lockStructureForAlter(); IDatabase::ASTModifier storage_modifier; if (primary_key_is_modified) @@ -400,7 +400,7 @@ bool StorageMergeTree::merge( bool deduplicate, String * out_disable_reason) { - auto structure_lock = lockStructure(true, __PRETTY_FUNCTION__); + auto structure_lock = lockStructure(true); MergeTreeDataMergerMutator::FuturePart future_part; @@ -505,7 +505,7 @@ bool StorageMergeTree::merge( bool StorageMergeTree::tryMutatePart() { - auto structure_lock = lockStructure(true, __PRETTY_FUNCTION__); + auto structure_lock = lockStructure(true); MergeTreeDataMergerMutator::FuturePart future_part; MutationCommands commands; @@ -705,7 +705,7 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi auto merge_blocker = merger_mutator.actions_blocker.cancel(); /// We don't change table structure, only data in some parts, parts are locked inside alterDataPart() function - auto lock_read_structure = lockStructure(false, __PRETTY_FUNCTION__); + auto lock_read_structure = lockStructure(false); String partition_id = data.getPartitionIDFromQuery(partition, context); auto parts = data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id); @@ -807,7 +807,7 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma case PartitionCommand::FREEZE_PARTITION: { - auto lock = lockStructure(false, __PRETTY_FUNCTION__); + auto lock = lockStructure(false); data.freezePartition(command.partition, command.with_name, context); } break; @@ -818,7 +818,7 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma case PartitionCommand::FREEZE_ALL_PARTITIONS: { - auto lock = lockStructure(false, __PRETTY_FUNCTION__); + auto lock = lockStructure(false); data.freezeAll(command.with_name, context); } break; @@ -836,7 +836,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons /// This protects against "revival" of data for a removed partition after completion of merge. auto merge_blocker = merger_mutator.actions_blocker.cancel(); /// Waits for completion of merge and does not start new ones. - auto lock = lockForAlter(__PRETTY_FUNCTION__); + auto lock = lockForAlter(); String partition_id = data.getPartitionIDFromQuery(partition, context); @@ -919,8 +919,8 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool part, cons void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) { - auto lock1 = lockStructure(false, __PRETTY_FUNCTION__); - auto lock2 = source_table->lockStructure(false, __PRETTY_FUNCTION__); + auto lock1 = lockStructure(false); + auto lock2 = source_table->lockStructure(false); Stopwatch watch; MergeTreeData * src_data = data.checkStructureAndGetMergeTreeData(source_table); diff --git a/dbms/src/Storages/StorageNull.cpp b/dbms/src/Storages/StorageNull.cpp index 25ecfecf2c9..0a1690d5d57 100644 --- a/dbms/src/Storages/StorageNull.cpp +++ b/dbms/src/Storages/StorageNull.cpp @@ -31,7 +31,7 @@ void registerStorageNull(StorageFactory & factory) void StorageNull::alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) { - auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto lock = lockStructureForAlter(); ColumnsDescription new_columns = getColumns(); params.apply(new_columns); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 7fe4a72c0eb..c0795c54291 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -1146,7 +1146,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) /// Can throw an exception. DiskSpaceMonitor::ReservationPtr reserved_space = DiskSpaceMonitor::reserve(full_path, estimated_space_for_merge); - auto table_lock = lockStructure(false, __PRETTY_FUNCTION__); + auto table_lock = lockStructure(false); MergeList::EntryPtr merge_entry = context.getMergeList().insert(database_name, table_name, entry.new_part_name, parts); @@ -1276,7 +1276,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM /// Can throw an exception. DiskSpaceMonitor::ReservationPtr reserved_space = DiskSpaceMonitor::reserve(full_path, estimated_space_for_result); - auto table_lock = lockStructure(false, __PRETTY_FUNCTION__); + auto table_lock = lockStructure(false); MergeTreeData::MutableDataPartPtr new_part; MergeTreeData::Transaction transaction(data); @@ -1582,7 +1582,7 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & /// We don't change table structure, only data in some parts /// To disable reading from these parts, we will sequentially acquire write lock for each part inside alterDataPart() /// If we will lock the whole table here, a deadlock can occur. For example, if use use Buffer table (CLICKHOUSE-3238) - auto lock_read_structure = lockStructure(false, __PRETTY_FUNCTION__); + auto lock_read_structure = lockStructure(false); auto zookeeper = getZooKeeper(); @@ -1683,7 +1683,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) PartDescriptions parts_to_add; MergeTreeData::DataPartsVector parts_to_remove; - auto structure_lock_dst_table = lockStructure(false, __PRETTY_FUNCTION__); + auto structure_lock_dst_table = lockStructure(false); for (size_t i = 0; i < entry_replace.new_part_names.size(); ++i) { @@ -1745,7 +1745,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) return 0; } - structure_lock_src_table = source_table->lockStructure(false, __PRETTY_FUNCTION__); + structure_lock_src_table = source_table->lockStructure(false); MergeTreeData::DataPartStates valid_states{MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}; @@ -2767,7 +2767,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin TableStructureReadLockPtr table_lock; if (!to_detached) - table_lock = lockStructure(true, __PRETTY_FUNCTION__); + table_lock = lockStructure(true); /// Logging Stopwatch stopwatch; @@ -3130,7 +3130,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, { /// Just to read current structure. Alter will be done in separate thread. - auto table_lock = lockStructure(false, __PRETTY_FUNCTION__); + auto table_lock = lockStructure(false); if (is_readonly) throw Exception("Can't ALTER readonly table", ErrorCodes::TABLE_IS_READ_ONLY); @@ -3312,7 +3312,7 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part case PartitionCommand::FREEZE_PARTITION: { - auto lock = lockStructure(false, __PRETTY_FUNCTION__); + auto lock = lockStructure(false); data.freezePartition(command.partition, command.with_name, context); } break; @@ -3323,7 +3323,7 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part case PartitionCommand::FREEZE_ALL_PARTITIONS: { - auto lock = lockStructure(false, __PRETTY_FUNCTION__); + auto lock = lockStructure(false); data.freezeAll(command.with_name, context); } break; @@ -4325,7 +4325,7 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() { /// Critical section is not required (since grabOldParts() returns unique part set on each call) - auto table_lock = lockStructure(false, __PRETTY_FUNCTION__); + auto table_lock = lockStructure(false); auto zookeeper = getZooKeeper(); MergeTreeData::DataPartsVector parts = data.grabOldParts(); @@ -4600,8 +4600,8 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) { - auto lock1 = lockStructure(false, __PRETTY_FUNCTION__); - auto lock2 = source_table->lockStructure(false, __PRETTY_FUNCTION__); + auto lock1 = lockStructure(false); + auto lock2 = source_table->lockStructure(false); Stopwatch watch; MergeTreeData * src_data = data.checkStructureAndGetMergeTreeData(source_table); diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index 91c514dba4b..449f833e259 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -98,7 +98,7 @@ protected: try { - table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__); + table_lock = storage->lockStructure(false); } catch (const Exception & e) { diff --git a/dbms/src/Storages/System/StorageSystemPartsBase.cpp b/dbms/src/Storages/System/StorageSystemPartsBase.cpp index 9d7b2745721..5d888b5bd10 100644 --- a/dbms/src/Storages/System/StorageSystemPartsBase.cpp +++ b/dbms/src/Storages/System/StorageSystemPartsBase.cpp @@ -166,7 +166,7 @@ public: try { /// For table not to be dropped and set of columns to remain constant. - info.table_lock = info.storage->lockStructure(false, __PRETTY_FUNCTION__); + info.table_lock = info.storage->lockStructure(false); } catch (const Exception & e) { diff --git a/suppress_base.json b/suppress_base.json index cc0ba5cbcb5..9a68c58fa55 100644 --- a/suppress_base.json +++ b/suppress_base.json @@ -6038,8 +6038,8 @@ "CodeNext": 0, "CodePrev": 0, "ErrorCode": "V001", - "FileName": "gtest_rw_lock_fifo.cpp.cpp", - "Message": "A code fragment from 'gtest_rw_lock_fifo.cpp.cpp' cannot be analyzed." + "FileName": "gtest_rw_lock.cpp", + "Message": "A code fragment from 'gtest_rw_lock.cpp' cannot be analyzed." }, { "CodeCurrent": 0, From f425803418478c008c6389873edbafbc7ce5ebd5 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 28 Nov 2018 19:05:29 +0300 Subject: [PATCH 30/69] don't overwrite table metadata file in case of offline ALTER after ATTACH [#CLIKCHOUSE-3859] --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 12 ++++++++++-- dbms/src/Storages/StorageReplicatedMergeTree.h | 7 ++++++- ...er_modify_order_by_replicated_zookeeper.reference | 2 ++ ...54_alter_modify_order_by_replicated_zookeeper.sql | 6 ++++++ 4 files changed, 24 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 050985c2851..d8fc93cfdd6 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -393,8 +393,13 @@ void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bo { LOG_WARNING(log, "Table structure in ZooKeeper is a little different from local table structure. Assuming ALTER."); - /// Without any locks, because table has not been created yet. - setTableStructure(std::move(columns_from_zk), metadata_diff); + /// We delay setting table structure till startup() because otherwise new table metadata file can + /// be overwritten in DatabaseOrdinary::createTable. + set_table_structure_at_startup = [columns_from_zk, metadata_diff, this]() + { + /// Without any locks, because table has not been created yet. + setTableStructure(std::move(columns_from_zk), metadata_diff); + }; } else { @@ -2785,6 +2790,9 @@ void StorageReplicatedMergeTree::startup() if (is_readonly) return; + if (set_table_structure_at_startup) + set_table_structure_at_startup(); + queue.initialize( zookeeper_path, replica_path, database_name + "." + table_name + " (ReplicatedMergeTreeQueue)", diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 6dcb642088e..753be7f088b 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -241,13 +241,17 @@ private: */ zkutil::EphemeralNodeHolderPtr replica_is_active_node; - /** Version node /columns in ZooKeeper corresponding to the current data.columns. + /** Version of the /columns node in ZooKeeper corresponding to the current data.columns. * Read and modify along with the data.columns - under TableStructureLock. */ int columns_version = -1; + /// Version of the /metadata node in ZooKeeper. int metadata_version = -1; + /// Used to delay setting table structure till startup() in case of an offline ALTER. + std::function set_table_structure_at_startup; + /** Is this replica "leading". The leader replica selects the parts to merge. */ std::atomic is_leader {false}; @@ -332,6 +336,7 @@ private: /** Verify that the list of columns and table settings match those specified in ZK (/metadata). * If not, throw an exception. + * Must be called before startup(). */ void checkTableStructure(bool skip_sanity_checks, bool allow_alter); diff --git a/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference b/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference index cf2141d2f85..5faefa91056 100644 --- a/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.reference @@ -10,3 +10,5 @@ 1 2 4 90 *** Check SHOW CREATE TABLE *** CREATE TABLE test.summing_r2 ( x UInt32, y UInt32, z UInt32, val UInt32) ENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test/summing\', \'r2\') PRIMARY KEY (x, y) ORDER BY (x, y, -z) SETTINGS index_granularity = 8192 +*** Check SHOW CREATE TABLE after offline ALTER *** +CREATE TABLE test.summing_r2 ( x UInt32, y UInt32, z UInt32, t UInt32, val UInt32) ENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test/summing\', \'r2\') PRIMARY KEY (x, y) ORDER BY (x, y, t * t) SETTINGS index_granularity = 8192 diff --git a/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql b/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql index c9358648d82..95092d55048 100644 --- a/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00754_alter_modify_order_by_replicated_zookeeper.sql @@ -39,5 +39,11 @@ SELECT * FROM test.summing_r2 FINAL ORDER BY x, y, z; SELECT '*** Check SHOW CREATE TABLE ***'; SHOW CREATE TABLE test.summing_r2; +DETACH TABLE test.summing_r2; +ALTER TABLE test.summing_r1 ADD COLUMN t UInt32 AFTER z, MODIFY ORDER BY (x, y, t * t); -- { serverError 341 } +ATTACH TABLE test.summing_r2; +SELECT '*** Check SHOW CREATE TABLE after offline ALTER ***'; +SHOW CREATE TABLE test.summing_r2; + DROP TABLE test.summing_r1; DROP TABLE test.summing_r2; From d4059ba50754af2bd8e120262757e09f86af1470 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 28 Nov 2018 20:21:27 +0300 Subject: [PATCH 31/69] Inject default column --- .../MergeTree/MergeTreeSequentialBlockInputStream.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp index 53bbd8de676..7430ca989f1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { @@ -28,14 +29,15 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( addTotalRowsApprox(data_part->rows_count); header = storage.getSampleBlockForColumns(columns_to_read); - LOG_INFO(log, "Reading columns:" << header.dumpNames()); fixHeader(header); - LOG_INFO(log, "Reading columns(after fix):" << header.dumpNames()); + const NamesAndTypesList & physical_columns = storage.getColumns().getAllPhysical(); + auto column_names_with_helper_columns = columns_to_read; + injectRequiredColumns(storage, data_part, column_names_with_helper_columns); reader = std::make_unique( - data_part->getFullPath(), data_part, header.getNamesAndTypesList(), /* uncompressed_cache = */ nullptr, + data_part->getFullPath(), data_part, physical_columns.addTypes(column_names_with_helper_columns), /* uncompressed_cache = */ nullptr, mark_cache.get(), /* save_marks_in_cache = */ false, storage, MarkRanges{MarkRange(0, data_part->marks_count)}, /* bytes to use AIO */ read_with_direct_io ? 1UL : std::numeric_limits::max(), @@ -46,7 +48,6 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( void MergeTreeSequentialBlockInputStream::fixHeader(Block & header_block) const { /// Types may be different during ALTER (when this stream is used to perform an ALTER). - /// NOTE: We may use similar code to implement non blocking ALTERs. for (const auto & name_type : data_part->columns) { if (header_block.has(name_type.name)) @@ -80,7 +81,6 @@ try current_row += rows_readed; current_mark += (rows_readed / storage.index_granularity); bool should_reorder = false, should_evaluate_missing_defaults = false; - LOG_INFO(log, "Block before filling: " << res.dumpStructure()); reader->fillMissingColumns(res, should_reorder, should_evaluate_missing_defaults, res.rows()); if (res && should_evaluate_missing_defaults) From e2e88324d0a2d2ab322cfb559acaabb339777a83 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 29 Nov 2018 12:19:42 +0300 Subject: [PATCH 32/69] Rename MergeTree streams for select --- ...=> MergeTreeBaseSelectBlockInputStream.cpp} | 14 +++++++------- ...h => MergeTreeBaseSelectBlockInputStream.h} | 8 ++++---- .../MergeTree/MergeTreeBlockReadUtils.h | 4 ++-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 ++ .../MergeTree/MergeTreeDataSelectExecutor.cpp | 10 +++++----- .../Storages/MergeTree/MergeTreeReadPool.cpp | 4 ++-- .../src/Storages/MergeTree/MergeTreeReadPool.h | 2 +- ...cpp => MergeTreeSelectBlockInputStream.cpp} | 18 ++++++++---------- ...eam.h => MergeTreeSelectBlockInputStream.h} | 16 ++++++++-------- .../MergeTreeSequentialBlockInputStream.cpp | 8 ++++++-- .../MergeTreeSequentialBlockInputStream.h | 6 +++++- ... MergeTreeThreadSelectBlockInputStream.cpp} | 12 ++++++------ ...=> MergeTreeThreadSelectBlockInputStream.h} | 8 ++++---- 13 files changed, 60 insertions(+), 52 deletions(-) rename dbms/src/Storages/MergeTree/{MergeTreeBaseBlockInputStream.cpp => MergeTreeBaseSelectBlockInputStream.cpp} (94%) rename dbms/src/Storages/MergeTree/{MergeTreeBaseBlockInputStream.h => MergeTreeBaseSelectBlockInputStream.h} (87%) rename dbms/src/Storages/MergeTree/{MergeTreeBlockInputStream.cpp => MergeTreeSelectBlockInputStream.cpp} (92%) rename dbms/src/Storages/MergeTree/{MergeTreeBlockInputStream.h => MergeTreeSelectBlockInputStream.h} (79%) rename dbms/src/Storages/MergeTree/{MergeTreeThreadBlockInputStream.cpp => MergeTreeThreadSelectBlockInputStream.cpp} (90%) rename dbms/src/Storages/MergeTree/{MergeTreeThreadBlockInputStream.h => MergeTreeThreadSelectBlockInputStream.h} (83%) diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.cpp similarity index 94% rename from dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp rename to dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.cpp index 23c14440bbd..a7613a740d5 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -19,7 +19,7 @@ namespace ErrorCodes } -MergeTreeBaseBlockInputStream::MergeTreeBaseBlockInputStream( +MergeTreeBaseSelectBlockInputStream::MergeTreeBaseSelectBlockInputStream( const MergeTreeData & storage, const PrewhereInfoPtr & prewhere_info, UInt64 max_block_size_rows, @@ -46,7 +46,7 @@ MergeTreeBaseBlockInputStream::MergeTreeBaseBlockInputStream( } -Block MergeTreeBaseBlockInputStream::readImpl() +Block MergeTreeBaseSelectBlockInputStream::readImpl() { Block res; @@ -68,7 +68,7 @@ Block MergeTreeBaseBlockInputStream::readImpl() } -Block MergeTreeBaseBlockInputStream::readFromPart() +Block MergeTreeBaseSelectBlockInputStream::readFromPart() { if (task->size_predictor) task->size_predictor->startBlock(); @@ -181,7 +181,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart() } -void MergeTreeBaseBlockInputStream::injectVirtualColumns(Block & block) const +void MergeTreeBaseSelectBlockInputStream::injectVirtualColumns(Block & block) const { /// add virtual columns /// Except _sample_factor, which is added from the outside. @@ -226,7 +226,7 @@ void MergeTreeBaseBlockInputStream::injectVirtualColumns(Block & block) const } -void MergeTreeBaseBlockInputStream::executePrewhereActions(Block & block, const PrewhereInfoPtr & prewhere_info) +void MergeTreeBaseSelectBlockInputStream::executePrewhereActions(Block & block, const PrewhereInfoPtr & prewhere_info) { if (prewhere_info) { @@ -243,6 +243,6 @@ void MergeTreeBaseBlockInputStream::executePrewhereActions(Block & block, const } -MergeTreeBaseBlockInputStream::~MergeTreeBaseBlockInputStream() = default; +MergeTreeBaseSelectBlockInputStream::~MergeTreeBaseSelectBlockInputStream() = default; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.h similarity index 87% rename from dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h rename to dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.h index 510408622e8..c9e7831fc78 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.h @@ -13,11 +13,11 @@ class UncompressedCache; class MarkCache; -/// Base class for MergeTreeThreadBlockInputStream and MergeTreeBlockInputStream -class MergeTreeBaseBlockInputStream : public IProfilingBlockInputStream +/// Base class for MergeTreeThreadSelectBlockInputStream and MergeTreeSelectBlockInputStream +class MergeTreeBaseSelectBlockInputStream : public IProfilingBlockInputStream { public: - MergeTreeBaseBlockInputStream( + MergeTreeBaseSelectBlockInputStream( const MergeTreeData & storage, const PrewhereInfoPtr & prewhere_info, UInt64 max_block_size_rows, @@ -29,7 +29,7 @@ public: bool save_marks_in_cache = true, const Names & virt_column_names = {}); - ~MergeTreeBaseBlockInputStream() override; + ~MergeTreeBaseSelectBlockInputStream() override; static void executePrewhereActions(Block & block, const PrewhereInfoPtr & prewhere_info); diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 615d0d93b3d..0ce69ebac84 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -25,13 +25,13 @@ using MergeTreeBlockSizePredictorPtr = std::unique_ptr #include -#include +#include #include -#include +#include #include #include #include @@ -654,7 +654,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( for (size_t i = 0; i < num_streams; ++i) { - res.emplace_back(std::make_shared( + res.emplace_back(std::make_shared( i, pool, min_marks_for_concurrent_read, max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, data, use_uncompressed_cache, prewhere_info, settings, virt_columns)); @@ -730,7 +730,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( parts.emplace_back(part); } - BlockInputStreamPtr source_stream = std::make_shared( + BlockInputStreamPtr source_stream = std::make_shared( data, part.data_part, max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, column_names, ranges_to_get_from_part, use_uncompressed_cache, prewhere_info, true, settings.min_bytes_to_use_direct_io, @@ -775,7 +775,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal { RangesInDataPart & part = parts[part_index]; - BlockInputStreamPtr source_stream = std::make_shared( + BlockInputStreamPtr source_stream = std::make_shared( data, part.data_part, max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, column_names, part.ranges, use_uncompressed_cache, prewhere_info, true, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, true, diff --git a/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp b/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp index 5b995e40853..84f8511f5ae 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace ProfileEvents @@ -114,7 +114,7 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read, need_marks -= marks_to_get_from_range; } - /** Change order to right-to-left, for MergeTreeThreadBlockInputStream to get ranges with .pop_back() + /** Change order to right-to-left, for MergeTreeThreadSelectBlockInputStream to get ranges with .pop_back() * (order was changed to left-to-right due to .pop_back() above). */ std::reverse(std::begin(ranges_to_get_from_part), std::end(ranges_to_get_from_part)); diff --git a/dbms/src/Storages/MergeTree/MergeTreeReadPool.h b/dbms/src/Storages/MergeTree/MergeTreeReadPool.h index 47f4258116f..531702a8dd4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReadPool.h @@ -12,7 +12,7 @@ namespace DB using MergeTreeReadTaskPtr = std::unique_ptr; -/** Provides read tasks for MergeTreeThreadBlockInputStream`s in fine-grained batches, allowing for more +/** Provides read tasks for MergeTreeThreadSelectBlockInputStream`s in fine-grained batches, allowing for more * uniform distribution of work amongst multiple threads. All parts and their ranges are divided into `threads` * workloads with at most `sum_marks / threads` marks. Then, threads are performing reads from these workloads * in "sequential" manner, requesting work in small batches. As soon as some thread has exhausted diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeSelectBlockInputStream.cpp similarity index 92% rename from dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp rename to dbms/src/Storages/MergeTree/MergeTreeSelectBlockInputStream.cpp index 16492b83275..76d63743079 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSelectBlockInputStream.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include @@ -9,13 +9,11 @@ namespace DB namespace ErrorCodes { - extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; extern const int MEMORY_LIMIT_EXCEEDED; - extern const int NOT_IMPLEMENTED; } -MergeTreeBlockInputStream::MergeTreeBlockInputStream( +MergeTreeSelectBlockInputStream::MergeTreeSelectBlockInputStream( const MergeTreeData & storage_, const MergeTreeData::DataPartPtr & owned_data_part_, size_t max_block_size_rows_, @@ -33,7 +31,7 @@ MergeTreeBlockInputStream::MergeTreeBlockInputStream( size_t part_index_in_query_, bool quiet) : - MergeTreeBaseBlockInputStream{storage_, prewhere_info, max_block_size_rows_, + MergeTreeBaseSelectBlockInputStream{storage_, prewhere_info, max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, min_bytes_to_use_direct_io_, max_read_buffer_size_, use_uncompressed_cache_, save_marks_in_cache_, virt_column_names}, required_columns{column_names}, @@ -84,13 +82,13 @@ MergeTreeBlockInputStream::MergeTreeBlockInputStream( } -Block MergeTreeBlockInputStream::getHeader() const +Block MergeTreeSelectBlockInputStream::getHeader() const { return header; } -bool MergeTreeBlockInputStream::getNewTask() +bool MergeTreeSelectBlockInputStream::getNewTask() try { /// Produce no more than one task @@ -196,7 +194,7 @@ catch (...) } -void MergeTreeBlockInputStream::finish() +void MergeTreeSelectBlockInputStream::finish() { /** Close the files (before destroying the object). * When many sources are created, but simultaneously reading only a few of them, @@ -209,7 +207,7 @@ void MergeTreeBlockInputStream::finish() } -MergeTreeBlockInputStream::~MergeTreeBlockInputStream() = default; +MergeTreeSelectBlockInputStream::~MergeTreeSelectBlockInputStream() = default; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeSelectBlockInputStream.h similarity index 79% rename from dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h rename to dbms/src/Storages/MergeTree/MergeTreeSelectBlockInputStream.h index 7411a7ff04e..7ff54838ffb 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSelectBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include #include #include #include @@ -10,13 +10,13 @@ namespace DB { -/// Used to read data from single part. -/// To read data from multiple parts, a Storage creates multiple such objects. -/// TODO: Make special lightweight version of the reader for merges and other utilites, remove this from SelectExecutor. -class MergeTreeBlockInputStream : public MergeTreeBaseBlockInputStream +/// Used to read data from single part with select query +/// Cares about PREWHERE, virtual columns, indexes etc. +/// To read data from multiple parts, Storage (MergeTree) creates multiple such objects. +class MergeTreeSelectBlockInputStream : public MergeTreeBaseSelectBlockInputStream { public: - MergeTreeBlockInputStream( + MergeTreeSelectBlockInputStream( const MergeTreeData & storage, const MergeTreeData::DataPartPtr & owned_data_part, size_t max_block_size_rows, @@ -34,7 +34,7 @@ public: size_t part_index_in_query = 0, bool quiet = false); - ~MergeTreeBlockInputStream() override; + ~MergeTreeSelectBlockInputStream() override; String getName() const override { return "MergeTree"; } @@ -74,7 +74,7 @@ private: String path; bool is_first_task = true; - Logger * log = &Logger::get("MergeTreeBlockInputStream"); + Logger * log = &Logger::get("MergeTreeSelectBlockInputStream"); }; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp index 7430ca989f1..49ede0622fd 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp @@ -24,7 +24,7 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( if (!quiet) LOG_TRACE(log, "Reading " << data_part->marks_count << " marks from part " << data_part->name << ", totaly " << data_part->rows_count - << " rows starting from the begging of the part"); + << " rows starting from the beginning of the part"); addTotalRowsApprox(data_part->rows_count); @@ -34,13 +34,16 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( const NamesAndTypesList & physical_columns = storage.getColumns().getAllPhysical(); auto column_names_with_helper_columns = columns_to_read; + + /// Add columns because we don't want to read empty blocks injectRequiredColumns(storage, data_part, column_names_with_helper_columns); reader = std::make_unique( data_part->getFullPath(), data_part, physical_columns.addTypes(column_names_with_helper_columns), /* uncompressed_cache = */ nullptr, mark_cache.get(), /* save_marks_in_cache = */ false, storage, MarkRanges{MarkRange(0, data_part->marks_count)}, - /* bytes to use AIO */ read_with_direct_io ? 1UL : std::numeric_limits::max(), + /* bytes to use AIO (this is hack) */ + read_with_direct_io ? 1UL : std::numeric_limits::max(), DBMS_DEFAULT_BUFFER_SIZE); } @@ -80,6 +83,7 @@ try current_row += rows_readed; current_mark += (rows_readed / storage.index_granularity); + bool should_reorder = false, should_evaluate_missing_defaults = false; reader->fillMissingColumns(res, should_reorder, should_evaluate_missing_defaults, res.rows()); diff --git a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h index 8ebf58d0ea1..8e81625d091 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h @@ -8,7 +8,7 @@ namespace DB { -/// Lightweight (in terms of logic) stream for reading single part in merge process +/// Lightweight (in terms of logic) stream for reading single part from MergeTree class MergeTreeSequentialBlockInputStream : public IProfilingBlockInputStream { public: @@ -29,6 +29,10 @@ public: /// Closes readers and unlock part locks void finish(); + size_t getCurrentMark() const { return current_mark; } + + size_t getCurrentRow() const { return current_row; } + protected: Block readImpl() override; diff --git a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp similarity index 90% rename from dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp rename to dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp index 0fa4d4c7019..432206270d9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.cpp @@ -1,13 +1,13 @@ #include #include -#include +#include namespace DB { -MergeTreeThreadBlockInputStream::MergeTreeThreadBlockInputStream( +MergeTreeThreadSelectBlockInputStream::MergeTreeThreadSelectBlockInputStream( const size_t thread, const MergeTreeReadPoolPtr & pool, const size_t min_marks_to_read_, @@ -20,7 +20,7 @@ MergeTreeThreadBlockInputStream::MergeTreeThreadBlockInputStream( const Settings & settings, const Names & virt_column_names) : - MergeTreeBaseBlockInputStream{storage, prewhere_info, max_block_size_rows, + MergeTreeBaseSelectBlockInputStream{storage, prewhere_info, max_block_size_rows, preferred_block_size_bytes, preferred_max_column_in_block_size_bytes, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, use_uncompressed_cache, true, virt_column_names}, thread{thread}, @@ -39,7 +39,7 @@ MergeTreeThreadBlockInputStream::MergeTreeThreadBlockInputStream( } -Block MergeTreeThreadBlockInputStream::getHeader() const +Block MergeTreeThreadSelectBlockInputStream::getHeader() const { auto res = pool->getHeader(); executePrewhereActions(res, prewhere_info); @@ -49,7 +49,7 @@ Block MergeTreeThreadBlockInputStream::getHeader() const /// Requests read task from MergeTreeReadPool and signals whether it got one -bool MergeTreeThreadBlockInputStream::getNewTask() +bool MergeTreeThreadSelectBlockInputStream::getNewTask() { task = pool->getTask(min_marks_to_read, thread, ordered_names); @@ -112,6 +112,6 @@ bool MergeTreeThreadBlockInputStream::getNewTask() } -MergeTreeThreadBlockInputStream::~MergeTreeThreadBlockInputStream() = default; +MergeTreeThreadSelectBlockInputStream::~MergeTreeThreadSelectBlockInputStream() = default; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.h similarity index 83% rename from dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.h rename to dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.h index 1a2009aac95..064dc88655a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputStream.h @@ -1,5 +1,5 @@ #pragma once -#include +#include namespace DB @@ -11,10 +11,10 @@ class MergeTreeReadPool; /** Used in conjunction with MergeTreeReadPool, asking it for more work to do and performing whatever reads it is asked * to perform. */ -class MergeTreeThreadBlockInputStream : public MergeTreeBaseBlockInputStream +class MergeTreeThreadSelectBlockInputStream : public MergeTreeBaseSelectBlockInputStream { public: - MergeTreeThreadBlockInputStream( + MergeTreeThreadSelectBlockInputStream( const size_t thread, const std::shared_ptr & pool, const size_t min_marks_to_read, @@ -29,7 +29,7 @@ public: String getName() const override { return "MergeTreeThread"; } - ~MergeTreeThreadBlockInputStream() override; + ~MergeTreeThreadSelectBlockInputStream() override; Block getHeader() const override; From b2fd3e804ab26779f9a017c0c7e49bf01926bb5c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 29 Nov 2018 14:55:34 +0300 Subject: [PATCH 33/69] Add special logic for alter --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 6 +-- .../MergeTreeSequentialBlockInputStream.cpp | 38 +++++++++++-------- .../MergeTreeSequentialBlockInputStream.h | 1 + 4 files changed, 28 insertions(+), 19 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index e326ffb96bb..c611653961c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1224,7 +1224,7 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( if (expression) { BlockInputStreamPtr part_in = std::make_shared( - *this, part, expression->getRequiredColumns(), false); + *this, part, expression->getRequiredColumns(), false, /* take_column_types_from_storage = */ false); auto compression_settings = this->context.chooseCompressionSettings( part->bytes_on_disk, diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 0dd5c0abab3..c7f25e372ae 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -607,7 +607,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor UInt64 watch_prev_elapsed = 0; /// We count total amount of bytes in parts - /// and use direct_io + aio is there are more than setting + /// and use direct_io + aio if there is more than min_merge_bytes_to_use_direct_io bool read_with_direct_io = false; if (data.settings.min_merge_bytes_to_use_direct_io != 0) { @@ -629,7 +629,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor for (const auto & part : parts) { auto input = std::make_unique( - data, part, merging_column_names, read_with_direct_io); + data, part, merging_column_names, read_with_direct_io, true); input->setProgressCallback(MergeProgressCallback( merge_entry, sum_input_rows_upper_bound, column_sizes, watch_prev_elapsed, merge_alg)); @@ -775,7 +775,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor for (size_t part_num = 0; part_num < parts.size(); ++part_num) { auto column_part_stream = std::make_shared( - data, parts[part_num], column_names, read_with_direct_io); + data, parts[part_num], column_names, read_with_direct_io, true); column_part_stream->setProgressCallback(MergeProgressCallbackVerticalStep( merge_entry, sum_input_rows_exact, column_sizes, column_name, watch_prev_elapsed)); diff --git a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp index 49ede0622fd..a82c745b7ff 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp @@ -13,6 +13,7 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( const MergeTreeData::DataPartPtr & data_part_, Names columns_to_read_, bool read_with_direct_io_, + bool take_column_types_from_storage, bool quiet) : storage(storage_) , data_part(data_part_) @@ -29,17 +30,21 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( addTotalRowsApprox(data_part->rows_count); header = storage.getSampleBlockForColumns(columns_to_read); - fixHeader(header); - const NamesAndTypesList & physical_columns = storage.getColumns().getAllPhysical(); - auto column_names_with_helper_columns = columns_to_read; + /// take columns from data_part (header was fixed by fixHeader) + NamesAndTypesList columns_for_reader = header.getNamesAndTypesList(); + if (take_column_types_from_storage) + { + const NamesAndTypesList & physical_columns = storage.getColumns().getAllPhysical(); + auto column_names_with_helper_columns = columns_to_read; + injectRequiredColumns(storage, data_part, column_names_with_helper_columns); + columns_for_reader = physical_columns.addTypes(column_names_with_helper_columns); + } /// Add columns because we don't want to read empty blocks - injectRequiredColumns(storage, data_part, column_names_with_helper_columns); - reader = std::make_unique( - data_part->getFullPath(), data_part, physical_columns.addTypes(column_names_with_helper_columns), /* uncompressed_cache = */ nullptr, + data_part->getFullPath(), data_part, columns_for_reader, /* uncompressed_cache = */ nullptr, mark_cache.get(), /* save_marks_in_cache = */ false, storage, MarkRanges{MarkRange(0, data_part->marks_count)}, /* bytes to use AIO (this is hack) */ @@ -79,19 +84,22 @@ try bool continue_reading = (current_mark != 0); size_t rows_readed = reader->readRows(current_mark, continue_reading, storage.index_granularity, res); - res.checkNumberOfRows(); + if (res) + { + res.checkNumberOfRows(); - current_row += rows_readed; - current_mark += (rows_readed / storage.index_granularity); + current_row += rows_readed; + current_mark += (rows_readed / storage.index_granularity); - bool should_reorder = false, should_evaluate_missing_defaults = false; - reader->fillMissingColumns(res, should_reorder, should_evaluate_missing_defaults, res.rows()); + bool should_reorder = false, should_evaluate_missing_defaults = false; + reader->fillMissingColumns(res, should_reorder, should_evaluate_missing_defaults, res.rows()); - if (res && should_evaluate_missing_defaults) - reader->evaluateMissingDefaults(res); + if (should_evaluate_missing_defaults) + reader->evaluateMissingDefaults(res); - if (res && should_reorder) - reader->reorderColumns(res, header.getNames(), nullptr); + if (should_reorder) + reader->reorderColumns(res, header.getNames(), nullptr); + } } else { diff --git a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h index 8e81625d091..7c0d2cc469c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h @@ -17,6 +17,7 @@ public: const MergeTreeData::DataPartPtr & data_part_, Names columns_to_read_, bool read_with_direct_io_, + bool take_column_types_from_storage, bool quiet = false ); From fc379c1d5c51da9868a6ed440ee8adfbc1c2a848 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 29 Nov 2018 15:03:58 +0300 Subject: [PATCH 34/69] Inject column names before types --- .../MergeTree/MergeTreeSequentialBlockInputStream.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp index a82c745b7ff..bfc23d76402 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp @@ -28,6 +28,7 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( << " rows starting from the beginning of the part"); addTotalRowsApprox(data_part->rows_count); + injectRequiredColumns(storage, data_part, columns_to_read); header = storage.getSampleBlockForColumns(columns_to_read); fixHeader(header); @@ -37,9 +38,7 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( if (take_column_types_from_storage) { const NamesAndTypesList & physical_columns = storage.getColumns().getAllPhysical(); - auto column_names_with_helper_columns = columns_to_read; - injectRequiredColumns(storage, data_part, column_names_with_helper_columns); - columns_for_reader = physical_columns.addTypes(column_names_with_helper_columns); + columns_for_reader = physical_columns.addTypes(columns_to_read); } /// Add columns because we don't want to read empty blocks From 51f73651ddce57f1e4722299f1aa23b4ddac7125 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 29 Nov 2018 18:16:08 +0300 Subject: [PATCH 35/69] Inject columns after creating header --- .../MergeTreeSequentialBlockInputStream.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp index bfc23d76402..d984a75bf23 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp @@ -28,20 +28,24 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( << " rows starting from the beginning of the part"); addTotalRowsApprox(data_part->rows_count); - injectRequiredColumns(storage, data_part, columns_to_read); header = storage.getSampleBlockForColumns(columns_to_read); fixHeader(header); - /// take columns from data_part (header was fixed by fixHeader) - NamesAndTypesList columns_for_reader = header.getNamesAndTypesList(); + /// Add columns because we don't want to read empty blocks + injectRequiredColumns(storage, data_part, columns_to_read); + NamesAndTypesList columns_for_reader; if (take_column_types_from_storage) { const NamesAndTypesList & physical_columns = storage.getColumns().getAllPhysical(); columns_for_reader = physical_columns.addTypes(columns_to_read); } + else + { + /// take columns from data_part + columns_for_reader = data_part->columns.addTypes(columns_to_read); + } - /// Add columns because we don't want to read empty blocks reader = std::make_unique( data_part->getFullPath(), data_part, columns_for_reader, /* uncompressed_cache = */ nullptr, mark_cache.get(), /* save_marks_in_cache = */ false, storage, From eb91b19a9981880da44b77f5b83b046891f0c58f Mon Sep 17 00:00:00 2001 From: Daniel Dao Date: Fri, 30 Nov 2018 02:17:01 +0000 Subject: [PATCH 36/69] Fix buffer overflow for EmptyImpl with FixedString Signed-off-by: Daniel Dao --- dbms/src/Functions/EmptyImpl.h | 2 +- .../00568_empty_function_with_fixed_string.reference | 1 + .../0_stateless/00568_empty_function_with_fixed_string.sql | 6 ++++++ 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/dbms/src/Functions/EmptyImpl.h b/dbms/src/Functions/EmptyImpl.h index a9702fe8869..abd406e0920 100644 --- a/dbms/src/Functions/EmptyImpl.h +++ b/dbms/src/Functions/EmptyImpl.h @@ -42,7 +42,7 @@ struct EmptyImpl size_t size = data.size() / n; for (size_t i = 0; i < size; ++i) - res[i] = negative ^ (0 == memcmp(&data[i * size], empty_chars.data(), n)); + res[i] = negative ^ (0 == memcmp(&data[i * n], empty_chars.data(), n)); } static void array(const ColumnString::Offsets & offsets, PaddedPODArray & res) diff --git a/dbms/tests/queries/0_stateless/00568_empty_function_with_fixed_string.reference b/dbms/tests/queries/0_stateless/00568_empty_function_with_fixed_string.reference index ae8f58a27fb..9c4bdb5d5ec 100644 --- a/dbms/tests/queries/0_stateless/00568_empty_function_with_fixed_string.reference +++ b/dbms/tests/queries/0_stateless/00568_empty_function_with_fixed_string.reference @@ -1,2 +1,3 @@ \0\0\0\0 1 \0abc 0 +32768 1 diff --git a/dbms/tests/queries/0_stateless/00568_empty_function_with_fixed_string.sql b/dbms/tests/queries/0_stateless/00568_empty_function_with_fixed_string.sql index 9ceb0fa4c75..7abc9067b7c 100644 --- a/dbms/tests/queries/0_stateless/00568_empty_function_with_fixed_string.sql +++ b/dbms/tests/queries/0_stateless/00568_empty_function_with_fixed_string.sql @@ -1,2 +1,8 @@ SELECT toFixedString('', 4) AS str, empty(str) AS is_empty; SELECT toFixedString('\0abc', 4) AS str, empty(str) AS is_empty; + +DROP TABLE IF EXISTS test.defaulted; +CREATE TABLE test.defaulted (v6 FixedString(16)) ENGINE=Memory; +INSERT INTO test.defaulted SELECT toFixedString('::0', 16) FROM numbers(32768); +SELECT count(), notEmpty(v6) e FROM test.defaulted GROUP BY e; +DROP TABLE test.defaulted; From c9b0cd454e8ae8b22c0af7d81cfa4a804da858ed Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 30 Nov 2018 15:30:05 +0300 Subject: [PATCH 37/69] There are more than two languages now + mention PDF --- docs/tools/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/README.md b/docs/tools/README.md index a0b853c0308..9b4dd26dd14 100644 --- a/docs/tools/README.md +++ b/docs/tools/README.md @@ -1,6 +1,6 @@ ## How ClickHouse documentation is generated? -ClickHouse documentation is built using [build.py](build.py) script that uses [mkdocs](https://www.mkdocs.org) library and it's dependencies to separately build all version of documentations (two languages in either single and multi page mode) as static HTMLs. The results are then put in correct directory structure. +ClickHouse documentation is built using [build.py](build.py) script that uses [mkdocs](https://www.mkdocs.org) library and it's dependencies to separately build all version of documentations (all languages in either single and multi page mode) as static HTMLs. The results are then put in correct directory structure. It can also generate PDF version. Finally [the infrustructure](../website) that builds ClickHouse [official website](https://clickhouse.yandex) just puts that directory structure into the same Docker container together with rest of website and deploys it to Yandex private cloud. From 078a192388e9de131a0aa3eafb8171f9759d31bd Mon Sep 17 00:00:00 2001 From: BayoNet Date: Fri, 30 Nov 2018 16:04:14 +0300 Subject: [PATCH 38/69] Update of JOIN docs (#3684) * Update of english version of descriprion of the table function `file`. * New syntax for ReplacingMergeTree. Some improvements in text. * Significantly change article about SummingMergeTree. Article is restructured, text is changed in many places of the document. New syntax for table creation is described. * Descriptions of AggregateFunction and AggregatingMergeTree are updated. Russian version. * New syntax for new syntax of CREATE TABLE * Added english docs on Aggregating, Replacing and SummingMergeTree. * CollapsingMergeTree docs. English version. * 1. Update of CollapsingMergeTree. 2. Minor changes in markup * Update aggregatefunction.md * Update aggregatefunction.md * Update aggregatefunction.md * Update aggregatingmergetree.md * GraphiteMergeTree docs update. New syntax for creation of Replicated* tables. Minor changes in *MergeTree tables creation syntax. * Markup fix * Markup and language fixes * Clarification in the CollapsingMergeTree article * DOCAPI-4821. Sync between ru and en versions of docs. * Fixed the ambiguity in geo functions description. * Example of JOIN in ru docs * Deleted misinforming example. * 1. Updated the JOIN clause description. 2. Added the new setting 'join_default_strictness' description. * Minor fixes in docs. * Deleted version of ClickHouse from setting. All info about new features are in changelog. --- docs/en/operations/settings/settings.md | 18 +- docs/en/operations/table_engines/join.md | 2 + docs/en/query_language/select.md | 225 ++++++++++++----------- 3 files changed, 139 insertions(+), 106 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8afea8e5d35..64f086930ef 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -72,7 +72,7 @@ Always pair it with `input_format_allow_errors_ratio`. To skip errors, both sett If an error occurred while reading rows but the error counter is still less than `input_format_allow_errors_num`, ClickHouse ignores the row and moves on to the next one. -If `input_format_allow_errors_num`is exceeded, ClickHouse throws an exception. +If `input_format_allow_errors_num` is exceeded, ClickHouse throws an exception. ## input_format_allow_errors_ratio @@ -87,6 +87,22 @@ If an error occurred while reading rows but the error counter is still less than If `input_format_allow_errors_ratio` is exceeded, ClickHouse throws an exception. + + +## join_default_strictness + +Sets default strictness for [JOIN clause](../../query_language/select.md#query-language-join). + +**Possible values** + +- `ALL` — If the right table has several matching rows, the data will be multiplied by the number of these rows. It is a normal `JOIN` behavior from standard SQL. +- `ANY` — If the right table has several matching rows, only the first one found is joined. If the right table has only one matching row, the results of `ANY` and `ALL` are the same. +- `Empty string` — If `ALL` or `ANY` not specified in query, ClickHouse throws exception. + +**Default value** + +`ALL` + ## max_block_size In ClickHouse, data is processed by blocks (sets of column parts). The internal processing cycles for a single block are efficient enough, but there are noticeable expenditures on each block. `max_block_size` is a recommendation for what size of block (in number of rows) to load from tables. The block size shouldn't be too small, so that the expenditures on each block are still noticeable, but not too large, so that the query with LIMIT that is completed after the first block is processed quickly, so that too much memory isn't consumed when extracting a large number of columns in multiple threads, and so that at least some cache locality is preserved. diff --git a/docs/en/operations/table_engines/join.md b/docs/en/operations/table_engines/join.md index 299071312b8..217e499305f 100644 --- a/docs/en/operations/table_engines/join.md +++ b/docs/en/operations/table_engines/join.md @@ -1,3 +1,5 @@ + + # Join A prepared data structure for JOIN that is always located in RAM. diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index 53164e67d80..b14410b9fa7 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -80,6 +80,8 @@ A sample with a relative coefficient is "consistent": if we look at all possible For example, a sample of user IDs takes rows with the same subset of all the possible user IDs from different tables. This allows using the sample in subqueries in the IN clause, as well as for manually correlating results of different queries with samples. + + ### ARRAY JOIN Clause Allows executing JOIN with an array or nested data structure. The intent is similar to the 'arrayJoin' function, but its functionality is broader. @@ -332,42 +334,55 @@ The query can only specify a single ARRAY JOIN clause. The corresponding conversion can be performed before the WHERE/PREWHERE clause (if its result is needed in this clause), or after completing WHERE/PREWHERE (to reduce the volume of calculations). + + ### JOIN Clause -The normal JOIN, which is not related to ARRAY JOIN described above. +Joins the data in the usual [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) sense. + +!!! info "Note" + Not related to [ARRAY JOIN](#select-array-join). + ``` sql -[GLOBAL] ANY|ALL INNER|LEFT [OUTER] JOIN (subquery)|table USING columns_list +SELECT +FROM +[GLOBAL] [ANY|ALL] INNER|LEFT|RIGHT|FULL|CROSS [OUTER] JOIN +(ON )|(USING ) ... ``` -Performs joins with data from the subquery. At the beginning of query processing, the subquery specified after JOIN is run, and its result is saved in memory. Then it is read from the "left" table specified in the FROM clause, and while it is being read, for each of the read rows from the "left" table, rows are selected from the subquery results table (the "right" table) that meet the condition for matching the values of the columns specified in USING. +The table names can be specified instead of `` and ``. This is equivalent to the `SELECT * FROM table` subquery, except in a special case when the table has the [Join](../operations/table_engines/join.md#table-engine-join) engine – an array prepared for joining. -The table name can be specified instead of a subquery. This is equivalent to the `SELECT * FROM table` subquery, except in a special case when the table has the Join engine – an array prepared for joining. +**Supported types of `JOIN`** -All columns that are not needed for the JOIN are deleted from the subquery. +- `INNER JOIN` +- `LEFT OUTER JOIN` +- `RIGHT OUTER JOIN` +- `FULL OUTER JOIN` +- `CROSS JOIN` -There are several types of JOINs: +You may skip the `OUTER` keyword it is implied by default. -`INNER` or `LEFT` type:If INNER is specified, the result will contain only those rows that have a matching row in the right table. -If LEFT is specified, any rows in the left table that don't have matching rows in the right table will be assigned the default value - zeros or empty rows. LEFT OUTER may be written instead of LEFT; the word OUTER does not affect anything. +**`ANY` or `ALL` strictness** -`ANY` or `ALL` stringency:If `ANY` is specified and the right table has several matching rows, only the first one found is joined. -If `ALL` is specified and the right table has several matching rows, the data will be multiplied by the number of these rows. +If `ALL` is specified and the right table has several matching rows, the data will be multiplied by the number of these rows. It is a normal `JOIN` behavior from standard SQL. +If `ANY` is specified and the right table has several matching rows, only the first one found is joined. If the right table has only one matching row, the results of `ANY` and `ALL` are the same. -Using ALL corresponds to the normal JOIN semantic from standard SQL. -Using ANY is optimal. If the right table has only one matching row, the results of ANY and ALL are the same. You must specify either ANY or ALL (neither of them is selected by default). +You can set the default value of strictness with session configuration parameter [join_default_strictness](../operations/settings/settings.md#session-setting-join_default_strictness). -`GLOBAL` distribution: +**`GLOBAL` distribution** -When using a normal JOIN, the query is sent to remote servers. Subqueries are run on each of them in order to make the right table, and the join is performed with this table. In other words, the right table is formed on each server separately. +When using a normal `JOIN`, the query is sent to remote servers. Subqueries are run on each of them in order to make the right table, and the join is performed with this table. In other words, the right table is formed on each server separately. When using `GLOBAL ... JOIN`, first the requestor server runs a subquery to calculate the right table. This temporary table is passed to each remote server, and queries are run on them using the temporary data that was transmitted. -Be careful when using GLOBAL JOINs. For more information, see the section "Distributed subqueries". +Be careful when using `GLOBAL`. For more information, see the section [Distributed subqueries](#queries-distributed-subqueries). -Any combination of JOINs is possible. For example, `GLOBAL ANY LEFT OUTER JOIN`. +**Usage Recommendations** -When running a JOIN, there is no optimization of the order of execution in relation to other stages of the query. The join (a search in the right table) is run before filtering in WHERE and before aggregation. In order to explicitly set the processing order, we recommend running a JOIN subquery with a subquery. +All columns that are not needed for the `JOIN` are deleted from the subquery. + +When running a `JOIN`, there is no optimization of the order of execution in relation to other stages of the query. The join (a search in the right table) is run before filtering in `WHERE` and before aggregation. In order to explicitly set the processing order, we recommend running a `JOIN` subquery with a subquery. Example: @@ -411,20 +426,20 @@ LIMIT 10 ``` Subqueries don't allow you to set names or use them for referencing a column from a specific subquery. -The columns specified in USING must have the same names in both subqueries, and the other columns must be named differently. You can use aliases to change the names of columns in subqueries (the example uses the aliases 'hits' and 'visits'). +The columns specified in `USING` must have the same names in both subqueries, and the other columns must be named differently. You can use aliases to change the names of columns in subqueries (the example uses the aliases 'hits' and 'visits'). -The USING clause specifies one or more columns to join, which establishes the equality of these columns. The list of columns is set without brackets. More complex join conditions are not supported. +The `USING` clause specifies one or more columns to join, which establishes the equality of these columns. The list of columns is set without brackets. More complex join conditions are not supported. -The right table (the subquery result) resides in RAM. If there isn't enough memory, you can't run a JOIN. +The right table (the subquery result) resides in RAM. If there isn't enough memory, you can't run a `JOIN`. -Only one JOIN can be specified in a query (on a single level). To run multiple JOINs, you can put them in subqueries. +Only one `JOIN` can be specified in a query (on a single level). To run multiple `JOIN`, you can put them in subqueries. -Each time a query is run with the same JOIN, the subquery is run again – the result is not cached. To avoid this, use the special 'Join' table engine, which is a prepared array for joining that is always in RAM. For more information, see the section "Table engines, Join". +Each time a query is run with the same `JOIN`, the subquery is run again – the result is not cached. To avoid this, use the special 'Join' table engine, which is a prepared array for joining that is always in RAM. For more information, see the section "Table engines, Join". -In some cases, it is more efficient to use IN instead of JOIN. -Among the various types of JOINs, the most efficient is ANY LEFT JOIN, then ANY INNER JOIN. The least efficient are ALL LEFT JOIN and ALL INNER JOIN. +In some cases, it is more efficient to use `IN` instead of `JOIN`. +Among the various types of `JOIN`, the most efficient is ANY `LEFT JOIN`, then `ANY INNER JOIN`. The least efficient are `ALL LEFT JOIN` and `ALL INNER JOIN`. -If you need a JOIN for joining with dimension tables (these are relatively small tables that contain dimension properties, such as names for advertising campaigns), a JOIN might not be very convenient due to the bulky syntax and the fact that the right table is re-accessed for every query. For such cases, there is an "external dictionaries" feature that you should use instead of JOIN. For more information, see the section "External dictionaries". +If you need a `JOIN` for joining with dimension tables (these are relatively small tables that contain dimension properties, such as names for advertising campaigns), a `JOIN` might not be very convenient due to the bulky syntax and the fact that the right table is re-accessed for every query. For such cases, there is an "external dictionaries" feature that you should use instead of `JOIN`. For more information, see the section [External dictionaries](dicts/external_dicts.md#dicts-external_dicts). @@ -500,35 +515,35 @@ A constant can't be specified as arguments for aggregate functions. Example: sum #### NULL processing - For grouping, ClickHouse interprets [NULL](syntax.md#null-literal) as a value, and `NULL=NULL`. +For grouping, ClickHouse interprets [NULL](syntax.md#null-literal) as a value, and `NULL=NULL`. - Here's an example to show what this means. +Here's an example to show what this means. - Assume you have this table: +Assume you have this table: - ``` - ┌─x─┬────y─┐ - │ 1 │ 2 │ - │ 2 │ ᴺᵁᴸᴸ │ - │ 3 │ 2 │ - │ 3 │ 3 │ - │ 3 │ ᴺᵁᴸᴸ │ - └───┴──────┘ - ``` +``` +┌─x─┬────y─┐ +│ 1 │ 2 │ +│ 2 │ ᴺᵁᴸᴸ │ +│ 3 │ 2 │ +│ 3 │ 3 │ +│ 3 │ ᴺᵁᴸᴸ │ +└───┴──────┘ +``` - The query `SELECT sum(x), y FROM t_null_big GROUP BY y` results in: +The query `SELECT sum(x), y FROM t_null_big GROUP BY y` results in: - ``` - ┌─sum(x)─┬────y─┐ - │ 4 │ 2 │ - │ 3 │ 3 │ - │ 5 │ ᴺᵁᴸᴸ │ - └────────┴──────┘ - ``` +``` +┌─sum(x)─┬────y─┐ +│ 4 │ 2 │ +│ 3 │ 3 │ +│ 5 │ ᴺᵁᴸᴸ │ +└────────┴──────┘ +``` - You can see that `GROUP BY` for `У = NULL` summed up `x`, as if `NULL` is this value. +You can see that `GROUP BY` for `У = NULL` summed up `x`, as if `NULL` is this value. - If you pass several keys to `GROUP BY`, the result will give you all the combinations of the selection, as if `NULL` were a specific value. +If you pass several keys to `GROUP BY`, the result will give you all the combinations of the selection, as if `NULL` were a specific value. #### WITH TOTALS Modifier @@ -615,45 +630,45 @@ If the ORDER BY clause is omitted, the order of the rows is also undefined, and `NaN` and `NULL` sorting order: - - With the modifier `NULLS FIRST` — First `NULL`, then `NaN`, then other values. - - With the modifier `NULLS LAST` — First the values, then `NaN`, then `NULL`. - - Default — The same as with the `NULLS LAST` modifier. +- With the modifier `NULLS FIRST` — First `NULL`, then `NaN`, then other values. +- With the modifier `NULLS LAST` — First the values, then `NaN`, then `NULL`. +- Default — The same as with the `NULLS LAST` modifier. - Example: +Example: - For the table +For the table - ``` - ┌─x─┬────y─┐ - │ 1 │ ᴺᵁᴸᴸ │ - │ 2 │ 2 │ - │ 1 │ nan │ - │ 2 │ 2 │ - │ 3 │ 4 │ - │ 5 │ 6 │ - │ 6 │ nan │ - │ 7 │ ᴺᵁᴸᴸ │ - │ 6 │ 7 │ - │ 8 │ 9 │ - └───┴──────┘ - ``` +``` +┌─x─┬────y─┐ +│ 1 │ ᴺᵁᴸᴸ │ +│ 2 │ 2 │ +│ 1 │ nan │ +│ 2 │ 2 │ +│ 3 │ 4 │ +│ 5 │ 6 │ +│ 6 │ nan │ +│ 7 │ ᴺᵁᴸᴸ │ +│ 6 │ 7 │ +│ 8 │ 9 │ +└───┴──────┘ +``` - Run the query `SELECT * FROM t_null_nan ORDER BY y NULLS FIRST` to get: +Run the query `SELECT * FROM t_null_nan ORDER BY y NULLS FIRST` to get: - ``` - ┌─x─┬────y─┐ - │ 1 │ ᴺᵁᴸᴸ │ - │ 7 │ ᴺᵁᴸᴸ │ - │ 1 │ nan │ - │ 6 │ nan │ - │ 2 │ 2 │ - │ 2 │ 2 │ - │ 3 │ 4 │ - │ 5 │ 6 │ - │ 6 │ 7 │ - │ 8 │ 9 │ - └───┴──────┘ - ``` +``` +┌─x─┬────y─┐ +│ 1 │ ᴺᵁᴸᴸ │ +│ 7 │ ᴺᵁᴸᴸ │ +│ 1 │ nan │ +│ 6 │ nan │ +│ 2 │ 2 │ +│ 2 │ 2 │ +│ 3 │ 4 │ +│ 5 │ 6 │ +│ 6 │ 7 │ +│ 8 │ 9 │ +└───┴──────┘ +``` When floating point numbers are sorted, NaNs are separate from the other values. Regardless of the sorting order, NaNs come at the end. In other words, for ascending sorting they are placed as if they are larger than all the other numbers, while for descending sorting they are placed as if they are smaller than the rest. @@ -804,38 +819,38 @@ A subquery in the IN clause is always run just one time on a single server. Ther #### NULL processing - During request processing, the IN operator assumes that the result of an operation with [NULL](syntax.md#null-literal) is always equal to `0`, regardless of whether `NULL` is on the right or left side of the operator. `NULL` values are not included in any dataset, do not correspond to each other and cannot be compared. +During request processing, the IN operator assumes that the result of an operation with [NULL](syntax.md#null-literal) is always equal to `0`, regardless of whether `NULL` is on the right or left side of the operator. `NULL` values are not included in any dataset, do not correspond to each other and cannot be compared. - Here is an example with the `t_null` table: +Here is an example with the `t_null` table: - ``` - ┌─x─┬────y─┐ - │ 1 │ ᴺᵁᴸᴸ │ - │ 2 │ 3 │ - └───┴──────┘ - ``` +``` +┌─x─┬────y─┐ +│ 1 │ ᴺᵁᴸᴸ │ +│ 2 │ 3 │ +└───┴──────┘ +``` - Running the query `SELECT x FROM t_null WHERE y IN (NULL,3)` gives you the following result: +Running the query `SELECT x FROM t_null WHERE y IN (NULL,3)` gives you the following result: - ``` - ┌─x─┐ - │ 2 │ - └───┘ - ``` +``` +┌─x─┐ +│ 2 │ +└───┘ +``` - You can see that the row in which `y = NULL` is thrown out of the query results. This is because ClickHouse can't decide whether `NULL` is included in the `(NULL,3)` set, returns `0` as the result of the operation, and `SELECT` excludes this row from the final output. +You can see that the row in which `y = NULL` is thrown out of the query results. This is because ClickHouse can't decide whether `NULL` is included in the `(NULL,3)` set, returns `0` as the result of the operation, and `SELECT` excludes this row from the final output. - ``` - SELECT y IN (NULL, 3) - FROM t_null +``` +SELECT y IN (NULL, 3) +FROM t_null - ┌─in(y, tuple(NULL, 3))─┐ - │ 0 │ - │ 1 │ - └───────────────────────┘ - ``` +┌─in(y, tuple(NULL, 3))─┐ +│ 0 │ +│ 1 │ +└───────────────────────┘ +``` - + #### Distributed Subqueries From f59bf174d777933b0d9080877fedae5adb0e6d66 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 30 Nov 2018 16:37:36 +0300 Subject: [PATCH 39/69] Fixed build of "widechar_width" library on ARM [#CLICKHOUSE-2] --- libs/libwidechar_width/widechar_width.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/libwidechar_width/widechar_width.h b/libs/libwidechar_width/widechar_width.h index 9465fdc5f97..f49180e7c54 100644 --- a/libs/libwidechar_width/widechar_width.h +++ b/libs/libwidechar_width/widechar_width.h @@ -495,7 +495,7 @@ static const struct widechar_range widechar_widened_table[] = { template bool widechar_in_table(const Collection &arr, int32_t c) { auto where = std::lower_bound(std::begin(arr), std::end(arr), c, - [](widechar_range p, wchar_t c) { return p.hi < c; }); + [](widechar_range p, int32_t c) { return p.hi < c; }); return where != std::end(arr) && where->lo <= c; } From cd2af4b6d2ae0bb0353fdcd9f367e9c75422afbe Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 30 Nov 2018 16:49:53 +0300 Subject: [PATCH 40/69] Update Contributors --- .../System/StorageSystemContributors.generated.cpp | 9 ++++++++- release_lib.sh | 2 +- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/System/StorageSystemContributors.generated.cpp b/dbms/src/Storages/System/StorageSystemContributors.generated.cpp index 135fb357ff5..fb4f9c4c1f4 100644 --- a/dbms/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/dbms/src/Storages/System/StorageSystemContributors.generated.cpp @@ -1,4 +1,4 @@ -// autogenerated by ./StorageSystemContributors.sh +// autogenerated by dbms/src/Storages/System/StorageSystemContributors.sh const char * auto_contributors[] { "Alberto", "Aleksandra (Ася)", @@ -7,6 +7,7 @@ const char * auto_contributors[] { "Alex Zatelepin", "Alexander Avdonkin", "Alexander Krasheninnikov", + "Alexander Kuranoff", "Alexander Lukin", "Alexander Makarov", "Alexander Marshalov", @@ -64,6 +65,7 @@ const char * auto_contributors[] { "Dmitry Luhtionov", "Dmitry Moskowski", "Dmitry S..ky / skype: dvska-at-skype", + "Elghazal Ahmed", "Eugene Klimov", "Eugene Konkov", "Evgeniy Gatov", @@ -87,6 +89,7 @@ const char * auto_contributors[] { "Ivan Babrou", "Ivan Blinkov", "Ivan He", + "Ivan Lezhankin", "Ivan Zhukov", "Jason", "Jean Baptiste Favre", @@ -142,6 +145,7 @@ const char * auto_contributors[] { "Pavel Litvinenko", "Pavel Patrin", "Pavel Yakunin", + "Pavlo Bashynskiy", "Pawel Rog", "Ravengg", "Reto Kromer", @@ -149,6 +153,7 @@ const char * auto_contributors[] { "Roman Nozdrin", "Roman Peshkurov", "Roman Tsisyk", + "Sabyanin Maxim", "SaltTan", "Sergei Tsetlin (rekub)", "Sergey Elantsev", @@ -179,6 +184,7 @@ const char * auto_contributors[] { "Veniamin Gvozdikov", "Victor Tarnavsky", "Vitaliy Lyudvichenko", + "Vitaly Baranov", "Vitaly Samigullin", "Vlad Arkhipov", "Vladimir Chebotarev", @@ -223,6 +229,7 @@ const char * auto_contributors[] { "glockbender", "hotid", "igor", + "igor.lapko", "ivanzhukov", "javi", "javi santana", diff --git a/release_lib.sh b/release_lib.sh index 328b91b8f5b..448c84d03a2 100644 --- a/release_lib.sh +++ b/release_lib.sh @@ -96,7 +96,7 @@ function gen_revision_author { gen_changelog "$VERSION_STRING" "" "$AUTHOR" "" gen_dockerfiles "$VERSION_STRING" - . dbms/src/Storages/System/StorageSystemContributors.sh + . dbms/src/Storages/System/StorageSystemContributors.sh ||: git commit -m "$auto_message [$VERSION_STRING] [$VERSION_REVISION]" dbms/cmake/version.cmake debian/changelog docker/*/Dockerfile dbms/src/Storages/System/StorageSystemContributors.generated.cpp git push From d092d35fa91b798d3877c07864db46c5c500a0a4 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 29 Nov 2018 23:44:12 +0800 Subject: [PATCH 41/69] NonJoinedStream[ALL] bypasses max_block_size --- dbms/src/Interpreters/Join.cpp | 13 ++++++++----- .../00779_all_right_join_max_block_size.reference | 1 + .../00779_all_right_join_max_block_size.sql | 2 ++ 3 files changed, 11 insertions(+), 5 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00779_all_right_join_max_block_size.reference create mode 100644 dbms/tests/queries/0_stateless/00779_all_right_join_max_block_size.sql diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 00d74cc0e2d..7707eed6933 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -960,7 +960,7 @@ struct AdderNonJoined; template struct AdderNonJoined { - static void add(const Mapped & mapped, + static void add(const Mapped & mapped, size_t & rows_added, size_t num_columns_left, MutableColumns & columns_left, size_t num_columns_right, MutableColumns & columns_right) { @@ -969,13 +969,15 @@ struct AdderNonJoined for (size_t j = 0; j < num_columns_right; ++j) columns_right[j]->insertFrom(*mapped.block->getByPosition(j).column.get(), mapped.row_num); + + ++rows_added; } }; template struct AdderNonJoined { - static void add(const Mapped & mapped, + static void add(const Mapped & mapped, size_t & rows_added, size_t num_columns_left, MutableColumns & columns_left, size_t num_columns_right, MutableColumns & columns_right) { @@ -986,6 +988,8 @@ struct AdderNonJoined for (size_t j = 0; j < num_columns_right; ++j) columns_right[j]->insertFrom(*current->block->getByPosition(j).column.get(), current->row_num); + + ++rows_added; } } }; @@ -1155,10 +1159,9 @@ private: if (it->second.getUsed()) continue; - AdderNonJoined::add(it->second, num_columns_left, columns_left, num_columns_right, columns_right); + AdderNonJoined::add(it->second, rows_added, num_columns_left, columns_left, num_columns_right, columns_right); - ++rows_added; - if (rows_added == max_block_size) + if (rows_added >= max_block_size) { ++it; break; diff --git a/dbms/tests/queries/0_stateless/00779_all_right_join_max_block_size.reference b/dbms/tests/queries/0_stateless/00779_all_right_join_max_block_size.reference new file mode 100644 index 00000000000..7f8f011eb73 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00779_all_right_join_max_block_size.reference @@ -0,0 +1 @@ +7 diff --git a/dbms/tests/queries/0_stateless/00779_all_right_join_max_block_size.sql b/dbms/tests/queries/0_stateless/00779_all_right_join_max_block_size.sql new file mode 100644 index 00000000000..64a41d92a2f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00779_all_right_join_max_block_size.sql @@ -0,0 +1,2 @@ +SET max_block_size = 6; +SELECT blockSize() bs FROM (SELECT 1 s) ALL RIGHT JOIN (SELECT arrayJoin([2, 2, 2, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3]) s) USING (s) GROUP BY bs ORDER BY bs; From 2d71338b78486ce309d0179e99143f7c14838c4a Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 30 Nov 2018 18:34:24 +0300 Subject: [PATCH 42/69] examine structure of the destination table of Buffer storage under lock --- dbms/src/Storages/StorageBuffer.cpp | 5 ++-- ...k_buffer_alter_destination_table.reference | 0 ...763_lock_buffer_alter_destination_table.sh | 30 +++++++++++++++++++ 3 files changed, 33 insertions(+), 2 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00763_lock_buffer_alter_destination_table.reference create mode 100755 dbms/tests/queries/0_stateless/00763_lock_buffer_alter_destination_table.sh diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 447852ab3b5..8ec803a5dba 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -150,6 +150,8 @@ BlockInputStreams StorageBuffer::read( /// Collect columns from the destination tables which can be requested. /// Find out if there is a struct mismatch and we need to convert read blocks from the destination tables. + auto destination_lock = destination->lockStructure(false, __PRETTY_FUNCTION__); + Names columns_intersection; bool struct_mismatch = false; for (const String & column_name : column_names) @@ -179,10 +181,9 @@ BlockInputStreams StorageBuffer::read( << " has no common columns with block in buffer. Block of data is skipped."); else { - auto lock = destination->lockStructure(false, __PRETTY_FUNCTION__); streams_from_dst = destination->read(columns_intersection, query_info, context, processed_stage, max_block_size, num_streams); for (auto & stream : streams_from_dst) - stream->addTableLock(lock); + stream->addTableLock(destination_lock); } if (struct_mismatch && !streams_from_dst.empty()) diff --git a/dbms/tests/queries/0_stateless/00763_lock_buffer_alter_destination_table.reference b/dbms/tests/queries/0_stateless/00763_lock_buffer_alter_destination_table.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00763_lock_buffer_alter_destination_table.sh b/dbms/tests/queries/0_stateless/00763_lock_buffer_alter_destination_table.sh new file mode 100755 index 00000000000..cebb69ba44a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00763_lock_buffer_alter_destination_table.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.mt" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.buffer" + +${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.buffer (s String) ENGINE = Buffer(test, mt, 1, 1, 1, 1, 1, 1, 1)" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.mt (x UInt32, s String) ENGINE = MergeTree ORDER BY x" +${CLICKHOUSE_CLIENT} --query="INSERT INTO test.mt VALUES (1, '1'), (2, '2'), (3, '3')" + +function thread1() +{ + seq 1 300 | sed -r -e 's/.+/ALTER TABLE test.mt MODIFY column s UInt32; ALTER TABLE test.mt MODIFY column s String;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: +} + +function thread2() +{ + seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM test.buffer;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^3$' +} + +thread1 & +thread2 & + +wait + +${CLICKHOUSE_CLIENT} --query="DROP TABLE test.mt" +${CLICKHOUSE_CLIENT} --query="DROP TABLE test.buffer" From 35f6fb5ae6f723a8f4ba43a7fe186e561f2478e8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 30 Nov 2018 18:36:41 +0300 Subject: [PATCH 43/69] Avoid memory over-allocation when large value of "max_query_size" is specified [#CLICKHOUSE-4169] --- dbms/src/Common/ShellCommand.cpp | 1 - dbms/src/Functions/FunctionsConversion.h | 5 ++- dbms/src/Functions/FunctionsFormatting.h | 4 +-- dbms/src/IO/LimitReadBuffer.h | 1 + dbms/src/IO/WriteBufferFromString.h | 41 ++---------------------- dbms/src/IO/WriteBufferFromVector.h | 32 ++++++++++++++---- dbms/src/IO/tests/parse_int_perf.cpp | 2 +- dbms/src/Interpreters/executeQuery.cpp | 10 ++++-- 8 files changed, 41 insertions(+), 55 deletions(-) diff --git a/dbms/src/Common/ShellCommand.cpp b/dbms/src/Common/ShellCommand.cpp index 675dd8391f1..2e19d442f9a 100644 --- a/dbms/src/Common/ShellCommand.cpp +++ b/dbms/src/Common/ShellCommand.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index d23a8104efa..74428782f9a 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -312,8 +312,7 @@ struct ConvertImpl - -#include - -#define WRITE_BUFFER_FROM_STRING_INITIAL_SIZE_IF_EMPTY 32 +#include namespace DB @@ -13,41 +10,7 @@ namespace DB /** Writes the data to a string. * Note: before using the resulting string, destroy this object. */ -class WriteBufferFromString : public WriteBuffer -{ -private: - std::string & s; - - void nextImpl() override - { - size_t old_size = s.size(); - s.resize(old_size * 2); - internal_buffer = Buffer(reinterpret_cast(&s[old_size]), reinterpret_cast(&s[s.size()])); - working_buffer = internal_buffer; - } - -protected: - void finish() - { - s.resize(count()); - } - -public: - WriteBufferFromString(std::string & s_) - : WriteBuffer(reinterpret_cast(s_.data()), s_.size()), s(s_) - { - if (s.empty()) - { - s.resize(WRITE_BUFFER_FROM_STRING_INITIAL_SIZE_IF_EMPTY); - set(reinterpret_cast(s.data()), s.size()); - } - } - - ~WriteBufferFromString() override - { - finish(); - } -}; +using WriteBufferFromString = WriteBufferFromVector; namespace detail diff --git a/dbms/src/IO/WriteBufferFromVector.h b/dbms/src/IO/WriteBufferFromVector.h index e3b626127f3..452e39cdde2 100644 --- a/dbms/src/IO/WriteBufferFromVector.h +++ b/dbms/src/IO/WriteBufferFromVector.h @@ -5,20 +5,22 @@ #include -#define WRITE_BUFFER_FROM_VECTOR_INITIAL_SIZE_IF_EMPTY 32 - - namespace DB { -/** Initialized by vector. Writes data to it. When the vector is finished, it doubles its size. - * CharType - char or unsigned char. +/** Writes data to existing std::vector or similar type. When not enough space, it doubles vector size. + * + * In destructor, vector is cutted to the size of written data. + * You can call to 'finish' to resize earlier. + * + * The vector should live until this object is destroyed or until the 'finish' method is called. */ -template > +template class WriteBufferFromVector : public WriteBuffer { private: VectorType & vector; + bool is_finished = false; void nextImpl() override { @@ -34,10 +36,26 @@ public: { if (vector.empty()) { - vector.resize(WRITE_BUFFER_FROM_VECTOR_INITIAL_SIZE_IF_EMPTY); + static constexpr size_t initial_size = 32; + vector.resize(initial_size); set(reinterpret_cast(vector.data()), vector.size()); } } + + void finish() + { + is_finished = true; + vector.resize( + ((position() - reinterpret_cast(vector.data())) + + sizeof(typename VectorType::value_type) - 1) /// Align up. + / sizeof(typename VectorType::value_type)); + } + + ~WriteBufferFromVector() override + { + if (!is_finished) + finish(); + } }; } diff --git a/dbms/src/IO/tests/parse_int_perf.cpp b/dbms/src/IO/tests/parse_int_perf.cpp index ae6762d39c9..e3b40927a85 100644 --- a/dbms/src/IO/tests/parse_int_perf.cpp +++ b/dbms/src/IO/tests/parse_int_perf.cpp @@ -61,7 +61,7 @@ int main(int argc, char ** argv) formatted.reserve(n * 21); { - DB::WriteBufferFromVector<> wb(formatted); + DB::WriteBufferFromVector wb(formatted); // DB::CompressedWriteBuffer wb2(wb1); // DB::AsynchronousWriteBuffer wb(wb2); Stopwatch watch; diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 5c4f082eefd..abc8d25c349 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -3,6 +3,9 @@ #include #include +#include +#include +#include #include #include @@ -445,8 +448,11 @@ void executeQuery( else { /// If not - copy enough data into 'parse_buf'. - parse_buf.resize(max_query_size + 1); - parse_buf.resize(istr.read(parse_buf.data(), max_query_size + 1)); + WriteBufferFromVector> out(parse_buf); + LimitReadBuffer limit(istr, max_query_size + 1, false); + copyData(limit, out); + out.finish(); + begin = parse_buf.data(); end = begin + parse_buf.size(); } From 621e622f798b0254bf7f853945ac291db83a2446 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 30 Nov 2018 18:41:24 +0300 Subject: [PATCH 44/69] Added test [#CLICKHOUSE-4169] --- .../0_stateless/00764_max_query_size_allocation.reference | 1 + .../0_stateless/00764_max_query_size_allocation.sh | 8 ++++++++ 2 files changed, 9 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00764_max_query_size_allocation.reference create mode 100755 dbms/tests/queries/0_stateless/00764_max_query_size_allocation.sh diff --git a/dbms/tests/queries/0_stateless/00764_max_query_size_allocation.reference b/dbms/tests/queries/0_stateless/00764_max_query_size_allocation.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00764_max_query_size_allocation.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00764_max_query_size_allocation.sh b/dbms/tests/queries/0_stateless/00764_max_query_size_allocation.sh new file mode 100755 index 00000000000..b8490970aa1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00764_max_query_size_allocation.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}?max_query_size=1000000000&max_memory_usage=10000000" -d "SELECT 1" From 0798db6c8c2b139ce5c1fd3b4b420e7a48545386 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Fri, 30 Nov 2018 20:54:14 +0300 Subject: [PATCH 45/69] Add variables to limit the number of concurrent linkings. --- CMakeLists.txt | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index d215c05250a..c5ae0ecbea6 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -25,6 +25,18 @@ endif () # Write compile_commands.json set(CMAKE_EXPORT_COMPILE_COMMANDS 1) +set(PARALLEL_COMPILE_JOBS "" CACHE STRING "Define the maximum number of concurrent compilation jobs") +if (PARALLEL_COMPILE_JOBS) + set_property(GLOBAL APPEND PROPERTY JOB_POOLS compile_job_pool="${PARALLEL_COMPILE_JOBS}") + set(CMAKE_JOB_POOL_COMPILE compile_job_pool) +endif () + +set(PARALLEL_LINK_JOBS "" CACHE STRING "Define the maximum number of concurrent link jobs") +if (LLVM_PARALLEL_LINK_JOBS) + set_property(GLOBAL APPEND PROPERTY JOB_POOLS link_job_pool=${PARALLEL_LINK_JOBS}) + set(CMAKE_JOB_POOL_LINK link_job_pool) +endif () + include (cmake/find_ccache.cmake) if (NOT CMAKE_BUILD_TYPE OR CMAKE_BUILD_TYPE STREQUAL "None") From 57d1b133d39d60cc57eb11ce5b2f3829f71e7e96 Mon Sep 17 00:00:00 2001 From: KochetovNicolai Date: Fri, 30 Nov 2018 22:02:22 +0300 Subject: [PATCH 46/69] Update CHANGELOG_RU.md --- CHANGELOG_RU.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index 216d80f100f..6c1c262cc77 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -1,3 +1,12 @@ +## ClickHouse release 18.14.17, 2018-11-30 + +### Исправления ошибок: +* Исправлена ситуация, при которой ODBC Bridge продолжал работу после завершения работы сервера ClickHouse. Теперь ODBC Bridge всегда завершает работу вместе с сервером. [#3642](https://github.com/yandex/ClickHouse/pull/3642) +* Исправлена синхронная вставка в `Distributed` таблицу в случае явного указания неполного списка столбцов или списка столбцов в измененном порядке. [#3673](https://github.com/yandex/ClickHouse/pull/3673) +* Исправлен race condition в `BackgroundProcessingPoolTaskInfo` возникающий при попытке выполнения задачи одновременно с инвалидацией её итератора очереди. [#3680](https://github.com/yandex/ClickHouse/pull/3680) +* Исправлен deadlock в `ParallelInputsProcessor` возникающий в случае исключения при создании потока. [#3643](https://github.com/yandex/ClickHouse/pull/3643) +* Исправлена ошибка парсинга `Engine` при создании таблицы с синтаксисом `AS table` в случае, когда `AS table` указывался после `Engine`, что приводило к игнорированию указанного движка. [#3692](https://github.com/yandex/ClickHouse/pull/3692) + ## ClickHouse release 18.14.15, 2018-11-21 ### Исправления ошибок: From d3894b52df6e45630340296c84b76efacc3dc860 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=A2=E7=A3=8A?= <2217232293@qq.com> Date: Sat, 1 Dec 2018 03:26:35 +0800 Subject: [PATCH 47/69] [WIP] translate about table-engines (#3660) * init zh/operations translate * finish table_engines about Integrations part * add table_engine index --- docs/zh/operations/access_rights.md | 104 ++- docs/zh/operations/configuration_files.md | 45 +- docs/zh/operations/index.md | 5 +- docs/zh/operations/quotas.md | 109 ++- docs/zh/operations/server_settings/index.md | 15 +- .../zh/operations/server_settings/settings.md | 722 +++++++++++++++++- docs/zh/operations/settings/index.md | 27 +- .../operations/settings/query_complexity.md | 198 ++++- docs/zh/operations/settings/settings.md | 422 +++++++++- .../operations/settings/settings_profiles.md | 68 +- docs/zh/operations/system_tables.md | 440 ++++++++++- .../table_engines/aggregatingmergetree.md | 96 ++- docs/zh/operations/table_engines/buffer.md | 57 +- .../table_engines/collapsingmergetree.md | 221 +++++- .../table_engines/custom_partitioning_key.md | 50 +- .../zh/operations/table_engines/dictionary.md | 113 ++- .../operations/table_engines/distributed.md | 127 ++- .../operations/table_engines/external_data.md | 65 +- docs/zh/operations/table_engines/file.md | 81 +- .../table_engines/graphitemergetree.md | 150 +++- docs/zh/operations/table_engines/join.md | 20 +- docs/zh/operations/table_engines/kafka.md | 140 +++- docs/zh/operations/table_engines/log.md | 9 +- .../table_engines/materializedview.md | 7 +- docs/zh/operations/table_engines/memory.md | 14 +- docs/zh/operations/table_engines/merge.md | 70 +- docs/zh/operations/table_engines/mergetree.md | 210 ++++- docs/zh/operations/table_engines/mysql.md | 28 +- docs/zh/operations/table_engines/null.md | 9 +- .../table_engines/replacingmergetree.md | 59 +- .../operations/table_engines/replication.md | 207 ++++- docs/zh/operations/table_engines/set.md | 14 +- .../table_engines/summingmergetree.md | 136 +++- docs/zh/operations/table_engines/tinylog.md | 22 +- docs/zh/operations/table_engines/url.md | 76 +- docs/zh/operations/table_engines/view.md | 7 +- docs/zh/operations/tips.md | 260 ++++++- docs/zh/operations/utils/clickhouse-copier.md | 164 +++- docs/zh/operations/utils/clickhouse-local.md | 76 +- docs/zh/operations/utils/index.md | 8 +- 40 files changed, 4611 insertions(+), 40 deletions(-) mode change 120000 => 100644 docs/zh/operations/access_rights.md mode change 120000 => 100644 docs/zh/operations/configuration_files.md mode change 120000 => 100644 docs/zh/operations/index.md mode change 120000 => 100644 docs/zh/operations/quotas.md mode change 120000 => 100644 docs/zh/operations/server_settings/index.md mode change 120000 => 100644 docs/zh/operations/server_settings/settings.md mode change 120000 => 100644 docs/zh/operations/settings/index.md mode change 120000 => 100644 docs/zh/operations/settings/query_complexity.md mode change 120000 => 100644 docs/zh/operations/settings/settings.md mode change 120000 => 100644 docs/zh/operations/settings/settings_profiles.md mode change 120000 => 100644 docs/zh/operations/system_tables.md mode change 120000 => 100644 docs/zh/operations/table_engines/aggregatingmergetree.md mode change 120000 => 100644 docs/zh/operations/table_engines/buffer.md mode change 120000 => 100644 docs/zh/operations/table_engines/collapsingmergetree.md mode change 120000 => 100644 docs/zh/operations/table_engines/custom_partitioning_key.md mode change 120000 => 100644 docs/zh/operations/table_engines/dictionary.md mode change 120000 => 100644 docs/zh/operations/table_engines/distributed.md mode change 120000 => 100644 docs/zh/operations/table_engines/external_data.md mode change 120000 => 100644 docs/zh/operations/table_engines/file.md mode change 120000 => 100644 docs/zh/operations/table_engines/graphitemergetree.md mode change 120000 => 100644 docs/zh/operations/table_engines/join.md mode change 120000 => 100644 docs/zh/operations/table_engines/kafka.md mode change 120000 => 100644 docs/zh/operations/table_engines/log.md mode change 120000 => 100644 docs/zh/operations/table_engines/materializedview.md mode change 120000 => 100644 docs/zh/operations/table_engines/memory.md mode change 120000 => 100644 docs/zh/operations/table_engines/merge.md mode change 120000 => 100644 docs/zh/operations/table_engines/mergetree.md mode change 120000 => 100644 docs/zh/operations/table_engines/mysql.md mode change 120000 => 100644 docs/zh/operations/table_engines/null.md mode change 120000 => 100644 docs/zh/operations/table_engines/replacingmergetree.md mode change 120000 => 100644 docs/zh/operations/table_engines/replication.md mode change 120000 => 100644 docs/zh/operations/table_engines/set.md mode change 120000 => 100644 docs/zh/operations/table_engines/summingmergetree.md mode change 120000 => 100644 docs/zh/operations/table_engines/tinylog.md mode change 120000 => 100644 docs/zh/operations/table_engines/url.md mode change 120000 => 100644 docs/zh/operations/table_engines/view.md mode change 120000 => 100644 docs/zh/operations/tips.md mode change 120000 => 100644 docs/zh/operations/utils/clickhouse-copier.md mode change 120000 => 100644 docs/zh/operations/utils/clickhouse-local.md mode change 120000 => 100644 docs/zh/operations/utils/index.md diff --git a/docs/zh/operations/access_rights.md b/docs/zh/operations/access_rights.md deleted file mode 120000 index 73463029569..00000000000 --- a/docs/zh/operations/access_rights.md +++ /dev/null @@ -1 +0,0 @@ -../../en/operations/access_rights.md \ No newline at end of file diff --git a/docs/zh/operations/access_rights.md b/docs/zh/operations/access_rights.md new file mode 100644 index 00000000000..451be2c7322 --- /dev/null +++ b/docs/zh/operations/access_rights.md @@ -0,0 +1,103 @@ +# Access Rights + +Users and access rights are set up in the user config. This is usually `users.xml`. + +Users are recorded in the `users` section. Here is a fragment of the `users.xml` file: + +```xml + + + + + + + + + + + + default + + + default + + + + + + + web + default + + test + + +``` + +You can see a declaration from two users: `default`and`web`. We added the `web` user separately. + +The `default` user is chosen in cases when the username is not passed. The `default` user is also used for distributed query processing, if the configuration of the server or cluster doesn't specify the `user` and `password` (see the section on the [Distributed](../operations/table_engines/distributed.md#table_engines-distributed) engine). + +The user that is used for exchanging information between servers combined in a cluster must not have substantial restrictions or quotas – otherwise, distributed queries will fail. + +The password is specified in clear text (not recommended) or in SHA-256. The hash isn't salted. In this regard, you should not consider these passwords as providing security against potential malicious attacks. Rather, they are necessary for protection from employees. + +A list of networks is specified that access is allowed from. In this example, the list of networks for both users is loaded from a separate file (`/etc/metrika.xml`) containing the `networks` substitution. Here is a fragment of it: + +```xml + + ... + + ::/64 + 203.0.113.0/24 + 2001:DB8::/32 + ... + + +``` + +You could define this list of networks directly in `users.xml`, or in a file in the `users.d` directory (for more information, see the section "[Configuration files](configuration_files.md#configuration_files)"). + +The config includes comments explaining how to open access from everywhere. + +For use in production, only specify `ip` elements (IP addresses and their masks), since using `host` and `hoost_regexp` might cause extra latency. + +Next the user settings profile is specified (see the section "[Settings profiles](settings/settings_profiles.md#settings_profiles)"). You can specify the default profile, `default'`. The profile can have any name. You can specify the same profile for different users. The most important thing you can write in the settings profile is `readonly=1`, which ensures read-only access. + +Then specify the quota to be used (see the section "[Quotas](quotas.md#quotas)"). You can specify the default quota: `default`. It is set in the config by default to only count resource usage, without restricting it. The quota can have any name. You can specify the same quota for different users – in this case, resource usage is calculated for each user individually. + +In the optional `` section, you can also specify a list of databases that the user can access. By default, all databases are available to the user. You can specify the `default` database. In this case, the user will receive access to the database by default. + +Access to the `system` database is always allowed (since this database is used for processing queries). + +The user can get a list of all databases and tables in them by using `SHOW` queries or system tables, even if access to individual databases isn't allowed. + +Database access is not related to the [readonly](settings/query_complexity.md#query_complexity_readonly) setting. You can't grant full access to one database and `readonly` access to another one. + + +[Original article](https://clickhouse.yandex/docs/en/operations/access_rights/) diff --git a/docs/zh/operations/configuration_files.md b/docs/zh/operations/configuration_files.md deleted file mode 120000 index a2d73dbaa25..00000000000 --- a/docs/zh/operations/configuration_files.md +++ /dev/null @@ -1 +0,0 @@ -../../en/operations/configuration_files.md \ No newline at end of file diff --git a/docs/zh/operations/configuration_files.md b/docs/zh/operations/configuration_files.md new file mode 100644 index 00000000000..a7cdfb124ee --- /dev/null +++ b/docs/zh/operations/configuration_files.md @@ -0,0 +1,44 @@ + + +# Configuration Files + +The main server config file is `config.xml`. It resides in the `/etc/clickhouse-server/` directory. + +Individual settings can be overridden in the `*.xml` and `*.conf` files in the `conf.d` and `config.d` directories next to the config file. + +The `replace` or `remove` attributes can be specified for the elements of these config files. + +If neither is specified, it combines the contents of elements recursively, replacing values of duplicate children. + +If `replace` is specified, it replaces the entire element with the specified one. + +If `remove` is specified, it deletes the element. + +The config can also define "substitutions". If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](server_settings/settings.md#server_settings-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros]() server_settings/settings.md#server_settings-macros)). + +Substitutions can also be performed from ZooKeeper. To do this, specify the attribute `from_zk = "/path/to/node"`. The element value is replaced with the contents of the node at `/path/to/node` in ZooKeeper. You can also put an entire XML subtree on the ZooKeeper node and it will be fully inserted into the source element. + +The `config.xml` file can specify a separate config with user settings, profiles, and quotas. The relative path to this config is set in the 'users_config' element. By default, it is `users.xml`. If `users_config` is omitted, the user settings, profiles, and quotas are specified directly in `config.xml`. + +In addition, `users_config` may have overrides in files from the `users_config.d` directory (for example, `users.d`) and substitutions. For example, you can have separate config file for each user like this: +``` xml +$ cat /etc/clickhouse-server/users.d/alice.xml + + + + analytics + + ::/0 + + ... + analytics + + + +``` + +For each config file, the server also generates `file-preprocessed.xml` files when starting. These files contain all the completed substitutions and overrides, and they are intended for informational use. If ZooKeeper substitutions were used in the config files but ZooKeeper is not available on the server start, the server loads the configuration from the preprocessed file. + +The server tracks changes in config files, as well as files and ZooKeeper nodes that were used when performing substitutions and overrides, and reloads the settings for users and clusters on the fly. This means that you can modify the cluster, users, and their settings without restarting the server. + +[Original article](https://clickhouse.yandex/docs/en/operations/configuration_files/) diff --git a/docs/zh/operations/index.md b/docs/zh/operations/index.md deleted file mode 120000 index ce854687b86..00000000000 --- a/docs/zh/operations/index.md +++ /dev/null @@ -1 +0,0 @@ -../../en/operations/index.md \ No newline at end of file diff --git a/docs/zh/operations/index.md b/docs/zh/operations/index.md new file mode 100644 index 00000000000..63cb19bb639 --- /dev/null +++ b/docs/zh/operations/index.md @@ -0,0 +1,4 @@ +# Operations + + +[Original article](https://clickhouse.yandex/docs/en/operations/) diff --git a/docs/zh/operations/quotas.md b/docs/zh/operations/quotas.md deleted file mode 120000 index 1c52cdf1e91..00000000000 --- a/docs/zh/operations/quotas.md +++ /dev/null @@ -1 +0,0 @@ -../../en/operations/quotas.md \ No newline at end of file diff --git a/docs/zh/operations/quotas.md b/docs/zh/operations/quotas.md new file mode 100644 index 00000000000..148cb39458c --- /dev/null +++ b/docs/zh/operations/quotas.md @@ -0,0 +1,108 @@ + + +# Quotas + +Quotas allow you to limit resource usage over a period of time, or simply track the use of resources. +Quotas are set up in the user config. This is usually 'users.xml'. + +The system also has a feature for limiting the complexity of a single query. See the section "Restrictions on query complexity"). + +In contrast to query complexity restrictions, quotas: + +- Place restrictions on a set of queries that can be run over a period of time, instead of limiting a single query. +- Account for resources spent on all remote servers for distributed query processing. + +Let's look at the section of the 'users.xml' file that defines quotas. + +```xml + + + + + + + + 3600 + + + 0 + 0 + 0 + 0 + 0 + + +``` + +By default, the quota just tracks resource consumption for each hour, without limiting usage. +The resource consumption calculated for each interval is output to the server log after each request. + +```xml + + + + + 3600 + + 1000 + 100 + 1000000000 + 100000000000 + 900 + + + + 86400 + + 10000 + 1000 + 5000000000 + 500000000000 + 7200 + + +``` + +For the 'statbox' quota, restrictions are set for every hour and for every 24 hours (86,400 seconds). The time interval is counted starting from an implementation-defined fixed moment in time. In other words, the 24-hour interval doesn't necessarily begin at midnight. + +When the interval ends, all collected values are cleared. For the next hour, the quota calculation starts over. + +Here are the amounts that can be restricted: + +`queries` – The total number of requests. + +`errors` – The number of queries that threw an exception. + +`result_rows` – The total number of rows given as the result. + +`read_rows` – The total number of source rows read from tables for running the query, on all remote servers. + +`execution_time` – The total query execution time, in seconds (wall time). + +If the limit is exceeded for at least one time interval, an exception is thrown with a text about which restriction was exceeded, for which interval, and when the new interval begins (when queries can be sent again). + +Quotas can use the "quota key" feature in order to report on resources for multiple keys independently. Here is an example of this: + +```xml + + + + +``` + +The quota is assigned to users in the 'users' section of the config. See the section "Access rights". + +For distributed query processing, the accumulated amounts are stored on the requestor server. So if the user goes to another server, the quota there will "start over". + +When the server is restarted, quotas are reset. + + +[Original article](https://clickhouse.yandex/docs/en/operations/quotas/) diff --git a/docs/zh/operations/server_settings/index.md b/docs/zh/operations/server_settings/index.md deleted file mode 120000 index 1d1a0585a42..00000000000 --- a/docs/zh/operations/server_settings/index.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/server_settings/index.md \ No newline at end of file diff --git a/docs/zh/operations/server_settings/index.md b/docs/zh/operations/server_settings/index.md new file mode 100644 index 00000000000..88f11c48f4b --- /dev/null +++ b/docs/zh/operations/server_settings/index.md @@ -0,0 +1,14 @@ + + +# Server configuration parameters + +This section contains descriptions of server settings that cannot be changed at the session or query level. + +These settings are stored in the `config.xml` file on the ClickHouse server. + +Other settings are described in the "[Settings](../settings/index.md#settings)" section. + +Before studying the settings, read the [Configuration files](../configuration_files.md#configuration_files) section and note the use of substitutions (the `incl` and `optional` attributes). + + +[Original article](https://clickhouse.yandex/docs/en/operations/server_settings/) diff --git a/docs/zh/operations/server_settings/settings.md b/docs/zh/operations/server_settings/settings.md deleted file mode 120000 index 19cd2e82ce7..00000000000 --- a/docs/zh/operations/server_settings/settings.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/server_settings/settings.md \ No newline at end of file diff --git a/docs/zh/operations/server_settings/settings.md b/docs/zh/operations/server_settings/settings.md new file mode 100644 index 00000000000..8e10969ed6b --- /dev/null +++ b/docs/zh/operations/server_settings/settings.md @@ -0,0 +1,721 @@ +# Server settings + + + +## builtin_dictionaries_reload_interval + +The interval in seconds before reloading built-in dictionaries. + +ClickHouse reloads built-in dictionaries every x seconds. This makes it possible to edit dictionaries "on the fly" without restarting the server. + +Default value: 3600. + +**Example** + +```xml +3600 +``` + + + +## compression + +Data compression settings. + +!!! warning + Don't use it if you have just started using ClickHouse. + +The configuration looks like this: + +```xml + + + + + ... + +``` + +You can configure multiple sections ``. + +Block field ``: + +- ``min_part_size`` – The minimum size of a table part. +- ``min_part_size_ratio`` – The ratio of the minimum size of a table part to the full size of the table. +- ``method`` – Compression method. Acceptable values ​: ``lz4`` or ``zstd``(experimental). + +ClickHouse checks `min_part_size` and `min_part_size_ratio` and processes the `case` blocks that match these conditions. If none of the `` matches, ClickHouse applies the `lz4` compression algorithm. + +**Example** + +```xml + + + 10000000000 + 0.01 + zstd + + +``` + + + +## default_database + +The default database. + +To get a list of databases, use the [ SHOW DATABASES](../../query_language/misc.md#query_language_queries_show_databases) query. + +**Example** + +```xml +default +``` + + + +## default_profile + +Default settings profile. + +Settings profiles are located in the file specified in the parameter [user_config](#server_settings-users_config). + +**Example** + +```xml +default +``` + + + +## dictionaries_config + +The path to the config file for external dictionaries. + +Path: + +- Specify the absolute path or the path relative to the server config file. +- The path can contain wildcards \* and ?. + +See also "[External dictionaries](../../query_language/dicts/external_dicts.md#dicts-external_dicts)". + +**Example** + +```xml +*_dictionary.xml +``` + + + +## dictionaries_lazy_load + +Lazy loading of dictionaries. + +If `true`, then each dictionary is created on first use. If dictionary creation failed, the function that was using the dictionary throws an exception. + +If `false`, all dictionaries are created when the server starts, and if there is an error, the server shuts down. + +The default is `true`. + +**Example** + +```xml +true +``` + + + +## format_schema_path + +The path to the directory with the schemes for the input data, such as schemas for the [CapnProto](../../interfaces/formats.md#format_capnproto) format. + +**Example** + +```xml + + format_schemas/ +``` + + + +## graphite + +Sending data to [Graphite](https://github.com/graphite-project). + +Settings: + +- host – The Graphite server. +- port – The port on the Graphite server. +- interval – The interval for sending, in seconds. +- timeout – The timeout for sending data, in seconds. +- root_path – Prefix for keys. +- metrics – Sending data from a :ref:`system_tables-system.metrics` table. +- events – Sending data from a :ref:`system_tables-system.events` table. +- asynchronous_metrics – Sending data from a :ref:`system_tables-system.asynchronous_metrics` table. + +You can configure multiple `` clauses. For instance, you can use this for sending different data at different intervals. + +**Example** + +```xml + + localhost + 42000 + 0.1 + 60 + one_min + true + true + true + +``` + + + +## graphite_rollup + +Settings for thinning data for Graphite. + +For more details, see [GraphiteMergeTree](../../operations/table_engines/graphitemergetree.md#table_engines-graphitemergetree). + +**Example** + +```xml + + + max + + 0 + 60 + + + 3600 + 300 + + + 86400 + 3600 + + + +``` + + + +## http_port/https_port + +The port for connecting to the server over HTTP(s). + +If `https_port` is specified, [openSSL](#server_settings-openSSL) must be configured. + +If `http_port` is specified, the openSSL configuration is ignored even if it is set. + +**Example** + +```xml +0000 +``` + + + +## http_server_default_response + +The page that is shown by default when you access the ClickHouse HTTP(s) server. + +**Example** + +Opens `https://tabix.io/` when accessing ` http://localhost: http_port`. + +```xml + +
]]> +
+``` + + + +## include_from + +The path to the file with substitutions. + +For more information, see the section "[Configuration files](../configuration_files.md#configuration_files)". + +**Example** + +```xml +/etc/metrica.xml +``` + + + +## interserver_http_port + +Port for exchanging data between ClickHouse servers. + +**Example** + +```xml +9009 +``` + + + +## interserver_http_host + +The host name that can be used by other servers to access this server. + +If omitted, it is defined in the same way as the `hostname-f` command. + +Useful for breaking away from a specific network interface. + +**Example** + +```xml +example.yandex.ru +``` + + + +## keep_alive_timeout + +The number of seconds that ClickHouse waits for incoming requests before closing the connection. Defaults to 10 seconds + +**Example** + +```xml +10 +``` + + + +## listen_host + +Restriction on hosts that requests can come from. If you want the server to answer all of them, specify `::`. + +Examples: + +```xml +::1 +127.0.0.1 +``` + + + +## logger + +Logging settings. + +Keys: + +- level – Logging level. Acceptable values: ``trace``, ``debug``, ``information``, ``warning``, ``error``. +- log – The log file. Contains all the entries according to `level`. +- errorlog – Error log file. +- size – Size of the file. Applies to ``log``and``errorlog``. Once the file reaches ``size``, ClickHouse archives and renames it, and creates a new log file in its place. +- count – The number of archived log files that ClickHouse stores. + +**Example** + +```xml + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + +``` + +Writing to the syslog is also supported. Config example: + +```xml + + 1 + +
syslog.remote:10514
+ myhost.local + LOG_LOCAL6 + syslog +
+
+``` + +Keys: + +- user_syslog — Required setting if you want to write to the syslog. +- address — The host[:порт] of syslogd. If omitted, the local daemon is used. +- hostname — Optional. The name of the host that logs are sent from. +- facility — [The syslog facility keyword](https://en.wikipedia.org/wiki/Syslog#Facility) +in uppercase letters with the "LOG_" prefix: (``LOG_USER``, ``LOG_DAEMON``, ``LOG_LOCAL3``, and so on). +Default value: ``LOG_USER`` if ``address`` is specified, ``LOG_DAEMON otherwise.`` +- format – Message format. Possible values: ``bsd`` and ``syslog.`` + + + +## macros + +Parameter substitutions for replicated tables. + +Can be omitted if replicated tables are not used. + +For more information, see the section "[Creating replicated tables](../../operations/table_engines/replication.md#table_engines-replication-creation_of_rep_tables)". + +**Example** + +```xml + +``` + + + +## mark_cache_size + +Approximate size (in bytes) of the cache of "marks" used by [MergeTree](../../operations/table_engines/mergetree.md#table_engines-mergetree). + +The cache is shared for the server and memory is allocated as needed. The cache size must be at least 5368709120. + +**Example** + +```xml +5368709120 +``` + + + +## max_concurrent_queries + +The maximum number of simultaneously processed requests. + +**Example** + +```xml +100 +``` + + + +## max_connections + +The maximum number of inbound connections. + +**Example** + +```xml +4096 +``` + + + +## max_open_files + +The maximum number of open files. + +By default: `maximum`. + +We recommend using this option in Mac OS X, since the `getrlimit()` function returns an incorrect value. + +**Example** + +```xml +262144 +``` + + + +## max_table_size_to_drop + +Restriction on deleting tables. + +If the size of a [MergeTree](../../operations/table_engines/mergetree.md#table_engines-mergetree) table exceeds `max_table_size_to_drop` (in bytes), you can't delete it using a DROP query. + +If you still need to delete the table without restarting the ClickHouse server, create the `/flags/force_drop_table` file and run the DROP query. + +Default value: 50 GB. + +The value 0 means that you can delete all tables without any restrictions. + +**Example** + +```xml +0 +``` + + + +## merge_tree + +Fine tuning for tables in the [ MergeTree](../../operations/table_engines/mergetree.md#table_engines-mergetree). + +For more information, see the MergeTreeSettings.h header file. + +**Example** + +```xml + + 5 + +``` + + + +## openSSL + +SSL client/server configuration. + +Support for SSL is provided by the `libpoco` library. The interface is described in the file [SSLManager.h](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h) + +Keys for server/client settings: + +- privateKeyFile – The path to the file with the secret key of the PEM certificate. The file may contain a key and certificate at the same time. +- certificateFile – The path to the client/server certificate file in PEM format. You can omit it if `privateKeyFile` contains the certificate. +- caConfig – The path to the file or directory that contains trusted root certificates. +- verificationMode – The method for checking the node's certificates. Details are in the description of the [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) class. Possible values: ``none``, ``relaxed``, ``strict``, ``once``. +- verificationDepth – The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value. +- loadDefaultCAFile – Indicates that built-in CA certificates for OpenSSL will be used. Acceptable values: `true`, `false`. | +- cipherList – Supported OpenSSL encryptions. For example: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`. +- cacheSessions – Enables or disables caching sessions. Must be used in combination with ``sessionIdContext``. Acceptable values: `true`, `false`. +- sessionIdContext – A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed ``SSL_MAX_SSL_SESSION_ID_LENGTH``. This parameter is always recommended, since it helps avoid problems both if the server caches the session and if the client requested caching. Default value: ``${application.name}``. +- sessionCacheSize – The maximum number of sessions that the server caches. Default value: 1024\*20. 0 – Unlimited sessions. +- sessionTimeout – Time for caching the session on the server. +- extendedVerification – Automatically extended verification of certificates after the session ends. Acceptable values: `true`, `false`. +- requireTLSv1 – Require a TLSv1 connection. Acceptable values: `true`, `false`. +- requireTLSv1_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. +- requireTLSv1 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. +- fips – Activates OpenSSL FIPS mode. Supported if the library's OpenSSL version supports FIPS. +- privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ````, ``KeyFileHandler``, ``test``, ````. +- invalidCertificateHandler – Class (subclass of CertificateHandler) for verifying invalid certificates. For example: `` ConsoleCertificateHandler `` . +- disableProtocols – Protocols that are not allowed to use. +- preferServerCiphers – Preferred server ciphers on the client. + +**Example of settings:** + +```xml + + + + /etc/clickhouse-server/server.crt + /etc/clickhouse-server/server.key + + /etc/clickhouse-server/dhparam.pem + none + true + true + sslv2,sslv3 + true + + + true + true + sslv2,sslv3 + true + + + + RejectCertificateHandler + + + +``` + + + +## part_log + +Logging events that are associated with [MergeTree](../../operations/table_engines/mergetree.md#table_engines-mergetree). For instance, adding or merging data. You can use the log to simulate merge algorithms and compare their characteristics. You can visualize the merge process. + +Queries are logged in the ClickHouse table, not in a separate file. + +Columns in the log: + +- event_time – Date of the event. +- duration_ms – Duration of the event. +- event_type – Type of event. 1 – new data part; 2 – merge result; 3 – data part downloaded from replica; 4 – data part deleted. +- database_name – The name of the database. +- table_name – Name of the table. +- part_name – Name of the data part. +- size_in_bytes – Size of the data part in bytes. +- merged_from – An array of names of data parts that make up the merge (also used when downloading a merged part). +- merge_time_ms – Time spent on the merge. + +Use the following parameters to configure logging: + +- database – Name of the database. +- table – Name of the table. +- partition_by – Sets a [custom partitioning key](../../operations/table_engines/custom_partitioning_key.md#custom-partitioning-key). +- flush_interval_milliseconds – Interval for flushing data from memory to the disk. + +**Example** + +```xml + + system + part_log
+ toMonday(event_date) + 7500 +
+``` + + + +## path + +The path to the directory containing data. + +!!! note + The trailing slash is mandatory. + +**Example** + +```xml +/var/lib/clickhouse/ +``` + + + +## query_log + +Setting for logging queries received with the [log_queries=1](../settings/settings.md#settings_settings-log_queries) setting. + +Queries are logged in the ClickHouse table, not in a separate file. + +Use the following parameters to configure logging: + +- database – Name of the database. +- table – Name of the table. +- partition_by – Sets a [custom partitioning key](../../operations/table_engines/custom_partitioning_key.md#custom-partitioning-key). +- flush_interval_milliseconds – Interval for flushing data from memory to the disk. + +If the table doesn't exist, ClickHouse will create it. If the structure of the query log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. + +**Example** + +```xml + + system + query_log
+ toMonday(event_date) + 7500 +
+``` + + + +## remote_servers + +Configuration of clusters used by the Distributed table engine. + +For more information, see the section "[Table engines/Distributed](../../operations/table_engines/distributed.md#table_engines-distributed)". + +**Example** + +```xml + +``` + +For the value of the `incl` attribute, see the section "[Configuration files](../configuration_files.md#configuration_files)". + + + +## timezone + +The server's time zone. + +Specified as an IANA identifier for the UTC time zone or geographic location (for example, Africa/Abidjan). + +The time zone is necessary for conversions between String and DateTime formats when DateTime fields are output to text format (printed on the screen or in a file), and when getting DateTime from a string. In addition, the time zone is used in functions that work with the time and date if they didn't receive the time zone in the input parameters. + +**Example** + +```xml +Europe/Moscow +``` + + + +## tcp_port + +Port for communicating with clients over the TCP protocol. + +**Example** + +```xml +9000 +``` + + + +## tmp_path + +Path to temporary data for processing large queries. + +!!! note + The trailing slash is mandatory. + +**Example** + +```xml +/var/lib/clickhouse/tmp/ +``` + + + +## uncompressed_cache_size + +Cache size (in bytes) for uncompressed data used by table engines from the [MergeTree](../../operations/table_engines/mergetree.md#table_engines-mergetree). + +There is one shared cache for the server. Memory is allocated on demand. The cache is used if the option [use_uncompressed_cache](../settings/settings.md#settings-use_uncompressed_cache) is enabled. + +The uncompressed cache is advantageous for very short queries in individual cases. + +**Example** + +```xml +8589934592 +``` + +## user_files_path + +The directory with user files. Used in the table function [file()](../../query_language/table_functions/file.md#table_functions-file). + +**Example** + +```xml +/var/lib/clickhouse/user_files/ +``` + + + +## users_config + +Path to the file that contains: + +- User configurations. +- Access rights. +- Settings profiles. +- Quota settings. + +**Example** + +```xml +users.xml +``` + + + +## zookeeper + +Configuration of ZooKeeper servers. + +ClickHouse uses ZooKeeper for storing replica metadata when using replicated tables. + +This parameter can be omitted if replicated tables are not used. + +For more information, see the section "[Replication](../../operations/table_engines/replication.md#table_engines-replication)". + +**Example** + +```xml + +``` + + +[Original article](https://clickhouse.yandex/docs/en/operations/server_settings/settings/) diff --git a/docs/zh/operations/settings/index.md b/docs/zh/operations/settings/index.md deleted file mode 120000 index fc3968d1f1e..00000000000 --- a/docs/zh/operations/settings/index.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/settings/index.md \ No newline at end of file diff --git a/docs/zh/operations/settings/index.md b/docs/zh/operations/settings/index.md new file mode 100644 index 00000000000..5676796fd10 --- /dev/null +++ b/docs/zh/operations/settings/index.md @@ -0,0 +1,26 @@ + + +# Settings + +There are multiple ways to make all the settings described below. +Settings are configured in layers, so each subsequent layer redefines the previous settings. + +Ways to configure settings, in order of priority: + +- Settings in the `users.xml` server configuration file. + + Set in the element ``. + +- Session settings. + + Send ` SET setting=value` from the ClickHouse console client in interactive mode. +Similarly, you can use ClickHouse sessions in the HTTP protocol. To do this, you need to specify the `session_id` HTTP parameter. + +- Query settings. + - When starting the ClickHouse console client in non-interactive mode, set the startup parameter `--setting=value`. + - When using the HTTP API, pass CGI parameters (`URL?setting_1=value&setting_2=value...`). + +Settings that can only be made in the server config file are not covered in this section. + + +[Original article](https://clickhouse.yandex/docs/en/operations/settings/) diff --git a/docs/zh/operations/settings/query_complexity.md b/docs/zh/operations/settings/query_complexity.md deleted file mode 120000 index 9a9c6d975a9..00000000000 --- a/docs/zh/operations/settings/query_complexity.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/settings/query_complexity.md \ No newline at end of file diff --git a/docs/zh/operations/settings/query_complexity.md b/docs/zh/operations/settings/query_complexity.md new file mode 100644 index 00000000000..9e49dc58ca3 --- /dev/null +++ b/docs/zh/operations/settings/query_complexity.md @@ -0,0 +1,197 @@ +# Restrictions on query complexity + +Restrictions on query complexity are part of the settings. +They are used in order to provide safer execution from the user interface. +Almost all the restrictions only apply to SELECTs.For distributed query processing, restrictions are applied on each server separately. + +Restrictions on the "maximum amount of something" can take the value 0, which means "unrestricted". +Most restrictions also have an 'overflow_mode' setting, meaning what to do when the limit is exceeded. +It can take one of two values: `throw` or `break`. Restrictions on aggregation (group_by_overflow_mode) also have the value `any`. + +`throw` – Throw an exception (default). + +`break` – Stop executing the query and return the partial result, as if the source data ran out. + +`any (only for group_by_overflow_mode)` – Continuing aggregation for the keys that got into the set, but don't add new keys to the set. + + + +## readonly + +With a value of 0, you can execute any queries. +With a value of 1, you can only execute read requests (such as SELECT and SHOW). Requests for writing and changing settings (INSERT, SET) are prohibited. +With a value of 2, you can process read queries (SELECT, SHOW) and change settings (SET). + +After enabling readonly mode, you can't disable it in the current session. + +When using the GET method in the HTTP interface, 'readonly = 1' is set automatically. In other words, for queries that modify data, you can only use the POST method. You can send the query itself either in the POST body, or in the URL parameter. + + + +## max_memory_usage + +The maximum amount of RAM to use for running a query on a single server. + +In the default configuration file, the maximum is 10 GB. + +The setting doesn't consider the volume of available memory or the total volume of memory on the machine. +The restriction applies to a single query within a single server. +You can use `SHOW PROCESSLIST` to see the current memory consumption for each query. +In addition, the peak memory consumption is tracked for each query and written to the log. + +Memory usage is not monitored for the states of certain aggregate functions. + +Memory usage is not fully tracked for states of the aggregate functions `min`, `max`, `any`, `anyLast`, `argMin`, `argMax` from `String` and `Array` arguments. + +Memory consumption is also restricted by the parameters `max_memory_usage_for_user` and `max_memory_usage_for_all_queries`. + +## max_memory_usage_for_user + +The maximum amount of RAM to use for running a user's queries on a single server. + +Default values are defined in [Settings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Interpreters/Settings.h#L244). By default, the amount is not restricted (`max_memory_usage_for_user = 0`). + +See also the description of [max_memory_usage](#settings_max_memory_usage). + +## max_memory_usage_for_all_queries + +The maximum amount of RAM to use for running all queries on a single server. + +Default values are defined in [Settings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Interpreters/Settings.h#L245). By default, the amount is not restricted (`max_memory_usage_for_all_queries = 0`). + +See also the description of [max_memory_usage](#settings_max_memory_usage). + +## max_rows_to_read + +The following restrictions can be checked on each block (instead of on each row). That is, the restrictions can be broken a little. +When running a query in multiple threads, the following restrictions apply to each thread separately. + +Maximum number of rows that can be read from a table when running a query. + +## max_bytes_to_read + +Maximum number of bytes (uncompressed data) that can be read from a table when running a query. + +## read_overflow_mode + +What to do when the volume of data read exceeds one of the limits: 'throw' or 'break'. By default, throw. + +## max_rows_to_group_by + +Maximum number of unique keys received from aggregation. This setting lets you limit memory consumption when aggregating. + +## group_by_overflow_mode + +What to do when the number of unique keys for aggregation exceeds the limit: 'throw', 'break', or 'any'. By default, throw. +Using the 'any' value lets you run an approximation of GROUP BY. The quality of this approximation depends on the statistical nature of the data. + +## max_rows_to_sort + +Maximum number of rows before sorting. This allows you to limit memory consumption when sorting. + +## max_bytes_to_sort + +Maximum number of bytes before sorting. + +## sort_overflow_mode + +What to do if the number of rows received before sorting exceeds one of the limits: 'throw' or 'break'. By default, throw. + +## max_result_rows + +Limit on the number of rows in the result. Also checked for subqueries, and on remote servers when running parts of a distributed query. + +## max_result_bytes + +Limit on the number of bytes in the result. The same as the previous setting. + +## result_overflow_mode + +What to do if the volume of the result exceeds one of the limits: 'throw' or 'break'. By default, throw. +Using 'break' is similar to using LIMIT. + +## max_execution_time + +Maximum query execution time in seconds. +At this time, it is not checked for one of the sorting stages, or when merging and finalizing aggregate functions. + +## timeout_overflow_mode + +What to do if the query is run longer than 'max_execution_time': 'throw' or 'break'. By default, throw. + +## min_execution_speed + +Minimal execution speed in rows per second. Checked on every data block when 'timeout_before_checking_execution_speed' expires. If the execution speed is lower, an exception is thrown. + +## timeout_before_checking_execution_speed + +Checks that execution speed is not too slow (no less than 'min_execution_speed'), after the specified time in seconds has expired. + +## max_columns_to_read + +Maximum number of columns that can be read from a table in a single query. If a query requires reading a greater number of columns, it throws an exception. + +## max_temporary_columns + +Maximum number of temporary columns that must be kept in RAM at the same time when running a query, including constant columns. If there are more temporary columns than this, it throws an exception. + +## max_temporary_non_const_columns + +The same thing as 'max_temporary_columns', but without counting constant columns. +Note that constant columns are formed fairly often when running a query, but they require approximately zero computing resources. + +## max_subquery_depth + +Maximum nesting depth of subqueries. If subqueries are deeper, an exception is thrown. By default, 100. + +## max_pipeline_depth + +Maximum pipeline depth. Corresponds to the number of transformations that each data block goes through during query processing. Counted within the limits of a single server. If the pipeline depth is greater, an exception is thrown. By default, 1000. + +## max_ast_depth + +Maximum nesting depth of a query syntactic tree. If exceeded, an exception is thrown. +At this time, it isn't checked during parsing, but only after parsing the query. That is, a syntactic tree that is too deep can be created during parsing, but the query will fail. By default, 1000. + +## max_ast_elements + +Maximum number of elements in a query syntactic tree. If exceeded, an exception is thrown. +In the same way as the previous setting, it is checked only after parsing the query. By default, 10,000. + +## max_rows_in_set + +Maximum number of rows for a data set in the IN clause created from a subquery. + +## max_bytes_in_set + +Maximum number of bytes (uncompressed data) used by a set in the IN clause created from a subquery. + +## set_overflow_mode + +What to do when the amount of data exceeds one of the limits: 'throw' or 'break'. By default, throw. + +## max_rows_in_distinct + +Maximum number of different rows when using DISTINCT. + +## max_bytes_in_distinct + +Maximum number of bytes used by a hash table when using DISTINCT. + +## distinct_overflow_mode + +What to do when the amount of data exceeds one of the limits: 'throw' or 'break'. By default, throw. + +## max_rows_to_transfer + +Maximum number of rows that can be passed to a remote server or saved in a temporary table when using GLOBAL IN. + +## max_bytes_to_transfer + +Maximum number of bytes (uncompressed data) that can be passed to a remote server or saved in a temporary table when using GLOBAL IN. + +## transfer_overflow_mode + +What to do when the amount of data exceeds one of the limits: 'throw' or 'break'. By default, throw. + +[Original article](https://clickhouse.yandex/docs/en/operations/settings/query_complexity/) diff --git a/docs/zh/operations/settings/settings.md b/docs/zh/operations/settings/settings.md deleted file mode 120000 index 0c8df3cfc90..00000000000 --- a/docs/zh/operations/settings/settings.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/settings/settings.md \ No newline at end of file diff --git a/docs/zh/operations/settings/settings.md b/docs/zh/operations/settings/settings.md new file mode 100644 index 00000000000..8afea8e5d35 --- /dev/null +++ b/docs/zh/operations/settings/settings.md @@ -0,0 +1,421 @@ +# Settings + + + +## distributed_product_mode + +Changes the behavior of [distributed subqueries](../../query_language/select.md#queries-distributed-subrequests). + +ClickHouse applies this setting when the query contains the product of distributed tables, i.e. when the query for a distributed table contains a non-GLOBAL subquery for the distributed table. + +Restrictions: + +- Only applied for IN and JOIN subqueries. +- Only if the FROM section uses a distributed table containing more than one shard. +- If the subquery concerns a distributed table containing more than one shard, +- Not used for a table-valued [remote](../../query_language/table_functions/remote.md#table_functions-remote) function. + +The possible values ​​are: + +- `deny` — Default value. Prohibits using these types of subqueries (returns the "Double-distributed in/JOIN subqueries is denied" exception). +- `local` — Replaces the database and table in the subquery with local ones for the destination server (shard), leaving the normal `IN` / `JOIN.` +- `global` — Replaces the `IN` / `JOIN` query with `GLOBAL IN` / `GLOBAL JOIN.` +- `allow` — Allows the use of these types of subqueries. + + + +## fallback_to_stale_replicas_for_distributed_queries + +Forces a query to an out-of-date replica if updated data is not available. See "[Replication](../../operations/table_engines/replication.md#table_engines-replication)". + +ClickHouse selects the most relevant from the outdated replicas of the table. + +Used when performing `SELECT` from a distributed table that points to replicated tables. + +By default, 1 (enabled). + + + +## force_index_by_date + +Disables query execution if the index can't be used by date. + +Works with tables in the MergeTree family. + +If `force_index_by_date=1`, ClickHouse checks whether the query has a date key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For example, the condition `Date != ' 2000-01-01 '` is acceptable even when it matches all the data in the table (i.e., running the query requires a full scan). For more information about ranges of data in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md#table_engines-mergetree)". + + + +## force_primary_key + +Disables query execution if indexing by the primary key is not possible. + +Works with tables in the MergeTree family. + +If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For more information about data ranges in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md#table_engines-mergetree)". + + + +## fsync_metadata + +Enable or disable fsync when writing .sql files. Enabled by default. + +It makes sense to disable it if the server has millions of tiny table chunks that are constantly being created and destroyed. + +## input_format_allow_errors_num + +Sets the maximum number of acceptable errors when reading from text formats (CSV, TSV, etc.). + +The default value is 0. + +Always pair it with `input_format_allow_errors_ratio`. To skip errors, both settings must be greater than 0. + +If an error occurred while reading rows but the error counter is still less than `input_format_allow_errors_num`, ClickHouse ignores the row and moves on to the next one. + +If `input_format_allow_errors_num`is exceeded, ClickHouse throws an exception. + +## input_format_allow_errors_ratio + +Sets the maximum percentage of errors allowed when reading from text formats (CSV, TSV, etc.). +The percentage of errors is set as a floating-point number between 0 and 1. + +The default value is 0. + +Always pair it with `input_format_allow_errors_num`. To skip errors, both settings must be greater than 0. + +If an error occurred while reading rows but the error counter is still less than `input_format_allow_errors_ratio`, ClickHouse ignores the row and moves on to the next one. + +If `input_format_allow_errors_ratio` is exceeded, ClickHouse throws an exception. + +## max_block_size + +In ClickHouse, data is processed by blocks (sets of column parts). The internal processing cycles for a single block are efficient enough, but there are noticeable expenditures on each block. `max_block_size` is a recommendation for what size of block (in number of rows) to load from tables. The block size shouldn't be too small, so that the expenditures on each block are still noticeable, but not too large, so that the query with LIMIT that is completed after the first block is processed quickly, so that too much memory isn't consumed when extracting a large number of columns in multiple threads, and so that at least some cache locality is preserved. + +By default, 65,536. + +Blocks the size of `max_block_size` are not always loaded from the table. If it is obvious that less data needs to be retrieved, a smaller block is processed. + +## preferred_block_size_bytes + +Used for the same purpose as `max_block_size`, but it sets the recommended block size in bytes by adapting it to the number of rows in the block. +However, the block size cannot be more than `max_block_size` rows. +Disabled by default (set to 0). It only works when reading from MergeTree engines. + + + +## log_queries + +Setting up query logging. + +Queries sent to ClickHouse with this setup are logged according to the rules in the [query_log](../server_settings/settings.md#server_settings-query_log) server configuration parameter. + +**Example**: + + log_queries=1 + + + +## max_insert_block_size + +The size of blocks to form for insertion into a table. +This setting only applies in cases when the server forms the blocks. +For example, for an INSERT via the HTTP interface, the server parses the data format and forms blocks of the specified size. +But when using clickhouse-client, the client parses the data itself, and the 'max_insert_block_size' setting on the server doesn't affect the size of the inserted blocks. +The setting also doesn't have a purpose when using INSERT SELECT, since data is inserted using the same blocks that are formed after SELECT. + +By default, it is 1,048,576. + +This is slightly more than `max_block_size`. The reason for this is because certain table engines (`*MergeTree`) form a data part on the disk for each inserted block, which is a fairly large entity. Similarly, `*MergeTree` tables sort data during insertion, and a large enough block size allows sorting more data in RAM. + + + +## max_replica_delay_for_distributed_queries + +Disables lagging replicas for distributed queries. See "[Replication](../../operations/table_engines/replication.md#table_engines-replication)". + +Sets the time in seconds. If a replica lags more than the set value, this replica is not used. + +Default value: 0 (off). + +Used when performing `SELECT` from a distributed table that points to replicated tables. + +## max_threads + +The maximum number of query processing threads + +- excluding threads for retrieving data from remote servers (see the 'max_distributed_connections' parameter). + +This parameter applies to threads that perform the same stages of the query processing pipeline in parallel. +For example, if reading from a table, evaluating expressions with functions, filtering with WHERE and pre-aggregating for GROUP BY can all be done in parallel using at least 'max_threads' number of threads, then 'max_threads' are used. + +By default, 8. + +If less than one SELECT query is normally run on a server at a time, set this parameter to a value slightly less than the actual number of processor cores. + +For queries that are completed quickly because of a LIMIT, you can set a lower 'max_threads'. For example, if the necessary number of entries are located in every block and max_threads = 8, 8 blocks are retrieved, although it would have been enough to read just one. + +The smaller the `max_threads` value, the less memory is consumed. + +## max_compress_block_size + +The maximum size of blocks of uncompressed data before compressing for writing to a table. By default, 1,048,576 (1 MiB). If the size is reduced, the compression rate is significantly reduced, the compression and decompression speed increases slightly due to cache locality, and memory consumption is reduced. There usually isn't any reason to change this setting. + +Don't confuse blocks for compression (a chunk of memory consisting of bytes) and blocks for query processing (a set of rows from a table). + +## min_compress_block_size + +For [MergeTree](../../operations/table_engines/mergetree.md#table_engines-mergetree)" tables. In order to reduce latency when processing queries, a block is compressed when writing the next mark if its size is at least 'min_compress_block_size'. By default, 65,536. + +The actual size of the block, if the uncompressed data is less than 'max_compress_block_size', is no less than this value and no less than the volume of data for one mark. + +Let's look at an example. Assume that 'index_granularity' was set to 8192 during table creation. + +We are writing a UInt32-type column (4 bytes per value). When writing 8192 rows, the total will be 32 KB of data. Since min_compress_block_size = 65,536, a compressed block will be formed for every two marks. + +We are writing a URL column with the String type (average size of 60 bytes per value). When writing 8192 rows, the average will be slightly less than 500 KB of data. Since this is more than 65,536, a compressed block will be formed for each mark. In this case, when reading data from the disk in the range of a single mark, extra data won't be decompressed. + +There usually isn't any reason to change this setting. + +## max_query_size + +The maximum part of a query that can be taken to RAM for parsing with the SQL parser. +The INSERT query also contains data for INSERT that is processed by a separate stream parser (that consumes O(1) RAM), which is not included in this restriction. + +The default is 256 KiB. + +## interactive_delay + +The interval in microseconds for checking whether request execution has been canceled and sending the progress. + +By default, 100,000 (check for canceling and send progress ten times per second). + +## connect_timeout + +## receive_timeout + +## send_timeout + +Timeouts in seconds on the socket used for communicating with the client. + +By default, 10, 300, 300. + +## poll_interval + +Lock in a wait loop for the specified number of seconds. + +By default, 10. + +## max_distributed_connections + +The maximum number of simultaneous connections with remote servers for distributed processing of a single query to a single Distributed table. We recommend setting a value no less than the number of servers in the cluster. + +By default, 100. + +The following parameters are only used when creating Distributed tables (and when launching a server), so there is no reason to change them at runtime. + +## distributed_connections_pool_size + +The maximum number of simultaneous connections with remote servers for distributed processing of all queries to a single Distributed table. We recommend setting a value no less than the number of servers in the cluster. + +By default, 128. + +## connect_timeout_with_failover_ms + +The timeout in milliseconds for connecting to a remote server for a Distributed table engine, if the 'shard' and 'replica' sections are used in the cluster definition. +If unsuccessful, several attempts are made to connect to various replicas. + +By default, 50. + +## connections_with_failover_max_tries + +The maximum number of connection attempts with each replica, for the Distributed table engine. + +By default, 3. + +## extremes + +Whether to count extreme values (the minimums and maximums in columns of a query result). Accepts 0 or 1. By default, 0 (disabled). +For more information, see the section "Extreme values". + + + +## use_uncompressed_cache + +Whether to use a cache of uncompressed blocks. Accepts 0 or 1. By default, 0 (disabled). +The uncompressed cache (only for tables in the MergeTree family) allows significantly reducing latency and increasing throughput when working with a large number of short queries. Enable this setting for users who send frequent short requests. Also pay attention to the 'uncompressed_cache_size' configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed; the least-used data is automatically deleted. + +For queries that read at least a somewhat large volume of data (one million rows or more), the uncompressed cache is disabled automatically in order to save space for truly small queries. So you can keep the 'use_uncompressed_cache' setting always set to 1. + +## replace_running_query + +When using the HTTP interface, the 'query_id' parameter can be passed. This is any string that serves as the query identifier. +If a query from the same user with the same 'query_id' already exists at this time, the behavior depends on the 'replace_running_query' parameter. + +`0` (default) – Throw an exception (don't allow the query to run if a query with the same 'query_id' is already running). + +`1` – Cancel the old query and start running the new one. + +Yandex.Metrica uses this parameter set to 1 for implementing suggestions for segmentation conditions. After entering the next character, if the old query hasn't finished yet, it should be canceled. + +## schema + +This parameter is useful when you are using formats that require a schema definition, such as [Cap'n Proto](https://capnproto.org/). The value depends on the format. + + + +## stream_flush_interval_ms + +Works for tables with streaming in the case of a timeout, or when a thread generates[max_insert_block_size](#settings-settings-max_insert_block_size) rows. + +The default value is 7500. + +The smaller the value, the more often data is flushed into the table. Setting the value too low leads to poor performance. + + + +## load_balancing + +Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing. + +### random (default) + +The number of errors is counted for each replica. The query is sent to the replica with the fewest errors, and if there are several of these, to any one of them. +Disadvantages: Server proximity is not accounted for; if the replicas have different data, you will also get different data. + +### nearest_hostname + +The number of errors is counted for each replica. Every 5 minutes, the number of errors is integrally divided by 2. Thus, the number of errors is calculated for a recent time with exponential smoothing. If there is one replica with a minimal number of errors (i.e. errors occurred recently on the other replicas), the query is sent to it. If there are multiple replicas with the same minimal number of errors, the query is sent to the replica with a host name that is most similar to the server's host name in the config file (for the number of different characters in identical positions, up to the minimum length of both host names). + +For instance, example01-01-1 and example01-01-2.yandex.ru are different in one position, while example01-01-1 and example01-02-2 differ in two places. +This method might seem a little stupid, but it doesn't use external data about network topology, and it doesn't compare IP addresses, which would be complicated for our IPv6 addresses. + +Thus, if there are equivalent replicas, the closest one by name is preferred. +We can also assume that when sending a query to the same server, in the absence of failures, a distributed query will also go to the same servers. So even if different data is placed on the replicas, the query will return mostly the same results. + +### in_order + +Replicas are accessed in the same order as they are specified. The number of errors does not matter. +This method is appropriate when you know exactly which replica is preferable. + +## totals_mode + +How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = 'any' are present. +See the section "WITH TOTALS modifier". + +## totals_auto_threshold + +The threshold for ` totals_mode = 'auto'`. +See the section "WITH TOTALS modifier". + +## default_sample + +Floating-point number from 0 to 1. By default, 1. +Allows you to set the default sampling ratio for all SELECT queries. +(For tables that do not support sampling, it throws an exception.) +If set to 1, sampling is not performed by default. + +## max_parallel_replicas + +The maximum number of replicas for each shard when executing a query. +For consistency (to get different parts of the same data split), this option only works when the sampling key is set. +Replica lag is not controlled. + +## compile + +Enable compilation of queries. By default, 0 (disabled). + +Compilation is only used for part of the query-processing pipeline: for the first stage of aggregation (GROUP BY). +If this portion of the pipeline was compiled, the query may run faster due to deployment of short cycles and inlining aggregate function calls. The maximum performance improvement (up to four times faster in rare cases) is seen for queries with multiple simple aggregate functions. Typically, the performance gain is insignificant. In very rare cases, it may slow down query execution. + +## min_count_to_compile + +How many times to potentially use a compiled chunk of code before running compilation. By default, 3. +If the value is zero, then compilation runs synchronously and the query waits for the end of the compilation process before continuing execution. This can be used for testing; otherwise, use values ​​starting with 1. Compilation normally takes about 5-10 seconds. +If the value is 1 or more, compilation occurs asynchronously in a separate thread. The result will be used as soon as it is ready, including by queries that are currently running. + +Compiled code is required for each different combination of aggregate functions used in the query and the type of keys in the GROUP BY clause. +The results of compilation are saved in the build directory in the form of .so files. There is no restriction on the number of compilation results, since they don't use very much space. Old results will be used after server restarts, except in the case of a server upgrade – in this case, the old results are deleted. + +## input_format_skip_unknown_fields + +If the value is true, running INSERT skips input data from columns with unknown names. Otherwise, this situation will generate an exception. +It works for JSONEachRow and TSKV formats. + +## output_format_json_quote_64bit_integers + +If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes. + + + +## format_csv_delimiter + +The character interpreted as a delimiter in the CSV data. By default, the delimiter is `,`. + + + +## join_use_nulls + +Affects the behavior of [JOIN](../../query_language/select.md#query_language-join). + +With `join_use_nulls=1,` `JOIN` behaves like in standard SQL, i.e. if empty cells appear when merging, the type of the corresponding field is converted to [Nullable](../../data_types/nullable.md#data_type-nullable), and empty cells are filled with [NULL](../../query_language/syntax.md#null-literal). + + + +## insert_quorum + +Enables quorum writes. + + - If `insert_quorum < 2`, the quorum writes are disabled. + - If `insert_quorum >= 2`, the quorum writes are enabled. + +The default value is 0. + +**Quorum writes** + +`INSERT` succeeds only when ClickHouse manages to correctly write data to the `insert_quorum` of replicas during the `insert_quorum_timeout`. If for any reason the number of replicas with successful writes does not reach the `insert_quorum`, the write is considered failed and ClickHouse will delete the inserted block from all the replicas where data has already been written. + +All the replicas in the quorum are consistent, i.e., they contain data from all previous `INSERT` queries. The `INSERT` sequence is linearized. + +When reading the data written from the `insert_quorum`, you can use the[select_sequential_consistency](#setting-select_sequential_consistency) option. + +**ClickHouse generates an exception** + +- If the number of available replicas at the time of the query is less than the `insert_quorum`. +- At an attempt to write data when the previous block has not yet been inserted in the `insert_quorum` of replicas. This situation may occur if the user tries to perform an `INSERT` before the previous one with the `insert_quorum` is completed. + +**See also the following parameters:** + +- [insert_quorum_timeout](#setting-insert_quorum_timeout) +- [select_sequential_consistency](#setting-select_sequential_consistency) + + + +## insert_quorum_timeout + +Quorum write timeout in seconds. If the timeout has passed and no write has taken place yet, ClickHouse will generate an exception and the client must repeat the query to write the same block to the same or any other replica. + +By default, 60 seconds. + +**See also the following parameters:** + +- [insert_quorum](#setting-insert_quorum) +- [select_sequential_consistency](#setting-select_sequential_consistency) + + + +## select_sequential_consistency + +Enables/disables sequential consistency for `SELECT` queries: + +- 0 — disabled. The default value is 0. +- 1 — enabled. + +When sequential consistency is enabled, ClickHouse allows the client to execute the `SELECT` query only for those replicas that contain data from all previous `INSERT` queries executed with `insert_quorum`. If the client refers to a partial replica, ClickHouse will generate an exception. The SELECT query will not include data that has not yet been written to the quorum of replicas. + +See also the following parameters: + +- [insert_quorum](#setting-insert_quorum) +- [insert_quorum_timeout](#setting-insert_quorum_timeout) + + +[Original article](https://clickhouse.yandex/docs/en/operations/settings/settings/) diff --git a/docs/zh/operations/settings/settings_profiles.md b/docs/zh/operations/settings/settings_profiles.md deleted file mode 120000 index 35d9747ad56..00000000000 --- a/docs/zh/operations/settings/settings_profiles.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/settings/settings_profiles.md \ No newline at end of file diff --git a/docs/zh/operations/settings/settings_profiles.md b/docs/zh/operations/settings/settings_profiles.md new file mode 100644 index 00000000000..338800fbee2 --- /dev/null +++ b/docs/zh/operations/settings/settings_profiles.md @@ -0,0 +1,67 @@ + + +# Settings profiles + +A settings profile is a collection of settings grouped under the same name. Each ClickHouse user has a profile. +To apply all the settings in a profile, set the `profile` setting. + +Example: + +Install the `web` profile. + +``` sql +SET profile = 'web' +``` + +Settings profiles are declared in the user config file. This is usually `users.xml`. + +Example: + +```xml + + + + + + 8 + + + + + 1000000000 + 100000000000 + + 1000000 + any + + 1000000 + 1000000000 + + 100000 + 100000000 + break + + 600 + 1000000 + 15 + + 25 + 100 + 50 + + 2 + 25 + 50 + 100 + + 1 + + +``` + +The example specifies two profiles: `default` and `web`. The `default` profile has a special purpose: it must always be present and is applied when starting the server. In other words, the `default` profile contains default settings. The `web` profile is a regular profile that can be set using the `SET` query or using a URL parameter in an HTTP query. + +Settings profiles can inherit from each other. To use inheritance, indicate the `profile` setting before the other settings that are listed in the profile. + + +[Original article](https://clickhouse.yandex/docs/en/operations/settings/settings_profiles/) diff --git a/docs/zh/operations/system_tables.md b/docs/zh/operations/system_tables.md deleted file mode 120000 index c5701190dca..00000000000 --- a/docs/zh/operations/system_tables.md +++ /dev/null @@ -1 +0,0 @@ -../../en/operations/system_tables.md \ No newline at end of file diff --git a/docs/zh/operations/system_tables.md b/docs/zh/operations/system_tables.md new file mode 100644 index 00000000000..a67fec3f9c5 --- /dev/null +++ b/docs/zh/operations/system_tables.md @@ -0,0 +1,439 @@ +# System tables + +System tables are used for implementing part of the system's functionality, and for providing access to information about how the system is working. +You can't delete a system table (but you can perform DETACH). +System tables don't have files with data on the disk or files with metadata. The server creates all the system tables when it starts. +System tables are read-only. +They are located in the 'system' database. + + +## system.asynchronous_metrics + +Contain metrics used for profiling and monitoring. +They usually reflect the number of events currently in the system, or the total resources consumed by the system. +Example: The number of SELECT queries currently running; the amount of memory in use.`system.asynchronous_metrics`and`system.metrics` differ in their sets of metrics and how they are calculated. + +## system.clusters + +Contains information about clusters available in the config file and the servers in them. +Columns: + +``` +cluster String — The cluster name. +shard_num UInt32 — The shard number in the cluster, starting from 1. +shard_weight UInt32 — The relative weight of the shard when writing data. +replica_num UInt32 — The replica number in the shard, starting from 1. +host_name String — The host name, as specified in the config. +String host_address — The host IP address obtained from DNS. +port UInt16 — The port to use for connecting to the server. +user String — The name of the user for connecting to the server. +``` + +## system.columns + +Contains information about the columns in all tables. +You can use this table to get information similar to `DESCRIBE TABLE`, but for multiple tables at once. + +``` +database String — The name of the database the table is in. +table String – Table name. +name String — Column name. +type String — Column type. +default_type String — Expression type (DEFAULT, MATERIALIZED, ALIAS) for the default value, or an empty string if it is not defined. +default_expression String — Expression for the default value, or an empty string if it is not defined. +``` + +## system.databases + +This table contains a single String column called 'name' – the name of a database. +Each database that the server knows about has a corresponding entry in the table. +This system table is used for implementing the `SHOW DATABASES` query. + +## system.dictionaries + +Contains information about external dictionaries. + +Columns: + +- `name String` — Dictionary name. +- `type String` — Dictionary type: Flat, Hashed, Cache. +- `origin String` — Path to the configuration file that describes the dictionary. +- `attribute.names Array(String)` — Array of attribute names provided by the dictionary. +- `attribute.types Array(String)` — Corresponding array of attribute types that are provided by the dictionary. +- `has_hierarchy UInt8` — Whether the dictionary is hierarchical. +- `bytes_allocated UInt64` — The amount of RAM the dictionary uses. +- `hit_rate Float64` — For cache dictionaries, the percentage of uses for which the value was in the cache. +- `element_count UInt64` — The number of items stored in the dictionary. +- `load_factor Float64` — The percentage full of the dictionary (for a hashed dictionary, the percentage filled in the hash table). +- `creation_time DateTime` — The time when the dictionary was created or last successfully reloaded. +- `last_exception String` — Text of the error that occurs when creating or reloading the dictionary if the dictionary couldn't be created. +- `source String` — Text describing the data source for the dictionary. + +Note that the amount of memory used by the dictionary is not proportional to the number of items stored in it. So for flat and cached dictionaries, all the memory cells are pre-assigned, regardless of how full the dictionary actually is. + + +## system.events + +Contains information about the number of events that have occurred in the system. This is used for profiling and monitoring purposes. +Example: The number of processed SELECT queries. +Columns: 'event String' – the event name, and 'value UInt64' – the quantity. + +## system.functions + +Contains information about normal and aggregate functions. + +Columns: + +- `name`(`String`) – The name of the function. +- `is_aggregate`(`UInt8`) — Whether the function is aggregate. + +## system.merges + +Contains information about merges currently in process for tables in the MergeTree family. + +Columns: + +- `database String` — The name of the database the table is in. +- `table String` — Table name. +- `elapsed Float64` — The time elapsed (in seconds) since the merge started. +- `progress Float64` — The percentage of completed work from 0 to 1. +- `num_parts UInt64` — The number of pieces to be merged. +- `result_part_name String` — The name of the part that will be formed as the result of merging. +- `total_size_bytes_compressed UInt64` — The total size of the compressed data in the merged chunks. +- `total_size_marks UInt64` — The total number of marks in the merged partss. +- `bytes_read_uncompressed UInt64` — Number of bytes read, uncompressed. +- `rows_read UInt64` — Number of rows read. +- `bytes_written_uncompressed UInt64` — Number of bytes written, uncompressed. +- `rows_written UInt64` — Number of lines rows written. + + +## system.metrics + +## system.numbers + +This table contains a single UInt64 column named 'number' that contains almost all the natural numbers starting from zero. +You can use this table for tests, or if you need to do a brute force search. +Reads from this table are not parallelized. + +## system.numbers_mt + +The same as 'system.numbers' but reads are parallelized. The numbers can be returned in any order. +Used for tests. + +## system.one + +This table contains a single row with a single 'dummy' UInt8 column containing the value 0. +This table is used if a SELECT query doesn't specify the FROM clause. +This is similar to the DUAL table found in other DBMSs. + +## system.parts + +Contains information about parts of [MergeTree](table_engines/mergetree.md#table_engines-mergetree) tables. + +Each row describes one part of the data. + +Columns: + +- partition (String) – The partition name. To learn what a partition is, see the description of the [ALTER](../query_language/alter.md#query_language_queries_alter) query. + +Formats: +- `YYYYMM` for automatic partitioning by month. +- `any_string` when partitioning manually. + +- name (String) – Name of the data part. + +- active (UInt8) – Indicates whether the part is active. If a part is active, it is used in a table; otherwise, it will be deleted. Inactive data parts remain after merging. + +- marks (UInt64) – The number of marks. To get the approximate number of rows in a data part, multiply ``marks`` by the index granularity (usually 8192). + +- marks_size (UInt64) – The size of the file with marks. + +- rows (UInt64) – The number of rows. + +- bytes (UInt64) – The number of bytes when compressed. + +- modification_time (DateTime) – The modification time of the directory with the data part. This usually corresponds to the time of data part creation.| + +- remove_time (DateTime) – The time when the data part became inactive. + +- refcount (UInt32) – The number of places where the data part is used. A value greater than 2 indicates that the data part is used in queries or merges. + +- min_date (Date) – The minimum value of the date key in the data part. + +- max_date (Date) – The maximum value of the date key in the data part. + +- min_block_number (UInt64) – The minimum number of data parts that make up the current part after merging. + +- max_block_number (UInt64) – The maximum number of data parts that make up the current part after merging. + +- level (UInt32) – Depth of the merge tree. If a merge was not performed, ``level=0``. + +- primary_key_bytes_in_memory (UInt64) – The amount of memory (in bytes) used by primary key values. + +- primary_key_bytes_in_memory_allocated (UInt64) – The amount of memory (in bytes) reserved for primary key values. + +- database (String) – Name of the database. + +- table (String) – Name of the table. + +- engine (String) – Name of the table engine without parameters. + +## system.processes + +This system table is used for implementing the `SHOW PROCESSLIST` query. +Columns: + +``` +user String – Name of the user who made the request. For distributed query processing, this is the user who helped the requestor server send the query to this server, not the user who made the distributed request on the requestor server. + +address String - The IP address the request was made from. The same for distributed processing. + +elapsed Float64 - The time in seconds since request execution started. + +rows_read UInt64 - The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. + +bytes_read UInt64 - The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. + +total_rows_approx UInt64 - The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known. + +memory_usage UInt64 - How much memory the request uses. It might not include some types of dedicated memory. + +query String - The query text. For INSERT, it doesn't include the data to insert. + +query_id String - Query ID, if defined. +``` + +## system.replicas + +Contains information and status for replicated tables residing on the local server. +This table can be used for monitoring. The table contains a row for every Replicated\* table. + +Example: + +``` sql +SELECT * +FROM system.replicas +WHERE table = 'visits' +FORMAT Vertical +``` + +``` +Row 1: +────── +database: merge +table: visits +engine: ReplicatedCollapsingMergeTree +is_leader: 1 +is_readonly: 0 +is_session_expired: 0 +future_parts: 1 +parts_to_check: 0 +zookeeper_path: /clickhouse/tables/01-06/visits +replica_name: example01-06-1.yandex.ru +replica_path: /clickhouse/tables/01-06/visits/replicas/example01-06-1.yandex.ru +columns_version: 9 +queue_size: 1 +inserts_in_queue: 0 +merges_in_queue: 1 +log_max_index: 596273 +log_pointer: 596274 +total_replicas: 2 +active_replicas: 2 +``` + +Columns: + +``` +database: Database name +table: Table name +engine: Table engine name + +is_leader: Whether the replica is the leader. + +Only one replica at a time can be the leader. The leader is responsible for selecting background merges to perform. +Note that writes can be performed to any replica that is available and has a session in ZK, regardless of whether it is a leader. + +is_readonly: Whether the replica is in read-only mode. +This mode is turned on if the config doesn't have sections with ZooKeeper, if an unknown error occurred when reinitializing sessions in ZooKeeper, and during session reinitialization in ZooKeeper. + +is_session_expired: Whether the session with ZooKeeper has expired. +Basically the same as 'is_readonly'. + +future_parts: The number of data parts that will appear as the result of INSERTs or merges that haven't been done yet. + +parts_to_check: The number of data parts in the queue for verification. +A part is put in the verification queue if there is suspicion that it might be damaged. + +zookeeper_path: Path to table data in ZooKeeper. +replica_name: Replica name in ZooKeeper. Different replicas of the same table have different names. +replica_path: Path to replica data in ZooKeeper. The same as concatenating 'zookeeper_path/replicas/replica_path'. + +columns_version: Version number of the table structure. +Indicates how many times ALTER was performed. If replicas have different versions, it means some replicas haven't made all of the ALTERs yet. + +queue_size: Size of the queue for operations waiting to be performed. +Operations include inserting blocks of data, merges, and certain other actions. +It usually coincides with 'future_parts'. + +inserts_in_queue: Number of inserts of blocks of data that need to be made. +Insertions are usually replicated fairly quickly. If this number is large, it means something is wrong. + +merges_in_queue: The number of merges waiting to be made. +Sometimes merges are lengthy, so this value may be greater than zero for a long time. + +The next 4 columns have a non-zero value only where there is an active session with ZK. + +log_max_index: Maximum entry number in the log of general activity. +log_pointer: Maximum entry number in the log of general activity that the replica copied to its execution queue, plus one. +If log_pointer is much smaller than log_max_index, something is wrong. + +total_replicas: The total number of known replicas of this table. +active_replicas: The number of replicas of this table that have a session in ZooKeeper (i.e., the number of functioning replicas). +``` + +If you request all the columns, the table may work a bit slowly, since several reads from ZooKeeper are made for each row. +If you don't request the last 4 columns (log_max_index, log_pointer, total_replicas, active_replicas), the table works quickly. + +For example, you can check that everything is working correctly like this: + +``` sql +SELECT + database, + table, + is_leader, + is_readonly, + is_session_expired, + future_parts, + parts_to_check, + columns_version, + queue_size, + inserts_in_queue, + merges_in_queue, + log_max_index, + log_pointer, + total_replicas, + active_replicas +FROM system.replicas +WHERE + is_readonly + OR is_session_expired + OR future_parts > 20 + OR parts_to_check > 10 + OR queue_size > 20 + OR inserts_in_queue > 10 + OR log_max_index - log_pointer > 10 + OR total_replicas < 2 + OR active_replicas < total_replicas +``` + +If this query doesn't return anything, it means that everything is fine. + +## system.settings + +Contains information about settings that are currently in use. +I.e. used for executing the query you are using to read from the system.settings table. + +Columns: + +``` +name String — Setting name. +value String — Setting value. +changed UInt8 — Whether the setting was explicitly defined in the config or explicitly changed. +``` + +Example: + +``` sql +SELECT * +FROM system.settings +WHERE changed +``` + +``` +┌─name───────────────────┬─value───────┬─changed─┐ +│ max_threads │ 8 │ 1 │ +│ use_uncompressed_cache │ 0 │ 1 │ +│ load_balancing │ random │ 1 │ +│ max_memory_usage │ 10000000000 │ 1 │ +└────────────────────────┴─────────────┴─────────┘ +``` + +## system.tables + +This table contains the String columns 'database', 'name', and 'engine'. +The table also contains three virtual columns: metadata_modification_time (DateTime type), create_table_query, and engine_full (String type). +Each table that the server knows about is entered in the 'system.tables' table. +This system table is used for implementing SHOW TABLES queries. + +## system.zookeeper + +The table does not exist if ZooKeeper is not configured. Allows reading data from the ZooKeeper cluster defined in the config. +The query must have a 'path' equality condition in the WHERE clause. This is the path in ZooKeeper for the children that you want to get data for. + +The query `SELECT * FROM system.zookeeper WHERE path = '/clickhouse'` outputs data for all children on the `/clickhouse` node. +To output data for all root nodes, write path = '/'. +If the path specified in 'path' doesn't exist, an exception will be thrown. + +Columns: + +- `name String` — The name of the node. +- `path String` — The path to the node. +- `value String` — Node value. +- `dataLength Int32` — Size of the value. +- `numChildren Int32` — Number of descendants. +- `czxid Int64` — ID of the transaction that created the node. +- `mzxid Int64` — ID of the transaction that last changed the node. +- `pzxid Int64` — ID of the transaction that last deleted or added descendants. +- `ctime DateTime` — Time of node creation. +- `mtime DateTime` — Time of the last modification of the node. +- `version Int32` — Node version: the number of times the node was changed. +- `cversion Int32` — Number of added or removed descendants. +- `aversion Int32` — Number of changes to the ACL. +- `ephemeralOwner Int64` — For ephemeral nodes, the ID of hte session that owns this node. + +Example: + +``` sql +SELECT * +FROM system.zookeeper +WHERE path = '/clickhouse/tables/01-08/visits/replicas' +FORMAT Vertical +``` + +``` +Row 1: +────── +name: example01-08-1.yandex.ru +value: +czxid: 932998691229 +mzxid: 932998691229 +ctime: 2015-03-27 16:49:51 +mtime: 2015-03-27 16:49:51 +version: 0 +cversion: 47 +aversion: 0 +ephemeralOwner: 0 +dataLength: 0 +numChildren: 7 +pzxid: 987021031383 +path: /clickhouse/tables/01-08/visits/replicas + +Row 2: +────── +name: example01-08-2.yandex.ru +value: +czxid: 933002738135 +mzxid: 933002738135 +ctime: 2015-03-27 16:57:01 +mtime: 2015-03-27 16:57:01 +version: 0 +cversion: 37 +aversion: 0 +ephemeralOwner: 0 +dataLength: 0 +numChildren: 7 +pzxid: 987021252247 +path: /clickhouse/tables/01-08/visits/replicas +``` + +[Original article](https://clickhouse.yandex/docs/en/operations/system_tables/) diff --git a/docs/zh/operations/table_engines/aggregatingmergetree.md b/docs/zh/operations/table_engines/aggregatingmergetree.md deleted file mode 120000 index 907a073e0c8..00000000000 --- a/docs/zh/operations/table_engines/aggregatingmergetree.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/aggregatingmergetree.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/aggregatingmergetree.md b/docs/zh/operations/table_engines/aggregatingmergetree.md new file mode 100644 index 00000000000..4ebb707a980 --- /dev/null +++ b/docs/zh/operations/table_engines/aggregatingmergetree.md @@ -0,0 +1,95 @@ + + +# AggregatingMergeTree + +The engine inherits from [MergeTree](mergetree.md#table_engines-mergetree), altering the logic for data parts merging. ClickHouse replaces all rows with the same primary key with a single row (within a one data part) that stores a combination of states of aggregate functions. + +You can use `AggregatingMergeTree` tables for incremental data aggregation, including for aggregated materialized views. + +The engine processes all columns with [AggregateFunction](../../data_types/nested_data_structures/aggregatefunction.md#data_type-aggregatefunction) type. + +It is appropriate to use `AggregatingMergeTree` if it reduces the number of rows by orders. + +## Creating a Table + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = AggregatingMergeTree() +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] +``` + +For a description of request parameters, see [request description](../../query_language/create.md#query_language-queries-create_table). + +**Query clauses** + +When creating a `ReplacingMergeTree` table the same [clauses](mergetree.md#table_engines-mergetree-configuring) are required, as when creating a `MergeTree` table. + +
Deprecated Method for Creating a Table + +!!! attention + Do not use this method in new projects and, if possible, switch the old projects to the method described above. + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] AggregatingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity) +``` + +All of the parameters have the same meaning as in `MergeTree`. +
+ +## SELECT and INSERT + +To insert data, use [INSERT SELECT](../../query_language/insert_into.md#queries-insert-select) query with aggregate `-State`- functions. + +When selecting data from `AggregatingMergeTree` table, use `GROUP BY` clause and the same aggregate functions as when inserting data, but using `-Merge` suffix. + +In the results of `SELECT` query the values of `AggregateFunction` type have implementation-specific binary representation for all of the ClickHouse output formats. If dump data into, for example, `TabSeparated` format with `SELECT` query then this dump can be loaded back using `INSERT` query. + +## Example of an Aggregated Materialized View + +`AggregatingMergeTree` materialized view that watches the `test.visits` table: + +``` sql +CREATE MATERIALIZED VIEW test.basic +ENGINE = AggregatingMergeTree() PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate) +AS SELECT + CounterID, + StartDate, + sumState(Sign) AS Visits, + uniqState(UserID) AS Users +FROM test.visits +GROUP BY CounterID, StartDate; +``` + +Inserting of data into the `test.visits` table. + +``` sql +INSERT INTO test.visits ... +``` + +The data are inserted in both the table and view `test.basic` that will perform the aggregation. + +To get the aggregated data, we need to execute a query such as `SELECT ... GROUP BY ...` from the view `test.basic`: + +``` sql +SELECT + StartDate, + sumMerge(Visits) AS Visits, + uniqMerge(Users) AS Users +FROM test.basic +GROUP BY StartDate +ORDER BY StartDate; +``` + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/aggregatingmergetree/) diff --git a/docs/zh/operations/table_engines/buffer.md b/docs/zh/operations/table_engines/buffer.md deleted file mode 120000 index 0a3c372fa67..00000000000 --- a/docs/zh/operations/table_engines/buffer.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/buffer.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/buffer.md b/docs/zh/operations/table_engines/buffer.md new file mode 100644 index 00000000000..24a990bb260 --- /dev/null +++ b/docs/zh/operations/table_engines/buffer.md @@ -0,0 +1,56 @@ +# Buffer + +Buffers the data to write in RAM, periodically flushing it to another table. During the read operation, data is read from the buffer and the other table simultaneously. + +``` +Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes) +``` + +Engine parameters:database, table – The table to flush data to. Instead of the database name, you can use a constant expression that returns a string.num_layers – Parallelism layer. Physically, the table will be represented as 'num_layers' of independent buffers. Recommended value: 16.min_time, max_time, min_rows, max_rows, min_bytes, and max_bytes are conditions for flushing data from the buffer. + +Data is flushed from the buffer and written to the destination table if all the 'min' conditions or at least one 'max' condition are met.min_time, max_time – Condition for the time in seconds from the moment of the first write to the buffer.min_rows, max_rows – Condition for the number of rows in the buffer.min_bytes, max_bytes – Condition for the number of bytes in the buffer. + +During the write operation, data is inserted to a 'num_layers' number of random buffers. Or, if the data part to insert is large enough (greater than 'max_rows' or 'max_bytes'), it is written directly to the destination table, omitting the buffer. + +The conditions for flushing the data are calculated separately for each of the 'num_layers' buffers. For example, if num_layers = 16 and max_bytes = 100000000, the maximum RAM consumption is 1.6 GB. + +Example: + +``` sql +CREATE TABLE merge.hits_buffer AS merge.hits ENGINE = Buffer(merge, hits, 16, 10, 100, 10000, 1000000, 10000000, 100000000) +``` + +Creating a 'merge.hits_buffer' table with the same structure as 'merge.hits' and using the Buffer engine. When writing to this table, data is buffered in RAM and later written to the 'merge.hits' table. 16 buffers are created. The data in each of them is flushed if either 100 seconds have passed, or one million rows have been written, or 100 MB of data have been written; or if simultaneously 10 seconds have passed and 10,000 rows and 10 MB of data have been written. For example, if just one row has been written, after 100 seconds it will be flushed, no matter what. But if many rows have been written, the data will be flushed sooner. + +When the server is stopped, with DROP TABLE or DETACH TABLE, buffer data is also flushed to the destination table. + +You can set empty strings in single quotation marks for the database and table name. This indicates the absence of a destination table. In this case, when the data flush conditions are reached, the buffer is simply cleared. This may be useful for keeping a window of data in memory. + +When reading from a Buffer table, data is processed both from the buffer and from the destination table (if there is one). +Note that the Buffer tables does not support an index. In other words, data in the buffer is fully scanned, which might be slow for large buffers. (For data in a subordinate table, the index that it supports will be used.) + +If the set of columns in the Buffer table doesn't match the set of columns in a subordinate table, a subset of columns that exist in both tables is inserted. + +If the types don't match for one of the columns in the Buffer table and a subordinate table, an error message is entered in the server log and the buffer is cleared. +The same thing happens if the subordinate table doesn't exist when the buffer is flushed. + +If you need to run ALTER for a subordinate table and the Buffer table, we recommend first deleting the Buffer table, running ALTER for the subordinate table, then creating the Buffer table again. + +If the server is restarted abnormally, the data in the buffer is lost. + +PREWHERE, FINAL and SAMPLE do not work correctly for Buffer tables. These conditions are passed to the destination table, but are not used for processing data in the buffer. Because of this, we recommend only using the Buffer table for writing, while reading from the destination table. + +When adding data to a Buffer, one of the buffers is locked. This causes delays if a read operation is simultaneously being performed from the table. + +Data that is inserted to a Buffer table may end up in the subordinate table in a different order and in different blocks. Because of this, a Buffer table is difficult to use for writing to a CollapsingMergeTree correctly. To avoid problems, you can set 'num_layers' to 1. + +If the destination table is replicated, some expected characteristics of replicated tables are lost when writing to a Buffer table. The random changes to the order of rows and sizes of data parts cause data deduplication to quit working, which means it is not possible to have a reliable 'exactly once' write to replicated tables. + +Due to these disadvantages, we can only recommend using a Buffer table in rare cases. + +A Buffer table is used when too many INSERTs are received from a large number of servers over a unit of time and data can't be buffered before insertion, which means the INSERTs can't run fast enough. + +Note that it doesn't make sense to insert data one row at a time, even for Buffer tables. This will only produce a speed of a few thousand rows per second, while inserting larger blocks of data can produce over a million rows per second (see the section "Performance"). + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/buffer/) diff --git a/docs/zh/operations/table_engines/collapsingmergetree.md b/docs/zh/operations/table_engines/collapsingmergetree.md deleted file mode 120000 index ef5cebb48d8..00000000000 --- a/docs/zh/operations/table_engines/collapsingmergetree.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/collapsingmergetree.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/collapsingmergetree.md b/docs/zh/operations/table_engines/collapsingmergetree.md new file mode 100644 index 00000000000..45106cb25e8 --- /dev/null +++ b/docs/zh/operations/table_engines/collapsingmergetree.md @@ -0,0 +1,220 @@ + + +# CollapsingMergeTree + +The engine inherits from [MergeTree](mergetree.md#table_engines-mergetree) and adds the logic of rows collapsing to data parts merge algorithm. + +`CollapsingMergeTree` asynchronously deletes (collapses) pairs of rows if all of the fields in a row are equivalent excepting the particular field `Sign` which can have `1` and `-1` values. Rows without a pair are kept. For more details see the [Collapsing](#collapsingmergetree-collapsing) section of the document. + +The engine may significantly reduce the volume of storage and increase efficiency of `SELECT` query as a consequence. + +## Creating a Table + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = CollapsingMergeTree(sign) +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] +``` + +For a description of request parameters, see [request description](../../query_language/create.md#query_language-queries-create_table). + +**CollapsingMergeTree Parameters** + +- `sign` — Name of the column with the type of row: `1` is a "state" row, `-1` is a "cancel" row. + + Column data type — `Int8`. + +**Query clauses** + +When creating a `CollapsingMergeTree` table, the same [clauses](mergetree.md#table_engines-mergetree-configuring) are required, as when creating a `MergeTree` table. + +
Deprecated Method for Creating a Table + +!!! attention + Do not use this method in new projects and, if possible, switch the old projects to the method described above. + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] CollapsingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, sign) +``` + +All of the parameters excepting `sign` have the same meaning as in `MergeTree`. + +- `sign` — Name of the column with the type of row: `1` — "state" row, `-1` — "cancel" row. + + Column Data Type — `Int8`. +
+ + + +## Collapsing + +### Data + +Consider the situation where you need to save continually changing data for some object. It sounds logical to have one row for an object and update it at any change, but update operation is expensive and slow for DBMS because it requires rewriting of the data in the storage. If you need to write data quickly, update not acceptable, but you can write the changes of an object sequentially as follows. + +Use the particular column `Sign` when writing row. If `Sign = 1` it means that the row is a state of an object, let's call it "state" row. If `Sign = -1` it means the cancellation of the state of an object with the same attributes, let's call it "cancel" row. + +For example, we want to calculate how much pages users checked at some site and how long they were there. At some moment of time we write the following row with the state of user activity: + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +At some moment later we register the change of user activity and write it with the following two rows. + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ -1 │ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +The first row cancels the previous state of the object (user). It should copy all of the fields of the canceled state excepting `Sign`. + +The second row contains the current state. + +As we need only the last state of user activity, the rows + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ 1 │ +│ 4324182021466249494 │ 5 │ 146 │ -1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +can be deleted collapsing the invalid (old) state of an object. `CollapsingMergeTree` does this while merging of the data parts. + +Why we need 2 rows for each change read in the "Algorithm" paragraph. + +**Peculiar properties of such approach** + +1. The program that writes the data should remember the state of an object to be able to cancel it. "Cancel" string should be the copy of "state" string with the opposite `Sign`. It increases the initial size of storage but allows to write the data quickly. +2. Long growing arrays in columns reduce the efficiency of the engine due to load for writing. The more straightforward data, the higher efficiency. +3. `SELECT` results depend strongly on the consistency of object changes history. Be accurate when preparing data for inserting. You can get unpredictable results in inconsistent data, for example, negative values for non-negative metrics such as session depth. + +### Algorithm + +When ClickHouse merges data parts, each group of consecutive rows with the same primary key is reduced to not more than two rows, one with `Sign = 1` ("state" row) and another with `Sign = -1` ("cancel" row). In other words, entries collapse. + +For each resulting data part ClickHouse saves: + + 1. The first "cancel" and the last "state" rows, if the number of "state" and "cancel" rows matches. + 1. The last "state" row, if there is one more "state" row than "cancel" rows. + 1. The first "cancel" row, if there is one more "cancel" row than "state" rows. + 1. None of the rows, in all other cases. + + The merge continues, but ClickHouse treats this situation as a logical error and records it in the server log. This error can occur if the same data were inserted more than once. + +Thus, collapsing should not change the results of calculating statistics. +Changes gradually collapsed so that in the end only the last state of almost every object left. + +The `Sign` is required because the merging algorithm doesn't guarantee that all of the rows with the same primary key will be in the same resulting data part and even on the same physical server. ClickHouse process `SELECT` queries with multiple threads, and it can not predict the order of rows in the result. The aggregation is required if there is a need to get completely "collapsed" data from `CollapsingMergeTree` table. + +To finalize collapsing write a query with `GROUP BY` clause and aggregate functions that account for the sign. For example, to calculate quantity, use `sum(Sign)` instead of `count()`. To calculate the sum of something, use `sum(Sign * x)` instead of `sum(x)`, and so on, and also add `HAVING sum(Sign) > 0`. + +The aggregates `count`, `sum` and `avg` could be calculated this way. The aggregate `uniq` could be calculated if an object has at list one state not collapsed. The aggregates `min` and `max` could not be calculated because `CollapsingMergeTree` does not save values history of the collapsed states. + +If you need to extract data without aggregation (for example, to check whether rows are present whose newest values match certain conditions), you can use the `FINAL` modifier for the `FROM` clause. This approach is significantly less efficient. + +## Example of use + +Example data: + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ 1 │ +│ 4324182021466249494 │ 5 │ 146 │ -1 │ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +Creation of the table: + +```sql +CREATE TABLE UAct +( + UserID UInt64, + PageViews UInt8, + Duration UInt8, + Sign Int8 +) +ENGINE = CollapsingMergeTree(Sign) +ORDER BY UserID +``` + +Insertion of the data: + +```sql +INSERT INTO UAct VALUES (4324182021466249494, 5, 146, 1) +``` +```sql +INSERT INTO UAct VALUES (4324182021466249494, 5, 146, -1),(4324182021466249494, 6, 185, 1) +``` + +We use two `INSERT` queries to create two different data parts. If we insert the data with one query ClickHouse creates one data part and will not perform any merge ever. + +Getting the data: + +``` +SELECT * FROM UAct +``` + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ -1 │ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +What do we see and where is collapsing? +With two `INSERT` queries, we created 2 data parts. The `SELECT` query was performed in 2 threads, and we got a random order of rows. +Collapsing not occurred because there was no merge of the data parts yet. ClickHouse merges data part in an unknown moment of time which we can not predict. + +Thus we need aggregation: + +```sql +SELECT + UserID, + sum(PageViews * Sign) AS PageViews, + sum(Duration * Sign) AS Duration +FROM UAct +GROUP BY UserID +HAVING sum(Sign) > 0 +``` +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┐ +│ 4324182021466249494 │ 6 │ 185 │ +└─────────────────────┴───────────┴──────────┘ +``` + +If we do not need aggregation and want to force collapsing, we can use `FINAL` modifier for `FROM` clause. + +```sql +SELECT * FROM UAct FINAL +``` +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +This way of selecting the data is very inefficient. Don't use it for big tables. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/collapsingmergetree/) diff --git a/docs/zh/operations/table_engines/custom_partitioning_key.md b/docs/zh/operations/table_engines/custom_partitioning_key.md deleted file mode 120000 index a9d18cacb25..00000000000 --- a/docs/zh/operations/table_engines/custom_partitioning_key.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/custom_partitioning_key.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/custom_partitioning_key.md b/docs/zh/operations/table_engines/custom_partitioning_key.md new file mode 100644 index 00000000000..55940db8ca9 --- /dev/null +++ b/docs/zh/operations/table_engines/custom_partitioning_key.md @@ -0,0 +1,49 @@ + + +# Custom Partitioning Key + +Starting with version 1.1.54310, you can create tables in the MergeTree family with any partitioning expression (not only partitioning by month). + +The partition key can be an expression from the table columns, or a tuple of such expressions (similar to the primary key). The partition key can be omitted. When creating a table, specify the partition key in the ENGINE description with the new syntax: + +``` +ENGINE [=] Name(...) [PARTITION BY expr] [ORDER BY expr] [SAMPLE BY expr] [SETTINGS name=value, ...] +``` + +For MergeTree tables, the partition expression is specified after `PARTITION BY`, the primary key after `ORDER BY`, the sampling key after `SAMPLE BY`, and `SETTINGS` can specify `index_granularity` (optional; the default value is 8192), as well as other settings from [MergeTreeSettings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Storages/MergeTree/MergeTreeSettings.h). The other engine parameters are specified in parentheses after the engine name, as previously. Example: + +``` sql +ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/name', 'replica1', Sign) + PARTITION BY (toMonday(StartDate), EventType) + ORDER BY (CounterID, StartDate, intHash32(UserID)) + SAMPLE BY intHash32(UserID) +``` + +The traditional partitioning by month is expressed as `toYYYYMM(date_column)`. + +You can't convert an old-style table to a table with custom partitions (only via INSERT SELECT). + +After this table is created, merge will only work for data parts that have the same value for the partitioning expression. Note: This means that you shouldn't make overly granular partitions (more than about a thousand partitions), or SELECT will perform poorly. + +To specify a partition in ALTER PARTITION commands, specify the value of the partition expression (or a tuple). Constants and constant expressions are supported. Example: + +``` sql +ALTER TABLE table DROP PARTITION (toMonday(today()), 1) +``` + +Deletes the partition for the current week with event type 1. The same is true for the OPTIMIZE query. To specify the only partition in a non-partitioned table, specify `PARTITION tuple()`. + +Note: For old-style tables, the partition can be specified either as a number `201710` or a string `'201710'`. The syntax for the new style of tables is stricter with types (similar to the parser for the VALUES input format). In addition, ALTER TABLE FREEZE PARTITION uses exact match for new-style tables (not prefix match). + +In the `system.parts` table, the `partition` column specifies the value of the partition expression to use in ALTER queries (if quotas are removed). The `name` column should specify the name of the data part that has a new format. + +Old: `20140317_20140323_2_2_0` (minimum date - maximum date - minimum block number - maximum block number - level). + +Now: `201403_2_2_0` (partition ID - minimum block number - maximum block number - level). + +The partition ID is its string identifier (human-readable, if possible) that is used for the names of data parts in the file system and in ZooKeeper. You can specify it in ALTER queries in place of the partition key. Example: Partition key `toYYYYMM(EventDate)`; ALTER can specify either `PARTITION 201710` or `PARTITION ID '201710'`. + +For more examples, see the tests [`00502_custom_partitioning_local`](https://github.com/yandex/ClickHouse/blob/master/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.sql) and [`00502_custom_partitioning_replicated_zookeeper`](https://github.com/yandex/ClickHouse/blob/master/dbms/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql). + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/custom_partitioning_key/) diff --git a/docs/zh/operations/table_engines/dictionary.md b/docs/zh/operations/table_engines/dictionary.md deleted file mode 120000 index 2a95f4a669b..00000000000 --- a/docs/zh/operations/table_engines/dictionary.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/dictionary.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/dictionary.md b/docs/zh/operations/table_engines/dictionary.md new file mode 100644 index 00000000000..eed7f7afaf4 --- /dev/null +++ b/docs/zh/operations/table_engines/dictionary.md @@ -0,0 +1,112 @@ + + +# Dictionary + +The `Dictionary` engine displays the dictionary data as a ClickHouse table. + +As an example, consider a dictionary of `products` with the following configuration: + +```xml + + + products + + + products
+ DSN=some-db-server +
+ + + 300 + 360 + + + + + + + product_id + + + title + String + + + +
+
+``` + +Query the dictionary data: + +``` sql +select name, type, key, attribute.names, attribute.types, bytes_allocated, element_count,source from system.dictionaries where name = 'products'; + +SELECT + name, + type, + key, + attribute.names, + attribute.types, + bytes_allocated, + element_count, + source +FROM system.dictionaries +WHERE name = 'products' +``` + +``` +┌─name─────┬─type─┬─key────┬─attribute.names─┬─attribute.types─┬─bytes_allocated─┬─element_count─┬─source──────────┐ +│ products │ Flat │ UInt64 │ ['title'] │ ['String'] │ 23065376 │ 175032 │ ODBC: .products │ +└──────────┴──────┴────────┴─────────────────┴─────────────────┴─────────────────┴───────────────┴─────────────────┘ +``` + +You can use the [dictGet*](../../query_language/functions/ext_dict_functions.md#ext_dict_functions) function to get the dictionary data in this format. + +This view isn't helpful when you need to get raw data, or when performing a `JOIN` operation. For these cases, you can use the `Dictionary` engine, which displays the dictionary data in a table. + +Syntax: + +``` +CREATE TABLE %table_name% (%fields%) engine = Dictionary(%dictionary_name%)` +``` + +Usage example: + +``` sql +create table products (product_id UInt64, title String) Engine = Dictionary(products); + +CREATE TABLE products +( + product_id UInt64, + title String, +) +ENGINE = Dictionary(products) +``` + +``` +Ok. + +0 rows in set. Elapsed: 0.004 sec. +``` + +Take a look at what's in the table. + +``` sql +select * from products limit 1; + +SELECT * +FROM products +LIMIT 1 +``` + +``` +┌────product_id─┬─title───────────┐ +│ 152689 │ Some item │ +└───────────────┴─────────────────┘ + +1 rows in set. Elapsed: 0.006 sec. +``` + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/dictionary/) diff --git a/docs/zh/operations/table_engines/distributed.md b/docs/zh/operations/table_engines/distributed.md deleted file mode 120000 index 46994303c35..00000000000 --- a/docs/zh/operations/table_engines/distributed.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/distributed.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/distributed.md b/docs/zh/operations/table_engines/distributed.md new file mode 100644 index 00000000000..6bd60c1591d --- /dev/null +++ b/docs/zh/operations/table_engines/distributed.md @@ -0,0 +1,126 @@ + + +# Distributed + +**The Distributed engine does not store data itself**, but allows distributed query processing on multiple servers. +Reading is automatically parallelized. During a read, the table indexes on remote servers are used, if there are any. +The Distributed engine accepts parameters: the cluster name in the server's config file, the name of a remote database, the name of a remote table, and (optionally) a sharding key. +Example: + +``` +Distributed(logs, default, hits[, sharding_key]) +``` + +Data will be read from all servers in the 'logs' cluster, from the default.hits table located on every server in the cluster. +Data is not only read, but is partially processed on the remote servers (to the extent that this is possible). +For example, for a query with GROUP BY, data will be aggregated on remote servers, and the intermediate states of aggregate functions will be sent to the requestor server. Then data will be further aggregated. + +Instead of the database name, you can use a constant expression that returns a string. For example: currentDatabase(). + +logs – The cluster name in the server's config file. + +Clusters are set like this: + +```xml + + + + + 1 + + false + + example01-01-1 + 9000 + + + example01-01-2 + 9000 + + + + 2 + false + + example01-02-1 + 9000 + + + example01-02-2 + 1 + 9440 + + + + +``` + +Here a cluster is defined with the name 'logs' that consists of two shards, each of which contains two replicas. +Shards refer to the servers that contain different parts of the data (in order to read all the data, you must access all the shards). +Replicas are duplicating servers (in order to read all the data, you can access the data on any one of the replicas). + +Cluster names must not contain dots. + +The parameters `host`, `port`, and optionally `user`, `password`, `secure`, `compression` are specified for each server: + +: - `host` – The address of the remote server. You can use either the domain or the IPv4 or IPv6 address. If you specify the domain, the server makes a DNS request when it starts, and the result is stored as long as the server is running. If the DNS request fails, the server doesn't start. If you change the DNS record, restart the server. +- `port`– The TCP port for messenger activity ('tcp_port' in the config, usually set to 9000). Do not confuse it with http_port. +- `user`– Name of the user for connecting to a remote server. Default value: default. This user must have access to connect to the specified server. Access is configured in the users.xml file. For more information, see the section "Access rights". +- `password` – The password for connecting to a remote server (not masked). Default value: empty string. +- `secure` - Use ssl for connection, usually you also should define `port` = 9440. Server should listen on 9440 and have correct certificates. +- `compression` - Use data compression. Default value: true. + +When specifying replicas, one of the available replicas will be selected for each of the shards when reading. You can configure the algorithm for load balancing (the preference for which replica to access) – see the 'load_balancing' setting. +If the connection with the server is not established, there will be an attempt to connect with a short timeout. If the connection failed, the next replica will be selected, and so on for all the replicas. If the connection attempt failed for all the replicas, the attempt will be repeated the same way, several times. +This works in favor of resiliency, but does not provide complete fault tolerance: a remote server might accept the connection, but might not work, or work poorly. + +You can specify just one of the shards (in this case, query processing should be called remote, rather than distributed) or up to any number of shards. In each shard, you can specify from one to any number of replicas. You can specify a different number of replicas for each shard. + +You can specify as many clusters as you wish in the configuration. + +To view your clusters, use the 'system.clusters' table. + +The Distributed engine allows working with a cluster like a local server. However, the cluster is inextensible: you must write its configuration in the server config file (even better, for all the cluster's servers). + +There is no support for Distributed tables that look at other Distributed tables (except in cases when a Distributed table only has one shard). As an alternative, make the Distributed table look at the "final" tables. + +The Distributed engine requires writing clusters to the config file. Clusters from the config file are updated on the fly, without restarting the server. If you need to send a query to an unknown set of shards and replicas each time, you don't need to create a Distributed table – use the 'remote' table function instead. See the section "Table functions". + +There are two methods for writing data to a cluster: + +First, you can define which servers to write which data to, and perform the write directly on each shard. In other words, perform INSERT in the tables that the distributed table "looks at". +This is the most flexible solution – you can use any sharding scheme, which could be non-trivial due to the requirements of the subject area. +This is also the most optimal solution, since data can be written to different shards completely independently. + +Second, you can perform INSERT in a Distributed table. In this case, the table will distribute the inserted data across servers itself. +In order to write to a Distributed table, it must have a sharding key set (the last parameter). In addition, if there is only one shard, the write operation works without specifying the sharding key, since it doesn't have any meaning in this case. + +Each shard can have a weight defined in the config file. By default, the weight is equal to one. Data is distributed across shards in the amount proportional to the shard weight. For example, if there are two shards and the first has a weight of 9 while the second has a weight of 10, the first will be sent 9 / 19 parts of the rows, and the second will be sent 10 / 19. + +Each shard can have the 'internal_replication' parameter defined in the config file. + +If this parameter is set to 'true', the write operation selects the first healthy replica and writes data to it. Use this alternative if the Distributed table "looks at" replicated tables. In other words, if the table where data will be written is going to replicate them itself. + +If it is set to 'false' (the default), data is written to all replicas. In essence, this means that the Distributed table replicates data itself. This is worse than using replicated tables, because the consistency of replicas is not checked, and over time they will contain slightly different data. + +To select the shard that a row of data is sent to, the sharding expression is analyzed, and its remainder is taken from dividing it by the total weight of the shards. The row is sent to the shard that corresponds to the half-interval of the remainders from 'prev_weight' to 'prev_weights + weight', where 'prev_weights' is the total weight of the shards with the smallest number, and 'weight' is the weight of this shard. For example, if there are two shards, and the first has a weight of 9 while the second has a weight of 10, the row will be sent to the first shard for the remainders from the range \[0, 9), and to the second for the remainders from the range \[9, 19). + +The sharding expression can be any expression from constants and table columns that returns an integer. For example, you can use the expression 'rand()' for random distribution of data, or 'UserID' for distribution by the remainder from dividing the user's ID (then the data of a single user will reside on a single shard, which simplifies running IN and JOIN by users). If one of the columns is not distributed evenly enough, you can wrap it in a hash function: intHash64(UserID). + +A simple remainder from division is a limited solution for sharding and isn't always appropriate. It works for medium and large volumes of data (dozens of servers), but not for very large volumes of data (hundreds of servers or more). In the latter case, use the sharding scheme required by the subject area, rather than using entries in Distributed tables. + +SELECT queries are sent to all the shards, and work regardless of how data is distributed across the shards (they can be distributed completely randomly). When you add a new shard, you don't have to transfer the old data to it. You can write new data with a heavier weight – the data will be distributed slightly unevenly, but queries will work correctly and efficiently. + +You should be concerned about the sharding scheme in the following cases: + +- Queries are used that require joining data (IN or JOIN) by a specific key. If data is sharded by this key, you can use local IN or JOIN instead of GLOBAL IN or GLOBAL JOIN, which is much more efficient. +- A large number of servers is used (hundreds or more) with a large number of small queries (queries of individual clients - websites, advertisers, or partners). In order for the small queries to not affect the entire cluster, it makes sense to locate data for a single client on a single shard. Alternatively, as we've done in Yandex.Metrica, you can set up bi-level sharding: divide the entire cluster into "layers", where a layer may consist of multiple shards. Data for a single client is located on a single layer, but shards can be added to a layer as necessary, and data is randomly distributed within them. Distributed tables are created for each layer, and a single shared distributed table is created for global queries. + +Data is written asynchronously. For an INSERT to a Distributed table, the data block is just written to the local file system. The data is sent to the remote servers in the background as soon as possible. You should check whether data is sent successfully by checking the list of files (data waiting to be sent) in the table directory: /var/lib/clickhouse/data/database/table/. + +If the server ceased to exist or had a rough restart (for example, after a device failure) after an INSERT to a Distributed table, the inserted data might be lost. If a damaged data part is detected in the table directory, it is transferred to the 'broken' subdirectory and no longer used. + +When the max_parallel_replicas option is enabled, query processing is parallelized across all replicas within a single shard. For more information, see the section "Settings, max_parallel_replicas". + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/distributed/) diff --git a/docs/zh/operations/table_engines/external_data.md b/docs/zh/operations/table_engines/external_data.md deleted file mode 120000 index 27a7b6acec2..00000000000 --- a/docs/zh/operations/table_engines/external_data.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/external_data.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/external_data.md b/docs/zh/operations/table_engines/external_data.md new file mode 100644 index 00000000000..ea27c9f0e1c --- /dev/null +++ b/docs/zh/operations/table_engines/external_data.md @@ -0,0 +1,64 @@ + + +# External Data for Query Processing + +ClickHouse allows sending a server the data that is needed for processing a query, together with a SELECT query. This data is put in a temporary table (see the section "Temporary tables") and can be used in the query (for example, in IN operators). + +For example, if you have a text file with important user identifiers, you can upload it to the server along with a query that uses filtration by this list. + +If you need to run more than one query with a large volume of external data, don't use this feature. It is better to upload the data to the DB ahead of time. + +External data can be uploaded using the command-line client (in non-interactive mode), or using the HTTP interface. + +In the command-line client, you can specify a parameters section in the format + +```bash +--external --file=... [--name=...] [--format=...] [--types=...|--structure=...] +``` + +You may have multiple sections like this, for the number of tables being transmitted. + +**--external** – Marks the beginning of a clause. +**--file** – Path to the file with the table dump, or -, which refers to stdin. +Only a single table can be retrieved from stdin. + +The following parameters are optional: **--name**– Name of the table. If omitted, _data is used. +**--format** – Data format in the file. If omitted, TabSeparated is used. + +One of the following parameters is required:**--types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named _1, _2, ... +**--structure**– The table structure in the format`UserID UInt64`, `URL String`. Defines the column names and types. + +The files specified in 'file' will be parsed by the format specified in 'format', using the data types specified in 'types' or 'structure'. The table will be uploaded to the server and accessible there as a temporary table with the name in 'name'. + +Examples: + +```bash +echo -ne "1\n2\n3\n" | clickhouse-client --query="SELECT count() FROM test.visits WHERE TraficSourceID IN _data" --external --file=- --types=Int8 +849897 +cat /etc/passwd | sed 's/:/\t/g' | clickhouse-client --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' +/bin/sh 20 +/bin/false 5 +/bin/bash 4 +/usr/sbin/nologin 1 +/bin/sync 1 +``` + +When using the HTTP interface, external data is passed in the multipart/form-data format. Each table is transmitted as a separate file. The table name is taken from the file name. The 'query_string' is passed the parameters 'name_format', 'name_types', and 'name_structure', where 'name' is the name of the table that these parameters correspond to. The meaning of the parameters is the same as when using the command-line client. + +Example: + +```bash +cat /etc/passwd | sed 's/:/\t/g' > passwd.tsv + +curl -F 'passwd=@passwd.tsv;' 'http://localhost:8123/?query=SELECT+shell,+count()+AS+c+FROM+passwd+GROUP+BY+shell+ORDER+BY+c+DESC&passwd_structure=login+String,+unused+String,+uid+UInt16,+gid+UInt16,+comment+String,+home+String,+shell+String' +/bin/sh 20 +/bin/false 5 +/bin/bash 4 +/usr/sbin/nologin 1 +/bin/sync 1 +``` + +For distributed query processing, the temporary tables are sent to all the remote servers. + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/external_data/) diff --git a/docs/zh/operations/table_engines/file.md b/docs/zh/operations/table_engines/file.md deleted file mode 120000 index 27dffc8d78f..00000000000 --- a/docs/zh/operations/table_engines/file.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/file.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/file.md b/docs/zh/operations/table_engines/file.md new file mode 100644 index 00000000000..ed49a693630 --- /dev/null +++ b/docs/zh/operations/table_engines/file.md @@ -0,0 +1,80 @@ + + +# File(InputFormat) + +The data source is a file that stores data in one of the supported input formats (TabSeparated, Native, etc.). + +Usage examples: + +- Data export from ClickHouse to file. +- Convert data from one format to another. +- Updating data in ClickHouse via editing a file on a disk. + +## Usage in ClickHouse Server + +``` +File(Format) +``` + +`Format` should be supported for either `INSERT` and `SELECT`. For the full list of supported formats see [Formats](../../interfaces/formats.md#formats). + +ClickHouse does not allow to specify filesystem path for`File`. It will use folder defined by [path](../server_settings/settings.md#server_settings-path) setting in server configuration. + +When creating table using `File(Format)` it creates empty subdirectory in that folder. When data is written to that table, it's put into `data.Format` file in that subdirectory. + +You may manually create this subfolder and file in server filesystem and then [ATTACH](../../query_language/misc.md#queries-attach) it to table information with matching name, so you can query data from that file. + +!!! warning + Be careful with this funcionality, because ClickHouse does not keep track of external changes to such files. The result of simultaneous writes via ClickHouse and outside of ClickHouse is undefined. + +**Example:** + +**1.** Set up the `file_engine_table` table: + +``` sql +CREATE TABLE file_engine_table (name String, value UInt32) ENGINE=File(TabSeparated) +``` + +By default ClickHouse will create folder `/var/lib/clickhouse/data/default/file_engine_table`. + +**2.** Manually create `/var/lib/clickhouse/data/default/file_engine_table/data.TabSeparated` containing: + +```bash +$ cat data.TabSeparated +one 1 +two 2 +``` + +**3.** Query the data: + +``` sql +SELECT * FROM file_engine_table +``` + +``` +┌─name─┬─value─┐ +│ one │ 1 │ +│ two │ 2 │ +└──────┴───────┘ +``` + +## Usage in Clickhouse-local + +In [clickhouse-local](../utils/clickhouse-local.md#utils-clickhouse-local) File engine accepts file path in addition to `Format`. Default input/output streams can be specified using numeric or human-readable names like `0` or `stdin`, `1` or `stdout`. + +**Example:** + +```bash +$ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64) ENGINE = File(CSV, stdin); SELECT a, b FROM table; DROP TABLE table" +``` + +## Details of Implementation + +- Reads can be parallel, but not writes +- Not supported: + - `ALTER` + - `SELECT ... SAMPLE` + - Indices + - Replication + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/file/) diff --git a/docs/zh/operations/table_engines/graphitemergetree.md b/docs/zh/operations/table_engines/graphitemergetree.md deleted file mode 120000 index 654425d050a..00000000000 --- a/docs/zh/operations/table_engines/graphitemergetree.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/graphitemergetree.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/graphitemergetree.md b/docs/zh/operations/table_engines/graphitemergetree.md new file mode 100644 index 00000000000..0aad07a13c3 --- /dev/null +++ b/docs/zh/operations/table_engines/graphitemergetree.md @@ -0,0 +1,149 @@ + + +# GraphiteMergeTree + +This engine is designed for rollup (thinning and aggregating/averaging) [Graphite](http://graphite.readthedocs.io/en/latest/index.html) data. It may be helpful to developers who want to use ClickHouse as a data store for Graphite. + +You can use any ClickHouse table engine to store the Graphite data if you don't need rollup, but if you need a rollup use `GraphiteMergeTree`. The engine reduces the volume of storage and increases the efficiency of queries from Graphite. + +The engine inherits properties from [MergeTree](mergetree.md#table_engines-mergetree). + +## Creating a Table + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + Path String, + Time DateTime, + Value , + Version + ... +) ENGINE = GraphiteMergeTree(config_section) +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] +``` + +For a description of request parameters, see [request description](../../query_language/create.md#query_language-queries-create_table). + +A table for the Graphite date should have the following columns: + +- Column with the metric name (Graphite sensor). Data type: `String`. +- Column with the time for measuring the metric. Data type: `DateTime`. +- Column with the value of the metric. Data type: any numeric. +- Column with the version of the metric with the same name and time of measurement. Data type: any numeric. + + ClickHouse saves the rows with the highest version or the last written if versions are the same. Other rows are deleted during the merge of data parts. + +The names of these columns should be set in the rollup configuration. + +**GraphiteMergeTree parameters** + +- `config_section` — Name of the section in the configuration file, where are the rules of rollup set. + +**Query clauses** + +When creating a `GraphiteMergeTree` table, the same [clauses](mergetree.md#table_engines-mergetree-configuring) are required, as when creating a `MergeTree` table. + +
Deprecated Method for Creating a Table + +!!! attention + Do not use this method in new projects and, if possible, switch the old projects to the method described above. + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + EventDate Date, + Path String, + Time DateTime, + Value , + Version + ... +) ENGINE [=] GraphiteMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, config_section) +``` + +All of the parameters excepting `config_section` have the same meaning as in `MergeTree`. + +- `config_section` — Name of the section in the configuration file, where are the rules of rollup set. +
+ +## Rollup configuration + +The settings for rollup are defined by the [graphite_rollup](../server_settings/settings.md#server_settings-graphite_rollup) parameter in the server configuration. The name of the parameter could be any. You can create several configurations and use them for different tables. + +Rollup configuration structure: + +``` +required-columns +pattern + regexp + function + age + precision + ... +pattern + ... +default + function + age + precision + ... +``` + +When processing a row, ClickHouse checks the rules in the `pattern` section. If the metric name matches the `regexp`, the rules from the `pattern`section are applied; otherwise, the rules from the `default` section are used. + +The rules are defined with fields `function` and `age + precision`. + +Fields for `pattern` and `default` sections: + +- `regexp`– A pattern for the metric name. +- `age` – The minimum age of the data in seconds. +- `precision`– How precisely to define the age of the data in seconds. +- `function` – The name of the aggregating function to apply to data whose age falls within the range `[age, age + precision]`. + +The `required-columns`: + +- `path_column_name` — Column with the metric name (Graphite sensor). +- `time_column_name` — Column with the time for measuring the metric. +- `value_column_name` — Column with the value of the metric at the time set in `time_column_name`. +- `version_column_name` — Column with the version timestamp of the metric with the same name and time remains in the database. + + +Example of settings: + +```xml + + Path + Time + Value + Version + + click_cost + any + + 0 + 5 + + + 86400 + 60 + + + + max + + 0 + 60 + + + 3600 + 300 + + + 86400 + 3600 + + + +``` + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/graphitemergetree/) diff --git a/docs/zh/operations/table_engines/join.md b/docs/zh/operations/table_engines/join.md deleted file mode 120000 index 0914ab950ed..00000000000 --- a/docs/zh/operations/table_engines/join.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/join.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/join.md b/docs/zh/operations/table_engines/join.md new file mode 100644 index 00000000000..299071312b8 --- /dev/null +++ b/docs/zh/operations/table_engines/join.md @@ -0,0 +1,19 @@ +# Join + +A prepared data structure for JOIN that is always located in RAM. + +``` +Join(ANY|ALL, LEFT|INNER, k1[, k2, ...]) +``` + +Engine parameters: `ANY|ALL` – strictness; `LEFT|INNER` – type. +These parameters are set without quotes and must match the JOIN that the table will be used for. k1, k2, ... are the key columns from the USING clause that the join will be made on. + +The table can't be used for GLOBAL JOINs. + +You can use INSERT to add data to the table, similar to the Set engine. For ANY, data for duplicated keys will be ignored. For ALL, it will be counted. You can't perform SELECT directly from the table. The only way to retrieve data is to use it as the "right-hand" table for JOIN. + +Storing data on the disk is the same as for the Set engine. + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/join/) diff --git a/docs/zh/operations/table_engines/kafka.md b/docs/zh/operations/table_engines/kafka.md deleted file mode 120000 index cb7bd5dd0f8..00000000000 --- a/docs/zh/operations/table_engines/kafka.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/kafka.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/kafka.md b/docs/zh/operations/table_engines/kafka.md new file mode 100644 index 00000000000..db86553676f --- /dev/null +++ b/docs/zh/operations/table_engines/kafka.md @@ -0,0 +1,139 @@ +# Kafka + +此引擎与 [Apache Kafka](http://kafka.apache.org/) 结合使用。 + +Kafka 特性: + +- 发布或者订阅数据流。 +- 容错存储机制。 +- 处理流数据。 + +老版格式: + +``` +Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format + [, kafka_row_delimiter, kafka_schema, kafka_num_consumers]) +``` + +新版格式: + +``` +Kafka SETTINGS + kafka_broker_list = 'localhost:9092', + kafka_topic_list = 'topic1,topic2', + kafka_group_name = 'group1', + kafka_format = 'JSONEachRow', + kafka_row_delimiter = '\n' + kafka_schema = '', + kafka_num_consumers = 2 +``` + +必要参数: + +- `kafka_broker_list` – 以逗号分隔的 brokers 列表 (`localhost:9092`)。 +- `kafka_topic_list` – topic 列表 (`my_topic`)。 +- `kafka_group_name` – Kafka 消费组名称 (`group1`)。如果不希望消息在集群中重复,请在每个分片中使用相同的组名。 +- `kafka_format` – 消息体格式。使用与 SQL 部分的 `FORMAT` 函数相同表示方法,例如 `JSONEachRow`。了解详细信息,请参考 `Formats` 部分。 + +可选参数: + +- `kafka_row_delimiter` - 每个消息体(记录)之间的分隔符。 +- `kafka_schema` – 如果解析格式需要一个 schema 时,此参数必填。例如,[Cap'n Proto](https://capnproto.org/) 需要 schema 文件路径以及根对象 `schema.capnp:Message` 的名字。 +- `kafka_num_consumers` – 单个表的消费者数量。默认值是:`1`,如果一个消费者的吞吐量不足,则指定更多的消费者。消费者的总数不应该超过 topic 中分区的数量,因为每个分区只能分配一个消费者。 + +示例: + +``` sql + CREATE TABLE queue ( + timestamp UInt64, + level String, + message String + ) ENGINE = Kafka('localhost:9092', 'topic', 'group1', 'JSONEachRow'); + + SELECT * FROM queue LIMIT 5; + + CREATE TABLE queue2 ( + timestamp UInt64, + level String, + message String + ) ENGINE = Kafka SETTINGS kafka_broker_list = 'localhost:9092', + kafka_topic_list = 'topic', + kafka_group_name = 'group1', + kafka_format = 'JSONEachRow', + kafka_num_consumers = 4; + + CREATE TABLE queue2 ( + timestamp UInt64, + level String, + message String + ) ENGINE = Kafka('localhost:9092', 'topic', 'group1') + SETTINGS kafka_format = 'JSONEachRow', + kafka_num_consumers = 4; +``` + +消费的消息会被自动追踪,因此每个消息在不同的消费组里只会记录一次。如果希望获得两次数据,则使用另一个组名创建副本。 + +消费组可以灵活配置并且在集群之间同步。例如,如果群集中有10个主题和5个表副本,则每个副本将获得2个主题。 如果副本数量发生变化,主题将自动在副本中重新分配。了解更多信息请访问 [http://kafka.apache.org/intro](http://kafka.apache.org/intro)。 + +`SELECT` 查询对于读取消息并不是很有用(调试除外),因为每条消息只能被读取一次。使用物化视图创建实时线程更实用。您可以这样做: + +1. 使用引擎创建一个 Kafka 消费者并作为一条数据流。 +2. 创建一个结构表。 +3. 创建物化视图,改视图会在后台转换引擎中的数据并将其放入之前创建的表中。 + +当 `MATERIALIZED VIEW` 添加至引擎,它将会在后台收集数据。可以持续不断地从 Kafka 收集数据并通过 `SELECT` 将数据转换为所需要的格式。 + +示例: + +``` sql + CREATE TABLE queue ( + timestamp UInt64, + level String, + message String + ) ENGINE = Kafka('localhost:9092', 'topic', 'group1', 'JSONEachRow'); + + CREATE TABLE daily ( + day Date, + level String, + total UInt64 + ) ENGINE = SummingMergeTree(day, (day, level), 8192); + + CREATE MATERIALIZED VIEW consumer TO daily + AS SELECT toDate(toDateTime(timestamp)) AS day, level, count() as total + FROM queue GROUP BY day, level; + + SELECT level, sum(total) FROM daily GROUP BY level; +``` + +为了提高性能,接受的消息被分组为 [max_insert_block_size](../settings/settings.md#settings-settings-max_insert_block_size) 大小的块。如果未在 [stream_flush_interval_ms](../settings/settings.md#settings-settings_stream_flush_interval_ms) 毫秒内形成块,则不关心块的完整性,都会将数据刷新到表中。 + +停止接收主题数据或更改转换逻辑,请 detach 物化视图: + +``` + DETACH TABLE consumer; + ATTACH MATERIALIZED VIEW consumer; +``` + +如果使用 `ALTER` 更改目标表,为了避免目标表与视图中的数据之间存在差异,推荐停止物化视图。 + +## 配置 + +与 `GraphiteMergeTree` 类似,Kafka 引擎支持使用ClickHouse配置文件进行扩展配置。可以使用两个配置键:全局 (`kafka`) 和 主题级别 (`kafka_*`)。首先应用全局配置,然后应用主题级配置(如果存在)。 + +```xml + + + cgrp + smallest + + + + + 250 + 100000 + +``` + +有关详细配置选项列表,请参阅 [librdkafka configuration reference](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md)。在 ClickHouse 配置中使用下划线 (`_`) ,并不是使用点 (`.`)。例如,`check.crcs=true` 将是 `true`。 + +[Original article](https://clickhouse.yandex/docs/zh/operations/table_engines/kafka/) diff --git a/docs/zh/operations/table_engines/log.md b/docs/zh/operations/table_engines/log.md deleted file mode 120000 index 2c39ba68522..00000000000 --- a/docs/zh/operations/table_engines/log.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/log.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/log.md b/docs/zh/operations/table_engines/log.md new file mode 100644 index 00000000000..fffc5a11aca --- /dev/null +++ b/docs/zh/operations/table_engines/log.md @@ -0,0 +1,8 @@ +# Log + +Log differs from TinyLog in that a small file of "marks" resides with the column files. These marks are written on every data block and contain offsets that indicate where to start reading the file in order to skip the specified number of rows. This makes it possible to read table data in multiple threads. +For concurrent data access, the read operations can be performed simultaneously, while write operations block reads and each other. +The Log engine does not support indexes. Similarly, if writing to a table failed, the table is broken, and reading from it returns an error. The Log engine is appropriate for temporary data, write-once tables, and for testing or demonstration purposes. + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/log/) diff --git a/docs/zh/operations/table_engines/materializedview.md b/docs/zh/operations/table_engines/materializedview.md deleted file mode 120000 index e3b5deb73dc..00000000000 --- a/docs/zh/operations/table_engines/materializedview.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/materializedview.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/materializedview.md b/docs/zh/operations/table_engines/materializedview.md new file mode 100644 index 00000000000..c13a1ac8710 --- /dev/null +++ b/docs/zh/operations/table_engines/materializedview.md @@ -0,0 +1,6 @@ +# MaterializedView + +Used for implementing materialized views (for more information, see [CREATE TABLE](../../query_language/create.md#query_language-queries-create_table)). For storing data, it uses a different engine that was specified when creating the view. When reading from a table, it just uses this engine. + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/materializedview/) diff --git a/docs/zh/operations/table_engines/memory.md b/docs/zh/operations/table_engines/memory.md deleted file mode 120000 index eee940c7bd3..00000000000 --- a/docs/zh/operations/table_engines/memory.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/memory.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/memory.md b/docs/zh/operations/table_engines/memory.md new file mode 100644 index 00000000000..d68ae923684 --- /dev/null +++ b/docs/zh/operations/table_engines/memory.md @@ -0,0 +1,13 @@ +# Memory + +The Memory engine stores data in RAM, in uncompressed form. Data is stored in exactly the same form as it is received when read. In other words, reading from this table is completely free. +Concurrent data access is synchronized. Locks are short: read and write operations don't block each other. +Indexes are not supported. Reading is parallelized. +Maximal productivity (over 10 GB/sec) is reached on simple queries, because there is no reading from the disk, decompressing, or deserializing data. (We should note that in many cases, the productivity of the MergeTree engine is almost as high.) +When restarting a server, data disappears from the table and the table becomes empty. +Normally, using this table engine is not justified. However, it can be used for tests, and for tasks where maximum speed is required on a relatively small number of rows (up to approximately 100,000,000). + +The Memory engine is used by the system for temporary tables with external query data (see the section "External data for processing a query"), and for implementing GLOBAL IN (see the section "IN operators"). + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/memory/) diff --git a/docs/zh/operations/table_engines/merge.md b/docs/zh/operations/table_engines/merge.md deleted file mode 120000 index 9e17d9bb939..00000000000 --- a/docs/zh/operations/table_engines/merge.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/merge.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/merge.md b/docs/zh/operations/table_engines/merge.md new file mode 100644 index 00000000000..e8de53bc286 --- /dev/null +++ b/docs/zh/operations/table_engines/merge.md @@ -0,0 +1,69 @@ +# Merge + +The `Merge` engine (not to be confused with `MergeTree`) does not store data itself, but allows reading from any number of other tables simultaneously. +Reading is automatically parallelized. Writing to a table is not supported. When reading, the indexes of tables that are actually being read are used, if they exist. +The `Merge` engine accepts parameters: the database name and a regular expression for tables. + +Example: + +``` +Merge(hits, '^WatchLog') +``` + +Data will be read from the tables in the `hits` database that have names that match the regular expression '`^WatchLog`'. + +Instead of the database name, you can use a constant expression that returns a string. For example, `currentDatabase()`. + +Regular expressions — [re2](https://github.com/google/re2) (supports a subset of PCRE), case-sensitive. +See the notes about escaping symbols in regular expressions in the "match" section. + +When selecting tables to read, the `Merge` table itself will not be selected, even if it matches the regex. This is to avoid loops. +It is possible to create two `Merge` tables that will endlessly try to read each others' data, but this is not a good idea. + +The typical way to use the `Merge` engine is for working with a large number of `TinyLog` tables as if with a single table. + +Example 2: + +Let's say you have a old table (WatchLog_old) and decided to change partitioning without moving data to a new table (WatchLog_new) and you need to see data from both tables. + +``` +CREATE TABLE WatchLog_old(date Date, UserId Int64, EventType String, Cnt UInt64) +ENGINE=MergeTree(date, (UserId, EventType), 8192); +INSERT INTO WatchLog_old VALUES ('2018-01-01', 1, 'hit', 3); + +CREATE TABLE WatchLog_new(date Date, UserId Int64, EventType String, Cnt UInt64) +ENGINE=MergeTree PARTITION BY date ORDER BY (UserId, EventType) SETTINGS index_granularity=8192; +INSERT INTO WatchLog_new VALUES ('2018-01-02', 2, 'hit', 3); + +CREATE TABLE WatchLog as WatchLog_old ENGINE=Merge(currentDatabase(), '^WatchLog'); + +SELECT * +FROM WatchLog + +┌───────date─┬─UserId─┬─EventType─┬─Cnt─┐ +│ 2018-01-01 │ 1 │ hit │ 3 │ +└────────────┴────────┴───────────┴─────┘ +┌───────date─┬─UserId─┬─EventType─┬─Cnt─┐ +│ 2018-01-02 │ 2 │ hit │ 3 │ +└────────────┴────────┴───────────┴─────┘ + +``` + +## Virtual Columns + +Virtual columns are columns that are provided by the table engine, regardless of the table definition. In other words, these columns are not specified in `CREATE TABLE`, but they are accessible for `SELECT`. + +Virtual columns differ from normal columns in the following ways: + +- They are not specified in table definitions. +- Data can't be added to them with `INSERT`. +- When using `INSERT` without specifying the list of columns, virtual columns are ignored. +- They are not selected when using the asterisk (`SELECT *`). +- Virtual columns are not shown in `SHOW CREATE TABLE` and `DESC TABLE` queries. + +The `Merge` type table contains a virtual `_table` column of the `String` type. (If the table already has a `_table` column, the virtual column is called `_table1`; if you already have `_table1`, it's called `_table2`, and so on.) It contains the name of the table that data was read from. + +If the `WHERE/PREWHERE` clause contains conditions for the `_table` column that do not depend on other table columns (as one of the conjunction elements, or as an entire expression), these conditions are used as an index. The conditions are performed on a data set of table names to read data from, and the read operation will be performed from only those tables that the condition was triggered on. + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/merge/) diff --git a/docs/zh/operations/table_engines/mergetree.md b/docs/zh/operations/table_engines/mergetree.md deleted file mode 120000 index cc6ac1e5297..00000000000 --- a/docs/zh/operations/table_engines/mergetree.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/mergetree.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/mergetree.md b/docs/zh/operations/table_engines/mergetree.md new file mode 100644 index 00000000000..7b4ecd51fe7 --- /dev/null +++ b/docs/zh/operations/table_engines/mergetree.md @@ -0,0 +1,209 @@ + + +# MergeTree + +The `MergeTree` engine and other engines of this family (`*MergeTree`) are the most robust ClickHousе table engines. + +The basic idea for `MergeTree` engines family is the following. When you have tremendous amount of a data that should be inserted into the table, you should write them quickly part by part and then merge parts by some rules in background. This method is much more efficient than constantly rewriting data in the storage at the insert. + +Main features: + +- Stores data sorted by primary key. + + This allows you to create a small sparse index that helps find data faster. + +- This allows you to use partitions if the [partitioning key](custom_partitioning_key.md#table_engines-custom_partitioning_key) is specified. + + ClickHouse supports certain operations with partitions that are more effective than general operations on the same data with the same result. ClickHouse also automatically cuts off the partition data where the partitioning key is specified in the query. This also increases the query performance. + +- Data replication support. + + The family of `ReplicatedMergeTree` tables is used for this. For more information, see the [Data replication](replication.md#table_engines-replication) section. + +- Data sampling support. + + If necessary, you can set the data sampling method in the table. + +!!! info + The [Merge](merge.md#table_engine-merge) engine does not belong to the `*MergeTree` family. + + + +## Creating a Table + +``` +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = MergeTree() +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] +``` + +For a description of request parameters, see [request description](../../query_language/create.md#query_language-queries-create_table). + +**Query clauses** + +- `ENGINE` - Name and parameters of the engine. `ENGINE = MergeTree()`. `MergeTree` engine does not have parameters. + +- `ORDER BY` — Primary key. + + A tuple of columns or arbitrary expressions. Example: `ORDER BY (CounterID, EventDate)`. +If a sampling expression is used, the primary key must contain it. Example: `ORDER BY (CounerID, EventDate, intHash32(UserID))`. + +- `PARTITION BY` — The [partitioning key](custom_partitioning_key.md#table_engines-custom_partitioning_key). + + For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](../../data_types/date.md#data_type-date). The partition names here have the `"YYYYMM"` format. + +- `SAMPLE BY` — An expression for sampling. Example: `intHash32(UserID))`. + +- `SETTINGS` — Additional parameters that control the behavior of the `MergeTree`: + - `index_granularity` — The granularity of an index. The number of data rows between the "marks" of an index. By default, 8192. + +**Example of sections setting** + +``` +ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192 +``` + +In the example, we set partitioning by month. + +We also set an expression for sampling as a hash by the user ID. This allows you to pseudorandomize the data in the table for each `CounterID` and `EventDate`. If, when selecting the data, you define a [SAMPLE](../../query_language/select.md#select-section-sample) clause, ClickHouse will return an evenly pseudorandom data sample for a subset of users. + +`index_granularity` could be omitted because 8192 is the default value. + +
Deprecated Method for Creating a Table + +!!! attention + Do not use this method in new projects and, if possible, switch the old projects to the method described above. + +``` +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] MergeTree(date-column [, sampling_expression], (primary, key), index_granularity) +``` + +**MergeTree() parameters** + +- `date-column` — The name of a column of the type [Date](../../data_types/date.md#data_type-date). ClickHouse automatically creates partitions by month on the basis of this column. The partition names are in the `"YYYYMM"` format. +- `sampling_expression` — an expression for sampling. +- `(primary, key)` — primary key. Type — [Tuple()](../../data_types/tuple.md#data_type-tuple). It may consist of arbitrary expressions, but it typically is a tuple of columns. It must include an expression for sampling if it is set. It must not include a column with a `date-column` date. +- `index_granularity` — The granularity of an index. The number of data rows between the "marks" of an index. The value 8192 is appropriate for most tasks. + +**Example** + +``` +MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID)), 8192) +``` + +The `MergeTree` engine is configured in the same way as in the example above for the main engine configuration method. +
+ +## Data Storage + +A table consists of data *parts* sorted by primary key. + +When data is inserted in a table, separate data parts are created and each of them is lexicographically sorted by primary key. For example, if the primary key is `(CounterID, Date)`, the data in the part is sorted by `CounterID`, and within each `CounterID`, it is ordered by `Date`. + +Data belonging to different partitions are separated into different parts. In the background, ClickHouse merges data parts for more efficient storage. Parts belonging to different partitions are not merged. The merge mechanism does not guarantee that all rows with the same primary key will be in the same data part. + +For each data part, ClickHouse creates an index file that contains the primary key value for each index row ("mark"). Index row numbers are defined as `n * index_granularity`. The maximum value `n` is equal to the integer part of dividing the total number of rows by the `index_granularity`. For each column, the "marks" are also written for the same index rows as the primary key. These "marks" allow you to find the data directly in the columns. + +You can use a single large table and continually add data to it in small chunks – this is what the `MergeTree` engine is intended for. + +## Primary Keys and Indexes in Queries + +Let's take the `(CounterID, Date)` primary key. In this case, the sorting and index can be illustrated as follows: + +``` +Whole data: [-------------------------------------------------------------------------] +CounterID: [aaaaaaaaaaaaaaaaaabbbbcdeeeeeeeeeeeeefgggggggghhhhhhhhhiiiiiiiiikllllllll] +Date: [1111111222222233331233211111222222333211111112122222223111112223311122333] +Marks: | | | | | | | | | | | + a,1 a,2 a,3 b,3 e,2 e,3 g,1 h,2 i,1 i,3 l,3 +Marks numbers: 0 1 2 3 4 5 6 7 8 9 10 +``` + +If the data query specifies: + +- `CounterID in ('a', 'h')`, the server reads the data in the ranges of marks `[0, 3)` and `[6, 8)`. +- `CounterID IN ('a', 'h') AND Date = 3`, the server reads the data in the ranges of marks `[1, 3)` and `[7, 8)`. +- `Date = 3`, the server reads the data in the range of marks `[1, 10)`. + +The examples above show that it is always more effective to use an index than a full scan. + +A sparse index allows extra strings to be read. When reading a single range of the primary key, up to `index_granularity * 2` extra rows in each data block can be read. In most cases, ClickHouse performance does not degrade when `index_granularity = 8192`. + +Sparse indexes allow you to work with a very large number of table rows, because such indexes are always stored in the computer's RAM. + +ClickHouse does not require a unique primary key. You can insert multiple rows with the same primary key. + +### Selecting the Primary Key + +The number of columns in the primary key is not explicitly limited. Depending on the data structure, you can include more or fewer columns in the primary key. This may: + +- Improve the performance of an index. + + If the primary key is `(a, b)`, then adding another column `c` will improve the performance if the following conditions are met: + - There are queries with a condition on column `c`. + - Long data ranges (several times longer than the `index_granularity`) with identical values for `(a, b)` are common. In other words, when adding another column allows you to skip quite long data ranges. + +- Improve data compression. + + ClickHouse sorts data by primary key, so the higher the consistency, the better the compression. + +- Provide additional logic when data parts merging in the [CollapsingMergeTree](collapsingmergetree.md#table_engine-collapsingmergetree) and [SummingMergeTree](summingmergetree.md#table_engine-summingmergetree) engines. + + You may need many fields in the primary key even if they are not necessary for the previous steps. + +A long primary key will negatively affect the insert performance and memory consumption, but extra columns in the primary key do not affect ClickHouse performance during `SELECT` queries. + +### Use of Indexes and Partitions in Queries + +For`SELECT` queries, ClickHouse analyzes whether an index can be used. An index can be used if the `WHERE/PREWHERE` clause has an expression (as one of the conjunction elements, or entirely) that represents an equality or inequality comparison operation, or if it has `IN` or `LIKE` with a fixed prefix on columns or expressions that are in the primary key or partitioning key, or on certain partially repetitive functions of these columns, or logical relationships of these expressions. + +Thus, it is possible to quickly run queries on one or many ranges of the primary key. In this example, queries will be fast when run for a specific tracking tag; for a specific tag and date range; for a specific tag and date; for multiple tags with a date range, and so on. + +Let's look at the engine configured as follows: + +``` +ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate) SETTINGS index_granularity=8192 +``` + +In this case, in queries: + +``` sql +SELECT count() FROM table WHERE EventDate = toDate(now()) AND CounterID = 34 +SELECT count() FROM table WHERE EventDate = toDate(now()) AND (CounterID = 34 OR CounterID = 42) +SELECT count() FROM table WHERE ((EventDate >= toDate('2014-01-01') AND EventDate <= toDate('2014-01-31')) OR EventDate = toDate('2014-05-01')) AND CounterID IN (101500, 731962, 160656) AND (CounterID = 101500 OR EventDate != toDate('2014-05-01')) +``` + +ClickHouse will use the primary key index to trim improper data and the monthly partitioning key to trim partitions that are in improper date ranges. + +The queries above show that the index is used even for complex expressions. Reading from the table is organized so that using the index can't be slower than a full scan. + +In the example below, the index can't be used. + +``` sql +SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' +``` + +To check whether ClickHouse can use the index when running a query, use the settings [force_index_by_date](../settings/settings.md#settings-settings-force_index_by_date) and [force_primary_key](../settings/settings.md#settings-settings-force_primary_key). + +The key for partitioning by month allows reading only those data blocks which contain dates from the proper range. In this case, the data block may contain data for many dates (up to an entire month). Within a block, data is sorted by primary key, which might not contain the date as the first column. Because of this, using a query with only a date condition that does not specify the primary key prefix will cause more data to be read than for a single date. + +## Concurrent Data Access + +For concurrent table access, we use multi-versioning. In other words, when a table is simultaneously read and updated, data is read from a set of parts that is current at the time of the query. There are no lengthy locks. Inserts do not get in the way of read operations. + +Reading from a table is automatically parallelized. + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/mergetree/) diff --git a/docs/zh/operations/table_engines/mysql.md b/docs/zh/operations/table_engines/mysql.md deleted file mode 120000 index e4c268658cf..00000000000 --- a/docs/zh/operations/table_engines/mysql.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/mysql.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/mysql.md b/docs/zh/operations/table_engines/mysql.md new file mode 100644 index 00000000000..8baceafc64c --- /dev/null +++ b/docs/zh/operations/table_engines/mysql.md @@ -0,0 +1,27 @@ +# MySQL + +MySQL 引擎可以对存储在远程 MySQL 服务器上的数据执行 `SELECT` 查询。 + +调用格式: + +``` +MySQL('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_duplicate_clause']); +``` + +**调用参数** + +- `host:port` — MySQL 服务器地址。 +- `database` — 数据库的名称。 +- `table` — 表名称。 +- `user` — 数据库用户。 +- `password` — 用户密码。 +- `replace_query` — 将 `INSERT INTO` 查询是否替换为 `REPLACE INTO` 的标志。如果 `replace_query=1`,则替换查询 +- `'on_duplicate_clause'` — 将 `ON DUPLICATE KEY UPDATE 'on_duplicate_clause'` 表达式添加到 `INSERT` 查询语句中。例如:`impression = VALUES(impression) + impression`。如果需要指定 `'on_duplicate_clause'`,则需要设置 `replace_query=0`。如果同时设置 `replace_query = 1` 和 `'on_duplicate_clause'`,则会抛出异常。 + +此时,简单的 `WHERE` 子句(例如 ` =, !=, >, >=, <, <=`)是在 MySQL 服务器上执行。 + +其余条件以及 `LIMIT` 采样约束语句仅在对MySQL的查询完成后才在ClickHouse中执行。 + +`MySQL` 引擎不支持 [Nullable](../../data_types/nullable.md#data_type-nullable) 数据类型,因此,当从MySQL表中读取数据时,`NULL` 将转换为指定列类型的默认值(通常为0或空字符串)。 + +[Original article](https://clickhouse.yandex/docs/zh/operations/table_engines/mysql/) diff --git a/docs/zh/operations/table_engines/null.md b/docs/zh/operations/table_engines/null.md deleted file mode 120000 index c7d9264571e..00000000000 --- a/docs/zh/operations/table_engines/null.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/null.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/null.md b/docs/zh/operations/table_engines/null.md new file mode 100644 index 00000000000..58d3552d19d --- /dev/null +++ b/docs/zh/operations/table_engines/null.md @@ -0,0 +1,8 @@ +# Null + +When writing to a Null table, data is ignored. When reading from a Null table, the response is empty. + +However, you can create a materialized view on a Null table. So the data written to the table will end up in the view. + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/null/) diff --git a/docs/zh/operations/table_engines/replacingmergetree.md b/docs/zh/operations/table_engines/replacingmergetree.md deleted file mode 120000 index 63ff25a4dd6..00000000000 --- a/docs/zh/operations/table_engines/replacingmergetree.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/replacingmergetree.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/replacingmergetree.md b/docs/zh/operations/table_engines/replacingmergetree.md new file mode 100644 index 00000000000..7a4f3ab7443 --- /dev/null +++ b/docs/zh/operations/table_engines/replacingmergetree.md @@ -0,0 +1,58 @@ +# ReplacingMergeTree + +The engine differs from [MergeTree](mergetree.md#table_engines-mergetree) in that it removes duplicate entries with the same primary key value. + +Data deduplication occurs only during a merge. Merging occurs in the background at an unknown time, so you can't plan for it. Some of the data may remain unprocessed. Although you can run an unscheduled merge using the `OPTIMIZE` query, don't count on using it, because the `OPTIMIZE` query will read and write a large amount of data. + +Thus, `ReplacingMergeTree` is suitable for clearing out duplicate data in the background in order to save space, but it doesn't guarantee the absence of duplicates. + +## Creating a Table + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = ReplacingMergeTree([ver]) +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] +``` + +For a description of request parameters, see [request description](../../query_language/create.md#query_language-queries-create_table). + +**ReplacingMergeTree Parameters** + +- `ver` — column with version. Type `UInt*`, `Date` or `DateTime`. Optional parameter. + + When merging, `ReplacingMergeTree` from all the rows with the same primary key leaves only one: + - Last in the selection, if `ver` not set. + - With the maximum version, if `ver` specified. + +**Query clauses** + +When creating a `ReplacingMergeTree` table the same [clauses](mergetree.md#table_engines-mergetree-configuring) are required, as when creating a `MergeTree` table. + +
Deprecated Method for Creating a Table + +!!! attention + Do not use this method in new projects and, if possible, switch the old projects to the method described above. + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] ReplacingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, [ver]) +``` + +All of the parameters excepting `ver` have the same meaning as in `MergeTree`. + + +- `ver` - column with the version. Optional parameter. For a description, see the text above. +
+ +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/replacingmergetree/) diff --git a/docs/zh/operations/table_engines/replication.md b/docs/zh/operations/table_engines/replication.md deleted file mode 120000 index b4b22ac708b..00000000000 --- a/docs/zh/operations/table_engines/replication.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/replication.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/replication.md b/docs/zh/operations/table_engines/replication.md new file mode 100644 index 00000000000..01245edf744 --- /dev/null +++ b/docs/zh/operations/table_engines/replication.md @@ -0,0 +1,206 @@ + + +# Data Replication + +Replication is only supported for tables in the MergeTree family: + +- ReplicatedMergeTree +- ReplicatedSummingMergeTree +- ReplicatedReplacingMergeTree +- ReplicatedAggregatingMergeTree +- ReplicatedCollapsingMergeTree +- ReplicatedGraphiteMergeTree + +Replication works at the level of an individual table, not the entire server. A server can store both replicated and non-replicated tables at the same time. + +Replication does not depend on sharding. Each shard has its own independent replication. + +Compressed data for `INSERT` and `ALTER` queries is replicated (for more information, see the documentation for [ALTER](../../query_language/alter.md#query_language_queries_alter)). + +`CREATE`, `DROP`, `ATTACH`, `DETACH` and `RENAME` queries are executed on a single server and are not replicated: + +- `The CREATE TABLE` query creates a new replicatable table on the server where the query is run. If this table already exists on other servers, it adds a new replica. +- `The DROP TABLE` query deletes the replica located on the server where the query is run. +- `The RENAME` query renames the table on one of the replicas. In other words, replicated tables can have different names on different replicas. + +To use replication, set the addresses of the ZooKeeper cluster in the config file. Example: + +```xml + + + example1 + 2181 + + + example2 + 2181 + + + example3 + 2181 + + +``` + +Use ZooKeeper version 3.4.5 or later. + +You can specify any existing ZooKeeper cluster and the system will use a directory on it for its own data (the directory is specified when creating a replicatable table). + +If ZooKeeper isn't set in the config file, you can't create replicated tables, and any existing replicated tables will be read-only. + +ZooKeeper is not used in `SELECT` queries because replication does not affect the performance of `SELECT` and queries run just as fast as they do for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max_replica_delay_for_distributed_queries](../settings/settings.md#settings_settings_max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](../settings/settings.md#settings-settings-fallback_to_stale_replicas_for_distributed_queries). + +For each `INSERT` query, approximately ten entries are added to ZooKeeper through several transactions. (To be more precise, this is for each inserted block of data; an INSERT query contains one block or one block per `max_insert_block_size = 1048576` rows.) This leads to slightly longer latencies for `INSERT` compared to non-replicated tables. But if you follow the recommendations to insert data in batches of no more than one `INSERT` per second, it doesn't create any problems. The entire ClickHouse cluster used for coordinating one ZooKeeper cluster has a total of several hundred `INSERTs` per second. The throughput on data inserts (the number of rows per second) is just as high as for non-replicated data. + +For very large clusters, you can use different ZooKeeper clusters for different shards. However, this hasn't proven necessary on the Yandex.Metrica cluster (approximately 300 servers). + +Replication is asynchronous and multi-master. `INSERT` queries (as well as `ALTER`) can be sent to any available server. Data is inserted on the server where the query is run, and then it is copied to the other servers. Because it is asynchronous, recently inserted data appears on the other replicas with some latency. If part of the replicas are not available, the data is written when they become available. If a replica is available, the latency is the amount of time it takes to transfer the block of compressed data over the network. + +By default, an INSERT query waits for confirmation of writing the data from only one replica. If the data was successfully written to only one replica and the server with this replica ceases to exist, the stored data will be lost. Tp enable getting confirmation of data writes from multiple replicas, use the `insert_quorum` option. + +Each block of data is written atomically. The INSERT query is divided into blocks up to `max_insert_block_size = 1048576` rows. In other words, if the `INSERT` query has less than 1048576 rows, it is made atomically. + +Data blocks are deduplicated. For multiple writes of the same data block (data blocks of the same size containing the same rows in the same order), the block is only written once. The reason for this is in case of network failures when the client application doesn't know if the data was written to the DB, so the `INSERT` query can simply be repeated. It doesn't matter which replica INSERTs were sent to with identical data. `INSERTs` are idempotent. Deduplication parameters are controlled by [merge_tree](../server_settings/settings.md#server_settings-merge_tree) server settings. + +During replication, only the source data to insert is transferred over the network. Further data transformation (merging) is coordinated and performed on all the replicas in the same way. This minimizes network usage, which means that replication works well when replicas reside in different datacenters. (Note that duplicating data in different datacenters is the main goal of replication.) + +You can have any number of replicas of the same data. Yandex.Metrica uses double replication in production. Each server uses RAID-5 or RAID-6, and RAID-10 in some cases. This is a relatively reliable and convenient solution. + +The system monitors data synchronicity on replicas and is able to recover after a failure. Failover is automatic (for small differences in data) or semi-automatic (when data differs too much, which may indicate a configuration error). + + + +## Creating Replicated Tables + +The `Replicated` prefix is added to the table engine name. For example:`ReplicatedMergeTree`. + +**Replicated\*MergeTree parameters** + +- `zoo_path` — The path to the table in ZooKeeper. +- `replica_name` — The replica name in ZooKeeper. + +Example: + +```sql +CREATE TABLE table_name +( + EventDate DateTime, + CounterID UInt32, + UserID UInt32 +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/hits', '{replica}') +PARTITION BY toYYYYMM(EventDate) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +``` + +Example in deprecated syntax: + +```sql +CREATE TABLE table_name +( + EventDate DateTime, + CounterID UInt32, + UserID UInt32 +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{layer}-{shard}/hits', '{replica}', EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID), EventTime), 8192) +``` + +As the example shows, these parameters can contain substitutions in curly brackets. The substituted values are taken from the 'macros' section of the configuration file. Example: + +```xml + + 05 + 02 + example05-02-1.yandex.ru + +``` + +The path to the table in ZooKeeper should be unique for each replicated table. Tables on different shards should have different paths. +In this case, the path consists of the following parts: + +`/clickhouse/tables/` is the common prefix. We recommend using exactly this one. + +`{layer}-{shard}` is the shard identifier. In this example it consists of two parts, since the Yandex.Metrica cluster uses bi-level sharding. For most tasks, you can leave just the {shard} substitution, which will be expanded to the shard identifier. + +`hits` is the name of the node for the table in ZooKeeper. It is a good idea to make it the same as the table name. It is defined explicitly, because in contrast to the table name, it doesn't change after a RENAME query. + +The replica name identifies different replicas of the same table. You can use the server name for this, as in the example. The name only needs to be unique within each shard. + +You can define the parameters explicitly instead of using substitutions. This might be convenient for testing and for configuring small clusters. However, you can't use distributed DDL queries (`ON CLUSTER`) in this case. + +When working with large clusters, we recommend using substitutions because they reduce the probability of error. + +Run the `CREATE TABLE` query on each replica. This query creates a new replicated table, or adds a new replica to an existing one. + +If you add a new replica after the table already contains some data on other replicas, the data will be copied from the other replicas to the new one after running the query. In other words, the new replica syncs itself with the others. + +To delete a replica, run `DROP TABLE`. However, only one replica is deleted – the one that resides on the server where you run the query. + +## Recovery After Failures + +If ZooKeeper is unavailable when a server starts, replicated tables switch to read-only mode. The system periodically attempts to connect to ZooKeeper. + +If ZooKeeper is unavailable during an `INSERT`, or an error occurs when interacting with ZooKeeper, an exception is thrown. + +After connecting to ZooKeeper, the system checks whether the set of data in the local file system matches the expected set of data (ZooKeeper stores this information). If there are minor inconsistencies, the system resolves them by syncing data with the replicas. + +If the system detects broken data parts (with the wrong size of files) or unrecognized parts (parts written to the file system but not recorded in ZooKeeper), it moves them to the 'detached' subdirectory (they are not deleted). Any missing parts are copied from the replicas. + +Note that ClickHouse does not perform any destructive actions such as automatically deleting a large amount of data. + +When the server starts (or establishes a new session with ZooKeeper), it only checks the quantity and sizes of all files. If the file sizes match but bytes have been changed somewhere in the middle, this is not detected immediately, but only when attempting to read the data for a `SELECT` query. The query throws an exception about a non-matching checksum or size of a compressed block. In this case, data parts are added to the verification queue and copied from the replicas if necessary. + +If the local set of data differs too much from the expected one, a safety mechanism is triggered. The server enters this in the log and refuses to launch. The reason for this is that this case may indicate a configuration error, such as if a replica on a shard was accidentally configured like a replica on a different shard. However, the thresholds for this mechanism are set fairly low, and this situation might occur during normal failure recovery. In this case, data is restored semi-automatically - by "pushing a button". + +To start recovery, create the node `/path_to_table/replica_name/flags/force_restore_data` in ZooKeeper with any content, or run the command to restore all replicated tables: + +```bash +sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data +``` + +Then restart the server. On start, the server deletes these flags and starts recovery. + +## Recovery After Complete Data Loss + +If all data and metadata disappeared from one of the servers, follow these steps for recovery: + +1. Install ClickHouse on the server. Define substitutions correctly in the config file that contains the shard identifier and replicas, if you use them. +2. If you had unreplicated tables that must be manually duplicated on the servers, copy their data from a replica (in the directory `/var/lib/clickhouse/data/db_name/table_name/`). +3. Copy table definitions located in `/var/lib/clickhouse/metadata/` from a replica. If a shard or replica identifier is defined explicitly in the table definitions, correct it so that it corresponds to this replica. (Alternatively, start the server and make all the `ATTACH TABLE` queries that should have been in the .sql files in `/var/lib/clickhouse/metadata/`.) +4. To start recovery, create the ZooKeeper node `/path_to_table/replica_name/flags/force_restore_data` with any content, or run the command to restore all replicated tables: `sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data` + +Then start the server (restart, if it is already running). Data will be downloaded from replicas. + +An alternative recovery option is to delete information about the lost replica from ZooKeeper (`/path_to_table/replica_name`), then create the replica again as described in "[Creating replicatable tables](#table_engines-replication-creation_of_rep_tables)". + +There is no restriction on network bandwidth during recovery. Keep this in mind if you are restoring many replicas at once. + + + +## Converting from MergeTree to ReplicatedMergeTree + +We use the term `MergeTree` to refer to all table engines in the ` MergeTree family`, the same as for `ReplicatedMergeTree`. + +If you had a `MergeTree` table that was manually replicated, you can convert it to a replicatable table. You might need to do this if you have already collected a large amount of data in a `MergeTree` table and now you want to enable replication. + +If the data differs on various replicas, first sync it, or delete this data on all the replicas except one. + +Rename the existing MergeTree table, then create a `ReplicatedMergeTree` table with the old name. +Move the data from the old table to the 'detached' subdirectory inside the directory with the new table data (`/var/lib/clickhouse/data/db_name/table_name/`). +Then run `ALTER TABLE ATTACH PARTITION` on one of the replicas to add these data parts to the working set. + +## Converting from ReplicatedMergeTree to MergeTree + +Create a MergeTree table with a different name. Move all the data from the directory with the `ReplicatedMergeTree` table data to the new table's data directory. Then delete the `ReplicatedMergeTree` table and restart the server. + +If you want to get rid of a `ReplicatedMergeTree` table without launching the server: + +- Delete the corresponding `.sql` file in the metadata directory (`/var/lib/clickhouse/metadata/`). +- Delete the corresponding path in ZooKeeper (`/path_to_table/replica_name`). + +After this, you can launch the server, create a `MergeTree` table, move the data to its directory, and then restart the server. + +## Recovery When Metadata in The ZooKeeper Cluster is Lost or Damaged + +If the data in ZooKeeper was lost or damaged, you can save data by moving it to an unreplicated table as described above. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/replication/) diff --git a/docs/zh/operations/table_engines/set.md b/docs/zh/operations/table_engines/set.md deleted file mode 120000 index d37e659badd..00000000000 --- a/docs/zh/operations/table_engines/set.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/set.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/set.md b/docs/zh/operations/table_engines/set.md new file mode 100644 index 00000000000..75c1f3072d4 --- /dev/null +++ b/docs/zh/operations/table_engines/set.md @@ -0,0 +1,13 @@ +# Set + +A data set that is always in RAM. It is intended for use on the right side of the IN operator (see the section "IN operators"). + +You can use INSERT to insert data in the table. New elements will be added to the data set, while duplicates will be ignored. +But you can't perform SELECT from the table. The only way to retrieve data is by using it in the right half of the IN operator. + +Data is always located in RAM. For INSERT, the blocks of inserted data are also written to the directory of tables on the disk. When starting the server, this data is loaded to RAM. In other words, after restarting, the data remains in place. + +For a rough server restart, the block of data on the disk might be lost or damaged. In the latter case, you may need to manually delete the file with damaged data. + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/set/) diff --git a/docs/zh/operations/table_engines/summingmergetree.md b/docs/zh/operations/table_engines/summingmergetree.md deleted file mode 120000 index 2b67e953d8a..00000000000 --- a/docs/zh/operations/table_engines/summingmergetree.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/summingmergetree.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/summingmergetree.md b/docs/zh/operations/table_engines/summingmergetree.md new file mode 100644 index 00000000000..a2d89bfa4c8 --- /dev/null +++ b/docs/zh/operations/table_engines/summingmergetree.md @@ -0,0 +1,135 @@ + + +# SummingMergeTree + +The engine inherits from [MergeTree](mergetree.md#table_engines-mergetree). The difference is that when merging data parts for `SummingMergeTree` tables ClickHouse replaces all the rows with the same primary key with one row which contains summarized values for the columns with the numeric data type. If the primary key is composed in a way that a single key value corresponds to large number of rows, this significantly reduces storage volume and speeds up data selection. + +We recommend to use the engine together with `MergeTree`. Store complete data in `MergeTree` table, and use `SummingMergeTree` for aggregated data storing, for example, when preparing reports. Such an approach will prevent you from losing valuable data due to an incorrectly composed primary key. + +## Creating a Table + +``` +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = MergeTree() +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] +``` + +For a description of request parameters, see [request description](../../query_language/create.md#query_language-queries-create_table). + +**Parameters of SummingMergeTree** + +- `columns` - a tuple with the names of columns where values will be summarized. Optional parameter. +The columns must be of a numeric type and must not be in the primary key. + + If `columns` not specified, ClickHouse summarizes the values in all columns with a numeric data type that are not in the primary key. + +**Query clauses** + +When creating a `SummingMergeTree` table the same [clauses](mergetree.md#table_engines-mergetree-configuring) are required, as when creating a `MergeTree` table. + +
Deprecated Method for Creating a Table + +!!! attention + Do not use this method in new projects and, if possible, switch the old projects to the method described above. + +``` +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] SummingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, [columns]) +``` + +All of the parameters excepting `columns` have the same meaning as in `MergeTree`. + +- `columns` — tuple with names of columns values of which will be summarized. Optional parameter. For a description, see the text above. +
+ +## Usage Example + +Consider the following table: + +```sql +CREATE TABLE summtt +( + key UInt32, + value UInt32 +) +ENGINE = SummingMergeTree() +ORDER BY key +``` + +Insert data to it: + +``` +:) INSERT INTO summtt Values(1,1),(1,2),(2,1) +``` + +ClickHouse may sum all the rows not completely ([see below](#summary-data-processing)), so we use an aggregate function `sum` and `GROUP BY` clause in the query. + +```sql +SELECT key, sum(value) FROM summtt GROUP BY key +``` + +``` +┌─key─┬─sum(value)─┐ +│ 2 │ 1 │ +│ 1 │ 3 │ +└─────┴────────────┘ +``` + + + +## Data Processing + +When data are inserted into a table, they are saved as-is. Clickhouse merges the inserted parts of data periodically and this is when rows with the same primary key are summed and replaced with one for each resulting part of data. + +ClickHouse can merge the data parts so that different resulting parts of data cat consist rows with the same primary key, i.e. the summation will be incomplete. Therefore (`SELECT`) an aggregate function [sum()](../../query_language/agg_functions/reference.md#agg_function-sum) and `GROUP BY` clause should be used in a query as described in the example above. + +### Common rules for summation + +The values in the columns with the numeric data type are summarized. The set of columns is defined by the parameter `columns`. + +If the values were 0 in all of the columns for summation, the row is deleted. + +If column is not in the primary key and is not summarized, an arbitrary value is selected from the existing ones. + +The values are not summarized for columns in the primary key. + +### The Summation in the AggregateFunction Columns + +For columns of [AggregateFunction type](../../data_types/nested_data_structures/aggregatefunction.md#data_type-aggregatefunction) ClickHouse behaves as [AggregatingMergeTree](aggregatingmergetree.md#table_engine-aggregatingmergetree) engine aggregating according to the function. + +### Nested Structures + +Table can have nested data structures that are processed in a special way. + +If the name of a nested table ends with `Map` and it contains at least two columns that meet the following criteria: + +- the first column is numeric `(*Int*, Date, DateTime)`, let's call it `key`, +- the other columns are arithmetic `(*Int*, Float32/64)`, let's call it `(values...)`, + +then this nested table is interpreted as a mapping of `key => (values...)`, and when merging its rows, the elements of two data sets are merged by `key` with a summation of the corresponding `(values...)`. + +Examples: + +``` +[(1, 100)] + [(2, 150)] -> [(1, 100), (2, 150)] +[(1, 100)] + [(1, 150)] -> [(1, 250)] +[(1, 100)] + [(1, 150), (2, 150)] -> [(1, 250), (2, 150)] +[(1, 100), (2, 150)] + [(1, -100)] -> [(2, 150)] +``` + +When requesting data, use the [sumMap(key, value)](../../query_language/agg_functions/reference.md#agg_function-summary) function for aggregation of `Map`. + +For nested data structure, you do not need to specify its columns in the tuple of columns for summation. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/summingmergetree/) diff --git a/docs/zh/operations/table_engines/tinylog.md b/docs/zh/operations/table_engines/tinylog.md deleted file mode 120000 index bda90c7d5ce..00000000000 --- a/docs/zh/operations/table_engines/tinylog.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/tinylog.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/tinylog.md b/docs/zh/operations/table_engines/tinylog.md new file mode 100644 index 00000000000..6ec1cb8173a --- /dev/null +++ b/docs/zh/operations/table_engines/tinylog.md @@ -0,0 +1,21 @@ +# TinyLog + +The simplest table engine, which stores data on a disk. +Each column is stored in a separate compressed file. +When writing, data is appended to the end of files. + +Concurrent data access is not restricted in any way: + +- If you are simultaneously reading from a table and writing to it in a different query, the read operation will complete with an error. +- If you are writing to a table in multiple queries simultaneously, the data will be broken. + +The typical way to use this table is write-once: first just write the data one time, then read it as many times as needed. +Queries are executed in a single stream. In other words, this engine is intended for relatively small tables (recommended up to 1,000,000 rows). +It makes sense to use this table engine if you have many small tables, since it is simpler than the Log engine (fewer files need to be opened). +The situation when you have a large number of small tables guarantees poor productivity, but may already be used when working with another DBMS, and you may find it easier to switch to using TinyLog types of tables. +**Indexes are not supported.** + +In Yandex.Metrica, TinyLog tables are used for intermediary data that is processed in small batches. + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/tinylog/) diff --git a/docs/zh/operations/table_engines/url.md b/docs/zh/operations/table_engines/url.md deleted file mode 120000 index d0de71dcf40..00000000000 --- a/docs/zh/operations/table_engines/url.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/url.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/url.md b/docs/zh/operations/table_engines/url.md new file mode 100644 index 00000000000..54d742e409a --- /dev/null +++ b/docs/zh/operations/table_engines/url.md @@ -0,0 +1,75 @@ + + +# URL(URL, Format) + +Manages data on a remote HTTP/HTTPS server. This engine is similar +to the [`File`](./file.md#) engine. + +## Using the engine in the ClickHouse server + +`The format` must be one that ClickHouse can use in +`SELECT` queries and, if necessary, in `INSERTs`. For the full list of supported formats, see +[Formats](../../interfaces/formats.md#formats). + +`The URL` must conform to the structure of a Uniform Resource Locator. The specified URL must point to a server +that uses HTTP or HTTPS. This does not require any +additional headers for getting a response from the server. + +`INSERT` and `SELECT` queries are transformed to `POST` and `GET` requests, +respectively. For processing `POST` requests, the remote server must support +[Chunked transfer encoding](https://en.wikipedia.org/wiki/Chunked_transfer_encoding). + +**Example:** + +**1.** Create a `url_engine_table` table on the server : + +``` sql +CREATE TABLE url_engine_table (word String, value UInt64) +ENGINE=URL('http://127.0.0.1:12345/', CSV) +``` + +**2.** Create a basic HTTP server using the standard Python 3 tools and +start it: + +```python3 +from http.server import BaseHTTPRequestHandler, HTTPServer + +class CSVHTTPServer(BaseHTTPRequestHandler): + def do_GET(self): + self.send_response(200) + self.send_header('Content-type', 'text/csv') + self.end_headers() + + self.wfile.write(bytes('Hello,1\nWorld,2\n', "utf-8")) + +if __name__ == "__main__": + server_address = ('127.0.0.1', 12345) + HTTPServer(server_address, CSVHTTPServer).serve_forever() +``` + +```bash +python3 server.py +``` + +**3.** Request data: + +``` sql +SELECT * FROM url_engine_table +``` + +``` +┌─word──┬─value─┐ +│ Hello │ 1 │ +│ World │ 2 │ +└───────┴───────┘ +``` + +## Details of Implementation + +- Reads and writes can be parallel +- Not supported: + - `ALTER` and `SELECT...SAMPLE` operations. + - Indexes. + - Replication. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/url/) diff --git a/docs/zh/operations/table_engines/view.md b/docs/zh/operations/table_engines/view.md deleted file mode 120000 index 3f2164181a7..00000000000 --- a/docs/zh/operations/table_engines/view.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/table_engines/view.md \ No newline at end of file diff --git a/docs/zh/operations/table_engines/view.md b/docs/zh/operations/table_engines/view.md new file mode 100644 index 00000000000..c74eab262e4 --- /dev/null +++ b/docs/zh/operations/table_engines/view.md @@ -0,0 +1,6 @@ +# View + +Used for implementing views (for more information, see the `CREATE VIEW query`). It does not store data, but only stores the specified `SELECT` query. When reading from a table, it runs this query (and deletes all unnecessary columns from the query). + + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/view/) diff --git a/docs/zh/operations/tips.md b/docs/zh/operations/tips.md deleted file mode 120000 index 9b3413bdbc3..00000000000 --- a/docs/zh/operations/tips.md +++ /dev/null @@ -1 +0,0 @@ -../../en/operations/tips.md \ No newline at end of file diff --git a/docs/zh/operations/tips.md b/docs/zh/operations/tips.md new file mode 100644 index 00000000000..a34f79f869f --- /dev/null +++ b/docs/zh/operations/tips.md @@ -0,0 +1,259 @@ +# Usage Recommendations + +## CPU + +The SSE 4.2 instruction set must be supported. Modern processors (since 2008) support it. + +When choosing a processor, prefer a large number of cores and slightly slower clock rate over fewer cores and a higher clock rate. +For example, 16 cores with 2600 MHz is better than 8 cores with 3600 MHz. + +## Hyper-threading + +Don't disable hyper-threading. It helps for some queries, but not for others. + +## Turbo Boost + +Turbo Boost is highly recommended. It significantly improves performance with a typical load. +You can use `turbostat` to view the CPU's actual clock rate under a load. + +## CPU Scaling Governor + +Always use the `performance` scaling governor. The `on-demand` scaling governor works much worse with constantly high demand. + +```bash +sudo echo 'performance' | tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor +``` + +## CPU Limitations + +Processors can overheat. Use `dmesg` to see if the CPU's clock rate was limited due to overheating. +The restriction can also be set externally at the datacenter level. You can use `turbostat` to monitor it under a load. + +## RAM + +For small amounts of data (up to \~200 GB compressed), it is best to use as much memory as the volume of data. +For large amounts of data and when processing interactive (online) queries, you should use a reasonable amount of RAM (128 GB or more) so the hot data subset will fit in the cache of pages. +Even for data volumes of \~50 TB per server, using 128 GB of RAM significantly improves query performance compared to 64 GB. + +## Swap File + +Always disable the swap file. The only reason for not doing this is if you are using ClickHouse on your personal laptop. + +## Huge Pages + +Always disable transparent huge pages. It interferes with memory allocators, which leads to significant performance degradation. + +```bash +echo 'never' | sudo tee /sys/kernel/mm/transparent_hugepage/enabled +``` + +Use `perf top` to watch the time spent in the kernel for memory management. +Permanent huge pages also do not need to be allocated. + +## Storage Subsystem + +If your budget allows you to use SSD, use SSD. +If not, use HDD. SATA HDDs 7200 RPM will do. + +Give preference to a lot of servers with local hard drives over a smaller number of servers with attached disk shelves. +But for storing archives with rare queries, shelves will work. + +## RAID + +When using HDD, you can combine their RAID-10, RAID-5, RAID-6 or RAID-50. +For Linux, software RAID is better (with `mdadm`). We don't recommend using LVM. +When creating RAID-10, select the `far` layout. +If your budget allows, choose RAID-10. + +If you have more than 4 disks, use RAID-6 (preferred) or RAID-50, instead of RAID-5. +When using RAID-5, RAID-6 or RAID-50, always increase stripe_cache_size, since the default value is usually not the best choice. + +```bash +echo 4096 | sudo tee /sys/block/md2/md/stripe_cache_size +``` + +Calculate the exact number from the number of devices and the block size, using the formula: `2 * num_devices * chunk_size_in_bytes / 4096`. + +A block size of 1025 KB is sufficient for all RAID configurations. +Never set the block size too small or too large. + +You can use RAID-0 on SSD. +Regardless of RAID use, always use replication for data security. + +Enable NCQ with a long queue. For HDD, choose the CFQ scheduler, and for SSD, choose noop. Don't reduce the 'readahead' setting. +For HDD, enable the write cache. + +## File System + +Ext4 is the most reliable option. Set the mount options `noatime, nobarrier`. +XFS is also suitable, but it hasn't been as thoroughly tested with ClickHouse. +Most other file systems should also work fine. File systems with delayed allocation work better. + +## Linux Kernel + +Don't use an outdated Linux kernel. + +## Network + +If you are using IPv6, increase the size of the route cache. +The Linux kernel prior to 3.2 had a multitude of problems with IPv6 implementation. + +Use at least a 10 GB network, if possible. 1 Gb will also work, but it will be much worse for patching replicas with tens of terabytes of data, or for processing distributed queries with a large amount of intermediate data. + +## ZooKeeper + +You are probably already using ZooKeeper for other purposes. You can use the same installation of ZooKeeper, if it isn't already overloaded. + +It's best to use a fresh version of ZooKeeper – 3.4.9 or later. The version in stable Linux distributions may be outdated. + +You should never use manually written scripts to transfer data between different ZooKeeper clusters, because the result will be incorrect for sequential nodes. Never use the "zkcopy" utility for the same reason: https://github.com/ksprojects/zkcopy/issues/15 + +If you want to divide an existing ZooKeeper cluster into two, the correct way is to increase the number of its replicas and then reconfigure it as two independent clusters. + +Do not run ZooKeeper on the same servers as ClickHouse. Because ZooKeeper is very sensitive for latency and ClickHouse may utilize all available system resources. + +With the default settings, ZooKeeper is a time bomb: + +> The ZooKeeper server won't delete files from old snapshots and logs when using the default configuration (see autopurge), and this is the responsibility of the operator. + +This bomb must be defused. + +The ZooKeeper (3.5.1) configuration below is used in the Yandex.Metrica production environment as of May 20, 2017: + +zoo.cfg: + +```bash +# http://hadoop.apache.org/zookeeper/docs/current/zookeeperAdmin.html + +# The number of milliseconds of each tick +tickTime=2000 +# The number of ticks that the initial +# synchronization phase can take +initLimit=30000 +# The number of ticks that can pass between +# sending a request and getting an acknowledgement +syncLimit=10 + +maxClientCnxns=2000 + +maxSessionTimeout=60000000 +# the directory where the snapshot is stored. +dataDir=/opt/zookeeper/{{ cluster['name'] }}/data +# Place the dataLogDir to a separate physical disc for better performance +dataLogDir=/opt/zookeeper/{{ cluster['name'] }}/logs + +autopurge.snapRetainCount=10 +autopurge.purgeInterval=1 + + +# To avoid seeks ZooKeeper allocates space in the transaction log file in +# blocks of preAllocSize kilobytes. The default block size is 64M. One reason +# for changing the size of the blocks is to reduce the block size if snapshots +# are taken more often. (Also, see snapCount). +preAllocSize=131072 + +# Clients can submit requests faster than ZooKeeper can process them, +# especially if there are a lot of clients. To prevent ZooKeeper from running +# out of memory due to queued requests, ZooKeeper will throttle clients so that +# there is no more than globalOutstandingLimit outstanding requests in the +# system. The default limit is 1,000.ZooKeeper logs transactions to a +# transaction log. After snapCount transactions are written to a log file a +# snapshot is started and a new transaction log file is started. The default +# snapCount is 10,000. +snapCount=3000000 + +# If this option is defined, requests will be will logged to a trace file named +# traceFile.year.month.day. +#traceFile= + +# Leader accepts client connections. Default value is "yes". The leader machine +# coordinates updates. For higher update throughput at thes slight expense of +# read throughput the leader can be configured to not accept clients and focus +# on coordination. +leaderServes=yes + +standaloneEnabled=false +dynamicConfigFile=/etc/zookeeper-{{ cluster['name'] }}/conf/zoo.cfg.dynamic +``` + +Java version: + +``` +Java(TM) SE Runtime Environment (build 1.8.0_25-b17) +Java HotSpot(TM) 64-Bit Server VM (build 25.25-b02, mixed mode) +``` + +JVM parameters: + +```bash +NAME=zookeeper-{{ cluster['name'] }} +ZOOCFGDIR=/etc/$NAME/conf + +# TODO this is really ugly +# How to find out, which jars are needed? +# seems, that log4j requires the log4j.properties file to be in the classpath +CLASSPATH="$ZOOCFGDIR:/usr/build/classes:/usr/build/lib/*.jar:/usr/share/zookeeper/zookeeper-3.5.1-metrika.jar:/usr/share/zookeeper/slf4j-log4j12-1.7.5.jar:/usr/share/zookeeper/slf4j-api-1.7.5.jar:/usr/share/zookeeper/servlet-api-2.5-20081211.jar:/usr/share/zookeeper/netty-3.7.0.Final.jar:/usr/share/zookeeper/log4j-1.2.16.jar:/usr/share/zookeeper/jline-2.11.jar:/usr/share/zookeeper/jetty-util-6.1.26.jar:/usr/share/zookeeper/jetty-6.1.26.jar:/usr/share/zookeeper/javacc.jar:/usr/share/zookeeper/jackson-mapper-asl-1.9.11.jar:/usr/share/zookeeper/jackson-core-asl-1.9.11.jar:/usr/share/zookeeper/commons-cli-1.2.jar:/usr/src/java/lib/*.jar:/usr/etc/zookeeper" + +ZOOCFG="$ZOOCFGDIR/zoo.cfg" +ZOO_LOG_DIR=/var/log/$NAME +USER=zookeeper +GROUP=zookeeper +PIDDIR=/var/run/$NAME +PIDFILE=$PIDDIR/$NAME.pid +SCRIPTNAME=/etc/init.d/$NAME +JAVA=/usr/bin/java +ZOOMAIN="org.apache.zookeeper.server.quorum.QuorumPeerMain" +ZOO_LOG4J_PROP="INFO,ROLLINGFILE" +JMXLOCALONLY=false +JAVA_OPTS="-Xms{{ cluster.get('xms','128M') }} \ + -Xmx{{ cluster.get('xmx','1G') }} \ + -Xloggc:/var/log/$NAME/zookeeper-gc.log \ + -XX:+UseGCLogFileRotation \ + -XX:NumberOfGCLogFiles=16 \ + -XX:GCLogFileSize=16M \ + -verbose:gc \ + -XX:+PrintGCTimeStamps \ + -XX:+PrintGCDateStamps \ + -XX:+PrintGCDetails + -XX:+PrintTenuringDistribution \ + -XX:+PrintGCApplicationStoppedTime \ + -XX:+PrintGCApplicationConcurrentTime \ + -XX:+PrintSafepointStatistics \ + -XX:+UseParNewGC \ + -XX:+UseConcMarkSweepGC \ +-XX:+CMSParallelRemarkEnabled" +``` + +Salt init: + +``` +description "zookeeper-{{ cluster['name'] }} centralized coordination service" + +start on runlevel [2345] +stop on runlevel [!2345] + +respawn + +limit nofile 8192 8192 + +pre-start script + [ -r "/etc/zookeeper-{{ cluster['name'] }}/conf/environment" ] || exit 0 + . /etc/zookeeper-{{ cluster['name'] }}/conf/environment + [ -d $ZOO_LOG_DIR ] || mkdir -p $ZOO_LOG_DIR + chown $USER:$GROUP $ZOO_LOG_DIR +end script + +script + . /etc/zookeeper-{{ cluster['name'] }}/conf/environment + [ -r /etc/default/zookeeper ] && . /etc/default/zookeeper + if [ -z "$JMXDISABLE" ]; then + JAVA_OPTS="$JAVA_OPTS -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.local.only=$JMXLOCALONLY" + fi + exec start-stop-daemon --start -c $USER --exec $JAVA --name zookeeper-{{ cluster['name'] }} \ + -- -cp $CLASSPATH $JAVA_OPTS -Dzookeeper.log.dir=${ZOO_LOG_DIR} \ + -Dzookeeper.root.logger=${ZOO_LOG4J_PROP} $ZOOMAIN $ZOOCFG +end script +``` + + +[Original article](https://clickhouse.yandex/docs/en/operations/tips/) diff --git a/docs/zh/operations/utils/clickhouse-copier.md b/docs/zh/operations/utils/clickhouse-copier.md deleted file mode 120000 index c9e89e33c7b..00000000000 --- a/docs/zh/operations/utils/clickhouse-copier.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/utils/clickhouse-copier.md \ No newline at end of file diff --git a/docs/zh/operations/utils/clickhouse-copier.md b/docs/zh/operations/utils/clickhouse-copier.md new file mode 100644 index 00000000000..361834a681d --- /dev/null +++ b/docs/zh/operations/utils/clickhouse-copier.md @@ -0,0 +1,163 @@ + + +# clickhouse-copier + +Copies data from the tables in one cluster to tables in another (or the same) cluster. + +You can run multiple `clickhouse-copier` instances on different servers to perform the same job. ZooKeeper is used for syncing the processes. + +After starting, `clickhouse-copier`: + +- Connects to ZooKeeper and receives: + - Copying jobs. + - The state of the copying jobs. + +- It performs the jobs. + + Each running process chooses the "closest" shard of the source cluster and copies the data into the destination cluster, resharding the data if necessary. + +`clickhouse-copier` tracks the changes in ZooKeeper and applies them on the fly. + +To reduce network traffic, we recommend running `clickhouse-copier` on the same server where the source data is located. + +## Running clickhouse-copier + +The utility should be run manually: + +```bash +clickhouse-copier copier --daemon --config zookeeper.xml --task-path /task/path --base-dir /path/to/dir +``` + +Parameters: + +- `daemon` — Starts `clickhouse-copier` in daemon mode. +- `config` — The path to the `zookeeper.xml` file with the parameters for the connection to ZooKeeper. +- `task-path` — The path to the ZooKeeper node. This node is used for syncing `clickhouse-copier` processes and storing tasks. Tasks are stored in `$task-path/description`. +- `base-dir` — The path to logs and auxiliary files. When it starts, `clickhouse-copier` creates `clickhouse-copier_YYYYMMHHSS_` subdirectories in `$base-dir`. If this parameter is omitted, the directories are created in the directory where `clickhouse-copier` was launched. + +## Format of zookeeper.xml + +```xml + + + + 127.0.0.1 + 2181 + + + +``` + +## Configuration of copying tasks + +```xml + + + + + + false + + 127.0.0.1 + 9000 + + + ... + + + + ... + + + + + 2 + + + + 1 + + + + + 0 + + + + + 3 + + 1 + + + + + + + + source_cluster + test + hits + + + destination_cluster + test + hits2 + + + + ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/hits2', '{replica}') + PARTITION BY toMonday(date) + ORDER BY (CounterID, EventDate) + + + + jumpConsistentHash(intHash64(UserID), 2) + + + CounterID != 0 + + + + '2018-02-26' + '2018-03-05' + ... + + + + + + ... + + ... + + +``` + +`clickhouse-copier` tracks the changes in `/task/path/description` and applies them on the fly. For instance, if you change the value of `max_workers`, the number of processes running tasks will also change. + + +[Original article](https://clickhouse.yandex/docs/en/operations/utils/clickhouse-copier/) diff --git a/docs/zh/operations/utils/clickhouse-local.md b/docs/zh/operations/utils/clickhouse-local.md deleted file mode 120000 index 032aaaa2b84..00000000000 --- a/docs/zh/operations/utils/clickhouse-local.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/utils/clickhouse-local.md \ No newline at end of file diff --git a/docs/zh/operations/utils/clickhouse-local.md b/docs/zh/operations/utils/clickhouse-local.md new file mode 100644 index 00000000000..4b20473cb42 --- /dev/null +++ b/docs/zh/operations/utils/clickhouse-local.md @@ -0,0 +1,75 @@ + + +# clickhouse-local + +The `clickhouse-local` program enables you to perform fast processing on local files, without having to deploy and configure the ClickHouse server. + +Accepts data that represent tables and queries them using [ClickHouse SQL dialect](../../query_language/index.md#queries). + +`clickhouse-local` uses the same core as ClickHouse server, so it supports most of the features and the same set of formats and table engines. + +By default `clickhouse-local` does not have access to data on the same host, but it supports loading server configuration using `--config-file` argument. + +!!! warning + It is not recommended to load production server configuration into `clickhouse-local` because data can be damaged in case of human error. + + +## Usage + +Basic usage: + +``` bash +clickhouse-local --structure "table_structure" --input-format "format_of_incoming_data" -q "query" +``` + +Arguments: + +- `-S`, `--structure` — table structure for input data. +- `-if`, `--input-format` — input format, `TSV` by default. +- `-f`, `--file` — path to data, `stdin` by default. +- `-q` `--query` — queries to execute with `;` as delimeter. +- `-N`, `--table` — table name where to put output data, `table` by default. +- `-of`, `--format`, `--output-format` — output format, `TSV` by default. +- `--stacktrace` — whether to dump debug output in case of exception. +- `--verbose` — more details on query execution. +- `-s` — disables `stderr` logging. +- `--config-file` — path to configuration file in same format as for ClickHouse server, by default the configuration empty. +- `--help` — arguments references for `clickhouse-local`. + +Also there are arguments for each ClickHouse configuration variable which are more commonly used instead of `--config-file`. + + +## Examples + +``` bash +echo -e "1,2\n3,4" | clickhouse-local -S "a Int64, b Int64" -if "CSV" -q "SELECT * FROM table" +Read 2 rows, 32.00 B in 0.000 sec., 5182 rows/sec., 80.97 KiB/sec. +1 2 +3 4 +``` + +Previous example is the same as: + +``` bash +$ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64) ENGINE = File(CSV, stdin); SELECT a, b FROM table; DROP TABLE table" +Read 2 rows, 32.00 B in 0.000 sec., 4987 rows/sec., 77.93 KiB/sec. +1 2 +3 4 +``` + +Now let's output memory user for each Unix user: + +``` bash +$ ps aux | tail -n +2 | awk '{ printf("%s\t%s\n", $1, $4) }' | clickhouse-local -S "user String, mem Float64" -q "SELECT user, round(sum(mem), 2) as memTotal FROM table GROUP BY user ORDER BY memTotal DESC FORMAT Pretty" +Read 186 rows, 4.15 KiB in 0.035 sec., 5302 rows/sec., 118.34 KiB/sec. +┏━━━━━━━━━━┳━━━━━━━━━━┓ +┃ user ┃ memTotal ┃ +┡━━━━━━━━━━╇━━━━━━━━━━┩ +│ bayonet │ 113.5 │ +├──────────┼──────────┤ +│ root │ 8.8 │ +├──────────┼──────────┤ +... +``` + +[Original article](https://clickhouse.yandex/docs/en/operations/utils/clickhouse-local/) diff --git a/docs/zh/operations/utils/index.md b/docs/zh/operations/utils/index.md deleted file mode 120000 index dd089d1ef4b..00000000000 --- a/docs/zh/operations/utils/index.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/utils/index.md \ No newline at end of file diff --git a/docs/zh/operations/utils/index.md b/docs/zh/operations/utils/index.md new file mode 100644 index 00000000000..6406b486cc8 --- /dev/null +++ b/docs/zh/operations/utils/index.md @@ -0,0 +1,7 @@ +# ClickHouse Utility + +* [clickhouse-local](clickhouse-local.md#utils-clickhouse-local) — Allows running SQL queries on data without stopping the ClickHouse server, similar to how `awk` does this. +* [clickhouse-copier](clickhouse-copier.md#utils-clickhouse-copier) — Copies (and reshards) data from one cluster to another cluster. + + +[Original article](https://clickhouse.yandex/docs/en/operations/utils/) From ca1d38914ab622f9b46c7d0417958ecc5d846d3a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 30 Nov 2018 22:37:31 +0300 Subject: [PATCH 48/69] Don't log to stderr within RE2 library [#CLICKHOUSE-2] --- dbms/src/Common/ErrorCodes.cpp | 1 + ...n.inl.h => OptimizedRegularExpression.cpp} | 34 ++++++++++++------- dbms/src/Common/OptimizedRegularExpression.h | 8 ++--- 3 files changed, 25 insertions(+), 18 deletions(-) rename dbms/src/Common/{OptimizedRegularExpression.inl.h => OptimizedRegularExpression.cpp} (93%) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 719e593e3b1..669e7f9c36b 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -401,6 +401,7 @@ namespace ErrorCodes extern const int CANNOT_LINK = 424; extern const int SYSTEM_ERROR = 425; extern const int NULL_POINTER_DEREFERENCE = 426; + extern const int CANNOT_COMPILE_REGEXP = 427; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Common/OptimizedRegularExpression.inl.h b/dbms/src/Common/OptimizedRegularExpression.cpp similarity index 93% rename from dbms/src/Common/OptimizedRegularExpression.inl.h rename to dbms/src/Common/OptimizedRegularExpression.cpp index 04860622fe7..16251ae1ee0 100644 --- a/dbms/src/Common/OptimizedRegularExpression.inl.h +++ b/dbms/src/Common/OptimizedRegularExpression.cpp @@ -1,7 +1,4 @@ -#include - -#include - +#include #include @@ -9,6 +6,15 @@ #define MAX_SUBPATTERNS 5 +namespace DB +{ + namespace ErrorCodes + { + extern const int CANNOT_COMPILE_REGEXP; + } +} + + template void OptimizedRegularExpressionImpl::analyze( const std::string & regexp, @@ -254,11 +260,11 @@ OptimizedRegularExpressionImpl::OptimizedRegularExpressionImpl(cons /// Just three following options are supported if (options & (~(RE_CASELESS | RE_NO_CAPTURE | RE_DOT_NL))) - throw Poco::Exception("OptimizedRegularExpression: Unsupported option."); + throw DB::Exception("OptimizedRegularExpression: Unsupported option.", DB::ErrorCodes::CANNOT_COMPILE_REGEXP); - is_case_insensitive = options & RE_CASELESS; - bool is_no_capture = options & RE_NO_CAPTURE; - bool is_dot_nl = options & RE_DOT_NL; + is_case_insensitive = options & RE_CASELESS; + bool is_no_capture = options & RE_NO_CAPTURE; + bool is_dot_nl = options & RE_DOT_NL; number_of_subpatterns = 0; if (!is_trivial) @@ -266,6 +272,9 @@ OptimizedRegularExpressionImpl::OptimizedRegularExpressionImpl(cons /// Compile the re2 regular expression. typename RegexType::Options regexp_options; + /// Never write error messages to stderr. It's ignorant to do it from library code. + regexp_options.set_log_errors(false); + if (is_case_insensitive) regexp_options.set_case_sensitive(false); @@ -274,13 +283,13 @@ OptimizedRegularExpressionImpl::OptimizedRegularExpressionImpl(cons re2 = std::make_unique(regexp_, regexp_options); if (!re2->ok()) - throw Poco::Exception("OptimizedRegularExpression: cannot compile re2: " + regexp_ + ", error: " + re2->error()); + throw DB::Exception("OptimizedRegularExpression: cannot compile re2: " + regexp_ + ", error: " + re2->error() + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", DB::ErrorCodes::CANNOT_COMPILE_REGEXP); if (!is_no_capture) { number_of_subpatterns = re2->NumberOfCapturingGroups(); if (number_of_subpatterns > MAX_SUBPATTERNS) - throw Poco::Exception("OptimizedRegularExpression: too many subpatterns in regexp: " + regexp_); + throw DB::Exception("OptimizedRegularExpression: too many subpatterns in regexp: " + regexp_, DB::ErrorCodes::CANNOT_COMPILE_REGEXP); } } } @@ -432,6 +441,5 @@ unsigned OptimizedRegularExpressionImpl::match(const char * subject } } -#undef MIN_LENGTH_FOR_STRSTR -#undef MAX_SUBPATTERNS - +template class OptimizedRegularExpressionImpl; +template class OptimizedRegularExpressionImpl; diff --git a/dbms/src/Common/OptimizedRegularExpression.h b/dbms/src/Common/OptimizedRegularExpression.h index 60566c9f029..729711277b1 100644 --- a/dbms/src/Common/OptimizedRegularExpression.h +++ b/dbms/src/Common/OptimizedRegularExpression.h @@ -45,9 +45,9 @@ class OptimizedRegularExpressionImpl public: enum Options { - RE_CASELESS = 0x00000001, - RE_NO_CAPTURE = 0x00000010, - RE_DOT_NL = 0x00000100 + RE_CASELESS = 0x00000001, + RE_NO_CAPTURE = 0x00000010, + RE_DOT_NL = 0x00000100 }; using Match = OptimizedRegularExpressionDetails::Match; @@ -106,5 +106,3 @@ private: }; using OptimizedRegularExpression = OptimizedRegularExpressionImpl; - -#include "OptimizedRegularExpression.inl.h" From be80b7b188d969d5553978f9a84fcd7574312112 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 30 Nov 2018 22:47:47 +0300 Subject: [PATCH 49/69] Added suggestion from Nikolay Kochetov #3720 --- dbms/src/IO/WriteBufferFromVector.h | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/dbms/src/IO/WriteBufferFromVector.h b/dbms/src/IO/WriteBufferFromVector.h index 452e39cdde2..3b6f75f354e 100644 --- a/dbms/src/IO/WriteBufferFromVector.h +++ b/dbms/src/IO/WriteBufferFromVector.h @@ -8,6 +8,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_WRITE_AFTER_END_OF_BUFFER; +} + /** Writes data to existing std::vector or similar type. When not enough space, it doubles vector size. * * In destructor, vector is cutted to the size of written data. @@ -24,6 +29,9 @@ private: void nextImpl() override { + if (is_finished) + throw Exception("WriteBufferFromVector is finished", ErrorCodes::CANNOT_WRITE_AFTER_END_OF_BUFFER); + size_t old_size = vector.size(); vector.resize(old_size * 2); internal_buffer = Buffer(reinterpret_cast(&vector[old_size]), reinterpret_cast(vector.data() + vector.size())); @@ -49,6 +57,9 @@ public: ((position() - reinterpret_cast(vector.data())) + sizeof(typename VectorType::value_type) - 1) /// Align up. / sizeof(typename VectorType::value_type)); + + /// Prevent further writes. + set(nullptr, 0); } ~WriteBufferFromVector() override From 2221638adec440a623a01f4cce351c6fcd573378 Mon Sep 17 00:00:00 2001 From: proller Date: Sat, 1 Dec 2018 01:30:34 +0300 Subject: [PATCH 50/69] Fix presence of both libc++ and -static-libstdc++ link flags (#3706) * Fix precence of both libc++ and -static-libstdc++ link flags * cmake: NO_WERROR for ARCH_ARM * Try fix test link * revert * Update CMakeLists.txt --- CMakeLists.txt | 14 +++++++------- debian/.pbuilderrc | 7 +++++-- libs/libglibc-compatibility/tests/CMakeLists.txt | 5 ++++- 3 files changed, 16 insertions(+), 10 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index c5ae0ecbea6..6b507c57a59 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -159,13 +159,6 @@ set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${COMPILER_FLAGS} -fn set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3 ${CMAKE_C_FLAGS_ADD}") set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O0 -g3 -ggdb3 -fno-inline ${CMAKE_C_FLAGS_ADD}") -if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (COMPILER_CLANG AND OS_FREEBSD)) - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libgcc -static-libstdc++") - - # Along with executables, we also build example of shared library for "library dictionary source"; and it also should be self-contained. - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -static-libgcc -static-libstdc++") -endif () - set(THREADS_PREFER_PTHREAD_FLAG ON) find_package (Threads) @@ -193,6 +186,13 @@ if (OS_LINUX AND COMPILER_CLANG) endif () endif () +if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (COMPILER_CLANG AND OS_FREEBSD)) + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libgcc -static-libstdc++") + + # Along with executables, we also build example of shared library for "library dictionary source"; and it also should be self-contained. + set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -static-libgcc -static-libstdc++") +endif () + if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${FLAG_NO_PIE}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}") diff --git a/debian/.pbuilderrc b/debian/.pbuilderrc index 2bc4fd75342..4e5722fe9e5 100644 --- a/debian/.pbuilderrc +++ b/debian/.pbuilderrc @@ -97,7 +97,8 @@ ALLOWUNTRUSTED=${SET_ALLOWUNTRUSTED:=${ALLOWUNTRUSTED}} if $(echo ${DEBIAN_SUITES[@]} | grep -q $DIST); then # Debian configuration OSNAME=debian - MIRRORSITE=${SET_MIRRORSITE="http://deb.debian.org/$OSNAME/"} + #MIRRORSITE=${SET_MIRRORSITE="http://deb.debian.org/$OSNAME/"} + MIRRORSITE=${SET_MIRRORSITE="http://mirror.yandex.ru/$OSNAME/"} COMPONENTS="main contrib non-free" if $(echo "$STABLE_CODENAME stable" | grep -q $DIST); then OTHERMIRROR="$OTHERMIRROR | deb $MIRRORSITE $STABLE_BACKPORTS_SUITE $COMPONENTS" @@ -117,7 +118,8 @@ elif $(echo ${UBUNTU_SUITES[@]} | grep -q $DIST); then OSNAME=ubuntu if [[ "$ARCH" == "amd64" || "$ARCH" == "i386" ]]; then - MIRRORSITE=${SET_MIRRORSITE="http://archive.ubuntu.com/ubuntu/"} + #MIRRORSITE=${SET_MIRRORSITE="http://archive.ubuntu.com/$OSNAME/"} + MIRRORSITE=${SET_MIRRORSITE="http://mirror.yandex.ru/$OSNAME/"} else MIRRORSITE=${SET_MIRRORSITE="http://ports.ubuntu.com/ubuntu-ports/"} fi @@ -126,6 +128,7 @@ elif $(echo ${UBUNTU_SUITES[@]} | grep -q $DIST); then OTHERMIRROR="$OTHERMIRROR | deb $MIRRORSITE $DIST-updates main restricted universe multiverse" OTHERMIRROR="$OTHERMIRROR | deb $MIRRORSITE $DIST-security main restricted universe multiverse" + OTHERMIRROR="$OTHERMIRROR | deb $MIRRORSITE $DIST-proposed main restricted universe multiverse" case "$DIST" in "trusty" | "xenial" ) diff --git a/libs/libglibc-compatibility/tests/CMakeLists.txt b/libs/libglibc-compatibility/tests/CMakeLists.txt index 70a9e056226..a4f95187485 100644 --- a/libs/libglibc-compatibility/tests/CMakeLists.txt +++ b/libs/libglibc-compatibility/tests/CMakeLists.txt @@ -2,7 +2,10 @@ include (${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake) foreach (T longjmp siglongjmp) add_executable (${T} ${T}.c) - target_link_libraries (${T} glibc-compatibility) + target_link_libraries (${T} PRIVATE glibc-compatibility) + if (USE_LIBCXX) + target_link_libraries (${T} PRIVATE Threads::Threads) + endif () set_target_properties (${T} PROPERTIES LINKER_LANGUAGE CXX) add_check (${T}) endforeach () From 024a4ba73cb9c24523f4b462a30e0a4952db2f69 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 1 Dec 2018 02:52:20 +0300 Subject: [PATCH 51/69] Fixed build after merge #3689 --- dbms/src/Storages/StorageBuffer.cpp | 2 +- dbms/src/Storages/StorageMaterializedView.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 26e2d10f695..626814468d5 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -150,7 +150,7 @@ BlockInputStreams StorageBuffer::read( /// Collect columns from the destination tables which can be requested. /// Find out if there is a struct mismatch and we need to convert read blocks from the destination tables. - auto destination_lock = destination->lockStructure(false, __PRETTY_FUNCTION__); + auto destination_lock = destination->lockStructure(false); Names columns_intersection; bool struct_mismatch = false; diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index ab691529ac9..b73ee12b473 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -186,7 +186,7 @@ BlockInputStreams StorageMaterializedView::read( const unsigned num_streams) { auto storage = getTargetTable(); - auto lock = storage->lockStructure(false, __PRETTY_FUNCTION__); + auto lock = storage->lockStructure(false); auto streams = storage->read(column_names, query_info, context, processed_stage, max_block_size, num_streams); for (auto & stream : streams) stream->addTableLock(lock); @@ -196,7 +196,7 @@ BlockInputStreams StorageMaterializedView::read( BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const Settings & settings) { auto storage = getTargetTable(); - auto lock = storage->lockStructure(true, __PRETTY_FUNCTION__); + auto lock = storage->lockStructure(true); auto stream = storage->write(query, settings); stream->addTableLock(lock); return stream; From 2cf627457aa19902744f353d7da9daee7cd6ad29 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 1 Dec 2018 05:38:54 +0300 Subject: [PATCH 52/69] Fixed error #3720 --- dbms/src/IO/WriteBufferFromVector.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/IO/WriteBufferFromVector.h b/dbms/src/IO/WriteBufferFromVector.h index 3b6f75f354e..70e6ef5e36e 100644 --- a/dbms/src/IO/WriteBufferFromVector.h +++ b/dbms/src/IO/WriteBufferFromVector.h @@ -52,6 +52,8 @@ public: void finish() { + if (is_finished) + return; is_finished = true; vector.resize( ((position() - reinterpret_cast(vector.data())) From 566644b171155132bc2919138be9d4a5907c124f Mon Sep 17 00:00:00 2001 From: Kirill Shvakov Date: Sat, 1 Dec 2018 10:01:20 +0200 Subject: [PATCH 53/69] timeSlots: add optional parameter 'Size' --- dbms/src/Functions/timeSlots.cpp | 63 +++++++++++++------ .../0_stateless/00734_timeslot.reference | 2 + .../queries/0_stateless/00734_timeslot.sql | 2 + .../functions/date_time_functions.md | 4 +- .../functions/date_time_functions.md | 5 +- 5 files changed, 52 insertions(+), 24 deletions(-) diff --git a/dbms/src/Functions/timeSlots.cpp b/dbms/src/Functions/timeSlots.cpp index e2de8cac702..1e48a952b0d 100644 --- a/dbms/src/Functions/timeSlots.cpp +++ b/dbms/src/Functions/timeSlots.cpp @@ -8,6 +8,7 @@ #include #include +#include namespace DB { @@ -30,10 +31,8 @@ namespace ErrorCodes template struct TimeSlotsImpl { - static constexpr UInt32 TIME_SLOT_SIZE = 1800; - static void vector_vector( - const PaddedPODArray & starts, const PaddedPODArray & durations, + const PaddedPODArray & starts, const PaddedPODArray & durations, UInt32 time_slot_size, PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets) { size_t size = starts.size(); @@ -44,9 +43,9 @@ struct TimeSlotsImpl ColumnArray::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - for (UInt32 value = starts[i] / TIME_SLOT_SIZE; value <= (starts[i] + durations[i]) / TIME_SLOT_SIZE; ++value) + for (UInt32 value = starts[i] / time_slot_size; value <= (starts[i] + durations[i]) / time_slot_size; ++value) { - result_values.push_back(value * TIME_SLOT_SIZE); + result_values.push_back(value * time_slot_size); ++current_offset; } @@ -55,7 +54,7 @@ struct TimeSlotsImpl } static void vector_constant( - const PaddedPODArray & starts, DurationType duration, + const PaddedPODArray & starts, DurationType duration, UInt32 time_slot_size, PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets) { size_t size = starts.size(); @@ -66,9 +65,9 @@ struct TimeSlotsImpl ColumnArray::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - for (UInt32 value = starts[i] / TIME_SLOT_SIZE; value <= (starts[i] + duration) / TIME_SLOT_SIZE; ++value) + for (UInt32 value = starts[i] / time_slot_size; value <= (starts[i] + duration) / time_slot_size; ++value) { - result_values.push_back(value * TIME_SLOT_SIZE); + result_values.push_back(value * time_slot_size); ++current_offset; } @@ -77,7 +76,7 @@ struct TimeSlotsImpl } static void constant_vector( - UInt32 start, const PaddedPODArray & durations, + UInt32 start, const PaddedPODArray & durations, UInt32 time_slot_size, PaddedPODArray & result_values, ColumnArray::Offsets & result_offsets) { size_t size = durations.size(); @@ -88,9 +87,9 @@ struct TimeSlotsImpl ColumnArray::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - for (UInt32 value = start / TIME_SLOT_SIZE; value <= (start + durations[i]) / TIME_SLOT_SIZE; ++value) + for (UInt32 value = start / time_slot_size; value <= (start + durations[i]) / time_slot_size; ++value) { - result_values.push_back(value * TIME_SLOT_SIZE); + result_values.push_back(value * time_slot_size); ++current_offset; } @@ -99,11 +98,11 @@ struct TimeSlotsImpl } static void constant_constant( - UInt32 start, DurationType duration, + UInt32 start, DurationType duration, UInt32 time_slot_size, Array & result) { - for (UInt32 value = start / TIME_SLOT_SIZE; value <= (start + duration) / TIME_SLOT_SIZE; ++value) - result.push_back(value * TIME_SLOT_SIZE); + for (UInt32 value = start / time_slot_size; value <= (start + duration) / time_slot_size; ++value) + result.push_back(value * time_slot_size); } }; @@ -112,6 +111,7 @@ class FunctionTimeSlots : public IFunction { public: static constexpr auto name = "timeSlots"; + static constexpr UInt32 TIME_SLOT_SIZE = 1800; static FunctionPtr create(const Context &) { return std::make_shared(); } String getName() const override @@ -119,10 +119,16 @@ public: return name; } - size_t getNumberOfArguments() const override { return 2; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { + if (arguments.size() != 2 && arguments.size() != 3) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 2 or 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + if (!WhichDataType(arguments[0].type).isDateTime()) throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + ". Must be DateTime.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -131,9 +137,13 @@ public: throw Exception("Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() + ". Must be UInt32.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isUInt32()) + throw Exception("Illegal type " + arguments[1].type->getName() + " of third argument of function " + getName() + ". Must be UInt32.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + /// If time zone is specified for source data type, attach it to the resulting type. /// Note that there is no explicit time zone argument for this function (we specify 2 as an argument number with explicit time zone). - return std::make_shared(std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0))); + return std::make_shared(std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0))); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override @@ -147,30 +157,43 @@ public: auto res = ColumnArray::create(ColumnUInt32::create()); ColumnUInt32::Container & res_values = typeid_cast(res->getData()).getData(); + auto time_slot_size = TIME_SLOT_SIZE; + + if (arguments.size() == 3) + { + auto time_slot_column = checkAndGetColumnConst(block.getByPosition(arguments[2]).column.get()); + if (!time_slot_column) + throw Exception("Third argument for function " + getName() + " must be constant UInt32", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (time_slot_size = time_slot_column->getValue(); time_slot_size == 0) + throw Exception("Third argument for function " + getName() + " must be greater than zero", ErrorCodes::ILLEGAL_COLUMN); + } + if (starts && durations) { - TimeSlotsImpl::vector_vector(starts->getData(), durations->getData(), res_values, res->getOffsets()); + TimeSlotsImpl::vector_vector(starts->getData(), durations->getData(), time_slot_size, res_values, res->getOffsets()); block.getByPosition(result).column = std::move(res); } else if (starts && const_durations) { - TimeSlotsImpl::vector_constant(starts->getData(), const_durations->getValue(), res_values, res->getOffsets()); + TimeSlotsImpl::vector_constant(starts->getData(), const_durations->getValue(), time_slot_size, res_values, res->getOffsets()); block.getByPosition(result).column = std::move(res); } else if (const_starts && durations) { - TimeSlotsImpl::constant_vector(const_starts->getValue(), durations->getData(), res_values, res->getOffsets()); + TimeSlotsImpl::constant_vector(const_starts->getValue(), durations->getData(), time_slot_size, res_values, res->getOffsets()); block.getByPosition(result).column = std::move(res); } else if (const_starts && const_durations) { Array const_res; - TimeSlotsImpl::constant_constant(const_starts->getValue(), const_durations->getValue(), const_res); + TimeSlotsImpl::constant_constant(const_starts->getValue(), const_durations->getValue(), time_slot_size, const_res); block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, const_res); } else throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName() + ", " + block.getByPosition(arguments[1]).column->getName() + + ", " + block.getByPosition(arguments[2]).column->getName() + " of arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } diff --git a/dbms/tests/queries/0_stateless/00734_timeslot.reference b/dbms/tests/queries/0_stateless/00734_timeslot.reference index 5acfa2b537c..17b1cd9cd10 100644 --- a/dbms/tests/queries/0_stateless/00734_timeslot.reference +++ b/dbms/tests/queries/0_stateless/00734_timeslot.reference @@ -1,2 +1,4 @@ 2000-01-02 03:00:00 ['2000-01-02 03:00:00','2000-01-02 03:30:00','2000-01-02 04:00:00','2000-01-02 04:30:00','2000-01-02 05:00:00','2000-01-02 05:30:00'] +['2000-01-02 03:00:00','2000-01-02 03:10:00','2000-01-02 03:20:00','2000-01-02 03:30:00','2000-01-02 03:40:00','2000-01-02 03:50:00','2000-01-02 04:00:00','2000-01-02 04:10:00','2000-01-02 04:20:00','2000-01-02 04:30:00','2000-01-02 04:40:00','2000-01-02 04:50:00','2000-01-02 05:00:00','2000-01-02 05:10:00','2000-01-02 05:20:00','2000-01-02 05:30:00','2000-01-02 05:40:00','2000-01-02 05:50:00'] +['2000-01-02 03:04:00','2000-01-02 03:04:30','2000-01-02 03:05:00','2000-01-02 03:05:30','2000-01-02 03:06:00','2000-01-02 03:06:30','2000-01-02 03:07:00','2000-01-02 03:07:30','2000-01-02 03:08:00','2000-01-02 03:08:30','2000-01-02 03:09:00','2000-01-02 03:09:30','2000-01-02 03:10:00','2000-01-02 03:10:30','2000-01-02 03:11:00','2000-01-02 03:11:30','2000-01-02 03:12:00','2000-01-02 03:12:30','2000-01-02 03:13:00','2000-01-02 03:13:30','2000-01-02 03:14:00'] diff --git a/dbms/tests/queries/0_stateless/00734_timeslot.sql b/dbms/tests/queries/0_stateless/00734_timeslot.sql index 7362074ccaf..ff7b10b8eee 100644 --- a/dbms/tests/queries/0_stateless/00734_timeslot.sql +++ b/dbms/tests/queries/0_stateless/00734_timeslot.sql @@ -1,2 +1,4 @@ SELECT timeSlot(toDateTime('2000-01-02 03:04:05', 'UTC')); SELECT timeSlots(toDateTime('2000-01-02 03:04:05', 'UTC'), toUInt32(10000)); +SELECT timeSlots(toDateTime('2000-01-02 03:04:05', 'UTC'), toUInt32(10000), toUInt32(600)); +SELECT timeSlots(toDateTime('2000-01-02 03:04:05', 'UTC'), toUInt32(600), toUInt32(30)); \ No newline at end of file diff --git a/docs/en/query_language/functions/date_time_functions.md b/docs/en/query_language/functions/date_time_functions.md index d451cf2f9d4..d07457e1410 100644 --- a/docs/en/query_language/functions/date_time_functions.md +++ b/docs/en/query_language/functions/date_time_functions.md @@ -148,9 +148,9 @@ The same as 'today() - 1'. Rounds the time to the half hour. This function is specific to Yandex.Metrica, since half an hour is the minimum amount of time for breaking a session into two sessions if a tracking tag shows a single user's consecutive pageviews that differ in time by strictly more than this amount. This means that tuples (the tag ID, user ID, and time slot) can be used to search for pageviews that are included in the corresponding session. -## timeSlots(StartTime, Duration) +## timeSlots(StartTime, Duration,\[, Size\]) -For a time interval starting at 'StartTime' and continuing for 'Duration' seconds, it returns an array of moments in time, consisting of points from this interval rounded down to the half hour. +For a time interval starting at 'StartTime' and continuing for 'Duration' seconds, it returns an array of moments in time, consisting of points from this interval rounded down to the 'Size' in seconds. 'Size' is an optional parameter: a constant UInt32, set to 1800 by default. For example, `timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]`. This is necessary for searching for pageviews in the corresponding session. diff --git a/docs/ru/query_language/functions/date_time_functions.md b/docs/ru/query_language/functions/date_time_functions.md index 048b8a9ce8c..141ee6d42a6 100644 --- a/docs/ru/query_language/functions/date_time_functions.md +++ b/docs/ru/query_language/functions/date_time_functions.md @@ -121,8 +121,9 @@ SELECT Округляет время до получаса. Эта функция является специфичной для Яндекс.Метрики, так как пол часа - минимальное время, для которого, если соседние по времени хиты одного посетителя на одном счётчике отстоят друг от друга строго более, чем на это время, визит может быть разбит на два визита. То есть, кортежи (номер счётчика, идентификатор посетителя, тайм-слот) могут использоваться для поиска хитов, входящий в соответствующий визит. -## timeSlots(StartTime, Duration) -Для интервала времени, начинающегося в StartTime и продолжающегося Duration секунд, возвращает массив моментов времени, состоящий из округлений вниз до получаса точек из этого интервала. +## timeSlots(StartTime, Duration,\[, Size\]) +Для интервала времени, начинающегося в 'StartTime' и продолжающегося 'Duration' секунд, возвращает массив моментов времени, состоящий из округлений вниз до 'Size' точек в секундах из этого интервала. 'Size' - необязательный параметр, константный UInt32, по умолчанию равен 1800. + Например, `timeSlots(toDateTime('2012-01-01 12:20:00'), toUInt32(600)) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]`. Это нужно для поиска хитов, входящих в соответствующий визит. From 7c03b35bc45375b946c877c66765d8c3b21ee512 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 1 Dec 2018 15:42:56 +0300 Subject: [PATCH 54/69] CLICKHOUSE-4127: Fix assertion in debug build. Do reading faster if the structure isn't altered. --- dbms/src/Storages/StorageBuffer.cpp | 82 ++++++++++++++++------------- 1 file changed, 46 insertions(+), 36 deletions(-) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 626814468d5..c93ef08801c 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -148,56 +149,65 @@ BlockInputStreams StorageBuffer::read( if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); - /// Collect columns from the destination tables which can be requested. - /// Find out if there is a struct mismatch and we need to convert read blocks from the destination tables. auto destination_lock = destination->lockStructure(false); - Names columns_intersection; - bool struct_mismatch = false; - for (const String & column_name : column_names) + const bool dst_has_same_structure = std::all_of(column_names.begin(), column_names.end(), [this, destination](const String& column_name) { - if (destination->hasColumn(column_name)) + return destination->hasColumn(column_name) && + destination->getColumn(column_name).type->equals(*getColumn(column_name).type); + }); + + if (dst_has_same_structure) + { + /// The destination table has the same structure of the requested columns and we can simply read blocks from there. + streams_from_dst = destination->read(column_names, query_info, context, processed_stage, max_block_size, num_streams); + } + else + { + /// There is a struct mismatch and we need to convert read blocks from the destination table. + const Block header = getSampleBlock(); + Names columns_intersection = column_names; + Block header_after_adding_defaults = header; + for (const String & column_name : column_names) { - columns_intersection.emplace_back(column_name); - if (!destination->getColumn(column_name).type->equals(*getColumn(column_name).type)) + if (!destination->hasColumn(column_name)) + { + LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) + << " doesn't have column " << backQuoteIfNeed(column_name) << ". The default values are used."); + boost::range::remove_erase(columns_intersection, column_name); + continue; + } + const auto & dst_col = destination->getColumn(column_name); + const auto & col = getColumn(column_name); + if (!dst_col.type->equals(*col.type)) { LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) << " has different type of column " << backQuoteIfNeed(column_name) << " (" - << destination->getColumn(column_name).type->getName() << " != " << getColumn(column_name).type->getName() - << "). Data from destination table is converted."); - struct_mismatch = true; + << dst_col.type->getName() << " != " << col.type->getName() << "). Data from destination table are converted."); + header_after_adding_defaults.getByName(column_name) = ColumnWithTypeAndName(dst_col.type, column_name); } } + + if (columns_intersection.empty()) + { + LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) + << " has no common columns with block in buffer. Block of data is skipped."); + } else { - LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) - << " doesn't have column " << backQuoteIfNeed(column_name) << ". The default values are used."); - struct_mismatch = true; + streams_from_dst = destination->read(columns_intersection, query_info, context, processed_stage, max_block_size, num_streams); + for (auto & stream : streams_from_dst) + { + stream = std::make_shared( + stream, header_after_adding_defaults, getColumns().defaults, context); + stream = std::make_shared( + context, stream, header, ConvertingBlockInputStream::MatchColumnsMode::Name); + } } } - if (columns_intersection.empty()) - LOG_WARNING(log, "Destination table " << backQuoteIfNeed(destination_database) << "." << backQuoteIfNeed(destination_table) - << " has no common columns with block in buffer. Block of data is skipped."); - else - { - streams_from_dst = destination->read(columns_intersection, query_info, context, processed_stage, max_block_size, num_streams); - for (auto & stream : streams_from_dst) - stream->addTableLock(destination_lock); - } - - if (struct_mismatch && !streams_from_dst.empty()) - { - /// Add streams to convert read blocks from the destination table. - auto header = getSampleBlock(); - for (auto & stream_from_dst : streams_from_dst) - { - stream_from_dst = std::make_shared( - stream_from_dst, header, getColumns().defaults, context); - stream_from_dst = std::make_shared( - context, stream_from_dst, header, ConvertingBlockInputStream::MatchColumnsMode::Name); - } - } + for (auto & stream : streams_from_dst) + stream->addTableLock(destination_lock); } BlockInputStreams streams_from_buffers; From c9b8cd06263a6511802e77123708cdfe1b541d71 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Dec 2018 03:47:34 +0300 Subject: [PATCH 55/69] Fixed errors and improved PR #3724 --- dbms/src/Functions/timeSlots.cpp | 35 +++++++------------ .../queries/0_stateless/00734_timeslot.sql | 4 +-- 2 files changed, 14 insertions(+), 25 deletions(-) diff --git a/dbms/src/Functions/timeSlots.cpp b/dbms/src/Functions/timeSlots.cpp index 1e48a952b0d..7ee0d1a1b2f 100644 --- a/dbms/src/Functions/timeSlots.cpp +++ b/dbms/src/Functions/timeSlots.cpp @@ -21,11 +21,12 @@ namespace ErrorCodes /** timeSlots(StartTime, Duration) * - for the time interval beginning at `StartTime` and continuing `Duration` seconds, - * returns an array of time points, consisting of rounding down to half an hour of points from this interval. + * returns an array of time points, consisting of rounding down to half an hour (default; or another value) of points from this interval. * For example, timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]. * This is necessary to search for hits that are part of the corresponding visit. * - * This is obsolete function. It was developed for Yandex.Metrica, but no longer used. + * This is obsolete function. It was developed for Yandex.Metrica, but no longer used in Yandex. + * But this function was adopted by wider audience. */ template @@ -43,7 +44,7 @@ struct TimeSlotsImpl ColumnArray::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - for (UInt32 value = starts[i] / time_slot_size; value <= (starts[i] + durations[i]) / time_slot_size; ++value) + for (UInt32 value = starts[i] / time_slot_size, end = (starts[i] + durations[i]) / time_slot_size; value <= end; ++value) { result_values.push_back(value * time_slot_size); ++current_offset; @@ -65,7 +66,7 @@ struct TimeSlotsImpl ColumnArray::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - for (UInt32 value = starts[i] / time_slot_size; value <= (starts[i] + duration) / time_slot_size; ++value) + for (UInt32 value = starts[i] / time_slot_size, end = (starts[i] + duration) / time_slot_size; value <= end; ++value) { result_values.push_back(value * time_slot_size); ++current_offset; @@ -87,7 +88,7 @@ struct TimeSlotsImpl ColumnArray::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - for (UInt32 value = start / time_slot_size; value <= (start + durations[i]) / time_slot_size; ++value) + for (UInt32 value = start / time_slot_size, end = (start + durations[i]) / time_slot_size; value <= end; ++value) { result_values.push_back(value * time_slot_size); ++current_offset; @@ -96,14 +97,6 @@ struct TimeSlotsImpl result_offsets[i] = current_offset; } } - - static void constant_constant( - UInt32 start, DurationType duration, UInt32 time_slot_size, - Array & result) - { - for (UInt32 value = start / time_slot_size; value <= (start + duration) / time_slot_size; ++value) - result.push_back(value * time_slot_size); - } }; @@ -121,6 +114,8 @@ public: bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {2}; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { @@ -137,8 +132,8 @@ public: throw Exception("Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() + ". Must be UInt32.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isUInt32()) - throw Exception("Illegal type " + arguments[1].type->getName() + " of third argument of function " + getName() + ". Must be UInt32.", + if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isNativeUInt()) + throw Exception("Illegal type " + arguments[2].type->getName() + " of third argument of function " + getName() + ". Must be UInt32.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); /// If time zone is specified for source data type, attach it to the resulting type. @@ -146,7 +141,7 @@ public: return std::make_shared(std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0))); } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override { auto starts = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); auto const_starts = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get()); @@ -161,7 +156,7 @@ public: if (arguments.size() == 3) { - auto time_slot_column = checkAndGetColumnConst(block.getByPosition(arguments[2]).column.get()); + auto time_slot_column = checkAndGetColumn(block.getByPosition(arguments[2]).column.get()); if (!time_slot_column) throw Exception("Third argument for function " + getName() + " must be constant UInt32", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -184,12 +179,6 @@ public: TimeSlotsImpl::constant_vector(const_starts->getValue(), durations->getData(), time_slot_size, res_values, res->getOffsets()); block.getByPosition(result).column = std::move(res); } - else if (const_starts && const_durations) - { - Array const_res; - TimeSlotsImpl::constant_constant(const_starts->getValue(), const_durations->getValue(), time_slot_size, const_res); - block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, const_res); - } else throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName() + ", " + block.getByPosition(arguments[1]).column->getName() diff --git a/dbms/tests/queries/0_stateless/00734_timeslot.sql b/dbms/tests/queries/0_stateless/00734_timeslot.sql index ff7b10b8eee..51e6a3f5330 100644 --- a/dbms/tests/queries/0_stateless/00734_timeslot.sql +++ b/dbms/tests/queries/0_stateless/00734_timeslot.sql @@ -1,4 +1,4 @@ SELECT timeSlot(toDateTime('2000-01-02 03:04:05', 'UTC')); SELECT timeSlots(toDateTime('2000-01-02 03:04:05', 'UTC'), toUInt32(10000)); -SELECT timeSlots(toDateTime('2000-01-02 03:04:05', 'UTC'), toUInt32(10000), toUInt32(600)); -SELECT timeSlots(toDateTime('2000-01-02 03:04:05', 'UTC'), toUInt32(600), toUInt32(30)); \ No newline at end of file +SELECT timeSlots(toDateTime('2000-01-02 03:04:05', 'UTC'), toUInt32(10000), 600); +SELECT timeSlots(toDateTime('2000-01-02 03:04:05', 'UTC'), toUInt32(600), 30); From dbbf148a98a5ef436da2ecf053baf70837f274a9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Dec 2018 04:57:01 +0300 Subject: [PATCH 56/69] Every function in its own file, part 8 [#CLICKHOUSE-2] --- dbms/src/Functions/FunctionsCoding.cpp | 1 - dbms/src/Functions/FunctionsCoding.h | 42 ----- dbms/src/Functions/FunctionsRandom.cpp | 62 +++++++- dbms/src/Functions/FunctionsRandom.h | 150 +----------------- dbms/src/Functions/generateUUIDv4.cpp | 57 +++++++ dbms/src/Functions/rand.cpp | 16 ++ dbms/src/Functions/rand64.cpp | 17 ++ dbms/src/Functions/randConstant.cpp | 62 ++++++++ .../src/Functions/registerFunctionsRandom.cpp | 21 +++ 9 files changed, 233 insertions(+), 195 deletions(-) create mode 100644 dbms/src/Functions/generateUUIDv4.cpp create mode 100644 dbms/src/Functions/rand.cpp create mode 100644 dbms/src/Functions/rand64.cpp create mode 100644 dbms/src/Functions/randConstant.cpp create mode 100644 dbms/src/Functions/registerFunctionsRandom.cpp diff --git a/dbms/src/Functions/FunctionsCoding.cpp b/dbms/src/Functions/FunctionsCoding.cpp index adc660df78f..7643e77b205 100644 --- a/dbms/src/Functions/FunctionsCoding.cpp +++ b/dbms/src/Functions/FunctionsCoding.cpp @@ -23,7 +23,6 @@ void registerFunctionsCoding(FunctionFactory & factory) factory.registerFunction>(); factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 5270cea0ac6..f505e97d423 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -17,7 +17,6 @@ #include #include #include -#include #include #include @@ -1093,47 +1092,6 @@ public: } }; -class FunctionGenerateUUIDv4 : public IFunction -{ -public: - static constexpr auto name = "generateUUIDv4"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const DataTypes &) const override - { - return std::make_shared(); - } - - bool isDeterministic() const override { return false; } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - auto col_res = ColumnVector::create(); - typename ColumnVector::Container & vec_to = col_res->getData(); - - size_t size = input_rows_count; - vec_to.resize(size); - Rand64Impl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * 2); - - for (UInt128 & uuid: vec_to) - { - /** https://tools.ietf.org/html/rfc4122#section-4.4 - */ - uuid.low = (uuid.low & 0xffffffffffff0fffull) | 0x0000000000004000ull; - uuid.high = (uuid.high & 0x3fffffffffffffffull) | 0x8000000000000000ull; - } - - block.getByPosition(result).column = std::move(col_res); - } -}; - class FunctionHex : public IFunction { diff --git a/dbms/src/Functions/FunctionsRandom.cpp b/dbms/src/Functions/FunctionsRandom.cpp index d51b0a3b03d..8bb309561f1 100644 --- a/dbms/src/Functions/FunctionsRandom.cpp +++ b/dbms/src/Functions/FunctionsRandom.cpp @@ -1,24 +1,72 @@ #include #include +#include +#include +#include + namespace DB { -namespace detail +namespace { + /// NOTE Probably + /// http://www.pcg-random.org/ + /// or http://www.math.sci.hiroshima-u.ac.jp/~m-mat/MT/SFMT/ + /// or http://docs.yeppp.info/c/group__yep_random___w_e_l_l1024a.html + /// could go better. + + struct LinearCongruentialGenerator + { + /// Constants from `man lrand48_r`. + static constexpr UInt64 a = 0x5DEECE66D; + static constexpr UInt64 c = 0xB; + + /// And this is from `head -c8 /dev/urandom | xxd -p` + UInt64 current = 0x09826f4a081cee35ULL; + + LinearCongruentialGenerator() {} + LinearCongruentialGenerator(UInt64 value) : current(value) {} + + void seed(UInt64 value) + { + current = value; + } + + UInt32 next() + { + current = current * a + c; + return current >> 16; + } + }; + void seed(LinearCongruentialGenerator & generator, intptr_t additional_seed) { generator.seed(intHash64(randomSeed() ^ intHash64(additional_seed))); } } - - -void registerFunctionsRandom(FunctionFactory & factory) +void RandImpl::execute(char * output, size_t size) { - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + LinearCongruentialGenerator generator0; + LinearCongruentialGenerator generator1; + LinearCongruentialGenerator generator2; + LinearCongruentialGenerator generator3; + + seed(generator0, 0xfb4121280b2ab902ULL + reinterpret_cast(output)); + seed(generator1, 0x0121cf76df39c673ULL + reinterpret_cast(output)); + seed(generator2, 0x17ae86e3a19a602fULL + reinterpret_cast(output)); + seed(generator3, 0x8b6e16da7e06d622ULL + reinterpret_cast(output)); + + for (const char * end = output + size; output < end; output += 16) + { + unalignedStore(output, generator0.next()); + unalignedStore(output + 4, generator1.next()); + unalignedStore(output + 8, generator2.next()); + unalignedStore(output + 12, generator3.next()); + } + + /// It is guaranteed (by PaddedPODArray) that we can overwrite up to 15 bytes after end. } } diff --git a/dbms/src/Functions/FunctionsRandom.h b/dbms/src/Functions/FunctionsRandom.h index 1f00f8b24ea..47cb275ee89 100644 --- a/dbms/src/Functions/FunctionsRandom.h +++ b/dbms/src/Functions/FunctionsRandom.h @@ -2,12 +2,8 @@ #include #include -#include #include -#include #include -#include -#include namespace DB @@ -38,95 +34,16 @@ namespace ErrorCodes * This means that the timer must be of sufficient resolution to give different values to each block. */ -namespace detail -{ - /// NOTE Probably - /// http://www.pcg-random.org/ - /// or http://www.math.sci.hiroshima-u.ac.jp/~m-mat/MT/SFMT/ - /// or http://docs.yeppp.info/c/group__yep_random___w_e_l_l1024a.html - /// could go better. - - struct LinearCongruentialGenerator - { - /// Constants from `man lrand48_r`. - static constexpr UInt64 a = 0x5DEECE66D; - static constexpr UInt64 c = 0xB; - - /// And this is from `head -c8 /dev/urandom | xxd -p` - UInt64 current = 0x09826f4a081cee35ULL; - - LinearCongruentialGenerator() {} - LinearCongruentialGenerator(UInt64 value) : current(value) {} - - void seed(UInt64 value) - { - current = value; - } - - UInt32 next() - { - current = current * a + c; - return current >> 16; - } - }; - - void seed(LinearCongruentialGenerator & generator, intptr_t additional_seed); -} - struct RandImpl { - using ReturnType = UInt32; - - static void execute(ReturnType * output, size_t size) - { - detail::LinearCongruentialGenerator generator0; - detail::LinearCongruentialGenerator generator1; - detail::LinearCongruentialGenerator generator2; - detail::LinearCongruentialGenerator generator3; - - detail::seed(generator0, 0xfb4121280b2ab902ULL + reinterpret_cast(output)); - detail::seed(generator1, 0x0121cf76df39c673ULL + reinterpret_cast(output)); - detail::seed(generator2, 0x17ae86e3a19a602fULL + reinterpret_cast(output)); - detail::seed(generator3, 0x8b6e16da7e06d622ULL + reinterpret_cast(output)); - - ReturnType * pos = output; - ReturnType * end = pos + size; - ReturnType * end4 = pos + size / 4 * 4; - - while (pos < end4) - { - pos[0] = generator0.next(); - pos[1] = generator1.next(); - pos[2] = generator2.next(); - pos[3] = generator3.next(); - pos += 4; - } - - while (pos < end) - { - pos[0] = generator0.next(); - ++pos; - } - } -}; - -struct Rand64Impl -{ - using ReturnType = UInt64; - - static void execute(ReturnType * output, size_t size) - { - RandImpl::execute(reinterpret_cast(output), size * 2); - } + /// Fill memory with random data. The memory region must be 15-bytes padded. + static void execute(char * output, size_t size); }; -template +template class FunctionRandom : public IFunction { -private: - using ToType = typename Impl::ReturnType; - public: static constexpr auto name = Name::name; static FunctionPtr create(const Context &) { return std::make_shared(); } @@ -147,7 +64,7 @@ public: + toString(arguments.size()) + ", should be 0 or 1.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return std::make_shared>(); + return std::make_shared>(); } void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override @@ -157,67 +74,10 @@ public: size_t size = input_rows_count; vec_to.resize(size); - Impl::execute(vec_to.data(), vec_to.size()); + RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(ToType)); block.getByPosition(result).column = std::move(col_to); } }; - -template -class FunctionRandomConstant : public IFunction -{ -private: - using ToType = typename Impl::ReturnType; - - /// The value is one for different blocks. - bool is_initialized = false; - ToType value; - -public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override - { - return name; - } - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (arguments.size() > 1) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be 0 or 1.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - return std::make_shared>(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - if (!is_initialized) - { - is_initialized = true; - typename ColumnVector::Container vec_to(1); - Impl::execute(vec_to.data(), vec_to.size()); - value = vec_to[0]; - } - - block.getByPosition(result).column = DataTypeNumber().createColumnConst(input_rows_count, toField(value)); - } -}; - - -struct NameRand { static constexpr auto name = "rand"; }; -struct NameRand64 { static constexpr auto name = "rand64"; }; -struct NameRandConstant { static constexpr auto name = "randConstant"; }; - -using FunctionRand = FunctionRandom ; -using FunctionRand64 = FunctionRandom; -using FunctionRandConstant = FunctionRandomConstant; - - } diff --git a/dbms/src/Functions/generateUUIDv4.cpp b/dbms/src/Functions/generateUUIDv4.cpp new file mode 100644 index 00000000000..63645614251 --- /dev/null +++ b/dbms/src/Functions/generateUUIDv4.cpp @@ -0,0 +1,57 @@ +#include +#include +#include + +namespace DB +{ + +class FunctionGenerateUUIDv4 : public IFunction +{ +public: + static constexpr auto name = "generateUUIDv4"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes &) const override + { + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_to = col_res->getData(); + + size_t size = input_rows_count; + vec_to.resize(size); + RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(UInt128)); + + for (UInt128 & uuid: vec_to) + { + /** https://tools.ietf.org/html/rfc4122#section-4.4 + */ + uuid.low = (uuid.low & 0xffffffffffff0fffull) | 0x0000000000004000ull; + uuid.high = (uuid.high & 0x3fffffffffffffffull) | 0x8000000000000000ull; + } + + block.getByPosition(result).column = std::move(col_res); + } +}; + +void registerFunctionGenerateUUIDv4(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + + + diff --git a/dbms/src/Functions/rand.cpp b/dbms/src/Functions/rand.cpp new file mode 100644 index 00000000000..cd4ced96b7e --- /dev/null +++ b/dbms/src/Functions/rand.cpp @@ -0,0 +1,16 @@ +#include +#include + +namespace DB +{ + +struct NameRand { static constexpr auto name = "rand"; }; +using FunctionRand = FunctionRandom; + +void registerFunctionRand(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + diff --git a/dbms/src/Functions/rand64.cpp b/dbms/src/Functions/rand64.cpp new file mode 100644 index 00000000000..cc942aee28c --- /dev/null +++ b/dbms/src/Functions/rand64.cpp @@ -0,0 +1,17 @@ +#include +#include + +namespace DB +{ + +struct NameRand64 { static constexpr auto name = "rand64"; }; +using FunctionRand64 = FunctionRandom; + +void registerFunctionRand64(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + + diff --git a/dbms/src/Functions/randConstant.cpp b/dbms/src/Functions/randConstant.cpp new file mode 100644 index 00000000000..91df7a1dfb8 --- /dev/null +++ b/dbms/src/Functions/randConstant.cpp @@ -0,0 +1,62 @@ +#include +#include +#include + +namespace DB +{ + +template +class FunctionRandomConstant : public IFunction +{ +private: + /// The value is one for different blocks. + bool is_initialized = false; + ToType value; + +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() > 1) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 0 or 1.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return std::make_shared>(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + if (!is_initialized) + { + is_initialized = true; + typename ColumnVector::Container vec_to(1); + RandImpl::execute(reinterpret_cast(vec_to.data()), sizeof(ToType)); + value = vec_to[0]; + } + + block.getByPosition(result).column = DataTypeNumber().createColumnConst(input_rows_count, toField(value)); + } +}; + +struct NameRandConstant { static constexpr auto name = "randConstant"; }; +using FunctionRandConstant = FunctionRandomConstant; + +void registerFunctionRandConstant(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + + diff --git a/dbms/src/Functions/registerFunctionsRandom.cpp b/dbms/src/Functions/registerFunctionsRandom.cpp new file mode 100644 index 00000000000..92a0505d530 --- /dev/null +++ b/dbms/src/Functions/registerFunctionsRandom.cpp @@ -0,0 +1,21 @@ +namespace DB +{ + +class FunctionFactory; + +void registerFunctionRand(FunctionFactory & factory); +void registerFunctionRand64(FunctionFactory & factory); +void registerFunctionRandConstant(FunctionFactory & factory); +void registerFunctionGenerateUUIDv4(FunctionFactory & factory); + +void registerFunctionsRandom(FunctionFactory & factory) +{ + registerFunctionRand(factory); + registerFunctionRand64(factory); + registerFunctionRandConstant(factory); + registerFunctionGenerateUUIDv4(factory); +} + +} + + From 70a1275d50407a0fa7a8db6fc7a591fd9c20b898 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Dec 2018 05:47:47 +0300 Subject: [PATCH 57/69] Every function in its own file, part 8 [#CLICKHOUSE-2] --- dbms/src/Functions/FunctionIfBase.h | 69 ++++ dbms/src/Functions/FunctionsConditional.cpp | 337 ------------------ dbms/src/Functions/caseWithExpression.cpp | 128 +++++++ .../{FunctionsConditional.h => if.cpp} | 120 +------ dbms/src/Functions/multiIf.cpp | 249 +++++++++++++ .../registerFunctionsConditional.cpp | 20 ++ 6 files changed, 471 insertions(+), 452 deletions(-) create mode 100644 dbms/src/Functions/FunctionIfBase.h delete mode 100644 dbms/src/Functions/FunctionsConditional.cpp create mode 100644 dbms/src/Functions/caseWithExpression.cpp rename dbms/src/Functions/{FunctionsConditional.h => if.cpp} (90%) create mode 100644 dbms/src/Functions/multiIf.cpp create mode 100644 dbms/src/Functions/registerFunctionsConditional.cpp diff --git a/dbms/src/Functions/FunctionIfBase.h b/dbms/src/Functions/FunctionIfBase.h new file mode 100644 index 00000000000..dfc399071bf --- /dev/null +++ b/dbms/src/Functions/FunctionIfBase.h @@ -0,0 +1,69 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +template +class FunctionIfBase : public IFunction +{ +#if USE_EMBEDDED_COMPILER +public: + bool isCompilableImpl(const DataTypes & types) const override + { + for (const auto & type : types) + if (!isCompilableType(removeNullable(type))) + return false; + return true; + } + + llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const DataTypes & types, ValuePlaceholders values) const override + { + auto & b = static_cast &>(builder); + auto type = getReturnTypeImpl(types); + llvm::Value * null = nullptr; + if (!null_is_false && type->isNullable()) + null = b.CreateInsertValue(llvm::Constant::getNullValue(toNativeType(b, type)), b.getTrue(), {1}); + auto * head = b.GetInsertBlock(); + auto * join = llvm::BasicBlock::Create(head->getContext(), "", head->getParent()); + std::vector> returns; + for (size_t i = 0; i + 1 < types.size(); i += 2) + { + auto * then = llvm::BasicBlock::Create(head->getContext(), "", head->getParent()); + auto * next = llvm::BasicBlock::Create(head->getContext(), "", head->getParent()); + auto * cond = values[i](); + if (!null_is_false && types[i]->isNullable()) + { + auto * nonnull = llvm::BasicBlock::Create(head->getContext(), "", head->getParent()); + returns.emplace_back(b.GetInsertBlock(), null); + b.CreateCondBr(b.CreateExtractValue(cond, {1}), join, nonnull); + b.SetInsertPoint(nonnull); + b.CreateCondBr(nativeBoolCast(b, removeNullable(types[i]), b.CreateExtractValue(cond, {0})), then, next); + } + else + { + b.CreateCondBr(nativeBoolCast(b, types[i], cond), then, next); + } + b.SetInsertPoint(then); + auto * value = nativeCast(b, types[i + 1], values[i + 1](), type); + returns.emplace_back(b.GetInsertBlock(), value); + b.CreateBr(join); + b.SetInsertPoint(next); + } + auto * value = nativeCast(b, types.back(), values.back()(), type); + returns.emplace_back(b.GetInsertBlock(), value); + b.CreateBr(join); + b.SetInsertPoint(join); + auto * phi = b.CreatePHI(toNativeType(b, type), returns.size()); + for (const auto & r : returns) + phi->addIncoming(r.second, r.first); + return phi; + } +#endif +}; + +} diff --git a/dbms/src/Functions/FunctionsConditional.cpp b/dbms/src/Functions/FunctionsConditional.cpp deleted file mode 100644 index 2b39a772f14..00000000000 --- a/dbms/src/Functions/FunctionsConditional.cpp +++ /dev/null @@ -1,337 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION; -} - -void registerFunctionsConditional(FunctionFactory & factory) -{ - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - /// These are obsolete function names. - factory.registerFunction("caseWithExpr"); - factory.registerFunction("caseWithoutExpr"); - factory.registerFunction("caseWithoutExpression"); -} - - -/// Implementation of FunctionMultiIf. - -FunctionPtr FunctionMultiIf::create(const Context & context) -{ - return std::make_shared(context); -} - -String FunctionMultiIf::getName() const -{ - return name; -} - - -void FunctionMultiIf::executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) -{ - /** We will gather values from columns in branches to result column, - * depending on values of conditions. - */ - struct Instruction - { - const IColumn * condition = nullptr; - const IColumn * source = nullptr; - - bool condition_always_true = false; - bool condition_is_nullable = false; - bool source_is_constant = false; - }; - - std::vector instructions; - instructions.reserve(args.size() / 2 + 1); - - Columns converted_columns_holder; - converted_columns_holder.reserve(instructions.size()); - - const DataTypePtr & return_type = block.getByPosition(result).type; - - for (size_t i = 0; i < args.size(); i += 2) - { - Instruction instruction; - size_t source_idx = i + 1; - - if (source_idx == args.size()) - { - /// The last, "else" branch can be treated as a branch with always true condition "else if (true)". - --source_idx; - instruction.condition_always_true = true; - } - else - { - const ColumnWithTypeAndName & cond_col = block.getByPosition(args[i]); - - /// We skip branches that are always false. - /// If we encounter a branch that is always true, we can finish. - - if (cond_col.column->onlyNull()) - continue; - - if (cond_col.column->isColumnConst()) - { - Field value = typeid_cast(*cond_col.column).getField(); - if (value.isNull()) - continue; - if (value.get() == 0) - continue; - instruction.condition_always_true = true; - } - else - { - if (cond_col.column->isColumnNullable()) - instruction.condition_is_nullable = true; - - instruction.condition = cond_col.column.get(); - } - } - - const ColumnWithTypeAndName & source_col = block.getByPosition(args[source_idx]); - if (source_col.type->equals(*return_type)) - { - instruction.source = source_col.column.get(); - } - else - { - /// Cast all columns to result type. - converted_columns_holder.emplace_back(castColumn(source_col, return_type, context)); - instruction.source = converted_columns_holder.back().get(); - } - - if (instruction.source && instruction.source->isColumnConst()) - instruction.source_is_constant = true; - - instructions.emplace_back(std::move(instruction)); - - if (instructions.back().condition_always_true) - break; - } - - size_t rows = input_rows_count; - MutableColumnPtr res = return_type->createColumn(); - - for (size_t i = 0; i < rows; ++i) - { - for (const auto & instruction : instructions) - { - bool insert = false; - - if (instruction.condition_always_true) - insert = true; - else if (!instruction.condition_is_nullable) - insert = static_cast(*instruction.condition).getData()[i]; - else - { - const ColumnNullable & condition_nullable = static_cast(*instruction.condition); - const ColumnUInt8 & condition_nested = static_cast(condition_nullable.getNestedColumn()); - const NullMap & condition_null_map = condition_nullable.getNullMapData(); - - insert = !condition_null_map[i] && condition_nested.getData()[i]; - } - - if (insert) - { - if (!instruction.source_is_constant) - res->insertFrom(*instruction.source, i); - else - res->insertFrom(static_cast(*instruction.source).getDataColumn(), 0); - - break; - } - } - } - - block.getByPosition(result).column = std::move(res); -} - -DataTypePtr FunctionMultiIf::getReturnTypeImpl(const DataTypes & args) const -{ - /// Arguments are the following: cond1, then1, cond2, then2, ... condN, thenN, else. - - auto for_conditions = [&args](auto && f) - { - size_t conditions_end = args.size() - 1; - for (size_t i = 0; i < conditions_end; i += 2) - f(args[i]); - }; - - auto for_branches = [&args](auto && f) - { - size_t branches_end = args.size(); - for (size_t i = 1; i < branches_end; i += 2) - f(args[i]); - f(args.back()); - }; - - if (!(args.size() >= 3 && args.size() % 2 == 1)) - throw Exception{"Invalid number of arguments for function " + getName(), - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - /// Conditions must be UInt8, Nullable(UInt8) or Null. If one of conditions is Nullable, the result is also Nullable. - bool have_nullable_condition = false; - - for_conditions([&](const DataTypePtr & arg) - { - const IDataType * nested_type; - if (arg->isNullable()) - { - have_nullable_condition = true; - - if (arg->onlyNull()) - return; - - const DataTypeNullable & nullable_type = static_cast(*arg); - nested_type = nullable_type.getNestedType().get(); - } - else - { - nested_type = arg.get(); - } - - if (!WhichDataType(nested_type).isUInt8()) - throw Exception{"Illegal type " + arg->getName() + " of argument (condition) " - "of function " + getName() + ". Must be UInt8.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - }); - - DataTypes types_of_branches; - types_of_branches.reserve(args.size() / 2 + 1); - - for_branches([&](const DataTypePtr & arg) - { - types_of_branches.emplace_back(arg); - }); - - DataTypePtr common_type_of_branches = getLeastSupertype(types_of_branches); - - return have_nullable_condition - ? makeNullable(common_type_of_branches) - : common_type_of_branches; -} - - -FunctionPtr FunctionCaseWithExpression::create(const Context & context_) -{ - return std::make_shared(context_); -} - -FunctionCaseWithExpression::FunctionCaseWithExpression(const Context & context_) - : context{context_} -{ -} - -String FunctionCaseWithExpression::getName() const -{ - return name; -} - -DataTypePtr FunctionCaseWithExpression::getReturnTypeImpl(const DataTypes & args) const -{ - if (!args.size()) - throw Exception{"Function " + getName() + " expects at least 1 arguments", - ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION}; - - /// See the comments in executeImpl() to understand why we actually have to - /// get the return type of a transform function. - - /// Get the types of the arrays that we pass to the transform function. - DataTypes src_array_types; - DataTypes dst_array_types; - - for (size_t i = 1; i < args.size() - 1; ++i) - ((i % 2) ? src_array_types : dst_array_types).push_back(args[i]); - - DataTypePtr src_array_type = std::make_shared(getLeastSupertype(src_array_types)); - DataTypePtr dst_array_type = std::make_shared(getLeastSupertype(dst_array_types)); - - /// Finally get the return type of the transform function. - FunctionTransform fun_transform; - ColumnsWithTypeAndName transform_args = {{nullptr, args.front(), {}}, {nullptr, src_array_type, {}}, - {nullptr, dst_array_type, {}}, {nullptr, args.back(), {}}}; - return fun_transform.getReturnType(transform_args); -} - -void FunctionCaseWithExpression::executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) -{ - if (!args.size()) - throw Exception{"Function " + getName() + " expects at least 1 arguments", - ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION}; - - /// In the following code, we turn the construction: - /// CASE expr WHEN val[0] THEN branch[0] ... WHEN val[N-1] then branch[N-1] ELSE branchN - /// into the construction transform(expr, src, dest, branchN) - /// where: - /// src = [val[0], val[1], ..., val[N-1]] - /// dst = [branch[0], ..., branch[N-1]] - /// then we perform it. - - /// Create the arrays required by the transform function. - ColumnNumbers src_array_args; - ColumnsWithTypeAndName src_array_elems; - DataTypes src_array_types; - - ColumnNumbers dst_array_args; - ColumnsWithTypeAndName dst_array_elems; - DataTypes dst_array_types; - - for (size_t i = 1; i < (args.size() - 1); ++i) - { - if (i % 2) - { - src_array_args.push_back(args[i]); - src_array_elems.push_back(block.getByPosition(args[i])); - src_array_types.push_back(block.getByPosition(args[i]).type); - } - else - { - dst_array_args.push_back(args[i]); - dst_array_elems.push_back(block.getByPosition(args[i])); - dst_array_types.push_back(block.getByPosition(args[i]).type); - } - } - - DataTypePtr src_array_type = std::make_shared(getLeastSupertype(src_array_types)); - DataTypePtr dst_array_type = std::make_shared(getLeastSupertype(dst_array_types)); - - Block temp_block = block; - - size_t src_array_pos = temp_block.columns(); - temp_block.insert({nullptr, src_array_type, ""}); - - size_t dst_array_pos = temp_block.columns(); - temp_block.insert({nullptr, dst_array_type, ""}); - - auto fun_array = FunctionFactory::instance().get("array", context); - - fun_array->build(src_array_elems)->execute(temp_block, src_array_args, src_array_pos, input_rows_count); - fun_array->build(dst_array_elems)->execute(temp_block, dst_array_args, dst_array_pos, input_rows_count); - - /// Execute transform. - FunctionTransform fun_transform; - - ColumnNumbers transform_args{args.front(), src_array_pos, dst_array_pos, args.back()}; - fun_transform.execute(temp_block, transform_args, result, input_rows_count); - - /// Put the result into the original block. - block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column); -} - -} diff --git a/dbms/src/Functions/caseWithExpression.cpp b/dbms/src/Functions/caseWithExpression.cpp new file mode 100644 index 00000000000..4af0cdb5161 --- /dev/null +++ b/dbms/src/Functions/caseWithExpression.cpp @@ -0,0 +1,128 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION; +} + +/// Implements the CASE construction when it is +/// provided an expression. Users should not call this function. +class FunctionCaseWithExpression : public IFunction +{ +public: + static constexpr auto name = "caseWithExpression"; + static FunctionPtr create(const Context & context_) { return std::make_shared(context_); } + +public: + FunctionCaseWithExpression(const Context & context_) : context(context_) {} + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + String getName() const override { return name; } + + DataTypePtr getReturnTypeImpl(const DataTypes & args) const override + { + if (!args.size()) + throw Exception{"Function " + getName() + " expects at least 1 arguments", + ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION}; + + /// See the comments in executeImpl() to understand why we actually have to + /// get the return type of a transform function. + + /// Get the types of the arrays that we pass to the transform function. + DataTypes dst_array_types; + + for (size_t i = 2; i < args.size() - 1; i += 2) + dst_array_types.push_back(args[i]); + + return getLeastSupertype(dst_array_types); + } + + void executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) override + { + if (!args.size()) + throw Exception{"Function " + getName() + " expects at least 1 argument", + ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION}; + + /// In the following code, we turn the construction: + /// CASE expr WHEN val[0] THEN branch[0] ... WHEN val[N-1] then branch[N-1] ELSE branchN + /// into the construction transform(expr, src, dest, branchN) + /// where: + /// src = [val[0], val[1], ..., val[N-1]] + /// dst = [branch[0], ..., branch[N-1]] + /// then we perform it. + + /// Create the arrays required by the transform function. + ColumnNumbers src_array_args; + ColumnsWithTypeAndName src_array_elems; + DataTypes src_array_types; + + ColumnNumbers dst_array_args; + ColumnsWithTypeAndName dst_array_elems; + DataTypes dst_array_types; + + for (size_t i = 1; i < (args.size() - 1); ++i) + { + if (i % 2) + { + src_array_args.push_back(args[i]); + src_array_elems.push_back(block.getByPosition(args[i])); + src_array_types.push_back(block.getByPosition(args[i]).type); + } + else + { + dst_array_args.push_back(args[i]); + dst_array_elems.push_back(block.getByPosition(args[i])); + dst_array_types.push_back(block.getByPosition(args[i]).type); + } + } + + DataTypePtr src_array_type = std::make_shared(getLeastSupertype(src_array_types)); + DataTypePtr dst_array_type = std::make_shared(getLeastSupertype(dst_array_types)); + + Block temp_block = block; + + size_t src_array_pos = temp_block.columns(); + temp_block.insert({nullptr, src_array_type, ""}); + + size_t dst_array_pos = temp_block.columns(); + temp_block.insert({nullptr, dst_array_type, ""}); + + auto fun_array = FunctionFactory::instance().get("array", context); + + fun_array->build(src_array_elems)->execute(temp_block, src_array_args, src_array_pos, input_rows_count); + fun_array->build(dst_array_elems)->execute(temp_block, dst_array_args, dst_array_pos, input_rows_count); + + /// Execute transform. + ColumnNumbers transform_args{args.front(), src_array_pos, dst_array_pos, args.back()}; + FunctionFactory::instance().get("transform", context)->build( + ext::map(transform_args, [&](auto i){ return temp_block.getByPosition(i); })) + ->execute(temp_block, transform_args, result, input_rows_count); + + /// Put the result into the original block. + block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column); + } + +private: + const Context & context; +}; + +void registerFunctionCaseWithExpression(FunctionFactory & factory) +{ + factory.registerFunction(); + + /// These are obsolete function names. + factory.registerFunction("caseWithExpr"); +} + +} + + + + diff --git a/dbms/src/Functions/FunctionsConditional.h b/dbms/src/Functions/if.cpp similarity index 90% rename from dbms/src/Functions/FunctionsConditional.h rename to dbms/src/Functions/if.cpp index 11364e6ed12..5879e45c810 100644 --- a/dbms/src/Functions/FunctionsConditional.h +++ b/dbms/src/Functions/if.cpp @@ -1,5 +1,3 @@ -#pragma once - #include #include #include @@ -7,7 +5,6 @@ #include #include #include -#include #include #include #include @@ -23,6 +20,8 @@ #include #include #include +#include +#include namespace DB @@ -165,64 +164,6 @@ public: }; -template -class FunctionIfBase : public IFunction -{ -#if USE_EMBEDDED_COMPILER -public: - bool isCompilableImpl(const DataTypes & types) const override - { - for (const auto & type : types) - if (!isCompilableType(removeNullable(type))) - return false; - return true; - } - - llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const DataTypes & types, ValuePlaceholders values) const override - { - auto & b = static_cast &>(builder); - auto type = getReturnTypeImpl(types); - llvm::Value * null = nullptr; - if (!null_is_false && type->isNullable()) - null = b.CreateInsertValue(llvm::Constant::getNullValue(toNativeType(b, type)), b.getTrue(), {1}); - auto * head = b.GetInsertBlock(); - auto * join = llvm::BasicBlock::Create(head->getContext(), "", head->getParent()); - std::vector> returns; - for (size_t i = 0; i + 1 < types.size(); i += 2) - { - auto * then = llvm::BasicBlock::Create(head->getContext(), "", head->getParent()); - auto * next = llvm::BasicBlock::Create(head->getContext(), "", head->getParent()); - auto * cond = values[i](); - if (!null_is_false && types[i]->isNullable()) - { - auto * nonnull = llvm::BasicBlock::Create(head->getContext(), "", head->getParent()); - returns.emplace_back(b.GetInsertBlock(), null); - b.CreateCondBr(b.CreateExtractValue(cond, {1}), join, nonnull); - b.SetInsertPoint(nonnull); - b.CreateCondBr(nativeBoolCast(b, removeNullable(types[i]), b.CreateExtractValue(cond, {0})), then, next); - } - else - { - b.CreateCondBr(nativeBoolCast(b, types[i], cond), then, next); - } - b.SetInsertPoint(then); - auto * value = nativeCast(b, types[i + 1], values[i + 1](), type); - returns.emplace_back(b.GetInsertBlock(), value); - b.CreateBr(join); - b.SetInsertPoint(next); - } - auto * value = nativeCast(b, types.back(), values.back()(), type); - returns.emplace_back(b.GetInsertBlock(), value); - b.CreateBr(join); - b.SetInsertPoint(join); - auto * phi = b.CreatePHI(toNativeType(b, type), returns.size()); - for (const auto & r : returns) - phi->addIncoming(r.second, r.first); - return phi; - } -#endif -}; - class FunctionIf : public FunctionIfBase { public: @@ -993,60 +934,9 @@ public: } }; - -/// Function multiIf, which generalizes the function if. -/// -/// Syntax: multiIf(cond_1, then_1, ..., cond_N, then_N, else) -/// where N >= 1. -/// -/// For all 1 <= i <= N, "cond_i" has type UInt8. -/// Types of all the branches "then_i" and "else" are either of the following: -/// - numeric types for which there exists a common type; -/// - dates; -/// - dates with time; -/// - strings; -/// - arrays of such types. -/// -/// Additionally the arguments, conditions or branches, support nullable types -/// and the NULL value, with a NULL condition treated as false. -class FunctionMultiIf final : public FunctionIfBase +void registerFunctionIf(FunctionFactory & factory) { -public: - static constexpr auto name = "multiIf"; - static FunctionPtr create(const Context & context); - FunctionMultiIf(const Context & context) : context(context) {} - -public: - String getName() const override; - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - bool useDefaultImplementationForNulls() const override { return false; } - DataTypePtr getReturnTypeImpl(const DataTypes & args) const override; - void executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) override; - -private: - const Context & context; -}; - - -/// Implements the CASE construction when it is -/// provided an expression. Users should not call this function. -class FunctionCaseWithExpression : public IFunction -{ -public: - static constexpr auto name = "caseWithExpression"; - static FunctionPtr create(const Context & context_); - -public: - FunctionCaseWithExpression(const Context & context_); - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - String getName() const override; - DataTypePtr getReturnTypeImpl(const DataTypes & args) const override; - void executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) override; - -private: - const Context & context; -}; + factory.registerFunction(); +} } diff --git a/dbms/src/Functions/multiIf.cpp b/dbms/src/Functions/multiIf.cpp new file mode 100644 index 00000000000..205bcff954f --- /dev/null +++ b/dbms/src/Functions/multiIf.cpp @@ -0,0 +1,249 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +/// Function multiIf, which generalizes the function if. +/// +/// Syntax: multiIf(cond_1, then_1, ..., cond_N, then_N, else) +/// where N >= 1. +/// +/// For all 1 <= i <= N, "cond_i" has type UInt8. +/// Types of all the branches "then_i" and "else" are either of the following: +/// - numeric types for which there exists a common type; +/// - dates; +/// - dates with time; +/// - strings; +/// - arrays of such types. +/// +/// Additionally the arguments, conditions or branches, support nullable types +/// and the NULL value, with a NULL condition treated as false. +class FunctionMultiIf final : public FunctionIfBase +{ +public: + static constexpr auto name = "multiIf"; + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + FunctionMultiIf(const Context & context) : context(context) {} + +public: + String getName() const override { return name; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForNulls() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & args) const override + { + /// Arguments are the following: cond1, then1, cond2, then2, ... condN, thenN, else. + + auto for_conditions = [&args](auto && f) + { + size_t conditions_end = args.size() - 1; + for (size_t i = 0; i < conditions_end; i += 2) + f(args[i]); + }; + + auto for_branches = [&args](auto && f) + { + size_t branches_end = args.size(); + for (size_t i = 1; i < branches_end; i += 2) + f(args[i]); + f(args.back()); + }; + + if (!(args.size() >= 3 && args.size() % 2 == 1)) + throw Exception{"Invalid number of arguments for function " + getName(), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + + /// Conditions must be UInt8, Nullable(UInt8) or Null. If one of conditions is Nullable, the result is also Nullable. + bool have_nullable_condition = false; + + for_conditions([&](const DataTypePtr & arg) + { + const IDataType * nested_type; + if (arg->isNullable()) + { + have_nullable_condition = true; + + if (arg->onlyNull()) + return; + + const DataTypeNullable & nullable_type = static_cast(*arg); + nested_type = nullable_type.getNestedType().get(); + } + else + { + nested_type = arg.get(); + } + + if (!WhichDataType(nested_type).isUInt8()) + throw Exception{"Illegal type " + arg->getName() + " of argument (condition) " + "of function " + getName() + ". Must be UInt8.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + }); + + DataTypes types_of_branches; + types_of_branches.reserve(args.size() / 2 + 1); + + for_branches([&](const DataTypePtr & arg) + { + types_of_branches.emplace_back(arg); + }); + + DataTypePtr common_type_of_branches = getLeastSupertype(types_of_branches); + + return have_nullable_condition + ? makeNullable(common_type_of_branches) + : common_type_of_branches; + } + + void executeImpl(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) override + { + /** We will gather values from columns in branches to result column, + * depending on values of conditions. + */ + struct Instruction + { + const IColumn * condition = nullptr; + const IColumn * source = nullptr; + + bool condition_always_true = false; + bool condition_is_nullable = false; + bool source_is_constant = false; + }; + + std::vector instructions; + instructions.reserve(args.size() / 2 + 1); + + Columns converted_columns_holder; + converted_columns_holder.reserve(instructions.size()); + + const DataTypePtr & return_type = block.getByPosition(result).type; + + for (size_t i = 0; i < args.size(); i += 2) + { + Instruction instruction; + size_t source_idx = i + 1; + + if (source_idx == args.size()) + { + /// The last, "else" branch can be treated as a branch with always true condition "else if (true)". + --source_idx; + instruction.condition_always_true = true; + } + else + { + const ColumnWithTypeAndName & cond_col = block.getByPosition(args[i]); + + /// We skip branches that are always false. + /// If we encounter a branch that is always true, we can finish. + + if (cond_col.column->onlyNull()) + continue; + + if (cond_col.column->isColumnConst()) + { + Field value = typeid_cast(*cond_col.column).getField(); + if (value.isNull()) + continue; + if (value.get() == 0) + continue; + instruction.condition_always_true = true; + } + else + { + if (cond_col.column->isColumnNullable()) + instruction.condition_is_nullable = true; + + instruction.condition = cond_col.column.get(); + } + } + + const ColumnWithTypeAndName & source_col = block.getByPosition(args[source_idx]); + if (source_col.type->equals(*return_type)) + { + instruction.source = source_col.column.get(); + } + else + { + /// Cast all columns to result type. + converted_columns_holder.emplace_back(castColumn(source_col, return_type, context)); + instruction.source = converted_columns_holder.back().get(); + } + + if (instruction.source && instruction.source->isColumnConst()) + instruction.source_is_constant = true; + + instructions.emplace_back(std::move(instruction)); + + if (instructions.back().condition_always_true) + break; + } + + size_t rows = input_rows_count; + MutableColumnPtr res = return_type->createColumn(); + + for (size_t i = 0; i < rows; ++i) + { + for (const auto & instruction : instructions) + { + bool insert = false; + + if (instruction.condition_always_true) + insert = true; + else if (!instruction.condition_is_nullable) + insert = static_cast(*instruction.condition).getData()[i]; + else + { + const ColumnNullable & condition_nullable = static_cast(*instruction.condition); + const ColumnUInt8 & condition_nested = static_cast(condition_nullable.getNestedColumn()); + const NullMap & condition_null_map = condition_nullable.getNullMapData(); + + insert = !condition_null_map[i] && condition_nested.getData()[i]; + } + + if (insert) + { + if (!instruction.source_is_constant) + res->insertFrom(*instruction.source, i); + else + res->insertFrom(static_cast(*instruction.source).getDataColumn(), 0); + + break; + } + } + } + + block.getByPosition(result).column = std::move(res); + } + +private: + const Context & context; +}; + +void registerFunctionMultiIf(FunctionFactory & factory) +{ + factory.registerFunction(); + + /// These are obsolete function names. + factory.registerFunction("caseWithoutExpr"); + factory.registerFunction("caseWithoutExpression"); +} + +} + + + diff --git a/dbms/src/Functions/registerFunctionsConditional.cpp b/dbms/src/Functions/registerFunctionsConditional.cpp new file mode 100644 index 00000000000..782399c1d4b --- /dev/null +++ b/dbms/src/Functions/registerFunctionsConditional.cpp @@ -0,0 +1,20 @@ +namespace DB +{ + +class FunctionFactory; + +void registerFunctionIf(FunctionFactory & factory); +void registerFunctionMultiIf(FunctionFactory & factory); +void registerFunctionCaseWithExpression(FunctionFactory & factory); + +void registerFunctionsConditional(FunctionFactory & factory) +{ + registerFunctionIf(factory); + registerFunctionMultiIf(factory); + registerFunctionCaseWithExpression(factory); +} + +} + + + From 2930da2880e0a631ea0e00bf4ea515e32c7c8f78 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Dec 2018 05:52:49 +0300 Subject: [PATCH 58/69] Every function in its own file, part 8 [#CLICKHOUSE-2] --- dbms/src/Functions/FunctionsTransform.cpp | 12 ------------ dbms/src/Functions/registerFunction.cpp.in | 14 -------------- dbms/src/Functions/registerFunction.h.in | 10 ---------- dbms/src/Functions/registerFunctions.cpp | 4 ++-- dbms/src/Functions/registerFunctions_area.cpp.in | 14 -------------- .../{FunctionsTransform.h => transform.cpp} | 8 ++++++-- 6 files changed, 8 insertions(+), 54 deletions(-) delete mode 100644 dbms/src/Functions/FunctionsTransform.cpp delete mode 100644 dbms/src/Functions/registerFunction.cpp.in delete mode 100644 dbms/src/Functions/registerFunction.h.in delete mode 100644 dbms/src/Functions/registerFunctions_area.cpp.in rename dbms/src/Functions/{FunctionsTransform.h => transform.cpp} (99%) diff --git a/dbms/src/Functions/FunctionsTransform.cpp b/dbms/src/Functions/FunctionsTransform.cpp deleted file mode 100644 index 180a9532c8e..00000000000 --- a/dbms/src/Functions/FunctionsTransform.cpp +++ /dev/null @@ -1,12 +0,0 @@ -#include -#include - -namespace DB -{ - -void registerFunctionsTransform(FunctionFactory & factory) -{ - factory.registerFunction(); -} - -} diff --git a/dbms/src/Functions/registerFunction.cpp.in b/dbms/src/Functions/registerFunction.cpp.in deleted file mode 100644 index 71bab6292d5..00000000000 --- a/dbms/src/Functions/registerFunction.cpp.in +++ /dev/null @@ -1,14 +0,0 @@ -#include "register@FUNCTION@.h" - -#include -#include - -namespace DB -{ - -void register@FUNCTION@(FunctionFactory & factory) -{ - factory.registerFunction<@FUNCTION@>(); -} - -} diff --git a/dbms/src/Functions/registerFunction.h.in b/dbms/src/Functions/registerFunction.h.in deleted file mode 100644 index 11ae2dc967e..00000000000 --- a/dbms/src/Functions/registerFunction.h.in +++ /dev/null @@ -1,10 +0,0 @@ -#pragma once - -namespace DB -{ - -class FunctionFactory; - -void register@FUNCTION@(FunctionFactory & factory); - -} diff --git a/dbms/src/Functions/registerFunctions.cpp b/dbms/src/Functions/registerFunctions.cpp index 292f6f608df..41164e5e65e 100644 --- a/dbms/src/Functions/registerFunctions.cpp +++ b/dbms/src/Functions/registerFunctions.cpp @@ -36,10 +36,10 @@ void registerFunctionsStringSearch(FunctionFactory &); void registerFunctionsURL(FunctionFactory &); void registerFunctionsVisitParam(FunctionFactory &); void registerFunctionsMath(FunctionFactory &); -void registerFunctionsTransform(FunctionFactory &); void registerFunctionsGeo(FunctionFactory &); void registerFunctionsNull(FunctionFactory &); void registerFunctionsFindCluster(FunctionFactory &); +void registerFunctionTransform(FunctionFactory &); #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -75,10 +75,10 @@ void registerFunctions() registerFunctionsURL(factory); registerFunctionsVisitParam(factory); registerFunctionsMath(factory); - registerFunctionsTransform(factory); registerFunctionsGeo(factory); registerFunctionsNull(factory); registerFunctionsFindCluster(factory); + registerFunctionTransform(factory); #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/dbms/src/Functions/registerFunctions_area.cpp.in b/dbms/src/Functions/registerFunctions_area.cpp.in deleted file mode 100644 index cd727885d57..00000000000 --- a/dbms/src/Functions/registerFunctions_area.cpp.in +++ /dev/null @@ -1,14 +0,0 @@ -#include -#include - -@REGISTER_HEADERS@ - -namespace DB -{ - -void registerFunctions@FUNCTION_AREA@(FunctionFactory & factory) -{ -@REGISTER_FUNCTIONS@ -} - -} diff --git a/dbms/src/Functions/FunctionsTransform.h b/dbms/src/Functions/transform.cpp similarity index 99% rename from dbms/src/Functions/FunctionsTransform.h rename to dbms/src/Functions/transform.cpp index f7096ef38dc..83aa379a3ba 100644 --- a/dbms/src/Functions/FunctionsTransform.h +++ b/dbms/src/Functions/transform.cpp @@ -1,5 +1,3 @@ -#pragma once - #include #include #include @@ -15,6 +13,7 @@ #include #include #include +#include #include @@ -836,4 +835,9 @@ private: } }; +void registerFunctionTransform(FunctionFactory & factory) +{ + factory.registerFunction(); +} + } From 3cc0829cc1b9eef07af23c06fabb7256651167d6 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 2 Dec 2018 19:00:23 +0800 Subject: [PATCH 59/69] Function execution with dry runs This commit prevents stateful functions like rowNumberInAllBlocks from being modified in getHeader() calls. --- .../ExpressionBlockInputStream.cpp | 2 +- dbms/src/Functions/FunctionsMiscellaneous.h | 4 +- dbms/src/Functions/IFunction.cpp | 28 ++++++++------ dbms/src/Functions/IFunction.h | 23 ++++++++---- dbms/src/Functions/rowNumberInAllBlocks.cpp | 6 +++ dbms/src/Interpreters/ExpressionActions.cpp | 14 +++---- dbms/src/Interpreters/ExpressionActions.h | 4 +- .../00799_function_dry_run.reference | 9 +++++ .../0_stateless/00799_function_dry_run.sql | 37 +++++++++++++++++++ 9 files changed, 96 insertions(+), 31 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00799_function_dry_run.reference create mode 100644 dbms/tests/queries/0_stateless/00799_function_dry_run.sql diff --git a/dbms/src/DataStreams/ExpressionBlockInputStream.cpp b/dbms/src/DataStreams/ExpressionBlockInputStream.cpp index f1840acd023..3eb559abaad 100644 --- a/dbms/src/DataStreams/ExpressionBlockInputStream.cpp +++ b/dbms/src/DataStreams/ExpressionBlockInputStream.cpp @@ -27,7 +27,7 @@ Block ExpressionBlockInputStream::getTotals() Block ExpressionBlockInputStream::getHeader() const { Block res = children.back()->getHeader(); - expression->execute(res); + expression->execute(res, true); return res; } diff --git a/dbms/src/Functions/FunctionsMiscellaneous.h b/dbms/src/Functions/FunctionsMiscellaneous.h index ee3a92bb6b6..6803e16abbe 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.h +++ b/dbms/src/Functions/FunctionsMiscellaneous.h @@ -34,7 +34,7 @@ public: return std::const_pointer_cast(shared_from_this()); } - void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/, bool) override { Block expr_block; for (size_t i = 0; i < arguments.size(); ++i) @@ -118,7 +118,7 @@ public: return std::const_pointer_cast(shared_from_this()); } - void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool) override { ColumnsWithTypeAndName columns; columns.reserve(arguments.size()); diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 7d6d09624d2..358fbfc425b 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -267,7 +267,7 @@ bool allArgumentsAreConstants(const Block & block, const ColumnNumbers & args) } bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result, - size_t input_rows_count) + size_t input_rows_count, bool dry_run) { ColumnNumbers arguments_to_remain_constants = getArgumentsThatAreAlwaysConstant(); @@ -312,7 +312,7 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo for (size_t i = 0; i < arguments_size; ++i) temporary_argument_numbers[i] = i; - executeWithoutLowCardinalityColumns(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows()); + executeWithoutLowCardinalityColumns(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows(), dry_run); ColumnPtr result_column; /// extremely rare case, when we have function with completely const arguments @@ -328,7 +328,7 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const ColumnNumbers & args, size_t result, - size_t input_rows_count) + size_t input_rows_count, bool dry_run) { if (args.empty() || !useDefaultImplementationForNulls()) return false; @@ -344,7 +344,7 @@ bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const Co if (null_presence.has_nullable) { Block temporary_block = createBlockWithNestedColumns(block, args, result); - executeWithoutLowCardinalityColumns(temporary_block, args, result, temporary_block.rows()); + executeWithoutLowCardinalityColumns(temporary_block, args, result, temporary_block.rows(), dry_run); block.getByPosition(result).column = wrapInNullable(temporary_block.getByPosition(result).column, block, args, result, input_rows_count); return true; @@ -353,15 +353,19 @@ bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const Co return false; } -void PreparedFunctionImpl::executeWithoutLowCardinalityColumns(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) +void PreparedFunctionImpl::executeWithoutLowCardinalityColumns(Block & block, const ColumnNumbers & args, size_t result, + size_t input_rows_count, bool dry_run) { - if (defaultImplementationForConstantArguments(block, args, result, input_rows_count)) + if (defaultImplementationForConstantArguments(block, args, result, input_rows_count, dry_run)) return; - if (defaultImplementationForNulls(block, args, result, input_rows_count)) + if (defaultImplementationForNulls(block, args, result, input_rows_count, dry_run)) return; - executeImpl(block, args, result, input_rows_count); + if (dry_run) + executeImplDryRun(block, args, result, input_rows_count); + else + executeImpl(block, args, result, input_rows_count); } static const ColumnLowCardinality * findLowCardinalityArgument(const Block & block, const ColumnNumbers & args) @@ -441,7 +445,7 @@ static void convertLowCardinalityColumnsToFull(Block & block, const ColumnNumber } } -void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count) +void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count, bool dry_run) { if (useDefaultImplementationForLowCardinalityColumns()) { @@ -477,7 +481,7 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si ColumnPtr indexes = replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( block_without_low_cardinality, args, can_be_executed_on_default_arguments); - executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, block_without_low_cardinality.rows()); + executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, block_without_low_cardinality.rows(), dry_run); auto & keys = block_without_low_cardinality.safeGetByPosition(result).column; if (auto full_column = keys->convertToFullColumnIfConst()) @@ -511,12 +515,12 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si else { convertLowCardinalityColumnsToFull(block_without_low_cardinality, args); - executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, input_rows_count); + executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, input_rows_count, dry_run); res.column = block_without_low_cardinality.safeGetByPosition(result).column; } } else - executeWithoutLowCardinalityColumns(block, args, result, input_rows_count); + executeWithoutLowCardinalityColumns(block, args, result, input_rows_count, dry_run); } void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) const diff --git a/dbms/src/Functions/IFunction.h b/dbms/src/Functions/IFunction.h index 4028a61ef7b..547229ecae1 100644 --- a/dbms/src/Functions/IFunction.h +++ b/dbms/src/Functions/IFunction.h @@ -40,7 +40,7 @@ public: /// Get the main function name. virtual String getName() const = 0; - virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) = 0; + virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count, bool dry_run) = 0; }; using PreparedFunctionPtr = std::shared_ptr; @@ -52,7 +52,7 @@ using PreparedFunctionLowCardinalityResultCachePtr = std::shared_ptrexecute(block, arguments, result, input_rows_count); + return prepare(block, arguments, result)->execute(block, arguments, result, input_rows_count, dry_run); } #if USE_EMBEDDED_COMPILER @@ -330,6 +334,7 @@ public: bool canBeExecutedOnLowCardinalityDictionary() const override { return isDeterministicInScopeOfQuery(); } using PreparedFunctionImpl::execute; + using PreparedFunctionImpl::executeImplDryRun; using FunctionBuilderImpl::getReturnTypeImpl; using FunctionBuilderImpl::getLambdaArgumentTypesImpl; using FunctionBuilderImpl::getReturnType; @@ -404,6 +409,10 @@ protected: { return function->executeImpl(block, arguments, result, input_rows_count); } + void executeImplDryRun(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) final + { + return function->executeImplDryRun(block, arguments, result, input_rows_count); + } bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); } bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); } bool useDefaultImplementationForLowCardinalityColumns() const final { return function->useDefaultImplementationForLowCardinalityColumns(); } diff --git a/dbms/src/Functions/rowNumberInAllBlocks.cpp b/dbms/src/Functions/rowNumberInAllBlocks.cpp index 0ee2ece13d1..cce7681cf9c 100644 --- a/dbms/src/Functions/rowNumberInAllBlocks.cpp +++ b/dbms/src/Functions/rowNumberInAllBlocks.cpp @@ -44,6 +44,12 @@ public: return std::make_shared(); } + void executeImplDryRun(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + auto column = ColumnUInt64::create(input_rows_count); + block.getByPosition(result).column = std::move(column); + } + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override { size_t current_row_number = rows.fetch_add(input_rows_count); diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index b1fab40a654..a33261ef385 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -204,7 +204,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) /// so we don't want to unfold non deterministic functions if (all_const && function_base->isSuitableForConstantFolding() && (!compile_expressions || function_base->isDeterministic())) { - function->execute(sample_block, arguments, result_position, sample_block.rows()); + function->execute(sample_block, arguments, result_position, sample_block.rows(), true); /// If the result is not a constant, just in case, we will consider the result as unknown. ColumnWithTypeAndName & col = sample_block.safeGetByPosition(result_position); @@ -325,7 +325,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) } -void ExpressionAction::execute(Block & block) const +void ExpressionAction::execute(Block & block, bool dry_run) const { size_t input_rows_count = block.rows(); @@ -355,7 +355,7 @@ void ExpressionAction::execute(Block & block) const ProfileEvents::increment(ProfileEvents::FunctionExecute); if (is_function_compiled) ProfileEvents::increment(ProfileEvents::CompiledFunctionExecute); - function->execute(block, arguments, num_columns_without_result, input_rows_count); + function->execute(block, arguments, num_columns_without_result, input_rows_count, dry_run); break; } @@ -383,7 +383,7 @@ void ExpressionAction::execute(Block & block) const Block tmp_block{src_col, {{}, src_col.type, {}}}; - function_builder->build({src_col})->execute(tmp_block, {0}, 1, src_col.column->size()); + function_builder->build({src_col})->execute(tmp_block, {0}, 1, src_col.column->size(), dry_run); non_empty_array_columns[name] = tmp_block.safeGetByPosition(1).column; } @@ -492,7 +492,7 @@ void ExpressionAction::execute(Block & block) const void ExpressionAction::executeOnTotals(Block & block) const { if (type != JOIN) - execute(block); + execute(block, false); else join->joinTotals(block); } @@ -704,11 +704,11 @@ bool ExpressionActions::popUnusedArrayJoin(const Names & required_columns, Expre return true; } -void ExpressionActions::execute(Block & block) const +void ExpressionActions::execute(Block & block, bool dry_run) const { for (const auto & action : actions) { - action.execute(block); + action.execute(block, dry_run); checkLimits(block); } } diff --git a/dbms/src/Interpreters/ExpressionActions.h b/dbms/src/Interpreters/ExpressionActions.h index 781134dbeb2..67b42febe16 100644 --- a/dbms/src/Interpreters/ExpressionActions.h +++ b/dbms/src/Interpreters/ExpressionActions.h @@ -136,7 +136,7 @@ private: friend class ExpressionActions; void prepare(Block & sample_block, const Settings & settings); - void execute(Block & block) const; + void execute(Block & block, bool dry_run) const; void executeOnTotals(Block & block) const; }; @@ -217,7 +217,7 @@ public: const NamesAndTypesList & getRequiredColumnsWithTypes() const { return input_columns; } /// Execute the expression on the block. The block must contain all the columns returned by getRequiredColumns. - void execute(Block & block) const; + void execute(Block & block, bool dry_run = false) const; /** Execute the expression on the block of total values. * Almost the same as `execute`. The difference is only when JOIN is executed. diff --git a/dbms/tests/queries/0_stateless/00799_function_dry_run.reference b/dbms/tests/queries/0_stateless/00799_function_dry_run.reference new file mode 100644 index 00000000000..35cebe7569a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00799_function_dry_run.reference @@ -0,0 +1,9 @@ +0.3 2018-11-19 13:00:00 \N +0.3 2018-11-19 13:05:00 \N +0.4 2018-11-19 13:10:00 1 +0.5 2018-11-19 13:15:00 1.2 +0.6 2018-11-19 13:15:00 1.5 +0.7 2018-11-19 13:20:00 1.8 +0.8 2018-11-19 13:25:00 2.1 +0.9 2018-11-19 13:25:00 2.4 +0.5 2018-11-19 13:30:00 2.2 diff --git a/dbms/tests/queries/0_stateless/00799_function_dry_run.sql b/dbms/tests/queries/0_stateless/00799_function_dry_run.sql new file mode 100644 index 00000000000..ac472c317d0 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00799_function_dry_run.sql @@ -0,0 +1,37 @@ +-- https://stackoverflow.com/questions/53416531/clickhouse-moving-average + +DROP TABLE IF EXISTS test.bm; + +USE test; + +CREATE TABLE bm (amount float, business_dttm DateTime) engine Log; + +INSERT INTO bm VALUES (0.3,'2018-11-19 13:00:00'), (0.3,'2018-11-19 13:05:00'), (0.4,'2018-11-19 13:10:00'), (0.5,'2018-11-19 13:15:00'), (0.6,'2018-11-19 13:15:00'), (0.7,'2018-11-19 13:20:00'), (0.8,'2018-11-19 13:25:00'), (0.9,'2018-11-19 13:25:00'), (0.5,'2018-11-19 13:30:00'); + +WITH + ( + SELECT arrayCumSum(groupArray(amount)) + FROM + ( + SELECT + amount + FROM bm + ORDER BY business_dttm + ) + ) AS arr, + 1 + rowNumberInAllBlocks() AS id, + 3 AS window_size +SELECT + amount, + business_dttm, + if(id < window_size, NULL, round(arr[id] - arr[id - window_size], 4)) AS moving_sum +FROM +( + SELECT + amount, + business_dttm + FROM bm + ORDER BY business_dttm +); + +DROP TABLE test.bm; From d35ba8b80b5f265f420ba97cbdae9ce9e6fcd2d6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Dec 2018 20:56:23 +0300 Subject: [PATCH 60/69] Removed outdated suppressions file for PVS-Studio [#CLICKHOUSE-2] --- suppress_base.json | 9917 -------------------------------------------- 1 file changed, 9917 deletions(-) delete mode 100644 suppress_base.json diff --git a/suppress_base.json b/suppress_base.json deleted file mode 100644 index 9a68c58fa55..00000000000 --- a/suppress_base.json +++ /dev/null @@ -1,9917 +0,0 @@ -{ - "version": 1, - "warnings": [ - { - "CodeCurrent": 1403077733, - "CodeNext": 951341, - "CodePrev": 1270846060, - "ErrorCode": "V550", - "FileName": "preciseExp10.c", - "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(y) < Epsilon." - }, - { - "CodeCurrent": 1403077733, - "CodeNext": 3854685, - "CodePrev": 1391368717, - "ErrorCode": "V550", - "FileName": "preciseExp10.c", - "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(y) < Epsilon." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "shift10.cpp", - "Message": "A code fragment from 'shift_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1360858873, - "CodeNext": 0, - "CodePrev": 864691173, - "ErrorCode": "V524", - "FileName": "DateLUTImpl.h", - "Message": "It is odd that the body of 'getValues' function is fully equivalent to the body of 'find' function." - }, - { - "CodeCurrent": 3167014141, - "CodeNext": 1528350630, - "CodePrev": 0, - "ErrorCode": "V524", - "FileName": "DateLUTImpl.h", - "Message": "It is odd that the body of 'toDate' function is fully equivalent to the body of 'fromDayNum' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DateLUTImpl.cpp", - "Message": "A code fragment from 'DateLUTImpl.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "demangle.cpp", - "Message": "A code fragment from 'demangle.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ThreadPool.cpp", - "Message": "A code fragment from 'ThreadPool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 18396, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "JSON.h", - "Message": "The 'JSON' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 1301811264, - "CodeNext": 123, - "CodePrev": 44642, - "ErrorCode": "V524", - "FileName": "JSON.cpp", - "Message": "It is odd that the body of 'get' function is fully equivalent to the body of 'getName' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "JSON.cpp", - "Message": "A code fragment from 'JSON.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DateLUT.cpp", - "Message": "A code fragment from 'DateLUT.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SetTerminalEcho.cpp", - "Message": "A code fragment from 'SetTerminalEcho.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "date_lut2.cpp", - "Message": "A code fragment from 'date_lut_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "date_lut3.cpp", - "Message": "A code fragment from 'date_lut_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "gtest_strong_typedef.cpp", - "Message": "A code fragment from 'gtest_strong_typedef.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "date_lut4.cpp", - "Message": "A code fragment from 'date_lut_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "dump_variable.cpp", - "Message": "A code fragment from 'dump_variable.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "date_lut_init.cpp", - "Message": "A code fragment from 'date_lut_init.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "gtest_json_test.cpp", - "Message": "A code fragment from 'gtest_json_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "multi_version.cpp", - "Message": "A code fragment from 'multi_version.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "date_lut_default_timezone.cpp", - "Message": "A code fragment from 'date_lut_default_timezone.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "LevelFilterChannel.cpp", - "Message": "A code fragment from 'LevelFilterChannel.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SessionPoolHelpers.cpp", - "Message": "A code fragment from 'SessionPoolHelpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 123, - "CodeNext": 2077384594, - "CodePrev": 2854941240, - "ErrorCode": "V565", - "FileName": "BaseDaemon.h", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "GraphiteWriter.cpp", - "Message": "A code fragment from 'GraphiteWriter.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3676031242, - "CodeNext": 3056897128, - "CodePrev": 1102711319, - "ErrorCode": "V707", - "FileName": "lgamma.c", - "Message": "Giving short names to global variables is considered to be bad practice. It is suggested to rename 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'a_', 'tc', 'tf', 'tt', 't_', 't_', 't_', 't_', 't_', 't_', 't_', 't_', 't_', 't_', 'u_', 'u_', 'u_', 'u_', 'u_', 'u_', 'v_', 'v_', 'v_', 'v_', 'v_', 's_', 's_', 's_', 's_', 's_', 's_', 's_', 'r_', 'r_', 'r_', 'r_', 'r_', 'r_', 'w_', 'w_', 'w_', 'w_', 'w_', 'w_', 'w_' variables." - }, - { - "CodeCurrent": 196849995, - "CodeNext": 2821665, - "CodePrev": 257609, - "ErrorCode": "V550", - "FileName": "lgamma.c", - "Message": "An odd precise comparison: t == _._. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 2367177615, - "CodeNext": 1620191745, - "CodePrev": 47045, - "ErrorCode": "V605", - "FileName": "posix_spawn.c", - "Message": "Consider verifying the expression. An unsigned value is compared to the number belonging to the range [-_.._]." - }, - { - "CodeCurrent": 2676964558, - "CodeNext": 17733, - "CodePrev": 17733, - "ErrorCode": "V526", - "FileName": "glob.c", - "Message": "The 'strcmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "exp2f.c", - "Message": "A code fragment from 'exp_f.c' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "OwnPatternFormatter.cpp", - "Message": "A code fragment from 'OwnPatternFormatter.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1586910037, - "CodeNext": 1159405304, - "CodePrev": 123, - "ErrorCode": "V769", - "FileName": "BaseDaemon.cpp", - "Message": "The 'name_start' pointer in the 'name_start - symbols[i]' expression could be nullptr. In such case, resulting value will be senseless and it should not be used." - }, - { - "CodeCurrent": 123, - "CodeNext": 125, - "CodePrev": 35837, - "ErrorCode": "V565", - "FileName": "BaseDaemon.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BaseDaemon.cpp", - "Message": "A code fragment from 'BaseDaemon.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Exception.cpp", - "Message": "A code fragment from 'Exception.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2045, - "CodeNext": 123, - "CodePrev": 2452368582, - "ErrorCode": "V730", - "FileName": "Row.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: lengths." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Connection.cpp", - "Message": "A code fragment from 'Connection.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 123, - "CodeNext": 2202583, - "CodePrev": 1128383580, - "ErrorCode": "V794", - "FileName": "Query.cpp", - "Message": "The assignment operator should be protected from the case of 'this == &other'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Query.cpp", - "Message": "A code fragment from 'Query.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ResultBase.cpp", - "Message": "A code fragment from 'ResultBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2377956688, - "CodeNext": 703533, - "CodePrev": 2042816737, - "ErrorCode": "V688", - "FileName": "StoreQueryResult.cpp", - "Message": "The 'fields' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StoreQueryResult.cpp", - "Message": "A code fragment from 'StoreQueryResult.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Row.cpp", - "Message": "A code fragment from 'Row.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Value.cpp", - "Message": "A code fragment from 'Value.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "UseQueryResult.cpp", - "Message": "A code fragment from 'UseQueryResult.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "consistent_hashing.cpp", - "Message": "A code fragment from 'consistent_hashing.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Pool.cpp", - "Message": "A code fragment from 'Pool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 188688957, - "CodeNext": 125, - "CodePrev": 769, - "ErrorCode": "V658", - "FileName": "sumbur.cpp", - "Message": "A value is being subtracted from the unsigned variable. This can result in an overflow. In such a case, the '<' comparison operation can potentially behave unexpectedly. Consider inspecting the '_xFFFFFFFF / _ - h < part' expression." - }, - { - "CodeCurrent": 450631639, - "CodeNext": 22890, - "CodePrev": 0, - "ErrorCode": "V1003", - "FileName": "sumbur.cpp", - "Message": "The macro 'curslice' is a dangerous expression. The parameter 'i' must be surrounded by parentheses." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "sumbur.cpp", - "Message": "A code fragment from 'sumbur.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PoolWithFailover.cpp", - "Message": "A code fragment from 'PoolWithFailover.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "mysqlxx_test.cpp", - "Message": "A code fragment from 'mysqlxx_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zstd_test.cpp", - "Message": "A code fragment from 'zstd_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "iotest.cpp", - "Message": "A code fragment from 'iotest.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "iotest_nonblock.cpp", - "Message": "A code fragment from 'iotest_nonblock.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "iotest_aio.cpp", - "Message": "A code fragment from 'iotest_aio.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "corrector_utf8.cpp", - "Message": "A code fragment from 'corrector_utf_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "config-processor.cpp", - "Message": "A code fragment from 'config-processor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2919138394, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "ZooKeeperImpl.h", - "Message": "The 'Exception' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zookeeper-cli.cpp", - "Message": "A code fragment from 'zookeeper-cli.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'main.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "main.cpp", - "Message": "A code fragment from 'main.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1244410449, - "CodeNext": 123, - "CodePrev": 182455598, - "ErrorCode": "V730", - "FileName": "HashTable.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: zero_value_storage." - }, - { - "CodeCurrent": 3824136199, - "CodeNext": 2663, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "HashTable.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: size." - }, - { - "CodeCurrent": 1712271, - "CodeNext": 0, - "CodePrev": 3363495029, - "ErrorCode": "V730", - "FileName": "MarkovModel.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: n." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MarkovModel.h", - "Message": "A code fragment from 'main.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 259986933, - "CodeNext": 0, - "CodePrev": 123, - "ErrorCode": "V730", - "FileName": "RWLock.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: type." - }, - { - "CodeCurrent": 6265, - "CodeNext": 5125961, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "Field.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: storage." - }, - { - "CodeCurrent": 101009567, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V730", - "FileName": "Field.h", - "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: storage, which." - }, - { - "CodeCurrent": 6341727, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "Field.h", - "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: storage, which." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AnalyzeColumns.cpp", - "Message": "A code fragment from 'AnalyzeColumns.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'markov-model.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AnalyzeLambdas.cpp", - "Message": "A code fragment from 'AnalyzeLambdas.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CollectAliases.cpp", - "Message": "A code fragment from 'CollectAliases.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3859608575, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 3859608575, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 728751127, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: f == static_cast< DB::Float_ > (u). It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 1801682850, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: static_cast< DB::Float_ > (u) == f. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 134227181, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: f == static_cast< DB::Float_ > (u). It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 940474457, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: static_cast< DB::Float_ > (u) == f. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 728751127, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: f == static_cast< DB::Float_ > (u). It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 1801682850, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: static_cast< DB::Float_ > (u) == f. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 134227181, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: f == static_cast< DB::Float_ > (u). It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 940474457, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "AccurateComparison.h", - "Message": "An odd precise comparison: static_cast< DB::Float_ > (u) == f. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 3765134313, - "CodeNext": 3685848331, - "CodePrev": 1405257529, - "ErrorCode": "V550", - "FileName": "FieldVisitors.h", - "Message": "An odd precise comparison: l == r. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 51079373, - "CodeNext": 0, - "CodePrev": 836015037, - "ErrorCode": "V550", - "FileName": "FieldVisitors.h", - "Message": "An odd precise comparison: x != _. It's probably better to use a comparison with defined precision: fabs(A - B) > Epsilon." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AnalyzeResultOfQuery.cpp", - "Message": "A code fragment from 'AnalyzeResultOfQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CollectTables.cpp", - "Message": "A code fragment from 'CollectTables.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ExecuteTableFunctions.cpp", - "Message": "A code fragment from 'ExecuteTableFunctions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "OptimizeGroupOrderLimitBy.cpp", - "Message": "A code fragment from 'OptimizeGroupOrderLimitBy.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TranslatePositionalArguments.cpp", - "Message": "A code fragment from 'TranslatePositionalArguments.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BlockInfo.cpp", - "Message": "A code fragment from 'BlockInfo.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 4174113710, - "CodeNext": 3531930912, - "CodePrev": 181464845, - "ErrorCode": "V659", - "FileName": "ColumnConst.h", - "Message": "Declarations of functions with 'getDataColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Block.cpp", - "Message": "A code fragment from 'Block.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 199355300, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V730", - "FileName": "ExpressionActions.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TypeAndConstantInference.cpp", - "Message": "A code fragment from 'TypeAndConstantInference.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnWithTypeAndName.cpp", - "Message": "A code fragment from 'ColumnWithTypeAndName.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Field.cpp", - "Message": "A code fragment from 'Field.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SortDescription.cpp", - "Message": "A code fragment from 'SortDescription.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NamesAndTypes.cpp", - "Message": "A code fragment from 'NamesAndTypes.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "iostream_debug_helpers.cpp", - "Message": "A code fragment from 'iostream_debug_helpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3826643199, - "CodeNext": 123, - "CodePrev": 22807034, - "ErrorCode": "V690", - "FileName": "ColumnVector.h", - "Message": "Copy constructor is declared as private in the 'ColumnVector' class, but the default '=' operator will still be generated by compiler. It is dangerous to use such a class." - }, - { - "CodeCurrent": 1132122309, - "CodeNext": 0, - "CodePrev": 931267045, - "ErrorCode": "V659", - "FileName": "ColumnArray.h", - "Message": "Declarations of functions with 'getData' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 1116407654, - "CodeNext": 0, - "CodePrev": 1356382257, - "ErrorCode": "V659", - "FileName": "ColumnArray.h", - "Message": "Declarations of functions with 'getOffsetsColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 2390648255, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V659", - "FileName": "ColumnArray.h", - "Message": "Declarations of functions with 'getOffsets' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AddingDefaultBlockOutputStream.cpp", - "Message": "A code fragment from 'AddingDefaultBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3168042121, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V690", - "FileName": "ColumnString.h", - "Message": "Copy constructor is declared as private in the 'ColumnString' class, but the default '=' operator will still be generated by compiler. It is dangerous to use such a class." - }, - { - "CodeCurrent": 4054773003, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V690", - "FileName": "ColumnAggregateFunction.h", - "Message": "Copy constructor is declared as private in the 'ColumnAggregateFunction' class, but the default '=' operator will still be generated by compiler. It is dangerous to use such a class." - }, - { - "CodeCurrent": 3194944389, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "AggregatingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 14743151, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "SortCursor.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: order." - }, - { - "CodeCurrent": 195436, - "CodeNext": 123, - "CodePrev": 45270, - "ErrorCode": "V730", - "FileName": "MergingSortedBlockInputStream.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: row_num." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregatingSortedBlockInputStream.cpp", - "Message": "A code fragment from 'AggregatingSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 4136571990, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V690", - "FileName": "ColumnFixedString.h", - "Message": "Copy constructor is declared as private in the 'ColumnFixedString' class, but the default '=' operator will still be generated by compiler. It is dangerous to use such a class." - }, - { - "CodeCurrent": 1054674186, - "CodeNext": 0, - "CodePrev": 925160603, - "ErrorCode": "V659", - "FileName": "ColumnNullable.h", - "Message": "Declarations of functions with 'getNestedColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 3317648788, - "CodeNext": 0, - "CodePrev": 3497477841, - "ErrorCode": "V659", - "FileName": "ColumnNullable.h", - "Message": "Declarations of functions with 'getNullMapColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 146511793, - "CodeNext": 1296505, - "CodePrev": 196592208, - "ErrorCode": "V522", - "FileName": "Aggregator.h", - "Message": "There might be dereferencing of a potential null pointer 'null_map'." - }, - { - "CodeCurrent": 132655, - "CodeNext": 0, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "TwoLevelHashTable.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: container, bucket, current_it." - }, - { - "CodeCurrent": 9156143, - "CodeNext": 1642228070, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "TwoLevelHashTable.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: container, bucket, current_it." - }, - { - "CodeCurrent": 3310514761, - "CodeNext": 2820625488, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "Aggregator.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: keys_size." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregatingBlockInputStream.cpp", - "Message": "A code fragment from 'AggregatingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BinaryRowInputStream.cpp", - "Message": "A code fragment from 'BinaryRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BinaryRowOutputStream.cpp", - "Message": "A code fragment from 'BinaryRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ApplyingMutationsBlockInputStream.cpp", - "Message": "A code fragment from 'ApplyingMutationsBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 389785, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "BlockIO.h", - "Message": "The 'BlockIO' class implements the '=' operator, but lacks a copy constructor. It is dangerous to use such a class." - }, - { - "CodeCurrent": 253306735, - "CodeNext": 4054391213, - "CodePrev": 0, - "ErrorCode": "V550", - "FileName": "ProcessList.h", - "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(memory_tracker_fault_probability) > Epsilon." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BlockIO.cpp", - "Message": "A code fragment from 'BlockIO.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BlockOutputStreamFromRowOutputStream.cpp", - "Message": "A code fragment from 'BlockOutputStreamFromRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2065830648, - "CodeNext": 2809, - "CodePrev": 0, - "ErrorCode": "V550", - "FileName": "BlockInputStreamFromRowInputStream.cpp", - "Message": "An odd precise comparison: allow_errors_ratio == _. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 123, - "CodeNext": 485783650, - "CodePrev": 35837, - "ErrorCode": "V565", - "FileName": "BlockInputStreamFromRowInputStream.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BlockInputStreamFromRowInputStream.cpp", - "Message": "A code fragment from 'BlockInputStreamFromRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BlockStreamProfileInfo.cpp", - "Message": "A code fragment from 'BlockStreamProfileInfo.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2244466177, - "CodeNext": 3040549265, - "CodePrev": 1036506627, - "ErrorCode": "V656", - "FileName": "CSVRowInputStream.cpp", - "Message": "Variables 'prev_position', 'curr_position' are initialized through the call to the same function. It's probably an error or un-optimized code. Consider inspecting the 'istr.position()' expression. Check lines: _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CSVRowInputStream.cpp", - "Message": "A code fragment from 'CSVRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CSVRowOutputStream.cpp", - "Message": "A code fragment from 'CSVRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 56687, - "CodeNext": 4160558847, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "CollapsingFinalBlockInputStream.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: pos." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CollapsingFinalBlockInputStream.cpp", - "Message": "A code fragment from 'CollapsingFinalBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3189192997, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "CollapsingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3474608216, - "CodeNext": 3529831664, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "CollapsingSortedBlockInputStream.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: current_key, next_key, first_negative, last_positive, last_negative." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CollapsingSortedBlockInputStream.cpp", - "Message": "A code fragment from 'CollapsingSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 194852, - "CodeNext": 123, - "CodePrev": 2043716261, - "ErrorCode": "V730", - "FileName": "CapnProtoRowInputStream.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'CapnProtoRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 92287, - "CodeNext": 231331854, - "CodePrev": 360716783, - "ErrorCode": "V550", - "FileName": "ColumnGathererStream.cpp", - "Message": "An odd precise comparison. It's probably better to use a comparison with defined precision: fabs(seconds) > Epsilon." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnGathererStream.cpp", - "Message": "A code fragment from 'ColumnGathererStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CountingBlockOutputStream.cpp", - "Message": "A code fragment from 'CountingBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ConvertingBlockInputStream.cpp", - "Message": "A code fragment from 'ConvertingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 578660774, - "CodeNext": 2441941198, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "HashSet.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: saved_hash." - }, - { - "CodeCurrent": 2441941198, - "CodeNext": 0, - "CodePrev": 578660774, - "ErrorCode": "V730", - "FileName": "HashSet.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: saved_hash." - }, - { - "CodeCurrent": 3398701039, - "CodeNext": 413159266, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "ClearableHashSet.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: version." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DistinctBlockInputStream.cpp", - "Message": "A code fragment from 'DistinctBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 233229516, - "CodeNext": 17813717, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "CreatingSetsBlockInputStream.cpp", - "Message": "There might be dereferencing of a potential null pointer 'subquery.set'." - }, - { - "CodeCurrent": 14128946, - "CodeNext": 35621461, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "CreatingSetsBlockInputStream.cpp", - "Message": "There might be dereferencing of a potential null pointer 'subquery.join'." - }, - { - "CodeCurrent": 207231, - "CodeNext": 150934975, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "BoolMask.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: can_be_true, can_be_false." - }, - { - "CodeCurrent": 61359, - "CodeNext": 1727213102, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "Join.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: block, row_num." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CreatingSetsBlockInputStream.cpp", - "Message": "A code fragment from 'CreatingSetsBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DistinctSortedBlockInputStream.cpp", - "Message": "A code fragment from 'DistinctSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ExpressionBlockInputStream.cpp", - "Message": "A code fragment from 'ExpressionBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FilterBlockInputStream.cpp", - "Message": "A code fragment from 'FilterBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FilterColumnsBlockInputStream.cpp", - "Message": "A code fragment from 'FilterColumnsBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3416694064, - "CodeNext": 1249324771, - "CodePrev": 2609586663, - "ErrorCode": "V526", - "FileName": "OptimizedRegularExpression.inl.h", - "Message": "The 'strncmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." - }, - { - "CodeCurrent": 1249324771, - "CodeNext": 611713731, - "CodePrev": 3416694064, - "ErrorCode": "V526", - "FileName": "OptimizedRegularExpression.inl.h", - "Message": "The 'strncmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." - }, - { - "CodeCurrent": 611713731, - "CodeNext": 3436934615, - "CodePrev": 1249324771, - "ErrorCode": "V526", - "FileName": "OptimizedRegularExpression.inl.h", - "Message": "The 'strncmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." - }, - { - "CodeCurrent": 3436934615, - "CodeNext": 123, - "CodePrev": 611713731, - "ErrorCode": "V526", - "FileName": "OptimizedRegularExpression.inl.h", - "Message": "The 'strncmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." - }, - { - "CodeCurrent": 3165904209, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "GraphiteRollupSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "GraphiteRollupSortedBlockInputStream.cpp", - "Message": "A code fragment from 'GraphiteRollupSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IBlockInputStream.cpp", - "Message": "A code fragment from 'IBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 505357, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "Quota.h", - "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: queries, errors, result_rows, result_bytes, read_rows, read_bytes, ..." - }, - { - "CodeCurrent": 1713353540, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "Quota.h", - "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: queries, errors, result_rows, result_bytes, read_rows, read_bytes, ..." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IProfilingBlockInputStream.cpp", - "Message": "A code fragment from 'IProfilingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'FormatFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IRowInputStream.cpp", - "Message": "A code fragment from 'IRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IRowOutputStream.cpp", - "Message": "A code fragment from 'IRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "JSONCompactRowOutputStream.cpp", - "Message": "A code fragment from 'JSONCompactRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InputStreamFromASTInsertQuery.cpp", - "Message": "A code fragment from 'InputStreamFromASTInsertQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "JSONEachRowRowInputStream.cpp", - "Message": "A code fragment from 'JSONEachRowRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "JSONEachRowRowOutputStream.cpp", - "Message": "A code fragment from 'JSONEachRowRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "JSONRowOutputStream.cpp", - "Message": "A code fragment from 'JSONRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "LimitBlockInputStream.cpp", - "Message": "A code fragment from 'LimitBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "LimitByBlockInputStream.cpp", - "Message": "A code fragment from 'LimitByBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MaterializingBlockInputStream.cpp", - "Message": "A code fragment from 'MaterializingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3161195889, - "CodeNext": 123, - "CodePrev": 1879495099, - "ErrorCode": "V688", - "FileName": "MergeSortingBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeSortingBlockInputStream.cpp", - "Message": "A code fragment from 'MergeSortingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergingAggregatedBlockInputStream.cpp", - "Message": "A code fragment from 'MergingAggregatedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1919288073, - "CodeNext": 123, - "CodePrev": 1879495099, - "ErrorCode": "V688", - "FileName": "MergingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3223507410, - "CodeNext": 123, - "CodePrev": 1879495099, - "ErrorCode": "V688", - "FileName": "MergingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2080638586, - "CodeNext": 123, - "CodePrev": 1879495099, - "ErrorCode": "V688", - "FileName": "MergingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergingSortedBlockInputStream.cpp", - "Message": "A code fragment from 'MergingSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergingAggregatedMemoryEfficientBlockInputStream.cpp", - "Message": "A code fragment from 'MergingAggregatedMemoryEfficientBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NativeBlockInputStream.cpp", - "Message": "A code fragment from 'NativeBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ODBCDriverBlockOutputStream.cpp", - "Message": "A code fragment from 'ODBCDriverBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NativeBlockOutputStream.cpp", - "Message": "A code fragment from 'NativeBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 429503, - "CodeNext": 3001760339, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "ParallelInputsProcessor.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: i." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParallelAggregatingBlockInputStream.cpp", - "Message": "A code fragment from 'ParallelAggregatingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PartialSortingBlockInputStream.cpp", - "Message": "A code fragment from 'PartialSortingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PrettyBlockOutputStream.cpp", - "Message": "A code fragment from 'PrettyBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PrettyCompactBlockOutputStream.cpp", - "Message": "A code fragment from 'PrettyCompactBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PrettySpaceBlockOutputStream.cpp", - "Message": "A code fragment from 'PrettySpaceBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "RemoteBlockOutputStream.cpp", - "Message": "A code fragment from 'RemoteBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 12514670, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "PoolWithFailoverBase.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: pool, state, index." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "RemoteBlockInputStream.cpp", - "Message": "A code fragment from 'RemoteBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SizeLimits.cpp", - "Message": "A code fragment from 'SizeLimits.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2251640247, - "CodeNext": 350966580, - "CodePrev": 576202780, - "ErrorCode": "V688", - "FileName": "PushingToViewsBlockOutputStream.cpp", - "Message": "The 'storage' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PushingToViewsBlockOutputStream.cpp", - "Message": "A code fragment from 'PushingToViewsBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3190533093, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ReplacingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2535084036, - "CodeNext": 1504104628, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "ReplacingSortedBlockInputStream.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: current_key, next_key, selected_row." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplacingSortedBlockInputStream.cpp", - "Message": "A code fragment from 'ReplacingSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SquashingBlockOutputStream.cpp", - "Message": "A code fragment from 'SquashingBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SquashingBlockInputStream.cpp", - "Message": "A code fragment from 'SquashingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SquashingTransform.cpp", - "Message": "A code fragment from 'SquashingTransform.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3967747240, - "CodeNext": 0, - "CodePrev": 893858483, - "ErrorCode": "V659", - "FileName": "ColumnTuple.h", - "Message": "Declarations of functions with 'getColumn' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 3191513253, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "SummingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SummingSortedBlockInputStream.cpp", - "Message": "A code fragment from 'SummingSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TSKVRowInputStream.cpp", - "Message": "A code fragment from 'TSKVRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TabSeparatedRowInputStream.cpp", - "Message": "A code fragment from 'TabSeparatedRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TSKVRowOutputStream.cpp", - "Message": "A code fragment from 'TSKVRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TabSeparatedRowOutputStream.cpp", - "Message": "A code fragment from 'TabSeparatedRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ValuesRowOutputStream.cpp", - "Message": "A code fragment from 'ValuesRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TotalsHavingBlockInputStream.cpp", - "Message": "A code fragment from 'TotalsHavingBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ValuesRowInputStream.cpp", - "Message": "A code fragment from 'ValuesRowInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "VerticalRowOutputStream.cpp", - "Message": "A code fragment from 'VerticalRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3998578981, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "VersionedCollapsingSortedBlockInputStream.cpp", - "Message": "The 'queue' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "VersionedCollapsingSortedBlockInputStream.cpp", - "Message": "A code fragment from 'VersionedCollapsingSortedBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "XMLRowOutputStream.cpp", - "Message": "A code fragment from 'XMLRowOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "copyData.cpp", - "Message": "A code fragment from 'copyData.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "materializeBlock.cpp", - "Message": "A code fragment from 'materializeBlock.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "verbosePrintString.cpp", - "Message": "A code fragment from 'verbosePrintString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "narrowBlockInputStreams.cpp", - "Message": "A code fragment from 'narrowBlockInputStreams.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeAggregateFunction.cpp", - "Message": "A code fragment from 'DataTypeAggregateFunction.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeArray.cpp", - "Message": "A code fragment from 'DataTypeArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeDate.cpp", - "Message": "A code fragment from 'DataTypeDate.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeDateTime.cpp", - "Message": "A code fragment from 'DataTypeDateTime.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeFactory.cpp", - "Message": "A code fragment from 'DataTypeFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeFunction.cpp", - "Message": "A code fragment from 'DataTypeFunction.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 566420924, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "DataTypeEnum.h", - "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2818381, - "CodeNext": 690132628, - "CodePrev": 3160185338, - "ErrorCode": "V688", - "FileName": "DataTypeEnum.cpp", - "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2818381, - "CodeNext": 368631651, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "DataTypeEnum.cpp", - "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2818381, - "CodeNext": 2806683281, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "DataTypeEnum.cpp", - "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2818381, - "CodeNext": 710643812, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "DataTypeEnum.cpp", - "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeEnum.cpp", - "Message": "A code fragment from 'DataTypeEnum.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeInterval.cpp", - "Message": "A code fragment from 'DataTypeInterval.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeFixedString.cpp", - "Message": "A code fragment from 'DataTypeFixedString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeNothing.cpp", - "Message": "A code fragment from 'DataTypeNothing.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeString.cpp", - "Message": "A code fragment from 'DataTypeString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeNumberBase.cpp", - "Message": "A code fragment from 'DataTypeNumberBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeNullable.cpp", - "Message": "A code fragment from 'DataTypeNullable.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypesNumber.cpp", - "Message": "A code fragment from 'DataTypesNumber.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FieldToDataType.cpp", - "Message": "A code fragment from 'FieldToDataType.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeTuple.cpp", - "Message": "A code fragment from 'DataTypeTuple.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataTypeUUID.cpp", - "Message": "A code fragment from 'DataTypeUUID.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IDataType.cpp", - "Message": "A code fragment from 'IDataType.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NestedUtils.cpp", - "Message": "A code fragment from 'NestedUtils.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getLeastSupertype.cpp", - "Message": "A code fragment from 'getLeastSupertype.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getMostSubtype.cpp", - "Message": "A code fragment from 'getMostSubtype.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DatabaseFactory.cpp", - "Message": "A code fragment from 'DatabaseFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DatabaseMemory.cpp", - "Message": "A code fragment from 'DatabaseMemory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2421950747, - "CodeNext": 281663836, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "DatabaseDictionary.cpp", - "Message": "The 'name' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'DatabaseDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 4574138, - "CodeNext": 2250845352, - "CodePrev": 72410316, - "ErrorCode": "V688", - "FileName": "DatabaseOrdinary.cpp", - "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DatabaseOrdinary.cpp", - "Message": "A code fragment from 'DatabaseOrdinary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DatabasesCommon.cpp", - "Message": "A code fragment from 'DatabasesCommon.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3735361261, - "CodeNext": 123, - "CodePrev": 1623468225, - "ErrorCode": "V688", - "FileName": "CatBoostModel.cpp", - "Message": "The 'cat_features_count' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2354497829, - "CodeNext": 2736089741, - "CodePrev": 2412400287, - "ErrorCode": "V688", - "FileName": "CatBoostModel.cpp", - "Message": "The 'float_features_count' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2354497829, - "CodeNext": 2736089741, - "CodePrev": 2412400287, - "ErrorCode": "V688", - "FileName": "CatBoostModel.cpp", - "Message": "The 'cat_features_count' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2575466056, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "CatBoostModel.cpp", - "Message": "The 'lib_path' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'CatBoostModel.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 207025177, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "CacheDictionary.h", - "Message": "The 'CacheDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 518403501, - "CodeNext": 610328694, - "CodePrev": 1500196055, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 411059078, - "CodeNext": 123, - "CodePrev": 610328694, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 518630893, - "CodeNext": 610328694, - "CodePrev": 1500196055, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 411059078, - "CodeNext": 123, - "CodePrev": 610328694, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 411059078, - "CodeNext": 123, - "CodePrev": 1382482652, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 411059078, - "CodeNext": 123, - "CodePrev": 1382482652, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3731206478, - "CodeNext": 123, - "CodePrev": 11900062, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3673035245, - "CodeNext": 1382482652, - "CodePrev": 3006984332, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2414448007, - "CodeNext": 123, - "CodePrev": 1382482652, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3673035245, - "CodeNext": 1382482652, - "CodePrev": 3467903158, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2414448007, - "CodeNext": 123, - "CodePrev": 1382482652, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 921248968, - "CodeNext": 123, - "CodePrev": 1342750387, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3690553619, - "CodeNext": 37308961, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "CacheDictionary.cpp", - "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 173283489, - "CodeNext": 1039775048, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "CacheDictionary.cpp", - "Message": "The 'size' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 549768760, - "CodeNext": 1152076000, - "CodePrev": 1919584191, - "ErrorCode": "V688", - "FileName": "CacheDictionary.cpp", - "Message": "The 'size' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 579497379, - "CodeNext": 93115, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "CacheDictionary.cpp", - "Message": "The 'size' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'CacheDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1825423383, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V690", - "FileName": "ClickHouseDictionarySource.h", - "Message": "The 'ClickHouseDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ClickHouseDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2185656967, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V690", - "FileName": "ComplexKeyCacheDictionary.h", - "Message": "The 'ComplexKeyCacheDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 3504260784, - "CodeNext": 37308961, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ComplexKeyCacheDictionary.cpp", - "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2142244675, - "CodeNext": 2508128848, - "CodePrev": 41371507, - "ErrorCode": "V688", - "FileName": "ComplexKeyCacheDictionary.cpp", - "Message": "The 'key_size' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyCacheDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyCacheDictionary_createAttributeWithType.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyCacheDictionary_generate_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyCacheDictionary_generate_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyCacheDictionary_generate_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 579497379, - "CodeNext": 93115, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ComplexKeyCacheDictionary_setAttributeValue.cpp", - "Message": "The 'size' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyCacheDictionary_setAttributeValue.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1173598882, - "CodeNext": 78734017, - "CodePrev": 38981872, - "ErrorCode": "V688", - "FileName": "DictionaryBlockInputStreamBase.cpp", - "Message": "The 'block' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DictionaryBlockInputStreamBase.cpp", - "Message": "A code fragment from 'DictionaryBlockInputStreamBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1259294311, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "ComplexKeyHashedDictionary.h", - "Message": "The 'ComplexKeyHashedDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ComplexKeyHashedDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'DictionarySourceHelpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2179468235, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "DictionaryStructure.cpp", - "Message": "The 'attributes' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'DictionaryStructure.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1562050581, - "CodeNext": 123, - "CodePrev": 2247801432, - "ErrorCode": "V690", - "FileName": "FileDictionarySource.h", - "Message": "The 'FileDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 1863286807, - "CodeNext": 123, - "CodePrev": 3087958036, - "ErrorCode": "V690", - "FileName": "ExecutableDictionarySource.h", - "Message": "The 'ExecutableDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 1560485909, - "CodeNext": 123, - "CodePrev": 3228890078, - "ErrorCode": "V690", - "FileName": "HTTPDictionarySource.h", - "Message": "The 'HTTPDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 314376213, - "CodeNext": 123, - "CodePrev": 4033243237, - "ErrorCode": "V690", - "FileName": "LibraryDictionarySource.h", - "Message": "The 'LibraryDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 313384981, - "CodeNext": 123, - "CodePrev": 247020956, - "ErrorCode": "V690", - "FileName": "MongoDBDictionarySource.h", - "Message": "The 'MongoDBDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 1561051157, - "CodeNext": 123, - "CodePrev": 1880965760, - "ErrorCode": "V690", - "FileName": "ODBCDictionarySource.h", - "Message": "The 'ODBCDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 1143955477, - "CodeNext": 123, - "CodePrev": 2162917980, - "ErrorCode": "V690", - "FileName": "MySQLDictionarySource.h", - "Message": "The 'MySQLDictionarySource' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'DictionarySourceFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3300655365, - "CodeNext": 2232737895, - "CodePrev": 3529503794, - "ErrorCode": "V669", - "FileName": "VarInt.h", - "Message": "The 'ostr' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 2232737895, - "CodeNext": 3413908603, - "CodePrev": 3300655365, - "ErrorCode": "V669", - "FileName": "VarInt.h", - "Message": "The 'ostr' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 2696744022, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "SettingsCommon.h", - "Message": "The 'buf' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 2696744022, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "SettingsCommon.h", - "Message": "The 'buf' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 2696744022, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "SettingsCommon.h", - "Message": "The 'buf' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 3674152177, - "CodeNext": 516173517, - "CodePrev": 5816, - "ErrorCode": "V688", - "FileName": "OwningBlockInputStream.h", - "Message": "The 'stream' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ExecutableDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ExternalResultDescription.cpp", - "Message": "A code fragment from 'ExternalResultDescription.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3266193319, - "CodeNext": 193, - "CodePrev": 123, - "ErrorCode": "V516", - "FileName": "IColumn.h", - "Message": "Consider inspecting an odd expression. Non-null function pointer is compared to null." - }, - { - "CodeCurrent": 3266193319, - "CodeNext": 193, - "CodePrev": 123, - "ErrorCode": "V516", - "FileName": "IColumn.h", - "Message": "Consider inspecting an odd expression. Non-null function pointer is compared to null: 'IsMutableColumns < Args >::value'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ExternalQueryBuilder.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FileDictionarySource.cpp", - "Message": "A code fragment from 'FileDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 207054359, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "FlatDictionary.h", - "Message": "The 'FlatDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'FlatDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'HTTPDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 206951383, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "HashedDictionary.h", - "Message": "The 'HashedDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'HashedDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 605484233, - "CodeNext": 26586574, - "CodePrev": 3065615242, - "ErrorCode": "V688", - "FileName": "LibraryDictionarySource.cpp", - "Message": "The 'config_prefix' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 26586574, - "CodeNext": 72410313, - "CodePrev": 605484233, - "ErrorCode": "V688", - "FileName": "LibraryDictionarySource.cpp", - "Message": "The 'sample_block' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'LibraryDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'MongoDBBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'MongoDBDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MySQLBlockInputStream.cpp", - "Message": "A code fragment from 'MySQLBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3362020784, - "CodeNext": 0, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "MySQLDictionarySource.cpp", - "Message": "The 'update_time' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 13569497, - "CodeNext": 985361697, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "MySQLDictionarySource.cpp", - "Message": "The 'sample_block' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'MySQLDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ODBCBlockInputStream.cpp", - "Message": "A code fragment from 'ODBCBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3238522471, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "RangeHashedDictionary.h", - "Message": "The 'RangeHashedDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 3406274085, - "CodeNext": 4182595664, - "CodePrev": 3042861068, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 4182595664, - "CodeNext": 123, - "CodePrev": 3406274085, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 456188231, - "CodeNext": 2414448007, - "CodePrev": 1469584987, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2414448007, - "CodeNext": 123, - "CodePrev": 456188231, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'dictionary' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2178677200, - "CodeNext": 2078138180, - "CodePrev": 2641309169, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'column_names' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2418126002, - "CodeNext": 123, - "CodePrev": 2901253088, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'start_dates' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2418126002, - "CodeNext": 123, - "CodePrev": 2901253088, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'end_dates' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1525601398, - "CodeNext": 2418126002, - "CodePrev": 682989491, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'ids' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2418126002, - "CodeNext": 123, - "CodePrev": 1525601398, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'start_dates' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2418126002, - "CodeNext": 123, - "CodePrev": 1525601398, - "ErrorCode": "V688", - "FileName": "RangeDictionaryBlockInputStream.h", - "Message": "The 'end_dates' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 127934457, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "RangeHashedDictionary.cpp", - "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'RangeHashedDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ODBCDictionarySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "GeoDictionariesLoader.cpp", - "Message": "A code fragment from 'GeoDictionariesLoader.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "readInvalidateQuery.cpp", - "Message": "A code fragment from 'readInvalidateQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "writeParenthesisedString.cpp", - "Message": "A code fragment from 'writeParenthesisedString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3312877383, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V690", - "FileName": "TrieDictionary.h", - "Message": "The 'TrieDictionary' class implements a copy constructor, but lacks the '=' operator. It is dangerous to use such a class." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'TrieDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "RegionsHierarchies.cpp", - "Message": "A code fragment from 'RegionsHierarchies.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "RegionsHierarchy.cpp", - "Message": "A code fragment from 'RegionsHierarchy.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "RegionsNames.cpp", - "Message": "A code fragment from 'RegionsNames.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TechDataHierarchy.cpp", - "Message": "A code fragment from 'TechDataHierarchy.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "HierarchiesProvider.cpp", - "Message": "A code fragment from 'HierarchiesProvider.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NamesProvider.cpp", - "Message": "A code fragment from 'NamesProvider.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "HierarchyFormatReader.cpp", - "Message": "A code fragment from 'HierarchyFormatReader.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NamesFormatReader.cpp", - "Message": "A code fragment from 'NamesFormatReader.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ActionLocksManager.cpp", - "Message": "A code fragment from 'ActionLocksManager.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ClientInfo.cpp", - "Message": "A code fragment from 'ClientInfo.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Compiler.cpp", - "Message": "A code fragment from 'Compiler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 871352, - "CodeNext": 223912592, - "CodePrev": 766487020, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 73089598, - "CodeNext": 4084406, - "CodePrev": 2724040669, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 3873811996, - "CodePrev": 3673856596, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 73089598, - "CodeNext": 4084406, - "CodePrev": 2724040669, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 36816, - "CodePrev": 3126615886, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 511803757, - "CodePrev": 3126615886, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 122870, - "CodePrev": 1474597293, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 73089598, - "CodeNext": 1179330, - "CodePrev": 2685314853, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 122870, - "CodePrev": 4240594574, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2338865922, - "CodeNext": 123, - "CodePrev": 2685314853, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 122870, - "CodePrev": 3859987971, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2338865922, - "CodeNext": 123, - "CodePrev": 4042434526, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 36816, - "CodePrev": 3126615886, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 73089598, - "CodeNext": 223912592, - "CodePrev": 207246, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 122870, - "CodePrev": 223912592, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 73089598, - "CodeNext": 223912592, - "CodePrev": 207246, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 122870, - "CodePrev": 223912592, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 871352, - "CodeNext": 109104, - "CodePrev": 3874875103, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'method' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 73089598, - "CodeNext": 4084406, - "CodePrev": 3397567370, - "ErrorCode": "V688", - "FileName": "Aggregator.cpp", - "Message": "The 'key_sizes' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Aggregator.cpp", - "Message": "A code fragment from 'Aggregator.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 304867129, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "Cluster.cpp", - "Message": "The 'host_name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 304867129, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "Cluster.cpp", - "Message": "The 'port' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3548849765, - "CodeNext": 2541401097, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "HexWriteBuffer.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: buf." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Cluster.cpp", - "Message": "A code fragment from 'Cluster.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AsynchronousMetrics.cpp", - "Message": "A code fragment from 'AsynchronousMetrics.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 123, - "CodeNext": 129791, - "CodePrev": 35837, - "ErrorCode": "V565", - "FileName": "DNSCacheUpdater.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 1922414329, - "CodeNext": 123, - "CodePrev": 125, - "ErrorCode": "V759", - "FileName": "DNSCacheUpdater.cpp", - "Message": "Incorrect order of exception handling blocks. Exception of 'DNSException' type will be caught in a catch block for 'Exception' base type." - }, - { - "CodeCurrent": 958195402, - "CodeNext": 123, - "CodePrev": 125, - "ErrorCode": "V759", - "FileName": "DNSCacheUpdater.cpp", - "Message": "Incorrect order of exception handling blocks. Exception of 'TimeoutException' type will be caught in a catch block for 'Exception' base type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DNSCacheUpdater.cpp", - "Message": "A code fragment from 'DNSCacheUpdater.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 390633, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "DDLWorker.cpp", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: host_shard_num, host_replica_num." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DDLWorker.cpp", - "Message": "A code fragment from 'DDLWorker.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'DictionaryFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3538236801, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V524", - "FileName": "RuntimeComponentsFactory.h", - "Message": "It is odd that the body of 'createExternalModelsConfigRepository' function is fully equivalent to the body of 'createExternalDictionariesConfigRepository' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'Context.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "EmbeddedDictionaries.cpp", - "Message": "A code fragment from 'EmbeddedDictionaries.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1141335469, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionActions.cpp", - "Message": "The 'sample_block' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ExpressionActions.cpp", - "Message": "A code fragment from 'ExpressionActions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ExternalDictionaries.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3392699298, - "CodeNext": 0, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "ExternalLoader.cpp", - "Message": "The 'config' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ExternalLoader.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ExternalLoaderConfigRepository.cpp", - "Message": "A code fragment from 'ExternalLoaderConfigRepository.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3405548629, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2136590205, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'aliases' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1478353069, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3783766598, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3829470883, - "CodeNext": 0, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "There might be dereferencing of a potential null pointer 'subquery'." - }, - { - "CodeCurrent": 3237744880, - "CodeNext": 123, - "CodePrev": 3248671758, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1459789653, - "CodeNext": 123, - "CodePrev": 4129363794, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3479198451, - "CodeNext": 4055667619, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'aliases' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 4216604761, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 427259459, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3061641111, - "CodeNext": 123, - "CodePrev": 137989537, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1240966709, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3626124927, - "CodeNext": 2885543787, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2661019153, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1362320973, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3178398591, - "CodeNext": 1853405735, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 238898558, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3409673381, - "CodeNext": 72789921, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ExpressionAnalyzer.cpp", - "Message": "The 'ast' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ExpressionAnalyzer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'ExternalModels.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InJoinSubqueriesPreprocessor.cpp", - "Message": "A code fragment from 'InJoinSubqueriesPreprocessor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 200100772, - "CodeNext": 123, - "CodePrev": 11496, - "ErrorCode": "V730", - "FileName": "InterpreterAlterQuery.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterAlterQuery.cpp", - "Message": "A code fragment from 'InterpreterAlterQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterCheckQuery.cpp", - "Message": "A code fragment from 'InterpreterCheckQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterDescribeQuery.cpp", - "Message": "A code fragment from 'InterpreterDescribeQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3494840295, - "CodeNext": 193, - "CodePrev": 0, - "ErrorCode": "V524", - "FileName": "StorageLog.h", - "Message": "It is odd that the body of 'getFullPath' function is fully equivalent to the body of 'full_path' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterCreateQuery.cpp", - "Message": "A code fragment from 'InterpreterCreateQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterExistsQuery.cpp", - "Message": "A code fragment from 'InterpreterExistsQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2270843740, - "CodeNext": 125, - "CodePrev": 4111337664, - "ErrorCode": "V522", - "FileName": "InterpreterDropQuery.cpp", - "Message": "There might be dereferencing of a potential null pointer 'database'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterDropQuery.cpp", - "Message": "A code fragment from 'InterpreterDropQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterFactory.cpp", - "Message": "A code fragment from 'InterpreterFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterKillQueryQuery.cpp", - "Message": "A code fragment from 'InterpreterKillQueryQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterInsertQuery.cpp", - "Message": "A code fragment from 'InterpreterInsertQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterOptimizeQuery.cpp", - "Message": "A code fragment from 'InterpreterOptimizeQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterRenameQuery.cpp", - "Message": "A code fragment from 'InterpreterRenameQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3481785836, - "CodeNext": 0, - "CodePrev": 263098193, - "ErrorCode": "V678", - "FileName": "InterpreterSelectWithUnionQuery.cpp", - "Message": "An object is used as an argument to its own method. Consider checking the first actual argument of the 'setQueryContext' function." - }, - { - "CodeCurrent": 146241864, - "CodeNext": 72410313, - "CodePrev": 607166188, - "ErrorCode": "V688", - "FileName": "InterpreterSelectWithUnionQuery.cpp", - "Message": "The 'query_ptr' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 72410313, - "CodeNext": 123, - "CodePrev": 146241864, - "ErrorCode": "V688", - "FileName": "InterpreterSelectWithUnionQuery.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterSelectWithUnionQuery.cpp", - "Message": "A code fragment from 'InterpreterSelectWithUnionQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterSetQuery.cpp", - "Message": "A code fragment from 'InterpreterSetQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterShowTablesQuery.cpp", - "Message": "A code fragment from 'InterpreterShowTablesQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3481785836, - "CodeNext": 0, - "CodePrev": 263098193, - "ErrorCode": "V678", - "FileName": "InterpreterSelectQuery.cpp", - "Message": "An object is used as an argument to its own method. Consider checking the first actual argument of the 'setQueryContext' function." - }, - { - "CodeCurrent": 1460847663, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "InterpreterSelectQuery.cpp", - "Message": "The 'input' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2449370409, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V524", - "FileName": "InterpreterSelectQuery.cpp", - "Message": "It is odd that the body of 'executeProjection' function is fully equivalent to the body of 'executeExpression' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterSelectQuery.cpp", - "Message": "A code fragment from 'InterpreterSelectQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterShowProcesslistQuery.cpp", - "Message": "A code fragment from 'InterpreterShowProcesslistQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1627430578, - "CodeNext": 520103033, - "CodePrev": 721433261, - "ErrorCode": "V522", - "FileName": "InterpreterShowCreateQuery.cpp", - "Message": "There might be dereferencing of a potential null pointer 'create_query'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterShowCreateQuery.cpp", - "Message": "A code fragment from 'InterpreterShowCreateQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterpreterUseQuery.cpp", - "Message": "A code fragment from 'InterpreterUseQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "LogicalExpressionsOptimizer.cpp", - "Message": "A code fragment from 'LogicalExpressionsOptimizer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1421843502, - "CodeNext": 2932320854, - "CodePrev": 3193279124, - "ErrorCode": "V688", - "FileName": "Join.cpp", - "Message": "The 'columns_left' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Join.cpp", - "Message": "A code fragment from 'Join.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NullableUtils.cpp", - "Message": "A code fragment from 'NullableUtils.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2628352566, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V659", - "FileName": "MergeTreeData.h", - "Message": "Declarations of functions with 'modifyPartState' name differ in the 'const' keyword only, but the bodies of these functions have different composition. This is suspicious and can possibly be an error. Check lines: _, _." - }, - { - "CodeCurrent": 3583851774, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "InterpreterSystemQuery.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3000727018, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "InterpreterSystemQuery.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'InterpreterSystemQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3170877678, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "ProcessList.cpp", - "Message": "The 'it' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ProcessList.cpp", - "Message": "A code fragment from 'ProcessList.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ProjectionManipulation.cpp", - "Message": "A code fragment from 'ProjectionManipulation.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PartLog.cpp", - "Message": "A code fragment from 'PartLog.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "QueryLog.cpp", - "Message": "A code fragment from 'QueryLog.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Quota.cpp", - "Message": "A code fragment from 'Quota.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SecurityManager.cpp", - "Message": "A code fragment from 'SecurityManager.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Set.cpp", - "Message": "A code fragment from 'Set.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SetVariants.cpp", - "Message": "A code fragment from 'SetVariants.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SystemLog.cpp", - "Message": "A code fragment from 'SystemLog.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Settings.cpp", - "Message": "A code fragment from 'Settings.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TablesStatus.cpp", - "Message": "A code fragment from 'TablesStatus.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Users.cpp", - "Message": "A code fragment from 'Users.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "createBlockSelector.cpp", - "Message": "A code fragment from 'createBlockSelector.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "castColumn.cpp", - "Message": "A code fragment from 'castColumn.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "convertFieldToType.cpp", - "Message": "A code fragment from 'convertFieldToType.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "evaluateConstantExpression.cpp", - "Message": "A code fragment from 'evaluateConstantExpression.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getClusterName.cpp", - "Message": "A code fragment from 'getClusterName.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "evaluateMissingDefaults.cpp", - "Message": "A code fragment from 'evaluateMissingDefaults.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 143231, - "CodeNext": 125, - "CodePrev": 125, - "ErrorCode": "V565", - "FileName": "executeQuery.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 3481785836, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V678", - "FileName": "executeQuery.cpp", - "Message": "An object is used as an argument to its own method. Consider checking the first actual argument of the 'setQueryContext' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "executeQuery.cpp", - "Message": "A code fragment from 'executeQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "sortBlock.cpp", - "Message": "A code fragment from 'sortBlock.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "loadMetadata.cpp", - "Message": "A code fragment from 'loadMetadata.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DescribeStreamFactory.cpp", - "Message": "A code fragment from 'DescribeStreamFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Collator.cpp", - "Message": "A code fragment from 'Collator.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "executeQuery.cpp", - "Message": "A code fragment from 'executeQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 39392417, - "CodeNext": 1261068192, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ColumnAggregateFunction.cpp", - "Message": "The 'data' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 699364582, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ColumnAggregateFunction.cpp", - "Message": "The 'src' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 4081716836, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ColumnAggregateFunction.cpp", - "Message": "The 'src' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnAggregateFunction.cpp", - "Message": "A code fragment from 'ColumnAggregateFunction.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SelectStreamFactory.cpp", - "Message": "A code fragment from 'SelectStreamFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2533242737, - "CodeNext": 3575089120, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "ColumnArray.cpp", - "Message": "The 'offsets' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnArray.cpp", - "Message": "A code fragment from 'ColumnArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnConst.cpp", - "Message": "A code fragment from 'ColumnConst.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnFixedString.cpp", - "Message": "A code fragment from 'ColumnFixedString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3418757250, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ColumnNullable.cpp", - "Message": "The 'null_map' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnNullable.cpp", - "Message": "A code fragment from 'ColumnNullable.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnFunction.cpp", - "Message": "A code fragment from 'ColumnFunction.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnString.cpp", - "Message": "A code fragment from 'ColumnString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FilterDescription.cpp", - "Message": "A code fragment from 'FilterDescription.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3124110550, - "CodeNext": 1303704304, - "CodePrev": 6015, - "ErrorCode": "V792", - "FileName": "ColumnsCommon.cpp", - "Message": "The '_mm_movemask_ps' function located to the right of the operator '|' will be called regardless of the value of the left operand. Perhaps, it is better to use '||'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnsCommon.cpp", - "Message": "A code fragment from 'ColumnsCommon.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnTuple.cpp", - "Message": "A code fragment from 'ColumnTuple.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 951091257, - "CodeNext": 0, - "CodePrev": 951484473, - "ErrorCode": "V656", - "FileName": "ColumnVector.cpp", - "Message": "Variables 'cur_min', 'cur_max' are initialized through the call to the same function. It's probably an error or un-optimized code. Consider inspecting the 'NaNOrZero < T > ()' expression. Check lines: _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnVector.cpp", - "Message": "A code fragment from 'ColumnVector.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnDefault.cpp", - "Message": "A code fragment from 'ColumnDefault.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IColumn.cpp", - "Message": "A code fragment from 'IColumn.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AlterCommands.cpp", - "Message": "A code fragment from 'AlterCommands.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IStorage.cpp", - "Message": "A code fragment from 'IStorage.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ColumnsDescription.cpp", - "Message": "A code fragment from 'ColumnsDescription.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ITableDeclaration.cpp", - "Message": "A code fragment from 'ITableDeclaration.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MutationCommands.cpp", - "Message": "A code fragment from 'MutationCommands.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2317070405, - "CodeNext": 813477413, - "CodePrev": 608950573, - "ErrorCode": "V688", - "FileName": "StorageCatBoostPool.cpp", - "Message": "The 'sample_block' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageCatBoostPool.cpp", - "Message": "A code fragment from 'StorageCatBoostPool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 72410316, - "CodeNext": 2690972308, - "CodePrev": 3645137672, - "ErrorCode": "V688", - "FileName": "StorageBuffer.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2232984829, - "CodeNext": 345608494, - "CodePrev": 0, - "ErrorCode": "V547", - "FileName": "StorageBuffer.cpp", - "Message": "Expression '!buffer.first_write_time' is always true." - }, - { - "CodeCurrent": 2935677976, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageBuffer.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageBuffer.cpp", - "Message": "A code fragment from 'StorageBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'StorageDictionary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageFile.cpp", - "Message": "A code fragment from 'StorageFile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 72410316, - "CodeNext": 2690972308, - "CodePrev": 3645137672, - "ErrorCode": "V688", - "FileName": "StorageDistributed.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3889328658, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageDistributed.cpp", - "Message": "The 'table_name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3889328658, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageDistributed.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2637534258, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageDistributed.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageDistributed.cpp", - "Message": "A code fragment from 'StorageDistributed.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageJoin.cpp", - "Message": "A code fragment from 'StorageJoin.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageFactory.cpp", - "Message": "A code fragment from 'StorageFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3969971195, - "CodeNext": 0, - "CodePrev": 2997480514, - "ErrorCode": "V656", - "FileName": "StorageMemory.cpp", - "Message": "Variables 'begin', 'end' are initialized through the call to the same function. It's probably an error or un-optimized code. Consider inspecting the 'data.begin()' expression. Check lines: _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageMemory.cpp", - "Message": "A code fragment from 'StorageMemory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 581555750, - "CodeNext": 2488995567, - "CodePrev": 2696082656, - "ErrorCode": "V688", - "FileName": "StorageLog.cpp", - "Message": "The 'column_names' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageLog.cpp", - "Message": "A code fragment from 'StorageLog.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 11714530, - "CodeNext": 0, - "CodePrev": 2931558369, - "ErrorCode": "V522", - "FileName": "StorageMaterializedView.cpp", - "Message": "There might be dereferencing of a potential null pointer 'query.storage'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageMaterializedView.cpp", - "Message": "A code fragment from 'StorageMaterializedView.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 72410316, - "CodeNext": 2690972308, - "CodePrev": 2488995567, - "ErrorCode": "V688", - "FileName": "StorageKafka.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'StorageKafka.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 72410316, - "CodeNext": 2690972308, - "CodePrev": 3645137672, - "ErrorCode": "V688", - "FileName": "StorageMySQL.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageMySQL.cpp", - "Message": "A code fragment from 'StorageMySQL.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 72410316, - "CodeNext": 2690972308, - "CodePrev": 3645137672, - "ErrorCode": "V688", - "FileName": "StorageMerge.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3595788056, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageMerge.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageMerge.cpp", - "Message": "A code fragment from 'StorageMerge.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3926377496, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageNull.cpp", - "Message": "The 'table_name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageNull.cpp", - "Message": "A code fragment from 'StorageNull.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 72410316, - "CodeNext": 2690972308, - "CodePrev": 3645137672, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2341879450, - "CodeNext": 292720666, - "CodePrev": 2322283894, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'database_name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 292720666, - "CodeNext": 72410313, - "CodePrev": 2341879450, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'table_name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 72410313, - "CodeNext": 123, - "CodePrev": 292720666, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2033765224, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3317773483, - "CodeNext": 123, - "CodePrev": 329476003, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2082196268, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 694167244, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 391214422, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3490868120, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageMergeTree.cpp", - "Message": "A code fragment from 'StorageMergeTree.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageODBC.cpp", - "Message": "A code fragment from 'StorageODBC.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSet.cpp", - "Message": "A code fragment from 'StorageSet.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageStripeLog.cpp", - "Message": "A code fragment from 'StorageStripeLog.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "VirtualColumnFactory.cpp", - "Message": "A code fragment from 'VirtualColumnFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageTinyLog.cpp", - "Message": "A code fragment from 'StorageTinyLog.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 630197398, - "CodeNext": 1056331065, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'replica_path' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 438297239, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'replica_path' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 72410316, - "CodeNext": 2690972308, - "CodePrev": 3645137672, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1038581435, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 4265130885, - "CodeNext": 123, - "CodePrev": 2362549484, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 423923019, - "CodeNext": 123, - "CodePrev": 3425255468, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1795911157, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1257284900, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 29091704, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2265033744, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 72410313, - "CodeNext": 123, - "CodePrev": 2533115030, - "ErrorCode": "V688", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1394960724, - "CodeNext": 2903347041, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "ReplicatedMergeTreeQuorumEntry.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: required_number_of_replicas." - }, - { - "CodeCurrent": 2903347041, - "CodeNext": 123, - "CodePrev": 1394960724, - "ErrorCode": "V730", - "FileName": "ReplicatedMergeTreeQuorumEntry.h", - "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: required_number_of_replicas." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageReplicatedMergeTree.cpp", - "Message": "A code fragment from 'StorageReplicatedMergeTree.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageView.cpp", - "Message": "A code fragment from 'StorageView.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerStorages.cpp", - "Message": "A code fragment from 'registerStorages.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "VirtualColumnUtils.cpp", - "Message": "A code fragment from 'VirtualColumnUtils.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getStructureOfRemoteTable.cpp", - "Message": "A code fragment from 'getStructureOfRemoteTable.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "transformQueryForExternalDatabase.cpp", - "Message": "A code fragment from 'transformQueryForExternalDatabase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ActiveDataPartSet.cpp", - "Message": "A code fragment from 'ActiveDataPartSet.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AllMergeSelector.cpp", - "Message": "A code fragment from 'AllMergeSelector.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3361369910, - "CodeNext": 123, - "CodePrev": 588179213, - "ErrorCode": "V658", - "FileName": "AbandonableLockInZooKeeper.cpp", - "Message": "A value is being subtracted from the unsigned variable. This can result in an overflow. In such a case, the '<' comparison operation can potentially behave unexpectedly. Consider inspecting the 'i < partitions.size() - holders.size()' expression." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AbandonableLockInZooKeeper.cpp", - "Message": "A code fragment from 'AbandonableLockInZooKeeper.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2664754306, - "CodeNext": 0, - "CodePrev": 1296085, - "ErrorCode": "V688", - "FileName": "DistributedBlockOutputStream.cpp", - "Message": "The 'query_string' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DistributedBlockOutputStream.cpp", - "Message": "A code fragment from 'DistributedBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1912588196, - "CodeNext": 979922832, - "CodePrev": 1045335318, - "ErrorCode": "V688", - "FileName": "DirectoryMonitor.cpp", - "Message": "The 'path' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'DirectoryMonitor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BackgroundProcessingPool.cpp", - "Message": "A code fragment from 'BackgroundProcessingPool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DiskSpaceMonitor.cpp", - "Message": "A code fragment from 'DiskSpaceMonitor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "LevelMergeSelector.cpp", - "Message": "A code fragment from 'LevelMergeSelector.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DataPartsExchange.cpp", - "Message": "A code fragment from 'DataPartsExchange.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3216131445, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V601", - "FileName": "KeyCondition.cpp", - "Message": "The 'true' value is implicitly cast to the integer type. Inspect the first argument." - }, - { - "CodeCurrent": 2137295876, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V601", - "FileName": "KeyCondition.cpp", - "Message": "The 'false' value is implicitly cast to the integer type. Inspect the first argument." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "KeyCondition.cpp", - "Message": "A code fragment from 'KeyCondition.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeList.cpp", - "Message": "A code fragment from 'MergeList.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1098322110, - "CodeNext": 279045268, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeBaseBlockInputStream.cpp", - "Message": "The 'max_block_size_rows' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 279045268, - "CodeNext": 2329935579, - "CodePrev": 1098322110, - "ErrorCode": "V688", - "FileName": "MergeTreeBaseBlockInputStream.cpp", - "Message": "The 'preferred_block_size_bytes' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2329935579, - "CodeNext": 3894021058, - "CodePrev": 279045268, - "ErrorCode": "V688", - "FileName": "MergeTreeBaseBlockInputStream.cpp", - "Message": "The 'preferred_max_column_in_block_size_bytes' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeBaseBlockInputStream.cpp", - "Message": "A code fragment from 'MergeTreeBaseBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeBlockInputStream.cpp", - "Message": "A code fragment from 'MergeTreeBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeBlockOutputStream.cpp", - "Message": "A code fragment from 'MergeTreeBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeBlockReadUtils.cpp", - "Message": "A code fragment from 'MergeTreeBlockReadUtils.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1550988213, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeDataPartChecksum.cpp", - "Message": "The 'full_checksums' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3214271742, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeDataPartChecksum.cpp", - "Message": "The 'full_checksums' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeDataPartChecksum.cpp", - "Message": "A code fragment from 'MergeTreeDataPartChecksum.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 123, - "CodeNext": 3380448882, - "CodePrev": 1441276206, - "ErrorCode": "V565", - "FileName": "MergeTreeData.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 2654040905, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeData.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2240934897, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeData.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeData.cpp", - "Message": "A code fragment from 'MergeTreeData.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 4013367774, - "CodeNext": 3698074834, - "CodePrev": 1857246075, - "ErrorCode": "V688", - "FileName": "MergeTreeDataMergerMutator.cpp", - "Message": "The 'data' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 896405418, - "CodeNext": 2464759539, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "MergeTreeDataMergerMutator.cpp", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: average_elem_progress." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeDataMergerMutator.cpp", - "Message": "A code fragment from 'MergeTreeDataMergerMutator.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3572738747, - "CodeNext": 123, - "CodePrev": 3750922986, - "ErrorCode": "V688", - "FileName": "MergeTreeDataPart.cpp", - "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1616975535, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeDataPart.cpp", - "Message": "The 'name' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2800774877, - "CodeNext": 403175306, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "MergeTreeDataPart.cpp", - "Message": "The 'columns' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeDataPart.cpp", - "Message": "A code fragment from 'MergeTreeDataPart.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreePartInfo.cpp", - "Message": "A code fragment from 'MergeTreePartInfo.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeDataWriter.cpp", - "Message": "A code fragment from 'MergeTreeDataWriter.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeDataSelectExecutor.cpp", - "Message": "A code fragment from 'MergeTreeDataSelectExecutor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreePartition.cpp", - "Message": "A code fragment from 'MergeTreePartition.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2300449615, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeRangeReader.cpp", - "Message": "The 'filter' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3630487317, - "CodeNext": 580147943, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeRangeReader.cpp", - "Message": "The 'filter' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeRangeReader.cpp", - "Message": "A code fragment from 'MergeTreeRangeReader.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeSettings.cpp", - "Message": "A code fragment from 'MergeTreeSettings.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 800849723, - "CodeNext": 355939354, - "CodePrev": 998672422, - "ErrorCode": "V688", - "FileName": "MergeTreeReadPool.cpp", - "Message": "The 'parts' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2641370323, - "CodeNext": 285699983, - "CodePrev": 807170999, - "ErrorCode": "V688", - "FileName": "MergeTreeReadPool.cpp", - "Message": "The 'parts' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1997550293, - "CodeNext": 0, - "CodePrev": 539, - "ErrorCode": "V601", - "FileName": "MergeTreeReadPool.cpp", - "Message": "The 'false' value is implicitly cast to the integer type. Inspect the first argument." - }, - { - "CodeCurrent": 2679702858, - "CodeNext": 123, - "CodePrev": 4119509742, - "ErrorCode": "V688", - "FileName": "MergeTreeReadPool.cpp", - "Message": "The 'parts' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeReadPool.cpp", - "Message": "A code fragment from 'MergeTreeReadPool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1269723304, - "CodeNext": 4205841567, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "MergeTreeReader.cpp", - "Message": "The 'all_mark_ranges' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeReader.cpp", - "Message": "A code fragment from 'MergeTreeReader.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1175987107, - "CodeNext": 123, - "CodePrev": 2568774401, - "ErrorCode": "V730", - "FileName": "ReplicatedMergeTreeAddress.h", - "Message": "It is possible that not all members of a class are initialized inside the constructor. Consider inspecting: replication_port, queries_port." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeAddress.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeAddress.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 763384319, - "CodeNext": 2422251580, - "CodePrev": 2529539154, - "ErrorCode": "V688", - "FileName": "MergeTreeThreadBlockInputStream.cpp", - "Message": "The 'max_block_size_rows' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 222867506, - "CodeNext": 163883724, - "CodePrev": 2381055092, - "ErrorCode": "V688", - "FileName": "MergeTreeThreadBlockInputStream.cpp", - "Message": "The 'storage' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeThreadBlockInputStream.cpp", - "Message": "A code fragment from 'MergeTreeThreadBlockInputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergeTreeWhereOptimizer.cpp", - "Message": "A code fragment from 'MergeTreeWhereOptimizer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MergedBlockOutputStream.cpp", - "Message": "A code fragment from 'MergedBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeLogEntry.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeLogEntry.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeAlterThread.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeAlterThread.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeBlockOutputStream.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeBlockOutputStream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeCleanupThread.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeCleanupThread.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeMutationEntry.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeMutationEntry.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SimpleMergeSelector.cpp", - "Message": "A code fragment from 'SimpleMergeSelector.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreePartCheckThread.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreePartCheckThread.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeQueue.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeQueue.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicatedMergeTreeRestartingThread.cpp", - "Message": "A code fragment from 'ReplicatedMergeTreeRestartingThread.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "checkDataPart.cpp", - "Message": "A code fragment from 'checkDataPart.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Connection.cpp", - "Message": "A code fragment from 'Connection.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 404950407, - "CodeNext": 125, - "CodePrev": 3626175879, - "ErrorCode": "V656", - "FileName": "ConnectionPoolWithFailover.cpp", - "Message": "Variables 'min_entries', 'max_entries' are initialized through the call to the same function. It's probably an error or un-optimized code. Consider inspecting the 'nested_pools.size()' expression. Check lines: _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ConnectionPoolWithFailover.cpp", - "Message": "A code fragment from 'ConnectionPoolWithFailover.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MultiplexedConnections.cpp", - "Message": "A code fragment from 'MultiplexedConnections.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerStorageMergeTree.cpp", - "Message": "A code fragment from 'registerStorageMergeTree.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionFactory.cpp", - "Message": "A code fragment from 'FunctionFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IFunction.cpp", - "Message": "A code fragment from 'IFunction.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionHelpers.cpp", - "Message": "A code fragment from 'FunctionHelpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionFactory.cpp", - "Message": "A code fragment from 'AggregateFunctionFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionCombinatorFactory.cpp", - "Message": "A code fragment from 'AggregateFunctionCombinatorFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FactoryHelpers.cpp", - "Message": "A code fragment from 'FactoryHelpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ITableFunction.cpp", - "Message": "A code fragment from 'ITableFunction.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parseAggregateFunctionParameters.cpp", - "Message": "A code fragment from 'parseAggregateFunctionParameters.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionState.cpp", - "Message": "A code fragment from 'AggregateFunctionState.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ActionLock.cpp", - "Message": "A code fragment from 'ActionLock.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CurrentMetrics.cpp", - "Message": "A code fragment from 'CurrentMetrics.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "BackgroundSchedulePool.cpp", - "Message": "A code fragment from 'BackgroundSchedulePool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Allocator.cpp", - "Message": "A code fragment from 'Allocator.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "DNSResolver.cpp", - "Message": "A code fragment from 'DNSResolver.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionFactory.cpp", - "Message": "A code fragment from 'TableFunctionFactory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 123, - "CodeNext": 125, - "CodePrev": 35837, - "ErrorCode": "V565", - "FileName": "Exception.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 143231, - "CodeNext": 125, - "CodePrev": 125, - "ErrorCode": "V565", - "FileName": "Exception.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 143231, - "CodeNext": 125, - "CodePrev": 125, - "ErrorCode": "V565", - "FileName": "Exception.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 143231, - "CodeNext": 125, - "CodePrev": 125, - "ErrorCode": "V565", - "FileName": "Exception.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 143231, - "CodeNext": 0, - "CodePrev": 125, - "ErrorCode": "V565", - "FileName": "Exception.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Exception.cpp", - "Message": "A code fragment from 'Exception.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Macros.cpp", - "Message": "A code fragment from 'Macros.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3496318343, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "FileChecker.cpp", - "Message": "The 'map' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FileChecker.cpp", - "Message": "A code fragment from 'FileChecker.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FieldVisitors.cpp", - "Message": "A code fragment from 'FieldVisitors.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1249266459, - "CodeNext": 125, - "CodePrev": 82483, - "ErrorCode": "V769", - "FileName": "SharedLibrary.cpp", - "Message": "The 'dlerror()' pointer in the expression could be nullptr. In such case, resulting value of arithmetic operations on this pointer will be senseless and it should not be used." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "SharedLibrary.cpp", - "Message": "A code fragment from 'SharedLibrary.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "RWLock.cpp", - "Message": "A code fragment from 'RWLock.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "config_build.cpp", - "Message": "A code fragment from 'config_build.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StackTrace.cpp", - "Message": "A code fragment from 'StackTrace.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "escapeForFileName.cpp", - "Message": "A code fragment from 'escapeForFileName.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MemoryTracker.cpp", - "Message": "A code fragment from 'MemoryTracker.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 348362777, - "CodeNext": 348528025, - "CodePrev": 897852770, - "ErrorCode": "V525", - "FileName": "ShellCommand.cpp", - "Message": "The code contains the collection of similar blocks. Check items 'write_fd', 'read_fd', 'read_fd' in lines _, _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ShellCommand.cpp", - "Message": "A code fragment from 'ShellCommand.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "formatIPv6.cpp", - "Message": "A code fragment from 'formatIPv_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getFQDNOrHostName.cpp", - "Message": "A code fragment from 'getFQDNOrHostName.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getMultipleKeysFromConfig.cpp", - "Message": "A code fragment from 'getMultipleKeysFromConfig.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getNumberOfPhysicalCPUCores.cpp", - "Message": "A code fragment from 'getNumberOfPhysicalCPUCores.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "formatReadable.cpp", - "Message": "A code fragment from 'formatReadable.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hex.cpp", - "Message": "A code fragment from 'hex.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "localBackup.cpp", - "Message": "A code fragment from 'localBackup.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "getPerformanceMessage.cpp", - "Message": "A code fragment from 'getPerformanceMessage.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "isLocalAddress.cpp", - "Message": "A code fragment from 'isLocalAddress.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "randomSeed.cpp", - "Message": "A code fragment from 'randomSeed.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "setThreadName.cpp", - "Message": "A code fragment from 'setThreadName.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parseAddress.cpp", - "Message": "A code fragment from 'parseAddress.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CascadeWriteBuffer.cpp", - "Message": "A code fragment from 'CascadeWriteBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CompressedReadBuffer.cpp", - "Message": "A code fragment from 'CompressedReadBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CachedCompressedReadBuffer.cpp", - "Message": "A code fragment from 'CachedCompressedReadBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CompressedReadBufferBase.cpp", - "Message": "A code fragment from 'CompressedReadBufferBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CompressedWriteBuffer.cpp", - "Message": "A code fragment from 'CompressedWriteBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CompressedReadBufferFromFile.cpp", - "Message": "A code fragment from 'CompressedReadBufferFromFile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CompressionSettings.cpp", - "Message": "A code fragment from 'CompressionSettings.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "HexWriteBuffer.cpp", - "Message": "A code fragment from 'HexWriteBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "HTTPCommon.cpp", - "Message": "A code fragment from 'HTTPCommon.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "HashingWriteBuffer.cpp", - "Message": "A code fragment from 'HashingWriteBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "LimitReadBuffer.cpp", - "Message": "A code fragment from 'LimitReadBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MemoryReadWriteBuffer.cpp", - "Message": "A code fragment from 'MemoryReadWriteBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterserverWriteBuffer.cpp", - "Message": "A code fragment from 'InterserverWriteBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Progress.cpp", - "Message": "A code fragment from 'Progress.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReadBufferFromFile.cpp", - "Message": "A code fragment from 'ReadBufferFromFile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 286621345, - "CodeNext": 0, - "CodePrev": 3895740, - "ErrorCode": "V688", - "FileName": "AIO.h", - "Message": "The 'id' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3733418444, - "CodeNext": 380034981, - "CodePrev": 729278174, - "ErrorCode": "V1007", - "FileName": "ReadBufferAIO.cpp", - "Message": "The value from the potentially uninitialized optional 'watch' is used. Probably it is a mistake." - }, - { - "CodeCurrent": 3788532949, - "CodeNext": 0, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "ReadBufferAIO.cpp", - "Message": "The 'metric_increment' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3788532949, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ReadBufferAIO.cpp", - "Message": "The 'metric_increment' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReadBufferAIO.cpp", - "Message": "A code fragment from 'ReadBufferAIO.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 743595643, - "CodeNext": 2465481192, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "ReadBufferFromFileBase.cpp", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: clock_type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReadBufferFromFileBase.cpp", - "Message": "A code fragment from 'ReadBufferFromFileBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReadBufferFromPocoSocket.cpp", - "Message": "A code fragment from 'ReadBufferFromPocoSocket.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3733418444, - "CodeNext": 380034981, - "CodePrev": 39251785, - "ErrorCode": "V1007", - "FileName": "ReadBufferFromFileDescriptor.cpp", - "Message": "The value from the potentially uninitialized optional 'watch' is used. Probably it is a mistake." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReadBufferFromFileDescriptor.cpp", - "Message": "A code fragment from 'ReadBufferFromFileDescriptor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3116768807, - "CodeNext": 3116768807, - "CodePrev": 127832661, - "ErrorCode": "V525", - "FileName": "ReadHelpers.cpp", - "Message": "The code contains the collection of similar blocks. Check items '_', '_', '_' in lines _, _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReadHelpers.cpp", - "Message": "A code fragment from 'ReadHelpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3282098908, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "WriteBufferAIO.cpp", - "Message": "The 'metric_increment' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferAIO.cpp", - "Message": "A code fragment from 'WriteBufferAIO.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2604535988, - "CodeNext": 1150311866, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ReadWriteBufferFromHTTP.cpp", - "Message": "The 'uri' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 376832871, - "CodeNext": 21716451, - "CodePrev": 3114652973, - "ErrorCode": "V688", - "FileName": "ReadWriteBufferFromHTTP.cpp", - "Message": "The 'timeouts' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReadWriteBufferFromHTTP.cpp", - "Message": "A code fragment from 'ReadWriteBufferFromHTTP.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferFromFile.cpp", - "Message": "A code fragment from 'WriteBufferFromFile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferFromFileBase.cpp", - "Message": "A code fragment from 'WriteBufferFromFileBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferFromPocoSocket.cpp", - "Message": "A code fragment from 'WriteBufferFromPocoSocket.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferFromHTTPServerResponse.cpp", - "Message": "A code fragment from 'WriteBufferFromHTTPServerResponse.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferFromTemporaryFile.cpp", - "Message": "A code fragment from 'WriteBufferFromTemporaryFile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferFromFileDescriptor.cpp", - "Message": "A code fragment from 'WriteBufferFromFileDescriptor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteBufferValidUTF8.cpp", - "Message": "A code fragment from 'WriteBufferValidUTF_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ZlibDeflatingWriteBuffer.cpp", - "Message": "A code fragment from 'ZlibDeflatingWriteBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ZlibInflatingReadBuffer.cpp", - "Message": "A code fragment from 'ZlibInflatingReadBuffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 622470420, - "CodeNext": 157897, - "CodePrev": 123, - "ErrorCode": "V525", - "FileName": "WriteHelpers.cpp", - "Message": "The code contains the collection of similar blocks. Check items '_', '_', '_', '_' in lines _, _, _, _." - }, - { - "CodeCurrent": 1208444161, - "CodeNext": 157897, - "CodePrev": 123, - "ErrorCode": "V525", - "FileName": "WriteHelpers.cpp", - "Message": "The code contains the collection of similar blocks. Check items '_', '_', '_' in lines _, _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "WriteHelpers.cpp", - "Message": "A code fragment from 'WriteHelpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "copyData.cpp", - "Message": "A code fragment from 'copyData.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "createWriteBufferFromFileBase.cpp", - "Message": "A code fragment from 'createWriteBufferFromFileBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "createReadBufferFromFileBase.cpp", - "Message": "A code fragment from 'createReadBufferFromFileBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parseDateTimeBestEffort.cpp", - "Message": "A code fragment from 'parseDateTimeBestEffort.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "readFloatText.cpp", - "Message": "A code fragment from 'readFloatText.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "gtest_unescapeForFileName.cpp", - "Message": "A code fragment from 'gtest_unescapeForFileName.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "gtest_rw_lock.cpp", - "Message": "A code fragment from 'gtest_rw_lock.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "gtest_zkutil_test_multi_exception.cpp", - "Message": "A code fragment from 'gtest_zkutil_test_multi_exception.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StringUtils.cpp", - "Message": "A code fragment from 'StringUtils.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3791607228, - "CodeNext": 60247417, - "CodePrev": 0, - "ErrorCode": "V522", - "FileName": "gtest_cascade_and_memory_write_buffer.cpp", - "Message": "There might be dereferencing of a potential null pointer 'wbuf_readable'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "gtest_cascade_and_memory_write_buffer.cpp", - "Message": "A code fragment from 'gtest_cascade_and_memory_write_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "stopwatch.cpp", - "Message": "A code fragment from 'stopwatch.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "allocator.cpp", - "Message": "A code fragment from 'allocator.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "thread_pool.cpp", - "Message": "A code fragment from 'thread_pool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "gtest_row_source_bits_test.cpp", - "Message": "A code fragment from 'gtest_row_source_bits_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "integer_hash_tables_and_hashes.cpp", - "Message": "A code fragment from 'integer_hash_tables_and_hashes.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3255795021, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "ArrayCache.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: ptr, size, chunk." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "array_cache.cpp", - "Message": "A code fragment from 'array_cache.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "thread_creation_latency.cpp", - "Message": "A code fragment from 'thread_creation_latency.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "lru_cache.cpp", - "Message": "A code fragment from 'lru_cache.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "sip_hash_perf.cpp", - "Message": "A code fragment from 'sip_hash_perf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "arena_with_free_lists.cpp", - "Message": "A code fragment from 'arena_with_free_lists.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "auto_array.cpp", - "Message": "A code fragment from 'auto_array.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "sip_hash.cpp", - "Message": "A code fragment from 'sip_hash.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hashes_test.cpp", - "Message": "A code fragment from 'hashes_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 178542355, - "CodeNext": 1913064305, - "CodePrev": 0, - "ErrorCode": "V760", - "FileName": "small_table.cpp", - "Message": "Two identical blocks of text were found. The second block begins from line _." - }, - { - "CodeCurrent": 3824136199, - "CodeNext": 2663, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "SmallTable.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: size." - }, - { - "CodeCurrent": 132655, - "CodeNext": 1795381173, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "SmallTable.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: container, ptr." - }, - { - "CodeCurrent": 9156143, - "CodeNext": 377744136, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "SmallTable.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: container, ptr." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "small_table.cpp", - "Message": "A code fragment from 'small_table.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parallel_aggregation.cpp", - "Message": "A code fragment from 'parallel_aggregation.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AvalancheTest.cpp", - "Message": "A code fragment from 'AvalancheTest.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parallel_aggregation2.cpp", - "Message": "A code fragment from 'parallel_aggregation_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 100335, - "CodeNext": 0, - "CodePrev": 123, - "ErrorCode": "V730", - "FileName": "SpaceSaving.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: slot, hash, count, error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "space_saving.cpp", - "Message": "A code fragment from 'space_saving.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "int_hashes_perf.cpp", - "Message": "A code fragment from 'int_hashes_perf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "simple_cache.cpp", - "Message": "A code fragment from 'simple_cache.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "cow_columns.cpp", - "Message": "A code fragment from 'cow_columns.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "pod_array.cpp", - "Message": "A code fragment from 'pod_array.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 178542355, - "CodeNext": 1913064305, - "CodePrev": 0, - "ErrorCode": "V760", - "FileName": "hash_table.cpp", - "Message": "Two identical blocks of text were found. The second block begins from line _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hash_table.cpp", - "Message": "A code fragment from 'hash_table.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 238596103, - "CodeNext": 2663, - "CodePrev": 5816, - "ErrorCode": "V730", - "FileName": "CompactArray.h", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: value_l, value_r, fits_in_byte." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "compact_array.cpp", - "Message": "A code fragment from 'compact_array.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1524853010, - "CodeNext": 193, - "CodePrev": 182716498, - "ErrorCode": "V524", - "FileName": "RadixSort.h", - "Message": "It is odd that the body of 'backward' function is fully equivalent to the body of 'forward' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "radix_sort.cpp", - "Message": "A code fragment from 'radix_sort.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "shell_command_test.cpp", - "Message": "A code fragment from 'shell_command_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "rvo_test.cpp", - "Message": "A code fragment from 'rvo_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "exception.cpp", - "Message": "A code fragment from 'exception.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "move_field.cpp", - "Message": "A code fragment from 'move_field.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "string_pool.cpp", - "Message": "A code fragment from 'string_pool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "string_ref_hash.cpp", - "Message": "A code fragment from 'string_ref_hash.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "field.cpp", - "Message": "A code fragment from 'field.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "filter_stream_hitlog.cpp", - "Message": "A code fragment from 'filter_stream_hitlog.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "filter_stream.cpp", - "Message": "A code fragment from 'filter_stream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "block_row_transforms.cpp", - "Message": "A code fragment from 'block_row_transforms.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "native_streams.cpp", - "Message": "A code fragment from 'native_streams.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "expression_stream.cpp", - "Message": "A code fragment from 'expression_stream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "tab_separated_streams.cpp", - "Message": "A code fragment from 'tab_separated_streams.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "sorting_stream.cpp", - "Message": "A code fragment from 'sorting_stream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "data_type_string.cpp", - "Message": "A code fragment from 'data_type_string.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "union_stream2.cpp", - "Message": "A code fragment from 'union_stream_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "collapsing_sorted_stream.cpp", - "Message": "A code fragment from 'collapsing_sorted_stream.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3646287644, - "CodeNext": 3487437108, - "CodePrev": 3633985876, - "ErrorCode": "V560", - "FileName": "data_type_get_common_type.cpp", - "Message": "A part of conditional expression is always false: !print_stack_trace." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "data_type_get_common_type.cpp", - "Message": "A code fragment from 'data_type_get_common_type.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "data_types_number_fixed.cpp", - "Message": "A code fragment from 'data_types_number_fixed.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemAsynchronousMetrics.cpp", - "Message": "A code fragment from 'StorageSystemAsynchronousMetrics.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemBuildOptions.cpp", - "Message": "A code fragment from 'StorageSystemBuildOptions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemClusters.cpp", - "Message": "A code fragment from 'StorageSystemClusters.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemDatabases.cpp", - "Message": "A code fragment from 'StorageSystemDatabases.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'StorageSystemDictionaries.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemColumns.cpp", - "Message": "A code fragment from 'StorageSystemColumns.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemEvents.cpp", - "Message": "A code fragment from 'StorageSystemEvents.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemFunctions.cpp", - "Message": "A code fragment from 'StorageSystemFunctions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1466671053, - "CodeNext": 1467990035, - "CodePrev": 1938442408, - "ErrorCode": "V601", - "FileName": "StorageSystemGraphite.cpp", - "Message": "The 'false' value is implicitly cast to the integer type." - }, - { - "CodeCurrent": 733335125, - "CodeNext": 2791142600, - "CodePrev": 1938442408, - "ErrorCode": "V601", - "FileName": "StorageSystemGraphite.cpp", - "Message": "The 'true' value is implicitly cast to the integer type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemGraphite.cpp", - "Message": "A code fragment from 'StorageSystemGraphite.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemMacros.cpp", - "Message": "A code fragment from 'StorageSystemMacros.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemMerges.cpp", - "Message": "A code fragment from 'StorageSystemMerges.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemMetrics.cpp", - "Message": "A code fragment from 'StorageSystemMetrics.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'StorageSystemModels.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemNumbers.cpp", - "Message": "A code fragment from 'StorageSystemNumbers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemOne.cpp", - "Message": "A code fragment from 'StorageSystemOne.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemParts.cpp", - "Message": "A code fragment from 'StorageSystemParts.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemProcesses.cpp", - "Message": "A code fragment from 'StorageSystemProcesses.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3754734131, - "CodeNext": 0, - "CodePrev": 3910557249, - "ErrorCode": "V688", - "FileName": "StorageSystemPartsBase.cpp", - "Message": "The 'database_column' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3833782239, - "CodeNext": 3076741407, - "CodePrev": 0, - "ErrorCode": "V525", - "FileName": "StorageSystemPartsBase.cpp", - "Message": "The code contains the collection of similar blocks. Check items 'ColumnString', 'ColumnString', 'ColumnUInt_' in lines _, _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemPartsBase.cpp", - "Message": "A code fragment from 'StorageSystemPartsBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemPartsColumns.cpp", - "Message": "A code fragment from 'StorageSystemPartsColumns.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemTables.cpp", - "Message": "A code fragment from 'StorageSystemTables.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemSettings.cpp", - "Message": "A code fragment from 'StorageSystemSettings.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemReplicas.cpp", - "Message": "A code fragment from 'StorageSystemReplicas.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemReplicationQueue.cpp", - "Message": "A code fragment from 'StorageSystemReplicationQueue.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "seek_speed_test.cpp", - "Message": "A code fragment from 'seek_speed_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StorageSystemZooKeeper.cpp", - "Message": "A code fragment from 'StorageSystemZooKeeper.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "part_name.cpp", - "Message": "A code fragment from 'part_name.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hit_log.cpp", - "Message": "A code fragment from 'hit_log.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "attachSystemTables.cpp", - "Message": "A code fragment from 'attachSystemTables.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "remove_symlink_directory.cpp", - "Message": "A code fragment from 'remove_symlink_directory.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "merge_selector2.cpp", - "Message": "A code fragment from 'merge_selector_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "storage_log.cpp", - "Message": "A code fragment from 'storage_log.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "part_checker.cpp", - "Message": "A code fragment from 'part_checker.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "get_current_inserts_in_replicated.cpp", - "Message": "A code fragment from 'get_current_inserts_in_replicated.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "system_numbers.cpp", - "Message": "A code fragment from 'system_numbers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTAlterQuery.cpp", - "Message": "A code fragment from 'ASTAlterQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTExpressionList.cpp", - "Message": "A code fragment from 'ASTExpressionList.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "merge_selector.cpp", - "Message": "A code fragment from 'merge_selector.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "get_abandonable_lock_in_all_partitions.cpp", - "Message": "A code fragment from 'get_abandonable_lock_in_all_partitions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTInsertQuery.cpp", - "Message": "A code fragment from 'ASTInsertQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTIdentifier.cpp", - "Message": "A code fragment from 'ASTIdentifier.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTFunction.cpp", - "Message": "A code fragment from 'ASTFunction.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTKillQueryQuery.cpp", - "Message": "A code fragment from 'ASTKillQueryQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTOrderByElement.cpp", - "Message": "A code fragment from 'ASTOrderByElement.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTLiteral.cpp", - "Message": "A code fragment from 'ASTLiteral.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTQualifiedAsterisk.cpp", - "Message": "A code fragment from 'ASTQualifiedAsterisk.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTPartition.cpp", - "Message": "A code fragment from 'ASTPartition.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTQueryWithOutput.cpp", - "Message": "A code fragment from 'ASTQueryWithOutput.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTSampleRatio.cpp", - "Message": "A code fragment from 'ASTSampleRatio.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTQueryWithOnCluster.cpp", - "Message": "A code fragment from 'ASTQueryWithOnCluster.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTSelectWithUnionQuery.cpp", - "Message": "A code fragment from 'ASTSelectWithUnionQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTSystemQuery.cpp", - "Message": "A code fragment from 'ASTSystemQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTSelectQuery.cpp", - "Message": "A code fragment from 'ASTSelectQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTTablesInSelectQuery.cpp", - "Message": "A code fragment from 'ASTTablesInSelectQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTSubquery.cpp", - "Message": "A code fragment from 'ASTSubquery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1818347258, - "CodeNext": 176909, - "CodePrev": 0, - "ErrorCode": "V526", - "FileName": "CommonParsers.cpp", - "Message": "The 'strncasecmp' function returns _ if corresponding strings are equal. Consider examining the condition for mistakes." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "CommonParsers.cpp", - "Message": "A code fragment from 'CommonParsers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ASTWithAlias.cpp", - "Message": "A code fragment from 'ASTWithAlias.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ExpressionListParsers.cpp", - "Message": "A code fragment from 'ExpressionListParsers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Lexer.cpp", - "Message": "A code fragment from 'Lexer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ExpressionElementParsers.cpp", - "Message": "A code fragment from 'ExpressionElementParsers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IParserBase.cpp", - "Message": "A code fragment from 'IParserBase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IAST.cpp", - "Message": "A code fragment from 'IAST.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserCheckQuery.cpp", - "Message": "A code fragment from 'ParserCheckQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserAlterQuery.cpp", - "Message": "A code fragment from 'ParserAlterQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserCase.cpp", - "Message": "A code fragment from 'ParserCase.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserDescribeTableQuery.cpp", - "Message": "A code fragment from 'ParserDescribeTableQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserCreateQuery.cpp", - "Message": "A code fragment from 'ParserCreateQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserDropQuery.cpp", - "Message": "A code fragment from 'ParserDropQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserKillQueryQuery.cpp", - "Message": "A code fragment from 'ParserKillQueryQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserInsertQuery.cpp", - "Message": "A code fragment from 'ParserInsertQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserOptimizeQuery.cpp", - "Message": "A code fragment from 'ParserOptimizeQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserQuery.cpp", - "Message": "A code fragment from 'ParserQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserPartition.cpp", - "Message": "A code fragment from 'ParserPartition.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserQueryWithOutput.cpp", - "Message": "A code fragment from 'ParserQueryWithOutput.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserRenameQuery.cpp", - "Message": "A code fragment from 'ParserRenameQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserSelectQuery.cpp", - "Message": "A code fragment from 'ParserSelectQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserSelectWithUnionQuery.cpp", - "Message": "A code fragment from 'ParserSelectWithUnionQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserSampleRatio.cpp", - "Message": "A code fragment from 'ParserSampleRatio.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserSetQuery.cpp", - "Message": "A code fragment from 'ParserSetQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserTablePropertiesQuery.cpp", - "Message": "A code fragment from 'ParserTablePropertiesQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserShowTablesQuery.cpp", - "Message": "A code fragment from 'ParserShowTablesQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserSystemQuery.cpp", - "Message": "A code fragment from 'ParserSystemQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TokenIterator.cpp", - "Message": "A code fragment from 'TokenIterator.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserTablesInSelectQuery.cpp", - "Message": "A code fragment from 'ParserTablesInSelectQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserUnionQueryElement.cpp", - "Message": "A code fragment from 'ParserUnionQueryElement.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ParserUseQuery.cpp", - "Message": "A code fragment from 'ParserUseQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "formatAST.cpp", - "Message": "A code fragment from 'formatAST.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parseDatabaseAndTableName.cpp", - "Message": "A code fragment from 'parseDatabaseAndTableName.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parseIdentifierOrStringLiteral.cpp", - "Message": "A code fragment from 'parseIdentifierOrStringLiteral.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "queryToString.cpp", - "Message": "A code fragment from 'queryToString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "iostream_debug_helpers.cpp", - "Message": "A code fragment from 'iostream_debug_helpers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parseQuery.cpp", - "Message": "A code fragment from 'parseQuery.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "create_parser.cpp", - "Message": "A code fragment from 'create_parser.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "select_parser.cpp", - "Message": "A code fragment from 'select_parser.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "lexer.cpp", - "Message": "A code fragment from 'lexer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parse_date_time_best_effort.cpp", - "Message": "A code fragment from 'parse_date_time_best_effort.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "limit_read_buffer2.cpp", - "Message": "A code fragment from 'limit_read_buffer_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zlib_buffers.cpp", - "Message": "A code fragment from 'zlib_buffers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "operators.cpp", - "Message": "A code fragment from 'operators.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "read_buffer_aio.cpp", - "Message": "A code fragment from 'read_buffer_aio.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hashing_read_buffer.cpp", - "Message": "A code fragment from 'hashing_read_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hashing_write_buffer.cpp", - "Message": "A code fragment from 'hashing_write_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "valid_utf8_perf.cpp", - "Message": "A code fragment from 'valid_utf__perf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "o_direct_and_dirty_pages.cpp", - "Message": "A code fragment from 'o_direct_and_dirty_pages.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "read_buffer.cpp", - "Message": "A code fragment from 'read_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "read_float_perf.cpp", - "Message": "A code fragment from 'read_float_perf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "compressed_buffer.cpp", - "Message": "A code fragment from 'compressed_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "write_buffer_perf.cpp", - "Message": "A code fragment from 'write_buffer_perf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "write_buffer.cpp", - "Message": "A code fragment from 'write_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "read_escaped_string.cpp", - "Message": "A code fragment from 'read_escaped_string.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "read_buffer_perf.cpp", - "Message": "A code fragment from 'read_buffer_perf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "limit_read_buffer.cpp", - "Message": "A code fragment from 'limit_read_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "cached_compressed_read_buffer.cpp", - "Message": "A code fragment from 'cached_compressed_read_buffer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "valid_utf8.cpp", - "Message": "A code fragment from 'valid_utf_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "mempbrk.cpp", - "Message": "A code fragment from 'mempbrk.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "var_uint.cpp", - "Message": "A code fragment from 'var_uint.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parse_int_perf2.cpp", - "Message": "A code fragment from 'parse_int_perf_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "write_buffer_aio.cpp", - "Message": "A code fragment from 'write_buffer_aio.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "async_write.cpp", - "Message": "A code fragment from 'async_write.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "parse_int_perf.cpp", - "Message": "A code fragment from 'parse_int_perf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "read_write_int.cpp", - "Message": "A code fragment from 'read_write_int.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsCharset.cpp", - "Message": "A code fragment from 'FunctionsCharset.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsCoding.cpp", - "Message": "A code fragment from 'FunctionsCoding.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2701979474, - "CodeNext": 125, - "CodePrev": 539, - "ErrorCode": "V522", - "FileName": "FunctionsArray.h", - "Message": "There might be dereferencing of a potential null pointer 'null_map_data'." - }, - { - "CodeCurrent": 2701979474, - "CodeNext": 125, - "CodePrev": 539, - "ErrorCode": "V522", - "FileName": "FunctionsArray.h", - "Message": "There might be dereferencing of a potential null pointer 'null_map_item'." - }, - { - "CodeCurrent": 2701979474, - "CodeNext": 125, - "CodePrev": 539, - "ErrorCode": "V522", - "FileName": "FunctionsArray.h", - "Message": "There might be dereferencing of a potential null pointer 'null_map_data'." - }, - { - "CodeCurrent": 2701979474, - "CodeNext": 125, - "CodePrev": 539, - "ErrorCode": "V522", - "FileName": "FunctionsArray.h", - "Message": "There might be dereferencing of a potential null pointer 'null_map_item'." - }, - { - "CodeCurrent": 1933433026, - "CodeNext": 352513873, - "CodePrev": 1066296242, - "ErrorCode": "V522", - "FileName": "FunctionsArray.cpp", - "Message": "There might be dereferencing of a potential null pointer 'col_const_array'." - }, - { - "CodeCurrent": 3479588408, - "CodeNext": 360719, - "CodePrev": 0, - "ErrorCode": "V519", - "FileName": "FunctionsArray.cpp", - "Message": "The 'col_array' variable is assigned values twice successively. Perhaps this is a mistake. Check lines: _, _." - }, - { - "CodeCurrent": 2226276340, - "CodeNext": 2372992004, - "CodePrev": 914871094, - "ErrorCode": "V601", - "FileName": "FunctionsArray.cpp", - "Message": "The 'false' value is implicitly cast to the integer type. Inspect the second argument." - }, - { - "CodeCurrent": 642252437, - "CodeNext": 1179013152, - "CodePrev": 123, - "ErrorCode": "V601", - "FileName": "FunctionsArray.cpp", - "Message": "The 'true' value is implicitly cast to the integer type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ValueSourceVisitor.h", - "Message": "A code fragment from 'FunctionsArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'FunctionsComparison.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsConsistentHashing.cpp", - "Message": "A code fragment from 'FunctionsConsistentHashing.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 780446959, - "CodeNext": 0, - "CodePrev": 1833977828, - "ErrorCode": "V728", - "FileName": "Algorithms.h", - "Message": "An excessive check can be simplified. The '||' operator is surrounded by opposite expressions '!size_null_map' and 'size_null_map'." - }, - { - "CodeCurrent": 2874700209, - "CodeNext": 0, - "CodePrev": 539, - "ErrorCode": "V522", - "FileName": "FunctionsConditional.h", - "Message": "There might be dereferencing of a potential null pointer 'col_right_const'." - }, - { - "CodeCurrent": 4053347441, - "CodeNext": 0, - "CodePrev": 539, - "ErrorCode": "V522", - "FileName": "FunctionsConditional.h", - "Message": "There might be dereferencing of a potential null pointer 'col_right_const'." - }, - { - "CodeCurrent": 1316020648, - "CodeNext": 67675834, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "FunctionsConditional.h", - "Message": "There might be dereferencing of a potential null pointer 'col_right_const_array'." - }, - { - "CodeCurrent": 1316020648, - "CodeNext": 67675834, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "FunctionsConditional.h", - "Message": "There might be dereferencing of a potential null pointer 'col_right_const_array'." - }, - { - "CodeCurrent": 3479214347, - "CodeNext": 328401, - "CodePrev": 0, - "ErrorCode": "V519", - "FileName": "FunctionsConditional.h", - "Message": "The 'col_left' variable is assigned values twice successively. Perhaps this is a mistake. Check lines: _, _." - }, - { - "CodeCurrent": 3244155324, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V524", - "FileName": "Sources.h", - "Message": "It is odd that the body of 'getColumnSize' function is fully equivalent to the body of 'getSizeForReserve' function." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ValueSourceVisitor.h", - "Message": "A code fragment from 'FunctionsConditional.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 71291086, - "CodeNext": 285164697, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "FunctionsConversion.h", - "Message": "There might be dereferencing of a potential null pointer 'col_from_string'." - }, - { - "CodeCurrent": 401577488, - "CodeNext": 3512469952, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "FunctionsConversion.h", - "Message": "There might be dereferencing of a potential null pointer 'col_from_fixed_string'." - }, - { - "CodeCurrent": 2536180031, - "CodeNext": 4123240192, - "CodePrev": 123, - "ErrorCode": "V522", - "FileName": "FunctionsConversion.h", - "Message": "There might be dereferencing of a potential null pointer 'offsets'." - }, - { - "CodeCurrent": 2581192971, - "CodeNext": 675433, - "CodePrev": 41145, - "ErrorCode": "V522", - "FileName": "FunctionsConversion.h", - "Message": "There might be dereferencing of a potential null pointer 'local_time_zone'." - }, - { - "CodeCurrent": 2581192971, - "CodeNext": 675433, - "CodePrev": 41145, - "ErrorCode": "V522", - "FileName": "FunctionsConversion.h", - "Message": "There might be dereferencing of a potential null pointer 'utc_time_zone'." - }, - { - "CodeCurrent": 1069551226, - "CodeNext": 125, - "CodePrev": 3401161010, - "ErrorCode": "V522", - "FileName": "FunctionsConversion.h", - "Message": "There might be dereferencing of a potential null pointer 'vec_null_map_to'." - }, - { - "CodeCurrent": 3903117136, - "CodeNext": 3272131897, - "CodePrev": 0, - "ErrorCode": "V788", - "FileName": "FunctionsConversion.h", - "Message": "Uninitialized variable 'from_nested_type' will be used in the lambda expression, as it was captured by value." - }, - { - "CodeCurrent": 3903117136, - "CodeNext": 3272131897, - "CodePrev": 0, - "ErrorCode": "V788", - "FileName": "FunctionsConversion.h", - "Message": "Uninitialized variable 'to_nested_type' will be used in the lambda expression, as it was captured by value." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsConversion.cpp", - "Message": "A code fragment from 'FunctionsConversion.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsDateTime.cpp", - "Message": "A code fragment from 'FunctionsDateTime.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsEmbeddedDictionaries.cpp", - "Message": "A code fragment from 'FunctionsEmbeddedDictionaries.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsFindCluster.cpp", - "Message": "A code fragment from 'FunctionsFindCluster.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'FunctionsExternalModels.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "IExternalLoadable.h", - "Message": "A code fragment from 'FunctionsExternalDictionaries.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsFormatting.cpp", - "Message": "A code fragment from 'FunctionsFormatting.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsHashing.cpp", - "Message": "A code fragment from 'FunctionsHashing.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsHigherOrder.cpp", - "Message": "A code fragment from 'FunctionsHigherOrder.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'FunctionsLogical.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsMath.cpp", - "Message": "A code fragment from 'FunctionsMath.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1915387808, - "CodeNext": 123, - "CodePrev": 388146916, - "ErrorCode": "V688", - "FileName": "GeoUtils.h", - "Message": "The 'polygon' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'FunctionsGeo.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'FunctionsNull.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsMiscellaneous.cpp", - "Message": "A code fragment from 'FunctionsMiscellaneous.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsProjection.cpp", - "Message": "A code fragment from 'FunctionsProjection.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsRandom.cpp", - "Message": "A code fragment from 'FunctionsRandom.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsReinterpret.cpp", - "Message": "A code fragment from 'FunctionsReinterpret.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ValueSourceVisitor.h", - "Message": "A code fragment from 'FunctionsString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsStringArray.cpp", - "Message": "A code fragment from 'FunctionsStringArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3108965974, - "CodeNext": 4119542900, - "CodePrev": 5816, - "ErrorCode": "V688", - "FileName": "StringSearcher.h", - "Message": "The 'needle_size' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3108965974, - "CodeNext": 4119542900, - "CodePrev": 5816, - "ErrorCode": "V688", - "FileName": "StringSearcher.h", - "Message": "The 'needle_size' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3108965974, - "CodeNext": 4119542900, - "CodePrev": 5816, - "ErrorCode": "V688", - "FileName": "StringSearcher.h", - "Message": "The 'needle_size' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 4259932362, - "CodeNext": 4031099091, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "Volnitsky.h", - "Message": "The 'needle_size' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2578531992, - "CodeNext": 1724192963, - "CodePrev": 3112257027, - "ErrorCode": "V519", - "FileName": "Volnitsky.h", - "Message": "The 'chars.c_' variable is assigned values twice successively. Perhaps this is a mistake. Check lines: _, _." - }, - { - "CodeCurrent": 1724192963, - "CodeNext": 371674977, - "CodePrev": 2578531992, - "ErrorCode": "V519", - "FileName": "Volnitsky.h", - "Message": "The 'chars.c_' variable is assigned values twice successively. Perhaps this is a mistake. Check lines: _, _." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'FunctionsStringSearch.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsTransform.cpp", - "Message": "A code fragment from 'FunctionsTransform.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsTuple.cpp", - "Message": "A code fragment from 'FunctionsTuple.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'FunctionsURL.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 88117, - "CodeNext": 125, - "CodePrev": 123, - "ErrorCode": "V601", - "FileName": "FunctionsVisitParam.h", - "Message": "The 'true' value is implicitly cast to the integer type." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'FunctionsVisitParam.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionAbs.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArray.cpp", - "Message": "A code fragment from 'registerFunctionArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayConcat.cpp", - "Message": "A code fragment from 'registerFunctionArrayConcat.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayElement.cpp", - "Message": "A code fragment from 'registerFunctionArrayElement.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayEnumerate.cpp", - "Message": "A code fragment from 'registerFunctionArrayEnumerate.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayHasAll.cpp", - "Message": "A code fragment from 'registerFunctionArrayHasAll.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayEnumerateUniq.cpp", - "Message": "A code fragment from 'registerFunctionArrayEnumerateUniq.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayHasAny.cpp", - "Message": "A code fragment from 'registerFunctionArrayHasAny.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayIntersect.cpp", - "Message": "A code fragment from 'registerFunctionArrayIntersect.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayPopBack.cpp", - "Message": "A code fragment from 'registerFunctionArrayPopBack.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayPopFront.cpp", - "Message": "A code fragment from 'registerFunctionArrayPopFront.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayPushFront.cpp", - "Message": "A code fragment from 'registerFunctionArrayPushFront.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayPushBack.cpp", - "Message": "A code fragment from 'registerFunctionArrayPushBack.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayReduce.cpp", - "Message": "A code fragment from 'registerFunctionArrayReduce.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayReverse.cpp", - "Message": "A code fragment from 'registerFunctionArrayReverse.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArraySlice.cpp", - "Message": "A code fragment from 'registerFunctionArraySlice.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayResize.cpp", - "Message": "A code fragment from 'registerFunctionArrayResize.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionArrayUniq.cpp", - "Message": "A code fragment from 'registerFunctionArrayUniq.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitAnd.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitNot.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitRotateRight.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitOr.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitRotateLeft.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitShiftLeft.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitTest.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitShiftRight.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitTestAny.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitXor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionBitTestAll.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'FunctionsRound.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionCountEqual.cpp", - "Message": "A code fragment from 'registerFunctionCountEqual.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionDivideFloating.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionBuildProjectionComposition.cpp", - "Message": "A code fragment from 'registerFunctionBuildProjectionComposition.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionDivideIntegral.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionDivideIntegralOrZero.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayDate.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayDate.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayDateTime.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayDateTime.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayFloat32.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayFloat_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayInt16.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayFloat64.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayFloat_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayInt64.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayInt32.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayInt8.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayString.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayString.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayToSingle.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayToSingle.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayUInt16.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayUInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayUInt32.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayUInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayUInt64.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayUInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionEmptyArrayUInt8.cpp", - "Message": "A code fragment from 'registerFunctionEmptyArrayUInt_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionGCD.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionGreatest.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionHas.cpp", - "Message": "A code fragment from 'registerFunctionHas.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionIndexOf.cpp", - "Message": "A code fragment from 'registerFunctionIndexOf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionIntExp_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionIntExp_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionLCM.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionLeast.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionMinus.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionModulo.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionMultiply.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionNegate.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionOneOrZero.cpp", - "Message": "A code fragment from 'registerFunctionOneOrZero.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionPlus.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionRange.cpp", - "Message": "A code fragment from 'registerFunctionRange.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionProject.cpp", - "Message": "A code fragment from 'registerFunctionProject.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctions.cpp", - "Message": "A code fragment from 'registerFunctions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionRestoreProjection.cpp", - "Message": "A code fragment from 'registerFunctionRestoreProjection.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionsArray.cpp", - "Message": "A code fragment from 'registerFunctionsArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "FunctionsArithmetic.h", - "Message": "A code fragment from 'registerFunctionsArithmetic.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1835696802, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "concat.cpp", - "Message": "The 'sink' argument is a non-constant reference. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'concat.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerFunctionsProjection.cpp", - "Message": "A code fragment from 'registerFunctionsProjection.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySinkVisitor.h", - "Message": "A code fragment from 'createArraySource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySinkVisitor.h", - "Message": "A code fragment from 'createArraySink.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySinkVisitor.h", - "Message": "A code fragment from 'createValueSource.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 412830845, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "has.cpp", - "Message": "The 'first', 'second', 'result' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'has.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3069352387, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "resizeConstantSize.cpp", - "Message": "The 'array_source', 'value_source', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'resizeConstantSize.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 652442793, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "push.cpp", - "Message": "The 'array_source', 'value_source', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'push.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3240218462, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "sliceDynamicOffsetBounded.cpp", - "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'sliceDynamicOffsetBounded.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1330432147, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "resizeDynamicSize.cpp", - "Message": "The 'array_source', 'value_source', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'resizeDynamicSize.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3966500669, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "sliceDynamicOffsetUnbounded.cpp", - "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'sliceDynamicOffsetUnbounded.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2727169871, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "sliceFromLeftConstantOffsetBounded.cpp", - "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'sliceFromLeftConstantOffsetBounded.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "number_traits.cpp", - "Message": "A code fragment from 'number_traits.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 566054038, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "sliceFromRightConstantOffsetBounded.cpp", - "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'sliceFromRightConstantOffsetBounded.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1515238783, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "sliceFromLeftConstantOffsetUnbounded.cpp", - "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'sliceFromLeftConstantOffsetUnbounded.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1741194879, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V669", - "FileName": "sliceFromRightConstantOffsetUnbounded.cpp", - "Message": "The 'src', 'sink' arguments are non-constant references. The analyzer is unable to determine the position at which this argument is being modified. It is possible that the function contains an error." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ArraySourceVisitor.h", - "Message": "A code fragment from 'sliceFromRightConstantOffsetUnbounded.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hash_map.cpp", - "Message": "A code fragment from 'hash_map.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1419470395, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V524", - "FileName": "hash_map_string_2.cpp", - "Message": "It is odd that the body of 'compare_byIntSSE' function is fully equivalent to the body of 'compareSSE_' function (StringRef.h, line _)." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hash_map_string_2.cpp", - "Message": "A code fragment from 'hash_map_string__.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hash_map3.cpp", - "Message": "A code fragment from 'hash_map_.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "select_query.cpp", - "Message": "A code fragment from 'select_query.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "aggregate.cpp", - "Message": "A code fragment from 'aggregate.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hash_map_string_3.cpp", - "Message": "A code fragment from 'hash_map_string__.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 15386799, - "CodeNext": 0, - "CodePrev": 193061534, - "ErrorCode": "V730", - "FileName": "hash_map_string_small.cpp", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: size." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hash_map_string_small.cpp", - "Message": "A code fragment from 'hash_map_string_small.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "two_level_hash_map.cpp", - "Message": "A code fragment from 'two_level_hash_map.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "expression.cpp", - "Message": "A code fragment from 'expression.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "compiler_test.cpp", - "Message": "A code fragment from 'compiler_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "create_query.cpp", - "Message": "A code fragment from 'create_query.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "logical_expressions_optimizer.cpp", - "Message": "A code fragment from 'logical_expressions_optimizer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "in_join_subqueries_preprocessor.cpp", - "Message": "A code fragment from 'in_join_subqueries_preprocessor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "hash_map_string.cpp", - "Message": "A code fragment from 'hash_map_string.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "users.cpp", - "Message": "A code fragment from 'users.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionArray.cpp", - "Message": "A code fragment from 'AggregateFunctionArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionAvg.cpp", - "Message": "A code fragment from 'AggregateFunctionAvg.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionBitwise.cpp", - "Message": "A code fragment from 'AggregateFunctionBitwise.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionCount.cpp", - "Message": "A code fragment from 'AggregateFunctionCount.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionForEach.cpp", - "Message": "A code fragment from 'AggregateFunctionForEach.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionGroupArray.cpp", - "Message": "A code fragment from 'AggregateFunctionGroupArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionGroupArrayInsertAt.cpp", - "Message": "A code fragment from 'AggregateFunctionGroupArrayInsertAt.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionGroupUniqArray.cpp", - "Message": "A code fragment from 'AggregateFunctionGroupUniqArray.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionIf.cpp", - "Message": "A code fragment from 'AggregateFunctionIf.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionMerge.cpp", - "Message": "A code fragment from 'AggregateFunctionMerge.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionMaxIntersections.cpp", - "Message": "A code fragment from 'AggregateFunctionMaxIntersections.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2360632089, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V522", - "FileName": "AggregateFunctionNull.cpp", - "Message": "There might be dereferencing of a potential null pointer 'nested_function'." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionNull.cpp", - "Message": "A code fragment from 'AggregateFunctionNull.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 2106398958, - "CodeNext": 123, - "CodePrev": 123, - "ErrorCode": "V730", - "FileName": "AggregateFunctionMinMaxAny.h", - "Message": "Not all members of a class are initialized inside the compiler generated constructor. Consider inspecting: large_data, small_data." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionMinMaxAny.cpp", - "Message": "A code fragment from 'AggregateFunctionMinMaxAny.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionStatistics.cpp", - "Message": "A code fragment from 'AggregateFunctionStatistics.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionSequenceMatch.cpp", - "Message": "A code fragment from 'AggregateFunctionSequenceMatch.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 707089, - "CodeNext": 3039935502, - "CodePrev": 123, - "ErrorCode": "V550", - "FileName": "QuantileTiming.h", - "Message": "An odd precise comparison: level != _. It's probably better to use a comparison with defined precision: fabs(A - B) > Epsilon." - }, - { - "CodeCurrent": 407249, - "CodeNext": 172219641, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "QuantileTiming.h", - "Message": "The 'count' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionQuantile.cpp", - "Message": "A code fragment from 'AggregateFunctionQuantile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionSum.cpp", - "Message": "A code fragment from 'AggregateFunctionSum.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionSumMap.cpp", - "Message": "A code fragment from 'AggregateFunctionSumMap.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionTopK.cpp", - "Message": "A code fragment from 'AggregateFunctionTopK.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionUniq.cpp", - "Message": "A code fragment from 'AggregateFunctionUniq.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionUniqUpTo.cpp", - "Message": "A code fragment from 'AggregateFunctionUniqUpTo.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "UniqCombinedBiasData.cpp", - "Message": "A code fragment from 'UniqCombinedBiasData.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerAggregateFunctions.cpp", - "Message": "A code fragment from 'registerAggregateFunctions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionWindowFunnel.cpp", - "Message": "A code fragment from 'AggregateFunctionWindowFunnel.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "AggregateFunctionsStatisticsSimple.cpp", - "Message": "A code fragment from 'AggregateFunctionsStatisticsSimple.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'ExtractFromConfig.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'Benchmark.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1973016635, - "CodeNext": 0, - "CodePrev": 1667937151, - "ErrorCode": "V688", - "FileName": "ClusterCopier.cpp", - "Message": "The 'pid' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 158502902, - "CodeNext": 73185914, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "ClusterCopier.cpp", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: task_descprtion_current_stat." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'ClusterCopier.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 254105, - "CodeNext": 8134297, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "Client.cpp", - "Message": "The 'query' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 254105, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "Client.cpp", - "Message": "The 'query' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 862425183, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V730", - "FileName": "Client.cpp", - "Message": "Not all members of a class are initialized inside the constructor. Consider inspecting: port, security, compression." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'Client.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'HTTPHandler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "NotFoundHandler.cpp", - "Message": "A code fragment from 'NotFoundHandler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "InterserverIOHTTPHandler.cpp", - "Message": "A code fragment from 'InterserverIOHTTPHandler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "MetricsTransmitter.cpp", - "Message": "A code fragment from 'MetricsTransmitter.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "PingRequestHandler.cpp", - "Message": "A code fragment from 'PingRequestHandler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "RootRequestHandler.cpp", - "Message": "A code fragment from 'RootRequestHandler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "StatusFile.cpp", - "Message": "A code fragment from 'StatusFile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ReplicasStatusHandler.cpp", - "Message": "A code fragment from 'ReplicasStatusHandler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Server.cpp", - "Message": "A code fragment from 'Server.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 1810641279, - "CodeNext": 2826637927, - "CodePrev": 123, - "ErrorCode": "V1002", - "FileName": "TCPHandler.cpp", - "Message": "The 'Context' class, containing pointers, constructor and destructor, is copied by the automatically generated operator=." - }, - { - "CodeCurrent": 2826637927, - "CodeNext": 0, - "CodePrev": 1810641279, - "ErrorCode": "V678", - "FileName": "TCPHandler.cpp", - "Message": "An object is used as an argument to its own method. Consider checking the first actual argument of the 'setSessionContext' function." - }, - { - "CodeCurrent": 143231, - "CodeNext": 0, - "CodePrev": 125, - "ErrorCode": "V565", - "FileName": "TCPHandler.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 4294129027, - "CodeNext": 0, - "CodePrev": 4133442322, - "ErrorCode": "V1002", - "FileName": "TCPHandler.cpp", - "Message": "The 'Context' class, containing pointers, constructor and destructor, is copied by the automatically generated operator=." - }, - { - "CodeCurrent": 1787830936, - "CodeNext": 123, - "CodePrev": 125, - "ErrorCode": "V759", - "FileName": "TCPHandler.cpp", - "Message": "Incorrect order of exception handling blocks. Exception of 'NetException' type will be caught in a catch block for 'Exception' base type." - }, - { - "CodeCurrent": 123, - "CodeNext": 2434086205, - "CodePrev": 35837, - "ErrorCode": "V565", - "FileName": "TCPHandler.cpp", - "Message": "An empty exception handler. Silent suppression of exceptions can hide the presence of bugs in source code during testing." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'TCPHandler.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3472545402, - "CodeNext": 123, - "CodePrev": 125, - "ErrorCode": "V688", - "FileName": "PerformanceTest.cpp", - "Message": "The 'padding' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 188108971, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V550", - "FileName": "PerformanceTest.cpp", - "Message": "An odd precise comparison: avg_speed_first == _. It's probably better to use a comparison with defined precision: fabs(A - B) < Epsilon." - }, - { - "CodeCurrent": 876333961, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "PerformanceTest.cpp", - "Message": "The 'input_files' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2173845642, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "PerformanceTest.cpp", - "Message": "The 'substitutions' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 596635228, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "PerformanceTest.cpp", - "Message": "The 'substitutions' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1500859456, - "CodeNext": 0, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "PerformanceTest.cpp", - "Message": "The 'queries' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3001731668, - "CodeNext": 3796428353, - "CodePrev": 388618255, - "ErrorCode": "V688", - "FileName": "PerformanceTest.cpp", - "Message": "The 'queries' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 2531644379, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "PerformanceTest.cpp", - "Message": "The 'main_metric' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'PerformanceTest.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'Compressor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 35857945, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "LocalServer.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 201131425, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "LocalServer.cpp", - "Message": "The 'context' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'LocalServer.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'Format.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionCatBoostPool.cpp", - "Message": "A code fragment from 'TableFunctionCatBoostPool.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3790144045, - "CodeNext": 100, - "CodePrev": 123, - "ErrorCode": "V601", - "FileName": "main.cpp", - "Message": "The 'false' value is implicitly cast to the integer type. Inspect the second argument." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "main.cpp", - "Message": "A code fragment from 'main.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'TableFunctionFile.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionMerge.cpp", - "Message": "A code fragment from 'TableFunctionMerge.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionNumbers.cpp", - "Message": "A code fragment from 'TableFunctionNumbers.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionMySQL.cpp", - "Message": "A code fragment from 'TableFunctionMySQL.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "registerTableFunctions.cpp", - "Message": "A code fragment from 'registerTableFunctions.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionODBC.cpp", - "Message": "A code fragment from 'TableFunctionODBC.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "translate_positional_arguments.cpp", - "Message": "A code fragment from 'translate_positional_arguments.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionRemote.cpp", - "Message": "A code fragment from 'TableFunctionRemote.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "TableFunctionShardByHash.cpp", - "Message": "A code fragment from 'TableFunctionShardByHash.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "collect_aliases.cpp", - "Message": "A code fragment from 'collect_aliases.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "analyze_result_of_query.cpp", - "Message": "A code fragment from 'analyze_result_of_query.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "type_and_constant_inference.cpp", - "Message": "A code fragment from 'type_and_constant_inference.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "analyze_columns.cpp", - "Message": "A code fragment from 'analyze_columns.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "collect_tables.cpp", - "Message": "A code fragment from 'collect_tables.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "Lock.cpp", - "Message": "A code fragment from 'Lock.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "analyze_lambdas.cpp", - "Message": "A code fragment from 'analyze_lambdas.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "optimize_group_order_limit_by.cpp", - "Message": "A code fragment from 'optimize_group_order_limit_by.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ZooKeeperHolder.cpp", - "Message": "A code fragment from 'ZooKeeperHolder.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ZooKeeperNodeCache.cpp", - "Message": "A code fragment from 'ZooKeeperNodeCache.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 4277102732, - "CodeNext": 3079148690, - "CodePrev": 123, - "ErrorCode": "V688", - "FileName": "ZooKeeperImpl.cpp", - "Message": "The 'out' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 20953377, - "CodeNext": 586135571, - "CodePrev": 4018551741, - "ErrorCode": "V688", - "FileName": "ZooKeeperImpl.cpp", - "Message": "The 'session_id' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 6647, - "CodeNext": 81911, - "CodePrev": 325699, - "ErrorCode": "V688", - "FileName": "ZooKeeperImpl.cpp", - "Message": "The 'xid' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 6647, - "CodeNext": 81911, - "CodePrev": 325699, - "ErrorCode": "V688", - "FileName": "ZooKeeperImpl.cpp", - "Message": "The 'xid' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 163059, - "CodeNext": 0, - "CodePrev": 18353, - "ErrorCode": "V688", - "FileName": "ZooKeeperImpl.cpp", - "Message": "The 'error' local variable possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ZooKeeperImpl.cpp", - "Message": "A code fragment from 'ZooKeeperImpl.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zookeeper_impl.cpp", - "Message": "A code fragment from 'zookeeper_impl.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 3719570876, - "CodeNext": 67617425, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ZooKeeper.cpp", - "Message": "The 'hosts' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3719570876, - "CodeNext": 67617425, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ZooKeeper.cpp", - "Message": "The 'identity' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 67617425, - "CodeNext": 123, - "CodePrev": 3719570876, - "ErrorCode": "V688", - "FileName": "ZooKeeper.cpp", - "Message": "The 'session_timeout_ms' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 67617425, - "CodeNext": 123, - "CodePrev": 3719570876, - "ErrorCode": "V688", - "FileName": "ZooKeeper.cpp", - "Message": "The 'chroot' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 1028915392, - "CodeNext": 2727151059, - "CodePrev": 0, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 1028915392, - "CodeNext": 2727151059, - "CodePrev": 0, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 1530511028, - "CodeNext": 3013342653, - "CodePrev": 1169372895, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 3013342653, - "CodeNext": 2123395508, - "CodePrev": 1530511028, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 2123395508, - "CodeNext": 2727151059, - "CodePrev": 3013342653, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 3807645422, - "CodeNext": 5607, - "CodePrev": 0, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 1530511028, - "CodeNext": 3013328497, - "CodePrev": 1169372895, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 3013328497, - "CodeNext": 1827075214, - "CodePrev": 1530511028, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 1827075214, - "CodeNext": 2727151059, - "CodePrev": 3013328497, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 1028915392, - "CodeNext": 2727151059, - "CodePrev": 0, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 1028915392, - "CodeNext": 2727151059, - "CodePrev": 0, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 1530511028, - "CodeNext": 3013328526, - "CodePrev": 1169372895, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 3013328526, - "CodeNext": 2727151059, - "CodePrev": 1530511028, - "ErrorCode": "V560", - "FileName": "ZooKeeper.cpp", - "Message": "A part of conditional expression is always false." - }, - { - "CodeCurrent": 3365467161, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ZooKeeper.cpp", - "Message": "The 'code' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 3365467161, - "CodeNext": 123, - "CodePrev": 0, - "ErrorCode": "V688", - "FileName": "ZooKeeper.cpp", - "Message": "The 'responses' function argument possesses the same name as one of the class members, which can result in a confusion." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'ZooKeeper.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zkutil_expiration_test.cpp", - "Message": "A code fragment from 'zkutil_expiration_test.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zkutil_zookeeper_holder.cpp", - "Message": "A code fragment from 'zkutil_zookeeper_holder.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zkutil_test_lock.cpp", - "Message": "A code fragment from 'zkutil_test_lock.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zkutil_test_commands.cpp", - "Message": "A code fragment from 'zkutil_test_commands.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zk_many_watches_reconnect.cpp", - "Message": "A code fragment from 'zk_many_watches_reconnect.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ConfigProcessor.cpp", - "Message": "A code fragment from 'ConfigProcessor.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "zkutil_test_async.cpp", - "Message": "A code fragment from 'zkutil_test_async.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "variant", - "Message": "A code fragment from 'zkutil_test_commands_new_lib.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "dictionary_library.cpp", - "Message": "A code fragment from 'dictionary_library.cpp' cannot be analyzed." - }, - { - "CodeCurrent": 0, - "CodeNext": 0, - "CodePrev": 0, - "ErrorCode": "V001", - "FileName": "ConfigReloader.cpp", - "Message": "A code fragment from 'ConfigReloader.cpp' cannot be analyzed." - } - ] -} From a7d3724d082eaca1b498d6e6cf5be68642ee1408 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 2 Dec 2018 22:37:01 +0300 Subject: [PATCH 61/69] Explicit specification of static linkage for standard, system and compiler libraries on clang #3706 --- CMakeLists.txt | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 6b507c57a59..3972cd66ca0 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -174,7 +174,8 @@ if (OS_LINUX AND COMPILER_CLANG) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") # Ok for clang6, for older can cause 'not used option' warning set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=0") # More checks in debug build. if (MAKE_STATIC_LIBRARIES) - link_libraries (-Wl,-Bstatic -stdlib=libc++ c++ c++abi -Wl,-Bdynamic) + execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-${CMAKE_SYSTEM_PROCESSOR}.a OUTPUT_VARIABLE BUILTINS_LIB_PATH OUTPUT_STRIP_TRAILING_WHITESPACE) + link_libraries (-nodefaultlibs -Wl,-Bstatic -stdlib=libc++ c++ c++abi gcc_eh ${BUILTINS_LIB_PATH} rt -Wl,-Bdynamic dl pthread m c) else () link_libraries (-stdlib=libc++ c++ c++abi) endif () @@ -186,11 +187,17 @@ if (OS_LINUX AND COMPILER_CLANG) endif () endif () +if (COMPILER_GCC) + set (STATIC_STDLIB_FLAGS "-static-libgcc -static-libstdc++") +else () + set (STATIC_STDLIB_FLAGS "") +endif () + if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (COMPILER_CLANG AND OS_FREEBSD)) - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libgcc -static-libstdc++") + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${STATIC_STDLIB_FLAGS}") # Along with executables, we also build example of shared library for "library dictionary source"; and it also should be self-contained. - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -static-libgcc -static-libstdc++") + set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} ${STATIC_STDLIB_FLAGS}") endif () if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE) From f6c264b70bc5c7f7c11674a4bbe809b3d91a6622 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Dec 2018 06:17:06 +0300 Subject: [PATCH 62/69] Every function in its own file, part 9 (#3733) * Every function in its own file, part 9 [#CLICKHOUSE-2] * Every function in its own file, part 9 [#CLICKHOUSE-2] --- ...ionsMath.h => FunctionMathBinaryFloat64.h} | 233 +----------------- dbms/src/Functions/FunctionMathConstFloat64.h | 34 +++ dbms/src/Functions/FunctionMathUnaryFloat64.h | 182 ++++++++++++++ dbms/src/Functions/FunctionsMath.cpp | 39 --- dbms/src/Functions/acos.cpp | 15 ++ dbms/src/Functions/asin.cpp | 15 ++ dbms/src/Functions/atan.cpp | 15 ++ dbms/src/Functions/cbrt.cpp | 15 ++ dbms/src/Functions/cos.cpp | 15 ++ dbms/src/Functions/e.cpp | 20 ++ dbms/src/Functions/erf.cpp | 15 ++ dbms/src/Functions/erfc.cpp | 15 ++ dbms/src/Functions/exp.cpp | 15 ++ dbms/src/Functions/exp10.cpp | 23 ++ dbms/src/Functions/exp2.cpp | 15 ++ dbms/src/Functions/lgamma.cpp | 15 ++ dbms/src/Functions/log.cpp | 16 ++ dbms/src/Functions/log10.cpp | 15 ++ dbms/src/Functions/log2.cpp | 15 ++ dbms/src/Functions/pi.cpp | 20 ++ dbms/src/Functions/pow.cpp | 16 ++ dbms/src/Functions/registerFunctionsMath.cpp | 54 ++++ dbms/src/Functions/sin.cpp | 15 ++ dbms/src/Functions/sqrt.cpp | 15 ++ dbms/src/Functions/tan.cpp | 15 ++ dbms/src/Functions/tgamma.cpp | 15 ++ 26 files changed, 608 insertions(+), 269 deletions(-) rename dbms/src/Functions/{FunctionsMath.h => FunctionMathBinaryFloat64.h} (53%) create mode 100644 dbms/src/Functions/FunctionMathConstFloat64.h create mode 100644 dbms/src/Functions/FunctionMathUnaryFloat64.h delete mode 100644 dbms/src/Functions/FunctionsMath.cpp create mode 100644 dbms/src/Functions/acos.cpp create mode 100644 dbms/src/Functions/asin.cpp create mode 100644 dbms/src/Functions/atan.cpp create mode 100644 dbms/src/Functions/cbrt.cpp create mode 100644 dbms/src/Functions/cos.cpp create mode 100644 dbms/src/Functions/e.cpp create mode 100644 dbms/src/Functions/erf.cpp create mode 100644 dbms/src/Functions/erfc.cpp create mode 100644 dbms/src/Functions/exp.cpp create mode 100644 dbms/src/Functions/exp10.cpp create mode 100644 dbms/src/Functions/exp2.cpp create mode 100644 dbms/src/Functions/lgamma.cpp create mode 100644 dbms/src/Functions/log.cpp create mode 100644 dbms/src/Functions/log10.cpp create mode 100644 dbms/src/Functions/log2.cpp create mode 100644 dbms/src/Functions/pi.cpp create mode 100644 dbms/src/Functions/pow.cpp create mode 100644 dbms/src/Functions/registerFunctionsMath.cpp create mode 100644 dbms/src/Functions/sin.cpp create mode 100644 dbms/src/Functions/sqrt.cpp create mode 100644 dbms/src/Functions/tan.cpp create mode 100644 dbms/src/Functions/tgamma.cpp diff --git a/dbms/src/Functions/FunctionsMath.h b/dbms/src/Functions/FunctionMathBinaryFloat64.h similarity index 53% rename from dbms/src/Functions/FunctionsMath.h rename to dbms/src/Functions/FunctionMathBinaryFloat64.h index 5ab1bf899c2..e4524138701 100644 --- a/dbms/src/Functions/FunctionsMath.h +++ b/dbms/src/Functions/FunctionMathBinaryFloat64.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -19,16 +18,15 @@ #if USE_VECTORCLASS #ifdef __clang__ - #pragma clang diagnostic push - #pragma clang diagnostic ignored "-Wshift-negative-value" + #pragma clang diagnostic push + #pragma clang diagnostic ignored "-Wshift-negative-value" #endif #include // Y_IGNORE #include // Y_IGNORE - #include // Y_IGNORE #ifdef __clang__ - #pragma clang diagnostic pop + #pragma clang diagnostic pop #endif #endif @@ -41,169 +39,6 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -template -class FunctionMathNullaryConstFloat64 : public IFunction -{ -public: - static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } - -private: - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override - { - block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, Impl::value); - } -}; - - -template -class FunctionMathUnaryFloat64 : public IFunction -{ -public: - static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } - static_assert(Impl::rows_per_iteration > 0, "Impl must process at least one row per iteration"); - -private: - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 1; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - const auto & arg = arguments.front(); - if (!isNumber(arg) && !isDecimal(arg)) - throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - return std::make_shared(); - } - - template - static void executeInIterations(const T * src_data, Float64 * dst_data, size_t size) - { - const size_t rows_remaining = size % Impl::rows_per_iteration; - const size_t rows_size = size - rows_remaining; - - for (size_t i = 0; i < rows_size; i += Impl::rows_per_iteration) - Impl::execute(&src_data[i], &dst_data[i]); - - if (rows_remaining != 0) - { - T src_remaining[Impl::rows_per_iteration]; - memcpy(src_remaining, &src_data[rows_size], rows_remaining * sizeof(T)); - memset(src_remaining + rows_remaining, 0, (Impl::rows_per_iteration - rows_remaining) * sizeof(T)); - Float64 dst_remaining[Impl::rows_per_iteration]; - - Impl::execute(src_remaining, dst_remaining); - - memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64)); - } - } - - template - static bool execute(Block & block, const ColumnVector * col, const size_t result) - { - const auto & src_data = col->getData(); - const size_t size = src_data.size(); - - auto dst = ColumnVector::create(); - auto & dst_data = dst->getData(); - dst_data.resize(size); - - executeInIterations(src_data.data(), dst_data.data(), size); - - block.getByPosition(result).column = std::move(dst); - return true; - } - - template - static bool execute(Block & block, const ColumnDecimal * col, const size_t result) - { - const auto & src_data = col->getData(); - const size_t size = src_data.size(); - UInt32 scale = src_data.getScale(); - - auto dst = ColumnVector::create(); - auto & dst_data = dst->getData(); - dst_data.resize(size); - - for (size_t i = 0; i < size; ++i) - dst_data[i] = convertFromDecimal, DataTypeNumber>(src_data[i], scale); - - executeInIterations(dst_data.data(), dst_data.data(), size); - - block.getByPosition(result).column = std::move(dst); - return true; - } - - bool useDefaultImplementationForConstants() const override { return true; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const ColumnWithTypeAndName & col = block.getByPosition(arguments[0]); - - auto call = [&](const auto & types) -> bool - { - using Types = std::decay_t; - using Type = typename Types::RightType; - using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; - - const auto col_vec = checkAndGetColumn(col.column.get()); - return execute(block, col_vec, result); - }; - - if (!callOnBasicType(col.type->getTypeId(), call)) - throw Exception{"Illegal column " + col.column->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN}; - } -}; - - -template -struct UnaryFunctionPlain -{ - static constexpr auto name = Name::name; - static constexpr auto rows_per_iteration = 1; - - template - static void execute(const T * src, Float64 * dst) - { - dst[0] = static_cast(Function(static_cast(src[0]))); - } -}; - -#if USE_VECTORCLASS - -template -struct UnaryFunctionVectorized -{ - static constexpr auto name = Name::name; - static constexpr auto rows_per_iteration = 2; - - template - static void execute(const T * src, Float64 * dst) - { - const auto result = Function(Vec2d(src[0], src[1])); - result.store(dst); - } -}; - -#else - -#define UnaryFunctionVectorized UnaryFunctionPlain - -#endif - template class FunctionMathBinaryFloat64 : public IFunction @@ -427,66 +262,4 @@ struct BinaryFunctionVectorized #endif - -struct EImpl -{ - static constexpr auto name = "e"; - static const double value; /// See .cpp -}; - -struct PiImpl -{ - static constexpr auto name = "pi"; - static const double value; -}; - -struct ExpName { static constexpr auto name = "exp"; }; -struct LogName { static constexpr auto name = "log"; }; -struct Exp2Name { static constexpr auto name = "exp2"; }; -struct Log2Name { static constexpr auto name = "log2"; }; -struct Exp10Name { static constexpr auto name = "exp10"; }; -struct Log10Name { static constexpr auto name = "log10"; }; -struct SqrtName { static constexpr auto name = "sqrt"; }; -struct CbrtName { static constexpr auto name = "cbrt"; }; -struct SinName { static constexpr auto name = "sin"; }; -struct CosName { static constexpr auto name = "cos"; }; -struct TanName { static constexpr auto name = "tan"; }; -struct AsinName { static constexpr auto name = "asin"; }; -struct AcosName { static constexpr auto name = "acos"; }; -struct AtanName { static constexpr auto name = "atan"; }; -struct ErfName { static constexpr auto name = "erf"; }; -struct ErfcName { static constexpr auto name = "erfc"; }; -struct LGammaName { static constexpr auto name = "lgamma"; }; -struct TGammaName { static constexpr auto name = "tgamma"; }; -struct PowName { static constexpr auto name = "pow"; }; - -using FunctionE = FunctionMathNullaryConstFloat64; -using FunctionPi = FunctionMathNullaryConstFloat64; -using FunctionExp = FunctionMathUnaryFloat64>; -using FunctionLog = FunctionMathUnaryFloat64>; -using FunctionExp2 = FunctionMathUnaryFloat64>; -using FunctionLog2 = FunctionMathUnaryFloat64>; -using FunctionExp10 = FunctionMathUnaryFloat64>; - -using FunctionLog10 = FunctionMathUnaryFloat64>; -using FunctionSqrt = FunctionMathUnaryFloat64>; -using FunctionCbrt = FunctionMathUnaryFloat64>; -using FunctionSin = FunctionMathUnaryFloat64>; -using FunctionCos = FunctionMathUnaryFloat64>; -using FunctionTan = FunctionMathUnaryFloat64>; -using FunctionAsin = FunctionMathUnaryFloat64>; -using FunctionAcos = FunctionMathUnaryFloat64>; -using FunctionAtan = FunctionMathUnaryFloat64>; -using FunctionErf = FunctionMathUnaryFloat64>; -using FunctionErfc = FunctionMathUnaryFloat64>; -using FunctionLGamma = FunctionMathUnaryFloat64>; -using FunctionTGamma = FunctionMathUnaryFloat64>; -using FunctionPow = FunctionMathBinaryFloat64>; - } diff --git a/dbms/src/Functions/FunctionMathConstFloat64.h b/dbms/src/Functions/FunctionMathConstFloat64.h new file mode 100644 index 00000000000..876582a850f --- /dev/null +++ b/dbms/src/Functions/FunctionMathConstFloat64.h @@ -0,0 +1,34 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +template +class FunctionMathConstFloat64 : public IFunction +{ +public: + static constexpr auto name = Impl::name; + static FunctionPtr create(const Context &) { return std::make_shared(); } + +private: + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, Impl::value); + } +}; + +} diff --git a/dbms/src/Functions/FunctionMathUnaryFloat64.h b/dbms/src/Functions/FunctionMathUnaryFloat64.h new file mode 100644 index 00000000000..f336847bee5 --- /dev/null +++ b/dbms/src/Functions/FunctionMathUnaryFloat64.h @@ -0,0 +1,182 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + +/** More efficient implementations of mathematical functions are possible when using a separate library. + * Disabled due to licence compatibility limitations. + * To enable: download http://www.agner.org/optimize/vectorclass.zip and unpack to contrib/vectorclass + * Then rebuild with -DENABLE_VECTORCLASS=1 + */ + +#if USE_VECTORCLASS + #ifdef __clang__ + #pragma clang diagnostic push + #pragma clang diagnostic ignored "-Wshift-negative-value" + #endif + + #include // Y_IGNORE + #include // Y_IGNORE + #include // Y_IGNORE + + #ifdef __clang__ + #pragma clang diagnostic pop + #endif +#endif + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + + +template +class FunctionMathUnaryFloat64 : public IFunction +{ +public: + static constexpr auto name = Impl::name; + static FunctionPtr create(const Context &) { return std::make_shared(); } + static_assert(Impl::rows_per_iteration > 0, "Impl must process at least one row per iteration"); + +private: + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const auto & arg = arguments.front(); + if (!isNumber(arg) && !isDecimal(arg)) + throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(); + } + + template + static void executeInIterations(const T * src_data, Float64 * dst_data, size_t size) + { + const size_t rows_remaining = size % Impl::rows_per_iteration; + const size_t rows_size = size - rows_remaining; + + for (size_t i = 0; i < rows_size; i += Impl::rows_per_iteration) + Impl::execute(&src_data[i], &dst_data[i]); + + if (rows_remaining != 0) + { + T src_remaining[Impl::rows_per_iteration]; + memcpy(src_remaining, &src_data[rows_size], rows_remaining * sizeof(T)); + memset(src_remaining + rows_remaining, 0, (Impl::rows_per_iteration - rows_remaining) * sizeof(T)); + Float64 dst_remaining[Impl::rows_per_iteration]; + + Impl::execute(src_remaining, dst_remaining); + + memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(Float64)); + } + } + + template + static bool execute(Block & block, const ColumnVector * col, const size_t result) + { + const auto & src_data = col->getData(); + const size_t size = src_data.size(); + + auto dst = ColumnVector::create(); + auto & dst_data = dst->getData(); + dst_data.resize(size); + + executeInIterations(src_data.data(), dst_data.data(), size); + + block.getByPosition(result).column = std::move(dst); + return true; + } + + template + static bool execute(Block & block, const ColumnDecimal * col, const size_t result) + { + const auto & src_data = col->getData(); + const size_t size = src_data.size(); + UInt32 scale = src_data.getScale(); + + auto dst = ColumnVector::create(); + auto & dst_data = dst->getData(); + dst_data.resize(size); + + for (size_t i = 0; i < size; ++i) + dst_data[i] = convertFromDecimal, DataTypeNumber>(src_data[i], scale); + + executeInIterations(dst_data.data(), dst_data.data(), size); + + block.getByPosition(result).column = std::move(dst); + return true; + } + + bool useDefaultImplementationForConstants() const override { return true; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const ColumnWithTypeAndName & col = block.getByPosition(arguments[0]); + + auto call = [&](const auto & types) -> bool + { + using Types = std::decay_t; + using Type = typename Types::RightType; + using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; + + const auto col_vec = checkAndGetColumn(col.column.get()); + return execute(block, col_vec, result); + }; + + if (!callOnBasicType(col.type->getTypeId(), call)) + throw Exception{"Illegal column " + col.column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN}; + } +}; + + +template +struct UnaryFunctionPlain +{ + static constexpr auto name = Name::name; + static constexpr auto rows_per_iteration = 1; + + template + static void execute(const T * src, Float64 * dst) + { + dst[0] = static_cast(Function(static_cast(src[0]))); + } +}; + +#if USE_VECTORCLASS + +template +struct UnaryFunctionVectorized +{ + static constexpr auto name = Name::name; + static constexpr auto rows_per_iteration = 2; + + template + static void execute(const T * src, Float64 * dst) + { + const auto result = Function(Vec2d(src[0], src[1])); + result.store(dst); + } +}; + +#else + +#define UnaryFunctionVectorized UnaryFunctionPlain + +#endif + +} diff --git a/dbms/src/Functions/FunctionsMath.cpp b/dbms/src/Functions/FunctionsMath.cpp deleted file mode 100644 index 0252361083f..00000000000 --- a/dbms/src/Functions/FunctionsMath.cpp +++ /dev/null @@ -1,39 +0,0 @@ -#include -#include - -namespace DB -{ - -const double EImpl::value = 2.7182818284590452353602874713526624977572470; -const double PiImpl::value = 3.1415926535897932384626433832795028841971693; - - -void registerFunctionsMath(FunctionFactory & factory) -{ - factory.registerFunction(); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(FunctionFactory::CaseInsensitive); - factory.registerFunction(FunctionFactory::CaseInsensitive); - - factory.registerAlias("power", "pow", FunctionFactory::CaseInsensitive); - factory.registerAlias("ln", "log", FunctionFactory::CaseInsensitive); -} - -} diff --git a/dbms/src/Functions/acos.cpp b/dbms/src/Functions/acos.cpp new file mode 100644 index 00000000000..a380ba9e5d5 --- /dev/null +++ b/dbms/src/Functions/acos.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct AcosName { static constexpr auto name = "acos"; }; +using FunctionAcos = FunctionMathUnaryFloat64>; + +void registerFunctionAcos(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/asin.cpp b/dbms/src/Functions/asin.cpp new file mode 100644 index 00000000000..c6edd2d3925 --- /dev/null +++ b/dbms/src/Functions/asin.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct AsinName { static constexpr auto name = "asin"; }; +using FunctionAsin = FunctionMathUnaryFloat64>; + +void registerFunctionAsin(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/atan.cpp b/dbms/src/Functions/atan.cpp new file mode 100644 index 00000000000..9918a2f76f4 --- /dev/null +++ b/dbms/src/Functions/atan.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct AtanName { static constexpr auto name = "atan"; }; +using FunctionAtan = FunctionMathUnaryFloat64>; + +void registerFunctionAtan(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/cbrt.cpp b/dbms/src/Functions/cbrt.cpp new file mode 100644 index 00000000000..3969063e0e4 --- /dev/null +++ b/dbms/src/Functions/cbrt.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct CbrtName { static constexpr auto name = "cbrt"; }; +using FunctionCbrt = FunctionMathUnaryFloat64>; + +void registerFunctionCbrt(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/cos.cpp b/dbms/src/Functions/cos.cpp new file mode 100644 index 00000000000..da5256c44cf --- /dev/null +++ b/dbms/src/Functions/cos.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct CosName { static constexpr auto name = "cos"; }; +using FunctionCos = FunctionMathUnaryFloat64>; + +void registerFunctionCos(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/e.cpp b/dbms/src/Functions/e.cpp new file mode 100644 index 00000000000..cb4cd66b44c --- /dev/null +++ b/dbms/src/Functions/e.cpp @@ -0,0 +1,20 @@ +#include +#include + +namespace DB +{ + +struct EImpl +{ + static constexpr auto name = "e"; + static constexpr double value = 2.7182818284590452353602874713526624977572470; +}; + +using FunctionE = FunctionMathConstFloat64; + +void registerFunctionE(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/erf.cpp b/dbms/src/Functions/erf.cpp new file mode 100644 index 00000000000..d8f9e7abb0c --- /dev/null +++ b/dbms/src/Functions/erf.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct ErfName { static constexpr auto name = "erf"; }; +using FunctionErf = FunctionMathUnaryFloat64>; + +void registerFunctionErf(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/erfc.cpp b/dbms/src/Functions/erfc.cpp new file mode 100644 index 00000000000..34802dc245f --- /dev/null +++ b/dbms/src/Functions/erfc.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct ErfcName { static constexpr auto name = "erfc"; }; +using FunctionErfc = FunctionMathUnaryFloat64>; + +void registerFunctionErfc(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/exp.cpp b/dbms/src/Functions/exp.cpp new file mode 100644 index 00000000000..0a387ac4031 --- /dev/null +++ b/dbms/src/Functions/exp.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct ExpName { static constexpr auto name = "exp"; }; +using FunctionExp = FunctionMathUnaryFloat64>; + +void registerFunctionExp(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/exp10.cpp b/dbms/src/Functions/exp10.cpp new file mode 100644 index 00000000000..caf949e0c62 --- /dev/null +++ b/dbms/src/Functions/exp10.cpp @@ -0,0 +1,23 @@ +#include +#include +#include + +namespace DB +{ + +struct Exp10Name { static constexpr auto name = "exp10"; }; + +using FunctionExp10 = FunctionMathUnaryFloat64>; + +void registerFunctionExp10(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/exp2.cpp b/dbms/src/Functions/exp2.cpp new file mode 100644 index 00000000000..25d7f683ecf --- /dev/null +++ b/dbms/src/Functions/exp2.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct Exp2Name { static constexpr auto name = "exp2"; }; +using FunctionExp2 = FunctionMathUnaryFloat64>; + +void registerFunctionExp2(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/lgamma.cpp b/dbms/src/Functions/lgamma.cpp new file mode 100644 index 00000000000..8f55fd75014 --- /dev/null +++ b/dbms/src/Functions/lgamma.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct LGammaName { static constexpr auto name = "lgamma"; }; +using FunctionLGamma = FunctionMathUnaryFloat64>; + +void registerFunctionLGamma(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/log.cpp b/dbms/src/Functions/log.cpp new file mode 100644 index 00000000000..643a20e7e28 --- /dev/null +++ b/dbms/src/Functions/log.cpp @@ -0,0 +1,16 @@ +#include +#include + +namespace DB +{ + +struct LogName { static constexpr auto name = "log"; }; +using FunctionLog = FunctionMathUnaryFloat64>; + +void registerFunctionLog(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerAlias("ln", "log", FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/log10.cpp b/dbms/src/Functions/log10.cpp new file mode 100644 index 00000000000..718a377e2f3 --- /dev/null +++ b/dbms/src/Functions/log10.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct Log10Name { static constexpr auto name = "log10"; }; +using FunctionLog10 = FunctionMathUnaryFloat64>; + +void registerFunctionLog10(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/log2.cpp b/dbms/src/Functions/log2.cpp new file mode 100644 index 00000000000..ef8281ecb0a --- /dev/null +++ b/dbms/src/Functions/log2.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct Log2Name { static constexpr auto name = "log2"; }; +using FunctionLog2 = FunctionMathUnaryFloat64>; + +void registerFunctionLog2(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/pi.cpp b/dbms/src/Functions/pi.cpp new file mode 100644 index 00000000000..4621ed8ecc0 --- /dev/null +++ b/dbms/src/Functions/pi.cpp @@ -0,0 +1,20 @@ +#include +#include + +namespace DB +{ + +struct PiImpl +{ + static constexpr auto name = "pi"; + static constexpr double value = 3.1415926535897932384626433832795028841971693; +}; + +using FunctionPi = FunctionMathConstFloat64; + +void registerFunctionPi(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/pow.cpp b/dbms/src/Functions/pow.cpp new file mode 100644 index 00000000000..db16bc993c7 --- /dev/null +++ b/dbms/src/Functions/pow.cpp @@ -0,0 +1,16 @@ +#include +#include + +namespace DB +{ + +struct PowName { static constexpr auto name = "pow"; }; +using FunctionPow = FunctionMathBinaryFloat64>; + +void registerFunctionPow(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerAlias("power", "pow", FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/registerFunctionsMath.cpp b/dbms/src/Functions/registerFunctionsMath.cpp new file mode 100644 index 00000000000..3f7bba95922 --- /dev/null +++ b/dbms/src/Functions/registerFunctionsMath.cpp @@ -0,0 +1,54 @@ +namespace DB +{ + +class FunctionFactory; + +void registerFunctionE(FunctionFactory & factory); +void registerFunctionPi(FunctionFactory & factory); +void registerFunctionExp(FunctionFactory & factory); +void registerFunctionLog(FunctionFactory & factory); +void registerFunctionExp2(FunctionFactory & factory); +void registerFunctionLog2(FunctionFactory & factory); +void registerFunctionExp10(FunctionFactory & factory); +void registerFunctionLog10(FunctionFactory & factory); +void registerFunctionSqrt(FunctionFactory & factory); +void registerFunctionCbrt(FunctionFactory & factory); +void registerFunctionErf(FunctionFactory & factory); +void registerFunctionErfc(FunctionFactory & factory); +void registerFunctionLGamma(FunctionFactory & factory); +void registerFunctionTGamma(FunctionFactory & factory); +void registerFunctionSin(FunctionFactory & factory); +void registerFunctionCos(FunctionFactory & factory); +void registerFunctionTan(FunctionFactory & factory); +void registerFunctionAsin(FunctionFactory & factory); +void registerFunctionAcos(FunctionFactory & factory); +void registerFunctionAtan(FunctionFactory & factory); +void registerFunctionPow(FunctionFactory & factory); + +void registerFunctionsMath(FunctionFactory & factory) +{ + registerFunctionE(factory); + registerFunctionPi(factory); + registerFunctionExp(factory); + registerFunctionLog(factory); + registerFunctionExp2(factory); + registerFunctionLog2(factory); + registerFunctionExp10(factory); + registerFunctionLog10(factory); + registerFunctionSqrt(factory); + registerFunctionCbrt(factory); + registerFunctionErf(factory); + registerFunctionErfc(factory); + registerFunctionLGamma(factory); + registerFunctionTGamma(factory); + registerFunctionSin(factory); + registerFunctionCos(factory); + registerFunctionTan(factory); + registerFunctionAsin(factory); + registerFunctionAcos(factory); + registerFunctionAtan(factory); + registerFunctionPow(factory); +} + +} + diff --git a/dbms/src/Functions/sin.cpp b/dbms/src/Functions/sin.cpp new file mode 100644 index 00000000000..2d1375c9a74 --- /dev/null +++ b/dbms/src/Functions/sin.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct SinName { static constexpr auto name = "sin"; }; +using FunctionSin = FunctionMathUnaryFloat64>; + +void registerFunctionSin(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/sqrt.cpp b/dbms/src/Functions/sqrt.cpp new file mode 100644 index 00000000000..817f6277f27 --- /dev/null +++ b/dbms/src/Functions/sqrt.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct SqrtName { static constexpr auto name = "sqrt"; }; +using FunctionSqrt = FunctionMathUnaryFloat64>; + +void registerFunctionSqrt(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/tan.cpp b/dbms/src/Functions/tan.cpp new file mode 100644 index 00000000000..ec83e83baf1 --- /dev/null +++ b/dbms/src/Functions/tan.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct TanName { static constexpr auto name = "tan"; }; +using FunctionTan = FunctionMathUnaryFloat64>; + +void registerFunctionTan(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/Functions/tgamma.cpp b/dbms/src/Functions/tgamma.cpp new file mode 100644 index 00000000000..d58732d3e84 --- /dev/null +++ b/dbms/src/Functions/tgamma.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ + +struct TGammaName { static constexpr auto name = "tgamma"; }; +using FunctionTGamma = FunctionMathUnaryFloat64>; + +void registerFunctionTGamma(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} From 0090aaf8eed74191ed88d111df61cbca250e0b16 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 3 Dec 2018 12:12:03 +0300 Subject: [PATCH 63/69] Add missed header --- dbms/src/Common/Config/ConfigProcessor.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Common/Config/ConfigProcessor.h b/dbms/src/Common/Config/ConfigProcessor.h index 78193484cce..d4dc48d8035 100644 --- a/dbms/src/Common/Config/ConfigProcessor.h +++ b/dbms/src/Common/Config/ConfigProcessor.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include From b96533d43307593ebb54726501761797351eb129 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 3 Dec 2018 15:44:41 +0300 Subject: [PATCH 64/69] more user-centric bug descriptions in v18.14.17 changelog --- CHANGELOG_RU.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index 6c1c262cc77..d43ceab40af 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -3,9 +3,9 @@ ### Исправления ошибок: * Исправлена ситуация, при которой ODBC Bridge продолжал работу после завершения работы сервера ClickHouse. Теперь ODBC Bridge всегда завершает работу вместе с сервером. [#3642](https://github.com/yandex/ClickHouse/pull/3642) * Исправлена синхронная вставка в `Distributed` таблицу в случае явного указания неполного списка столбцов или списка столбцов в измененном порядке. [#3673](https://github.com/yandex/ClickHouse/pull/3673) -* Исправлен race condition в `BackgroundProcessingPoolTaskInfo` возникающий при попытке выполнения задачи одновременно с инвалидацией её итератора очереди. [#3680](https://github.com/yandex/ClickHouse/pull/3680) -* Исправлен deadlock в `ParallelInputsProcessor` возникающий в случае исключения при создании потока. [#3643](https://github.com/yandex/ClickHouse/pull/3643) -* Исправлена ошибка парсинга `Engine` при создании таблицы с синтаксисом `AS table` в случае, когда `AS table` указывался после `Engine`, что приводило к игнорированию указанного движка. [#3692](https://github.com/yandex/ClickHouse/pull/3692) +* Исправлен редкий race condition, который мог привести к падению сервера при удалении MergeTree-таблиц. [#3680](https://github.com/yandex/ClickHouse/pull/3680) +* Исправлен deadlock при выполнении запроса, возникающий если создание новых потоков выполнения невозможно из-за ошибки `Resource temporarily unavailable`. [#3643](https://github.com/yandex/ClickHouse/pull/3643) +* Исправлена ошибка парсинга `ENGINE` при создании таблицы с синтаксисом `AS table` в случае, когда `AS table` указывался после `ENGINE`, что приводило к игнорированию указанного движка. [#3692](https://github.com/yandex/ClickHouse/pull/3692) ## ClickHouse release 18.14.15, 2018-11-21 @@ -18,7 +18,7 @@ * Исправлена работа запросов `ON CLUSTER` в случае, когда в конфигурации кластера включено шифрование (флаг ``). [#3599](https://github.com/yandex/ClickHouse/pull/3599) ### Улучшения процесса сборки ClickHouse: -* Испрпавлены проблемы сборки (llvm-7 из системы, macos) [#3582](https://github.com/yandex/ClickHouse/pull/3582) +* Исправлены проблемы сборки (llvm-7 из системы, macos) [#3582](https://github.com/yandex/ClickHouse/pull/3582) ## ClickHouse release 18.14.13, 2018-11-08 From 9407e925fa9b9ff3e5c3266b6dff55853480f6a3 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 3 Dec 2018 15:45:21 +0300 Subject: [PATCH 65/69] add missing entries to en changelog --- CHANGELOG.md | 32 ++++++++++++++++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4e81432edd0..1b36d5f99e4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,12 @@ +## ClickHouse release 18.14.17, 2018-11-30 + +### Bug fixes: +* Fixed cases when the ODBC bridge process did not terminate with the main server process. [#3642](https://github.com/yandex/ClickHouse/pull/3642) +* Fixed synchronous insertion into the `Distributed` table with a columns list that differs from the column list of the remote table. [#3673](https://github.com/yandex/ClickHouse/pull/3673) +* Fixed a rare race condition that can lead to a crash when dropping a MergeTree table. [#3643](https://github.com/yandex/ClickHouse/pull/3643) +* Fixed a query deadlock in case when query thread creation fails with the `Resource temporarily unavailable` error. [#3643](https://github.com/yandex/ClickHouse/pull/3643) +* Fixed parsing of the `ENGINE` clause when the `CREATE AS table` syntax was used and the `ENGINE` clause was specified before the `AS table` (the error resulted in ignoring the specified engine). [#3692](https://github.com/yandex/ClickHouse/pull/3692) + ## ClickHouse release 18.14.15, 2018-11-21 ### Bug fixes: @@ -11,6 +20,29 @@ ### Build changes: * Fixed problems (llvm-7 from system, macos) [#3582](https://github.com/yandex/ClickHouse/pull/3582) +## ClickHouse release 18.14.13, 2018-11-08 + +### Bug fixes: +* Fixed the `Block structure mismatch in MergingSorted stream` error. [#3162](https://github.com/yandex/ClickHouse/issues/3162) +* Fixed `ON CLUSTER` queries in case when secure connections were turned on in the cluster config (the `` flag). [#3465](https://github.com/yandex/ClickHouse/pull/3465) +* Fixed an error in queries that used `SAMPLE`, `PREWHERE` and alias columns. [#3543](https://github.com/yandex/ClickHouse/pull/3543) +* Fixed a rare `unknown compression method` error when the `min_bytes_to_use_direct_io` setting was enabled. [3544](https://github.com/yandex/ClickHouse/pull/3544) + +### Performance improvements: +* Fixed performance regression of queries with `GROUP BY` of columns of UInt16 or Date type when executing on AMD EPYC processors. [Igor Lapko](https://github.com/yandex/ClickHouse/pull/3512) +* Fixed performance regression of queries that process long strings. [#3530](https://github.com/yandex/ClickHouse/pull/3530) + +### Build improvements: +* Improvements for simplifying the Arcadia build. [#3475](https://github.com/yandex/ClickHouse/pull/3475), [#3535](https://github.com/yandex/ClickHouse/pull/3535) + +## ClickHouse release 18.14.12, 2018-11-02 + +### Bug fixes: + +* Fixed a crash on joining two unnamed subqueries. [#3505](https://github.com/yandex/ClickHouse/pull/3505) +* Fixed generating incorrect queries (with an empty `WHERE` clause) when querying external databases. [hotid](https://github.com/yandex/ClickHouse/pull/3477) +* Fixed using an incorrect timeout value in ODBC dictionaries. [Marek Vavruša](https://github.com/yandex/ClickHouse/pull/3511) + ## ClickHouse release 18.14.11, 2018-10-29 ### Bug fixes: From 051883bdf9c3653ac77182769452103041760749 Mon Sep 17 00:00:00 2001 From: moscas Date: Mon, 3 Dec 2018 18:52:31 +0300 Subject: [PATCH 66/69] Update gui.md --- docs/en/interfaces/third-party/gui.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docs/en/interfaces/third-party/gui.md b/docs/en/interfaces/third-party/gui.md index 280b0bee218..855e9fffc7a 100644 --- a/docs/en/interfaces/third-party/gui.md +++ b/docs/en/interfaces/third-party/gui.md @@ -46,4 +46,17 @@ Key features: - Table preview. - Autocompletion. +## DataGrip + +[DataGrip](https://www.jetbrains.com/datagrip/) - Database IDE from JetBrains with dedicated support for ClickHouse. The same is embedded into other IntelliJ-based tools: PyCharm, IntelliJIDEA, GoLand, PhpStorm etc. + +Features: + +- Very fast code completion. +- Clickhouse synthax highlighting. +- Specific Clickhouse features support in SQL, i.e. nested columns, table engines. +- Data Editor. +- Refactorings. +- Search and Navigation. + [Original article](https://clickhouse.yandex/docs/en/interfaces/third-party_gui/) From 170f90ffe7ed26894aaac263049293b385dd33ef Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 3 Dec 2018 19:08:26 +0300 Subject: [PATCH 67/69] Update README.md --- dbms/tests/integration/README.md | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/dbms/tests/integration/README.md b/dbms/tests/integration/README.md index 84db3876727..8a808dd4300 100644 --- a/dbms/tests/integration/README.md +++ b/dbms/tests/integration/README.md @@ -33,19 +33,7 @@ set the following environment variables: ### Running with runner script -The only requirement is fresh docker with access to the internet. You can check it with: -``` -$ docker run ubuntu:14.04 ping github.com -PING github.com (140.82.118.3) 56(84) bytes of data. -64 bytes from 140.82.118.3: icmp_seq=1 ttl=53 time=40.1 ms -64 bytes from 140.82.118.3: icmp_seq=2 ttl=53 time=40.4 ms -64 bytes from 140.82.118.3: icmp_seq=3 ttl=53 time=40.3 ms -64 bytes from 140.82.118.3: icmp_seq=4 ttl=53 time=40.1 ms - ---- github.com ping statistics --- -4 packets transmitted, 4 received, 0% packet loss, time 19823ms -rtt min/avg/max/mdev = 40.157/40.284/40.463/0.278 ms -``` +The only requirement is fresh docker. You can run tests via `./runner` script and pass pytest arguments as last arg: ``` From 9805c4ae39fce5ab595eac21147d476a91232e6e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=A2=E7=A3=8A?= <2217232293@qq.com> Date: Tue, 4 Dec 2018 01:02:00 +0800 Subject: [PATCH 68/69] [hotfix][doc] affect reading documents, add a space (#3738) I hereby agree to the terms of the CLA available at: https://yandex.ru/legal/cla/?lang=en affect reading documents, add a space --- docs/en/operations/table_engines/buffer.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/table_engines/buffer.md b/docs/en/operations/table_engines/buffer.md index 24a990bb260..65cd59eaebe 100644 --- a/docs/en/operations/table_engines/buffer.md +++ b/docs/en/operations/table_engines/buffer.md @@ -6,7 +6,7 @@ Buffers the data to write in RAM, periodically flushing it to another table. Dur Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes) ``` -Engine parameters:database, table – The table to flush data to. Instead of the database name, you can use a constant expression that returns a string.num_layers – Parallelism layer. Physically, the table will be represented as 'num_layers' of independent buffers. Recommended value: 16.min_time, max_time, min_rows, max_rows, min_bytes, and max_bytes are conditions for flushing data from the buffer. +Engine parameters:database, table – The table to flush data to. Instead of the database name, you can use a constant expression that returns a string. num_layers – Parallelism layer. Physically, the table will be represented as 'num_layers' of independent buffers. Recommended value: 16.min_time, max_time, min_rows, max_rows, min_bytes, and max_bytes are conditions for flushing data from the buffer. Data is flushed from the buffer and written to the destination table if all the 'min' conditions or at least one 'max' condition are met.min_time, max_time – Condition for the time in seconds from the moment of the first write to the buffer.min_rows, max_rows – Condition for the number of rows in the buffer.min_bytes, max_bytes – Condition for the number of bytes in the buffer. From d84df9c28395bbdda72d749c77f01b2d9161504a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=A2=E7=A3=8A?= <2217232293@qq.com> Date: Tue, 4 Dec 2018 03:38:27 +0800 Subject: [PATCH 69/69] [hotfix][doc] no space affect reading buffer doc (#3739) I hereby agree to the terms of the CLA available at: https://yandex.ru/legal/cla/?lang=en Backport of #3738 . --- docs/en/operations/table_engines/buffer.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/table_engines/buffer.md b/docs/en/operations/table_engines/buffer.md index 65cd59eaebe..7f6a17223be 100644 --- a/docs/en/operations/table_engines/buffer.md +++ b/docs/en/operations/table_engines/buffer.md @@ -6,9 +6,9 @@ Buffers the data to write in RAM, periodically flushing it to another table. Dur Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes) ``` -Engine parameters:database, table – The table to flush data to. Instead of the database name, you can use a constant expression that returns a string. num_layers – Parallelism layer. Physically, the table will be represented as 'num_layers' of independent buffers. Recommended value: 16.min_time, max_time, min_rows, max_rows, min_bytes, and max_bytes are conditions for flushing data from the buffer. +Engine parameters:database, table – The table to flush data to. Instead of the database name, you can use a constant expression that returns a string. num_layers – Parallelism layer. Physically, the table will be represented as 'num_layers' of independent buffers. Recommended value: 16. min_time, max_time, min_rows, max_rows, min_bytes, and max_bytes are conditions for flushing data from the buffer. -Data is flushed from the buffer and written to the destination table if all the 'min' conditions or at least one 'max' condition are met.min_time, max_time – Condition for the time in seconds from the moment of the first write to the buffer.min_rows, max_rows – Condition for the number of rows in the buffer.min_bytes, max_bytes – Condition for the number of bytes in the buffer. +Data is flushed from the buffer and written to the destination table if all the 'min' conditions or at least one 'max' condition are met.min_time, max_time – Condition for the time in seconds from the moment of the first write to the buffer. min_rows, max_rows – Condition for the number of rows in the buffer. min_bytes, max_bytes – Condition for the number of bytes in the buffer. During the write operation, data is inserted to a 'num_layers' number of random buffers. Or, if the data part to insert is large enough (greater than 'max_rows' or 'max_bytes'), it is written directly to the destination table, omitting the buffer.