From 1389105a30f64dce7f7bb1cbbe88cfb1e5218992 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Tue, 9 Apr 2019 18:36:33 +0300 Subject: [PATCH 001/161] parsing --- dbms/src/Parsers/ASTAlterQuery.cpp | 11 +++++++++++ dbms/src/Parsers/ASTAlterQuery.h | 1 + dbms/src/Parsers/ParserAlterQuery.cpp | 18 ++++++++++++++++++ 3 files changed, 30 insertions(+) diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index 12491c83762d..c0e4584f8c3e 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -100,6 +100,17 @@ void ASTAlterCommand::formatImpl( << "DROP INDEX " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); index->formatImpl(settings, state, frame); } + else if (type == ASTAlterCommand::MATERIALIZE_INDEX) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str + << "MATERIALIZE INDEX " << (settings.hilite ? hilite_none : ""); + index->formatImpl(settings, state, frame); + if (partition) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str<< " IN PARTITION " << (settings.hilite ? hilite_none : ""); + partition->formatImpl(settings, state, frame); + } + } 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 7261170288ac..6a078b029023 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -30,6 +30,7 @@ class ASTAlterCommand : public IAST ADD_INDEX, DROP_INDEX, + MATERIALIZE_INDEX, DROP_PARTITION, ATTACH_PARTITION, diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index b33679ad26bf..7864d2915593 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -30,6 +30,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_add_index("ADD INDEX"); ParserKeyword s_drop_index("DROP INDEX"); + ParserKeyword s_materialize_index("MATERIALIZE INDEX"); ParserKeyword s_attach_partition("ATTACH PARTITION"); ParserKeyword s_detach_partition("DETACH PARTITION"); @@ -124,6 +125,23 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::DROP_INDEX; command->detach = false; } + else if (s_materialize_index.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_name.parse(pos, command->index, expected)) + return false; + + command->type = ASTAlterCommand::MATERIALIZE_INDEX; + command->detach = false; + + if (s_in_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + } + } else if (s_clear_column.ignore(pos, expected)) { if (s_if_exists.ignore(pos, expected)) From 20f0b17cf4bb0302b459d062fced8360e9570de6 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Wed, 10 Apr 2019 20:44:39 +0300 Subject: [PATCH 002/161] mutations --- dbms/src/Interpreters/MutationsInterpreter.h | 2 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 21 +++- .../MergeTree/MergedBlockOutputStream.cpp | 113 +++++++++++++++++- .../MergeTree/MergedBlockOutputStream.h | 6 + dbms/src/Storages/MutationCommands.cpp | 9 ++ dbms/src/Storages/MutationCommands.h | 4 + 7 files changed, 144 insertions(+), 13 deletions(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.h b/dbms/src/Interpreters/MutationsInterpreter.h index 8c91c9cef605..6a83b14900f5 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.h +++ b/dbms/src/Interpreters/MutationsInterpreter.h @@ -13,7 +13,7 @@ namespace DB class Context; -/// Create an input stream that will read data from storage and apply mutation commands (UPDATEs, DELETEs) +/// Create an input stream that will read data from storage and apply mutation commands (UPDATEs, DELETEs, MATERIALIZEs) /// to this data. class MutationsInterpreter { diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index a33fa8f5f5e7..ee659527ecd6 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1404,7 +1404,7 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( */ IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets; MergedColumnOnlyOutputStream out( - *this, in.getHeader(), full_path + part->name + '/', true /* sync */, compression_codec, true /* skip_offsets */, unused_written_offsets); + *this, in.getHeader(), full_path + part->name + '/', true /* sync */, compression_codec, true /* skip_offsets */, {}, unused_written_offsets); in.readPrefix(); out.writePrefix(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index f5ee7fe1ee72..74e965e32d12 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -758,7 +758,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor rows_sources_read_buf.seek(0, 0); ColumnGathererStream column_gathered_stream(column_name, column_part_streams, rows_sources_read_buf); MergedColumnOnlyOutputStream column_to( - data, column_gathered_stream.getHeader(), new_part_tmp_path, false, compression_codec, false, written_offset_columns); + data, column_gathered_stream.getHeader(), new_part_tmp_path, false, + compression_codec, false, {}, written_offset_columns); size_t column_elems_written = 0; column_to.writePrefix(); @@ -915,7 +916,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor /// We will modify only some of the columns. Other columns and key values can be copied as-is. /// TODO: check that we modify only non-key columns in this case. - /// Checks if columns used in skipping indexes modified/ + /// Checks if columns used in skipping indexes modified + std::set indices_to_recalc; for (const auto & col : in_header.getNames()) { for (const auto & index : data.skip_indices) @@ -923,9 +925,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor const auto & index_cols = index->expr->getRequiredColumns(); auto it = find(cbegin(index_cols), cend(index_cols), col); if (it != cend(index_cols)) - throw Exception("You can not modify columns used in index. Index name: '" + indices_to_recalc.insert(index); + /*throw Exception("You can not modify columns used in index. Index name: '" + index->name - + "' bad column: '" + *it + "'", ErrorCodes::ILLEGAL_COLUMN); + + "' bad column: '" + *it + "'", ErrorCodes::ILLEGAL_COLUMN);*/ } } @@ -942,6 +945,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor IDataType::SubstreamPath stream_path; entry.type->enumerateStreams(callback, stream_path); } + for (const auto & index : indices_to_recalc) + { + files_to_skip.insert(index->getFileName() + ".idx"); + files_to_skip.insert(index->getFileName() + ".mrk"); + } Poco::DirectoryIterator dir_end; for (Poco::DirectoryIterator dir_it(source_part->getFullPath()); dir_it != dir_end; ++dir_it) @@ -959,7 +967,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets; MergedColumnOnlyOutputStream out( - data, in_header, new_part_tmp_path, /* sync = */ false, compression_codec, /* skip_offsets = */ false, unused_written_offsets); + data, in_header, new_part_tmp_path, /* sync = */ false, + compression_codec, /* skip_offsets = */ false, + std::vector(indices_to_recalc.begin(), indices_to_recalc.end()), + unused_written_offsets); in->readPrefix(); out.writePrefix(); diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 49fa5da2afa3..87f4b0a3e359 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -632,13 +632,14 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, CompressionCodecPtr default_codec_, bool skip_offsets_, + const std::vector & indices_to_recalc, WrittenOffsetColumns & already_written_offset_columns) : IMergedBlockOutputStream( storage_, storage_.global_context.getSettings().min_compress_block_size, storage_.global_context.getSettings().max_compress_block_size, default_codec_, storage_.global_context.getSettings().min_bytes_to_use_direct_io), header(header_), part_path(part_path_), sync(sync_), skip_offsets(skip_offsets_), - already_written_offset_columns(already_written_offset_columns) + skip_indices(indices_to_recalc), already_written_offset_columns(already_written_offset_columns) { } @@ -648,13 +649,13 @@ void MergedColumnOnlyOutputStream::write(const Block & block) { column_streams.clear(); serialization_states.clear(); - serialization_states.reserve(block.columns()); + serialization_states.reserve(header.columns()); WrittenOffsetColumns tmp_offset_columns; IDataType::SerializeBinaryBulkSettings settings; - for (size_t i = 0; i < block.columns(); ++i) + for (size_t i = 0; i < header.columns(); ++i) { - const auto & col = block.safeGetByPosition(i); + const auto & col = header.safeGetByPosition(i); const auto columns = storage.getColumns(); addStreams(part_path, col.name, *col.type, columns.getCodecOrDefault(col.name, codec), 0, skip_offsets); @@ -663,15 +664,96 @@ void MergedColumnOnlyOutputStream::write(const Block & block) col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back()); } + for (const auto & index : skip_indices) + { + String stream_name = index->getFileName(); + skip_indices_streams.emplace_back( + std::make_unique( + stream_name, + part_path + stream_name, INDEX_FILE_EXTENSION, + part_path + stream_name, MARKS_FILE_EXTENSION, + codec, max_compress_block_size, + 0, aio_threshold)); + skip_indices_aggregators.push_back(index->createIndexAggregator()); + skip_index_filling.push_back(0); + } + initialized = true; } + std::set skip_indexes_column_names_set; + for (const auto & index : skip_indices) + std::copy(index->columns.cbegin(), index->columns.cend(), + std::inserter(skip_indexes_column_names_set, skip_indexes_column_names_set.end())); + Names skip_indexes_column_names(skip_indexes_column_names_set.begin(), skip_indexes_column_names_set.end()); + + std::vector skip_indexes_columns(skip_indexes_column_names.size()); + std::map skip_indexes_column_name_to_position; + for (size_t i = 0, size = skip_indexes_column_names.size(); i < size; ++i) + { + const auto & name = skip_indexes_column_names[i]; + skip_indexes_column_name_to_position.emplace(name, i); + skip_indexes_columns[i] = block.getByName(name); + } + size_t rows = block.rows(); + { + /// Creating block for update + Block indices_update_block(skip_indexes_columns); + /// Filling and writing skip indices like in IMergedBlockOutputStream::writeData + for (size_t i = 0; i < skip_indices.size(); ++i) + { + const auto index = skip_indices[i]; + auto & stream = *skip_indices_streams[i]; + size_t prev_pos = 0; + + while (prev_pos < rows) + { + UInt64 limit = 0; + if (prev_pos == 0 && index_offset != 0) + { + limit = index_offset; + } + else + { + limit = storage.index_granularity; + if (skip_indices_aggregators[i]->empty()) + { + skip_indices_aggregators[i] = index->createIndexAggregator(); + skip_index_filling[i] = 0; + + if (stream.compressed.offset() >= min_compress_block_size) + stream.compressed.next(); + + writeIntBinary(stream.plain_hashing.count(), stream.marks); + writeIntBinary(stream.compressed.offset(), stream.marks); + } + } + + size_t pos = prev_pos; + skip_indices_aggregators[i]->update(indices_update_block, &pos, limit); + + if (pos == prev_pos + limit) + { + ++skip_index_filling[i]; + + /// write index if it is filled + if (skip_index_filling[i] == index->granularity) + { + skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed); + skip_index_filling[i] = 0; + } + } + prev_pos = pos; + } + } + } + WrittenOffsetColumns offset_columns = already_written_offset_columns; - for (size_t i = 0; i < block.columns(); ++i) + for (size_t i = 0; i < header.columns(); ++i) { - const ColumnWithTypeAndName & column = block.safeGetByPosition(i); + const ColumnWithTypeAndName & column = block.getByName(header.getColumnsWithTypeAndName()[i].name); writeData(column.name, *column.type, *column.column, offset_columns, skip_offsets, serialization_states[i]); } @@ -699,6 +781,15 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG column.type->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[i]); } + /// Finish skip index serialization + for (size_t i = 0; i < storage.skip_indices.size(); ++i) + { + auto & stream = *skip_indices_streams[i]; + if (!skip_indices_aggregators[i]->empty()) + skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed); + } + + MergeTreeData::DataPart::Checksums checksums; for (auto & column_stream : column_streams) @@ -710,10 +801,20 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG column_stream.second->addToChecksums(checksums); } + for (auto & stream : skip_indices_streams) + { + stream->finalize(); + stream->addToChecksums(checksums); + } + column_streams.clear(); serialization_states.clear(); initialized = false; + skip_indices_streams.clear(); + skip_indices_aggregators.clear(); + skip_index_filling.clear(); + return checksums; } diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h index 06acba26804b..4db07f017824 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -166,6 +166,7 @@ class MergedColumnOnlyOutputStream final : public IMergedBlockOutputStream MergedColumnOnlyOutputStream( MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, CompressionCodecPtr default_codec_, bool skip_offsets_, + const std::vector & indices_to_recalc, WrittenOffsetColumns & already_written_offset_columns); Block getHeader() const override { return header; } @@ -182,6 +183,11 @@ class MergedColumnOnlyOutputStream final : public IMergedBlockOutputStream bool sync; bool skip_offsets; + std::vector skip_indices; + std::vector> skip_indices_streams; + MergeTreeIndexAggregators skip_indices_aggregators; + std::vector skip_index_filling; + /// To correctly write Nested elements column-by-column. WrittenOffsetColumns & already_written_offset_columns; }; diff --git a/dbms/src/Storages/MutationCommands.cpp b/dbms/src/Storages/MutationCommands.cpp index 349ecb669801..03c4b65c8596 100644 --- a/dbms/src/Storages/MutationCommands.cpp +++ b/dbms/src/Storages/MutationCommands.cpp @@ -43,6 +43,15 @@ std::optional MutationCommand::parse(ASTAlterCommand * command) } return res; } + else if (command->type == ASTAlterCommand::MATERIALIZE_INDEX) + { + MutationCommand res; + res.ast = command->ptr(); + res.type = MATERIALIZE_INDEX; + res.partition = command->partition; + res.predicate = nullptr; + return res; + } else return {}; } diff --git a/dbms/src/Storages/MutationCommands.h b/dbms/src/Storages/MutationCommands.h index b5894c935988..3a8616a76777 100644 --- a/dbms/src/Storages/MutationCommands.h +++ b/dbms/src/Storages/MutationCommands.h @@ -22,6 +22,7 @@ struct MutationCommand EMPTY, /// Not used. DELETE, UPDATE, + MATERIALIZE_INDEX }; Type type = EMPTY; @@ -30,6 +31,9 @@ struct MutationCommand std::unordered_map column_to_update_expression; + /// For MATERIALIZE INDEX + ASTPtr partition; + static std::optional parse(ASTAlterCommand * command); }; From 8e8c77a46bf04c59cc80c6fd8e227e30258d985a Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Wed, 17 Apr 2019 20:07:07 +0300 Subject: [PATCH 003/161] added indices to mutationsInterpreter --- .../src/Interpreters/MutationsInterpreter.cpp | 36 +++++++++++++++++-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 26 ++++++++++---- .../MergeTree/StorageFromMergeTreeDataPart.h | 2 ++ 3 files changed, 56 insertions(+), 8 deletions(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 6415799af108..b1d5db407034 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -173,6 +173,7 @@ void MutationsInterpreter::prepare(bool dry_run) throw Exception("Empty mutation commands list", ErrorCodes::LOGICAL_ERROR); const ColumnsDescription & columns_desc = storage->getColumns(); + const IndicesDescription & indices_desc = storage->getIndicesDescription(); NamesAndTypesList all_columns = columns_desc.getAllPhysical(); NameSet updated_columns; @@ -182,9 +183,10 @@ void MutationsInterpreter::prepare(bool dry_run) updated_columns.insert(kv.first); } - /// We need to know which columns affect which MATERIALIZED columns to recalculate them if dependencies - /// are updated. + /// We need to know which columns affect which MATERIALIZED columns and data skipping indices + /// to recalculate them if dependencies are updated. std::unordered_map column_to_affected_materialized; + NameSet affected_indices_columns; if (!updated_columns.empty()) { for (const auto & column : columns_desc) @@ -201,6 +203,22 @@ void MutationsInterpreter::prepare(bool dry_run) } } } + for (const auto & index : indices_desc.indices) + { + auto query = index->expr->clone(); + auto syntax_result = SyntaxAnalyzer(context).analyze(query, all_columns); + ExpressionAnalyzer analyzer(query, syntax_result, context); + const auto required_columns = analyzer.getRequiredSourceColumns(); + + for (const String & dependency : required_columns) + { + if (updated_columns.count(dependency)) + { + affected_indices_columns.insert(std::cbegin(required_columns), std::cend(required_columns)); + break; + } + } + } validateUpdateColumns(storage, updated_columns, column_to_affected_materialized); } @@ -263,6 +281,20 @@ void MutationsInterpreter::prepare(bool dry_run) throw Exception("Unknown mutation command type: " + DB::toString(command.type), ErrorCodes::UNKNOWN_MUTATION_COMMAND); } + /// Special step to recalculate affected indices. + if (!affected_indices_columns.empty()) + { + stages.emplace_back(context); + for (const auto & column : affected_indices_columns) + { + stages.back().column_to_updated.emplace( + column, + std::make_shared( + columns_desc.getPhysical(column).type->getName())); + } + } + + /// Next, for each stage calculate columns changed by this and previous stages. for (size_t i = 0; i < stages.size(); ++i) { diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 74e965e32d12..e64a5648897d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -918,19 +918,33 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor /// Checks if columns used in skipping indexes modified std::set indices_to_recalc; + ASTPtr indices_recalc_expr_list = std::make_shared(); for (const auto & col : in_header.getNames()) { - for (const auto & index : data.skip_indices) + for (size_t i = 0; i < data.skip_indices.size(); ++i) { + const auto & index = data.skip_indices[i]; const auto & index_cols = index->expr->getRequiredColumns(); auto it = find(cbegin(index_cols), cend(index_cols), col); - if (it != cend(index_cols)) - indices_to_recalc.insert(index); - /*throw Exception("You can not modify columns used in index. Index name: '" - + index->name - + "' bad column: '" + *it + "'", ErrorCodes::ILLEGAL_COLUMN);*/ + if (it != cend(index_cols) && indices_to_recalc.insert(index).second) + { + ASTPtr expr_list = MergeTreeData::extractKeyExpressionList( + storage_from_source_part->getIndicesDescription().indices[i]->expr->clone()); + for (const auto & expr : expr_list->children) + indices_recalc_expr_list->children.push_back(expr->clone()); + } } } + if (!indices_to_recalc.empty()) + { + auto indices_recalc_syntax = SyntaxAnalyzer(context, {}).analyze( + indices_recalc_expr_list, in_header.getNamesAndTypesList()); + auto indices_recalc_expr = ExpressionAnalyzer( + indices_recalc_expr_list, + indices_recalc_syntax, context).getActions(false); + in = std::make_shared( + std::make_shared(in, indices_recalc_expr)); + } NameSet files_to_skip = {"checksums.txt", "columns.txt"}; for (const auto & entry : in_header) diff --git a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 314474b3cae1..0a13252b58bd 100644 --- a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -37,6 +37,8 @@ class StorageFromMergeTreeDataPart : public ext::shared_ptr_helperstorage.mayBenefitFromIndexForIn(left_in_operand); } + const IndicesDescription & getIndicesDescription() const override { return part->storage.getIndicesDescription(); } + protected: StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_) : IStorage(part_->storage.getColumns()), part(part_) From 0c2684b33e2787b82d1bbfc3d5254c11314e8ed4 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 18 Apr 2019 18:41:07 +0300 Subject: [PATCH 004/161] fix --- .../src/Interpreters/MutationsInterpreter.cpp | 6 +-- .../0_stateless/00935_mutate_index.reference | 4 ++ .../queries/0_stateless/00935_mutate_index.sh | 45 +++++++++++++++++++ 3 files changed, 50 insertions(+), 5 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00935_mutate_index.reference create mode 100644 dbms/tests/queries/0_stateless/00935_mutate_index.sh diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index b1d5db407034..0c2e28c56dfc 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -286,12 +286,8 @@ void MutationsInterpreter::prepare(bool dry_run) { stages.emplace_back(context); for (const auto & column : affected_indices_columns) - { stages.back().column_to_updated.emplace( - column, - std::make_shared( - columns_desc.getPhysical(column).type->getName())); - } + column, std::make_shared(column)); } diff --git a/dbms/tests/queries/0_stateless/00935_mutate_index.reference b/dbms/tests/queries/0_stateless/00935_mutate_index.reference new file mode 100644 index 000000000000..6582a420c512 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00935_mutate_index.reference @@ -0,0 +1,4 @@ +10 +0 +0 +10 diff --git a/dbms/tests/queries/0_stateless/00935_mutate_index.sh b/dbms/tests/queries/0_stateless/00935_mutate_index.sh new file mode 100644 index 000000000000..5faa9c95bf9e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00935_mutate_index.sh @@ -0,0 +1,45 @@ +#!/usr/bin/env bash + +CLICKHOUSE_CLIENT_OPT="--allow_experimental_data_skipping_indices=1" + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" + + +$CLICKHOUSE_CLIENT -n --query=" +CREATE TABLE test.minmax_idx +( + u64 UInt64, + i64 Int64, + i32 Int32, + INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1 +) ENGINE = MergeTree() +ORDER BY u64 +SETTINGS index_granularity = 2;" + + +$CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES +(0, 1, 1), +(1, 1, 2), +(2, 1, 3), +(3, 1, 4), +(4, 1, 5), +(5, 1, 6), +(6, 1, 7), +(7, 1, 8), +(8, 1, 9), +(9, 1, 10)" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 1;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 5;" + +$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx UPDATE i64 = 5 WHERE i64 = 1;" + +sleep 0.1 + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 1;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 5;" + +$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" \ No newline at end of file From 11d2785e8e9dfeb585488380951921197d912ed3 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 5 May 2019 10:59:07 +0300 Subject: [PATCH 005/161] parts drop --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index e64a5648897d..e330b4b74e9c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -836,7 +836,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor "This is a bug.", ErrorCodes::LOGICAL_ERROR); CurrentMetrics::Increment num_mutations{CurrentMetrics::PartMutation}; - const auto & source_part = future_part.parts[0]; auto storage_from_source_part = StorageFromMergeTreeDataPart::create(source_part); @@ -844,7 +843,18 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor context_for_reading.getSettingsRef().merge_tree_uniform_read_distribution = 0; context_for_reading.getSettingsRef().max_threads = 1; - MutationsInterpreter mutations_interpreter(storage_from_source_part, commands, context_for_reading); + std::vector commands_for_part; + std::copy_if( + std::cbegin(commands), std::cend(commands), + std::back_inserter(commands_for_part), + [&] (const MutationCommand & command) { + return command.partition == nullptr || + future_part.parts[0]->info.partition_id != data.getPartitionIDFromQuery( + command.partition, context_for_reading); + }); + + + MutationsInterpreter mutations_interpreter(storage_from_source_part, commands_for_part, context_for_reading); if (!mutations_interpreter.isStorageTouchedByMutations()) { From f68670140600826e8e5699845e52100e394319e3 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 5 May 2019 12:12:59 +0300 Subject: [PATCH 006/161] fix test --- dbms/tests/queries/0_stateless/00935_mutate_index.sh | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00935_mutate_index.sh b/dbms/tests/queries/0_stateless/00935_mutate_index.sh index 5faa9c95bf9e..20316d975e73 100644 --- a/dbms/tests/queries/0_stateless/00935_mutate_index.sh +++ b/dbms/tests/queries/0_stateless/00935_mutate_index.sh @@ -1,14 +1,10 @@ -#!/usr/bin/env bash - -CLICKHOUSE_CLIENT_OPT="--allow_experimental_data_skipping_indices=1" - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. $CURDIR/../shell_config.sh +#!/usr/bin/bash $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" $CLICKHOUSE_CLIENT -n --query=" +SET allow_experimental_data_skipping_indices=1; CREATE TABLE test.minmax_idx ( u64 UInt64, @@ -42,4 +38,4 @@ sleep 0.1 $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 1;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 5;" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" \ No newline at end of file +$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" From 87131d9e99741b3626b289786b37440affae8689 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 5 May 2019 15:17:30 +0300 Subject: [PATCH 007/161] fix --- dbms/src/Storages/IndicesDescription.cpp | 6 +++--- dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 3 ++- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/IndicesDescription.cpp b/dbms/src/Storages/IndicesDescription.cpp index 8c539190e3f2..d658335fadb7 100644 --- a/dbms/src/Storages/IndicesDescription.cpp +++ b/dbms/src/Storages/IndicesDescription.cpp @@ -8,11 +8,11 @@ namespace DB { -IndicesDescription::IndicesDescription(const IndicesDescription & indices_) { - for (const auto & index : indices_.indices) { +IndicesDescription::IndicesDescription(const IndicesDescription & indices_) +{ + for (const auto & index : indices_.indices) indices.push_back( std::dynamic_pointer_cast(index->clone())); - } } String IndicesDescription::toString() const diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 530e3b4218d1..e16bb247ac40 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -895,7 +895,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor std::copy_if( std::cbegin(commands), std::cend(commands), std::back_inserter(commands_for_part), - [&] (const MutationCommand & command) { + [&] (const MutationCommand & command) + { return command.partition == nullptr || future_part.parts[0]->info.partition_id != data.getPartitionIDFromQuery( command.partition, context_for_reading); From 9e3140a45a55d1e0067caa5dc792d78744eb7243 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 5 May 2019 15:40:00 +0300 Subject: [PATCH 008/161] chmod 775 --- dbms/tests/queries/0_stateless/00935_mutate_index.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 dbms/tests/queries/0_stateless/00935_mutate_index.sh diff --git a/dbms/tests/queries/0_stateless/00935_mutate_index.sh b/dbms/tests/queries/0_stateless/00935_mutate_index.sh old mode 100644 new mode 100755 From d079e80659ef2a0dfb89b33366199268ec61e801 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 5 May 2019 15:42:53 +0300 Subject: [PATCH 009/161] operator= --- dbms/src/Storages/IndicesDescription.cpp | 7 +++++++ dbms/src/Storages/IndicesDescription.h | 1 + 2 files changed, 8 insertions(+) diff --git a/dbms/src/Storages/IndicesDescription.cpp b/dbms/src/Storages/IndicesDescription.cpp index d658335fadb7..c0e3a5144dd8 100644 --- a/dbms/src/Storages/IndicesDescription.cpp +++ b/dbms/src/Storages/IndicesDescription.cpp @@ -15,6 +15,13 @@ IndicesDescription::IndicesDescription(const IndicesDescription & indices_) std::dynamic_pointer_cast(index->clone())); } +IndicesDescription & IndicesDescription::operator=(const IndicesDescription & indices_) +{ + for (const auto & index : indices_.indices) + indices.push_back( + std::dynamic_pointer_cast(index->clone())); +} + String IndicesDescription::toString() const { if (indices.empty()) diff --git a/dbms/src/Storages/IndicesDescription.h b/dbms/src/Storages/IndicesDescription.h index 3471ed033fa5..2f349127766a 100644 --- a/dbms/src/Storages/IndicesDescription.h +++ b/dbms/src/Storages/IndicesDescription.h @@ -14,6 +14,7 @@ struct IndicesDescription IndicesDescription() = default; IndicesDescription(const IndicesDescription & indices_); + IndicesDescription & operator=(const IndicesDescription & indices_); bool empty() const { return indices.empty(); } String toString() const; From dea174620b4fa05856b04b5a16a6c9b731a35242 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 5 May 2019 18:12:33 +0300 Subject: [PATCH 010/161] fix --- dbms/src/Storages/IndicesDescription.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Storages/IndicesDescription.cpp b/dbms/src/Storages/IndicesDescription.cpp index c0e3a5144dd8..1b68364e4ec7 100644 --- a/dbms/src/Storages/IndicesDescription.cpp +++ b/dbms/src/Storages/IndicesDescription.cpp @@ -20,6 +20,7 @@ IndicesDescription & IndicesDescription::operator=(const IndicesDescription & in for (const auto & index : indices_.indices) indices.push_back( std::dynamic_pointer_cast(index->clone())); + return *this; } String IndicesDescription::toString() const From 3bf239733297bf1cd39ee61e0d0f8ce28c68efaf Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 5 May 2019 20:01:54 +0300 Subject: [PATCH 011/161] materialize --- .../src/Interpreters/MutationsInterpreter.cpp | 25 +++++++++++++++++++ dbms/src/Parsers/ASTAlterQuery.h | 3 ++- dbms/src/Storages/IndicesDescription.cpp | 4 +-- dbms/src/Storages/IndicesDescription.h | 4 +-- dbms/src/Storages/MutationCommands.cpp | 2 ++ dbms/src/Storages/MutationCommands.h | 1 + 6 files changed, 34 insertions(+), 5 deletions(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 183942f11af1..3a8e8182e1e3 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -270,6 +270,31 @@ void MutationsInterpreter::prepare(bool dry_run) } } } + else if (command.type == MutationCommand::MATERIALIZE_INDEX) + { + auto it = std::find_if( + std::cbegin(indices_desc.indices), std::end(indices_desc.indices), + [&](const std::shared_ptr & index) + { + return index->name == command.index_name; + }); + if (it == std::cend(indices_desc.indices)) + throw Exception("Unknown index: " + command.index_name, ErrorCodes::BAD_ARGUMENTS); + + auto query = (*it)->expr->clone(); + auto syntax_result = SyntaxAnalyzer(context).analyze(query, all_columns); + ExpressionAnalyzer analyzer(query, syntax_result, context); + const auto required_columns = analyzer.getRequiredSourceColumns(); + + for (const String & dependency : required_columns) + { + if (updated_columns.count(dependency)) + { + affected_indices_columns.insert(std::cbegin(required_columns), std::cend(required_columns)); + break; + } + } + } else throw Exception("Unknown mutation command type: " + DB::toString(command.type), ErrorCodes::UNKNOWN_MUTATION_COMMAND); } diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index b348eb5fb9a4..1be61c459a87 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -69,8 +69,9 @@ class ASTAlterCommand : public IAST /** The ADD INDEX query stores the name of the index following AFTER. * The DROP INDEX query stores the name for deletion. + * The MATERIALIZE INDEX query stores the name of the index to materialize. */ - ASTPtr index; + ASTPtr index; /** Used in DROP PARTITION and ATTACH PARTITION FROM queries. * The value or ID of the partition is stored here. diff --git a/dbms/src/Storages/IndicesDescription.cpp b/dbms/src/Storages/IndicesDescription.cpp index 1b68364e4ec7..32c1790e3c1f 100644 --- a/dbms/src/Storages/IndicesDescription.cpp +++ b/dbms/src/Storages/IndicesDescription.cpp @@ -8,7 +8,7 @@ namespace DB { -IndicesDescription::IndicesDescription(const IndicesDescription & indices_) +/*IndicesDescription::IndicesDescription(const IndicesDescription & indices_) { for (const auto & index : indices_.indices) indices.push_back( @@ -21,7 +21,7 @@ IndicesDescription & IndicesDescription::operator=(const IndicesDescription & in indices.push_back( std::dynamic_pointer_cast(index->clone())); return *this; -} +}*/ String IndicesDescription::toString() const { diff --git a/dbms/src/Storages/IndicesDescription.h b/dbms/src/Storages/IndicesDescription.h index 2f349127766a..6c12f9071859 100644 --- a/dbms/src/Storages/IndicesDescription.h +++ b/dbms/src/Storages/IndicesDescription.h @@ -13,8 +13,8 @@ struct IndicesDescription IndicesASTs indices; IndicesDescription() = default; - IndicesDescription(const IndicesDescription & indices_); - IndicesDescription & operator=(const IndicesDescription & indices_); + /*IndicesDescription(const IndicesDescription & indices_); + IndicesDescription & operator=(const IndicesDescription & indices_);*/ bool empty() const { return indices.empty(); } String toString() const; diff --git a/dbms/src/Storages/MutationCommands.cpp b/dbms/src/Storages/MutationCommands.cpp index 03c4b65c8596..41469347e24b 100644 --- a/dbms/src/Storages/MutationCommands.cpp +++ b/dbms/src/Storages/MutationCommands.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -50,6 +51,7 @@ std::optional MutationCommand::parse(ASTAlterCommand * command) res.type = MATERIALIZE_INDEX; res.partition = command->partition; res.predicate = nullptr; + res.index_name = command->index->as().name; return res; } else diff --git a/dbms/src/Storages/MutationCommands.h b/dbms/src/Storages/MutationCommands.h index 3a8616a76777..b8b8fee024aa 100644 --- a/dbms/src/Storages/MutationCommands.h +++ b/dbms/src/Storages/MutationCommands.h @@ -32,6 +32,7 @@ struct MutationCommand std::unordered_map column_to_update_expression; /// For MATERIALIZE INDEX + String index_name; ASTPtr partition; static std::optional parse(ASTAlterCommand * command); From 27aeb3a36d007f48e15b0a1d7af29cdc626a2a4a Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 5 May 2019 21:19:07 +0300 Subject: [PATCH 012/161] fix --- dbms/src/Interpreters/MutationsInterpreter.cpp | 10 +--------- dbms/src/Storages/IndicesDescription.cpp | 15 --------------- dbms/src/Storages/IndicesDescription.h | 2 -- .../MergeTree/MergeTreeDataMergerMutator.cpp | 8 +++++++- ...dex.reference => 00942_mutate_index.reference} | 0 ...0935_mutate_index.sh => 00942_mutate_index.sh} | 0 6 files changed, 8 insertions(+), 27 deletions(-) rename dbms/tests/queries/0_stateless/{00935_mutate_index.reference => 00942_mutate_index.reference} (100%) rename dbms/tests/queries/0_stateless/{00935_mutate_index.sh => 00942_mutate_index.sh} (100%) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 3a8e8182e1e3..1aa9a8472614 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -285,15 +285,7 @@ void MutationsInterpreter::prepare(bool dry_run) auto syntax_result = SyntaxAnalyzer(context).analyze(query, all_columns); ExpressionAnalyzer analyzer(query, syntax_result, context); const auto required_columns = analyzer.getRequiredSourceColumns(); - - for (const String & dependency : required_columns) - { - if (updated_columns.count(dependency)) - { - affected_indices_columns.insert(std::cbegin(required_columns), std::cend(required_columns)); - break; - } - } + affected_indices_columns.insert(std::cbegin(required_columns), std::cend(required_columns)); } else throw Exception("Unknown mutation command type: " + DB::toString(command.type), ErrorCodes::UNKNOWN_MUTATION_COMMAND); diff --git a/dbms/src/Storages/IndicesDescription.cpp b/dbms/src/Storages/IndicesDescription.cpp index 32c1790e3c1f..35e5ae456299 100644 --- a/dbms/src/Storages/IndicesDescription.cpp +++ b/dbms/src/Storages/IndicesDescription.cpp @@ -8,21 +8,6 @@ namespace DB { -/*IndicesDescription::IndicesDescription(const IndicesDescription & indices_) -{ - for (const auto & index : indices_.indices) - indices.push_back( - std::dynamic_pointer_cast(index->clone())); -} - -IndicesDescription & IndicesDescription::operator=(const IndicesDescription & indices_) -{ - for (const auto & index : indices_.indices) - indices.push_back( - std::dynamic_pointer_cast(index->clone())); - return *this; -}*/ - String IndicesDescription::toString() const { if (indices.empty()) diff --git a/dbms/src/Storages/IndicesDescription.h b/dbms/src/Storages/IndicesDescription.h index 6c12f9071859..53b492198ce0 100644 --- a/dbms/src/Storages/IndicesDescription.h +++ b/dbms/src/Storages/IndicesDescription.h @@ -13,8 +13,6 @@ struct IndicesDescription IndicesASTs indices; IndicesDescription() = default; - /*IndicesDescription(const IndicesDescription & indices_); - IndicesDescription & operator=(const IndicesDescription & indices_);*/ bool empty() const { return indices.empty(); } String toString() const; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index e16bb247ac40..4b164d4c6979 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -976,11 +976,14 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor /// We will modify only some of the columns. Other columns and key values can be copied as-is. /// TODO: check that we modify only non-key columns in this case. - /// Checks if columns used in skipping indexes modified + /// Checks if columns used in skipping indexes modified. std::set indices_to_recalc; ASTPtr indices_recalc_expr_list = std::make_shared(); + auto * log = &Poco::Logger::get("kek>>"); + LOG_DEBUG(log, "columns: " << in_header.columns()); for (const auto & col : in_header.getNames()) { + LOG_DEBUG(log, col); for (size_t i = 0; i < data.skip_indices.size(); ++i) { const auto & index = data.skip_indices[i]; @@ -988,6 +991,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor auto it = find(cbegin(index_cols), cend(index_cols), col); if (it != cend(index_cols) && indices_to_recalc.insert(index).second) { + LOG_DEBUG(log, ": " << index->name); ASTPtr expr_list = MergeTreeData::extractKeyExpressionList( storage_from_source_part->getIndices().indices[i]->expr->clone()); for (const auto & expr : expr_list->children) @@ -995,6 +999,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor } } } + /// TODO: add indices from materialize + if (!indices_to_recalc.empty()) { auto indices_recalc_syntax = SyntaxAnalyzer(context, {}).analyze( diff --git a/dbms/tests/queries/0_stateless/00935_mutate_index.reference b/dbms/tests/queries/0_stateless/00942_mutate_index.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00935_mutate_index.reference rename to dbms/tests/queries/0_stateless/00942_mutate_index.reference diff --git a/dbms/tests/queries/0_stateless/00935_mutate_index.sh b/dbms/tests/queries/0_stateless/00942_mutate_index.sh similarity index 100% rename from dbms/tests/queries/0_stateless/00935_mutate_index.sh rename to dbms/tests/queries/0_stateless/00942_mutate_index.sh From 18593391b4ec65b91515381064d3865d8a57b73e Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 5 May 2019 21:53:48 +0300 Subject: [PATCH 013/161] materialize query --- dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 4 ---- dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp | 6 +++--- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 4b164d4c6979..faa4ca6f1e0c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -979,11 +979,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor /// Checks if columns used in skipping indexes modified. std::set indices_to_recalc; ASTPtr indices_recalc_expr_list = std::make_shared(); - auto * log = &Poco::Logger::get("kek>>"); - LOG_DEBUG(log, "columns: " << in_header.columns()); for (const auto & col : in_header.getNames()) { - LOG_DEBUG(log, col); for (size_t i = 0; i < data.skip_indices.size(); ++i) { const auto & index = data.skip_indices[i]; @@ -991,7 +988,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor auto it = find(cbegin(index_cols), cend(index_cols), col); if (it != cend(index_cols) && indices_to_recalc.insert(index).second) { - LOG_DEBUG(log, ": " << index->name); ASTPtr expr_list = MergeTreeData::extractKeyExpressionList( storage_from_source_part->getIndices().indices[i]->expr->clone()); for (const auto & expr : expr_list->children) diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 8f56adf1f968..f115fa602566 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -842,9 +842,9 @@ void MergedColumnOnlyOutputStream::write(const Block & block) /// Creating block for update Block indices_update_block(skip_indexes_columns); /// Filling and writing skip indices like in IMergedBlockOutputStream::writeColumn - for (size_t i = 0; i < storage.skip_indices.size(); ++i) + for (size_t i = 0; i < skip_indices.size(); ++i) { - const auto index = storage.skip_indices[i]; + const auto index = skip_indices[i]; auto & stream = *skip_indices_streams[i]; size_t prev_pos = 0; @@ -930,7 +930,7 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG } /// Finish skip index serialization - for (size_t i = 0; i < storage.skip_indices.size(); ++i) + for (size_t i = 0; i < skip_indices.size(); ++i) { auto & stream = *skip_indices_streams[i]; if (!skip_indices_aggregators[i]->empty()) From e09f177b27e029be072931f269ca6a209a55e01b Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 9 May 2019 15:14:29 +0300 Subject: [PATCH 014/161] fixed test --- dbms/tests/queries/0_stateless/00942_mutate_index.sh | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00942_mutate_index.sh b/dbms/tests/queries/0_stateless/00942_mutate_index.sh index 20316d975e73..c6dd1dfb8361 100755 --- a/dbms/tests/queries/0_stateless/00942_mutate_index.sh +++ b/dbms/tests/queries/0_stateless/00942_mutate_index.sh @@ -1,4 +1,7 @@ -#!/usr/bin/bash +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" From d7c7f03a78cc404d7fce9853e7083092d253fe3f Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 9 May 2019 15:26:25 +0300 Subject: [PATCH 015/161] materialize test --- .../00943_materialize_index.reference | 4 ++ .../0_stateless/00943_materialize_index.sh | 42 +++++++++++++++++++ 2 files changed, 46 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00943_materialize_index.reference create mode 100755 dbms/tests/queries/0_stateless/00943_materialize_index.sh diff --git a/dbms/tests/queries/0_stateless/00943_materialize_index.reference b/dbms/tests/queries/0_stateless/00943_materialize_index.reference new file mode 100644 index 000000000000..8cc6d8ad18e9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00943_materialize_index.reference @@ -0,0 +1,4 @@ +5 + "rows_read": 10, +5 + "rows_read": 6, diff --git a/dbms/tests/queries/0_stateless/00943_materialize_index.sh b/dbms/tests/queries/0_stateless/00943_materialize_index.sh new file mode 100755 index 000000000000..6638065480f5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00943_materialize_index.sh @@ -0,0 +1,42 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" + + +$CLICKHOUSE_CLIENT -n --query=" +SET allow_experimental_data_skipping_indices=1; +CREATE TABLE test.minmax_idx +( + u64 UInt64, + i64 Int64, + i32 Int32, + INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1 +) ENGINE = MergeTree() +ORDER BY u64 +SETTINGS index_granularity = 2;" + + +$CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES +(0, 2, 1), +(1, 1, 2), +(2, 1, 3), +(3, 1, 4), +(4, 2, 5), +(5, 2, 6), +(6, 2, 7), +(7, 2, 8), +(8, 1, 9), +(9, 1, 10)" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx;" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" From 95fb9132d1ed523f627cb1ced3299f5d3b814629 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 9 May 2019 15:30:32 +0300 Subject: [PATCH 016/161] fix materialize test --- dbms/tests/queries/0_stateless/00943_materialize_index.sh | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00943_materialize_index.sh b/dbms/tests/queries/0_stateless/00943_materialize_index.sh index 6638065480f5..4625025da207 100755 --- a/dbms/tests/queries/0_stateless/00943_materialize_index.sh +++ b/dbms/tests/queries/0_stateless/00943_materialize_index.sh @@ -7,13 +7,11 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" $CLICKHOUSE_CLIENT -n --query=" -SET allow_experimental_data_skipping_indices=1; CREATE TABLE test.minmax_idx ( u64 UInt64, i64 Int64, - i32 Int32, - INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1 + i32 Int32 ) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 2;" @@ -34,6 +32,10 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT -n --query=" +SET allow_experimental_data_skipping_indices=1; +ALTER TABLE test.minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1;" + $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" From 0a4761a8fae8e9b9b938b2286951a35e4c053fe4 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 9 May 2019 15:49:00 +0300 Subject: [PATCH 017/161] test for clear index --- .../00944_clear_index_in_partition.reference | 4 ++ .../00944_clear_index_in_partition.sh | 43 +++++++++++++++++++ 2 files changed, 47 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference create mode 100755 dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh diff --git a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference new file mode 100644 index 000000000000..c3e1b8bdb9aa --- /dev/null +++ b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference @@ -0,0 +1,4 @@ +5 + "rows_read": 6, +5 + "rows_read": 10, diff --git a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh new file mode 100755 index 000000000000..1d08f9facfc2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" + + +$CLICKHOUSE_CLIENT -n --query=" +SET allow_experimental_data_skipping_indices=1; +CREATE TABLE test.minmax_idx +( + u64 UInt64, + i64 Int64, + i32 Int32, + INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1 +) ENGINE = MergeTree() +PARTITION BY i32 +ORDER BY u64 +SETTINGS index_granularity = 2;" + + +$CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES +(0, 2, 1), +(1, 1, 1), +(2, 1, 1), +(3, 1, 1), +(4, 2, 3), +(5, 2, 3), +(6, 2, 2), +(7, 2, 2), +(8, 1, 2), +(9, 1, 2)" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 1;" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" From 1dc2eed16949717bee8f4c852ea2324fa08f2067 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 9 May 2019 15:53:08 +0300 Subject: [PATCH 018/161] materialize in partition --- .../00943_materialize_index.reference | 4 +++ .../0_stateless/00943_materialize_index.sh | 29 +++++++++++++------ 2 files changed, 24 insertions(+), 9 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00943_materialize_index.reference b/dbms/tests/queries/0_stateless/00943_materialize_index.reference index 8cc6d8ad18e9..00aa71e8bd10 100644 --- a/dbms/tests/queries/0_stateless/00943_materialize_index.reference +++ b/dbms/tests/queries/0_stateless/00943_materialize_index.reference @@ -1,4 +1,8 @@ 5 "rows_read": 10, +5 + "rows_read": 8, +5 + "rows_read": 6, 5 "rows_read": 6, diff --git a/dbms/tests/queries/0_stateless/00943_materialize_index.sh b/dbms/tests/queries/0_stateless/00943_materialize_index.sh index 4625025da207..4ab061d93474 100755 --- a/dbms/tests/queries/0_stateless/00943_materialize_index.sh +++ b/dbms/tests/queries/0_stateless/00943_materialize_index.sh @@ -13,21 +13,22 @@ CREATE TABLE test.minmax_idx i64 Int64, i32 Int32 ) ENGINE = MergeTree() +PARTITION BY i32 ORDER BY u64 SETTINGS index_granularity = 2;" $CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES (0, 2, 1), -(1, 1, 2), -(2, 1, 3), -(3, 1, 4), -(4, 2, 5), -(5, 2, 6), -(6, 2, 7), -(7, 2, 8), -(8, 1, 9), -(9, 1, 10)" +(1, 1, 1), +(2, 1, 1), +(3, 1, 1), +(4, 2, 2), +(5, 2, 2), +(6, 2, 2), +(7, 2, 2), +(8, 1, 2), +(9, 1, 2)" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" @@ -36,6 +37,16 @@ $CLICKHOUSE_CLIENT -n --query=" SET allow_experimental_data_skipping_indices=1; ALTER TABLE test.minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1;" +$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 2;" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" + $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" From 8d2b32fda896e5e9c5965620b8f5e2e1218a882d Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 9 May 2019 17:25:18 +0300 Subject: [PATCH 019/161] clear index --- dbms/src/Parsers/ASTAlterQuery.cpp | 7 ++- dbms/src/Parsers/ASTAlterQuery.h | 7 ++- dbms/src/Parsers/ParserAlterQuery.cpp | 31 ++++++++--- dbms/src/Storages/IndicesDescription.cpp | 10 ++++ dbms/src/Storages/IndicesDescription.h | 3 +- .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 12 +++++ .../MergeTree/ReplicatedMergeTreeLogEntry.h | 5 +- dbms/src/Storages/PartitionCommands.cpp | 21 ++++++++ dbms/src/Storages/PartitionCommands.h | 4 +- dbms/src/Storages/StorageMergeTree.cpp | 29 ++++++++--- dbms/src/Storages/StorageMergeTree.h | 3 +- .../Storages/StorageReplicatedMergeTree.cpp | 52 ++++++++++++++----- .../src/Storages/StorageReplicatedMergeTree.h | 4 +- 13 files changed, 151 insertions(+), 37 deletions(-) diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index acef31d09467..f970db34c821 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -102,8 +102,13 @@ void ASTAlterCommand::formatImpl( else if (type == ASTAlterCommand::DROP_INDEX) { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str - << "DROP INDEX " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); + << (clear_index ? "CLEAR " : "DROP ") << "INDEX " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); index->formatImpl(settings, state, frame); + if (partition) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str<< " IN PARTITION " << (settings.hilite ? hilite_none : ""); + partition->formatImpl(settings, state, frame); + } } else if (type == ASTAlterCommand::MATERIALIZE_INDEX) { diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index 1be61c459a87..f660dc7b9268 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -70,6 +70,7 @@ class ASTAlterCommand : public IAST /** The ADD INDEX query stores the name of the index following AFTER. * The DROP INDEX query stores the name for deletion. * The MATERIALIZE INDEX query stores the name of the index to materialize. + * The CLEAR INDEX query stores the name of the index to clear. */ ASTPtr index; @@ -96,9 +97,11 @@ class ASTAlterCommand : public IAST bool clear_column = false; /// for CLEAR COLUMN (do not drop column from metadata) - bool if_not_exists = false; /// option for ADD_COLUMN + bool clear_index = false; /// for CLEAR INDEX (do not drop index from metadata) - bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN + bool if_not_exists = false; /// option for ADD_COLUMN + + bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN /** For FETCH PARTITION - the path in ZK to the shard, from which to download the partition. */ diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 71e10a8dc972..f823ee846238 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -31,6 +31,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_add_index("ADD INDEX"); ParserKeyword s_drop_index("DROP INDEX"); + ParserKeyword s_clear_index("CLEAR INDEX"); ParserKeyword s_materialize_index("MATERIALIZE INDEX"); ParserKeyword s_attach_partition("ATTACH PARTITION"); @@ -99,6 +100,24 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::DROP_COLUMN; command->detach = false; } + else if (s_clear_column.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_name.parse(pos, command->column, expected)) + return false; + + command->type = ASTAlterCommand::DROP_COLUMN; + command->clear_column = true; + command->detach = false; + + if (s_in_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + } + } else if (s_add_index.ignore(pos, expected)) { if (s_if_not_exists.ignore(pos, expected)) @@ -126,7 +145,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::DROP_INDEX; command->detach = false; } - else if (s_materialize_index.ignore(pos, expected)) + else if (s_clear_index.ignore(pos, expected)) { if (s_if_exists.ignore(pos, expected)) command->if_exists = true; @@ -134,7 +153,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!parser_name.parse(pos, command->index, expected)) return false; - command->type = ASTAlterCommand::MATERIALIZE_INDEX; + command->type = ASTAlterCommand::DROP_INDEX; + command->clear_index = true; command->detach = false; if (s_in_partition.ignore(pos, expected)) @@ -143,16 +163,15 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; } } - else if (s_clear_column.ignore(pos, expected)) + else if (s_materialize_index.ignore(pos, expected)) { if (s_if_exists.ignore(pos, expected)) command->if_exists = true; - if (!parser_name.parse(pos, command->column, expected)) + if (!parser_name.parse(pos, command->index, expected)) return false; - command->type = ASTAlterCommand::DROP_COLUMN; - command->clear_column = true; + command->type = ASTAlterCommand::MATERIALIZE_INDEX; command->detach = false; if (s_in_partition.ignore(pos, expected)) diff --git a/dbms/src/Storages/IndicesDescription.cpp b/dbms/src/Storages/IndicesDescription.cpp index 35e5ae456299..c2e461cee8ab 100644 --- a/dbms/src/Storages/IndicesDescription.cpp +++ b/dbms/src/Storages/IndicesDescription.cpp @@ -8,6 +8,16 @@ namespace DB { +bool IndicesDescription::has(const String & name) const +{ + return std::cend(indices) != std::find_if( + std::cbegin(indices), std::cend(indices), + [&name](const auto & index) + { + return index->name == name; + }); +} + String IndicesDescription::toString() const { if (indices.empty()) diff --git a/dbms/src/Storages/IndicesDescription.h b/dbms/src/Storages/IndicesDescription.h index 53b492198ce0..06ce2d366b10 100644 --- a/dbms/src/Storages/IndicesDescription.h +++ b/dbms/src/Storages/IndicesDescription.h @@ -15,8 +15,9 @@ struct IndicesDescription IndicesDescription() = default; bool empty() const { return indices.empty(); } - String toString() const; + bool has(const String & name) const; + String toString() const; static IndicesDescription parse(const String & str); }; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index fb3fe01bf871..29878cc064e1 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -46,6 +46,13 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const << new_part_name; break; + case CLEAR_INDEX: + out << "clear_index\n" + << escape << index_name + << "\nfrom\n" + << new_part_name; + break; + case REPLACE_RANGE: out << typeToString(REPLACE_RANGE) << "\n"; replace_range_entry->writeText(out); @@ -125,6 +132,11 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) type = CLEAR_COLUMN; in >> escape >> column_name >> "\nfrom\n" >> new_part_name; } + else if (type_str == "clear_index") + { + type = CLEAR_INDEX; + in >> escape >> index_name >> "\nfrom\n" >> new_part_name; + } else if (type_str == typeToString(REPLACE_RANGE)) { type = REPLACE_RANGE; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 602249c86ea1..5a446e7baeea 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -34,6 +34,7 @@ struct ReplicatedMergeTreeLogEntryData MERGE_PARTS, /// Merge the parts. DROP_RANGE, /// Delete the parts in the specified partition in the specified number range. CLEAR_COLUMN, /// Drop specific column from specified partition. + CLEAR_INDEX, /// Drop specific index from specified partition. REPLACE_RANGE, /// Drop certain range of partitions and replace them by new ones MUTATE_PART, /// Apply one or several mutations to the part. }; @@ -46,6 +47,7 @@ struct ReplicatedMergeTreeLogEntryData case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS"; case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE"; case ReplicatedMergeTreeLogEntryData::CLEAR_COLUMN: return "CLEAR_COLUMN"; + case ReplicatedMergeTreeLogEntryData::CLEAR_INDEX: return "CLEAR_INDEX"; case ReplicatedMergeTreeLogEntryData::REPLACE_RANGE: return "REPLACE_RANGE"; case ReplicatedMergeTreeLogEntryData::MUTATE_PART: return "MUTATE_PART"; default: @@ -76,6 +78,7 @@ struct ReplicatedMergeTreeLogEntryData Strings source_parts; bool deduplicate = false; /// Do deduplicate on merge String column_name; + String index_name; /// For DROP_RANGE, true means that the parts need not be deleted, but moved to the `detached` directory. bool detach = false; @@ -108,7 +111,7 @@ struct ReplicatedMergeTreeLogEntryData /// Return {} because selection of merges in the partition where the column is cleared /// should not be blocked (only execution of merges should be blocked). - if (type == CLEAR_COLUMN) + if (type == CLEAR_COLUMN || type == CLEAR_INDEX) return {}; if (type == REPLACE_RANGE) diff --git a/dbms/src/Storages/PartitionCommands.cpp b/dbms/src/Storages/PartitionCommands.cpp index f6aaee4c70eb..054a8be4e4eb 100644 --- a/dbms/src/Storages/PartitionCommands.cpp +++ b/dbms/src/Storages/PartitionCommands.cpp @@ -68,6 +68,17 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * res.column_name = *getIdentifierName(command_ast->column); return res; } + else if (command_ast->type == ASTAlterCommand::DROP_INDEX && command_ast->partition) + { + if (!command_ast->clear_index) + throw Exception("Can't DROP INDEX from partition. It is possible only to CLEAR INDEX in partition", ErrorCodes::BAD_ARGUMENTS); + + PartitionCommand res; + res.type = CLEAR_INDEX; + res.partition = command_ast->partition; + res.index_name = *getIdentifierName(command_ast->index); + return res; + } else if (command_ast->type == ASTAlterCommand::FREEZE_ALL) { PartitionCommand command; @@ -93,6 +104,16 @@ void PartitionCommands::validate(const IStorage & table) DB::ErrorCodes::ILLEGAL_COLUMN); } } + else if (command.type == PartitionCommand::CLEAR_INDEX) + { + String index_name = command.index_name.safeGet(); + + if (!table.getIndices().has(index_name)) + { + throw Exception("Wrong index name. Cannot find index " + index_name + " to clear it from partition", + DB::ErrorCodes::BAD_ARGUMENTS); + } + } } } diff --git a/dbms/src/Storages/PartitionCommands.h b/dbms/src/Storages/PartitionCommands.h index 987af8baef45..7f9beedfc57c 100644 --- a/dbms/src/Storages/PartitionCommands.h +++ b/dbms/src/Storages/PartitionCommands.h @@ -19,17 +19,19 @@ struct PartitionCommand { ATTACH_PARTITION, CLEAR_COLUMN, + CLEAR_INDEX, DROP_PARTITION, FETCH_PARTITION, FREEZE_ALL_PARTITIONS, FREEZE_PARTITION, - REPLACE_PARTITION, + REPLACE_PARTITION }; Type type; ASTPtr partition; Field column_name; + Field index_name; /// true for DETACH PARTITION. bool detach = false; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 4075e1dc5275..8f3539e0cc2a 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -781,7 +781,7 @@ void StorageMergeTree::clearOldMutations() } -void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context) +void StorageMergeTree::clearColumnOrIndexInPartition(const ASTPtr & partition, const AlterCommand & alter_command, const Context & context) { /// Asks to complete merges and does not allow them to start. /// This protects against "revival" of data for a removed partition after completion of merge. @@ -795,10 +795,6 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi std::vector transactions; - AlterCommand alter_command; - alter_command.type = AlterCommand::DROP_COLUMN; - alter_command.column_name = get(column_name); - auto new_columns = getColumns(); auto new_indices = getIndices(); ASTPtr ignored_order_by_ast; @@ -815,7 +811,10 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi if (auto transaction = alterDataPart(part, columns_for_parts, new_indices.indices, false)) transactions.push_back(std::move(transaction)); - LOG_DEBUG(log, "Removing column " << get(column_name) << " from part " << part->name); + if (alter_command.type == AlterCommand::DROP_COLUMN) + LOG_DEBUG(log, "Removing column " << alter_command.column_name << " from part " << part->name); + else if (alter_command.type == AlterCommand::DROP_INDEX) + LOG_DEBUG(log, "Removing index " << alter_command.index_name << " from part " << part->name); } if (transactions.empty()) @@ -900,8 +899,22 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma break; case PartitionCommand::CLEAR_COLUMN: - clearColumnInPartition(command.partition, command.column_name, context); - break; + { + AlterCommand alter_command; + alter_command.type = AlterCommand::DROP_COLUMN; + alter_command.column_name = get(command.column_name); + clearColumnOrIndexInPartition(command.partition, alter_command, context); + } + break; + + case PartitionCommand::CLEAR_INDEX: + { + AlterCommand alter_command; + alter_command.type = AlterCommand::DROP_COLUMN; + alter_command.column_name = get(command.index_name); + clearColumnOrIndexInPartition(command.partition, alter_command, context); + } + break; case PartitionCommand::FREEZE_ALL_PARTITIONS: { diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index ff48682a03e5..0f7a07cc72ce 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -115,7 +116,7 @@ class StorageMergeTree : public ext::shared_ptr_helper, public // Partition helpers void dropPartition(const ASTPtr & partition, bool detach, const Context & context); - void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context); + void clearColumnOrIndexInPartition(const ASTPtr & partition, const AlterCommand & alter_command, const Context & context); void attachPartition(const ASTPtr & partition, bool part, const Context & context); void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index c5187995d4d8..b5b1f5b4f33b 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -866,7 +866,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) if (entry.type == LogEntry::CLEAR_COLUMN) { - executeClearColumnInPartition(entry); + executeClearColumnOrIndexInPartition(entry); return true; } @@ -1487,7 +1487,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) } -void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & entry) +void StorageReplicatedMergeTree::executeClearColumnOrIndexInPartition(const LogEntry & entry) { LOG_INFO(log, "Clear column " << entry.column_name << " in parts inside " << entry.new_part_name << " range"); @@ -1501,8 +1501,16 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & auto zookeeper = getZooKeeper(); AlterCommand alter_command; - alter_command.type = AlterCommand::DROP_COLUMN; - alter_command.column_name = entry.column_name; + if (entry.type == LogEntry::CLEAR_COLUMN) + { + alter_command.type = AlterCommand::DROP_COLUMN; + alter_command.column_name = entry.column_name; + } + else if (entry.type == LogEntry::CLEAR_INDEX) + { + alter_command.type = AlterCommand::DROP_INDEX; + alter_command.index_name = entry.index_name; + } auto new_columns = getColumns(); auto new_indices = getIndices(); @@ -1527,7 +1535,10 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & if (!entry_part_info.contains(part->info)) continue; - LOG_DEBUG(log, "Clearing column " << entry.column_name << " in part " << part->name); + if (entry.type == LogEntry::CLEAR_COLUMN) + LOG_DEBUG(log, "Clearing column " << alter_command.column_name << " in part " << part->name); + else if (entry.type == LogEntry::CLEAR_INDEX) + LOG_DEBUG(log, "Clearing index " << alter_command.index_name << " in part " << part->name); auto transaction = alterDataPart(part, columns_for_parts, new_indices.indices, false); if (!transaction) @@ -1538,7 +1549,10 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & ++modified_parts; } - LOG_DEBUG(log, "Cleared column " << entry.column_name << " in " << modified_parts << " parts"); + if (entry.type == LogEntry::CLEAR_COLUMN) + LOG_DEBUG(log, "Cleared column " << entry.column_name << " in " << modified_parts << " parts"); + else if (entry.type == LogEntry::CLEAR_INDEX) + LOG_DEBUG(log, "Cleared index " << entry.index_name << " in " << modified_parts << " parts"); /// Recalculate columns size (not only for the modified column) recalculateColumnSizes(); @@ -3373,8 +3387,22 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part break; case PartitionCommand::CLEAR_COLUMN: - clearColumnInPartition(command.partition, command.column_name, query_context); - break; + { + LogEntry entry; + entry.type = LogEntry::CLEAR_COLUMN; + entry.column_name = command.column_name.safeGet(); + clearColumnOrIndexInPartition(command.partition, std::move(entry), query_context); + } + break; + + case PartitionCommand::CLEAR_INDEX: + { + LogEntry entry; + entry.type = LogEntry::CLEAR_INDEX; + entry.index_name = command.index_name.safeGet(); + clearColumnOrIndexInPartition(command.partition, std::move(entry), query_context); + } + break; case PartitionCommand::FREEZE_ALL_PARTITIONS: { @@ -3438,8 +3466,8 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St } -void StorageReplicatedMergeTree::clearColumnInPartition( - const ASTPtr & partition, const Field & column_name, const Context & query_context) +void StorageReplicatedMergeTree::clearColumnOrIndexInPartition( + const ASTPtr & partition, LogEntry && entry, const Context & query_context) { assertNotReadonly(); @@ -3455,11 +3483,7 @@ void StorageReplicatedMergeTree::clearColumnInPartition( } /// We allocated new block number for this part, so new merges can't merge clearing parts with new ones - - LogEntry entry; - entry.type = LogEntry::CLEAR_COLUMN; entry.new_part_name = getPartNamePossiblyFake(format_version, drop_range_info); - entry.column_name = column_name.safeGet(); entry.create_time = time(nullptr); String log_znode_path = getZooKeeper()->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index eba0511e15ef..133535e16398 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -374,7 +374,7 @@ class StorageReplicatedMergeTree : public ext::shared_ptr_helper findClusterAddress(const ReplicatedMergeTreeAddress & leader_address) const; // Partition helpers - void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & query_context); + void clearColumnOrIndexInPartition(const ASTPtr & partition, LogEntry && entry, const Context & query_context); void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & query_context); void attachPartition(const ASTPtr & partition, bool part, const Context & query_context); void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & query_context); From ad07dd2e1ffdaa0a5f1f13e8d2313212f924144d Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 9 May 2019 20:04:05 +0300 Subject: [PATCH 020/161] fix --- dbms/src/Parsers/ParserAlterQuery.cpp | 9 ++++----- dbms/src/Storages/AlterCommands.cpp | 4 ++-- dbms/src/Storages/StorageMergeTree.cpp | 4 ++-- .../00944_clear_index_in_partition.reference | 4 +++- .../00944_clear_index_in_partition.sh | 16 +++++++++++----- 5 files changed, 22 insertions(+), 15 deletions(-) diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index f823ee846238..a4ffcab7c983 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -157,11 +157,10 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->clear_index = true; command->detach = false; - if (s_in_partition.ignore(pos, expected)) - { - if (!parser_partition.parse(pos, command->partition, expected)) - return false; - } + if (!s_in_partition.ignore(pos, expected)) + return false; + if (!parser_partition.parse(pos, command->partition, expected)) + return false; } else if (s_materialize_index.ignore(pos, expected)) { diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 88f3e909f49c..5d8e8c97ec16 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -153,10 +153,10 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ return command; } - else if (command_ast->type == ASTAlterCommand::DROP_INDEX) + else if (command_ast->type == ASTAlterCommand::DROP_INDEX && !command_ast->partition) { if (command_ast->clear_column) - throw Exception("\"ALTER TABLE table CLEAR COLUMN column\" queries are not supported yet. Use \"CLEAR COLUMN column IN PARTITION\".", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("\"ALTER TABLE table CLEAR INDEX index\" queries are not supported yet. Use \"CLEAR INDEX index IN PARTITION\".", ErrorCodes::NOT_IMPLEMENTED); AlterCommand command; command.type = AlterCommand::DROP_INDEX; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 8f3539e0cc2a..08599a6dd72d 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -910,8 +910,8 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma case PartitionCommand::CLEAR_INDEX: { AlterCommand alter_command; - alter_command.type = AlterCommand::DROP_COLUMN; - alter_command.column_name = get(command.index_name); + alter_command.type = AlterCommand::DROP_INDEX; + alter_command.index_name = get(command.index_name); clearColumnOrIndexInPartition(command.partition, alter_command, context); } break; diff --git a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference index c3e1b8bdb9aa..a2feb54381d3 100644 --- a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference +++ b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference @@ -1,4 +1,6 @@ +5 + "rows_read": 4, 5 "rows_read": 6, 5 - "rows_read": 10, + "rows_read": 4, \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh index 1d08f9facfc2..123b2eb38a47 100755 --- a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh +++ b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh @@ -25,10 +25,10 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES (1, 1, 1), (2, 1, 1), (3, 1, 1), -(4, 2, 3), -(5, 2, 3), -(6, 2, 2), -(7, 2, 2), +(4, 1, 1), +(5, 2, 1), +(6, 1, 2), +(7, 1, 2), (8, 1, 2), (9, 1, 2)" @@ -40,4 +40,10 @@ $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTI $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" +$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" + + +#$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" From 6c0722db04b7940e110e8bc2af81b7ac671fc9e6 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 9 May 2019 20:27:20 +0300 Subject: [PATCH 021/161] fix mutations with parts --- dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index faa4ca6f1e0c..03805c333746 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -898,7 +898,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor [&] (const MutationCommand & command) { return command.partition == nullptr || - future_part.parts[0]->info.partition_id != data.getPartitionIDFromQuery( + future_part.parts[0]->info.partition_id == data.getPartitionIDFromQuery( command.partition, context_for_reading); }); From f74fd052771e3489261347e78e3a631f795f1e22 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 9 May 2019 21:58:54 +0300 Subject: [PATCH 022/161] fix test --- .../0_stateless/00944_clear_index_in_partition.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference index a2feb54381d3..59e597fe70be 100644 --- a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference +++ b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference @@ -1,6 +1,6 @@ -5 +2 "rows_read": 4, -5 +2 "rows_read": 6, -5 +2 "rows_read": 4, \ No newline at end of file From cad2501b27477471b23cf121b85fe52a953dde6b Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 9 May 2019 23:26:30 +0300 Subject: [PATCH 023/161] fix test --- .../0_stateless/00944_clear_index_in_partition.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference index 59e597fe70be..9c5382e5cbe4 100644 --- a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference +++ b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.reference @@ -3,4 +3,4 @@ 2 "rows_read": 6, 2 - "rows_read": 4, \ No newline at end of file + "rows_read": 4, From 9873b046119be84e09b0901efc042bb3eade0a00 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Tue, 25 Jun 2019 16:10:09 +0300 Subject: [PATCH 024/161] StringRef -> String (race cond) --- dbms/src/Parsers/ASTPartition.h | 2 +- dbms/src/Parsers/ParserPartition.cpp | 8 ++++---- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- .../queries/0_stateless/00944_clear_index_in_partition.sh | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dbms/src/Parsers/ASTPartition.h b/dbms/src/Parsers/ASTPartition.h index d87206d7bb4a..8a837a10451f 100644 --- a/dbms/src/Parsers/ASTPartition.h +++ b/dbms/src/Parsers/ASTPartition.h @@ -12,7 +12,7 @@ class ASTPartition : public IAST { public: ASTPtr value; - StringRef fields_str; /// The extent of comma-separated partition expression fields without parentheses. + String fields_str; /// The extent of comma-separated partition expression fields without parentheses. size_t fields_count = 0; String id; diff --git a/dbms/src/Parsers/ParserPartition.cpp b/dbms/src/Parsers/ParserPartition.cpp index 6d2c259f8bf3..4dc1f4cabf92 100644 --- a/dbms/src/Parsers/ParserPartition.cpp +++ b/dbms/src/Parsers/ParserPartition.cpp @@ -35,7 +35,7 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; size_t fields_count; - StringRef fields_str; + String fields_str; const auto * tuple_ast = value->as(); if (tuple_ast && tuple_ast->name == "tuple") @@ -59,17 +59,17 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (right_paren->type != TokenType::ClosingRoundBracket) return false; - fields_str = StringRef(left_paren->end, right_paren->begin - left_paren->end); + fields_str = String(left_paren->end, right_paren->begin - left_paren->end); } else { fields_count = 1; - fields_str = StringRef(begin->begin, pos->begin - begin->begin); + fields_str = String(begin->begin, pos->begin - begin->begin); } partition->value = value; partition->children.push_back(value); - partition->fields_str = fields_str; + partition->fields_str = std::move(fields_str); partition->fields_count = fields_count; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 894ee53e2daf..f21e75eef75c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -2492,7 +2492,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context if (fields_count) { ReadBufferFromMemory left_paren_buf("(", 1); - ReadBufferFromMemory fields_buf(partition_ast.fields_str.data, partition_ast.fields_str.size); + ReadBufferFromMemory fields_buf(partition_ast.fields_str.data(), partition_ast.fields_str.size()); ReadBufferFromMemory right_paren_buf(")", 1); ConcatReadBuffer buf({&left_paren_buf, &fields_buf, &right_paren_buf}); @@ -2502,7 +2502,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context RowReadExtension unused; if (!input_stream.read(columns, unused)) throw Exception( - "Could not parse partition value: `" + partition_ast.fields_str.toString() + "`", + "Could not parse partition value: `" + partition_ast.fields_str + "`", ErrorCodes::INVALID_PARTITION_VALUE); for (size_t i = 0; i < fields_count; ++i) diff --git a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh index 5a179a8b20a7..9047bbb3a721 100755 --- a/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh +++ b/dbms/tests/queries/0_stateless/00944_clear_index_in_partition.sh @@ -48,4 +48,4 @@ $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -#$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" +$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" From 1118ce04b951271ddac4619a4d86476a9cfe7ffa Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sat, 27 Jul 2019 18:36:40 +0300 Subject: [PATCH 025/161] fixed --- dbms/src/Storages/IStorage.cpp | 10 ++++++++++ dbms/src/Storages/IStorage.h | 2 ++ .../MergeTree/MergedColumnOnlyOutputStream.cpp | 17 ++++++++++------- .../MergeTree/MergedColumnOnlyOutputStream.h | 1 + .../MergeTree/StorageFromMergeTreeDataPart.h | 2 +- 5 files changed, 24 insertions(+), 8 deletions(-) diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index b93c6ae478c8..07b2b94c1910 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -25,6 +25,11 @@ IStorage::IStorage(ColumnsDescription columns_) setColumns(std::move(columns_)); } +IStorage::IStorage(ColumnsDescription columns_, ColumnsDescription virtuals_) : virtuals(std::move(virtuals_)) +{ + setColumns(std::move(columns_)); +} + IStorage::IStorage(ColumnsDescription columns_, ColumnsDescription virtuals_, IndicesDescription indices_) : virtuals(std::move(virtuals_)) { setColumns(std::move(columns_)); @@ -36,6 +41,11 @@ const ColumnsDescription & IStorage::getColumns() const return columns; } +const ColumnsDescription & IStorage::getVirtuals() const +{ + return virtuals; +} + const IndicesDescription & IStorage::getIndices() const { return indices; diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index a2f5199486fd..38441c10ee94 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -64,6 +64,7 @@ class IStorage : public std::enable_shared_from_this public: IStorage() = default; explicit IStorage(ColumnsDescription columns_); + IStorage(ColumnsDescription columns_, ColumnsDescription virtuals_); IStorage(ColumnsDescription columns_, ColumnsDescription virtuals_, IndicesDescription indices_); virtual ~IStorage() = default; @@ -102,6 +103,7 @@ class IStorage : public std::enable_shared_from_this public: /// thread-unsafe part. lockStructure must be acquired const ColumnsDescription & getColumns() const; /// returns combined set of columns + const ColumnsDescription & getVirtuals() const; const IndicesDescription & getIndices() const; /// NOTE: these methods should include virtual columns, diff --git a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index faf82321c835..d8eaef2a7814 100644 --- a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -85,14 +85,15 @@ void MergedColumnOnlyOutputStream::write(const Block & block) { /// Creating block for update Block indices_update_block(skip_indexes_columns); + size_t skip_index_current_mark = 0; + /// Filling and writing skip indices like in IMergedBlockOutputStream::writeColumn - for (size_t i = 0; i < skip_indices.size(); ++i) + for (size_t i = 0; i < storage.skip_indices.size(); ++i) { - const auto index = skip_indices[i]; + const auto index = storage.skip_indices[i]; auto & stream = *skip_indices_streams[i]; size_t prev_pos = 0; - - size_t current_mark = 0; + skip_index_current_mark = skip_index_mark; while (prev_pos < rows) { UInt64 limit = 0; @@ -102,7 +103,7 @@ void MergedColumnOnlyOutputStream::write(const Block & block) } else { - limit = index_granularity.getMarkRows(current_mark); + limit = index_granularity.getMarkRows(skip_index_current_mark); if (skip_indices_aggregators[i]->empty()) { skip_indices_aggregators[i] = index->createIndexAggregator(); @@ -115,8 +116,10 @@ void MergedColumnOnlyOutputStream::write(const Block & block) writeIntBinary(stream.compressed.offset(), stream.marks); /// Actually this numbers is redundant, but we have to store them /// to be compatible with normal .mrk2 file format - if (storage.index_granularity_info.is_adaptive) + if (storage.canUseAdaptiveGranularity()) writeIntBinary(1UL, stream.marks); + + ++skip_index_current_mark; } } @@ -135,9 +138,9 @@ void MergedColumnOnlyOutputStream::write(const Block & block) } } prev_pos = pos; - current_mark++; } } + skip_index_mark = skip_index_current_mark; } size_t new_index_offset = 0; diff --git a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index cdcbe8372205..645e9f2ff365 100644 --- a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -32,6 +32,7 @@ class MergedColumnOnlyOutputStream final : public IMergedBlockOutputStream bool initialized = false; bool sync; bool skip_offsets; + size_t skip_index_mark = 0; std::vector skip_indices; std::vector> skip_indices_streams; diff --git a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index c5d12b2fc988..83582c9779f9 100644 --- a/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -40,7 +40,7 @@ class StorageFromMergeTreeDataPart : public ext::shared_ptr_helperstorage.getColumns(), part_->storage.getIndices()), part(part_) + : IStorage(part_->storage.getColumns(), part_->storage.getVirtuals(), part_->storage.getIndices()), part(part_) {} private: From 63fe71442003a17c708247731484365a183faba8 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 28 Jul 2019 14:10:35 +0300 Subject: [PATCH 026/161] fix --- .../MergeTree/IMergedBlockOutputStream.cpp | 105 ++++++++++++++++ .../MergeTree/IMergedBlockOutputStream.h | 15 ++- .../MergeTree/MergeTreeDataMergerMutator.cpp | 1 - .../MergeTree/MergedBlockOutputStream.cpp | 114 ++---------------- .../MergeTree/MergedBlockOutputStream.h | 7 -- .../MergedColumnOnlyOutputStream.cpp | 108 ++--------------- .../MergeTree/MergedColumnOnlyOutputStream.h | 8 -- 7 files changed, 140 insertions(+), 218 deletions(-) diff --git a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index fc19fbd67929..a7475ae7c188 100644 --- a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -12,18 +12,22 @@ namespace ErrorCodes namespace { constexpr auto DATA_FILE_EXTENSION = ".bin"; + constexpr auto INDEX_FILE_EXTENSION = ".idx"; } IMergedBlockOutputStream::IMergedBlockOutputStream( MergeTreeData & storage_, + String part_path_, size_t min_compress_block_size_, size_t max_compress_block_size_, CompressionCodecPtr codec_, size_t aio_threshold_, bool blocks_are_granules_size_, + const std::vector & indices_to_recalc, const MergeTreeIndexGranularity & index_granularity_) : storage(storage_) + , part_path(part_path_) , min_compress_block_size(min_compress_block_size_) , max_compress_block_size(max_compress_block_size_) , aio_threshold(aio_threshold_) @@ -32,6 +36,7 @@ IMergedBlockOutputStream::IMergedBlockOutputStream( , index_granularity(index_granularity_) , compute_granularity(index_granularity.empty()) , codec(std::move(codec_)) + , skip_indices(indices_to_recalc) , with_final_mark(storage.settings.write_final_mark && storage.canUseAdaptiveGranularity()) { if (blocks_are_granules_size && !index_granularity.empty()) @@ -304,6 +309,106 @@ void IMergedBlockOutputStream::writeFinalMark( }, path); } +void IMergedBlockOutputStream::initSkipIndices() +{ + for (const auto & index : skip_indices) + { + String stream_name = index->getFileName(); + skip_indices_streams.emplace_back( + std::make_unique( + stream_name, + part_path + stream_name, INDEX_FILE_EXTENSION, + part_path + stream_name, marks_file_extension, + codec, max_compress_block_size, + 0, aio_threshold)); + skip_indices_aggregators.push_back(index->createIndexAggregator()); + skip_index_filling.push_back(0); + } +} + +void IMergedBlockOutputStream::calculateAndSerializeSkipIndices( + const ColumnsWithTypeAndName & skip_indexes_columns, size_t rows) +{ + /// Creating block for update + Block indices_update_block(skip_indexes_columns); + size_t skip_index_current_mark = 0; + + /// Filling and writing skip indices like in IMergedBlockOutputStream::writeColumn + for (size_t i = 0; i < storage.skip_indices.size(); ++i) + { + const auto index = storage.skip_indices[i]; + auto & stream = *skip_indices_streams[i]; + size_t prev_pos = 0; + skip_index_current_mark = skip_index_mark; + while (prev_pos < rows) + { + UInt64 limit = 0; + if (prev_pos == 0 && index_offset != 0) + { + limit = index_offset; + } + else + { + limit = index_granularity.getMarkRows(skip_index_current_mark); + if (skip_indices_aggregators[i]->empty()) + { + skip_indices_aggregators[i] = index->createIndexAggregator(); + skip_index_filling[i] = 0; + + if (stream.compressed.offset() >= min_compress_block_size) + stream.compressed.next(); + + writeIntBinary(stream.plain_hashing.count(), stream.marks); + writeIntBinary(stream.compressed.offset(), stream.marks); + /// Actually this numbers is redundant, but we have to store them + /// to be compatible with normal .mrk2 file format + if (storage.canUseAdaptiveGranularity()) + writeIntBinary(1UL, stream.marks); + + ++skip_index_current_mark; + } + } + + size_t pos = prev_pos; + skip_indices_aggregators[i]->update(indices_update_block, &pos, limit); + + if (pos == prev_pos + limit) + { + ++skip_index_filling[i]; + + /// write index if it is filled + if (skip_index_filling[i] == index->granularity) + { + skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed); + skip_index_filling[i] = 0; + } + } + prev_pos = pos; + } + } + skip_index_mark = skip_index_current_mark; +} + +void IMergedBlockOutputStream::finishSkipIndicesSerialization( + MergeTreeData::DataPart::Checksums & checksums) +{ + for (size_t i = 0; i < skip_indices.size(); ++i) + { + auto & stream = *skip_indices_streams[i]; + if (!skip_indices_aggregators[i]->empty()) + skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed); + } + + for (auto & stream : skip_indices_streams) + { + stream->finalize(); + stream->addToChecksums(checksums); + } + + skip_indices_streams.clear(); + skip_indices_aggregators.clear(); + skip_index_filling.clear(); +} /// Implementation of IMergedBlockOutputStream::ColumnStream. diff --git a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.h b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.h index b9d083f3a19d..276a5c37d17a 100644 --- a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -16,11 +16,13 @@ class IMergedBlockOutputStream : public IBlockOutputStream public: IMergedBlockOutputStream( MergeTreeData & storage_, + String part_path_, size_t min_compress_block_size_, size_t max_compress_block_size_, CompressionCodecPtr default_codec_, size_t aio_threshold_, bool blocks_are_granules_size_, + const std::vector & indices_to_recalc, const MergeTreeIndexGranularity & index_granularity_); using WrittenOffsetColumns = std::set; @@ -117,10 +119,15 @@ class IMergedBlockOutputStream : public IBlockOutputStream bool skip_offsets, DB::IDataType::SubstreamPath & path); + void initSkipIndices(); + void calculateAndSerializeSkipIndices(const ColumnsWithTypeAndName & skip_indexes_columns, size_t rows); + void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums); protected: - MergeTreeData & storage; + SerializationStates serialization_states; + String part_path; + ColumnStreams column_streams; /// The offset to the first row of the block for which you want to write the index. @@ -132,6 +139,7 @@ class IMergedBlockOutputStream : public IBlockOutputStream size_t aio_threshold; size_t current_mark = 0; + size_t skip_index_mark = 0; const std::string marks_file_extension; const bool blocks_are_granules_size; @@ -141,6 +149,11 @@ class IMergedBlockOutputStream : public IBlockOutputStream const bool compute_granularity; CompressionCodecPtr codec; + std::vector skip_indices; + std::vector> skip_indices_streams; + MergeTreeIndexAggregators skip_indices_aggregators; + std::vector skip_index_filling; + const bool with_final_mark; }; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index c7e853aa647a..27d013da32b6 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1000,7 +1000,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor } } } - /// TODO: add indices from materialize if (!indices_to_recalc.empty()) { diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index b3a481b48c5e..703cf0dd01b4 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -18,11 +18,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -namespace -{ - constexpr auto INDEX_FILE_EXTENSION = ".idx"; -} - MergedBlockOutputStream::MergedBlockOutputStream( MergeTreeData & storage_, @@ -31,12 +26,13 @@ MergedBlockOutputStream::MergedBlockOutputStream( CompressionCodecPtr default_codec_, bool blocks_are_granules_size_) : IMergedBlockOutputStream( - storage_, storage_.global_context.getSettings().min_compress_block_size, + storage_, part_path_, storage_.global_context.getSettings().min_compress_block_size, storage_.global_context.getSettings().max_compress_block_size, default_codec_, storage_.global_context.getSettings().min_bytes_to_use_direct_io, blocks_are_granules_size_, - {}), - columns_list(columns_list_), part_path(part_path_) + std::vector(std::begin(storage_.skip_indices), std::end(storage_.skip_indices)), + {}) + , columns_list(columns_list_) { init(); for (const auto & it : columns_list) @@ -55,10 +51,11 @@ MergedBlockOutputStream::MergedBlockOutputStream( size_t aio_threshold_, bool blocks_are_granules_size_) : IMergedBlockOutputStream( - storage_, storage_.global_context.getSettings().min_compress_block_size, + storage_, part_path_, storage_.global_context.getSettings().min_compress_block_size, storage_.global_context.getSettings().max_compress_block_size, default_codec_, - aio_threshold_, blocks_are_granules_size_, {}), - columns_list(columns_list_), part_path(part_path_) + aio_threshold_, blocks_are_granules_size_, + std::vector(std::begin(storage_.skip_indices), std::end(storage_.skip_indices)), {}) + , columns_list(columns_list_) { init(); @@ -134,15 +131,6 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( if (with_final_mark && rows_count != 0) index_granularity.appendMark(0); /// last mark - /// Finish skip index serialization - for (size_t i = 0; i < storage.skip_indices.size(); ++i) - { - auto & stream = *skip_indices_streams[i]; - if (!skip_indices_aggregators[i]->empty()) - skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed); - } - - if (!total_column_list) total_column_list = &columns_list; @@ -171,22 +159,14 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( index_stream = nullptr; } - for (auto & stream : skip_indices_streams) - { - stream->finalize(); - stream->addToChecksums(checksums); - } - - skip_indices_streams.clear(); - skip_indices_aggregators.clear(); - skip_index_filling.clear(); - for (ColumnStreams::iterator it = column_streams.begin(); it != column_streams.end(); ++it) { it->second->finalize(); it->second->addToChecksums(checksums); } + finishSkipIndicesSerialization(checksums); + column_streams.clear(); if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) @@ -248,19 +228,7 @@ void MergedBlockOutputStream::init() index_stream = std::make_unique(*index_file_stream); } - for (const auto & index : storage.skip_indices) - { - String stream_name = index->getFileName(); - skip_indices_streams.emplace_back( - std::make_unique( - stream_name, - part_path + stream_name, INDEX_FILE_EXTENSION, - part_path + stream_name, marks_file_extension, - codec, max_compress_block_size, - 0, aio_threshold)); - skip_indices_aggregators.push_back(index->createIndexAggregator()); - skip_index_filling.push_back(0); - } + initSkipIndices(); } @@ -380,66 +348,8 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm } rows_count += rows; - { - /// Creating block for update - Block indices_update_block(skip_indexes_columns); - size_t skip_index_current_mark = 0; - - /// Filling and writing skip indices like in IMergedBlockOutputStream::writeColumn - for (size_t i = 0; i < storage.skip_indices.size(); ++i) - { - const auto index = storage.skip_indices[i]; - auto & stream = *skip_indices_streams[i]; - size_t prev_pos = 0; - skip_index_current_mark = skip_index_mark; - while (prev_pos < rows) - { - UInt64 limit = 0; - if (prev_pos == 0 && index_offset != 0) - { - limit = index_offset; - } - else - { - limit = index_granularity.getMarkRows(skip_index_current_mark); - if (skip_indices_aggregators[i]->empty()) - { - skip_indices_aggregators[i] = index->createIndexAggregator(); - skip_index_filling[i] = 0; - - if (stream.compressed.offset() >= min_compress_block_size) - stream.compressed.next(); - - writeIntBinary(stream.plain_hashing.count(), stream.marks); - writeIntBinary(stream.compressed.offset(), stream.marks); - /// Actually this numbers is redundant, but we have to store them - /// to be compatible with normal .mrk2 file format - if (storage.canUseAdaptiveGranularity()) - writeIntBinary(1UL, stream.marks); - - ++skip_index_current_mark; - } - } - - size_t pos = prev_pos; - skip_indices_aggregators[i]->update(indices_update_block, &pos, limit); - if (pos == prev_pos + limit) - { - ++skip_index_filling[i]; - - /// write index if it is filled - if (skip_index_filling[i] == index->granularity) - { - skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed); - skip_index_filling[i] = 0; - } - } - prev_pos = pos; - } - } - skip_index_mark = skip_index_current_mark; - } + calculateAndSerializeSkipIndices(skip_indexes_columns, rows); { /** While filling index (index_columns), disable memory tracker. diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h index 3acb01c3c0aa..4efb5b528a09 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -64,11 +64,8 @@ class MergedBlockOutputStream final : public IMergedBlockOutputStream private: NamesAndTypesList columns_list; - SerializationStates serialization_states; - String part_path; size_t rows_count = 0; - size_t skip_index_mark = 0; std::unique_ptr index_file_stream; std::unique_ptr index_stream; @@ -76,10 +73,6 @@ class MergedBlockOutputStream final : public IMergedBlockOutputStream /// Index columns values from the last row from the last block /// It's written to index file in the `writeSuffixAndFinalizePart` method ColumnsWithTypeAndName last_index_row; - - std::vector> skip_indices_streams; - MergeTreeIndexAggregators skip_indices_aggregators; - std::vector skip_index_filling; }; } diff --git a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index d8eaef2a7814..68f033cf401e 100644 --- a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -3,10 +3,6 @@ namespace DB { -namespace -{ - constexpr auto INDEX_FILE_EXTENSION = ".idx"; -} MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, @@ -15,13 +11,14 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( WrittenOffsetColumns & already_written_offset_columns, const MergeTreeIndexGranularity & index_granularity_) : IMergedBlockOutputStream( - storage_, storage_.global_context.getSettings().min_compress_block_size, + storage_, part_path_, storage_.global_context.getSettings().min_compress_block_size, storage_.global_context.getSettings().max_compress_block_size, default_codec_, storage_.global_context.getSettings().min_bytes_to_use_direct_io, - false, - index_granularity_), - header(header_), part_path(part_path_), sync(sync_), skip_offsets(skip_offsets_), - skip_indices(indices_to_recalc), already_written_offset_columns(already_written_offset_columns) + false, indices_to_recalc, index_granularity_) + , header(header_) + , sync(sync_) + , skip_offsets(skip_offsets_) + , already_written_offset_columns(already_written_offset_columns) { } @@ -46,19 +43,7 @@ void MergedColumnOnlyOutputStream::write(const Block & block) col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back()); } - for (const auto & index : skip_indices) - { - String stream_name = index->getFileName(); - skip_indices_streams.emplace_back( - std::make_unique( - stream_name, - part_path + stream_name, INDEX_FILE_EXTENSION, - part_path + stream_name, marks_file_extension, - codec, max_compress_block_size, - 0, aio_threshold)); - skip_indices_aggregators.push_back(index->createIndexAggregator()); - skip_index_filling.push_back(0); - } + initSkipIndices(); initialized = true; } @@ -82,66 +67,7 @@ void MergedColumnOnlyOutputStream::write(const Block & block) if (!rows) return; - { - /// Creating block for update - Block indices_update_block(skip_indexes_columns); - size_t skip_index_current_mark = 0; - - /// Filling and writing skip indices like in IMergedBlockOutputStream::writeColumn - for (size_t i = 0; i < storage.skip_indices.size(); ++i) - { - const auto index = storage.skip_indices[i]; - auto & stream = *skip_indices_streams[i]; - size_t prev_pos = 0; - skip_index_current_mark = skip_index_mark; - while (prev_pos < rows) - { - UInt64 limit = 0; - if (prev_pos == 0 && index_offset != 0) - { - limit = index_offset; - } - else - { - limit = index_granularity.getMarkRows(skip_index_current_mark); - if (skip_indices_aggregators[i]->empty()) - { - skip_indices_aggregators[i] = index->createIndexAggregator(); - skip_index_filling[i] = 0; - - if (stream.compressed.offset() >= min_compress_block_size) - stream.compressed.next(); - - writeIntBinary(stream.plain_hashing.count(), stream.marks); - writeIntBinary(stream.compressed.offset(), stream.marks); - /// Actually this numbers is redundant, but we have to store them - /// to be compatible with normal .mrk2 file format - if (storage.canUseAdaptiveGranularity()) - writeIntBinary(1UL, stream.marks); - - ++skip_index_current_mark; - } - } - - size_t pos = prev_pos; - skip_indices_aggregators[i]->update(indices_update_block, &pos, limit); - - if (pos == prev_pos + limit) - { - ++skip_index_filling[i]; - - /// write index if it is filled - if (skip_index_filling[i] == index->granularity) - { - skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed); - skip_index_filling[i] = 0; - } - } - prev_pos = pos; - } - } - skip_index_mark = skip_index_current_mark; - } + calculateAndSerializeSkipIndices(skip_indexes_columns, rows); size_t new_index_offset = 0; size_t new_current_mark = 0; @@ -181,14 +107,6 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG writeFinalMark(column.name, column.type, offset_columns, skip_offsets, serialize_settings.path); } - /// Finish skip index serialization - for (size_t i = 0; i < skip_indices.size(); ++i) - { - auto & stream = *skip_indices_streams[i]; - if (!skip_indices_aggregators[i]->empty()) - skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed); - } - MergeTreeData::DataPart::Checksums checksums; for (auto & column_stream : column_streams) @@ -200,20 +118,12 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG column_stream.second->addToChecksums(checksums); } - for (auto & stream : skip_indices_streams) - { - stream->finalize(); - stream->addToChecksums(checksums); - } + finishSkipIndicesSerialization(checksums); column_streams.clear(); serialization_states.clear(); initialized = false; - skip_indices_streams.clear(); - skip_indices_aggregators.clear(); - skip_index_filling.clear(); - return checksums; } diff --git a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index 645e9f2ff365..7daaf03fd66d 100644 --- a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -26,18 +26,10 @@ class MergedColumnOnlyOutputStream final : public IMergedBlockOutputStream private: Block header; - SerializationStates serialization_states; - String part_path; bool initialized = false; bool sync; bool skip_offsets; - size_t skip_index_mark = 0; - - std::vector skip_indices; - std::vector> skip_indices_streams; - MergeTreeIndexAggregators skip_indices_aggregators; - std::vector skip_index_filling; /// To correctly write Nested elements column-by-column. WrittenOffsetColumns & already_written_offset_columns; From 908623c029a0a2ed6ee0c19b7f94efb39049842e Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 28 Jul 2019 14:32:03 +0300 Subject: [PATCH 027/161] style --- dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 68f033cf401e..129a84d1e769 100644 --- a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -3,7 +3,6 @@ namespace DB { - MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, CompressionCodecPtr default_codec_, bool skip_offsets_, From 4c5003b3b9fc593ccbb031ec4e4e30cec081b13d Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 28 Jul 2019 14:45:50 +0300 Subject: [PATCH 028/161] const & --- dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp | 2 +- dbms/src/Storages/MergeTree/IMergedBlockOutputStream.h | 2 +- dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp | 4 ++-- dbms/src/Storages/MergeTree/MergedBlockOutputStream.h | 4 ++-- dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp | 2 +- dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index a7475ae7c188..8321d7dc8b28 100644 --- a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -18,7 +18,7 @@ namespace IMergedBlockOutputStream::IMergedBlockOutputStream( MergeTreeData & storage_, - String part_path_, + const String & part_path_, size_t min_compress_block_size_, size_t max_compress_block_size_, CompressionCodecPtr codec_, diff --git a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.h b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.h index 276a5c37d17a..f1fbb0584360 100644 --- a/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -16,7 +16,7 @@ class IMergedBlockOutputStream : public IBlockOutputStream public: IMergedBlockOutputStream( MergeTreeData & storage_, - String part_path_, + const String & part_path_, size_t min_compress_block_size_, size_t max_compress_block_size_, CompressionCodecPtr default_codec_, diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 703cf0dd01b4..b923337d71a5 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes MergedBlockOutputStream::MergedBlockOutputStream( MergeTreeData & storage_, - String part_path_, + const String & part_path_, const NamesAndTypesList & columns_list_, CompressionCodecPtr default_codec_, bool blocks_are_granules_size_) @@ -44,7 +44,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( MergedBlockOutputStream::MergedBlockOutputStream( MergeTreeData & storage_, - String part_path_, + const String & part_path_, const NamesAndTypesList & columns_list_, CompressionCodecPtr default_codec_, const MergeTreeData::DataPart::ColumnToSize & merged_column_to_size_, diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h index 4efb5b528a09..ff45934f1063 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -15,14 +15,14 @@ class MergedBlockOutputStream final : public IMergedBlockOutputStream public: MergedBlockOutputStream( MergeTreeData & storage_, - String part_path_, + const String & part_path_, const NamesAndTypesList & columns_list_, CompressionCodecPtr default_codec_, bool blocks_are_granules_size_ = false); MergedBlockOutputStream( MergeTreeData & storage_, - String part_path_, + const String & part_path_, const NamesAndTypesList & columns_list_, CompressionCodecPtr default_codec_, const MergeTreeData::DataPart::ColumnToSize & merged_column_to_size_, diff --git a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 129a84d1e769..a694d509b687 100644 --- a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -4,7 +4,7 @@ namespace DB { MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( - MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, + MergeTreeData & storage_, const Block & header_, const String & part_path_, bool sync_, CompressionCodecPtr default_codec_, bool skip_offsets_, const std::vector & indices_to_recalc, WrittenOffsetColumns & already_written_offset_columns, diff --git a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index 7daaf03fd66d..adf5d7f7bfdd 100644 --- a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -13,7 +13,7 @@ class MergedColumnOnlyOutputStream final : public IMergedBlockOutputStream /// Pass empty 'already_written_offset_columns' first time then and pass the same object to subsequent instances of MergedColumnOnlyOutputStream /// if you want to serialize elements of Nested data structure in different instances of MergedColumnOnlyOutputStream. MergedColumnOnlyOutputStream( - MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, + MergeTreeData & storage_, const Block & header_, const String & part_path_, bool sync_, CompressionCodecPtr default_codec_, bool skip_offsets_, const std::vector & indices_to_recalc, WrittenOffsetColumns & already_written_offset_columns, From 538c17be3f6c719b377d350e81462be775b7891c Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 28 Jul 2019 14:56:16 +0300 Subject: [PATCH 029/161] replicated test --- ...es_mutation_replicated_zookeeper.reference | 6 ++ ...5_indices_mutation_replicated_zookeeper.sh | 61 +++++++++++++++++++ 2 files changed, 67 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.reference create mode 100644 dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh diff --git a/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.reference b/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.reference new file mode 100644 index 000000000000..9c5382e5cbe4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.reference @@ -0,0 +1,6 @@ +2 + "rows_read": 4, +2 + "rows_read": 6, +2 + "rows_read": 4, diff --git a/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh b/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh new file mode 100644 index 000000000000..4490487663f8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh @@ -0,0 +1,61 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" + + +$CLICKHOUSE_CLIENT -n --query=" +SET allow_experimental_data_skipping_indices=1; +CREATE TABLE test.indices_mutaions1 +( + u64 UInt64, + i64 Int64, + i32 Int32, + INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1 +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/indices_mutaions', 'r1') +PARTITION BY i32 +ORDER BY u64 +SETTINGS index_granularity = 2; +CREATE TABLE test.indices_mutaions2 +( + u64 UInt64, + i64 Int64, + i32 Int32, + INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1 +) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/indices_mutaions', 'r2') +PARTITION BY i32 +ORDER BY u64 +SETTINGS index_granularity = 2;" + + +$CLICKHOUSE_CLIENT --query="INSERT INTO test.indices_mutaions1 VALUES +(0, 2, 1), +(1, 1, 1), +(2, 1, 1), +(3, 1, 1), +(4, 1, 1), +(5, 2, 1), +(6, 1, 2), +(7, 1, 2), +(8, 1, 2), +(9, 1, 2)" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 CLEAR INDEX idx IN PARTITION 1;" +sleep 1 + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 MATERIALIZE INDEX idx IN PARTITION 1;" +sleep 1 + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="DROP TABLE test.indices_mutaions1" +$CLICKHOUSE_CLIENT --query="DROP TABLE test.indices_mutaions2" From 21ce5331d19d2fd09c6029606cb440fd6ecfe071 Mon Sep 17 00:00:00 2001 From: Yuriy Date: Sun, 28 Jul 2019 16:12:26 +0300 Subject: [PATCH 030/161] implemented mysql_native_password auth plugin for compatibility with mysqljs --- dbms/programs/server/MySQLHandler.cpp | 142 ++------ dbms/programs/server/MySQLHandler.h | 6 +- dbms/programs/server/users.xml | 8 + dbms/src/Core/MySQLProtocol.h | 330 +++++++++++++++--- dbms/src/Interpreters/Context.cpp | 4 + dbms/src/Interpreters/Context.h | 5 + dbms/src/Interpreters/Users.cpp | 18 +- dbms/src/Interpreters/Users.h | 1 + dbms/src/Interpreters/UsersManager.cpp | 62 ++-- .../clients/mysqljs/Dockerfile | 5 + .../clients/mysqljs/docker_compose.yml | 8 + .../clients/mysqljs/test.js | 21 ++ .../test_mysql_protocol/configs/users.xml | 21 ++ .../integration/test_mysql_protocol/test.py | 32 +- 14 files changed, 460 insertions(+), 203 deletions(-) create mode 100644 dbms/tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile create mode 100644 dbms/tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml create mode 100644 dbms/tests/integration/test_mysql_protocol/clients/mysqljs/test.js diff --git a/dbms/programs/server/MySQLHandler.cpp b/dbms/programs/server/MySQLHandler.cpp index c71e0554fb54..52dbc0a135ad 100644 --- a/dbms/programs/server/MySQLHandler.cpp +++ b/dbms/programs/server/MySQLHandler.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -45,6 +44,7 @@ MySQLHandler::MySQLHandler(IServer & server_, const Poco::Net::StreamSocket & so , connection_id(connection_id) , public_key(public_key) , private_key(private_key) + , auth_plugin(new Authentication::Native41()) { server_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_SECURE_CONNECTION | CLIENT_PLUGIN_AUTH | CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA | CLIENT_CONNECT_WITH_DB | CLIENT_DEPRECATE_EOF; if (ssl_enabled) @@ -62,9 +62,7 @@ void MySQLHandler::run() try { - String scramble = generateScramble(); - - Handshake handshake(server_capability_flags, connection_id, VERSION_STRING + String("-") + VERSION_NAME, Authentication::Native, scramble + '\0'); + Handshake handshake(server_capability_flags, connection_id, VERSION_STRING + String("-") + VERSION_NAME, auth_plugin->getName(), auth_plugin->getAuthPluginData()); packet_sender->sendPacket(handshake, true); LOG_TRACE(log, "Sent handshake"); @@ -96,10 +94,21 @@ void MySQLHandler::run() client_capability_flags = handshake_response.capability_flags; if (!(client_capability_flags & CLIENT_PROTOCOL_41)) throw Exception("Required capability: CLIENT_PROTOCOL_41.", ErrorCodes::MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES); - if (!(client_capability_flags & CLIENT_PLUGIN_AUTH)) - throw Exception("Required capability: CLIENT_PLUGIN_AUTH.", ErrorCodes::MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES); - authenticate(handshake_response, scramble); + authenticate(handshake_response.username, handshake_response.auth_plugin_name, handshake_response.auth_response); + + try + { + if (!handshake_response.database.empty()) + connection_context.setCurrentDatabase(handshake_response.database); + connection_context.setCurrentQueryId(""); + } + catch (const Exception & exc) + { + log->log(exc); + packet_sender->sendPacket(ERR_Packet(exc.code(), "00000", exc.message()), true); + } + OK_Packet ok_packet(0, handshake_response.capability_flags, 0, 0, 0); packet_sender->sendPacket(ok_packet, true); @@ -216,121 +225,24 @@ void MySQLHandler::finishHandshake(MySQLProtocol::HandshakeResponse & packet) } } -String MySQLHandler::generateScramble() +void MySQLHandler::authenticate(const String & user_name, const String & auth_plugin_name, const String & initial_auth_response) { - String scramble(MySQLProtocol::SCRAMBLE_LENGTH, 0); - Poco::RandomInputStream generator; - for (size_t i = 0; i < scramble.size(); i++) - { - generator >> scramble[i]; - } - return scramble; -} - -void MySQLHandler::authenticate(const HandshakeResponse & handshake_response, const String & scramble) -{ - - String auth_response; - AuthSwitchResponse response; - if (handshake_response.auth_plugin_name != Authentication::SHA256) - { - /** Native authentication sent 20 bytes + '\0' character = 21 bytes. - * This plugin must do the same to stay consistent with historical behavior if it is set to operate as a default plugin. - * https://github.com/mysql/mysql-server/blob/8.0/sql/auth/sql_authentication.cc#L3994 - */ - packet_sender->sendPacket(AuthSwitchRequest(Authentication::SHA256, scramble + '\0'), true); - if (in->eof()) - throw Exception( - "Client doesn't support authentication method " + String(Authentication::SHA256) + " used by ClickHouse", - ErrorCodes::MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES); - packet_sender->receivePacket(response); - auth_response = response.value; - LOG_TRACE(log, "Authentication method mismatch."); - } - else - { - auth_response = handshake_response.auth_response; - LOG_TRACE(log, "Authentication method match."); - } - - if (auth_response == "\1") - { - LOG_TRACE(log, "Client requests public key."); - - BIO * mem = BIO_new(BIO_s_mem()); - SCOPE_EXIT(BIO_free(mem)); - if (PEM_write_bio_RSA_PUBKEY(mem, &public_key) != 1) - { - throw Exception("Failed to write public key to memory. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR); - } - char * pem_buf = nullptr; - long pem_size = BIO_get_mem_data(mem, &pem_buf); - String pem(pem_buf, pem_size); - - LOG_TRACE(log, "Key: " << pem); - - AuthMoreData data(pem); - packet_sender->sendPacket(data, true); - packet_sender->receivePacket(response); - auth_response = response.value; - } - else - { - LOG_TRACE(log, "Client didn't request public key."); - } - - String password; - - /** Decrypt password, if it's not empty. - * The original intention was that the password is a string[NUL] but this never got enforced properly so now we have to accept that - * an empty packet is a blank password, thus the check for auth_response.empty() has to be made too. - * https://github.com/mysql/mysql-server/blob/8.0/sql/auth/sql_authentication.cc#L4017 - */ - if (!secure_connection && !auth_response.empty() && auth_response != String("\0", 1)) - { - LOG_TRACE(log, "Received nonempty password"); - auto ciphertext = reinterpret_cast(auth_response.data()); - - unsigned char plaintext[RSA_size(&private_key)]; - int plaintext_size = RSA_private_decrypt(auth_response.size(), ciphertext, plaintext, &private_key, RSA_PKCS1_OAEP_PADDING); - if (plaintext_size == -1) - { - throw Exception("Failed to decrypt auth data. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR); - } - - password.resize(plaintext_size); - for (int i = 0; i < plaintext_size; i++) - { - password[i] = plaintext[i] ^ static_cast(scramble[i % scramble.size()]); - } - } - else if (secure_connection) - { - password = auth_response; - } - else - { - LOG_TRACE(log, "Received empty password"); - } - - if (!password.empty() && password.back() == 0) - { - password.pop_back(); - } - - try - { - connection_context.setUser(handshake_response.username, password, socket().address(), ""); - if (!handshake_response.database.empty()) connection_context.setCurrentDatabase(handshake_response.database); - connection_context.setCurrentQueryId(""); - LOG_INFO(log, "Authentication for user " << handshake_response.username << " succeeded."); + // For compatibility with JavaScript MySQL client, Native41 authentication plugin is used when it is possible. If password is specified using SHA-2, then SHA256 plugin is used. + auto user = connection_context.getUser(user_name); + if (!user->password_sha256_hex.empty()) + auth_plugin = std::make_unique(public_key, private_key, log); + + try { + std::optional auth_response = auth_plugin_name == auth_plugin->getName() ? std::make_optional(initial_auth_response) : std::nullopt; + auth_plugin->authenticate(user_name, auth_response, connection_context, packet_sender, secure_connection, socket().address()); } catch (const Exception & exc) { - LOG_ERROR(log, "Authentication for user " << handshake_response.username << " failed."); + LOG_ERROR(log, "Authentication for user " << user_name << " failed."); packet_sender->sendPacket(ERR_Packet(exc.code(), "00000", exc.message()), true); throw; } + LOG_INFO(log, "Authentication for user " << user_name << " succeeded."); } void MySQLHandler::comInitDB(ReadBuffer & payload) diff --git a/dbms/programs/server/MySQLHandler.h b/dbms/programs/server/MySQLHandler.h index e899d8ef5013..3e521004aca7 100644 --- a/dbms/programs/server/MySQLHandler.h +++ b/dbms/programs/server/MySQLHandler.h @@ -30,9 +30,7 @@ class MySQLHandler : public Poco::Net::TCPServerConnection void comInitDB(ReadBuffer & payload); - static String generateScramble(); - - void authenticate(const MySQLProtocol::HandshakeResponse &, const String & scramble); + void authenticate(const String & user_name, const String & auth_plugin_name, const String & auth_response); IServer & server; Poco::Logger * log; @@ -48,6 +46,8 @@ class MySQLHandler : public Poco::Net::TCPServerConnection RSA & public_key; RSA & private_key; + std::unique_ptr auth_plugin; + std::shared_ptr ss; std::shared_ptr in; std::shared_ptr out; diff --git a/dbms/programs/server/users.xml b/dbms/programs/server/users.xml index 24b8f628c3a8..9755c29d480b 100644 --- a/dbms/programs/server/users.xml +++ b/dbms/programs/server/users.xml @@ -39,10 +39,18 @@ If you want to specify SHA256, place it in 'password_sha256_hex' element. Example: 65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5 + Restrictions of SHA256: impossibility to connect to ClickHouse using MySQL JS client (as of July 2019). + + If you want to specify double SHA1, place it in 'password_double_sha1_hex' element. + Example: e395796d6546b1b65db9d665cd43f0e858dd4303 How to generate decent password: Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | sha256sum | tr -d '-' In first line will be password and in second - corresponding SHA256. + + How to generate double SHA1: + Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | openssl dgst -sha1 -binary | openssl dgst -sha1 + In first line will be password and in second - corresponding double SHA1. --> diff --git a/dbms/src/Core/MySQLProtocol.h b/dbms/src/Core/MySQLProtocol.h index cb0e6ab001c8..0704cec0edb9 100644 --- a/dbms/src/Core/MySQLProtocol.h +++ b/dbms/src/Core/MySQLProtocol.h @@ -1,9 +1,17 @@ #pragma once +#include +#include +#include +#include +#include #include +#include #include #include +#include #include +#include #include #include #include @@ -12,11 +20,12 @@ #include #include #include +#include +#include #include #include -#include -#include -#include +#include +#include /// Implementation of MySQL wire protocol. /// Works only on little-endian architecture. @@ -27,6 +36,9 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_PACKET_FROM_CLIENT; + extern const int MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES; + extern const int OPENSSL_ERROR; + extern const int UNKNOWN_EXCEPTION; } namespace MySQLProtocol @@ -39,11 +51,6 @@ const size_t MYSQL_ERRMSG_SIZE = 512; const size_t PACKET_HEADER_SIZE = 4; const size_t SSL_REQUEST_PAYLOAD_SIZE = 32; -namespace Authentication -{ - const String Native = "mysql_native_password"; - const String SHA256 = "sha256_password"; /// Caching SHA2 plugin is not used because it would be possible to authenticate knowing hash from users.xml. -} enum CharacterSet { @@ -139,8 +146,7 @@ class PacketPayloadReadBuffer : public ReadBuffer public: PacketPayloadReadBuffer(ReadBuffer & in, uint8_t & sequence_id) : ReadBuffer(in.position(), 0) // not in.buffer().begin(), because working buffer may include previous packet - , in(in) - , sequence_id(sequence_id) + , in(in), sequence_id(sequence_id) { } @@ -149,6 +155,8 @@ class PacketPayloadReadBuffer : public ReadBuffer uint8_t & sequence_id; const size_t max_packet_size = MAX_PACKET_LENGTH; + bool has_read_header = false; + // Size of packet which is being read now. size_t payload_length = 0; @@ -158,8 +166,9 @@ class PacketPayloadReadBuffer : public ReadBuffer protected: bool nextImpl() override { - if (payload_length == 0 || (payload_length == max_packet_size && offset == payload_length)) + if (!has_read_header || (payload_length == max_packet_size && offset == payload_length)) { + has_read_header = true; working_buffer.resize(0); offset = 0; payload_length = 0; @@ -171,10 +180,6 @@ class PacketPayloadReadBuffer : public ReadBuffer tmp << "Received packet with payload larger than max_packet_size: " << payload_length; throw ProtocolError(tmp.str(), ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT); } - else if (payload_length == 0) - { - return false; - } size_t packet_sequence_id = 0; in.read(reinterpret_cast(packet_sequence_id)); @@ -185,6 +190,9 @@ class PacketPayloadReadBuffer : public ReadBuffer throw ProtocolError(tmp.str(), ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT); } sequence_id++; + + if (payload_length == 0) + return false; } else if (offset == payload_length) { @@ -208,6 +216,7 @@ class ClientPacket { public: ClientPacket() = default; + ClientPacket(ClientPacket &&) = default; virtual void read(ReadBuffer & in, uint8_t & sequence_id) @@ -246,10 +255,7 @@ class PacketPayloadWriteBuffer : public WriteBuffer { public: PacketPayloadWriteBuffer(WriteBuffer & out, size_t payload_length, uint8_t & sequence_id) - : WriteBuffer(out.position(), 0) - , out(out) - , sequence_id(sequence_id) - , total_left(payload_length) + : WriteBuffer(out.position(), 0), out(out), sequence_id(sequence_id), total_left(payload_length) { startPacket(); } @@ -265,7 +271,9 @@ class PacketPayloadWriteBuffer : public WriteBuffer } } - ~PacketPayloadWriteBuffer() override { next(); } + ~PacketPayloadWriteBuffer() override + { next(); } + private: WriteBuffer & out; uint8_t & sequence_id; @@ -286,6 +294,7 @@ class PacketPayloadWriteBuffer : public WriteBuffer working_buffer = WriteBuffer::Buffer(out.position(), out.position() + std::min(payload_length - bytes_written, out.available())); pos = working_buffer.begin(); } + protected: void nextImpl() override { @@ -345,17 +354,13 @@ class PacketSender /// For reading and writing. PacketSender(ReadBuffer & in, WriteBuffer & out, uint8_t & sequence_id) - : sequence_id(sequence_id) - , in(&in) - , out(&out) + : sequence_id(sequence_id), in(&in), out(&out) { } /// For writing. PacketSender(WriteBuffer & out, uint8_t & sequence_id) - : sequence_id(sequence_id) - , in(nullptr) - , out(&out) + : sequence_id(sequence_id), in(nullptr), out(&out) { } @@ -419,14 +424,8 @@ class Handshake : public WritePacket String auth_plugin_data; public: explicit Handshake(uint32_t capability_flags, uint32_t connection_id, String server_version, String auth_plugin_name, String auth_plugin_data) - : protocol_version(0xa) - , server_version(std::move(server_version)) - , connection_id(connection_id) - , capability_flags(capability_flags) - , character_set(CharacterSet::utf8_general_ci) - , status_flags(0) - , auth_plugin_name(std::move(auth_plugin_name)) - , auth_plugin_data(std::move(auth_plugin_data)) + : protocol_version(0xa), server_version(std::move(server_version)), connection_id(connection_id), capability_flags(capability_flags), character_set(CharacterSet::utf8_general_ci), + status_flags(0), auth_plugin_name(std::move(auth_plugin_name)), auth_plugin_data(std::move(auth_plugin_data)) { } @@ -449,9 +448,6 @@ class Handshake : public WritePacket buffer.write(static_cast(auth_plugin_data.size())); writeChar(0x0, 10, buffer); writeString(auth_plugin_data.substr(AUTH_PLUGIN_DATA_PART_1_LENGTH, auth_plugin_data.size() - AUTH_PLUGIN_DATA_PART_1_LENGTH), buffer); - // A workaround for PHP mysqlnd extension bug which occurs when sha256_password is used as a default authentication plugin. - // Instead of using client response for mysql_native_password plugin, the server will always generate authentication method mismatch - // and switch to sha256_password to simulate that mysql_native_password is used as a default plugin. writeString(auth_plugin_name, buffer); writeChar(0x0, 1, buffer); } @@ -563,7 +559,8 @@ class AuthMoreData : public WritePacket { String data; public: - explicit AuthMoreData(String data): data(std::move(data)) {} + explicit AuthMoreData(String data) : data(std::move(data)) + {} protected: size_t getPayloadSize() const override @@ -590,19 +587,14 @@ class OK_Packet : public WritePacket String info; public: OK_Packet(uint8_t header, - uint32_t capabilities, - uint64_t affected_rows, - uint32_t status_flags, - int16_t warnings, - String session_state_changes = "", - String info = "") - : header(header) - , capabilities(capabilities) - , affected_rows(affected_rows) - , warnings(warnings) - , status_flags(status_flags) - , session_state_changes(std::move(session_state_changes)) - , info(std::move(info)) + uint32_t capabilities, + uint64_t affected_rows, + uint32_t status_flags, + int16_t warnings, + String session_state_changes = "", + String info = "") + : header(header), capabilities(capabilities), affected_rows(affected_rows), warnings(warnings), status_flags(status_flags), session_state_changes(std::move(session_state_changes)), + info(std::move(info)) { } @@ -798,7 +790,7 @@ class LengthEncodedNumber : public WritePacket { uint64_t value; public: - explicit LengthEncodedNumber(uint64_t value): value(value) + explicit LengthEncodedNumber(uint64_t value) : value(value) { } @@ -840,5 +832,239 @@ class ResultsetRow : public WritePacket } }; +namespace Authentication +{ + +class IPlugin +{ +public: + virtual String getName() = 0; + + virtual String getAuthPluginData() = 0; + + virtual void authenticate(const String & user_name, std::optional auth_response, Context & context, std::shared_ptr packet_sender, bool is_secure_connection, + const Poco::Net::SocketAddress & address) = 0; + + virtual ~IPlugin() = default; +}; + +/// https://dev.mysql.com/doc/internals/en/secure-password-authentication.html +class Native41 : public IPlugin +{ +public: + Native41() + { + scramble.resize(SCRAMBLE_LENGTH + 1, 0); + Poco::RandomInputStream generator; + + for (size_t i = 0; i < SCRAMBLE_LENGTH; i++) + generator >> scramble[i]; + } + + String getName() override + { + return "mysql_native_password"; + } + + String getAuthPluginData() override + { + return scramble; + } + + void authenticate( + const String & user_name, + std::optional auth_response, + Context & context, + std::shared_ptr packet_sender, + bool /* is_secure_connection */, + const Poco::Net::SocketAddress & address) override + { + if (!auth_response) + { + packet_sender->sendPacket(AuthSwitchRequest(getName(), scramble), true); + AuthSwitchResponse response; + packet_sender->receivePacket(response); + auth_response = response.value; + } + + if (auth_response->empty()) + { + context.setUser(user_name, "", address, ""); + return; + } + + if (auth_response->size() != Poco::SHA1Engine::DIGEST_SIZE) + throw Exception("Wrong size of auth response. Expected: " + std::to_string(Poco::SHA1Engine::DIGEST_SIZE) + " bytes, received: " + std::to_string(auth_response->size()) + " bytes.", + ErrorCodes::UNKNOWN_EXCEPTION); + + auto user = context.getUser(user_name); + + if (!user->password_sha256_hex.empty()) + throw Exception("Cannot use " + getName() + " auth plugin for user " + user_name + " since its password is specified using SHA256.", ErrorCodes::UNKNOWN_EXCEPTION); + + Poco::SHA1Engine::Digest double_sha1_value; + Poco::SHA1Engine engine; + + /// If password is specified using double SHA1, than it is non-empty (unlike plaintext password, which can be empty). + if (!user->password_double_sha1_hex.empty()) + { + double_sha1_value = Poco::DigestEngine::digestFromHex(user->password_double_sha1_hex); + assert(double_sha1_value.size() == Poco::SHA1Engine::DIGEST_SIZE); + } + else + { + engine.update(user->password); + const Poco::SHA1Engine::Digest & first_sha1 = engine.digest(); + engine.update(first_sha1.data(), first_sha1.size()); + double_sha1_value = engine.digest(); + } + + engine.update(scramble.data(), SCRAMBLE_LENGTH); + engine.update(double_sha1_value.data(), double_sha1_value.size()); + + String password_sha1(Poco::SHA1Engine::DIGEST_SIZE, 0x0); + const Poco::SHA1Engine::Digest & digest = engine.digest(); + for (size_t i = 0; i < password_sha1.size(); i++) + { + password_sha1[i] = digest[i] ^ static_cast((*auth_response)[i]); + } + context.setUser(user_name, password_sha1, address, ""); + } +private: + String scramble; +}; + +/// Caching SHA2 plugin is not used because it would be possible to authenticate knowing hash from users.xml. +/// https://dev.mysql.com/doc/internals/en/sha256.html +class Sha256Password : public IPlugin +{ +public: + Sha256Password(RSA & public_key, RSA & private_key, Logger * log) : public_key(public_key), private_key(private_key), log(log) + { + /** Native authentication sent 20 bytes + '\0' character = 21 bytes. + * This plugin must do the same to stay consistent with historical behavior if it is set to operate as a default plugin. [1] + * https://github.com/mysql/mysql-server/blob/8.0/sql/auth/sql_authentication.cc#L3994 + */ + scramble.resize(SCRAMBLE_LENGTH + 1, 0); + Poco::RandomInputStream generator; + + for (char & c : scramble) + generator >> c; + } + + String getName() override + { + return "sha256_password"; + } + + String getAuthPluginData() override + { + return scramble; + } + + void authenticate( + const String & user_name, + std::optional auth_response, + Context & context, + std::shared_ptr packet_sender, + bool is_secure_connection, + const Poco::Net::SocketAddress & address) override + { + if (!auth_response) + { + packet_sender->sendPacket(AuthSwitchRequest(getName(), scramble), true); + + if (packet_sender->in->eof()) + throw Exception("Client doesn't support authentication method " + getName() + " used by ClickHouse", ErrorCodes::MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES); + + AuthSwitchResponse response; + packet_sender->receivePacket(response); + auth_response = response.value; + LOG_TRACE(log, "Authentication method mismatch."); + } + else + { + LOG_TRACE(log, "Authentication method match."); + } + + if (auth_response == "\1") + { + LOG_TRACE(log, "Client requests public key."); + + BIO * mem = BIO_new(BIO_s_mem()); + SCOPE_EXIT(BIO_free(mem)); + if (PEM_write_bio_RSA_PUBKEY(mem, &public_key) != 1) + { + throw Exception("Failed to write public key to memory. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR); + } + char * pem_buf = nullptr; + long pem_size = BIO_get_mem_data(mem, &pem_buf); + String pem(pem_buf, pem_size); + + LOG_TRACE(log, "Key: " << pem); + + AuthMoreData data(pem); + packet_sender->sendPacket(data, true); + + AuthSwitchResponse response; + packet_sender->receivePacket(response); + auth_response = response.value; + } + else + { + LOG_TRACE(log, "Client didn't request public key."); + } + + String password; + + /** Decrypt password, if it's not empty. + * The original intention was that the password is a string[NUL] but this never got enforced properly so now we have to accept that + * an empty packet is a blank password, thus the check for auth_response.empty() has to be made too. + * https://github.com/mysql/mysql-server/blob/8.0/sql/auth/sql_authentication.cc#L4017 + */ + if (!is_secure_connection && !auth_response->empty() && auth_response != String("\0", 1)) + { + LOG_TRACE(log, "Received nonempty password"); + auto ciphertext = reinterpret_cast(auth_response->data()); + + unsigned char plaintext[RSA_size(&private_key)]; + int plaintext_size = RSA_private_decrypt(auth_response->size(), ciphertext, plaintext, &private_key, RSA_PKCS1_OAEP_PADDING); + if (plaintext_size == -1) + { + throw Exception("Failed to decrypt auth data. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR); + } + + password.resize(plaintext_size); + for (int i = 0; i < plaintext_size; i++) + { + password[i] = plaintext[i] ^ static_cast(scramble[i % scramble.size()]); + } + } + else if (is_secure_connection) + { + password = *auth_response; + } + else + { + LOG_TRACE(log, "Received empty password"); + } + + if (!password.empty() && password.back() == 0) + { + password.pop_back(); + } + + context.setUser(user_name, password, address, ""); + } + +private: + RSA & public_key; + RSA & private_key; + Logger * log; + String scramble; +}; + +} + } } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index cec36f424693..4b9599942db8 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -661,6 +661,10 @@ void Context::setProfile(const String & profile) settings_constraints = std::move(new_constraints); } +std::shared_ptr Context::getUser(const String & user_name) +{ + return shared->users_manager->getUser(user_name); +} void Context::setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key) { diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index d2e08a450268..a168e2af5f1f 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -197,6 +198,10 @@ class Context /// Must be called before getClientInfo. void setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key); + + /// Used by MySQL Secure Password Authentication plugin. + std::shared_ptr getUser(const String & user_name); + /// Compute and set actual user settings, client_info.current_user should be set void calculateUserSettings(); diff --git a/dbms/src/Interpreters/Users.cpp b/dbms/src/Interpreters/Users.cpp index 1f5a474d6f12..2dff8c32a692 100644 --- a/dbms/src/Interpreters/Users.cpp +++ b/dbms/src/Interpreters/Users.cpp @@ -278,12 +278,14 @@ User::User(const String & name_, const String & config_elem, const Poco::Util::A { bool has_password = config.has(config_elem + ".password"); bool has_password_sha256_hex = config.has(config_elem + ".password_sha256_hex"); + bool has_password_double_sha1_hex = config.has(config_elem + ".password_double_sha1_hex"); - if (has_password && has_password_sha256_hex) - throw Exception("Both fields 'password' and 'password_sha256_hex' are specified for user " + name + ". Must be only one of them.", ErrorCodes::BAD_ARGUMENTS); + if (has_password + has_password_sha256_hex + has_password_double_sha1_hex > 1) + throw Exception("More than one field of 'password', 'password_sha256_hex', 'password_double_sha1_hex' is used to specify password for user " + name + ". Must be only one of them.", + ErrorCodes::BAD_ARGUMENTS); - if (!has_password && !has_password_sha256_hex) - throw Exception("Either 'password' or 'password_sha256_hex' must be specified for user " + name + ".", ErrorCodes::BAD_ARGUMENTS); + if (!has_password && !has_password_sha256_hex && !has_password_double_sha1_hex) + throw Exception("Either 'password' or 'password_sha256_hex' or 'password_double_sha1_hex' must be specified for user " + name + ".", ErrorCodes::BAD_ARGUMENTS); if (has_password) password = config.getString(config_elem + ".password"); @@ -296,6 +298,14 @@ User::User(const String & name_, const String & config_elem, const Poco::Util::A throw Exception("password_sha256_hex for user " + name + " has length " + toString(password_sha256_hex.size()) + " but must be exactly 64 symbols.", ErrorCodes::BAD_ARGUMENTS); } + if (has_password_double_sha1_hex) + { + password_double_sha1_hex = Poco::toLower(config.getString(config_elem + ".password_double_sha1_hex")); + + if (password_double_sha1_hex.size() != 40) + throw Exception("password_double_sha1_hex for user " + name + " has length " + toString(password_double_sha1_hex.size()) + " but must be exactly 40 symbols.", ErrorCodes::BAD_ARGUMENTS); + } + profile = config.getString(config_elem + ".profile"); quota = config.getString(config_elem + ".quota"); diff --git a/dbms/src/Interpreters/Users.h b/dbms/src/Interpreters/Users.h index 95dde778b62d..8baeec57e509 100644 --- a/dbms/src/Interpreters/Users.h +++ b/dbms/src/Interpreters/Users.h @@ -56,6 +56,7 @@ struct User /// Required password. Could be stored in plaintext or in SHA256. String password; String password_sha256_hex; + String password_double_sha1_hex; String profile; String quota; diff --git a/dbms/src/Interpreters/UsersManager.cpp b/dbms/src/Interpreters/UsersManager.cpp index 9c8434cdbe93..d5b454f2acc7 100644 --- a/dbms/src/Interpreters/UsersManager.cpp +++ b/dbms/src/Interpreters/UsersManager.cpp @@ -1,17 +1,17 @@ #include -#include -#include -#include +#include "config_core.h" #include +#include #include #include #include -#include -#include "config_core.h" -#if USE_SSL -# include -#endif +#include +#include +#include +#include +#include +#include namespace DB @@ -70,32 +70,40 @@ UserPtr UsersManager::authorizeAndGetUser( if (!it->second->password_sha256_hex.empty()) { -#if USE_SSL - unsigned char hash[32]; - - SHA256_CTX ctx; - SHA256_Init(&ctx); - SHA256_Update(&ctx, reinterpret_cast(password.data()), password.size()); - SHA256_Final(hash, &ctx); - - String hash_hex; + Poco::SHA2Engine engine; + engine.update(password); + if (Poco::SHA2Engine::digestToHex(engine.digest()) != it->second->password_sha256_hex) + on_wrong_password(); + } + else if (!it->second->password_double_sha1_hex.empty()) + { + /// MySQL compatibility server passes SHA1 instead of a password. If password length equals to 20, it is treated as a SHA1. Server stores double SHA1. + Poco::SHA1Engine engine; + if (password.size() == Poco::SHA1Engine::DIGEST_SIZE) { - WriteBufferFromString buf(hash_hex); - HexWriteBuffer hex_buf(buf); - hex_buf.write(reinterpret_cast(hash), sizeof(hash)); + engine.update(password); + } + else + { + engine.update(password); + const auto & first_sha1 = engine.digest(); + engine.update(first_sha1.data(), first_sha1.size()); } - Poco::toLowerInPlace(hash_hex); - - if (hash_hex != it->second->password_sha256_hex) + if (Poco::SHA1Engine::digestToHex(engine.digest()) != it->second->password_double_sha1_hex) on_wrong_password(); -#else - throw DB::Exception("SHA256 passwords support is disabled, because ClickHouse was built without SSL library", DB::ErrorCodes::SUPPORT_IS_DISABLED); -#endif } else if (password != it->second->password) { - on_wrong_password(); + /// MySQL compatibility server passes SHA1 instead of a password. + if (password.size() != Poco::SHA1Engine::DIGEST_SIZE) + on_wrong_password(); + + Poco::SHA1Engine engine; + engine.update(it->second->password); + + if (engine.digest() != Poco::SHA1Engine::Digest(password.begin(), password.end())) + on_wrong_password(); } return it->second; diff --git a/dbms/tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile b/dbms/tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile new file mode 100644 index 000000000000..5381915efbae --- /dev/null +++ b/dbms/tests/integration/test_mysql_protocol/clients/mysqljs/Dockerfile @@ -0,0 +1,5 @@ +FROM node:8 + +RUN npm install mysql + +COPY ./test.js test.js diff --git a/dbms/tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml b/dbms/tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml new file mode 100644 index 000000000000..9485662dd6c9 --- /dev/null +++ b/dbms/tests/integration/test_mysql_protocol/clients/mysqljs/docker_compose.yml @@ -0,0 +1,8 @@ +version: '2.2' +services: + mysqljs1: + build: + context: ./ + network: host + # to keep container running + command: sleep infinity diff --git a/dbms/tests/integration/test_mysql_protocol/clients/mysqljs/test.js b/dbms/tests/integration/test_mysql_protocol/clients/mysqljs/test.js new file mode 100644 index 000000000000..0cbe38acb079 --- /dev/null +++ b/dbms/tests/integration/test_mysql_protocol/clients/mysqljs/test.js @@ -0,0 +1,21 @@ +var mysql = require('mysql'); + +var connection = mysql.createConnection({ + host : process.argv[2], + port : process.argv[3], + user : process.argv[4], + password : process.argv[5], + database : 'system', +}); + +connection.connect(); + +connection.query('SELECT 1 + 1 AS solution', function (error, results, fields) { + if (error) throw error; + + if (results[0].solution.toString() !== '2') { + throw Error('Wrong result of a query. Expected: "2", received: ' + results[0].solution + '.') + } +}); + +connection.end(); diff --git a/dbms/tests/integration/test_mysql_protocol/configs/users.xml b/dbms/tests/integration/test_mysql_protocol/configs/users.xml index 2be13dca4997..fb4b948d896d 100644 --- a/dbms/tests/integration/test_mysql_protocol/configs/users.xml +++ b/dbms/tests/integration/test_mysql_protocol/configs/users.xml @@ -14,6 +14,27 @@ default default + + + + e395796d6546b1b65db9d665cd43f0e858dd4303 + + ::/0 + + default + default + + + + + + + + ::/0 + + default + default + diff --git a/dbms/tests/integration/test_mysql_protocol/test.py b/dbms/tests/integration/test_mysql_protocol/test.py index edacfb751c9b..39f3793cc5c2 100644 --- a/dbms/tests/integration/test_mysql_protocol/test.py +++ b/dbms/tests/integration/test_mysql_protocol/test.py @@ -50,8 +50,22 @@ def php_container(): yield docker.from_env().containers.get(cluster.project_name + '_php1_1') +@pytest.fixture(scope='module') +def nodejs_container(): + docker_compose = os.path.join(SCRIPT_DIR, 'clients', 'mysqljs', 'docker_compose.yml') + subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) + yield docker.from_env().containers.get(cluster.project_name + '_mysqljs1_1') + + def test_mysql_client(mysql_client, server_address): # type: (Container, str) -> None + code, (stdout, stderr) = mysql_client.exec_run(''' + mysql --protocol tcp -h {host} -P {port} default -u user_with_double_sha1 --password=abacaba + -e "SELECT 1;" + '''.format(host=server_address, port=server_port), demux=True) + + assert stdout == '\n'.join(['1', '1', '']) + code, (stdout, stderr) = mysql_client.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "SELECT 1 as a;" @@ -149,10 +163,24 @@ def test_golang_client(server_address, golang_container): def test_php_client(server_address, php_container): # type: (str, Container) -> None - code, (stdout, stderr) = php_container.exec_run('php -f test.php {host} {port} default 123 '.format(host=server_address, port=server_port), demux=True) + code, (stdout, stderr) = php_container.exec_run('php -f test.php {host} {port} default 123'.format(host=server_address, port=server_port), demux=True) assert code == 0 assert stdout == 'tables\n' - code, (stdout, stderr) = php_container.exec_run('php -f test_ssl.php {host} {port} default 123 '.format(host=server_address, port=server_port), demux=True) + code, (stdout, stderr) = php_container.exec_run('php -f test_ssl.php {host} {port} default 123'.format(host=server_address, port=server_port), demux=True) assert code == 0 assert stdout == 'tables\n' + + +def test_mysqljs_client(server_address, nodejs_container): + code, (_, _) = nodejs_container.exec_run('node test.js {host} {port} default 123'.format(host=server_address, port=server_port), demux=True) + assert code == 0 + + code, (_, _) = nodejs_container.exec_run('node test.js {host} {port} user_with_double_sha1 abacaba'.format(host=server_address, port=server_port), demux=True) + assert code == 0 + + code, (_, _) = nodejs_container.exec_run('node test.js {host} {port} user_with_empty_password ""'.format(host=server_address, port=server_port), demux=True) + assert code == 0 + + code, (_, _) = nodejs_container.exec_run('node test.js {host} {port} user_with_empty_password 123'.format(host=server_address, port=server_port), demux=True) + assert code == 1 From 760afb007c402b518cfa02bb07ad46f3554d951b Mon Sep 17 00:00:00 2001 From: Yuriy Date: Sun, 28 Jul 2019 16:36:27 +0300 Subject: [PATCH 031/161] arbitrary passwords --- dbms/src/Interpreters/UsersManager.cpp | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/dbms/src/Interpreters/UsersManager.cpp b/dbms/src/Interpreters/UsersManager.cpp index d5b454f2acc7..057257f1754c 100644 --- a/dbms/src/Interpreters/UsersManager.cpp +++ b/dbms/src/Interpreters/UsersManager.cpp @@ -77,18 +77,15 @@ UserPtr UsersManager::authorizeAndGetUser( } else if (!it->second->password_double_sha1_hex.empty()) { - /// MySQL compatibility server passes SHA1 instead of a password. If password length equals to 20, it is treated as a SHA1. Server stores double SHA1. Poco::SHA1Engine engine; - if (password.size() == Poco::SHA1Engine::DIGEST_SIZE) - { - engine.update(password); - } - else - { - engine.update(password); - const auto & first_sha1 = engine.digest(); - engine.update(first_sha1.data(), first_sha1.size()); - } + engine.update(password); + const auto & first_sha1 = engine.digest(); + + /// If it was MySQL compatibility server, then first_sha1 already contains double SHA1. + if (Poco::SHA1Engine::digestToHex(first_sha1) != it->second->password_double_sha1_hex) + return it->second; + + engine.update(first_sha1.data(), first_sha1.size()); if (Poco::SHA1Engine::digestToHex(engine.digest()) != it->second->password_double_sha1_hex) on_wrong_password(); From b1d5f4ca20a5275499dd6940bee60c4840459494 Mon Sep 17 00:00:00 2001 From: Yuriy Date: Sun, 28 Jul 2019 17:17:33 +0300 Subject: [PATCH 032/161] disabled mysql_native_password when password is specified as a plain text as it allows to connect to ClickHouse knowing only SHA1 instead of a password --- dbms/programs/server/MySQLHandler.cpp | 4 +-- dbms/src/Core/MySQLProtocol.h | 30 ++++++------------- dbms/src/Interpreters/UsersManager.cpp | 12 ++------ .../integration/test_mysql_protocol/test.py | 12 ++++---- 4 files changed, 20 insertions(+), 38 deletions(-) diff --git a/dbms/programs/server/MySQLHandler.cpp b/dbms/programs/server/MySQLHandler.cpp index 52dbc0a135ad..d225b7a58e87 100644 --- a/dbms/programs/server/MySQLHandler.cpp +++ b/dbms/programs/server/MySQLHandler.cpp @@ -227,9 +227,9 @@ void MySQLHandler::finishHandshake(MySQLProtocol::HandshakeResponse & packet) void MySQLHandler::authenticate(const String & user_name, const String & auth_plugin_name, const String & initial_auth_response) { - // For compatibility with JavaScript MySQL client, Native41 authentication plugin is used when it is possible. If password is specified using SHA-2, then SHA256 plugin is used. + // For compatibility with JavaScript MySQL client, Native41 authentication plugin is used when possible (if password is specified using double SHA1). Otherwise SHA256 plugin is used. auto user = connection_context.getUser(user_name); - if (!user->password_sha256_hex.empty()) + if (user->password_double_sha1_hex.empty()) auth_plugin = std::make_unique(public_key, private_key, log); try { diff --git a/dbms/src/Core/MySQLProtocol.h b/dbms/src/Core/MySQLProtocol.h index 0704cec0edb9..625f689d02ff 100644 --- a/dbms/src/Core/MySQLProtocol.h +++ b/dbms/src/Core/MySQLProtocol.h @@ -899,26 +899,13 @@ class Native41 : public IPlugin auto user = context.getUser(user_name); - if (!user->password_sha256_hex.empty()) - throw Exception("Cannot use " + getName() + " auth plugin for user " + user_name + " since its password is specified using SHA256.", ErrorCodes::UNKNOWN_EXCEPTION); + if (user->password_double_sha1_hex.empty()) + throw Exception("Cannot use " + getName() + " auth plugin for user " + user_name + " since its password isn't specified using double SHA1.", ErrorCodes::UNKNOWN_EXCEPTION); - Poco::SHA1Engine::Digest double_sha1_value; - Poco::SHA1Engine engine; - - /// If password is specified using double SHA1, than it is non-empty (unlike plaintext password, which can be empty). - if (!user->password_double_sha1_hex.empty()) - { - double_sha1_value = Poco::DigestEngine::digestFromHex(user->password_double_sha1_hex); - assert(double_sha1_value.size() == Poco::SHA1Engine::DIGEST_SIZE); - } - else - { - engine.update(user->password); - const Poco::SHA1Engine::Digest & first_sha1 = engine.digest(); - engine.update(first_sha1.data(), first_sha1.size()); - double_sha1_value = engine.digest(); - } + Poco::SHA1Engine::Digest double_sha1_value = Poco::DigestEngine::digestFromHex(user->password_double_sha1_hex); + assert(double_sha1_value.size() == Poco::SHA1Engine::DIGEST_SIZE); + Poco::SHA1Engine engine; engine.update(scramble.data(), SCRAMBLE_LENGTH); engine.update(double_sha1_value.data(), double_sha1_value.size()); @@ -948,8 +935,8 @@ class Sha256Password : public IPlugin scramble.resize(SCRAMBLE_LENGTH + 1, 0); Poco::RandomInputStream generator; - for (char & c : scramble) - generator >> c; + for (size_t i = 0; i < SCRAMBLE_LENGTH; i++) + generator >> scramble[i]; } String getName() override @@ -975,7 +962,8 @@ class Sha256Password : public IPlugin packet_sender->sendPacket(AuthSwitchRequest(getName(), scramble), true); if (packet_sender->in->eof()) - throw Exception("Client doesn't support authentication method " + getName() + " used by ClickHouse", ErrorCodes::MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES); + throw Exception("Client doesn't support authentication method " + getName() + " used by ClickHouse. Specifying user password using 'password_double_sha1_hex' may fix the problem.", + ErrorCodes::MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES); AuthSwitchResponse response; packet_sender->receivePacket(response); diff --git a/dbms/src/Interpreters/UsersManager.cpp b/dbms/src/Interpreters/UsersManager.cpp index 057257f1754c..6c33187f526d 100644 --- a/dbms/src/Interpreters/UsersManager.cpp +++ b/dbms/src/Interpreters/UsersManager.cpp @@ -82,7 +82,7 @@ UserPtr UsersManager::authorizeAndGetUser( const auto & first_sha1 = engine.digest(); /// If it was MySQL compatibility server, then first_sha1 already contains double SHA1. - if (Poco::SHA1Engine::digestToHex(first_sha1) != it->second->password_double_sha1_hex) + if (Poco::SHA1Engine::digestToHex(first_sha1) == it->second->password_double_sha1_hex) return it->second; engine.update(first_sha1.data(), first_sha1.size()); @@ -92,15 +92,7 @@ UserPtr UsersManager::authorizeAndGetUser( } else if (password != it->second->password) { - /// MySQL compatibility server passes SHA1 instead of a password. - if (password.size() != Poco::SHA1Engine::DIGEST_SIZE) - on_wrong_password(); - - Poco::SHA1Engine engine; - engine.update(it->second->password); - - if (engine.digest() != Poco::SHA1Engine::Digest(password.begin(), password.end())) - on_wrong_password(); + on_wrong_password(); } return it->second; diff --git a/dbms/tests/integration/test_mysql_protocol/test.py b/dbms/tests/integration/test_mysql_protocol/test.py index 39f3793cc5c2..f8d79cb2e320 100644 --- a/dbms/tests/integration/test_mysql_protocol/test.py +++ b/dbms/tests/integration/test_mysql_protocol/test.py @@ -173,13 +173,15 @@ def test_php_client(server_address, php_container): def test_mysqljs_client(server_address, nodejs_container): - code, (_, _) = nodejs_container.exec_run('node test.js {host} {port} default 123'.format(host=server_address, port=server_port), demux=True) - assert code == 0 + code, (_, stderr) = nodejs_container.exec_run('node test.js {host} {port} default 123'.format(host=server_address, port=server_port), demux=True) + assert code == 1 + assert 'MySQL is requesting the sha256_password authentication method, which is not supported.' in stderr - code, (_, _) = nodejs_container.exec_run('node test.js {host} {port} user_with_double_sha1 abacaba'.format(host=server_address, port=server_port), demux=True) - assert code == 0 + code, (_, stderr) = nodejs_container.exec_run('node test.js {host} {port} user_with_empty_password ""'.format(host=server_address, port=server_port), demux=True) + assert code == 1 + assert 'MySQL is requesting the sha256_password authentication method, which is not supported.' in stderr - code, (_, _) = nodejs_container.exec_run('node test.js {host} {port} user_with_empty_password ""'.format(host=server_address, port=server_port), demux=True) + code, (_, _) = nodejs_container.exec_run('node test.js {host} {port} user_with_double_sha1 abacaba'.format(host=server_address, port=server_port), demux=True) assert code == 0 code, (_, _) = nodejs_container.exec_run('node test.js {host} {port} user_with_empty_password 123'.format(host=server_address, port=server_port), demux=True) From dd2e3ab7f78b87d045664b0d15038f904d6a0825 Mon Sep 17 00:00:00 2001 From: Yuriy Date: Sun, 28 Jul 2019 17:24:52 +0300 Subject: [PATCH 033/161] removed wrong comment --- dbms/tests/integration/test_mysql_protocol/configs/users.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/tests/integration/test_mysql_protocol/configs/users.xml b/dbms/tests/integration/test_mysql_protocol/configs/users.xml index fb4b948d896d..ebcd1a297e14 100644 --- a/dbms/tests/integration/test_mysql_protocol/configs/users.xml +++ b/dbms/tests/integration/test_mysql_protocol/configs/users.xml @@ -27,7 +27,6 @@ - ::/0 From 39b40f8e429ace52f19d206eec89bb897f2af079 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 28 Jul 2019 17:55:51 +0300 Subject: [PATCH 034/161] fix replicated indices mutations --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../0_stateless/00975_indices_mutation_replicated_zookeeper.sh | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) mode change 100644 => 100755 dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 0134c425a946..ef474e612cf5 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -865,7 +865,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) return true; } - if (entry.type == LogEntry::CLEAR_COLUMN) + if (entry.type == LogEntry::CLEAR_COLUMN || entry.type == LogEntry::CLEAR_INDEX) { executeClearColumnOrIndexInPartition(entry); return true; diff --git a/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh b/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh old mode 100644 new mode 100755 index 4490487663f8..613226a3fb79 --- a/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh @@ -3,7 +3,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.indices_mutaions1;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.indices_mutaions2;" $CLICKHOUSE_CLIENT -n --query=" From f1694a17467bcae6f42398c2870837ccea200568 Mon Sep 17 00:00:00 2001 From: Yuriy Date: Mon, 29 Jul 2019 03:09:17 +0300 Subject: [PATCH 035/161] build fix --- dbms/src/Core/MySQLProtocol.h | 2 -- dbms/src/Interpreters/UsersManager.cpp | 1 - 2 files changed, 3 deletions(-) diff --git a/dbms/src/Core/MySQLProtocol.h b/dbms/src/Core/MySQLProtocol.h index 625f689d02ff..046fb297fb10 100644 --- a/dbms/src/Core/MySQLProtocol.h +++ b/dbms/src/Core/MySQLProtocol.h @@ -20,8 +20,6 @@ #include #include #include -#include -#include #include #include #include diff --git a/dbms/src/Interpreters/UsersManager.cpp b/dbms/src/Interpreters/UsersManager.cpp index 6c33187f526d..6d1f7152b9e2 100644 --- a/dbms/src/Interpreters/UsersManager.cpp +++ b/dbms/src/Interpreters/UsersManager.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include From b8dff6ebb1b26f8818c78dfcee193163421d68fb Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Thu, 1 Aug 2019 22:03:39 +0300 Subject: [PATCH 036/161] more optimal ranges selection at reading in order --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 70 +++++++++++-------- 1 file changed, 42 insertions(+), 28 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ccbcd04857d6..d528755005b1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -831,39 +831,11 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithO size_t adaptive_parts = 0; std::vector sum_marks_in_parts(parts.size()); - /// In case of reverse order let's split ranges to avoid reading much data. - auto split_ranges = [max_block_size](const auto & ranges, size_t rows_granularity, size_t num_marks_in_part) - { - /// Constants is just a guess. - const size_t min_rows_in_range = max_block_size; - const size_t max_num_ranges = 64; - - size_t min_marks_in_range = std::max( - (min_rows_in_range + rows_granularity - 1) / rows_granularity, - (num_marks_in_part + max_num_ranges - 1) / max_num_ranges); - - MarkRanges new_ranges; - for (auto range : ranges) - { - while (range.begin + min_marks_in_range < range.end) - { - new_ranges.emplace_back(range.begin, range.begin + min_marks_in_range); - range.begin += min_marks_in_range; - } - new_ranges.emplace_back(range.begin, range.end); - } - - return new_ranges; - }; - for (size_t i = 0; i < parts.size(); ++i) { sum_marks_in_parts[i] = parts[i].getMarksCount(); sum_marks += sum_marks_in_parts[i]; - if (sorting_info->direction == -1) - parts[i].ranges = split_ranges(parts[i].ranges, data.settings.index_granularity, sum_marks_in_parts[i]); - /// Let the ranges be listed from right to left so that the leftmost range can be dropped using `pop_back()`. std::reverse(parts[i].ranges.begin(), parts[i].ranges.end()); @@ -895,6 +867,46 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithO if (sum_marks == 0) return streams; + /// In case of reverse order let's split ranges to avoid reading much data. + auto split_ranges = [rows_granularity = data.settings.index_granularity, max_block_size](const auto & ranges, int direction) + { + if (direction == 1) + return ranges; + + MarkRanges new_ranges; + + /// Probably it useful to make it larger. + const size_t max_marks_in_range = (max_block_size + rows_granularity - 1) / rows_granularity; + size_t marks_in_range = 1; + + for (auto range : ranges) + { + while (range.begin + marks_in_range < range.end) + { + if (direction == 1) + { + /// Comment + new_ranges.emplace_back(range.begin, range.begin + marks_in_range); + range.begin += marks_in_range; + marks_in_range *= 2; + + if (marks_in_range > max_block_size) + break; + } + else + { + /// Comment + new_ranges.emplace_back(range.end - marks_in_range, range.end); + range.end -= marks_in_range; + marks_in_range = std::min(marks_in_range * 2, max_marks_in_range); + } + } + new_ranges.emplace_back(range.begin, range.end); + } + + return new_ranges; + }; + const size_t min_marks_per_stream = (sum_marks - 1) / num_streams + 1; for (size_t i = 0; i < num_streams && !parts.empty(); ++i) @@ -959,6 +971,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithO parts.emplace_back(part); } + ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, sorting_info->direction); + BlockInputStreamPtr source_stream; if (sorting_info->direction == 1) { From 5f7ebb18eda0ff629984d90f2443a95b50940f1f Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 2 Aug 2019 19:16:18 +0300 Subject: [PATCH 037/161] more optimal ranges selection at reading in order --- .../FinishSortingBlockInputStream.cpp | 2 -- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 16 +++++++++++----- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 14 +++++--------- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp b/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp index 461a1b36a654..e7382bf8b6df 100644 --- a/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp @@ -102,8 +102,6 @@ Block FinishSortingBlockInputStream::readImpl() if (block.rows() == 0) continue; - // We need to sort each block separately before merging. - sortBlock(block, description_to_sort, limit); removeConstantsFromBlock(block); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 11d90fd4f4ad..a2beadc3ddf1 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1915,13 +1915,12 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr so * At this stage we merge per-thread streams into one. */ - if (sorting_info->prefix_order_descr.size() < order_descr.size()) + bool need_finish_sorting = (sorting_info->prefix_order_descr.size() < order_descr.size()); + if (need_finish_sorting) { pipeline.transform([&](auto & stream) { - stream = std::make_shared( - stream, sorting_info->prefix_order_descr, - order_descr, settings.max_block_size, limit); + stream = std::make_shared(stream, order_descr, limit); }); } @@ -1933,10 +1932,17 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr so }); pipeline.firstStream() = std::make_shared( - pipeline.streams, order_descr, + pipeline.streams, sorting_info->prefix_order_descr, settings.max_block_size, limit); pipeline.streams.resize(1); } + + if (need_finish_sorting) + { + pipeline.firstStream() = std::make_shared( + pipeline.firstStream(), sorting_info->prefix_order_descr, + order_descr, settings.max_block_size, limit); + } } else { diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index d528755005b1..d1179787d9a7 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -867,25 +867,20 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithO if (sum_marks == 0) return streams; - /// In case of reverse order let's split ranges to avoid reading much data. + /// Let's split ranges to avoid reading much data. auto split_ranges = [rows_granularity = data.settings.index_granularity, max_block_size](const auto & ranges, int direction) { - if (direction == 1) - return ranges; - MarkRanges new_ranges; - - /// Probably it useful to make it larger. const size_t max_marks_in_range = (max_block_size + rows_granularity - 1) / rows_granularity; - size_t marks_in_range = 1; for (auto range : ranges) { + size_t marks_in_range = 1; while (range.begin + marks_in_range < range.end) { if (direction == 1) { - /// Comment + /// Split first few ranges to avoid reading much data. new_ranges.emplace_back(range.begin, range.begin + marks_in_range); range.begin += marks_in_range; marks_in_range *= 2; @@ -895,7 +890,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithO } else { - /// Comment + /// Split all ranges to avoid reading much data, because we have to + /// store whole range in memory to reverse it. new_ranges.emplace_back(range.end - marks_in_range, range.end); range.end -= marks_in_range; marks_in_range = std::min(marks_in_range * 2, max_marks_in_range); From a0f61e8cb3c1391210f48d287c4620a0708ac675 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 2 Aug 2019 19:17:18 +0300 Subject: [PATCH 038/161] better performance test for order by optimization --- dbms/tests/performance/order_by_read_in_order.xml | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/dbms/tests/performance/order_by_read_in_order.xml b/dbms/tests/performance/order_by_read_in_order.xml index f52c9a0ce55d..d0c5350b3c6c 100644 --- a/dbms/tests/performance/order_by_read_in_order.xml +++ b/dbms/tests/performance/order_by_read_in_order.xml @@ -25,9 +25,13 @@ - test.hits + default.hits_100m_single -SELECT CounterID FROM test.hits ORDER BY CounterID, EventDate DESC LIMIT 50 +SELECT * FROM hits_100m_single ORDER BY CounterID, EventDate LIMIT 100 +SELECT * FROM hits_100m_single ORDER BY CounterID DESC, toStartOfWeek(EventDate) DESC LIMIT 100 +SELECT * FROM hits_100m_single ORDER BY CounterID, EventDate, URL LIMIT 100 +SELECT * FROM hits_100m_single WHERE CounterID IN (152220, 168777, 149234, 149234) ORDER BY CounterID DESC, EventDate DESC LIMIT 100 +SELECT * FROM hits_100m_single WHERE UserID=1988954671305023629 ORDER BY CounterID, EventDate LIMIT 100 From a6e85a331a58a58a11176c00fbe5ee3726c2840b Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 2 Aug 2019 19:37:43 +0300 Subject: [PATCH 039/161] more optimal ranges selection at reading in order --- dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index d1179787d9a7..db73b54f288b 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -885,7 +885,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithO range.begin += marks_in_range; marks_in_range *= 2; - if (marks_in_range > max_block_size) + if (marks_in_range * rows_granularity > max_block_size) break; } else From e06a53a5d90f454b4c1e6849095bfca5c20a29fd Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 2 Aug 2019 20:34:29 +0300 Subject: [PATCH 040/161] fix ranges selection at reading in order --- .../Interpreters/InterpreterSelectQuery.cpp | 3 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 32 +++++++++----- .../00940_order_by_read_in_order.sql | 43 +++++++++---------- 3 files changed, 44 insertions(+), 34 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index a2beadc3ddf1..0b9f24590fb0 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1931,9 +1931,10 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr so stream = std::make_shared(stream); }); + UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit); pipeline.firstStream() = std::make_shared( pipeline.streams, sorting_info->prefix_order_descr, - settings.max_block_size, limit); + settings.max_block_size, limit_for_merging); pipeline.streams.resize(1); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index db73b54f288b..394f5ddc326d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -872,32 +872,42 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithO { MarkRanges new_ranges; const size_t max_marks_in_range = (max_block_size + rows_granularity - 1) / rows_granularity; + size_t marks_in_range = 1; - for (auto range : ranges) + if (direction == 1) { - size_t marks_in_range = 1; - while (range.begin + marks_in_range < range.end) + /// Split first few ranges to avoid reading much data. + bool splitted = false; + for (auto range : ranges) { - if (direction == 1) + while (!splitted && range.begin + marks_in_range < range.end) { - /// Split first few ranges to avoid reading much data. new_ranges.emplace_back(range.begin, range.begin + marks_in_range); range.begin += marks_in_range; marks_in_range *= 2; - if (marks_in_range * rows_granularity > max_block_size) - break; + if (marks_in_range > max_marks_in_range) + splitted = true; } - else + new_ranges.emplace_back(range.begin, range.end); + } + } + else + { + /// Split all ranges to avoid reading much data, because we have to + /// store whole range in memory to reverse it. + for (auto it = ranges.rbegin(); it != ranges.rend(); ++it) + { + auto range = *it; + while (range.begin + marks_in_range < range.end) { - /// Split all ranges to avoid reading much data, because we have to - /// store whole range in memory to reverse it. new_ranges.emplace_back(range.end - marks_in_range, range.end); range.end -= marks_in_range; marks_in_range = std::min(marks_in_range * 2, max_marks_in_range); } + new_ranges.emplace_back(range.begin, range.end); } - new_ranges.emplace_back(range.begin, range.end); + std::reverse(new_ranges.begin(), new_ranges.end()); } return new_ranges; diff --git a/dbms/tests/queries/0_stateless/00940_order_by_read_in_order.sql b/dbms/tests/queries/0_stateless/00940_order_by_read_in_order.sql index 55fca9700dd9..34c0d16716e3 100644 --- a/dbms/tests/queries/0_stateless/00940_order_by_read_in_order.sql +++ b/dbms/tests/queries/0_stateless/00940_order_by_read_in_order.sql @@ -1,33 +1,32 @@ -CREATE DATABASE IF NOT EXISTS test; -DROP TABLE IF EXISTS test.pk_order; +DROP TABLE IF EXISTS pk_order; SET optimize_read_in_order = 1; -CREATE TABLE test.pk_order(a UInt64, b UInt64, c UInt64, d UInt64) ENGINE=MergeTree() ORDER BY (a, b); -INSERT INTO test.pk_order(a, b, c, d) VALUES (1, 1, 101, 1), (1, 2, 102, 1), (1, 3, 103, 1), (1, 4, 104, 1); -INSERT INTO test.pk_order(a, b, c, d) VALUES (1, 5, 104, 1), (1, 6, 105, 1), (2, 1, 106, 2), (2, 1, 107, 2); +CREATE TABLE pk_order(a UInt64, b UInt64, c UInt64, d UInt64) ENGINE=MergeTree() ORDER BY (a, b); +INSERT INTO pk_order(a, b, c, d) VALUES (1, 1, 101, 1), (1, 2, 102, 1), (1, 3, 103, 1), (1, 4, 104, 1); +INSERT INTO pk_order(a, b, c, d) VALUES (1, 5, 104, 1), (1, 6, 105, 1), (2, 1, 106, 2), (2, 1, 107, 2); -INSERT INTO test.pk_order(a, b, c, d) VALUES (2, 2, 107, 2), (2, 3, 108, 2), (2, 4, 109, 2); +INSERT INTO pk_order(a, b, c, d) VALUES (2, 2, 107, 2), (2, 3, 108, 2), (2, 4, 109, 2); -SELECT b FROM test.pk_order ORDER BY a, b; -SELECT a FROM test.pk_order ORDER BY a, b; +SELECT b FROM pk_order ORDER BY a, b; +SELECT a FROM pk_order ORDER BY a, b; -SELECT a, b FROM test.pk_order ORDER BY a, b; -SELECT a, b FROM test.pk_order ORDER BY a DESC, b; -SELECT a, b FROM test.pk_order ORDER BY a, b DESC; -SELECT a, b FROM test.pk_order ORDER BY a DESC, b DESC; -SELECT a FROM test.pk_order ORDER BY a DESC; +SELECT a, b FROM pk_order ORDER BY a, b; +SELECT a, b FROM pk_order ORDER BY a DESC, b; +SELECT a, b FROM pk_order ORDER BY a, b DESC; +SELECT a, b FROM pk_order ORDER BY a DESC, b DESC; +SELECT a FROM pk_order ORDER BY a DESC; -SELECT a, b, c FROM test.pk_order ORDER BY a, b, c; -SELECT a, b, c FROM test.pk_order ORDER BY a DESC, b, c; -SELECT a, b, c FROM test.pk_order ORDER BY a, b DESC, c; -SELECT a, b, c FROM test.pk_order ORDER BY a, b, c DESC; -SELECT a, b, c FROM test.pk_order ORDER BY a DESC, b DESC, c; -SELECT a, b, c FROM test.pk_order ORDER BY a DESC, b, c DESC; -SELECT a, b, c FROM test.pk_order ORDER BY a, b DESC, c DESC; -SELECT a, b, c FROM test.pk_order ORDER BY a DESC, b DESC, c DESC; +SELECT a, b, c FROM pk_order ORDER BY a, b, c; +SELECT a, b, c FROM pk_order ORDER BY a DESC, b, c; +SELECT a, b, c FROM pk_order ORDER BY a, b DESC, c; +SELECT a, b, c FROM pk_order ORDER BY a, b, c DESC; +SELECT a, b, c FROM pk_order ORDER BY a DESC, b DESC, c; +SELECT a, b, c FROM pk_order ORDER BY a DESC, b, c DESC; +SELECT a, b, c FROM pk_order ORDER BY a, b DESC, c DESC; +SELECT a, b, c FROM pk_order ORDER BY a DESC, b DESC, c DESC; -DROP TABLE IF EXISTS test.pk_order; +DROP TABLE IF EXISTS pk_order; CREATE TABLE pk_order (d DateTime, a Int32, b Int32) ENGINE = MergeTree ORDER BY (d, a) PARTITION BY toDate(d) SETTINGS index_granularity=1; From 5804398eaefe64bc7065e429e87e74e4e36aaa56 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 4 Aug 2019 17:32:33 +0300 Subject: [PATCH 041/161] new test --- .../queries/0_stateless/00943_materialize_index.reference | 2 ++ dbms/tests/queries/0_stateless/00943_materialize_index.sh | 5 +++++ 2 files changed, 7 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00943_materialize_index.reference b/dbms/tests/queries/0_stateless/00943_materialize_index.reference index 00aa71e8bd10..aa4014f798d8 100644 --- a/dbms/tests/queries/0_stateless/00943_materialize_index.reference +++ b/dbms/tests/queries/0_stateless/00943_materialize_index.reference @@ -4,5 +4,7 @@ "rows_read": 8, 5 "rows_read": 6, +5 + "rows_read": 10, 5 "rows_read": 6, diff --git a/dbms/tests/queries/0_stateless/00943_materialize_index.sh b/dbms/tests/queries/0_stateless/00943_materialize_index.sh index 4ab061d93474..96cdc27a7ec8 100755 --- a/dbms/tests/queries/0_stateless/00943_materialize_index.sh +++ b/dbms/tests/queries/0_stateless/00943_materialize_index.sh @@ -47,6 +47,11 @@ $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx;" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" + $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" From 4c53b90e66b7b5f8df33dfaeea6c36a6c090dc58 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 4 Aug 2019 21:28:46 +0300 Subject: [PATCH 042/161] fixed MergedColumnOnlyOutputStream --- .../MergeTree/MergedColumnOnlyOutputStream.cpp | 10 +++++----- .../queries/0_stateless/00943_materialize_index.sh | 3 ++- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index a694d509b687..0c0b1785addc 100644 --- a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -27,13 +27,13 @@ void MergedColumnOnlyOutputStream::write(const Block & block) { column_streams.clear(); serialization_states.clear(); - serialization_states.reserve(block.columns()); + serialization_states.reserve(header.columns()); WrittenOffsetColumns tmp_offset_columns; IDataType::SerializeBinaryBulkSettings settings; - for (size_t i = 0; i < block.columns(); ++i) + for (const auto & column_name : header.getNames()) { - const auto & col = block.safeGetByPosition(i); + const auto & col = block.getByName(column_name); const auto columns = storage.getColumns(); addStreams(part_path, col.name, *col.type, columns.getCodecOrDefault(col.name, codec), 0, skip_offsets); @@ -71,9 +71,9 @@ void MergedColumnOnlyOutputStream::write(const Block & block) size_t new_index_offset = 0; size_t new_current_mark = 0; WrittenOffsetColumns offset_columns = already_written_offset_columns; - for (size_t i = 0; i < block.columns(); ++i) + for (size_t i = 0; i < header.columns(); ++i) { - const ColumnWithTypeAndName & column = block.safeGetByPosition(i); + const ColumnWithTypeAndName & column = block.getByName(header.getByPosition(i).name); std::tie(new_current_mark, new_index_offset) = writeColumn(column.name, *column.type, *column.column, offset_columns, skip_offsets, serialization_states[i], current_mark); } diff --git a/dbms/tests/queries/0_stateless/00943_materialize_index.sh b/dbms/tests/queries/0_stateless/00943_materialize_index.sh index 96cdc27a7ec8..f51b66993aa4 100755 --- a/dbms/tests/queries/0_stateless/00943_materialize_index.sh +++ b/dbms/tests/queries/0_stateless/00943_materialize_index.sh @@ -47,7 +47,8 @@ $CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx;" +$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 1;" +$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" $CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" From 7b8c0f1750032863fa72bd29bd8bbaae2531cbd7 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Mon, 5 Aug 2019 21:06:05 +0300 Subject: [PATCH 043/161] experiment --- .../src/Interpreters/MutationsInterpreter.cpp | 51 +++++++++++++++---- dbms/src/Interpreters/MutationsInterpreter.h | 9 +++- .../MergeTree/MergeTreeDataMergerMutator.cpp | 12 +++-- 3 files changed, 56 insertions(+), 16 deletions(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 541b77e0f60c..7eb4562961e4 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -217,19 +217,20 @@ void MutationsInterpreter::prepare(bool dry_run) } /// First, break a sequence of commands into stages. - stages.emplace_back(context); for (const auto & command : commands) { - if (!stages.back().column_to_updated.empty()) - stages.emplace_back(context); - if (command.type == MutationCommand::DELETE) { + if (stages.empty() || !stages.back().column_to_updated.empty()) + stages.emplace_back(context); + auto negated_predicate = makeASTFunction("not", command.predicate->clone()); stages.back().filters.push_back(negated_predicate); } else if (command.type == MutationCommand::UPDATE) { + if (stages.empty() || !stages.back().column_to_updated.empty()) + stages.emplace_back(context); if (stages.size() == 1) /// First stage only supports filtering and can't update columns. stages.emplace_back(context); @@ -291,15 +292,39 @@ void MutationsInterpreter::prepare(bool dry_run) throw Exception("Unknown mutation command type: " + DB::toString(command.type), ErrorCodes::UNKNOWN_MUTATION_COMMAND); } - /// Special step to recalculate affected indices. if (!affected_indices_columns.empty()) { + if (!stages.empty()) + { + std::vector stages_copy; + /// Copy all filled stages except index calculation stage. + for (const auto &stage : stages) + { + stages_copy.emplace_back(context); + stages_copy.back().column_to_updated = stage.column_to_updated; + stages_copy.back().output_columns = stage.output_columns; + stages_copy.back().filters = stage.filters; + } + auto first_stage_header = prepareInterpreterSelect(/* dry_run = */ true)->getSampleBlock(); + auto in = std::make_shared(first_stage_header); + updated_header = std::make_unique(addStreamsForLaterStages(in)->getHeader()); + std::swap(stages, stages_copy); + } + /// Special step to recalculate affected indices. stages.emplace_back(context); for (const auto & column : affected_indices_columns) stages.back().column_to_updated.emplace( column, std::make_shared(column)); } + interpreter_select = prepareInterpreterSelect(dry_run); + + is_prepared = true; +} + +std::unique_ptr MutationsInterpreter::prepareInterpreterSelect(bool dry_run) +{ + NamesAndTypesList all_columns = storage->getColumns().getAllPhysical(); /// Next, for each stage calculate columns changed by this and previous stages. for (size_t i = 0; i < stages.size(); ++i) @@ -363,7 +388,7 @@ void MutationsInterpreter::prepare(bool dry_run) for (const auto & kv : stage.column_to_updated) { actions_chain.getLastActions()->add(ExpressionAction::copyColumn( - kv.second->getColumnName(), kv.first, /* can_replace = */ true)); + kv.second->getColumnName(), kv.first, /* can_replace = */ true)); } } @@ -403,9 +428,7 @@ void MutationsInterpreter::prepare(bool dry_run) select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where_expression)); } - interpreter_select = std::make_unique(select, context, storage, SelectQueryOptions().analyze(dry_run).ignoreLimits()); - - is_prepared = true; + return std::make_unique(select, context, storage, SelectQueryOptions().analyze(dry_run).ignoreLimits()); } BlockInputStreamPtr MutationsInterpreter::addStreamsForLaterStages(BlockInputStreamPtr in) const @@ -451,7 +474,15 @@ BlockInputStreamPtr MutationsInterpreter::execute() { prepare(/* dry_run = */ false); BlockInputStreamPtr in = interpreter_select->execute().in; - return addStreamsForLaterStages(in); + auto result_stream = addStreamsForLaterStages(in); + if (!updated_header) + updated_header = std::make_unique(result_stream->getHeader()); + return result_stream; +} + +const Block & MutationsInterpreter::getUpdatedHeader() const +{ + return *updated_header; } } diff --git a/dbms/src/Interpreters/MutationsInterpreter.h b/dbms/src/Interpreters/MutationsInterpreter.h index 28168f02d2d0..9880b07e686a 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.h +++ b/dbms/src/Interpreters/MutationsInterpreter.h @@ -30,12 +30,18 @@ class MutationsInterpreter /// Return false if the data isn't going to be changed by mutations. bool isStorageTouchedByMutations() const; - /// The resulting stream will return blocks containing changed columns only. + /// The resulting stream will return blocks containing only changed columns and columns, that we need to recalculate indices. BlockInputStreamPtr execute(); + /// Only changed columns. + const Block & getUpdatedHeader() const; + private: void prepare(bool dry_run); + struct Stage; + + std::unique_ptr prepareInterpreterSelect(bool dry_run); BlockInputStreamPtr addStreamsForLaterStages(BlockInputStreamPtr in) const; private: @@ -78,6 +84,7 @@ class MutationsInterpreter }; std::unique_ptr interpreter_select; + std::unique_ptr updated_header; std::vector stages; bool is_prepared = false; /// Has the sequence of stages been prepared. }; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 93c62593daab..19d775890d88 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -949,6 +949,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor Poco::File(new_part_tmp_path).createDirectories(); auto in = mutations_interpreter.execute(); + const auto & updated_header = mutations_interpreter.getUpdatedHeader(); + NamesAndTypesList all_columns = data.getColumns().getAllPhysical(); Block in_header = in->getHeader(); @@ -957,7 +959,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor MergeStageProgress stage_progress(1.0); in->setProgressCallback(MergeProgressCallback(merge_entry, watch_prev_elapsed, stage_progress)); - if (in_header.columns() == all_columns.size()) + if (updated_header.columns() == all_columns.size()) { /// All columns are modified, proceed to write a new part from scratch. if (data.hasPrimaryKey() || data.hasSkipIndices()) @@ -1025,7 +1027,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor NameSet files_to_skip = {"checksums.txt", "columns.txt"}; auto mrk_extension = data.settings.index_granularity_bytes ? getAdaptiveMrkExtension() : getNonAdaptiveMrkExtension(); - for (const auto & entry : in_header) + for (const auto & entry : updated_header) { IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path) { @@ -1055,12 +1057,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor createHardLink(dir_it.path().toString(), destination.toString()); } - merge_entry->columns_written = all_columns.size() - in_header.columns(); + merge_entry->columns_written = all_columns.size() - updated_header.columns(); IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets; MergedColumnOnlyOutputStream out( data, - in_header, + updated_header, new_part_tmp_path, /* sync = */ false, compression_codec, @@ -1099,7 +1101,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor NameSet source_columns_name_set(source_column_names.begin(), source_column_names.end()); for (auto it = new_data_part->columns.begin(); it != new_data_part->columns.end();) { - if (source_columns_name_set.count(it->name) || in_header.has(it->name)) + if (source_columns_name_set.count(it->name) || updated_header.has(it->name)) ++it; else it = new_data_part->columns.erase(it); From 4e8ab12816b0f790edaebf885904412f88b17512 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Mon, 5 Aug 2019 21:22:44 +0300 Subject: [PATCH 044/161] refactoring --- .../src/Interpreters/MutationsInterpreter.cpp | 49 +++++++++---------- dbms/src/Interpreters/MutationsInterpreter.h | 4 +- 2 files changed, 26 insertions(+), 27 deletions(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 7eb4562961e4..bf822312821e 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -305,10 +305,9 @@ void MutationsInterpreter::prepare(bool dry_run) stages_copy.back().output_columns = stage.output_columns; stages_copy.back().filters = stage.filters; } - auto first_stage_header = prepareInterpreterSelect(/* dry_run = */ true)->getSampleBlock(); + auto first_stage_header = prepareInterpreterSelect(stages_copy, /* dry_run = */ true)->getSampleBlock(); auto in = std::make_shared(first_stage_header); - updated_header = std::make_unique(addStreamsForLaterStages(in)->getHeader()); - std::swap(stages, stages_copy); + updated_header = std::make_unique(addStreamsForLaterStages(stages_copy, in)->getHeader()); } /// Special step to recalculate affected indices. stages.emplace_back(context); @@ -317,40 +316,40 @@ void MutationsInterpreter::prepare(bool dry_run) column, std::make_shared(column)); } - interpreter_select = prepareInterpreterSelect(dry_run); + interpreter_select = prepareInterpreterSelect(stages, dry_run); is_prepared = true; } -std::unique_ptr MutationsInterpreter::prepareInterpreterSelect(bool dry_run) +std::unique_ptr MutationsInterpreter::prepareInterpreterSelect(std::vector & prepared_stages, bool dry_run) { NamesAndTypesList all_columns = storage->getColumns().getAllPhysical(); /// Next, for each stage calculate columns changed by this and previous stages. - for (size_t i = 0; i < stages.size(); ++i) + for (size_t i = 0; i < prepared_stages.size(); ++i) { - if (!stages[i].filters.empty()) + if (!prepared_stages[i].filters.empty()) { for (const auto & column : all_columns) - stages[i].output_columns.insert(column.name); + prepared_stages[i].output_columns.insert(column.name); continue; } if (i > 0) - stages[i].output_columns = stages[i - 1].output_columns; + prepared_stages[i].output_columns = prepared_stages[i - 1].output_columns; - if (stages[i].output_columns.size() < all_columns.size()) + if (prepared_stages[i].output_columns.size() < all_columns.size()) { - for (const auto & kv : stages[i].column_to_updated) - stages[i].output_columns.insert(kv.first); + for (const auto & kv : prepared_stages[i].column_to_updated) + prepared_stages[i].output_columns.insert(kv.first); } } /// Now, calculate `expressions_chain` for each stage except the first. /// Do it backwards to propagate information about columns required as input for a stage to the previous stage. - for (size_t i = stages.size() - 1; i > 0; --i) + for (size_t i = prepared_stages.size() - 1; i > 0; --i) { - auto & stage = stages[i]; + auto & stage = prepared_stages[i]; ASTPtr all_asts = std::make_shared(); @@ -400,7 +399,7 @@ std::unique_ptr MutationsInterpreter::prepareInterpreter /// Propagate information about columns needed as input. for (const auto & column : actions_chain.steps.front().actions->getRequiredColumnsWithTypes()) - stages[i - 1].output_columns.insert(column.name); + prepared_stages[i - 1].output_columns.insert(column.name); } /// Execute first stage as a SELECT statement. @@ -408,21 +407,21 @@ std::unique_ptr MutationsInterpreter::prepareInterpreter auto select = std::make_shared(); select->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); - for (const auto & column_name : stages[0].output_columns) + for (const auto & column_name : prepared_stages[0].output_columns) select->select()->children.push_back(std::make_shared(column_name)); - if (!stages[0].filters.empty()) + if (!prepared_stages[0].filters.empty()) { ASTPtr where_expression; - if (stages[0].filters.size() == 1) - where_expression = stages[0].filters[0]; + if (prepared_stages[0].filters.size() == 1) + where_expression = prepared_stages[0].filters[0]; else { auto coalesced_predicates = std::make_shared(); coalesced_predicates->name = "and"; coalesced_predicates->arguments = std::make_shared(); coalesced_predicates->children.push_back(coalesced_predicates->arguments); - coalesced_predicates->arguments->children = stages[0].filters; + coalesced_predicates->arguments->children = prepared_stages[0].filters; where_expression = std::move(coalesced_predicates); } select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where_expression)); @@ -431,11 +430,11 @@ std::unique_ptr MutationsInterpreter::prepareInterpreter return std::make_unique(select, context, storage, SelectQueryOptions().analyze(dry_run).ignoreLimits()); } -BlockInputStreamPtr MutationsInterpreter::addStreamsForLaterStages(BlockInputStreamPtr in) const +BlockInputStreamPtr MutationsInterpreter::addStreamsForLaterStages(const std::vector & prepared_stages, BlockInputStreamPtr in) const { - for (size_t i_stage = 1; i_stage < stages.size(); ++i_stage) + for (size_t i_stage = 1; i_stage < prepared_stages.size(); ++i_stage) { - const Stage & stage = stages[i_stage]; + const Stage & stage = prepared_stages[i_stage]; for (size_t i = 0; i < stage.expressions_chain.steps.size(); ++i) { @@ -467,14 +466,14 @@ void MutationsInterpreter::validate() prepare(/* dry_run = */ true); Block first_stage_header = interpreter_select->getSampleBlock(); BlockInputStreamPtr in = std::make_shared(first_stage_header); - addStreamsForLaterStages(in)->getHeader(); + addStreamsForLaterStages(stages, in)->getHeader(); } BlockInputStreamPtr MutationsInterpreter::execute() { prepare(/* dry_run = */ false); BlockInputStreamPtr in = interpreter_select->execute().in; - auto result_stream = addStreamsForLaterStages(in); + auto result_stream = addStreamsForLaterStages(stages, in); if (!updated_header) updated_header = std::make_unique(result_stream->getHeader()); return result_stream; diff --git a/dbms/src/Interpreters/MutationsInterpreter.h b/dbms/src/Interpreters/MutationsInterpreter.h index 9880b07e686a..268e5f4b081b 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.h +++ b/dbms/src/Interpreters/MutationsInterpreter.h @@ -41,8 +41,8 @@ class MutationsInterpreter struct Stage; - std::unique_ptr prepareInterpreterSelect(bool dry_run); - BlockInputStreamPtr addStreamsForLaterStages(BlockInputStreamPtr in) const; + std::unique_ptr prepareInterpreterSelect(std::vector & prepared_stages, bool dry_run); + BlockInputStreamPtr addStreamsForLaterStages(const std::vector & prepared_stages, BlockInputStreamPtr in) const; private: StoragePtr storage; From cfe3112bf2ba3c971a18a05e5011ea4931a2a1aa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2019 16:46:15 +0300 Subject: [PATCH 045/161] Enabled SIMDJSON for SSE 4.2 with PCLMUL instruction set --- dbms/src/Functions/FunctionsJSON.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsJSON.h b/dbms/src/Functions/FunctionsJSON.h index a1826b22d308..a0aad24d0b93 100644 --- a/dbms/src/Functions/FunctionsJSON.h +++ b/dbms/src/Functions/FunctionsJSON.h @@ -64,7 +64,7 @@ class FunctionJSON : public IFunction { /// Choose JSONParser. #if USE_SIMDJSON - if (context.getSettings().allow_simdjson && Cpu::CpuFlagsCache::have_AVX2) + if (context.getSettings().allow_simdjson && Cpu::CpuFlagsCache::have_SSE42 && Cpu::CpuFlagsCache::have_PCLMUL) { Executor::run(block, arguments, result_pos, input_rows_count); return; From 0154cdf13b0a703eec3bea354344f4e88474c749 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 7 Aug 2019 17:08:15 +0300 Subject: [PATCH 046/161] Remove checks for AVX2 from cmake lists which are no longer needed. --- cmake/find_simdjson.cmake | 7 +++++-- cmake/test_cpu.cmake | 11 +++++++++++ contrib/simdjson-cmake/CMakeLists.txt | 4 ---- 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/cmake/find_simdjson.cmake b/cmake/find_simdjson.cmake index a556fa5f2b29..2a9f50a88598 100644 --- a/cmake/find_simdjson.cmake +++ b/cmake/find_simdjson.cmake @@ -3,8 +3,11 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/simdjson/include/simdjson/jsonp return() endif () -if (NOT HAVE_AVX2) - message (WARNING "submodule contrib/simdjson requires AVX2 support") +if (NOT HAVE_SSE42) + message (WARNING "submodule contrib/simdjson requires support of SSE4.2 instructions") + return() +elseif (NOT HAVE_PCLMULQDQ) + message (WARNING "submodule contrib/simdjson requires support of PCLMULQDQ instructions") return() endif () diff --git a/cmake/test_cpu.cmake b/cmake/test_cpu.cmake index 42f9bac936ed..d12eac2e3c41 100644 --- a/cmake/test_cpu.cmake +++ b/cmake/test_cpu.cmake @@ -81,6 +81,17 @@ check_cxx_source_compiles(" } " HAVE_AVX2) +set (TEST_FLAG "-mpclmul") +set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") +check_cxx_source_compiles(" + #include + int main() { + auto a = _mm_clmulepi64_si128(__m128i(), __m128i(), 0); + (void)a; + return 0; + } +" HAVE_PCLMULQDQ) + # gcc -dM -E -mpopcnt - < /dev/null | sort > gcc-dump-popcnt #define __POPCNT__ 1 diff --git a/contrib/simdjson-cmake/CMakeLists.txt b/contrib/simdjson-cmake/CMakeLists.txt index bbb2d8e389f4..747b85e6a94e 100644 --- a/contrib/simdjson-cmake/CMakeLists.txt +++ b/contrib/simdjson-cmake/CMakeLists.txt @@ -1,6 +1,3 @@ -if (NOT HAVE_AVX2) - message (FATAL_ERROR "No AVX2 support") -endif () set(SIMDJSON_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/simdjson/include") set(SIMDJSON_SRC_DIR "${SIMDJSON_INCLUDE_DIR}/../src") set(SIMDJSON_SRC @@ -16,4 +13,3 @@ set(SIMDJSON_SRC add_library(${SIMDJSON_LIBRARY} ${SIMDJSON_SRC}) target_include_directories(${SIMDJSON_LIBRARY} SYSTEM PUBLIC "${SIMDJSON_INCLUDE_DIR}") -target_compile_options(${SIMDJSON_LIBRARY} PRIVATE -mavx2 -mbmi -mbmi2 -mpclmul) From d8684bef2801c528895db0545694241750502d00 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 9 Aug 2019 21:43:19 +0300 Subject: [PATCH 047/161] Unfork simdjson (they accepted our changes). --- .gitmodules | 2 +- contrib/simdjson | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index bb8c730baa91..847abf7d9313 100644 --- a/.gitmodules +++ b/.gitmodules @@ -93,7 +93,7 @@ url = https://github.com/ClickHouse-Extras/libunwind.git [submodule "contrib/simdjson"] path = contrib/simdjson - url = https://github.com/ClickHouse-Extras/simdjson.git + url = https://github.com/lemire/simdjson.git [submodule "contrib/rapidjson"] path = contrib/rapidjson url = https://github.com/Tencent/rapidjson diff --git a/contrib/simdjson b/contrib/simdjson index e3f6322af762..9dfab9d9a4c1 160000 --- a/contrib/simdjson +++ b/contrib/simdjson @@ -1 +1 @@ -Subproject commit e3f6322af762213ff2087ce3366bf9541c7fd355 +Subproject commit 9dfab9d9a4c111690a101ea0a7506a2b2f3fa414 From 82fc0f6b15b77e749d6b1b91d9cdd3383ff9aea0 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sat, 10 Aug 2019 16:52:01 +0300 Subject: [PATCH 048/161] docs --- docs/en/query_language/alter.md | 15 +++++++++++++++ docs/ru/query_language/alter.md | 15 +++++++++++++++ 2 files changed, 30 insertions(+) diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index 6e8e712ff30b..92bd44fc6994 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -175,6 +175,7 @@ The following operations with [partitions](../operations/table_engines/custom_pa - [ATTACH PART|PARTITION](#alter_attach-partition) – Adds a part or partition from the `detached` directory to the table. - [REPLACE PARTITION](#alter_replace-partition) - Copies the data partition from one table to another. - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) - Resets the value of a specified column in a partition. +- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - Resets the secondary index in a partition. - [FREEZE PARTITION](#alter_freeze-partition) – Creates a backup of a partition. - [FETCH PARTITION](#alter_fetch-partition) – Downloads a partition from another server. @@ -292,6 +293,14 @@ Restoring from a backup doesn't require stopping the server. For more information about backups and restoring data, see the [Data Backup](../operations/backup.md) section. +#### CLEAR INDEX IN PARTITION {#alter_clear-index-partition} + +``` sql +ALTER TABLE table_name CLEAR INDEX index_name IN PARTITION partition_expr +``` + +The query works similar to `CLEAR COLUMN`, but it resets an index instead of a column. + #### FETCH PARTITION {#alter_fetch-partition} ``` sql @@ -370,6 +379,12 @@ ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr The command is available starting with the 18.12.14 version. The `filter_expr` must be of type UInt8. This query updates values of specified columns to the values of corresponding expressions in rows for which the `filter_expr` takes a non-zero value. Values are casted to the column type using the `CAST` operator. Updating columns that are used in the calculation of the primary or the partition key is not supported. +``` sql +ALTER TABLE [db.]table MATERIALIZE INDEX name IN PARTITION partition_name +``` + +The query recalculates the secondary index `name` in the partition `partition_name`. + One query can contain several commands separated by commas. For *MergeTree tables mutations execute by rewriting whole data parts. There is no atomicity - parts are substituted for mutated parts as soon as they are ready and a `SELECT` query that started executing during a mutation will see data from parts that have already been mutated along with data from parts that have not been mutated yet. diff --git a/docs/ru/query_language/alter.md b/docs/ru/query_language/alter.md index 2367386172ad..fcc0c3dbd51e 100644 --- a/docs/ru/query_language/alter.md +++ b/docs/ru/query_language/alter.md @@ -174,6 +174,7 @@ ALTER TABLE [db].name DROP INDEX name - [ATTACH PARTITION|PART](#alter_attach-partition) – добавить партицию/кусок в таблицу из директории `detached`; - [REPLACE PARTITION](#alter_replace-partition) – скопировать партицию из другой таблицы; - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) – удалить все значения в столбце для заданной партиции; +- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - очистить посчитанные вторичные индексы для заданной партиции; - [FREEZE PARTITION](#alter_freeze-partition) – создать резервную копию партиции; - [FETCH PARTITION](#alter_fetch-partition) – скачать партицию с другого сервера. @@ -257,6 +258,14 @@ ALTER TABLE table_name CLEAR COLUMN column_name IN PARTITION partition_expr ALTER TABLE visits CLEAR COLUMN hour in PARTITION 201902 ``` +#### CLEAR INDEX IN PARTITION {#alter_clear-index-partition} + +``` sql +ALTER TABLE table_name CLEAR INDEX index_name IN PARTITION partition_expr +``` + +Работает как `CLEAR COLUMN`, но сбрасывает индексы вместо столбцов. + #### FREEZE PARTITION {#alter_freeze-partition} ```sql @@ -371,6 +380,12 @@ ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr Команда доступна начиная с версии 18.12.14. Выражение `filter_expr` должно иметь тип UInt8. Запрос изменяет значение указанных столбцов на вычисленное значение соответствующих выражений в каждой строке, для которой `filter_expr` принимает ненулевое значение. Вычисленные значения преобразуются к типу столбца с помощью оператора `CAST`. Изменение столбцов, которые используются при вычислении первичного ключа или ключа партиционирования, не поддерживается. +``` sql +ALTER TABLE [db.]table MATERIALIZE INDEX name IN PARTITION partition_name +``` + +Команда пересчитает вторичный индекс `name` для партиции `partition_name`. + В одном запросе можно указать несколько команд через запятую. Для \*MergeTree-таблиц мутации выполняются, перезаписывая данные по кускам (parts). При этом атомарности нет — куски заменяются на помутированные по мере выполнения и запрос `SELECT`, заданный во время выполнения мутации, увидит данные как из измененных кусков, так и из кусков, которые еще не были изменены. From 22870c58695a55b989d34c7a56208355c56ba545 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 12 Aug 2019 11:47:18 +0300 Subject: [PATCH 049/161] few files --- dbms/src/Interpreters/MetricLog.cpp | 88 +++++++++++++++++++++++++++++ dbms/src/Interpreters/MetricLog.h | 24 ++++++++ dbms/src/Interpreters/SystemLog.cpp | 3 + dbms/src/Interpreters/SystemLog.h | 2 + 4 files changed, 117 insertions(+) create mode 100644 dbms/src/Interpreters/MetricLog.cpp create mode 100644 dbms/src/Interpreters/MetricLog.h diff --git a/dbms/src/Interpreters/MetricLog.cpp b/dbms/src/Interpreters/MetricLog.cpp new file mode 100644 index 000000000000..e968b568c06e --- /dev/null +++ b/dbms/src/Interpreters/MetricLog.cpp @@ -0,0 +1,88 @@ +#include +#include +#include + +namespace DB +{ + +Block MetricLogElement::createBlock() +{ + ColumnsWithTypeAndName columns_with_type_and_name; + //ProfileEvents + for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) + { + UInt64 value = ProfileEvents::global_counters[i]; + + if (0 != value) + { + std::string name; + name += "ProfileEvent_"; + name += ProfileEvents::getName(ProfileEvents::Event(i)); + columns_with_type_and_name.emplace_back(std::make_shared(), name); + } + } + + //CurrentMetrics + for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) + { + UInt64 value = CurrentMetrics::values[i]; + + if (0 != value) + { + std::string name; + name += "CurrentMetric_"; + name += CurrentMetrics::getName(ProfileEvents::Event(i)); + columns_with_type_and_name.emplace_back(std::make_shared(), name); + } + } + + //AsyncMetrics + auto async_metrics_values = async_metrics_ptr.getValues(); + for (const auto & name_value : async_metrics_values) + { + std::string name; + name += "AsynchronousMetrics_"; + name += name_value.first; + columns_with_type_and_name.emplace_back(std::make_shared(), name); + } + + return Block(columns_with_type_and_name); +} + +void MetricLogElement::appendToBlock(Block & block) const +{ + MutableColumns columns = block.mutateColumns(); + + size_t iter = 0; + + //ProfileEvents + for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) + { + UInt64 value = ProfileEvents::global_counters[i]; + + if (0 != value) + { + columns[iter++]->insert(value); + } + } + + //CurrentMetrics + for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) + { + UInt64 value = CurrentMetrics::values[i]; + + if (0 != value) + { + columns[iter++]->insert(value); + } + } + + //AsyncMetrics + auto async_metrics_values = async_metrics_ptr.getValues(); + for (const auto & name_value : async_metrics_values) + { + columns[iter++]->insert(name_value.second); + } +} + +} diff --git a/dbms/src/Interpreters/MetricLog.h b/dbms/src/Interpreters/MetricLog.h new file mode 100644 index 000000000000..ed97578afd5b --- /dev/null +++ b/dbms/src/Interpreters/MetricLog.h @@ -0,0 +1,24 @@ +#pragma once +#include +#include + +namespace DB +{ + +using Poco::Message; + +struct MetricLogElement +{ + std::shared_ptr async_metrics_ptr{nullptr}; + + static std::string name() { return "MetricLog"; } + static Block createBlock(); + void appendToBlock(Block & block) const; +}; + +class MetricLog : public SystemLog +{ + using SystemLog::SystemLog; +}; + +} diff --git a/dbms/src/Interpreters/SystemLog.cpp b/dbms/src/Interpreters/SystemLog.cpp index f1f65dfe8838..a070c43c8830 100644 --- a/dbms/src/Interpreters/SystemLog.cpp +++ b/dbms/src/Interpreters/SystemLog.cpp @@ -48,6 +48,7 @@ SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfi part_log = createSystemLog(global_context, "system", "part_log", config, "part_log"); trace_log = createSystemLog(global_context, "system", "trace_log", config, "trace_log"); text_log = createSystemLog(global_context, "system", "text_log", config, "text_log"); + metric_log = createSystemLog(global_context, "system", "metric_log", config, "metric_log"); part_log_database = config.getString("part_log.database", "system"); } @@ -70,6 +71,8 @@ void SystemLogs::shutdown() trace_log->shutdown(); if (text_log) text_log->shutdown(); + if (metric_log) + metric_log->shutdown(); } } diff --git a/dbms/src/Interpreters/SystemLog.h b/dbms/src/Interpreters/SystemLog.h index 3dd329d577b3..b00f77b76222 100644 --- a/dbms/src/Interpreters/SystemLog.h +++ b/dbms/src/Interpreters/SystemLog.h @@ -61,6 +61,7 @@ class QueryThreadLog; class PartLog; class TextLog; class TraceLog; +class MetricLog; /// System logs should be destroyed in destructor of the last Context and before tables, /// because SystemLog destruction makes insert query while flushing data into underlying tables @@ -76,6 +77,7 @@ struct SystemLogs std::shared_ptr part_log; /// Used to log operations with parts std::shared_ptr trace_log; /// Used to log traces from query profiler std::shared_ptr text_log; /// Used to log all text messages. + std::shared_ptr metric_log; /// Used to log all metrics. String part_log_database; }; From 3f63d41baa8cbf2f70d5d8fdc7f6adf5be4462d1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 12 Aug 2019 16:35:20 +0300 Subject: [PATCH 050/161] submodules --- contrib/simdjson | 2 +- contrib/snappy | 2 +- contrib/thrift | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/contrib/simdjson b/contrib/simdjson index e3f6322af762..3fb82502f7f1 160000 --- a/contrib/simdjson +++ b/contrib/simdjson @@ -1 +1 @@ -Subproject commit e3f6322af762213ff2087ce3366bf9541c7fd355 +Subproject commit 3fb82502f7f19a098006e7ff74c9b6e7c7dd4a84 diff --git a/contrib/snappy b/contrib/snappy index 3f194acb57e0..156cd8939c5f 160000 --- a/contrib/snappy +++ b/contrib/snappy @@ -1 +1 @@ -Subproject commit 3f194acb57e0487531c96b97af61dcbd025a78a3 +Subproject commit 156cd8939c5fba7fa68ae08db843377ecc07b4b5 diff --git a/contrib/thrift b/contrib/thrift index 010ccf0a0c70..74d6d9d3d640 160000 --- a/contrib/thrift +++ b/contrib/thrift @@ -1 +1 @@ -Subproject commit 010ccf0a0c7023fea0f6bf4e4078ebdff7e61982 +Subproject commit 74d6d9d3d6400d1672f48b4acf5bc7d1260ad96d From 3d2e9867cc8c398f21a1222726617d79e1d66d59 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 12 Aug 2019 16:38:30 +0300 Subject: [PATCH 051/161] arrow submodule --- contrib/arrow | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/arrow b/contrib/arrow index 87ac6fddaf21..93688e8c1fa2 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 87ac6fddaf21d0b4ee8b8090533ff293db0da1b4 +Subproject commit 93688e8c1fa2f22d46394c548a9edbd3d2d7c62d From 46a5ec5c16ed274956cc2051d0d2da213d975051 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 12 Aug 2019 17:56:58 +0300 Subject: [PATCH 052/161] some changes --- dbms/src/Interpreters/MetricLog.cpp | 17 ----------------- dbms/src/Interpreters/MetricLog.h | 2 -- dbms/src/Interpreters/SystemLog.cpp | 6 ++++++ dbms/src/Interpreters/SystemLog.h | 2 ++ 4 files changed, 8 insertions(+), 19 deletions(-) diff --git a/dbms/src/Interpreters/MetricLog.cpp b/dbms/src/Interpreters/MetricLog.cpp index e968b568c06e..8f421a65dea9 100644 --- a/dbms/src/Interpreters/MetricLog.cpp +++ b/dbms/src/Interpreters/MetricLog.cpp @@ -36,16 +36,6 @@ Block MetricLogElement::createBlock() } } - //AsyncMetrics - auto async_metrics_values = async_metrics_ptr.getValues(); - for (const auto & name_value : async_metrics_values) - { - std::string name; - name += "AsynchronousMetrics_"; - name += name_value.first; - columns_with_type_and_name.emplace_back(std::make_shared(), name); - } - return Block(columns_with_type_and_name); } @@ -76,13 +66,6 @@ void MetricLogElement::appendToBlock(Block & block) const columns[iter++]->insert(value); } } - - //AsyncMetrics - auto async_metrics_values = async_metrics_ptr.getValues(); - for (const auto & name_value : async_metrics_values) - { - columns[iter++]->insert(name_value.second); - } } } diff --git a/dbms/src/Interpreters/MetricLog.h b/dbms/src/Interpreters/MetricLog.h index ed97578afd5b..5e034463d0ee 100644 --- a/dbms/src/Interpreters/MetricLog.h +++ b/dbms/src/Interpreters/MetricLog.h @@ -9,8 +9,6 @@ using Poco::Message; struct MetricLogElement { - std::shared_ptr async_metrics_ptr{nullptr}; - static std::string name() { return "MetricLog"; } static Block createBlock(); void appendToBlock(Block & block) const; diff --git a/dbms/src/Interpreters/SystemLog.cpp b/dbms/src/Interpreters/SystemLog.cpp index a070c43c8830..b8e8c5d7c1ce 100644 --- a/dbms/src/Interpreters/SystemLog.cpp +++ b/dbms/src/Interpreters/SystemLog.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -75,4 +76,9 @@ void SystemLogs::shutdown() metric_log->shutdown(); } +void SystemLogs::threadFunction() +{ + +} + } diff --git a/dbms/src/Interpreters/SystemLog.h b/dbms/src/Interpreters/SystemLog.h index b00f77b76222..f42febdf090c 100644 --- a/dbms/src/Interpreters/SystemLog.h +++ b/dbms/src/Interpreters/SystemLog.h @@ -79,6 +79,8 @@ struct SystemLogs std::shared_ptr text_log; /// Used to log all text messages. std::shared_ptr metric_log; /// Used to log all metrics. + void threadFunction(); + String part_log_database; }; From 15d2561ac80d2547e7f16624d601d3a1bf954a35 Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 12 Aug 2019 22:27:09 +0300 Subject: [PATCH 053/161] more ExpressionAnalyzer refactoring --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 88 +++++++------------ dbms/src/Interpreters/ExpressionAnalyzer.h | 5 +- dbms/src/Interpreters/GetAggregatesVisitor.h | 59 +++++++++++++ .../Interpreters/InterpreterSelectQuery.cpp | 3 +- 4 files changed, 92 insertions(+), 63 deletions(-) create mode 100644 dbms/src/Interpreters/GetAggregatesVisitor.h diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 9405f7cedeb0..ef40fd5b047f 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -58,6 +58,7 @@ #include #include #include +#include namespace DB { @@ -68,7 +69,6 @@ using LogAST = DebugASTLog; /// set to true to enable logs namespace ErrorCodes { extern const int UNKNOWN_IDENTIFIER; - extern const int ILLEGAL_AGGREGATION; extern const int EXPECTED_ALL_OR_ANY; } @@ -238,20 +238,6 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables() } -void ExpressionAnalyzer::makeSetsForIndex() -{ - const auto * select_query = query->as(); - - if (storage() && select_query && storage()->supportsIndexForIn()) - { - if (select_query->where()) - makeSetsForIndexImpl(select_query->where()); - if (select_query->prewhere()) - makeSetsForIndexImpl(select_query->prewhere()); - } -} - - void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name) { auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name); @@ -277,8 +263,12 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_ } -void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node) +/// Perfomance optimisation for IN() if storage supports it. +void ExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) { + if (!node || !storage() || !storage()->supportsIndexForIn()) + return; + for (auto & child : node->children) { /// Don't descend into subqueries. @@ -290,15 +280,16 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node) if (func && func->name == "lambda") continue; - makeSetsForIndexImpl(child); + makeSetsForIndex(child); } const auto * func = node->as(); if (func && functionIsInOperator(func->name)) { const IAST & args = *func->arguments; + const ASTPtr & left_in_operand = args.children.at(0); - if (storage() && storage()->mayBenefitFromIndexForIn(args.children.at(0), context)) + if (storage()->mayBenefitFromIndexForIn(left_in_operand, context)) { const ASTPtr & arg = args.children.at(1); if (arg->as() || arg->as()) @@ -310,13 +301,13 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node) { NamesAndTypesList temp_columns = sourceColumns(); temp_columns.insert(temp_columns.end(), array_join_columns.begin(), array_join_columns.end()); - for (const auto & joined_column : columnsAddedByJoin()) - temp_columns.push_back(joined_column); + temp_columns.insert(temp_columns.end(), columnsAddedByJoin().begin(), columnsAddedByJoin().end()); + ExpressionActionsPtr temp_actions = std::make_shared(temp_columns, context); - getRootActions(func->arguments->children.at(0), true, temp_actions); + getRootActions(left_in_operand, true, temp_actions); Block sample_block_with_calculated_columns = temp_actions->getSampleBlock(); - if (sample_block_with_calculated_columns.has(args.children.at(0)->getColumnName())) + if (sample_block_with_calculated_columns.has(left_in_operand->getColumnName())) makeExplicitSet(func, sample_block_with_calculated_columns, true, context, settings.size_limits_for_set, prepared_sets); } @@ -356,17 +347,15 @@ void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join, actions = actions_visitor.popActionsLevel(); } +static void assertNoAggregates(const ASTPtr & ast, const char * description) +{ + GetAggregatesVisitor::Data data{true, description, {}, {}}; + GetAggregatesVisitor(data).visit(ast); +} void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions) { - const auto * select_query = query->as(); - - /// There can not be aggregate functions inside the WHERE and PREWHERE. - if (select_query && (ast.get() == select_query->where().get() || ast.get() == select_query->prewhere().get())) - { - assertNoAggregates(ast, "in WHERE or PREWHERE"); - return; - } + auto * select_query = query->as(); /// If we are not analyzing a SELECT query, but a separate expression, then there can not be aggregate functions in it. if (!select_query) @@ -375,18 +364,21 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr return; } - const auto * node = ast->as(); - if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name)) + /// There can not be aggregate functions inside the WHERE and PREWHERE. + if (select_query->where()) + assertNoAggregates(select_query->where(), "in WHERE"); + if (select_query->prewhere()) + assertNoAggregates(select_query->prewhere(), "in PREWHERE"); + + GetAggregatesVisitor::Data data; + GetAggregatesVisitor(data).visit(ast); + has_aggregation = !data.aggregates.empty(); + + for (const ASTFunction * node : data.aggregates) { - has_aggregation = true; AggregateDescription aggregate; aggregate.column_name = node->getColumnName(); - /// Make unique aggregate functions. - for (size_t i = 0; i < aggregate_descriptions.size(); ++i) - if (aggregate_descriptions[i].column_name == aggregate.column_name) - return; - const ASTs & arguments = node->arguments->children; aggregate.argument_names.resize(arguments.size()); DataTypes types(arguments.size()); @@ -407,26 +399,6 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr aggregate_descriptions.push_back(aggregate); } - else - { - for (const auto & child : ast->children) - if (!child->as() && !child->as()) - getAggregates(child, actions); - } -} - - -void ExpressionAnalyzer::assertNoAggregates(const ASTPtr & ast, const char * description) -{ - const auto * node = ast->as(); - - if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name)) - throw Exception("Aggregate function " + node->getColumnName() - + " is found " + String(description) + " in query", ErrorCodes::ILLEGAL_AGGREGATION); - - for (const auto & child : ast->children) - if (!child->as() && !child->as()) - assertNoAggregates(child, description); } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 1959d98ca649..0749c644aee4 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -161,7 +161,7 @@ class ExpressionAnalyzer : private ExpressionAnalyzerData, private boost::noncop const ExpressionAnalyzerData & getAnalyzedData() const { return *this; } /// Create Set-s that we can from IN section to use the index on them. - void makeSetsForIndex(); + void makeSetsForIndex(const ASTPtr & node); bool hasGlobalSubqueries() { return has_global_subqueries; } @@ -200,7 +200,6 @@ class ExpressionAnalyzer : private ExpressionAnalyzerData, private boost::noncop */ void analyzeAggregation(); void getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions); - void assertNoAggregates(const ASTPtr & ast, const char * description); /// columns - the columns that are present before the transformations begin. void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const; @@ -214,8 +213,6 @@ class ExpressionAnalyzer : private ExpressionAnalyzerData, private boost::noncop */ void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name); - void makeSetsForIndexImpl(const ASTPtr & node); - bool isRemoteStorage() const; }; diff --git a/dbms/src/Interpreters/GetAggregatesVisitor.h b/dbms/src/Interpreters/GetAggregatesVisitor.h new file mode 100644 index 000000000000..2c8491020203 --- /dev/null +++ b/dbms/src/Interpreters/GetAggregatesVisitor.h @@ -0,0 +1,59 @@ +#pragma once + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_AGGREGATION; +} + +class GetAggregatesMatcher +{ +public: + using Visitor = ConstInDepthNodeVisitor; + + struct Data + { + bool assert_no_aggregates = false; + const char * description = nullptr; + std::unordered_set uniq_names; + std::vector aggregates; + }; + + static bool needChildVisit(const ASTPtr &, const ASTPtr & child) + { + if (child->as() || child->as()) + return false; + return true; + } + + static void visit(const ASTPtr & ast, Data & data) + { + if (auto * func = ast->as()) + visit(*func, ast, data); + } + +private: + static void visit(const ASTFunction & node, const ASTPtr &, Data & data) + { + if (!AggregateFunctionFactory::instance().isAggregateFunctionName(node.name)) + return; + + if (data.assert_no_aggregates) + throw Exception("Aggregate function " + node.getColumnName() + " is found " + String(data.description) + " in query", + ErrorCodes::ILLEGAL_AGGREGATION); + + String column_name = node.getColumnName(); + if (data.uniq_names.count(column_name)) + return; + + data.aggregates.push_back(&node); + } +}; + +using GetAggregatesVisitor = GetAggregatesMatcher::Visitor; + +} diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 899686ced962..db65c8a95b10 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -790,7 +790,8 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if (!dry_run) from_stage = storage->getQueryProcessingStage(context); - query_analyzer->makeSetsForIndex(); + query_analyzer->makeSetsForIndex(query.where()); + query_analyzer->makeSetsForIndex(query.prewhere()); auto optimize_prewhere = [&](auto & merge_tree) { From bdd79653be348a4e75b25ff648671457855409c2 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev <31595000+nikvas0@users.noreply.github.com> Date: Mon, 12 Aug 2019 22:33:50 +0300 Subject: [PATCH 054/161] Update docs/ru/query_language/alter.md Co-Authored-By: Ivan Blinkov --- docs/ru/query_language/alter.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/query_language/alter.md b/docs/ru/query_language/alter.md index fcc0c3dbd51e..a73a6851f818 100644 --- a/docs/ru/query_language/alter.md +++ b/docs/ru/query_language/alter.md @@ -384,7 +384,7 @@ ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr ALTER TABLE [db.]table MATERIALIZE INDEX name IN PARTITION partition_name ``` -Команда пересчитает вторичный индекс `name` для партиции `partition_name`. +Команда перестроит вторичный индекс `name` для партиции `partition_name`. В одном запросе можно указать несколько команд через запятую. From 0df2f4a79c33a4bdb46ff8c16bee825a8ac1ed0c Mon Sep 17 00:00:00 2001 From: Nikita Vasilev <31595000+nikvas0@users.noreply.github.com> Date: Mon, 12 Aug 2019 22:34:03 +0300 Subject: [PATCH 055/161] Update docs/ru/query_language/alter.md Co-Authored-By: Ivan Blinkov --- docs/ru/query_language/alter.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/query_language/alter.md b/docs/ru/query_language/alter.md index a73a6851f818..1d0c7451e162 100644 --- a/docs/ru/query_language/alter.md +++ b/docs/ru/query_language/alter.md @@ -264,7 +264,7 @@ ALTER TABLE visits CLEAR COLUMN hour in PARTITION 201902 ALTER TABLE table_name CLEAR INDEX index_name IN PARTITION partition_expr ``` -Работает как `CLEAR COLUMN`, но сбрасывает индексы вместо столбцов. +Работает как `CLEAR COLUMN`, но сбрасывает индексы вместо данных в столбцах. #### FREEZE PARTITION {#alter_freeze-partition} From a32f68fa40056cb3b16eaf5c23cb6f3c4442a3b2 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev <31595000+nikvas0@users.noreply.github.com> Date: Mon, 12 Aug 2019 22:34:13 +0300 Subject: [PATCH 056/161] Update docs/ru/query_language/alter.md Co-Authored-By: Ivan Blinkov --- docs/ru/query_language/alter.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/query_language/alter.md b/docs/ru/query_language/alter.md index 1d0c7451e162..a172134fd165 100644 --- a/docs/ru/query_language/alter.md +++ b/docs/ru/query_language/alter.md @@ -174,7 +174,7 @@ ALTER TABLE [db].name DROP INDEX name - [ATTACH PARTITION|PART](#alter_attach-partition) – добавить партицию/кусок в таблицу из директории `detached`; - [REPLACE PARTITION](#alter_replace-partition) – скопировать партицию из другой таблицы; - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) – удалить все значения в столбце для заданной партиции; -- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - очистить посчитанные вторичные индексы для заданной партиции; +- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - очистить построенные вторичные индексы для заданной партиции; - [FREEZE PARTITION](#alter_freeze-partition) – создать резервную копию партиции; - [FETCH PARTITION](#alter_fetch-partition) – скачать партицию с другого сервера. From fc913b6c4f9ecc56feeb2de9394ccdbfae19384f Mon Sep 17 00:00:00 2001 From: Nikita Vasilev <31595000+nikvas0@users.noreply.github.com> Date: Mon, 12 Aug 2019 22:34:27 +0300 Subject: [PATCH 057/161] Update docs/en/query_language/alter.md Co-Authored-By: Ivan Blinkov --- docs/en/query_language/alter.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index 92bd44fc6994..d55cbb4139f9 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -383,7 +383,7 @@ The command is available starting with the 18.12.14 version. The `filter_expr` m ALTER TABLE [db.]table MATERIALIZE INDEX name IN PARTITION partition_name ``` -The query recalculates the secondary index `name` in the partition `partition_name`. +The query rebuilds the secondary index `name` in the partition `partition_name`. One query can contain several commands separated by commas. From 5afa7b5148bae332c825f57212626cd0263f719c Mon Sep 17 00:00:00 2001 From: Nikita Vasilev <31595000+nikvas0@users.noreply.github.com> Date: Mon, 12 Aug 2019 22:34:40 +0300 Subject: [PATCH 058/161] Update docs/en/query_language/alter.md Co-Authored-By: Ivan Blinkov --- docs/en/query_language/alter.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index d55cbb4139f9..eaed7fbbd192 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -175,7 +175,7 @@ The following operations with [partitions](../operations/table_engines/custom_pa - [ATTACH PART|PARTITION](#alter_attach-partition) – Adds a part or partition from the `detached` directory to the table. - [REPLACE PARTITION](#alter_replace-partition) - Copies the data partition from one table to another. - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) - Resets the value of a specified column in a partition. -- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - Resets the secondary index in a partition. +- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - Resets the specified secondary index in a partition. - [FREEZE PARTITION](#alter_freeze-partition) – Creates a backup of a partition. - [FETCH PARTITION](#alter_fetch-partition) – Downloads a partition from another server. From cf2dda0419e2ceedb86efd4532209152160b5e8a Mon Sep 17 00:00:00 2001 From: Nikita Vasilev <31595000+nikvas0@users.noreply.github.com> Date: Mon, 12 Aug 2019 22:34:53 +0300 Subject: [PATCH 059/161] Update docs/en/query_language/alter.md Co-Authored-By: Ivan Blinkov --- docs/en/query_language/alter.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index eaed7fbbd192..102c65528e2d 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -299,7 +299,7 @@ For more information about backups and restoring data, see the [Data Backup](../ ALTER TABLE table_name CLEAR INDEX index_name IN PARTITION partition_expr ``` -The query works similar to `CLEAR COLUMN`, but it resets an index instead of a column. +The query works similar to `CLEAR COLUMN`, but it resets an index instead of a column data. #### FETCH PARTITION {#alter_fetch-partition} From f2a03d9d515ac9a20efbdb393006a70311c8aa19 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 13 Aug 2019 03:25:51 +0300 Subject: [PATCH 060/161] several fixes --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 13 +++++-------- dbms/src/Interpreters/GetAggregatesVisitor.h | 20 ++++++++++++++++++-- 2 files changed, 23 insertions(+), 10 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index ef40fd5b047f..cb2fbaf33bb8 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -347,15 +347,9 @@ void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join, actions = actions_visitor.popActionsLevel(); } -static void assertNoAggregates(const ASTPtr & ast, const char * description) -{ - GetAggregatesVisitor::Data data{true, description, {}, {}}; - GetAggregatesVisitor(data).visit(ast); -} - void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions) { - auto * select_query = query->as(); + const auto * select_query = query->as(); /// If we are not analyzing a SELECT query, but a separate expression, then there can not be aggregate functions in it. if (!select_query) @@ -371,11 +365,14 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr assertNoAggregates(select_query->prewhere(), "in PREWHERE"); GetAggregatesVisitor::Data data; + /// @warning It's not clear that aggregate_descriptions could be not empty here. But they do. + for (const AggregateDescription & agg : aggregate_descriptions) + data.uniq_names.insert(agg.column_name); GetAggregatesVisitor(data).visit(ast); - has_aggregation = !data.aggregates.empty(); for (const ASTFunction * node : data.aggregates) { + has_aggregation = true; AggregateDescription aggregate; aggregate.column_name = node->getColumnName(); diff --git a/dbms/src/Interpreters/GetAggregatesVisitor.h b/dbms/src/Interpreters/GetAggregatesVisitor.h index 2c8491020203..fbd0b111ad16 100644 --- a/dbms/src/Interpreters/GetAggregatesVisitor.h +++ b/dbms/src/Interpreters/GetAggregatesVisitor.h @@ -23,10 +23,13 @@ class GetAggregatesMatcher std::vector aggregates; }; - static bool needChildVisit(const ASTPtr &, const ASTPtr & child) + static bool needChildVisit(const ASTPtr & node, const ASTPtr & child) { if (child->as() || child->as()) return false; + if (auto * func = node->as()) + if (isAggregateFunction(func->name)) + return false; return true; } @@ -39,7 +42,7 @@ class GetAggregatesMatcher private: static void visit(const ASTFunction & node, const ASTPtr &, Data & data) { - if (!AggregateFunctionFactory::instance().isAggregateFunctionName(node.name)) + if (!isAggregateFunction(node.name)) return; if (data.assert_no_aggregates) @@ -50,10 +53,23 @@ class GetAggregatesMatcher if (data.uniq_names.count(column_name)) return; + data.uniq_names.insert(column_name); data.aggregates.push_back(&node); } + + static bool isAggregateFunction(const String & name) + { + return AggregateFunctionFactory::instance().isAggregateFunctionName(name); + } }; using GetAggregatesVisitor = GetAggregatesMatcher::Visitor; + +inline void assertNoAggregates(const ASTPtr & ast, const char * description) +{ + GetAggregatesVisitor::Data data{true, description, {}, {}}; + GetAggregatesVisitor(data).visit(ast); +} + } From 3d00e4b374f87536deca4dfb5d85c31f8a030ec0 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 13 Aug 2019 13:56:12 +0300 Subject: [PATCH 061/161] remove wrong logic --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index cb2fbaf33bb8..edd3b44943dd 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -365,9 +365,6 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr assertNoAggregates(select_query->prewhere(), "in PREWHERE"); GetAggregatesVisitor::Data data; - /// @warning It's not clear that aggregate_descriptions could be not empty here. But they do. - for (const AggregateDescription & agg : aggregate_descriptions) - data.uniq_names.insert(agg.column_name); GetAggregatesVisitor(data).visit(ast); for (const ASTFunction * node : data.aggregates) From 1edc2a264729e54bf791d5a092e5b472dc1d807a Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 13 Aug 2019 14:02:26 +0300 Subject: [PATCH 062/161] Add link to Hong Kong Meetup --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 8f25b2156b62..dccb75b42827 100644 --- a/README.md +++ b/README.md @@ -13,7 +13,7 @@ ClickHouse is an open-source column-oriented database management system that all * You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person. ## Upcoming Events -* [ClickHouse Meetup in Mountain View](https://www.eventbrite.com/e/meetup-clickhouse-in-the-south-bay-registration-65935505873) on August 13. * [ClickHouse Meetup in Moscow](https://yandex.ru/promo/clickhouse/moscow-2019) on September 5. +* [ClickHouse Meetup in Hong Kong](https://www.meetup.com/Hong-Kong-Machine-Learning-Meetup/events/263580542/) on October 17. * [ClickHouse Meetup in Shenzhen](https://www.huodongxing.com/event/3483759917300) on October 20. * [ClickHouse Meetup in Shanghai](https://www.huodongxing.com/event/4483760336000) on October 27. From 684580e3311c834eb1b2d899c6a98df057b76208 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 13 Aug 2019 14:04:23 +0300 Subject: [PATCH 063/161] Add link to Hong Kong Meetup to website front page (#6465) --- website/index.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/index.html b/website/index.html index 26090e5d857d..017080d56479 100644 --- a/website/index.html +++ b/website/index.html @@ -94,7 +94,7 @@

- Upcoming Meetups: Mountain View on August 13, Moscow on September 5, Shenzhen on October 20 and Shanghai on October 27 + Upcoming Meetups: Moscow on September 5, Hong Kong on October 17, Shenzhen on October 20 and Shanghai on October 27
From 4b9ae60bd12ed7124ab52f9beac91d7e8627ab88 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 13 Aug 2019 14:25:37 +0300 Subject: [PATCH 064/161] Check bad URIs in Poco library --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 7a2d304c2154..6216cc01a107 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 7a2d304c21549427460428c9039009ef4bbfd899 +Subproject commit 6216cc01a107ce149863411ca29013a224f80343 From 10a199185519a877ed816765348d90175c3162f1 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 13 Aug 2019 15:39:03 +0300 Subject: [PATCH 065/161] move getAggregates() into SyntaxAnalyzer --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 63 ++++++-------------- dbms/src/Interpreters/ExpressionAnalyzer.h | 5 +- dbms/src/Interpreters/GetAggregatesVisitor.h | 8 +-- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 26 ++++++++ dbms/src/Interpreters/SyntaxAnalyzer.h | 3 + 5 files changed, 54 insertions(+), 51 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index edd3b44943dd..f47a5eb30229 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -116,9 +116,6 @@ void ExpressionAnalyzer::analyzeAggregation() auto * select_query = query->as(); - if (select_query && (select_query->groupBy() || select_query->having())) - has_aggregation = true; - ExpressionActionsPtr temp_actions = std::make_shared(sourceColumns(), context); if (select_query) @@ -150,7 +147,9 @@ void ExpressionAnalyzer::analyzeAggregation() } } - getAggregates(query, temp_actions); + has_aggregation = makeAggregateDescriptions(temp_actions); + if (select_query && (select_query->groupBy() || select_query->having())) + has_aggregation = true; if (has_aggregation) { @@ -347,29 +346,10 @@ void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join, actions = actions_visitor.popActionsLevel(); } -void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions) +bool ExpressionAnalyzer::makeAggregateDescriptions(ExpressionActionsPtr & actions) { - const auto * select_query = query->as(); - - /// If we are not analyzing a SELECT query, but a separate expression, then there can not be aggregate functions in it. - if (!select_query) - { - assertNoAggregates(ast, "in wrong place"); - return; - } - - /// There can not be aggregate functions inside the WHERE and PREWHERE. - if (select_query->where()) - assertNoAggregates(select_query->where(), "in WHERE"); - if (select_query->prewhere()) - assertNoAggregates(select_query->prewhere(), "in PREWHERE"); - - GetAggregatesVisitor::Data data; - GetAggregatesVisitor(data).visit(ast); - - for (const ASTFunction * node : data.aggregates) + for (const ASTFunction * node : aggregates()) { - has_aggregation = true; AggregateDescription aggregate; aggregate.column_name = node->getColumnName(); @@ -379,9 +359,6 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr for (size_t i = 0; i < arguments.size(); ++i) { - /// There can not be other aggregate functions within the aggregate functions. - assertNoAggregates(arguments[i], "inside another aggregate function"); - getRootActions(arguments[i], true, actions); const std::string & name = arguments[i]->getColumnName(); types[i] = actions->getSampleBlock().getByName(name).type; @@ -393,6 +370,8 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr aggregate_descriptions.push_back(aggregate); } + + return !aggregates().empty(); } @@ -724,13 +703,22 @@ void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChai } } - getActionsBeforeAggregation(select_query->select(), step.actions, only_types); + /// Collect aggregates removing duplicates by node.getColumnName() + /// It's not clear why we recollect aggregates (for query parts) while we're able to use previously collected ones (for entire query) + /// @note The original recollection logic didn't remove duplicates. + GetAggregatesVisitor::Data data; + GetAggregatesVisitor(data).visit(select_query->select()); if (select_query->having()) - getActionsBeforeAggregation(select_query->having(), step.actions, only_types); + GetAggregatesVisitor(data).visit(select_query->having()); if (select_query->orderBy()) - getActionsBeforeAggregation(select_query->orderBy(), step.actions, only_types); + GetAggregatesVisitor(data).visit(select_query->orderBy()); + + /// TODO: data.aggregates -> aggregates() + for (const ASTFunction * node : data.aggregates) + for (auto & argument : node->arguments->children) + getRootActions(argument, only_types, step.actions); } bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types) @@ -858,19 +846,6 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const } -void ExpressionAnalyzer::getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries) -{ - const auto * node = ast->as(); - - if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name)) - for (auto & argument : node->arguments->children) - getRootActions(argument, no_subqueries, actions); - else - for (auto & child : ast->children) - getActionsBeforeAggregation(child, actions, no_subqueries); -} - - ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result) { ExpressionActionsPtr actions = std::make_shared(sourceColumns(), context); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 0749c644aee4..c21ec500b923 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -178,6 +178,7 @@ class ExpressionAnalyzer : private ExpressionAnalyzerData, private boost::noncop const AnalyzedJoin & analyzedJoin() const { return syntax->analyzed_join; } const NamesAndTypesList & sourceColumns() const { return syntax->required_source_columns; } const NamesAndTypesList & columnsAddedByJoin() const { return syntax->columns_added_by_join; } + const std::vector & aggregates() const { return syntax->aggregates; } /// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables. void initGlobalSubqueriesAndExternalTables(); @@ -191,15 +192,13 @@ class ExpressionAnalyzer : private ExpressionAnalyzerData, private boost::noncop void getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false); - void getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries); - /** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions, * Create a set of columns aggregated_columns resulting after the aggregation, if any, * or after all the actions that are normally performed before aggregation. * Set has_aggregation = true if there is GROUP BY or at least one aggregate function. */ void analyzeAggregation(); - void getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions); + bool makeAggregateDescriptions(ExpressionActionsPtr & actions); /// columns - the columns that are present before the transformations begin. void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const; diff --git a/dbms/src/Interpreters/GetAggregatesVisitor.h b/dbms/src/Interpreters/GetAggregatesVisitor.h index fbd0b111ad16..ba1501fc6245 100644 --- a/dbms/src/Interpreters/GetAggregatesVisitor.h +++ b/dbms/src/Interpreters/GetAggregatesVisitor.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -17,8 +18,7 @@ class GetAggregatesMatcher struct Data { - bool assert_no_aggregates = false; - const char * description = nullptr; + const char * assert_no_aggregates = nullptr; std::unordered_set uniq_names; std::vector aggregates; }; @@ -46,7 +46,7 @@ class GetAggregatesMatcher return; if (data.assert_no_aggregates) - throw Exception("Aggregate function " + node.getColumnName() + " is found " + String(data.description) + " in query", + throw Exception("Aggregate function " + node.getColumnName() + " is found " + String(data.assert_no_aggregates) + " in query", ErrorCodes::ILLEGAL_AGGREGATION); String column_name = node.getColumnName(); @@ -68,7 +68,7 @@ using GetAggregatesVisitor = GetAggregatesMatcher::Visitor; inline void assertNoAggregates(const ASTPtr & ast, const char * description) { - GetAggregatesVisitor::Data data{true, description, {}, {}}; + GetAggregatesVisitor::Data data{description, {}, {}}; GetAggregatesVisitor(data).visit(ast); } diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 282b19991b17..77f259820a7b 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -558,6 +559,30 @@ void checkJoin(const ASTTablesInSelectQueryElement * join) ErrorCodes::NOT_IMPLEMENTED); } +std::vector getAggregates(const ASTPtr & query) +{ + if (const auto * select_query = query->as()) + { + /// There can not be aggregate functions inside the WHERE and PREWHERE. + if (select_query->where()) + assertNoAggregates(select_query->where(), "in WHERE"); + if (select_query->prewhere()) + assertNoAggregates(select_query->prewhere(), "in PREWHERE"); + + GetAggregatesVisitor::Data data; + GetAggregatesVisitor(data).visit(query); + + /// There can not be other aggregate functions within the aggregate functions. + for (const ASTFunction * node : data.aggregates) + for (auto & arg : node->arguments->children) + assertNoAggregates(arg, "inside another aggregate function"); + return data.aggregates; + } + else + assertNoAggregates(query, "in wrong place"); + return {}; +} + } /// Calculate which columns are required to execute the expression. @@ -840,6 +865,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( collectJoinedColumns(result.analyzed_join, *select_query, source_columns_set, result.aliases, settings.join_use_nulls); } + result.aggregates = getAggregates(query); result.collectUsedColumns(query, additional_source_columns); return std::make_shared(result); } diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.h b/dbms/src/Interpreters/SyntaxAnalyzer.h index c7addb03526f..a31dfef7e824 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.h +++ b/dbms/src/Interpreters/SyntaxAnalyzer.h @@ -10,6 +10,8 @@ namespace DB NameSet removeDuplicateColumns(NamesAndTypesList & columns); +class ASTFunction; + struct SyntaxAnalyzerResult { StoragePtr storage; @@ -22,6 +24,7 @@ struct SyntaxAnalyzerResult NamesAndTypesList columns_added_by_join; Aliases aliases; + std::vector aggregates; /// Which column is needed to be ARRAY-JOIN'ed to get the specified. /// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v". From 87fbc2a5cb67c8cc760b267ace743d76e6f92151 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 13 Aug 2019 16:48:09 +0300 Subject: [PATCH 066/161] minor improvements --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 64 +++++++------------- dbms/src/Interpreters/ExpressionAnalyzer.h | 16 ++--- 2 files changed, 27 insertions(+), 53 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index f47a5eb30229..edd26dd3bd19 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -78,16 +78,15 @@ ExpressionAnalyzer::ExpressionAnalyzer( const Context & context_, const NameSet & required_result_columns_, size_t subquery_depth_, - bool do_global_, - const SubqueriesForSets & subqueries_for_sets_) - : ExpressionAnalyzerData(required_result_columns_, subqueries_for_sets_) + bool do_global) + : ExpressionAnalyzerData(required_result_columns_) , query(query_), context(context_), settings(context.getSettings()) - , subquery_depth(subquery_depth_), do_global(do_global_) + , subquery_depth(subquery_depth_) , syntax(syntax_analyzer_result_) { /// external_tables, subqueries_for_sets for global subqueries. /// Replaces global subqueries with the generated names of temporary tables that will be sent to remote servers. - initGlobalSubqueriesAndExternalTables(); + initGlobalSubqueriesAndExternalTables(do_global); /// has_aggregation, aggregation_keys, aggregate_descriptions, aggregated_columns. /// This analysis should be performed after processing global subqueries, because otherwise, @@ -153,7 +152,7 @@ void ExpressionAnalyzer::analyzeAggregation() if (has_aggregation) { - assertSelect(); + getSelectQuery(); /// assertSelect() /// Find out aggregation keys. if (select_query->groupBy()) @@ -222,7 +221,7 @@ void ExpressionAnalyzer::analyzeAggregation() } -void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables() +void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global) { /// Adds existing external tables (not subqueries) to the external_tables dictionary. ExternalTablesVisitor::Data tables_data{context, external_tables}; @@ -375,18 +374,19 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ExpressionActionsPtr & action } -void ExpressionAnalyzer::assertSelect() const +const ASTSelectQuery * ExpressionAnalyzer::getSelectQuery() const { const auto * select_query = query->as(); - if (!select_query) throw Exception("Not a select query", ErrorCodes::LOGICAL_ERROR); + return select_query; } -void ExpressionAnalyzer::assertAggregation() const +const ASTSelectQuery * ExpressionAnalyzer::getAggregatingQuery() const { if (!has_aggregation) throw Exception("No aggregation", ErrorCodes::LOGICAL_ERROR); + return getSelectQuery(); } void ExpressionAnalyzer::initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const @@ -416,9 +416,7 @@ void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actio bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types) { - const auto * select_query = query->as(); - - assertSelect(); + const auto * select_query = getSelectQuery(); bool is_array_join_left; ASTPtr array_join_expression_list = select_query->array_join_expression_list(is_array_join_left); @@ -460,9 +458,7 @@ static void appendRequiredColumns( bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types) { - const auto * select_query = query->as(); - - assertSelect(); + const auto * select_query = getSelectQuery(); if (!select_query->join()) return false; @@ -571,9 +567,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty bool ExpressionAnalyzer::appendPrewhere( ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns) { - const auto * select_query = query->as(); - - assertSelect(); + const auto * select_query = getSelectQuery(); if (!select_query->prewhere()) return false; @@ -646,9 +640,7 @@ bool ExpressionAnalyzer::appendPrewhere( bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_types) { - const auto * select_query = query->as(); - - assertSelect(); + const auto * select_query = getSelectQuery(); if (!select_query->where()) return false; @@ -666,9 +658,7 @@ bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_t bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only_types) { - const auto * select_query = query->as(); - - assertAggregation(); + const auto * select_query = getAggregatingQuery(); if (!select_query->groupBy()) return false; @@ -688,9 +678,7 @@ bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types) { - const auto * select_query = query->as(); - - assertAggregation(); + const auto * select_query = getAggregatingQuery(); initChain(chain, sourceColumns()); ExpressionActionsChain::Step & step = chain.steps.back(); @@ -723,9 +711,7 @@ void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChai bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types) { - const auto * select_query = query->as(); - - assertAggregation(); + const auto * select_query = getAggregatingQuery(); if (!select_query->having()) return false; @@ -741,9 +727,7 @@ bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_ void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_types) { - const auto * select_query = query->as(); - - assertSelect(); + const auto * select_query = getSelectQuery(); initChain(chain, aggregated_columns); ExpressionActionsChain::Step & step = chain.steps.back(); @@ -756,9 +740,7 @@ void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_ bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types) { - const auto * select_query = query->as(); - - assertSelect(); + const auto * select_query = getSelectQuery(); if (!select_query->orderBy()) return false; @@ -782,9 +764,7 @@ bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only_types) { - const auto * select_query = query->as(); - - assertSelect(); + const auto * select_query = getSelectQuery(); if (!select_query->limitBy()) return false; @@ -813,9 +793,7 @@ bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only void ExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const { - const auto * select_query = query->as(); - - assertSelect(); + const auto * select_query = getSelectQuery(); initChain(chain, aggregated_columns); ExpressionActionsChain::Step & step = chain.steps.back(); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index c21ec500b923..5a530add810c 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -51,10 +51,8 @@ struct ExpressionAnalyzerData Tables external_tables; protected: - ExpressionAnalyzerData(const NameSet & required_result_columns_, - const SubqueriesForSets & subqueries_for_sets_) - : required_result_columns(required_result_columns_), - subqueries_for_sets(subqueries_for_sets_) + ExpressionAnalyzerData(const NameSet & required_result_columns_) + : required_result_columns(required_result_columns_) {} }; @@ -91,8 +89,7 @@ class ExpressionAnalyzer : private ExpressionAnalyzerData, private boost::noncop const Context & context_, const NameSet & required_result_columns_ = {}, size_t subquery_depth_ = 0, - bool do_global_ = false, - const SubqueriesForSets & subqueries_for_set_ = {}); + bool do_global_ = false); /// Does the expression have aggregate functions or a GROUP BY or HAVING section. bool hasAggregation() const { return has_aggregation; } @@ -170,7 +167,6 @@ class ExpressionAnalyzer : private ExpressionAnalyzerData, private boost::noncop const Context & context; const ExtractedSettings settings; size_t subquery_depth; - bool do_global; /// Do I need to prepare for execution global subqueries when analyzing the query. SyntaxAnalyzerResultPtr syntax; @@ -181,7 +177,7 @@ class ExpressionAnalyzer : private ExpressionAnalyzerData, private boost::noncop const std::vector & aggregates() const { return syntax->aggregates; } /// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables. - void initGlobalSubqueriesAndExternalTables(); + void initGlobalSubqueriesAndExternalTables(bool do_global); void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const; @@ -203,8 +199,8 @@ class ExpressionAnalyzer : private ExpressionAnalyzerData, private boost::noncop /// columns - the columns that are present before the transformations begin. void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const; - void assertSelect() const; - void assertAggregation() const; + const ASTSelectQuery * getSelectQuery() const; + const ASTSelectQuery * getAggregatingQuery() const; /** * Create Set from a subquery or a table expression in the query. The created set is suitable for using the index. From 7ae73befd6de39ab58e5e362efb0bb0d62d02329 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Tue, 13 Aug 2019 17:07:36 +0300 Subject: [PATCH 067/161] wait for finish calculations at underlying streams while cancelling AsynchronousBlockInputStream --- dbms/src/DataStreams/AsynchronousBlockInputStream.h | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index 6cfa247ab448..c741d8090e93 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -73,6 +73,18 @@ class AsynchronousBlockInputStream : public IBlockInputStream Block getHeader() const override { return children.at(0)->getHeader(); } + void cancel(bool kill) override + { + IBlockInputStream::cancel(kill); + + /// Wait for some backgroud calculations to be sure, + /// that after end of stream nothing is being executing. + if (started) + { + pool.wait(); + started = false; + } + } ~AsynchronousBlockInputStream() override { From 8b9284be241460626b0017cc2a7eb7f8eb273281 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Tue, 13 Aug 2019 17:08:28 +0300 Subject: [PATCH 068/161] Fix shared build (#6453) * Fix shared build * Enable no-undefined check only in CI --- CMakeLists.txt | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index d369dca7e786..8466fa5d33d7 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -264,7 +264,9 @@ if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE) set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}") endif () -if (NOT SANITIZE AND NOT SPLIT_SHARED_LIBRARIES) +# TODO: only make this extra-checks in CI builds, since a lot of contrib libs won't link - +# CI works around this problem by explicitly adding GLIBC_COMPATIBILITY flag. +if (NOT SANITIZE AND YANDEX_OFFICIAL_BUILD) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-undefined") set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-undefined") endif () @@ -328,7 +330,7 @@ if (OS_LINUX AND NOT UNBUNDLED AND (GLIBC_COMPATIBILITY OR USE_INTERNAL_UNWIND_L if (USE_INTERNAL_LIBCXX_LIBRARY) set (LIBCXX_LIBS "${ClickHouse_BINARY_DIR}/contrib/libcxx-cmake/libcxx_static${${CMAKE_POSTFIX_VARIABLE}}.a ${ClickHouse_BINARY_DIR}/contrib/libcxxabi-cmake/libcxxabi_static${${CMAKE_POSTFIX_VARIABLE}}.a") else () - set (LIBCXX_LIBS "-lc++ -lc++abi") + set (LIBCXX_LIBS "-lc++ -lc++abi -lc++fs") endif () set (DEFAULT_LIBS "${DEFAULT_LIBS} -Wl,-Bstatic ${LIBCXX_LIBS} ${EXCEPTION_HANDLING_LIBRARY} ${BUILTINS_LIB_PATH} -Wl,-Bdynamic") From bbe20b3f9f083fa04979294c6025796891614eb3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 13 Aug 2019 17:31:46 +0300 Subject: [PATCH 069/161] done --- dbms/src/Interpreters/Context.cpp | 17 +++-- dbms/src/Interpreters/Context.h | 2 + .../Interpreters/InterpreterSystemQuery.cpp | 4 +- dbms/src/Interpreters/MetricLog.cpp | 68 +++++++++++++++++++ dbms/src/Interpreters/MetricLog.h | 23 +++++++ dbms/src/Interpreters/SystemLog.cpp | 39 +++++++++++ dbms/src/Interpreters/SystemLog.h | 10 ++- .../MergeTreeSequentialBlockInputStream.cpp | 4 +- docker/test/stateful/Dockerfile | 1 + docker/test/stateful_with_coverage/run.sh | 1 + docker/test/stateless/Dockerfile | 1 + docker/test/stateless_with_coverage/run.sh | 1 + 12 files changed, 162 insertions(+), 9 deletions(-) create mode 100644 dbms/src/Interpreters/MetricLog.cpp create mode 100644 dbms/src/Interpreters/MetricLog.h diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 83f3763bb11c..ea78df254857 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -38,10 +38,6 @@ #include #include #include -#include -#include -#include -#include #include #include #include @@ -1701,6 +1697,7 @@ std::shared_ptr Context::getPartLog(const String & part_database) return shared->system_logs->part_log; } + std::shared_ptr Context::getTraceLog() { auto lock = getLock(); @@ -1711,6 +1708,7 @@ std::shared_ptr Context::getTraceLog() return shared->system_logs->trace_log; } + std::shared_ptr Context::getTextLog() { auto lock = getLock(); @@ -1722,6 +1720,17 @@ std::shared_ptr Context::getTextLog() } +std::shared_ptr Context::getMetricLog() +{ + auto lock = getLock(); + + if (!shared->system_logs || !shared->system_logs->metric_log) + return {}; + + return shared->system_logs->metric_log; +} + + CompressionCodecPtr Context::chooseCompressionCodec(size_t part_size, double part_size_ratio) const { auto lock = getLock(); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 50b7ab3eba23..2ef123b74b1f 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -64,6 +64,7 @@ class QueryThreadLog; class PartLog; class TextLog; class TraceLog; +class MetricLog; struct MergeTreeSettings; class IDatabase; class DDLGuard; @@ -434,6 +435,7 @@ class Context std::shared_ptr getQueryThreadLog(); std::shared_ptr getTraceLog(); std::shared_ptr getTextLog(); + std::shared_ptr getMetricLog(); /// Returns an object used to log opertaions with parts if it possible. /// Provide table name to make required cheks. diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 818aae6d0488..0797a3deab42 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -232,7 +233,8 @@ BlockIO InterpreterSystemQuery::execute() [&] () { if (auto part_log = context.getPartLog("")) part_log->flush(); }, [&] () { if (auto query_thread_log = context.getQueryThreadLog()) query_thread_log->flush(); }, [&] () { if (auto trace_log = context.getTraceLog()) trace_log->flush(); }, - [&] () { if (auto text_log = context.getTextLog()) text_log->flush(); } + [&] () { if (auto text_log = context.getTextLog()) text_log->flush(); }, + [&] () { if (auto metric_log = context.getMetricLog()) metric_log->flush(); } ); break; case Type::STOP_LISTEN_QUERIES: diff --git a/dbms/src/Interpreters/MetricLog.cpp b/dbms/src/Interpreters/MetricLog.cpp new file mode 100644 index 000000000000..69d9425b17e4 --- /dev/null +++ b/dbms/src/Interpreters/MetricLog.cpp @@ -0,0 +1,68 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +Block MetricLogElement::createBlock() +{ + ColumnsWithTypeAndName columns_with_type_and_name; + + columns_with_type_and_name.emplace_back(std::make_shared(), "event_date"); + columns_with_type_and_name.emplace_back(std::make_shared(), "event_time"); + + //ProfileEvents + for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) + { + std::string name; + name += "ProfileEvent_"; + name += ProfileEvents::getName(ProfileEvents::Event(i)); + columns_with_type_and_name.emplace_back(std::make_shared(), name); + } + + //CurrentMetrics + for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) + { + + std::string name; + name += "CurrentMetric_"; + name += CurrentMetrics::getName(ProfileEvents::Event(i)); + columns_with_type_and_name.emplace_back(std::make_shared(), name); + + } + + return Block(columns_with_type_and_name); +} + +void MetricLogElement::appendToBlock(Block & block) const +{ + MutableColumns columns = block.mutateColumns(); + + size_t iter = 0; + + columns[iter++]->insert(DateLUT::instance().toDayNum(event_time)); + columns[iter++]->insert(event_time); + + //ProfileEvents + for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) + { + UInt64 value = ProfileEvents::global_counters[i]; + + columns[iter++]->insert(value); + + } + + //CurrentMetrics + for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) + { + UInt64 value = CurrentMetrics::values[i]; + + columns[iter++]->insert(value); + + } +} + +} diff --git a/dbms/src/Interpreters/MetricLog.h b/dbms/src/Interpreters/MetricLog.h new file mode 100644 index 000000000000..13112d4934eb --- /dev/null +++ b/dbms/src/Interpreters/MetricLog.h @@ -0,0 +1,23 @@ +#pragma once +#include +#include + +namespace DB +{ + +using Poco::Message; + +struct MetricLogElement +{ + time_t event_time{}; + static std::string name() { return "MetricLog"; } + static Block createBlock(); + void appendToBlock(Block & block) const; +}; + +class MetricLog : public SystemLog +{ + using SystemLog::SystemLog; +}; + +} diff --git a/dbms/src/Interpreters/SystemLog.cpp b/dbms/src/Interpreters/SystemLog.cpp index f1f65dfe8838..6e63951d9ec5 100644 --- a/dbms/src/Interpreters/SystemLog.cpp +++ b/dbms/src/Interpreters/SystemLog.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -48,7 +49,10 @@ SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfi part_log = createSystemLog(global_context, "system", "part_log", config, "part_log"); trace_log = createSystemLog(global_context, "system", "trace_log", config, "trace_log"); text_log = createSystemLog(global_context, "system", "text_log", config, "text_log"); + metric_log = createSystemLog(global_context, "system", "metric_log", config, "metric_log"); + if (metric_log) + metric_flush_thread = ThreadFromGlobalPool([this] { metricThreadFunction(); }); part_log_database = config.getString("part_log.database", "system"); } @@ -70,6 +74,41 @@ void SystemLogs::shutdown() trace_log->shutdown(); if (text_log) text_log->shutdown(); + if (metric_log) + { + bool old_val = false; + if (!is_shutdown_metric_thread.compare_exchange_strong(old_val, true)) + return; + metric_flush_thread.join(); + metric_log->shutdown(); + } +} + +void SystemLogs::metricThreadFunction() +{ + const size_t flush_interval_milliseconds = 1000; + while (true) + { + try + { + const auto prev_timepoint = std::chrono::system_clock::now(); + + if (is_shutdown_metric_thread) + break; + + MetricLogElement elem; + elem.event_time = std::time(nullptr); + metric_log->add(elem); + + const auto next_timepoint = prev_timepoint + std::chrono::milliseconds(flush_interval_milliseconds); + std::this_thread::sleep_until(next_timepoint); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + } } } diff --git a/dbms/src/Interpreters/SystemLog.h b/dbms/src/Interpreters/SystemLog.h index 3dd329d577b3..545519aeb9df 100644 --- a/dbms/src/Interpreters/SystemLog.h +++ b/dbms/src/Interpreters/SystemLog.h @@ -61,6 +61,7 @@ class QueryThreadLog; class PartLog; class TextLog; class TraceLog; +class MetricLog; /// System logs should be destroyed in destructor of the last Context and before tables, /// because SystemLog destruction makes insert query while flushing data into underlying tables @@ -76,6 +77,11 @@ struct SystemLogs std::shared_ptr part_log; /// Used to log operations with parts std::shared_ptr trace_log; /// Used to log traces from query profiler std::shared_ptr text_log; /// Used to log all text messages. + std::shared_ptr metric_log; /// Used to log all metrics. + + ThreadFromGlobalPool metric_flush_thread; + void metricThreadFunction(); + std::atomic is_shutdown_metric_thread{false}; String part_log_database; }; @@ -115,6 +121,8 @@ class SystemLog : private boost::noncopyable /// Stop the background flush thread before destructor. No more data will be written. void shutdown(); + size_t getFlushInterval() { return flush_interval_milliseconds; } + protected: Context & context; const String database_name; @@ -179,7 +187,7 @@ SystemLog::SystemLog(Context & context_, flush_interval_milliseconds(flush_interval_milliseconds_) { log = &Logger::get("SystemLog (" + database_name + "." + table_name + ")"); - +qq data.reserve(DBMS_SYSTEM_LOG_QUEUE_SIZE); saving_thread = ThreadFromGlobalPool([this] { threadFunction(); }); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp index 1c9c3457fe58..74cff479e5f2 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.cpp @@ -27,9 +27,7 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream( std::stringstream message; message << "Reading " << data_part->getMarksCount() << " marks from part " << data_part->name << ", total " << data_part->rows_count - << " rows starting from the beginning of the part, columns: "; - for (size_t i = 0, size = columns_to_read.size(); i < size; ++i) - message << (i == 0 ? "" : ", ") << columns_to_read[i]; + << " rows starting from the beginning of the part"; LOG_TRACE(log, message.rdbuf()); } diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile index 516d63fa3303..cab4ebbe8bb5 100644 --- a/docker/test/stateful/Dockerfile +++ b/docker/test/stateful/Dockerfile @@ -20,6 +20,7 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \ + ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \ diff --git a/docker/test/stateful_with_coverage/run.sh b/docker/test/stateful_with_coverage/run.sh index 6253a07c7454..a6d2ba0e9e8e 100755 --- a/docker/test/stateful_with_coverage/run.sh +++ b/docker/test/stateful_with_coverage/run.sh @@ -46,6 +46,7 @@ ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/con ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \ + ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \ diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index eea48c3c032a..60ab18cd5f23 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -40,6 +40,7 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \ + ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \ diff --git a/docker/test/stateless_with_coverage/run.sh b/docker/test/stateless_with_coverage/run.sh index 5d63f9f49d07..ccf3e53f7158 100755 --- a/docker/test/stateless_with_coverage/run.sh +++ b/docker/test/stateless_with_coverage/run.sh @@ -48,6 +48,7 @@ ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/con ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \ + ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \ From 875d7aef8b690711a121799fd7eeaba634da5006 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 13 Aug 2019 17:32:32 +0300 Subject: [PATCH 070/161] better --- dbms/src/Interpreters/SystemLog.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/SystemLog.h b/dbms/src/Interpreters/SystemLog.h index 545519aeb9df..da340fab5d5c 100644 --- a/dbms/src/Interpreters/SystemLog.h +++ b/dbms/src/Interpreters/SystemLog.h @@ -187,7 +187,7 @@ SystemLog::SystemLog(Context & context_, flush_interval_milliseconds(flush_interval_milliseconds_) { log = &Logger::get("SystemLog (" + database_name + "." + table_name + ")"); -qq + data.reserve(DBMS_SYSTEM_LOG_QUEUE_SIZE); saving_thread = ThreadFromGlobalPool([this] { threadFunction(); }); } From c69684e807277d2e172ca6cd5512346fea4bce7a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 13 Aug 2019 17:40:19 +0300 Subject: [PATCH 071/161] better --- contrib/arrow | 2 +- contrib/simdjson | 2 +- contrib/snappy | 2 +- contrib/thrift | 2 +- dbms/src/Interpreters/MetricLog.cpp | 9 +++------ dbms/src/Interpreters/SystemLog.cpp | 8 ++++++++ 6 files changed, 15 insertions(+), 10 deletions(-) diff --git a/contrib/arrow b/contrib/arrow index 87ac6fddaf21..93688e8c1fa2 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 87ac6fddaf21d0b4ee8b8090533ff293db0da1b4 +Subproject commit 93688e8c1fa2f22d46394c548a9edbd3d2d7c62d diff --git a/contrib/simdjson b/contrib/simdjson index e3f6322af762..3fb82502f7f1 160000 --- a/contrib/simdjson +++ b/contrib/simdjson @@ -1 +1 @@ -Subproject commit e3f6322af762213ff2087ce3366bf9541c7fd355 +Subproject commit 3fb82502f7f19a098006e7ff74c9b6e7c7dd4a84 diff --git a/contrib/snappy b/contrib/snappy index 3f194acb57e0..156cd8939c5f 160000 --- a/contrib/snappy +++ b/contrib/snappy @@ -1 +1 @@ -Subproject commit 3f194acb57e0487531c96b97af61dcbd025a78a3 +Subproject commit 156cd8939c5fba7fa68ae08db843377ecc07b4b5 diff --git a/contrib/thrift b/contrib/thrift index 010ccf0a0c70..74d6d9d3d640 160000 --- a/contrib/thrift +++ b/contrib/thrift @@ -1 +1 @@ -Subproject commit 010ccf0a0c7023fea0f6bf4e4078ebdff7e61982 +Subproject commit 74d6d9d3d6400d1672f48b4acf5bc7d1260ad96d diff --git a/dbms/src/Interpreters/MetricLog.cpp b/dbms/src/Interpreters/MetricLog.cpp index 69d9425b17e4..52775ecb8ea9 100644 --- a/dbms/src/Interpreters/MetricLog.cpp +++ b/dbms/src/Interpreters/MetricLog.cpp @@ -1,8 +1,11 @@ #include #include #include +<<<<<<< HEAD #include #include +======= +>>>>>>> 46a5ec5c16ed274956cc2051d0d2da213d975051 namespace DB { @@ -26,12 +29,10 @@ Block MetricLogElement::createBlock() //CurrentMetrics for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) { - std::string name; name += "CurrentMetric_"; name += CurrentMetrics::getName(ProfileEvents::Event(i)); columns_with_type_and_name.emplace_back(std::make_shared(), name); - } return Block(columns_with_type_and_name); @@ -50,18 +51,14 @@ void MetricLogElement::appendToBlock(Block & block) const for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) { UInt64 value = ProfileEvents::global_counters[i]; - columns[iter++]->insert(value); - } //CurrentMetrics for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) { UInt64 value = CurrentMetrics::values[i]; - columns[iter++]->insert(value); - } } diff --git a/dbms/src/Interpreters/SystemLog.cpp b/dbms/src/Interpreters/SystemLog.cpp index 6e63951d9ec5..f59af169a4af 100644 --- a/dbms/src/Interpreters/SystemLog.cpp +++ b/dbms/src/Interpreters/SystemLog.cpp @@ -109,6 +109,14 @@ void SystemLogs::metricThreadFunction() } } +======= + metric_log->shutdown(); +} + +void SystemLogs::threadFunction() +{ + +>>>>>>> 46a5ec5c16ed274956cc2051d0d2da213d975051 } } From e9fcccff5997d996421649475654a46cf216c8ad Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 13 Aug 2019 17:47:19 +0300 Subject: [PATCH 072/161] conflict resolved --- dbms/src/Interpreters/MetricLog.cpp | 3 --- dbms/src/Interpreters/MetricLog.h | 1 + dbms/src/Interpreters/SystemLog.cpp | 9 --------- 3 files changed, 1 insertion(+), 12 deletions(-) diff --git a/dbms/src/Interpreters/MetricLog.cpp b/dbms/src/Interpreters/MetricLog.cpp index 52775ecb8ea9..5c526d002d6a 100644 --- a/dbms/src/Interpreters/MetricLog.cpp +++ b/dbms/src/Interpreters/MetricLog.cpp @@ -1,11 +1,8 @@ #include #include #include -<<<<<<< HEAD #include #include -======= ->>>>>>> 46a5ec5c16ed274956cc2051d0d2da213d975051 namespace DB { diff --git a/dbms/src/Interpreters/MetricLog.h b/dbms/src/Interpreters/MetricLog.h index 13112d4934eb..ce435b9e3cf4 100644 --- a/dbms/src/Interpreters/MetricLog.h +++ b/dbms/src/Interpreters/MetricLog.h @@ -10,6 +10,7 @@ using Poco::Message; struct MetricLogElement { time_t event_time{}; + static std::string name() { return "MetricLog"; } static Block createBlock(); void appendToBlock(Block & block) const; diff --git a/dbms/src/Interpreters/SystemLog.cpp b/dbms/src/Interpreters/SystemLog.cpp index f59af169a4af..bd8a20daff17 100644 --- a/dbms/src/Interpreters/SystemLog.cpp +++ b/dbms/src/Interpreters/SystemLog.cpp @@ -109,14 +109,5 @@ void SystemLogs::metricThreadFunction() } } -======= - metric_log->shutdown(); -} - -void SystemLogs::threadFunction() -{ - ->>>>>>> 46a5ec5c16ed274956cc2051d0d2da213d975051 -} } From c8970ea6d5a298cbc3e5fb8ee149f69cbc7bfb29 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 13 Aug 2019 17:52:35 +0300 Subject: [PATCH 073/161] nothing --- dbms/src/Interpreters/SystemLog.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Interpreters/SystemLog.cpp b/dbms/src/Interpreters/SystemLog.cpp index bd8a20daff17..8df20c4c8b54 100644 --- a/dbms/src/Interpreters/SystemLog.cpp +++ b/dbms/src/Interpreters/SystemLog.cpp @@ -38,7 +38,6 @@ std::shared_ptr createSystemLog( return std::make_shared(context, database, table, engine, flush_interval_milliseconds); } - } From 9632c4102a78d9f53749938134acf2f97fbe24f2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 13 Aug 2019 18:26:31 +0300 Subject: [PATCH 074/161] Change test to pass the exceeding of memory limit in ParsedJson::Iterator::Iterator when UBSan is used. --- dbms/src/Functions/FunctionsJSON.h | 2 +- dbms/tests/queries/0_stateless/00975_json_hang.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/FunctionsJSON.h b/dbms/src/Functions/FunctionsJSON.h index a0aad24d0b93..0d229f8e2f32 100644 --- a/dbms/src/Functions/FunctionsJSON.h +++ b/dbms/src/Functions/FunctionsJSON.h @@ -7,6 +7,7 @@ #include "config_functions.h" #include #include +#include #include #include #include @@ -15,7 +16,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/tests/queries/0_stateless/00975_json_hang.sql b/dbms/tests/queries/0_stateless/00975_json_hang.sql index d60411cb7965..0618b4ba8f79 100644 --- a/dbms/tests/queries/0_stateless/00975_json_hang.sql +++ b/dbms/tests/queries/0_stateless/00975_json_hang.sql @@ -1 +1 @@ -SELECT DISTINCT JSONExtractRaw(concat('{"x":', rand() % 2 ? 'true' : 'false', '}'), 'x') AS res FROM numbers(1000000) ORDER BY res; +SELECT DISTINCT JSONExtractRaw(concat('{"x":', rand() % 2 ? 'true' : 'false', '}'), 'x') AS res FROM numbers(100000) ORDER BY res; From e227611f0133cbadbc6999f5e51b35a83d19dcf6 Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Tue, 13 Aug 2019 19:05:13 +0300 Subject: [PATCH 075/161] New function currentUser() implemented --- dbms/src/Functions/currentUser.cpp | 54 +++++++++++++++++++ .../registerFunctionsMiscellaneous.cpp | 2 + .../00990_function_current_user.reference | 3 ++ .../00990_function_current_user.sql | 4 ++ 4 files changed, 63 insertions(+) create mode 100644 dbms/src/Functions/currentUser.cpp create mode 100644 dbms/tests/queries/0_stateless/00990_function_current_user.reference create mode 100644 dbms/tests/queries/0_stateless/00990_function_current_user.sql diff --git a/dbms/src/Functions/currentUser.cpp b/dbms/src/Functions/currentUser.cpp new file mode 100644 index 000000000000..ee7c4d589a01 --- /dev/null +++ b/dbms/src/Functions/currentUser.cpp @@ -0,0 +1,54 @@ +#include +#include +#include +#include + + +namespace DB +{ + +class FunctionCurrentUser : public IFunction +{ + const String user_name; + +public: + static constexpr auto name = "currentUser"; + static FunctionPtr create(const Context & context) + { + return std::make_shared(context.getClientInfo().current_user); + } + + explicit FunctionCurrentUser(const String & user_name_) : user_name{user_name_} + { + } + + String getName() const override + { + return name; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) 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 + { + block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, user_name); + } +}; + + +void registerFunctionCurrentUser(FunctionFactory & factory) +{ + factory.registerFunction(); + factory.registerAlias("user", FunctionCurrentUser::name, FunctionFactory::CaseSensitive); +} + +} diff --git a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp index 6d201d65bd35..5c86587cb842 100644 --- a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp @@ -6,6 +6,7 @@ namespace DB class FunctionFactory; void registerFunctionCurrentDatabase(FunctionFactory &); +void registerFunctionCurrentUser(FunctionFactory &); void registerFunctionHostName(FunctionFactory &); void registerFunctionVisibleWidth(FunctionFactory &); void registerFunctionToTypeName(FunctionFactory &); @@ -56,6 +57,7 @@ void registerFunctionConvertCharset(FunctionFactory &); void registerFunctionsMiscellaneous(FunctionFactory & factory) { registerFunctionCurrentDatabase(factory); + registerFunctionCurrentUser(factory); registerFunctionHostName(factory); registerFunctionVisibleWidth(factory); registerFunctionToTypeName(factory); diff --git a/dbms/tests/queries/0_stateless/00990_function_current_user.reference b/dbms/tests/queries/0_stateless/00990_function_current_user.reference new file mode 100644 index 000000000000..e8183f05f5db --- /dev/null +++ b/dbms/tests/queries/0_stateless/00990_function_current_user.reference @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/dbms/tests/queries/0_stateless/00990_function_current_user.sql b/dbms/tests/queries/0_stateless/00990_function_current_user.sql new file mode 100644 index 000000000000..c925dfd8e477 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00990_function_current_user.sql @@ -0,0 +1,4 @@ +-- since actual user name is unknown, have to perform just smoke tests +select currentUser() IS NOT NULL; +select length(currentUser()) > 0; +select currentUser() = user(); \ No newline at end of file From 4d145cb72be73617b9ebb17365863ba6b4defd9d Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Tue, 13 Aug 2019 19:08:12 +0300 Subject: [PATCH 076/161] Documentation added --- docs/en/query_language/functions/other_functions.md | 3 +++ docs/ru/query_language/functions/other_functions.md | 3 +++ 2 files changed, 6 insertions(+) diff --git a/docs/en/query_language/functions/other_functions.md b/docs/en/query_language/functions/other_functions.md index 57fa8acfee35..05961caadc8a 100644 --- a/docs/en/query_language/functions/other_functions.md +++ b/docs/en/query_language/functions/other_functions.md @@ -102,6 +102,9 @@ Sleeps 'seconds' seconds on each row. You can specify an integer or a floating-p Returns the name of the current database. You can use this function in table engine parameters in a CREATE TABLE query where you need to specify the database. +## currentUser() +Returns the login of authorized user. + ## isFinite(x) Accepts Float32 and Float64 and returns UInt8 equal to 1 if the argument is not infinite and not a NaN, otherwise 0. diff --git a/docs/ru/query_language/functions/other_functions.md b/docs/ru/query_language/functions/other_functions.md index 1637c7bda936..dd4264e411fd 100644 --- a/docs/ru/query_language/functions/other_functions.md +++ b/docs/ru/query_language/functions/other_functions.md @@ -97,6 +97,9 @@ SELECT visibleWidth(NULL) Возвращает имя текущей базы данных. Эта функция может использоваться в параметрах движка таблицы в запросе CREATE TABLE там, где нужно указать базу данных. +## currentUser() +Возвращает логин пользователя, от имени которого исполняется запрос. + ## isFinite(x) Принимает Float32 или Float64 и возвращает UInt8, равный 1, если аргумент не бесконечный и не NaN, иначе 0. From aab140ceff49bccbc3808ccb78f9f042bd646449 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 13 Aug 2019 19:13:08 +0300 Subject: [PATCH 077/161] submodules --- contrib/arrow | 2 +- contrib/simdjson | 2 +- contrib/snappy | 2 +- contrib/thrift | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/contrib/arrow b/contrib/arrow index 93688e8c1fa2..87ac6fddaf21 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 93688e8c1fa2f22d46394c548a9edbd3d2d7c62d +Subproject commit 87ac6fddaf21d0b4ee8b8090533ff293db0da1b4 diff --git a/contrib/simdjson b/contrib/simdjson index 3fb82502f7f1..e3f6322af762 160000 --- a/contrib/simdjson +++ b/contrib/simdjson @@ -1 +1 @@ -Subproject commit 3fb82502f7f19a098006e7ff74c9b6e7c7dd4a84 +Subproject commit e3f6322af762213ff2087ce3366bf9541c7fd355 diff --git a/contrib/snappy b/contrib/snappy index 156cd8939c5f..3f194acb57e0 160000 --- a/contrib/snappy +++ b/contrib/snappy @@ -1 +1 @@ -Subproject commit 156cd8939c5fba7fa68ae08db843377ecc07b4b5 +Subproject commit 3f194acb57e0487531c96b97af61dcbd025a78a3 diff --git a/contrib/thrift b/contrib/thrift index 74d6d9d3d640..010ccf0a0c70 160000 --- a/contrib/thrift +++ b/contrib/thrift @@ -1 +1 @@ -Subproject commit 74d6d9d3d6400d1672f48b4acf5bc7d1260ad96d +Subproject commit 010ccf0a0c7023fea0f6bf4e4078ebdff7e61982 From 7dba25ff9d195fa910706ed12bb6f9227a6276d7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 13 Aug 2019 19:17:18 +0300 Subject: [PATCH 078/161] const --- dbms/src/Interpreters/MetricLog.cpp | 4 ++-- dbms/src/Interpreters/SystemLog.cpp | 2 ++ 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/MetricLog.cpp b/dbms/src/Interpreters/MetricLog.cpp index 5c526d002d6a..088cb7cabd24 100644 --- a/dbms/src/Interpreters/MetricLog.cpp +++ b/dbms/src/Interpreters/MetricLog.cpp @@ -47,14 +47,14 @@ void MetricLogElement::appendToBlock(Block & block) const //ProfileEvents for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) { - UInt64 value = ProfileEvents::global_counters[i]; + const UInt64 value = ProfileEvents::global_counters[i]; columns[iter++]->insert(value); } //CurrentMetrics for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) { - UInt64 value = CurrentMetrics::values[i]; + const UInt64 value = CurrentMetrics::values[i]; columns[iter++]->insert(value); } } diff --git a/dbms/src/Interpreters/SystemLog.cpp b/dbms/src/Interpreters/SystemLog.cpp index 8df20c4c8b54..6e63951d9ec5 100644 --- a/dbms/src/Interpreters/SystemLog.cpp +++ b/dbms/src/Interpreters/SystemLog.cpp @@ -38,6 +38,7 @@ std::shared_ptr createSystemLog( return std::make_shared(context, database, table, engine, flush_interval_milliseconds); } + } @@ -108,5 +109,6 @@ void SystemLogs::metricThreadFunction() } } +} } From 226ef15c2ac90fded69ecb3b943cef66fe3097f0 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 13 Aug 2019 19:21:22 +0300 Subject: [PATCH 079/161] ZH docs build fix (#6469) * Link fix. * Fixed ZH docs build. --- docs/zh/data_types/{domain => domains}/ipv4.md | 0 docs/zh/data_types/{domain => domains}/ipv6.md | 0 docs/zh/data_types/{domain => domains}/overview.md | 0 3 files changed, 0 insertions(+), 0 deletions(-) rename docs/zh/data_types/{domain => domains}/ipv4.md (100%) rename docs/zh/data_types/{domain => domains}/ipv6.md (100%) rename docs/zh/data_types/{domain => domains}/overview.md (100%) diff --git a/docs/zh/data_types/domain/ipv4.md b/docs/zh/data_types/domains/ipv4.md similarity index 100% rename from docs/zh/data_types/domain/ipv4.md rename to docs/zh/data_types/domains/ipv4.md diff --git a/docs/zh/data_types/domain/ipv6.md b/docs/zh/data_types/domains/ipv6.md similarity index 100% rename from docs/zh/data_types/domain/ipv6.md rename to docs/zh/data_types/domains/ipv6.md diff --git a/docs/zh/data_types/domain/overview.md b/docs/zh/data_types/domains/overview.md similarity index 100% rename from docs/zh/data_types/domain/overview.md rename to docs/zh/data_types/domains/overview.md From e07f3d9d10e58245800a3c565f454c2dc826a0b7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 13 Aug 2019 19:47:12 +0300 Subject: [PATCH 080/161] refactor --- dbms/src/Interpreters/MetricLog.cpp | 42 +++++++++++++++++++++++++++++ dbms/src/Interpreters/MetricLog.h | 14 ++++++++++ dbms/src/Interpreters/SystemLog.cpp | 39 +++++---------------------- dbms/src/Interpreters/SystemLog.h | 6 ----- 4 files changed, 63 insertions(+), 38 deletions(-) diff --git a/dbms/src/Interpreters/MetricLog.cpp b/dbms/src/Interpreters/MetricLog.cpp index 088cb7cabd24..3e8bf21ad80c 100644 --- a/dbms/src/Interpreters/MetricLog.cpp +++ b/dbms/src/Interpreters/MetricLog.cpp @@ -59,4 +59,46 @@ void MetricLogElement::appendToBlock(Block & block) const } } +void MetricLog::startCollectMetric(size_t collect_interval_milliseconds_) +{ + collect_interval_milliseconds = collect_interval_milliseconds_; + is_shutdown_metric_thread = false; + metric_flush_thread = ThreadFromGlobalPool([this] { metricThreadFunction(); }); +} + +void MetricLog::stopCollectMetric() +{ + bool old_val = false; + if (!is_shutdown_metric_thread.compare_exchange_strong(old_val, true)) + return; + metric_flush_thread.join(); +} + +void MetricLog::metricThreadFunction() +{ + const size_t flush_interval_milliseconds = 1000; + while (true) + { + try + { + const auto prev_timepoint = std::chrono::system_clock::now(); + + if (is_shutdown_metric_thread) + break; + + MetricLogElement elem; + elem.event_time = std::time(nullptr); + this->add(elem); + + const auto next_timepoint = prev_timepoint + std::chrono::milliseconds(flush_interval_milliseconds); + std::this_thread::sleep_until(next_timepoint); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + } +} + } diff --git a/dbms/src/Interpreters/MetricLog.h b/dbms/src/Interpreters/MetricLog.h index ce435b9e3cf4..7c041a19d7d3 100644 --- a/dbms/src/Interpreters/MetricLog.h +++ b/dbms/src/Interpreters/MetricLog.h @@ -19,6 +19,20 @@ struct MetricLogElement class MetricLog : public SystemLog { using SystemLog::SystemLog; + +public: + /// Launches a background thread to collect metrics with interval + void startCollectMetric(size_t collect_interval_milliseconds_); + + /// Stop background thread. Call before shutdown. + void stopCollectMetric(); + +private: + void metricThreadFunction(); + + ThreadFromGlobalPool metric_flush_thread; + size_t collect_interval_milliseconds; + std::atomic is_shutdown_metric_thread{false}; }; } diff --git a/dbms/src/Interpreters/SystemLog.cpp b/dbms/src/Interpreters/SystemLog.cpp index 6e63951d9ec5..ce1d787562d9 100644 --- a/dbms/src/Interpreters/SystemLog.cpp +++ b/dbms/src/Interpreters/SystemLog.cpp @@ -52,7 +52,12 @@ SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfi metric_log = createSystemLog(global_context, "system", "metric_log", config, "metric_log"); if (metric_log) - metric_flush_thread = ThreadFromGlobalPool([this] { metricThreadFunction(); }); + { + constexpr size_t DEFAULT_METRIC_LOG_COLLECT_INTERVAL_MILLISECONDS = 1000; + size_t collect_interval_milliseconds = config.getUInt64("metric_log.collect_interval_milliseconds", DEFAULT_METRIC_LOG_COLLECT_INTERVAL_MILLISECONDS); + metric_log->startCollectMetric(collect_interval_milliseconds); + } + part_log_database = config.getString("part_log.database", "system"); } @@ -76,39 +81,9 @@ void SystemLogs::shutdown() text_log->shutdown(); if (metric_log) { - bool old_val = false; - if (!is_shutdown_metric_thread.compare_exchange_strong(old_val, true)) - return; - metric_flush_thread.join(); + metric_log->stopCollectMetric(); metric_log->shutdown(); } } -void SystemLogs::metricThreadFunction() -{ - const size_t flush_interval_milliseconds = 1000; - while (true) - { - try - { - const auto prev_timepoint = std::chrono::system_clock::now(); - - if (is_shutdown_metric_thread) - break; - - MetricLogElement elem; - elem.event_time = std::time(nullptr); - metric_log->add(elem); - - const auto next_timepoint = prev_timepoint + std::chrono::milliseconds(flush_interval_milliseconds); - std::this_thread::sleep_until(next_timepoint); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - } -} - } diff --git a/dbms/src/Interpreters/SystemLog.h b/dbms/src/Interpreters/SystemLog.h index da340fab5d5c..b00f77b76222 100644 --- a/dbms/src/Interpreters/SystemLog.h +++ b/dbms/src/Interpreters/SystemLog.h @@ -79,10 +79,6 @@ struct SystemLogs std::shared_ptr text_log; /// Used to log all text messages. std::shared_ptr metric_log; /// Used to log all metrics. - ThreadFromGlobalPool metric_flush_thread; - void metricThreadFunction(); - std::atomic is_shutdown_metric_thread{false}; - String part_log_database; }; @@ -121,8 +117,6 @@ class SystemLog : private boost::noncopyable /// Stop the background flush thread before destructor. No more data will be written. void shutdown(); - size_t getFlushInterval() { return flush_interval_milliseconds; } - protected: Context & context; const String database_name; From eed8dd713824cd710753188d790292b781eb0b52 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 13 Aug 2019 19:48:23 +0300 Subject: [PATCH 081/161] better --- dbms/src/Interpreters/MetricLog.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/MetricLog.cpp b/dbms/src/Interpreters/MetricLog.cpp index 3e8bf21ad80c..001ad1a4bd21 100644 --- a/dbms/src/Interpreters/MetricLog.cpp +++ b/dbms/src/Interpreters/MetricLog.cpp @@ -76,7 +76,6 @@ void MetricLog::stopCollectMetric() void MetricLog::metricThreadFunction() { - const size_t flush_interval_milliseconds = 1000; while (true) { try @@ -90,7 +89,7 @@ void MetricLog::metricThreadFunction() elem.event_time = std::time(nullptr); this->add(elem); - const auto next_timepoint = prev_timepoint + std::chrono::milliseconds(flush_interval_milliseconds); + const auto next_timepoint = prev_timepoint + std::chrono::milliseconds(collect_interval_milliseconds); std::this_thread::sleep_until(next_timepoint); } catch (...) From a3d4fbd07d91530d5153bc043aa8e50652ee4226 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 13 Aug 2019 20:02:17 +0300 Subject: [PATCH 082/161] simple test added --- .../0_stateless/00990_metric_log_table_not_empty.reference | 1 + .../queries/0_stateless/00990_metric_log_table_not_empty.sql | 5 +++++ 2 files changed, 6 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00990_metric_log_table_not_empty.reference create mode 100644 dbms/tests/queries/0_stateless/00990_metric_log_table_not_empty.sql diff --git a/dbms/tests/queries/0_stateless/00990_metric_log_table_not_empty.reference b/dbms/tests/queries/0_stateless/00990_metric_log_table_not_empty.reference new file mode 100644 index 000000000000..d00491fd7e5b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00990_metric_log_table_not_empty.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00990_metric_log_table_not_empty.sql b/dbms/tests/queries/0_stateless/00990_metric_log_table_not_empty.sql new file mode 100644 index 000000000000..700312d0ba1b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00990_metric_log_table_not_empty.sql @@ -0,0 +1,5 @@ +select sleep(2) format Null; + +system flush logs; + +select count()>0 from system.metric_log \ No newline at end of file From c3e0ceecde177007381549499fc00597fc352d8e Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Tue, 13 Aug 2019 21:28:18 +0300 Subject: [PATCH 083/161] wait for finish calculations at underlying streams while cancelling AsynchronousBlockInputStream --- dbms/src/DataStreams/AsynchronousBlockInputStream.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index c741d8090e93..93c695f20c9b 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -80,10 +80,7 @@ class AsynchronousBlockInputStream : public IBlockInputStream /// Wait for some backgroud calculations to be sure, /// that after end of stream nothing is being executing. if (started) - { pool.wait(); - started = false; - } } ~AsynchronousBlockInputStream() override From 09ecd865fc9d75c0f1e6ef05ead663ed0abc5028 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 14 Aug 2019 03:26:38 +0300 Subject: [PATCH 084/161] Allow to use library dictionary source with ASan. May impose troubles. --- dbms/src/Dictionaries/LibraryDictionarySource.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Dictionaries/LibraryDictionarySource.cpp b/dbms/src/Dictionaries/LibraryDictionarySource.cpp index b4de6506db12..2cb74b944d33 100644 --- a/dbms/src/Dictionaries/LibraryDictionarySource.cpp +++ b/dbms/src/Dictionaries/LibraryDictionarySource.cpp @@ -1,5 +1,6 @@ #include "LibraryDictionarySource.h" #include +#include #include #include #include @@ -134,7 +135,7 @@ LibraryDictionarySource::LibraryDictionarySource( ErrorCodes::FILE_DOESNT_EXIST); description.init(sample_block); library = std::make_shared(path, RTLD_LAZY -#if defined(RTLD_DEEPBIND) // Does not exists in freebsd +#if defined(RTLD_DEEPBIND) && !defined(ADDRESS_SANITIZER) // Does not exists in FreeBSD. Cannot work with Address Sanitizer. | RTLD_DEEPBIND #endif ); From 5c0c661b6e5a97bb57f9582dbee04ca33db4c021 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 14 Aug 2019 09:02:56 +0300 Subject: [PATCH 085/161] Add Paris Meetup link --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index dccb75b42827..ac579d3f2a3d 100644 --- a/README.md +++ b/README.md @@ -14,6 +14,7 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events * [ClickHouse Meetup in Moscow](https://yandex.ru/promo/clickhouse/moscow-2019) on September 5. +* [ClickHouse Meetup in Paris](https://www.eventbrite.com/e/clickhouse-paris-meetup-2019-registration-68493270215) on October 3. * [ClickHouse Meetup in Hong Kong](https://www.meetup.com/Hong-Kong-Machine-Learning-Meetup/events/263580542/) on October 17. * [ClickHouse Meetup in Shenzhen](https://www.huodongxing.com/event/3483759917300) on October 20. * [ClickHouse Meetup in Shanghai](https://www.huodongxing.com/event/4483760336000) on October 27. From ccab7ce467c1f0fbfee1fa5f08407efdb210e491 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 14 Aug 2019 09:05:00 +0300 Subject: [PATCH 086/161] Add Paris Meetup link to front page (#6485) --- website/index.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/index.html b/website/index.html index 017080d56479..5ab51289c920 100644 --- a/website/index.html +++ b/website/index.html @@ -94,7 +94,7 @@

- Upcoming Meetups: Moscow on September 5, Hong Kong on October 17, Shenzhen on October 20 and Shanghai on October 27 + Upcoming Meetups: Moscow on September 5, Paris on October 3, Hong Kong on October 17, Shenzhen on October 20 and Shanghai on October 27
From e5d32785d4a0c7e41809a69410e379987de049b8 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 14 Aug 2019 09:45:24 +0300 Subject: [PATCH 087/161] DOCAPI-7443: Virtual columns docs update. (#6382) --- docs/en/operations/table_engines/index.md | 68 ++++++++++++++++++++++- docs/en/operations/table_engines/kafka.md | 13 +++++ docs/en/operations/table_engines/merge.md | 24 +++----- docs/toc_en.yml | 4 +- docs/toc_fa.yml | 4 +- docs/toc_ru.yml | 4 +- docs/toc_zh.yml | 4 +- 7 files changed, 94 insertions(+), 27 deletions(-) diff --git a/docs/en/operations/table_engines/index.md b/docs/en/operations/table_engines/index.md index e370e6173bbd..41680a5b3aff 100644 --- a/docs/en/operations/table_engines/index.md +++ b/docs/en/operations/table_engines/index.md @@ -2,15 +2,77 @@ The table engine (type of table) determines: -- How and where data is stored, where to write it to, and where to read it from. +- How and where data is stored, where to write it to, and from where to read it. - Which queries are supported, and how. - Concurrent data access. - Use of indexes, if present. - Whether multithreaded request execution is possible. - Data replication parameters. -When reading, the engine is only required to output the requested columns, but in some cases the engine can partially process data when responding to the request. +## Engine Families -For most serious tasks, you should use engines from the `MergeTree` family. +### *MergeTree + +The most universal and functional table engines for high-load tasks. The common property of these engines is quick data insertion with subsequent data processing in the background. The `*MergeTree` engines support data replication (with [Replicated*](replication.md) versions of engines), partitioning and other features not supported in other engines. + +Engines of the family: + +- [MergTree](mergetree.md) +- [ReplacingMergeTree](replacingmergetree.md) +- [SummingMergeTree](summingmergetree.md) +- [AggregatingMergeTree](aggregatingmergetree.md) +- [CollapsingMergeTree](collapsingmergetree.md) +- [VersionedCollapsingMergeTree](versionedcollapsingmergetree.md) +- [GraphiteMergeTree](graphitemergetree.md) + +### *Log + +Lightweight [engines](log_family.md) with minimum functionality. They are the most effective in scenarios when you need to quickly write many small tables (up to about 1 million rows) and read them later as a whole. + +Engines of the family: + +- [TinyLog](tinylog.md) +- [StripeLog](stripelog.md) +- [Log](log.md) + +### Intergation engines + +Engines for communicating with other data storage and processing systems. + +Engines of the family: + +- [Kafka](kafka.md) +- [MySQL](mysql.md) +- [ODBC](odbc.md) +- [JDBC](jdbc.md) + +### Special engines + +Engines solving special tasks. + +Engines of the family: + +- [Distributed](distributed.md) +- [MaterializedView](materializedview.md) +- [Dictionary](dictionary.md) +- [Merge](merge.md) +- [File](file.md) +- [Null](null.md) +- [Set](set.md) +- [Join](join.md) +- [URL](url.md) +- [View](view.md) +- [Memory](memory.md) +- [Buffer](buffer.md) + +## Virtual columns {#table_engines-virtual_columns} + +Virtual column is an integral attribute of a table engine that is defined in the source code of the engine. + +You should not specify virtual columns in the `CREATE TABLE` query, and you cannot see them in the results of `SHOW CREATE TABLE` and `DESCRIBE TABLE` queries. Also, virtual columns are read-only, so you can't insert data into virtual columns. + +To select data from a virtual column, you must specify its name in the `SELECT` query. The `SELECT *` doesn't return values from virtual columns. + +If you create a table with a column that has the same name as one of the table virtual columns, the virtual column becomes inaccessible. Doing so is not recommended. To help avoiding conflicts virtual column names are usually prefixed with an underscore. [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/) diff --git a/docs/en/operations/table_engines/kafka.md b/docs/en/operations/table_engines/kafka.md index 7bedd8f7ac92..90745ebb4cfc 100644 --- a/docs/en/operations/table_engines/kafka.md +++ b/docs/en/operations/table_engines/kafka.md @@ -155,4 +155,17 @@ Similar to GraphiteMergeTree, the Kafka engine supports extended configuration u For a list of possible configuration options, see the [librdkafka configuration reference](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md). Use the underscore (`_`) instead of a dot in the ClickHouse configuration. For example, `check.crcs=true` will be `true`. +## Virtual Columns + +- `_topic` — Kafka topic. +- `_key` — Key of the message. +- `_offset` — Offset of the message. +- `_timestamp` — Timestamp of the message. +- `_partition` — Partition of Kafka topic. + +**See Also** + +- [Virtual columns](index.md#table_engines-virtual_columns) + + [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/kafka/) diff --git a/docs/en/operations/table_engines/merge.md b/docs/en/operations/table_engines/merge.md index f29075ec9734..f2347d9777d0 100644 --- a/docs/en/operations/table_engines/merge.md +++ b/docs/en/operations/table_engines/merge.md @@ -6,7 +6,7 @@ The `Merge` engine accepts parameters: the database name and a regular expressio Example: -``` +```sql Merge(hits, '^WatchLog') ``` @@ -26,7 +26,7 @@ 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. -``` +```sql 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); @@ -39,33 +39,25 @@ CREATE TABLE WatchLog as WatchLog_old ENGINE=Merge(currentDatabase(), '^WatchLog SELECT * FROM WatchLog - +``` +```text ┌───────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. +- `_table` — Contains the name of the table from which data was read. Type: [String](../../data_types/string.md). -The `Merge` type table contains the virtual column `_table` of the type `String`. It contains the name of the table that data was read from. If any underlying table already has the column `_table`, then the virtual column is shadowed and is not accessible. + You can set the constant conditions on `_table` in the `WHERE/PREWHERE` clause (for example, `WHERE _table='xyz'`). In this case the read operation is performed only for that tables where the condition on `_table` is satisfied, so the `_table` column acts as an index. - +**See Also** -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. +- [Virtual columns](index.md#table_engines-virtual_columns) [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/merge/) diff --git a/docs/toc_en.yml b/docs/toc_en.yml index d75f81a556c0..5e6854002ca7 100644 --- a/docs/toc_en.yml +++ b/docs/toc_en.yml @@ -85,6 +85,8 @@ nav: - 'Integrations': - 'Kafka': 'operations/table_engines/kafka.md' - 'MySQL': 'operations/table_engines/mysql.md' + - 'JDBC': 'operations/table_engines/jdbc.md' + - 'ODBC': 'operations/table_engines/odbc.md' - 'Special': - 'Distributed': 'operations/table_engines/distributed.md' - 'External data': 'operations/table_engines/external_data.md' @@ -99,8 +101,6 @@ nav: - 'MaterializedView': 'operations/table_engines/materializedview.md' - 'Memory': 'operations/table_engines/memory.md' - 'Buffer': 'operations/table_engines/buffer.md' - - 'JDBC': 'operations/table_engines/jdbc.md' - - 'ODBC': 'operations/table_engines/odbc.md' - 'SQL Reference': - 'hidden': 'query_language/index.md' diff --git a/docs/toc_fa.yml b/docs/toc_fa.yml index 70a0f222d4d6..1799093df24c 100644 --- a/docs/toc_fa.yml +++ b/docs/toc_fa.yml @@ -85,6 +85,8 @@ nav: - 'Integrations': - 'Kafka': 'operations/table_engines/kafka.md' - 'MySQL': 'operations/table_engines/mysql.md' + - 'JDBC': 'operations/table_engines/jdbc.md' + - 'ODBC': 'operations/table_engines/odbc.md' - 'Special': - 'Distributed': 'operations/table_engines/distributed.md' - 'External data': 'operations/table_engines/external_data.md' @@ -99,8 +101,6 @@ nav: - 'MaterializedView': 'operations/table_engines/materializedview.md' - 'Memory': 'operations/table_engines/memory.md' - 'Buffer': 'operations/table_engines/buffer.md' - - 'JDBC': 'operations/table_engines/jdbc.md' - - 'ODBC': 'operations/table_engines/odbc.md' - 'SQL Reference': - 'hidden': 'query_language/index.md' diff --git a/docs/toc_ru.yml b/docs/toc_ru.yml index d2f463fcc852..682f171a1f34 100644 --- a/docs/toc_ru.yml +++ b/docs/toc_ru.yml @@ -86,6 +86,8 @@ nav: - 'Интеграции': - 'Kafka': 'operations/table_engines/kafka.md' - 'MySQL': 'operations/table_engines/mysql.md' + - 'JDBC': 'operations/table_engines/jdbc.md' + - 'ODBC': 'operations/table_engines/odbc.md' - 'Особые': - 'Distributed': 'operations/table_engines/distributed.md' - 'Внешние данные': 'operations/table_engines/external_data.md' @@ -100,8 +102,6 @@ nav: - 'MaterializedView': 'operations/table_engines/materializedview.md' - 'Memory': 'operations/table_engines/memory.md' - 'Buffer': 'operations/table_engines/buffer.md' - - 'JDBC': 'operations/table_engines/jdbc.md' - - 'ODBC': 'operations/table_engines/odbc.md' - 'Справка по SQL': - 'hidden': 'query_language/index.md' diff --git a/docs/toc_zh.yml b/docs/toc_zh.yml index 40b1c97d2a8a..6f9abc9b36cb 100644 --- a/docs/toc_zh.yml +++ b/docs/toc_zh.yml @@ -84,6 +84,8 @@ nav: - 'Integrations': - 'Kafka': 'operations/table_engines/kafka.md' - 'MySQL': 'operations/table_engines/mysql.md' + - 'JDBC': 'operations/table_engines/jdbc.md' + - 'ODBC': 'operations/table_engines/odbc.md' - 'Special': - 'Distributed': 'operations/table_engines/distributed.md' - 'External data': 'operations/table_engines/external_data.md' @@ -98,8 +100,6 @@ nav: - 'MaterializedView': 'operations/table_engines/materializedview.md' - 'Memory': 'operations/table_engines/memory.md' - 'Buffer': 'operations/table_engines/buffer.md' - - 'JDBC': 'operations/table_engines/jdbc.md' - - 'ODBC': 'operations/table_engines/odbc.md' - 'SQL语法': - 'hidden': 'query_language/index.md' From 09a67981b7655158190ffe1817eef35ef72495e9 Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 14 Aug 2019 14:56:43 +0800 Subject: [PATCH 088/161] build fix --- dbms/src/Core/SettingsCommon.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCommon.h index 080640961331..a2a587b74053 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCommon.h @@ -555,7 +555,7 @@ class SettingsCollection for (const auto & member : members()) { if (member.isChanged(castToDerived())) - found_changes.emplace_back(member.name.toString(), member.get_field(castToDerived())); + found_changes.push_back({member.name.toString(), member.get_field(castToDerived())}); } return found_changes; } From 6472768c07668c7749f3d18694abf22e665c6385 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 14 Aug 2019 10:58:30 +0300 Subject: [PATCH 089/161] DOCAPI-7090: SYSTEM DISTRIBUTED docs. EN review. RU translation. (#6262) --- docs/en/query_language/system.md | 19 ++++++++++++------- docs/ru/query_language/system.md | 15 +++++++++------ 2 files changed, 21 insertions(+), 13 deletions(-) diff --git a/docs/en/query_language/system.md b/docs/en/query_language/system.md index c5f4830723f9..b7797df490bf 100644 --- a/docs/en/query_language/system.md +++ b/docs/en/query_language/system.md @@ -4,31 +4,36 @@ - [FLUSH DISTRIBUTED](#query_language-system-flush-distributed) - [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends) + ## Managing Distributed Tables {#query_language-system-distributed} -ClickHouse can manage [distributed](../operations/table_engines/distributed.md) tables. When a user inserts data into such table, ClickHouse creates a queue of the data which should be sent to servers of the cluster, then asynchronously sends them. You can control the processing of queue by using the requests [STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends), [FLUSH DISTRIBUTED](#query_language-system-flush-distributed) and [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends). Also, you can synchronously insert distributed data with the `insert_distributed_sync` setting. +ClickHouse can manage [distributed](../operations/table_engines/distributed.md) tables. When a user inserts data into these tables, ClickHouse first creates a queue of the data that should be sent to cluster nodes, then asynchronously sends it. You can manage queue processing with the [STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends), [FLUSH DISTRIBUTED](#query_language-system-flush-distributed), and [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends) queries. You can also synchronously insert distributed data with the `insert_distributed_sync` setting. ### STOP DISTRIBUTED SENDS {#query_language-system-stop-distributed-sends} -Disables background data distributing, when inserting data into the distributed tables. +Disables background data distribution when inserting data into distributed tables. -``` +```sql SYSTEM STOP DISTRIBUTED SENDS [db.] ``` + ### FLUSH DISTRIBUTED {#query_language-system-flush-distributed} -Forces ClickHouse to send data to the servers of the cluster in synchronous mode. If some of the servers are not available, ClickHouse throws an exception and stops query processing. When servers are back into operation, you should repeat the query. +Forces ClickHouse to send data to cluster nodes synchronously. If any nodes are unavailable, ClickHouse throws an exception and stops query execution. You can retry the query until it succeeds, which will happen when all nodes are back online. -``` +```sql SYSTEM FLUSH DISTRIBUTED [db.] ``` + ### START DISTRIBUTED SENDS {#query_language-system-start-distributed-sends} -Enables background data distributing, when inserting data into the distributed tables. +Enables background data distribution when inserting data into distributed tables. -``` +```sql SYSTEM START DISTRIBUTED SENDS [db.] ``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/system/) diff --git a/docs/ru/query_language/system.md b/docs/ru/query_language/system.md index fcf6fb8eced2..f35b4a390611 100644 --- a/docs/ru/query_language/system.md +++ b/docs/ru/query_language/system.md @@ -1,16 +1,16 @@ -# SYSTEM Queries {#query_language-system} +# Запросы SYSTEM {#query_language-system} - [STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends) - [FLUSH DISTRIBUTED](#query_language-system-flush-distributed) - [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends) -## Managing Distributed Tables {#query_language-system-distributed} +## Управление распределёнными таблицами {#query_language-system-distributed} -ClickHouse can manage [distributed](../operations/table_engines/distributed.md) tables. When a user inserts data into such table, ClickHouse creates a queue of the data which should be sent to servers of the cluster, then asynchronously sends them. You can control the processing of queue by using the requests [STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends), [FLUSH DISTRIBUTED](#query_language-system-flush-distributed) and [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends). +ClickHouse может оперировать [распределёнными](../operations/table_engines/distributed.md) таблицами. Когда пользователь вставляет данные в эти таблицы, ClickHouse сначала формирует очередь из данных, которые должны быть отправлены на узлы кластера, а затем асинхронно отправляет подготовленные данные. Вы пожете управлять очередью с помощью запросов [STOP DISTRIBUTED SENDS](#query_language-system-stop-distributed-sends), [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends) и [FLUSH DISTRIBUTED](#query_language-system-flush-distributed). Также есть возможность синхронно вставлять распределенные данные с помощью настройки `insert_distributed_sync`. ### STOP DISTRIBUTED SENDS {#query_language-system-stop-distributed-sends} -Disables asynchronous distribution of data between servers of the cluster. +Отключает фоновую отправку при вставке данных в распределённые таблицы. ``` SYSTEM STOP DISTRIBUTED SENDS [db.] @@ -18,7 +18,7 @@ SYSTEM STOP DISTRIBUTED SENDS [db.] ### FLUSH DISTRIBUTED {#query_language-system-flush-distributed} -Forces ClickHouse to send data to the servers of the cluster in synchronous mode. If some of the servers are not available, ClickHouse throws an exception and stops query processing. When servers back into operation, you should repeat the query. +В синхронном режиме отправляет все данные на узлы кластера. Если какие-либо узлы недоступны, ClickHouse генерирует исключение и останавливает выполнение запроса. Такой запрос можно повторять до успешного завершения, что будет означать возвращение связанности с остальными узлами кластера. ``` SYSTEM FLUSH DISTRIBUTED [db.] @@ -26,8 +26,11 @@ SYSTEM FLUSH DISTRIBUTED [db.] ### START DISTRIBUTED SENDS {#query_language-system-start-distributed-sends} -Enables asynchronous distribution of data between servers of cluster. +Включает фоновую отправку при вставке данных в распределенные таблицы. ``` SYSTEM START DISTRIBUTED SENDS [db.] ``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/system/) + From bc4177f64cefd43c477d05dc0f92ca0c62fc4185 Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Wed, 14 Aug 2019 13:20:15 +0300 Subject: [PATCH 090/161] Make alias case-insenstitive --- dbms/src/Functions/currentUser.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/currentUser.cpp b/dbms/src/Functions/currentUser.cpp index ee7c4d589a01..fa8b3103be2c 100644 --- a/dbms/src/Functions/currentUser.cpp +++ b/dbms/src/Functions/currentUser.cpp @@ -48,7 +48,7 @@ class FunctionCurrentUser : public IFunction void registerFunctionCurrentUser(FunctionFactory & factory) { factory.registerFunction(); - factory.registerAlias("user", FunctionCurrentUser::name, FunctionFactory::CaseSensitive); + factory.registerAlias("user", FunctionCurrentUser::name, FunctionFactory::CaseInsensitive); } } From 6bcfe51edfd621d4239b2ffea5fba3170c6de1d0 Mon Sep 17 00:00:00 2001 From: akuzm <36882414+akuzm@users.noreply.github.com> Date: Wed, 14 Aug 2019 14:04:11 +0300 Subject: [PATCH 091/161] In performance test, do not read query log for queries we didn't run. (#6427) --- .../performance-test/PerformanceTest.cpp | 91 ++++++++++--------- 1 file changed, 50 insertions(+), 41 deletions(-) diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index 45e9dfa89a70..ab55cd3d6cf8 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -249,6 +251,54 @@ std::vector PerformanceTest::execute() runQueries(queries_with_indexes, statistics_by_run); } + + if (got_SIGINT) + { + return statistics_by_run; + } + + // Pull memory usage data from query log. The log is normally filled in + // background, so we have to flush it synchronously here to see all the + // previous queries. + { + NullBlockOutputStream null_output(Block{}); + RemoteBlockInputStream flush_log(connection, "system flush logs", + {} /* header */, context); + copyData(flush_log, null_output); + } + + for (auto & statistics : statistics_by_run) + { + if (statistics.query_id.empty()) + { + // We have statistics structs for skipped queries as well, so we + // have to filter them out. + continue; + } + + // We run some test queries several times, specifying the same query id, + // so this query to the log may return several records. Choose the + // last one, because this is when the query performance has stabilized. + RemoteBlockInputStream log_reader(connection, + "select memory_usage, query_start_time from system.query_log " + "where type = 2 and query_id = '" + statistics.query_id + "' " + "order by query_start_time desc", + {} /* header */, context); + + log_reader.readPrefix(); + Block block = log_reader.read(); + if (block.columns() == 0) + { + LOG_WARNING(log, "Query '" << statistics.query_id << "' is not found in query log."); + continue; + } + + auto column = block.getByName("memory_usage").column; + statistics.memory_usage = column->get64(0); + + log_reader.readSuffix(); + } + return statistics_by_run; } @@ -298,47 +348,6 @@ void PerformanceTest::runQueries( break; } } - - if (got_SIGINT) - { - return; - } - - // Pull memory usage data from query log. The log is normally filled in - // background, so we have to flush it synchronously here to see all the - // previous queries. - { - RemoteBlockInputStream flush_log(connection, "system flush logs", - {} /* header */, context); - flush_log.readPrefix(); - while (flush_log.read()); - flush_log.readSuffix(); - } - - for (auto & statistics : statistics_by_run) - { - RemoteBlockInputStream log_reader(connection, - "select memory_usage from system.query_log where type = 2 and query_id = '" - + statistics.query_id + "'", - {} /* header */, context); - - log_reader.readPrefix(); - Block block = log_reader.read(); - if (block.columns() == 0) - { - LOG_WARNING(log, "Query '" << statistics.query_id << "' is not found in query log."); - continue; - } - - assert(block.columns() == 1); - assert(block.getDataTypes()[0]->getName() == "UInt64"); - ColumnPtr column = block.getByPosition(0).column; - assert(column->size() == 1); - StringRef ref = column->getDataAt(0); - assert(ref.size == sizeof(UInt64)); - statistics.memory_usage = *reinterpret_cast(ref.data); - log_reader.readSuffix(); - } } From d9234a64c478d44c2b03bcf02c523a2f19b93485 Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Wed, 14 Aug 2019 15:18:11 +0300 Subject: [PATCH 092/161] Make function return initial user --- dbms/src/Functions/currentUser.cpp | 2 +- .../queries/0_stateless/00990_function_current_user.reference | 1 + dbms/tests/queries/0_stateless/00990_function_current_user.sql | 3 ++- docs/en/query_language/functions/other_functions.md | 2 +- 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/currentUser.cpp b/dbms/src/Functions/currentUser.cpp index fa8b3103be2c..4501047fbb02 100644 --- a/dbms/src/Functions/currentUser.cpp +++ b/dbms/src/Functions/currentUser.cpp @@ -15,7 +15,7 @@ class FunctionCurrentUser : public IFunction static constexpr auto name = "currentUser"; static FunctionPtr create(const Context & context) { - return std::make_shared(context.getClientInfo().current_user); + return std::make_shared(context.getClientInfo().initial_user); } explicit FunctionCurrentUser(const String & user_name_) : user_name{user_name_} diff --git a/dbms/tests/queries/0_stateless/00990_function_current_user.reference b/dbms/tests/queries/0_stateless/00990_function_current_user.reference index e8183f05f5db..f1f321b1ecda 100644 --- a/dbms/tests/queries/0_stateless/00990_function_current_user.reference +++ b/dbms/tests/queries/0_stateless/00990_function_current_user.reference @@ -1,3 +1,4 @@ 1 1 +1 1 1 diff --git a/dbms/tests/queries/0_stateless/00990_function_current_user.sql b/dbms/tests/queries/0_stateless/00990_function_current_user.sql index c925dfd8e477..57cd14c2b65b 100644 --- a/dbms/tests/queries/0_stateless/00990_function_current_user.sql +++ b/dbms/tests/queries/0_stateless/00990_function_current_user.sql @@ -1,4 +1,5 @@ -- since actual user name is unknown, have to perform just smoke tests select currentUser() IS NOT NULL; select length(currentUser()) > 0; -select currentUser() = user(); \ No newline at end of file +select currentUser() = user(), currentUser() = USER(); +select currentUser() = initial_user from system.processes where query like '%$!@#%'; \ No newline at end of file diff --git a/docs/en/query_language/functions/other_functions.md b/docs/en/query_language/functions/other_functions.md index 05961caadc8a..cc5fadcdd98c 100644 --- a/docs/en/query_language/functions/other_functions.md +++ b/docs/en/query_language/functions/other_functions.md @@ -103,7 +103,7 @@ Returns the name of the current database. You can use this function in table engine parameters in a CREATE TABLE query where you need to specify the database. ## currentUser() -Returns the login of authorized user. +Returns the login of authorized user (initiator of query execution). ## isFinite(x) From d44d2d1731637cf6d259b6db147ae729ce21a7fe Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Wed, 14 Aug 2019 15:29:21 +0300 Subject: [PATCH 093/161] rewrite trash code in optimizeReadInOrder and disable read in order optimization with joins --- .../Interpreters/InterpreterSelectQuery.cpp | 34 +++++-------------- 1 file changed, 9 insertions(+), 25 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index e969e4fca3a2..882bb44411b7 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -665,7 +665,8 @@ static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & c } -static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, const ASTSelectQuery & query, const Context & context) +static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, const ASTSelectQuery & query, + const Context & context, const SyntaxAnalyzerResultPtr & syntax_result) { if (!merge_tree.hasSortingKey()) return {}; @@ -677,20 +678,12 @@ static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, cons const auto & sorting_key_columns = merge_tree.getSortingKeyColumns(); size_t prefix_size = std::min(order_descr.size(), sorting_key_columns.size()); - auto order_by_expr = query.orderBy(); - SyntaxAnalyzerResultPtr syntax_result; - try - { - syntax_result = SyntaxAnalyzer(context).analyze(order_by_expr, merge_tree.getColumns().getAllPhysical()); - } - catch (const Exception &) - { - return {}; - } - for (size_t i = 0; i < prefix_size; ++i) { - /// Read in pk order in case of exact match with order key element + if (syntax_result->array_join_alias_to_name.count(order_descr[i].column_name)) + break; + + /// Optimize in case of exact match with order key element /// or in some simple cases when order key element is wrapped into monotonic function. int current_direction = order_descr[i].direction; if (order_descr[i].column_name == sorting_key_columns[i] && current_direction == read_direction) @@ -699,16 +692,7 @@ static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, cons { const auto & ast = query.orderBy()->children[i]; ExpressionActionsPtr actions; - try - { - actions = ExpressionAnalyzer(ast->children.at(0), syntax_result, context).getActions(false); - } - catch (const Exception &) - { - /// Can't analyze order expression at this stage. - /// May be some actions required for order will be executed later. - break; - } + actions = ExpressionAnalyzer(ast->children.at(0), syntax_result, context).getActions(true); const auto & input_columns = actions->getRequiredColumnsWithTypes(); if (input_columns.size() != 1 || input_columns.front().name != sorting_key_columns[i]) @@ -820,10 +804,10 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS } SortingInfoPtr sorting_info; - if (settings.optimize_read_in_order && storage && query.orderBy() && !query.groupBy() && !query.final()) + if (settings.optimize_read_in_order && storage && query.orderBy() && !query.groupBy() && !query.final() && !query.join()) { if (const MergeTreeData * merge_tree_data = dynamic_cast(storage.get())) - sorting_info = optimizeReadInOrder(*merge_tree_data, query, context); + sorting_info = optimizeReadInOrder(*merge_tree_data, query, context, syntax_analyzer_result); } if (dry_run) From 15bbf080f36ecf1eb91bd4956639e6169ffee015 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 14 Aug 2019 15:54:41 +0300 Subject: [PATCH 094/161] changes after review --- dbms/programs/server/config.xml | 9 +++++++++ dbms/src/Interpreters/MetricLog.cpp | 19 ++++++++++--------- dbms/src/Interpreters/SystemLog.cpp | 3 +-- .../00990_metric_log_table_not_empty.sql | 4 ++-- 4 files changed, 22 insertions(+), 13 deletions(-) diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index c09913cbd870..28901a3b8971 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -329,6 +329,15 @@ text_log
7500 + + diff --git a/docs/en/database_engines/mysql.md b/docs/en/database_engines/mysql.md index fd5417ee27d7..24352c1924c8 100644 --- a/docs/en/database_engines/mysql.md +++ b/docs/en/database_engines/mysql.md @@ -1,10 +1,10 @@ # MySQL -Allows to connect to some database on remote MySQL server and perform `INSERT` and `SELECT` queries with tables to exchange data between ClickHouse and MySQL. +Allows to connect to databases on a remote MySQL server and perform `INSERT` and `SELECT` queries with tables to exchange data between ClickHouse and MySQL. -The `MySQL` database engine translate queries to the MySQL server, so you can perform operations such as `SHOW TABLES` or `SHOW CREATE TABLE`. +The `MySQL` database engine translate queries to the MySQL server so you can perform operations such as `SHOW TABLES` or `SHOW CREATE TABLE`. -You cannot perform with tables the following queries: +You cannot perform the following queries: - `ATTACH`/`DETACH` - `DROP` @@ -48,7 +48,7 @@ BINARY | [FixedString](../data_types/fixedstring.md) All other MySQL data types are converted into [String](../data_types/string.md). -[Nullable](../data_types/nullable.md) data type is supported. +[Nullable](../data_types/nullable.md) is supported. ## Examples of Use @@ -120,3 +120,5 @@ SELECT * FROM mysql_db.mysql_table │ 3 │ 4 │ └────────┴───────┘ ``` + +[Original article](https://clickhouse.yandex/docs/en/database_engines/mysql/) diff --git a/docs/en/operations/table_engines/mergetree.md b/docs/en/operations/table_engines/mergetree.md index 2a099a8947d5..56cc99f5653b 100644 --- a/docs/en/operations/table_engines/mergetree.md +++ b/docs/en/operations/table_engines/mergetree.md @@ -28,7 +28,7 @@ Main features: ## Creating a Table {#table_engine-mergetree-creating-a-table} -``` +```sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ( name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1], @@ -47,7 +47,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] For descriptions of request parameters, see the [request description](../../query_language/create.md). -**Query clauses** +### Query Clauses - `ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. The `MergeTree` engine does not have parameters. @@ -81,7 +81,7 @@ For descriptions of request parameters, see the [request description](../../quer - `merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with TTL. Default value: 86400 (1 day). -**Example of setting the sections** +**Example of Sections Setting** ```sql ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192 @@ -107,7 +107,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ) ENGINE [=] MergeTree(date-column [, sampling_expression], (primary, key), index_granularity) ``` -**MergeTree() parameters** +**MergeTree() Parameters** - `date-column` — The name of a column of the [Date](../../data_types/date.md) type. ClickHouse automatically creates partitions by month based on this column. The partition names are in the `"YYYYMM"` format. - `sampling_expression` — An expression for sampling. @@ -312,16 +312,16 @@ Reading from a table is automatically parallelized. Determines the lifetime of values. -The `TTL` clause can be set for the whole table and for each individual column. If `TTL` is set for the whole table, individual `TTL` for columns are ignored. - +The `TTL` clause can be set for the whole table and for each individual column. If both `TTL` are set, ClickHouse uses that `TTL` which expires earlier. -The table must have the column of the [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md) data type. This date column should be used in the `TTL` clause. You can only set lifetime of the data as an interval from the date column value. +The table must have the column in the [Date](../../data_types/date.md) or [DateTime](../../data_types/datetime.md) data type. To define the lifetime of data, use operations on this time column, for example: ``` -TTL date_time + interval +TTL time_column +TTL time_column + interval ``` -You can set the `interval` by any expression, returning the value of the `DateTime` data type. For example, you can use [time interval](../../query_language/operators.md#operators-datetime) operators. +To define `interval`, use [time interval](../../query_language/operators.md#operators-datetime) operators. ``` TTL date_time + INTERVAL 1 MONTH @@ -330,20 +330,20 @@ TTL date_time + INTERVAL 15 HOUR **Column TTL** -When the values in the column expire, ClickHouse replace them with the default values for the column data type. If all the column values in the data part become expired, ClickHouse deletes this column from the data part in a filesystem. +When the values in the column expire, ClickHouse replaces them with the default values for the column data type. If all the column values in the data part expire, ClickHouse deletes this column from the data part in a filesystem. -The `TTL` clause cannot be used for key columns. +The `TTL` clause can't be used for key columns. **Table TTL** -When some data in table expires, ClickHouse deletes all the corresponding rows. +When data in a table expires, ClickHouse deletes all corresponding rows. -**Cleaning up of Data** +**Removing Data** -Data with expired TTL is removed, when ClickHouse merges data parts. +Data with an expired TTL is removed when ClickHouse merges data parts. -When ClickHouse see that some data is expired, it performs off-schedule merge. To control frequency of such merges, you can set [merge_with_ttl_timeout](#mergetree_setting-merge_with_ttl_timeout). If it is too low, many off-schedule merges consume much resources. +When ClickHouse see that data is expired, it performs an off-schedule merge. To control the frequency of such merges, you can set [merge_with_ttl_timeout](#mergetree_setting-merge_with_ttl_timeout). If the value is too low, it will perform many off-schedule merges that may consume a lot of resources. -If you perform the `SELECT` query between merges you can get the expired data. To avoid it, use the [OPTIMIZE](../../query_language/misc.md#misc_operations-optimize) query before `SELECT`. +If you perform the `SELECT` query between merges, you may get expired data. To avoid it, use the [OPTIMIZE](../../query_language/misc.md#misc_operations-optimize) query before `SELECT`. [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/mergetree/) diff --git a/docs/en/query_language/create.md b/docs/en/query_language/create.md index 81d7982eb00e..e40a452cfe06 100644 --- a/docs/en/query_language/create.md +++ b/docs/en/query_language/create.md @@ -10,10 +10,10 @@ CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] [ENGINE = engine(.. - `IF NOT EXISTS` - If the `db_name` database already exists then: + If the `db_name` database already exists, then ClickHouse doesn't create a new database and: - - If clause is specified, ClickHouse doesn't create a new database and doesn't throw an exception. - - If clause is not specified, then ClickHouse doesn't create a new database and throw and exception. + - Doesn't throw an exception if clause is specified. + - Throws an exception if clause isn't specified. - `ON CLUSTER` @@ -23,7 +23,7 @@ CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] [ENGINE = engine(.. - [MySQL](../database_engines/mysql.md) - Allows to retrieve data from the remote MySQL server. + Allows you to retrieve data from the remote MySQL server. By default, ClickHouse uses its own [database engine](../database_engines/index.md). diff --git a/docs/ru/database_engines/index.md b/docs/ru/database_engines/index.md index 3190a79017f8..52aefe1bba2f 100644 --- a/docs/ru/database_engines/index.md +++ b/docs/ru/database_engines/index.md @@ -1,9 +1,10 @@ -# Database Engines +# Движки баз данных -Database engines provide working with tables. +Движки баз данных обеспечивают работу с таблицами. -By default, ClickHouse uses its native database engine which provides configurable [table engines](../operations/table_engines/index.md) and [SQL dialect](../query_language/syntax.md). +По умолчанию ClickHouse использует собственный движок баз данных, который поддерживает конфигурируемые [движки таблиц](../operations/table_engines/index.md) и [диалект SQL](../query_language/syntax.md). -Also you can use the following database engines: +Также можно использовать следующие движки баз данных: - [MySQL](mysql.md) + diff --git a/docs/ru/database_engines/mysql.md b/docs/ru/database_engines/mysql.md index fd5417ee27d7..acfb71d839cd 100644 --- a/docs/ru/database_engines/mysql.md +++ b/docs/ru/database_engines/mysql.md @@ -1,10 +1,10 @@ # MySQL -Allows to connect to some database on remote MySQL server and perform `INSERT` and `SELECT` queries with tables to exchange data between ClickHouse and MySQL. +Позволяет подключаться к базам данных на удалённом MySQL сервере и выполнять запросы `INSERT` и `SELECT` для обмена данными между ClickHouse и MySQL. -The `MySQL` database engine translate queries to the MySQL server, so you can perform operations such as `SHOW TABLES` or `SHOW CREATE TABLE`. +Движок баз данных `MySQL` транслирует запросы при передаче на сервер MySQL, что позволяет выполнять и другие виды запросов, например `SHOW TABLES` или `SHOW CREATE TABLE`. -You cannot perform with tables the following queries: +Не поддерживаемые виды запросов: - `ATTACH`/`DETACH` - `DROP` @@ -12,48 +12,45 @@ You cannot perform with tables the following queries: - `CREATE TABLE` - `ALTER` +## Создание базы данных -## Creating a Database - -``` sql +```sql CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] ENGINE = MySQL('host:port', 'database', 'user', 'password') ``` -**Engine Parameters** - -- `host:port` — MySQL server address. -- `database` — Remote database name. -- `user` — MySQL user. -- `password` — User password. - +**Параметры движка** -## Data Types Support +- `host:port` — адрес сервера MySQL. +- `database` — имя базы данных на удалённом сервере. +- `user` — пользователь MySQL. +- `password` — пароль пользователя. -MySQL | ClickHouse -------|------------ -UNSIGNED TINYINT | [UInt8](../data_types/int_uint.md) -TINYINT | [Int8](../data_types/int_uint.md) -UNSIGNED SMALLINT | [UInt16](../data_types/int_uint.md) -SMALLINT | [Int16](../data_types/int_uint.md) -UNSIGNED INT, UNSIGNED MEDIUMINT | [UInt32](../data_types/int_uint.md) -INT, MEDIUMINT | [Int32](../data_types/int_uint.md) -UNSIGNED BIGINT | [UInt64](../data_types/int_uint.md) -BIGINT | [Int64](../data_types/int_uint.md) -FLOAT | [Float32](../data_types/float.md) -DOUBLE | [Float64](../data_types/float.md) -DATE | [Date](../data_types/date.md) -DATETIME, TIMESTAMP | [DateTime](../data_types/datetime.md) -BINARY | [FixedString](../data_types/fixedstring.md) +## Поддержка типов данных -All other MySQL data types are converted into [String](../data_types/string.md). +| MySQL | ClickHouse | +| ------ | ------------ | +| UNSIGNED TINYINT | [UInt8](../data_types/int_uint.md) | +| TINYINT | [Int8](../data_types/int_uint.md) | +| UNSIGNED SMALLINT | [UInt16](../data_types/int_uint.md) | +| SMALLINT | [Int16](../data_types/int_uint.md) | +| UNSIGNED INT, UNSIGNED MEDIUMINT | [UInt32](../data_types/int_uint.md) | +| INT, MEDIUMINT | [Int32](../data_types/int_uint.md) | +| UNSIGNED BIGINT | [UInt64](../data_types/int_uint.md) | +| BIGINT | [Int64](../data_types/int_uint.md) | +| FLOAT | [Float32](../data_types/float.md) | +| DOUBLE | [Float64](../data_types/float.md) | +| DATE | [Date](../data_types/date.md) | +| DATETIME, TIMESTAMP | [DateTime](../data_types/datetime.md) | +| BINARY | [FixedString](../data_types/fixedstring.md) | -[Nullable](../data_types/nullable.md) data type is supported. +Все прочие типы данных преобразуются в [String](../data_types/string.md). +[Nullable](../data_types/nullable.md) поддержан. -## Examples of Use +## Примеры использования -Table in MySQL: +Таблица в MySQL: ``` mysql> USE test; @@ -77,14 +74,16 @@ mysql> select * from mysql_table; 1 row in set (0,00 sec) ``` -Database in ClickHouse, exchanging data with the MySQL server: +База данных в ClickHouse, позволяющая обмениваться данными с сервером MySQL: ```sql CREATE DATABASE mysql_db ENGINE = MySQL('localhost:3306', 'test', 'my_user', 'user_password') ``` + ```sql SHOW DATABASES ``` + ```text ┌─name─────┐ │ default │ @@ -92,31 +91,39 @@ SHOW DATABASES │ system │ └──────────┘ ``` + ```sql SHOW TABLES FROM mysql_db ``` + ```text ┌─name─────────┐ │ mysql_table │ └──────────────┘ ``` + ```sql SELECT * FROM mysql_db.mysql_table ``` + ```text ┌─int_id─┬─value─┐ │ 1 │ 2 │ └────────┴───────┘ ``` + ```sql INSERT INTO mysql_db.mysql_table VALUES (3,4) ``` + ```sql SELECT * FROM mysql_db.mysql_table ``` + ```text ┌─int_id─┬─value─┐ │ 1 │ 2 │ │ 3 │ 4 │ └────────┴───────┘ ``` + diff --git a/docs/ru/operations/table_engines/mergetree.md b/docs/ru/operations/table_engines/mergetree.md index 84ca187fd6aa..d47336c25934 100644 --- a/docs/ru/operations/table_engines/mergetree.md +++ b/docs/ru/operations/table_engines/mergetree.md @@ -40,14 +40,15 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [ORDER BY expr] [PRIMARY KEY expr] [SAMPLE BY expr] +[TTL expr] [SETTINGS name=value, ...] ``` Описание параметров запроса смотрите в [описании запроса](../../query_language/create.md). -**Секции запроса** +### Секции запроса -- `ENGINE` — Имя и параметры движка. `ENGINE = MergeTree()`. `MergeTree` не имеет параметров. +- `ENGINE` — имя и параметры движка. `ENGINE = MergeTree()`. `MergeTree` не имеет параметров. - `PARTITION BY` — [ключ партиционирования](custom_partitioning_key.md). @@ -66,22 +67,22 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] Если используется выражение для сэмплирования, то первичный ключ должен содержать его. Пример: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. -- `TTL` - выражение для задания времени хранения строк. +- `TTL` — выражение, определяющее длительность хранения строк. - Оно должно зависеть от стобца типа `Date` или `DateTime` и в качестве результата вычислять столбец типа `Date` или `DateTime`. Пример: - `TTL date + INTERVAL 1 DAY` + Должно зависеть от столбца `Date` или `DateTime` и возвращать столбец `Date` или `DateTime`. Пример:`TTL date + INTERVAL 1 DAY` - Подробнее смотрите в [TTL для стоблцов и таблиц](mergetree.md) + Дополнительные сведения смотрите в разделе [TTL для столбцов и таблиц](mergetree.md) - `SETTINGS` — дополнительные параметры, регулирующие поведение `MergeTree`: - `index_granularity` — гранулярность индекса. Число строк данных между «засечками» индекса. По умолчанию — 8192. Список всех доступных параметров можно посмотреть в [MergeTreeSettings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Storages/MergeTree/MergeTreeSettings.h). - `min_merge_bytes_to_use_direct_io` — минимальный объем данных, необходимый для прямого (небуферизованного) чтения/записи (direct I/O) на диск. При слиянии частей данных ClickHouse вычисляет общий объем хранения всех данных, подлежащих слиянию. Если общий объем хранения всех данных для чтения превышает `min_bytes_to_use_direct_io` байт, тогда ClickHouse использует флаг `O_DIRECT` при чтении данных с диска. Если `min_merge_bytes_to_use_direct_io = 0`, тогда прямой ввод-вывод отключен. Значение по умолчанию: `10 * 1024 * 1024 * 1024` байт. + - `merge_with_ttl_timeout` - Минимальное время в секундах для повторного выполнения слияний с TTL. По умолчанию - 86400 (1 день). **Пример задания секций** -``` +```sql ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity=8192 ``` @@ -109,7 +110,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] - `date-column` — имя столбца с типом [Date](../../data_types/date.md). На основе этого столбца ClickHouse автоматически создаёт партиции по месяцам. Имена партиций имеют формат `"YYYYMM"`. - `sampling_expression` — выражение для сэмплирования. -- `(primary, key)` — первичный ключ. Тип — [Tuple()](../../data_types/tuple.md- `index_granularity` — гранулярность индекса. Число строк данных между «засечками» индекса. Для большинства задач подходит значение 8192. +- `(primary, key)` — первичный ключ. Тип — [Tuple()](../../data_types/tuple.md) +- `index_granularity` — гранулярность индекса. Число строк данных между «засечками» индекса. Для большинства задач подходит значение 8192. **Пример** @@ -118,6 +120,7 @@ MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID) ``` Движок `MergeTree` сконфигурирован таким же образом, как и в примере выше для основного способа конфигурирования движка. + ## Хранение данных @@ -298,13 +301,42 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT Чтения из таблицы автоматически распараллеливаются. +## TTL для столбцов и таблиц {#table_engine-mergetree-ttl} + +Определяет время жизни значений. + +Секция `TTL` может быть установлена как для всей таблицы, так и для каждого отдельного столбца. Если установлены оба `TTL`, то ClickHouse использует тот, что истекает раньше. + +Таблица должна иметь столбец типа [Date](../../data_types/date.md) или [DateTime](../../data_types/datetime.md). Для установки времени жизни данных, следует использовать операцию со столбцом с временем, например: + +``` +TTL time_column +TTL time_column + interval +``` + +Чтобы задать `interval`, используйте операторы [интервала времени](../../query_language/operators.md#operators-datetime). + +``` +TTL date_time + INTERVAL 1 MONTH +TTL date_time + INTERVAL 15 HOUR +``` + +**TTL столбца** + +Когда срок действия значений в столбце истечет, ClickHouse заменит их значениями по умолчанию для типа данных столбца. Если срок действия всех значений столбцов в части данных истек, ClickHouse удаляет столбец из куска данных в файловой системе. + +Секцию `TTL` нельзя использовать для ключевых столбцов. + +**TTL таблицы** + +Когда некоторые данные в таблице устаревают, ClickHouse удаляет все соответствующие строки. -## TTL для столбцов и таблиц +**Удаление данных** -Данные с истекшим TTL удаляются во время слияний. +Данные с истекшим TTL удаляются, когда ClickHouse мёржит куски данных. -Если TTL указан для столбца, то когда он истекает, значение заменяется на значение по умолчанию. Если все значения столбца обнулены в куске, то данные этого столбца удаляются с диска в куске. Если TTL указан для таблицы, то когда он истекает, удаляется строка. +Когда ClickHouse видит, что некоторые данные устарели, он выполняет внеплановые мёржи. Для управление частотой подобных мёржей, можно задать настройку [merge_with_ttl_timeout](#mergetree_setting-merge_with_ttl_timeout). Если её значение слишком низкое, придется выполнять много внеплановых мёржей, которые могут начать потреблять значительную долю ресурсов сервера. -Когда истекает TTL для какого-нибудь значения или строки в куске, назначается внеочередное слияние. Чтобы контролировать частоту слияний с TTL, вы можете задать настройку `merge_with_ttl_timeout`. Если ее значение слишком мало, то будет происходить слишком много внеочередных слияний и мало обычных, вследствие чего может ухудшиться производительность. +Если вы выполните запрос `SELECT` между слияниями вы можете получить устаревшие данные. Чтобы избежать этого используйте запрос [OPTIMIZE](../../query_language/misc.md#misc_operations-optimize) перед `SELECT`. [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/mergetree/) diff --git a/docs/ru/query_language/create.md b/docs/ru/query_language/create.md index be75bb9ce514..ebbef603390d 100644 --- a/docs/ru/query_language/create.md +++ b/docs/ru/query_language/create.md @@ -1,13 +1,30 @@ -## CREATE DATABASE +## CREATE DATABASE {#query_language-create-database} -Создание базы данных db\_name. +Создает базу данных. ```sql -CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] +CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] [ENGINE = engine(...)] ``` -`База данных` - это просто директория для таблиц. -Если написано `IF NOT EXISTS`, то запрос не будет возвращать ошибку, если база данных уже существует. +### Секции + +- `IF NOT EXISTS` + + Если база данных с именем `db_name` уже существует, то ClickHouse не создаёт базу данных и: + - Не генерирует исключение, если секция указана. + - Генерирует исключение, если секция не указана. + +- `ON CLUSTER` + + ClickHouse создаёт базу данных `db_name` на всех серверах указанного кластера. + +- `ENGINE` + + - [MySQL](../database_engines/mysql.md) + + Позволяет получать данные с удаленного сервера MySQL. + + По умолчанию ClickHouse использует собственный [движок баз данных](../database_engines/index.md). ## CREATE TABLE {#create-table-query} @@ -48,7 +65,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ... Во всех случаях, если указано `IF NOT EXISTS`, то запрос не будет возвращать ошибку, если таблица уже существует. В этом случае, запрос будет ничего не делать. -После секции `ENGINE` в запросе могут использоваться и другие секции в зависимости от движка. Подробную документацию по созданию таблиц смотрите в описаниях [движков](../operations/table_engines/index.md#table_engines). +После секции `ENGINE` в запросе могут использоваться и другие секции в зависимости от движка. Подробную документацию по созданию таблиц смотрите в описаниях [движков таблиц](../operations/table_engines/index.md#table_engines). ### Значения по умолчанию {#create-default-values} @@ -88,11 +105,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ... ### Выражение для TTL -Может быть указано только для таблиц семейства MergeTree. Выражение для указания времени хранения значений. Оно должно зависеть от стобца типа `Date` или `DateTime` и в качестве результата вычислять столбец типа `Date` или `DateTime`. Пример: - `TTL date + INTERVAL 1 DAY` - -Нельзя указывать TTL для ключевых столбцов. Подробнее смотрите в [TTL для стоблцов и таблиц](../operations/table_engines/mergetree.md) - +Определяет время хранения значений. Может быть указано только для таблиц семейства MergeTree. Подробнее смотрите в [TTL для столбцов и таблиц](../operations/table_engines/mergetree.md#table_engine-mergetree-ttl). ## Форматы сжатия для колонок diff --git a/docs/ru/query_language/operators.md b/docs/ru/query_language/operators.md index a493e555af74..de38bb5b1938 100644 --- a/docs/ru/query_language/operators.md +++ b/docs/ru/query_language/operators.md @@ -65,13 +65,13 @@ `a GLOBAL NOT IN ...` - функция `globalNotIn(a, b)` -## Оператор для работы с датами и временем +## Оператор для работы с датами и временем {#operators-datetime} ``` sql EXTRACT(part FROM date); ``` -Позволяет извлечь отдельные части из переданной даты. Например, можно получить месяц из даты, или минуты из времени. +Позволяет извлечь отдельные части из переданной даты. Например, можно получить месяц из даты, или минуты из времени. В параметре `part` указывается, какой фрагмент даты нужно получить. Доступные значения: @@ -99,8 +99,8 @@ SELECT EXTRACT(YEAR FROM toDate('2017-06-15')); ``` sql CREATE TABLE test.Orders ( - OrderId UInt64, - OrderName String, + OrderId UInt64, + OrderName String, OrderDate DateTime ) ENGINE = Log; @@ -110,11 +110,11 @@ ENGINE = Log; INSERT INTO test.Orders VALUES (1, 'Jarlsberg Cheese', toDateTime('2008-10-11 13:23:44')); ``` ``` sql -SELECT - toYear(OrderDate) AS OrderYear, - toMonth(OrderDate) AS OrderMonth, - toDayOfMonth(OrderDate) AS OrderDay, - toHour(OrderDate) AS OrderHour, +SELECT + toYear(OrderDate) AS OrderYear, + toMonth(OrderDate) AS OrderMonth, + toDayOfMonth(OrderDate) AS OrderDay, + toHour(OrderDate) AS OrderHour, toMinute(OrderDate) AS OrderMinute, toSecond(OrderDate) AS OrderSecond FROM test.Orders; From 047a14a1893910b37b0fe287d012a715abd84f7a Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 14 Aug 2019 19:53:30 +0300 Subject: [PATCH 105/161] one more minor refactoring --- dbms/src/Interpreters/ActionsVisitor.h | 10 ++++++---- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 4 ++-- dbms/src/Interpreters/ExpressionAnalyzer.h | 2 +- dbms/src/Interpreters/GlobalSubqueriesVisitor.h | 1 - 4 files changed, 9 insertions(+), 8 deletions(-) diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h index 9841c8e9df8b..4d03f758f617 100644 --- a/dbms/src/Interpreters/ActionsVisitor.h +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -54,7 +54,6 @@ struct ScopeStack /// Collect ExpressionAction from AST. Returns PreparedSets and SubqueriesForSets too. -/// After AST is visited source ExpressionActions should be updated with popActionsLevel() method. class ActionsVisitor { public: @@ -63,9 +62,11 @@ class ActionsVisitor PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_ = nullptr); - void visit(const ASTPtr & ast); - - ExpressionActionsPtr popActionsLevel() { return actions_stack.popLevel(); } + void visit(const ASTPtr & ast, ExpressionActionsPtr & actions) + { + visit(ast); + actions = actions_stack.popLevel(); + } private: const Context & context; @@ -81,6 +82,7 @@ class ActionsVisitor std::ostream * ostr; ScopeStack actions_stack; + void visit(const ASTPtr & ast); SetPtr makeSet(const ASTFunction * node, const Block & sample_block); }; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 1990c4270203..2a96a58c4f29 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -24,6 +24,7 @@ #include #include +#include #include #include #include @@ -320,8 +321,7 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, ActionsVisitor actions_visitor(context, settings.size_limits_for_set, subquery_depth, sourceColumns(), actions, prepared_sets, subqueries_for_sets, no_subqueries, only_consts, !isRemoteStorage(), log.stream()); - actions_visitor.visit(ast); - actions = actions_visitor.popActionsLevel(); + actions_visitor.visit(ast, actions); } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index d40b1c219b50..2e70662f1964 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -2,9 +2,9 @@ #include #include -#include #include #include +#include #include #include diff --git a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h index 1622c27f62f6..583ec026af08 100644 --- a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include From e83482b51220991c95060d3df44d8938a5f9afc8 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 14 Aug 2019 20:21:45 +0300 Subject: [PATCH 106/161] metric_log.xml --- dbms/programs/server/config.d/metric_log.xml | 8 ++++++++ dbms/programs/server/config.xml | 1 + 2 files changed, 9 insertions(+) create mode 100644 dbms/programs/server/config.d/metric_log.xml diff --git a/dbms/programs/server/config.d/metric_log.xml b/dbms/programs/server/config.d/metric_log.xml new file mode 100644 index 000000000000..0ca9f1624169 --- /dev/null +++ b/dbms/programs/server/config.d/metric_log.xml @@ -0,0 +1,8 @@ + + + system + metric_log
+ 7500 + 1000 +
+
diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index 28901a3b8971..44b3c25c19d7 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -329,6 +329,7 @@ text_log
7500 + --> diff --git a/docs/en/query_language/agg_functions/reference.md b/docs/en/query_language/agg_functions/reference.md index 350803f5aefb..3e3e1e2cc01e 100644 --- a/docs/en/query_language/agg_functions/reference.md +++ b/docs/en/query_language/agg_functions/reference.md @@ -650,7 +650,7 @@ The function takes a variable number of parameters. Parameters can be `Tuple`, ` - [uniqHLL12](#agg_function-uniqhll12) -## groupArray(x), groupArray(max_size)(x) +## groupArray(x), groupArray(max_size)(x) {#agg_function-grouparray} Creates an array of argument values. Values can be added to the array in any (indeterminate) order. From b1d1a23318758c74bb7a883bf39022db584474b5 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Thu, 15 Aug 2019 13:10:59 +0300 Subject: [PATCH 111/161] DOCAPI-7431: Formatted queries docs (#6490) --- docs/en/interfaces/cli.md | 25 +++++++++++++++++++++++++ docs/en/interfaces/http.md | 9 +++++++++ 2 files changed, 34 insertions(+) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 09c8baed5f51..9f9448f27c8f 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -65,6 +65,31 @@ You can cancel a long query by pressing Ctrl+C. However, you will still need to The command-line client allows passing external data (external temporary tables) for querying. For more information, see the section "External data for query processing". +### Queries with Parameters {#cli-queries-with-parameters} + +You can create a query with parameters, and pass values for these parameters with the parameters of the client app. For example: + +```bash +clickhouse-client --param_parName="[1, 2]" -q "SELECT * FROM table WHERE a = {parName:Array(UInt16)}" +``` + +#### Syntax of a Query {#cli-queries-with-parameters-syntax} + +Format a query by the standard method. Values that you want to put into the query from the app parameters place in braces and format as follows: + +``` +{:} +``` + +- `name` — Identifier of a placeholder that should be used in app parameter as `--param_name = value`. +- `data type` — A data type of app parameter value. For example, data structure like `(integer, ('string', integer))` can have a data type `Tuple(UInt8, Tuple(String, UInt8))` (also you can use another [integer](../data_types/int_uint.md) types). + +#### Example + +```bash +clickhouse-client --param_tuple_in_tuple="(10, ('dt', 10))" -q "SELECT * FROM table WHERE val = {tuple_in_tuple:Tuple(UInt8, Tuple(String, UInt8))}" +``` + ## Configuring {#interfaces_cli_configuration} You can pass parameters to `clickhouse-client` (all parameters have a default value) using: diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 37da364942c4..ef32b101b719 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -244,5 +244,14 @@ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&wa Use buffering to avoid situations where a query processing error occurred after the response code and HTTP headers were sent to the client. In this situation, an error message is written at the end of the response body, and on the client side, the error can only be detected at the parsing stage. +### Queries with Parameters {#cli-queries-with-parameters} + +You can create a query with parameters, and pass values for these parameters with the parameters of the HTTP request. For more information, see [CLI Formatted Queries](cli.md#cli-queries-with-parameters). + +### Example + +```bash +curl -sS "
?param_id=2¶m_phrase=test" -d "SELECT * FROM table WHERE int_column = {id:UInt8} and string_column = {phrase:String}" +``` [Original article](https://clickhouse.yandex/docs/en/interfaces/http_interface/) From 6ef3b5f9c758962f6175405da46170ed5555c83a Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Thu, 15 Aug 2019 13:15:04 +0300 Subject: [PATCH 112/161] fix usage of global syntax_result in optimizeReadInOrder --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 9a6c15f87f61..6786f64795d6 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -666,7 +666,7 @@ static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & c static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, const ASTSelectQuery & query, - const Context & context, const SyntaxAnalyzerResultPtr & syntax_result) + const Context & context, const SyntaxAnalyzerResultPtr & global_syntax_result) { if (!merge_tree.hasSortingKey()) return {}; @@ -680,7 +680,7 @@ static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, cons for (size_t i = 0; i < prefix_size; ++i) { - if (syntax_result->array_join_result_to_source.count(order_descr[i].column_name)) + if (global_syntax_result->array_join_result_to_source.count(order_descr[i].column_name)) break; /// Optimize in case of exact match with order key element @@ -690,9 +690,9 @@ static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, cons prefix_order_descr.push_back(order_descr[i]); else { - const auto & ast = query.orderBy()->children[i]; - ExpressionActionsPtr actions; - actions = ExpressionAnalyzer(ast->children.at(0), syntax_result, context).getActions(true); + auto ast = query.orderBy()->children[i]->children.at(0); + auto syntax_result = SyntaxAnalyzer(context).analyze(ast, global_syntax_result->required_source_columns); + auto actions = ExpressionAnalyzer(ast, syntax_result, context).getActions(true); const auto & input_columns = actions->getRequiredColumnsWithTypes(); if (input_columns.size() != 1 || input_columns.front().name != sorting_key_columns[i]) From 1b500ade459d51845fa881f250ddd58d07b8cfe8 Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 15 Aug 2019 14:22:19 +0300 Subject: [PATCH 113/161] remove duplicated include --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 86cf8cd0d689..eaba3d568e37 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -56,7 +56,6 @@ #include #include -#include #include #include #include From 994f9f3cc6f21beacf0e0f643daf62cd0beda5af Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 15 Aug 2019 16:54:59 +0300 Subject: [PATCH 114/161] unify ActionsVisitor: rewrite as InDepthNodeVisitor --- dbms/src/Interpreters/ActionsVisitor.cpp | 523 ++++++++++--------- dbms/src/Interpreters/ActionsVisitor.h | 82 ++- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 7 +- 3 files changed, 326 insertions(+), 286 deletions(-) diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index 523343a288e7..7c6f97d5ed55 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -1,3 +1,5 @@ +#include + #include #include @@ -19,8 +21,6 @@ #include #include -#include -#include #include #include #include @@ -228,346 +228,351 @@ const Block & ScopeStack::getSampleBlock() const return stack.back().actions->getSampleBlock(); } +struct CachedColumnName +{ + String cached; + + const String & get(const ASTPtr & ast) + { + if (cached.empty()) + cached = ast->getColumnName(); + return cached; + } +}; + +bool ActionsMatcher::needChildVisit(const ASTPtr & node, const ASTPtr & child) +{ + /// Visit children themself + if (node->as() || + node->as() || + node->as()) + return false; + + /// Do not go to FROM, JOIN, UNION. + if (child->as() || + child->as()) + return false; + + return true; +} -ActionsVisitor::ActionsVisitor( - const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, - const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions, - PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, - bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_) -: context(context_), - set_size_limit(set_size_limit_), - subquery_depth(subquery_depth_), - source_columns(source_columns_), - prepared_sets(prepared_sets_), - subqueries_for_sets(subqueries_for_sets_), - no_subqueries(no_subqueries_), - only_consts(only_consts_), - no_storage_or_local(no_storage_or_local_), - visit_depth(0), - ostr(ostr_), - actions_stack(actions, context) +void ActionsMatcher::visit(const ASTPtr & ast, Data & data) { + if (const auto * identifier = ast->as()) + visit(*identifier, ast, data); + else if (const auto * node = ast->as()) + visit(*node, ast, data); + else if (const auto * literal = ast->as()) + visit(*literal, ast, data); } -void ActionsVisitor::visit(const ASTPtr & ast) +void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr & ast, Data & data) { - DumpASTNode dump(*ast, ostr, visit_depth, "getActions"); + CachedColumnName column_name; - String ast_column_name; - auto getColumnName = [&ast, &ast_column_name]() + if (!data.only_consts && !data.actions_stack.getSampleBlock().has(column_name.get(ast))) { - if (ast_column_name.empty()) - ast_column_name = ast->getColumnName(); + /// The requested column is not in the block. + /// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY. - return ast_column_name; - }; + bool found = false; + for (const auto & column_name_type : data.source_columns) + if (column_name_type.name == column_name.get(ast)) + found = true; - /// If the result of the calculation already exists in the block. - if ((ast->as() || ast->as()) && actions_stack.getSampleBlock().has(getColumnName())) - return; + if (found) + throw Exception("Column " + column_name.get(ast) + " is not under aggregate function and not in GROUP BY.", + ErrorCodes::NOT_AN_AGGREGATE); - if (const auto * identifier = ast->as()) - { - if (!only_consts && !actions_stack.getSampleBlock().has(getColumnName())) - { - /// The requested column is not in the block. - /// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY. + /// Special check for WITH statement alias. Add alias action to be able to use this alias. + if (identifier.prefer_alias_to_column_name && !identifier.alias.empty()) + data.actions_stack.addAction(ExpressionAction::addAliases({{identifier.name, identifier.alias}})); + } +} - bool found = false; - for (const auto & column_name_type : source_columns) - if (column_name_type.name == getColumnName()) - found = true; +void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data) +{ + CachedColumnName column_name; - if (found) - throw Exception("Column " + getColumnName() + " is not under aggregate function and not in GROUP BY.", - ErrorCodes::NOT_AN_AGGREGATE); + if (data.hasColumn(column_name.get(ast))) + return; - /// Special check for WITH statement alias. Add alias action to be able to use this alias. - if (identifier->prefer_alias_to_column_name && !identifier->alias.empty()) - actions_stack.addAction(ExpressionAction::addAliases({{identifier->name, identifier->alias}})); - } - } - else if (const auto * node = ast->as()) + if (node.name == "lambda") + throw Exception("Unexpected lambda expression", ErrorCodes::UNEXPECTED_EXPRESSION); + + /// Function arrayJoin. + if (node.name == "arrayJoin") { - if (node->name == "lambda") - throw Exception("Unexpected lambda expression", ErrorCodes::UNEXPECTED_EXPRESSION); + if (node.arguments->children.size() != 1) + throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH); - /// Function arrayJoin. - if (node->name == "arrayJoin") + ASTPtr arg = node.arguments->children.at(0); + visit(arg, data); + if (!data.only_consts) { - if (node->arguments->children.size() != 1) - throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH); + String result_name = column_name.get(ast); + data.actions_stack.addAction(ExpressionAction::copyColumn(arg->getColumnName(), result_name)); + NameSet joined_columns; + joined_columns.insert(result_name); + data.actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns, false, data.context)); + } - ASTPtr arg = node->arguments->children.at(0); - visit(arg); - if (!only_consts) - { - String result_name = getColumnName(); - actions_stack.addAction(ExpressionAction::copyColumn(arg->getColumnName(), result_name)); - NameSet joined_columns; - joined_columns.insert(result_name); - actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns, false, context)); - } + return; + } - return; - } + SetPtr prepared_set; + if (functionIsInOrGlobalInOperator(node.name)) + { + /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). + visit(node.arguments->children.at(0), data); - SetPtr prepared_set; - if (functionIsInOrGlobalInOperator(node->name)) + if (!data.no_subqueries) { - /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). - visit(node->arguments->children.at(0)); - - if (!no_subqueries) - { - /// Transform tuple or subquery into a set. - prepared_set = makeSet(node, actions_stack.getSampleBlock()); - } - else + /// Transform tuple or subquery into a set. + prepared_set = makeSet(node, data); + } + else + { + if (!data.only_consts) { - if (!only_consts) - { - /// We are in the part of the tree that we are not going to compute. You just need to define types. - /// Do not subquery and create sets. We treat "IN" as "ignoreExceptNull" function. + /// We are in the part of the tree that we are not going to compute. You just need to define types. + /// Do not subquery and create sets. We treat "IN" as "ignoreExceptNull" function. - actions_stack.addAction(ExpressionAction::applyFunction( - FunctionFactory::instance().get("ignoreExceptNull", context), - { node->arguments->children.at(0)->getColumnName() }, - getColumnName())); - } - return; + data.actions_stack.addAction(ExpressionAction::applyFunction( + FunctionFactory::instance().get("ignoreExceptNull", data.context), + { node.arguments->children.at(0)->getColumnName() }, + column_name.get(ast))); } - } - - /// A special function `indexHint`. Everything that is inside it is not calculated - /// (and is used only for index analysis, see KeyCondition). - if (node->name == "indexHint") - { - actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName( - ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared(), - getColumnName()))); return; } + } - if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name)) - return; + /// A special function `indexHint`. Everything that is inside it is not calculated + /// (and is used only for index analysis, see KeyCondition). + if (node.name == "indexHint") + { + data.actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName( + ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared(), + column_name.get(ast)))); + return; + } - /// Context object that we pass to function should live during query. - const Context & function_context = context.hasQueryContext() - ? context.getQueryContext() - : context; + if (AggregateFunctionFactory::instance().isAggregateFunctionName(node.name)) + return; - FunctionBuilderPtr function_builder; - try - { - function_builder = FunctionFactory::instance().get(node->name, function_context); - } - catch (DB::Exception & e) - { - auto hints = AggregateFunctionFactory::instance().getHints(node->name); - if (!hints.empty()) - e.addMessage("Or unknown aggregate function " + node->name + ". Maybe you meant: " + toString(hints)); - e.rethrow(); - } + /// Context object that we pass to function should live during query. + const Context & function_context = data.context.hasQueryContext() + ? data.context.getQueryContext() + : data.context; - Names argument_names; - DataTypes argument_types; - bool arguments_present = true; + FunctionBuilderPtr function_builder; + try + { + function_builder = FunctionFactory::instance().get(node.name, function_context); + } + catch (DB::Exception & e) + { + auto hints = AggregateFunctionFactory::instance().getHints(node.name); + if (!hints.empty()) + e.addMessage("Or unknown aggregate function " + node.name + ". Maybe you meant: " + toString(hints)); + e.rethrow(); + } - /// If the function has an argument-lambda expression, you need to determine its type before the recursive call. - bool has_lambda_arguments = false; + Names argument_names; + DataTypes argument_types; + bool arguments_present = true; - for (size_t arg = 0; arg < node->arguments->children.size(); ++arg) - { - auto & child = node->arguments->children[arg]; - auto child_column_name = child->getColumnName(); + /// If the function has an argument-lambda expression, you need to determine its type before the recursive call. + bool has_lambda_arguments = false; - const auto * lambda = child->as(); - if (lambda && lambda->name == "lambda") - { - /// If the argument is a lambda expression, just remember its approximate type. - if (lambda->arguments->children.size() != 2) - throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + for (size_t arg = 0; arg < node.arguments->children.size(); ++arg) + { + auto & child = node.arguments->children[arg]; + auto child_column_name = child->getColumnName(); - const auto * lambda_args_tuple = lambda->arguments->children.at(0)->as(); + const auto * lambda = child->as(); + if (lambda && lambda->name == "lambda") + { + /// If the argument is a lambda expression, just remember its approximate type. + if (lambda->arguments->children.size() != 2) + throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") - throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH); + const auto * lambda_args_tuple = lambda->arguments->children.at(0)->as(); - has_lambda_arguments = true; - argument_types.emplace_back(std::make_shared(DataTypes(lambda_args_tuple->arguments->children.size()))); - /// Select the name in the next cycle. - argument_names.emplace_back(); - } - else if (functionIsInOrGlobalInOperator(node->name) && arg == 1 && prepared_set) - { - ColumnWithTypeAndName column; - column.type = std::make_shared(); + if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") + throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH); - /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, - /// so that sets with the same literal representation do not fuse together (they can have different types). - if (!prepared_set->empty()) - column.name = getUniqueName(actions_stack.getSampleBlock(), "__set"); - else - column.name = child_column_name; + has_lambda_arguments = true; + argument_types.emplace_back(std::make_shared(DataTypes(lambda_args_tuple->arguments->children.size()))); + /// Select the name in the next cycle. + argument_names.emplace_back(); + } + else if (functionIsInOrGlobalInOperator(node.name) && arg == 1 && prepared_set) + { + ColumnWithTypeAndName column; + column.type = std::make_shared(); - if (!actions_stack.getSampleBlock().has(column.name)) - { - column.column = ColumnSet::create(1, prepared_set); + /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, + /// so that sets with the same literal representation do not fuse together (they can have different types). + if (!prepared_set->empty()) + column.name = getUniqueName(data.actions_stack.getSampleBlock(), "__set"); + else + column.name = child_column_name; - actions_stack.addAction(ExpressionAction::addColumn(column)); - } + if (!data.actions_stack.getSampleBlock().has(column.name)) + { + column.column = ColumnSet::create(1, prepared_set); - argument_types.push_back(column.type); - argument_names.push_back(column.name); + data.actions_stack.addAction(ExpressionAction::addColumn(column)); + } + + argument_types.push_back(column.type); + argument_names.push_back(column.name); + } + else + { + /// If the argument is not a lambda expression, call it recursively and find out its type. + visit(child, data); + std::string name = child_column_name; + if (data.actions_stack.getSampleBlock().has(name)) + { + argument_types.push_back(data.actions_stack.getSampleBlock().getByName(name).type); + argument_names.push_back(name); } else { - /// If the argument is not a lambda expression, call it recursively and find out its type. - visit(child); - std::string name = child_column_name; - if (actions_stack.getSampleBlock().has(name)) - { - argument_types.push_back(actions_stack.getSampleBlock().getByName(name).type); - argument_names.push_back(name); - } + if (data.only_consts) + arguments_present = false; else - { - if (only_consts) - arguments_present = false; - else - throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER); - } + throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER); } } + } - if (only_consts && !arguments_present) - return; + if (data.only_consts && !arguments_present) + return; + + if (has_lambda_arguments && !data.only_consts) + { + function_builder->getLambdaArgumentTypes(argument_types); - if (has_lambda_arguments && !only_consts) + /// Call recursively for lambda expressions. + for (size_t i = 0; i < node.arguments->children.size(); ++i) { - function_builder->getLambdaArgumentTypes(argument_types); + ASTPtr child = node.arguments->children[i]; - /// Call recursively for lambda expressions. - for (size_t i = 0; i < node->arguments->children.size(); ++i) + const auto * lambda = child->as(); + if (lambda && lambda->name == "lambda") { - ASTPtr child = node->arguments->children[i]; + const DataTypeFunction * lambda_type = typeid_cast(argument_types[i].get()); + const auto * lambda_args_tuple = lambda->arguments->children.at(0)->as(); + const ASTs & lambda_arg_asts = lambda_args_tuple->arguments->children; + NamesAndTypesList lambda_arguments; - const auto * lambda = child->as(); - if (lambda && lambda->name == "lambda") + for (size_t j = 0; j < lambda_arg_asts.size(); ++j) { - const DataTypeFunction * lambda_type = typeid_cast(argument_types[i].get()); - const auto * lambda_args_tuple = lambda->arguments->children.at(0)->as(); - const ASTs & lambda_arg_asts = lambda_args_tuple->arguments->children; - NamesAndTypesList lambda_arguments; - - for (size_t j = 0; j < lambda_arg_asts.size(); ++j) - { - auto opt_arg_name = tryGetIdentifierName(lambda_arg_asts[j]); - if (!opt_arg_name) - throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); - - lambda_arguments.emplace_back(*opt_arg_name, lambda_type->getArgumentTypes()[j]); - } - - actions_stack.pushLevel(lambda_arguments); - visit(lambda->arguments->children.at(1)); - ExpressionActionsPtr lambda_actions = actions_stack.popLevel(); - - String result_name = lambda->arguments->children.at(1)->getColumnName(); - lambda_actions->finalize(Names(1, result_name)); - DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; - - Names captured; - Names required = lambda_actions->getRequiredColumns(); - for (const auto & required_arg : required) - if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end()) - captured.push_back(required_arg); - - /// We can not name `getColumnName()`, - /// because it does not uniquely define the expression (the types of arguments can be different). - String lambda_name = getUniqueName(actions_stack.getSampleBlock(), "__lambda"); - - auto function_capture = std::make_shared( - lambda_actions, captured, lambda_arguments, result_type, result_name); - actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name)); - - argument_types[i] = std::make_shared(lambda_type->getArgumentTypes(), result_type); - argument_names[i] = lambda_name; + auto opt_arg_name = tryGetIdentifierName(lambda_arg_asts[j]); + if (!opt_arg_name) + throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); + + lambda_arguments.emplace_back(*opt_arg_name, lambda_type->getArgumentTypes()[j]); } + + data.actions_stack.pushLevel(lambda_arguments); + visit(lambda->arguments->children.at(1), data); + ExpressionActionsPtr lambda_actions = data.actions_stack.popLevel(); + + String result_name = lambda->arguments->children.at(1)->getColumnName(); + lambda_actions->finalize(Names(1, result_name)); + DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; + + Names captured; + Names required = lambda_actions->getRequiredColumns(); + for (const auto & required_arg : required) + if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end()) + captured.push_back(required_arg); + + /// We can not name `getColumnName()`, + /// because it does not uniquely define the expression (the types of arguments can be different). + String lambda_name = getUniqueName(data.actions_stack.getSampleBlock(), "__lambda"); + + auto function_capture = std::make_shared( + lambda_actions, captured, lambda_arguments, result_type, result_name); + data.actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name)); + + argument_types[i] = std::make_shared(lambda_type->getArgumentTypes(), result_type); + argument_names[i] = lambda_name; } } + } - if (only_consts) + if (data.only_consts) + { + for (const auto & argument_name : argument_names) { - for (const auto & argument_name : argument_names) + if (!data.actions_stack.getSampleBlock().has(argument_name)) { - if (!actions_stack.getSampleBlock().has(argument_name)) - { - arguments_present = false; - break; - } + arguments_present = false; + break; } } - - if (arguments_present) - { - actions_stack.addAction( - ExpressionAction::applyFunction(function_builder, argument_names, getColumnName())); - } } - else if (const auto * literal = ast->as()) - { - DataTypePtr type = applyVisitor(FieldToDataType(), literal->value); - - ColumnWithTypeAndName column; - column.column = type->createColumnConst(1, convertFieldToType(literal->value, *type)); - column.type = type; - column.name = getColumnName(); - actions_stack.addAction(ExpressionAction::addColumn(column)); - } - else + if (arguments_present) { - for (auto & child : ast->children) - { - /// Do not go to FROM, JOIN, UNION. - if (!child->as() && !child->as()) - visit(child); - } + data.actions_stack.addAction( + ExpressionAction::applyFunction(function_builder, argument_names, column_name.get(ast))); } } -SetPtr ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_block) +void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data) +{ + CachedColumnName column_name; + + if (data.hasColumn(column_name.get(ast))) + return; + + DataTypePtr type = applyVisitor(FieldToDataType(), literal.value); + + ColumnWithTypeAndName column; + column.column = type->createColumnConst(1, convertFieldToType(literal.value, *type)); + column.type = type; + column.name = column_name.get(ast); + + data.actions_stack.addAction(ExpressionAction::addColumn(column)); +} + +SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data) { /** You need to convert the right argument to a set. * This can be a table name, a value, a value enumeration, or a subquery. * The enumeration of values is parsed as a function `tuple`. */ - const IAST & args = *node->arguments; + const IAST & args = *node.arguments; const ASTPtr & arg = args.children.at(1); + const Block & sample_block = data.actions_stack.getSampleBlock(); /// If the subquery or table name for SELECT. const auto * identifier = arg->as(); if (arg->as() || identifier) { auto set_key = PreparedSetKey::forSubquery(*arg); - if (prepared_sets.count(set_key)) - return prepared_sets.at(set_key); + if (data.prepared_sets.count(set_key)) + return data.prepared_sets.at(set_key); /// A special case is if the name of the table is specified on the right side of the IN statement, /// and the table has the type Set (a previously prepared set). if (identifier) { DatabaseAndTableWithAlias database_table(*identifier); - StoragePtr table = context.tryGetTable(database_table.database, database_table.table); + StoragePtr table = data.context.tryGetTable(database_table.database, database_table.table); if (table) { StorageSet * storage_set = dynamic_cast(table.get()); if (storage_set) { - prepared_sets[set_key] = storage_set->getSet(); + data.prepared_sets[set_key] = storage_set->getSet(); return storage_set->getSet(); } } @@ -576,25 +581,25 @@ SetPtr ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bl /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. String set_id = arg->getColumnName(); - SubqueryForSet & subquery_for_set = subqueries_for_sets[set_id]; + SubqueryForSet & subquery_for_set = data.subqueries_for_sets[set_id]; /// If you already created a Set with the same subquery / table. if (subquery_for_set.set) { - prepared_sets[set_key] = subquery_for_set.set; + data.prepared_sets[set_key] = subquery_for_set.set; return subquery_for_set.set; } - SetPtr set = std::make_shared(set_size_limit, false); + SetPtr set = std::make_shared(data.set_size_limit, false); /** The following happens for GLOBAL INs: * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, * in the subquery_for_set object, this subquery is set as source and the temporary table _data1 as the table. * - this function shows the expression IN_data1. */ - if (!subquery_for_set.source && no_storage_or_local) + if (!subquery_for_set.source && data.no_storage_or_local) { - auto interpreter = interpretSubquery(arg, context, subquery_depth, {}); + auto interpreter = interpretSubquery(arg, data.context, data.subquery_depth, {}); subquery_for_set.source = std::make_shared( interpreter->getSampleBlock(), [interpreter]() mutable { return interpreter->execute().in; }); @@ -627,13 +632,13 @@ SetPtr ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bl } subquery_for_set.set = set; - prepared_sets[set_key] = set; + data.prepared_sets[set_key] = set; return set; } else { /// An explicit enumeration of values in parentheses. - return makeExplicitSet(node, sample_block, false, context, set_size_limit, prepared_sets); + return makeExplicitSet(&node, sample_block, false, data.context, data.set_size_limit, data.prepared_sets); } } diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h index 4d03f758f617..963dd9f8675a 100644 --- a/dbms/src/Interpreters/ActionsVisitor.h +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -52,38 +53,71 @@ struct ScopeStack const Block & getSampleBlock() const; }; +class ASTIdentifier; +class ASTFunction; +class ASTLiteral; /// Collect ExpressionAction from AST. Returns PreparedSets and SubqueriesForSets too. -class ActionsVisitor +class ActionsMatcher { public: - ActionsVisitor(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, - const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions, - PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, - bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_ = nullptr); + using Visitor = ConstInDepthNodeVisitor; - void visit(const ASTPtr & ast, ExpressionActionsPtr & actions) + struct Data { - visit(ast); - actions = actions_stack.popLevel(); - } + const Context & context; + SizeLimits set_size_limit; + size_t subquery_depth; + const NamesAndTypesList & source_columns; + PreparedSets & prepared_sets; + SubqueriesForSets & subqueries_for_sets; + bool no_subqueries; + bool only_consts; + bool no_storage_or_local; + size_t visit_depth; + ScopeStack actions_stack; + + Data(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, + const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions, + PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, + bool no_subqueries_, bool only_consts_, bool no_storage_or_local_) + : context(context_), + set_size_limit(set_size_limit_), + subquery_depth(subquery_depth_), + source_columns(source_columns_), + prepared_sets(prepared_sets_), + subqueries_for_sets(subqueries_for_sets_), + no_subqueries(no_subqueries_), + only_consts(only_consts_), + no_storage_or_local(no_storage_or_local_), + visit_depth(0), + actions_stack(actions, context) + {} + + void updateActions(ExpressionActionsPtr & actions) + { + actions = actions_stack.popLevel(); + } + + /// Does result of the calculation already exists in the block. + bool hasColumn(const String & columnName) const + { + return actions_stack.getSampleBlock().has(columnName); + } + }; + + static void visit(const ASTPtr & ast, Data & data); + static bool needChildVisit(const ASTPtr & node, const ASTPtr & child); private: - const Context & context; - SizeLimits set_size_limit; - size_t subquery_depth; - const NamesAndTypesList & source_columns; - PreparedSets & prepared_sets; - SubqueriesForSets & subqueries_for_sets; - const bool no_subqueries; - const bool only_consts; - const bool no_storage_or_local; - mutable size_t visit_depth; - std::ostream * ostr; - ScopeStack actions_stack; - - void visit(const ASTPtr & ast); - SetPtr makeSet(const ASTFunction * node, const Block & sample_block); + + static void visit(const ASTIdentifier & identifier, const ASTPtr & ast, Data & data); + static void visit(const ASTFunction & node, const ASTPtr & ast, Data & data); + static void visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data); + + static SetPtr makeSet(const ASTFunction & node, Data & data); }; +using ActionsVisitor = ActionsMatcher::Visitor; + } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index eaba3d568e37..2132259ecaa8 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -315,10 +315,11 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts) { LogAST log; - ActionsVisitor actions_visitor(context, settings.size_limits_for_set, subquery_depth, + ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, sourceColumns(), actions, prepared_sets, subqueries_for_sets, - no_subqueries, only_consts, !isRemoteStorage(), log.stream()); - actions_visitor.visit(ast, actions); + no_subqueries, only_consts, !isRemoteStorage()); + ActionsVisitor(visitor_data, log.stream()).visit(ast); + visitor_data.updateActions(actions); } From 24e5b95b98de432f0cdb828546a825bf37c7cd66 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 15 Aug 2019 17:00:36 +0300 Subject: [PATCH 115/161] Fix test_external_dictionaries for non root user. --- dbms/tests/integration/helpers/cluster.py | 2 +- .../test_external_dictionaries/external_sources.py | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/dbms/tests/integration/helpers/cluster.py b/dbms/tests/integration/helpers/cluster.py index bd3ecb9ae9cc..1288aaa23f27 100644 --- a/dbms/tests/integration/helpers/cluster.py +++ b/dbms/tests/integration/helpers/cluster.py @@ -562,7 +562,7 @@ def copy_file_to_container(self, local_path, dest_path): with open(local_path, 'r') as fdata: data = fdata.read() encoded_data = base64.b64encode(data) - self.exec_in_container(["bash", "-c", "echo {} | base64 --decode > {}".format(encoded_data, dest_path)]) + self.exec_in_container(["bash", "-c", "echo {} | base64 --decode > {}".format(encoded_data, dest_path)], user='root') def get_process_pid(self, process_name): output = self.exec_in_container(["bash", "-c", "ps ax | grep '{}' | grep -v 'grep' | grep -v 'bash -c' | awk '{{print $1}}'".format(process_name)]) diff --git a/dbms/tests/integration/test_external_dictionaries/external_sources.py b/dbms/tests/integration/test_external_dictionaries/external_sources.py index 71dc05ca78c7..7ff24b4b28cc 100644 --- a/dbms/tests/integration/test_external_dictionaries/external_sources.py +++ b/dbms/tests/integration/test_external_dictionaries/external_sources.py @@ -246,7 +246,7 @@ def get_source_str(self, table_name): def prepare(self, structure, table_name, cluster): self.node = cluster.instances[self.docker_hostname] path = "/" + table_name + ".tsv" - self.node.exec_in_container(["bash", "-c", "touch {}".format(path)]) + self.node.exec_in_container(["bash", "-c", "touch {}".format(path)], user="root") self.ordered_names = structure.get_ordered_names() self.prepared = True @@ -260,7 +260,7 @@ def load_data(self, data, table_name): sorted_row.append(str(row.data[name])) str_data = '\t'.join(sorted_row) - self.node.exec_in_container(["bash", "-c", "echo \"{row}\" >> {fname}".format(row=str_data, fname=path)]) + self.node.exec_in_container(["bash", "-c", "echo \"{row}\" >> {fname}".format(row=str_data, fname=path)], user="root") def compatible_with_layout(self, layout): return 'cache' not in layout.name @@ -286,7 +286,7 @@ def get_source_str(self, table_name): def prepare(self, structure, table_name, cluster): self.node = cluster.instances[self.docker_hostname] path = "/" + table_name + ".tsv" - self.node.exec_in_container(["bash", "-c", "touch {}".format(path)]) + self.node.exec_in_container(["bash", "-c", "touch {}".format(path)], user="root") self.ordered_names = structure.get_ordered_names() self.prepared = True @@ -300,7 +300,7 @@ def load_data(self, data, table_name): sorted_row.append(str(row.data[name])) str_data = '\t'.join(sorted_row) - self.node.exec_in_container(["bash", "-c", "echo \"{row}\" >> {fname}".format(row=str_data, fname=path)]) + self.node.exec_in_container(["bash", "-c", "echo \"{row}\" >> {fname}".format(row=str_data, fname=path)], user='root') class SourceExecutableCache(_SourceExecutableBase): @@ -337,7 +337,7 @@ def get_source_str(self, table_name): def prepare(self, structure, table_name, cluster): self.node = cluster.instances[self.docker_hostname] path = "/" + table_name + ".tsv" - self.node.exec_in_container(["bash", "-c", "touch {}".format(path)]) + self.node.exec_in_container(["bash", "-c", "touch {}".format(path)], user='root') script_dir = os.path.dirname(os.path.realpath(__file__)) self.node.copy_file_to_container(os.path.join(script_dir, './http_server.py'), '/http_server.py') @@ -361,7 +361,7 @@ def load_data(self, data, table_name): sorted_row.append(str(row.data[name])) str_data = '\t'.join(sorted_row) - self.node.exec_in_container(["bash", "-c", "echo \"{row}\" >> {fname}".format(row=str_data, fname=path)]) + self.node.exec_in_container(["bash", "-c", "echo \"{row}\" >> {fname}".format(row=str_data, fname=path)], user='root') class SourceHTTP(SourceHTTPBase): From c52a34fd15bc834ab40c8ef91ee281a89d58e2b3 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 15 Aug 2019 16:49:49 +0300 Subject: [PATCH 116/161] Add test for multiple mat. views --- .../integration/test_storage_kafka/test.py | 48 +++++++++++++++++++ 1 file changed, 48 insertions(+) diff --git a/dbms/tests/integration/test_storage_kafka/test.py b/dbms/tests/integration/test_storage_kafka/test.py index c47f68bf6738..09b13d884f6b 100644 --- a/dbms/tests/integration/test_storage_kafka/test.py +++ b/dbms/tests/integration/test_storage_kafka/test.py @@ -325,6 +325,54 @@ def test_kafka_materialized_view(kafka_cluster): kafka_check_result(result, True) +@pytest.mark.timeout(60) +def test_kafka_many_materialized_views(kafka_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.view1; + DROP TABLE IF EXISTS test.view2; + DROP TABLE IF EXISTS test.consumer1; + DROP TABLE IF EXISTS test.consumer2; + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'mmv', + kafka_group_name = 'mmv', + kafka_format = 'JSONEachRow', + kafka_row_delimiter = '\\n'; + CREATE TABLE test.view1 (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE TABLE test.view2 (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer1 TO test.view1 AS + SELECT * FROM test.kafka; + CREATE MATERIALIZED VIEW test.consumer2 TO test.view2 AS + SELECT * FROM test.kafka; + ''') + + messages = [] + for i in range(50): + messages.append(json.dumps({'key': i, 'value': i})) + kafka_produce('mmv', messages) + + while True: + result1 = instance.query('SELECT * FROM test.view1') + result2 = instance.query('SELECT * FROM test.view2') + if kafka_check_result(result1) and kafka_check_result(result2): + break + + instance.query(''' + DROP TABLE test.consumer1; + DROP TABLE test.consumer2; + DROP TABLE test.view1; + DROP TABLE test.view2; + ''') + + kafka_check_result(result1, True) + kafka_check_result(result2, True) + + @pytest.mark.timeout(300) def test_kafka_flush_on_big_message(kafka_cluster): # Create batchs of messages of size ~100Kb From e895251bc67330299f53b8fa42222726098fcbf5 Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 15 Aug 2019 17:22:33 +0300 Subject: [PATCH 117/161] minor changes --- dbms/src/Interpreters/ActionsVisitor.cpp | 39 ++++++++++++------------ dbms/src/Interpreters/ActionsVisitor.h | 10 ++++++ 2 files changed, 29 insertions(+), 20 deletions(-) diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index 7c6f97d5ed55..c519d75b8127 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -269,8 +269,10 @@ void ActionsMatcher::visit(const ASTPtr & ast, Data & data) void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr & ast, Data & data) { CachedColumnName column_name; + if (data.hasColumn(column_name.get(ast))) + return; - if (!data.only_consts && !data.actions_stack.getSampleBlock().has(column_name.get(ast))) + if (!data.only_consts) { /// The requested column is not in the block. /// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY. @@ -286,14 +288,13 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr & ast, /// Special check for WITH statement alias. Add alias action to be able to use this alias. if (identifier.prefer_alias_to_column_name && !identifier.alias.empty()) - data.actions_stack.addAction(ExpressionAction::addAliases({{identifier.name, identifier.alias}})); + data.addAction(ExpressionAction::addAliases({{identifier.name, identifier.alias}})); } } void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data) { CachedColumnName column_name; - if (data.hasColumn(column_name.get(ast))) return; @@ -311,10 +312,10 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (!data.only_consts) { String result_name = column_name.get(ast); - data.actions_stack.addAction(ExpressionAction::copyColumn(arg->getColumnName(), result_name)); + data.addAction(ExpressionAction::copyColumn(arg->getColumnName(), result_name)); NameSet joined_columns; joined_columns.insert(result_name); - data.actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns, false, data.context)); + data.addAction(ExpressionAction::arrayJoin(joined_columns, false, data.context)); } return; @@ -338,7 +339,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// We are in the part of the tree that we are not going to compute. You just need to define types. /// Do not subquery and create sets. We treat "IN" as "ignoreExceptNull" function. - data.actions_stack.addAction(ExpressionAction::applyFunction( + data.addAction(ExpressionAction::applyFunction( FunctionFactory::instance().get("ignoreExceptNull", data.context), { node.arguments->children.at(0)->getColumnName() }, column_name.get(ast))); @@ -351,7 +352,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// (and is used only for index analysis, see KeyCondition). if (node.name == "indexHint") { - data.actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName( + data.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName( ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared(), column_name.get(ast)))); return; @@ -415,15 +416,15 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, /// so that sets with the same literal representation do not fuse together (they can have different types). if (!prepared_set->empty()) - column.name = getUniqueName(data.actions_stack.getSampleBlock(), "__set"); + column.name = getUniqueName(data.getSampleBlock(), "__set"); else column.name = child_column_name; - if (!data.actions_stack.getSampleBlock().has(column.name)) + if (!data.hasColumn(column.name)) { column.column = ColumnSet::create(1, prepared_set); - data.actions_stack.addAction(ExpressionAction::addColumn(column)); + data.addAction(ExpressionAction::addColumn(column)); } argument_types.push_back(column.type); @@ -434,9 +435,9 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// If the argument is not a lambda expression, call it recursively and find out its type. visit(child, data); std::string name = child_column_name; - if (data.actions_stack.getSampleBlock().has(name)) + if (data.hasColumn(name)) { - argument_types.push_back(data.actions_stack.getSampleBlock().getByName(name).type); + argument_types.push_back(data.getSampleBlock().getByName(name).type); argument_names.push_back(name); } else @@ -494,11 +495,11 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// We can not name `getColumnName()`, /// because it does not uniquely define the expression (the types of arguments can be different). - String lambda_name = getUniqueName(data.actions_stack.getSampleBlock(), "__lambda"); + String lambda_name = getUniqueName(data.getSampleBlock(), "__lambda"); auto function_capture = std::make_shared( lambda_actions, captured, lambda_arguments, result_type, result_name); - data.actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name)); + data.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name)); argument_types[i] = std::make_shared(lambda_type->getArgumentTypes(), result_type); argument_names[i] = lambda_name; @@ -510,7 +511,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & { for (const auto & argument_name : argument_names) { - if (!data.actions_stack.getSampleBlock().has(argument_name)) + if (!data.hasColumn(argument_name)) { arguments_present = false; break; @@ -520,15 +521,13 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (arguments_present) { - data.actions_stack.addAction( - ExpressionAction::applyFunction(function_builder, argument_names, column_name.get(ast))); + data.addAction(ExpressionAction::applyFunction(function_builder, argument_names, column_name.get(ast))); } } void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data) { CachedColumnName column_name; - if (data.hasColumn(column_name.get(ast))) return; @@ -539,7 +538,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & ast, Data column.type = type; column.name = column_name.get(ast); - data.actions_stack.addAction(ExpressionAction::addColumn(column)); + data.addAction(ExpressionAction::addColumn(column)); } SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data) @@ -550,7 +549,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data) */ const IAST & args = *node.arguments; const ASTPtr & arg = args.children.at(1); - const Block & sample_block = data.actions_stack.getSampleBlock(); + const Block & sample_block = data.getSampleBlock(); /// If the subquery or table name for SELECT. const auto * identifier = arg->as(); diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h index 963dd9f8675a..def72c7ad1c1 100644 --- a/dbms/src/Interpreters/ActionsVisitor.h +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -99,6 +99,16 @@ class ActionsMatcher actions = actions_stack.popLevel(); } + void addAction(const ExpressionAction & action) + { + actions_stack.addAction(action); + } + + const Block & getSampleBlock() const + { + return actions_stack.getSampleBlock(); + } + /// Does result of the calculation already exists in the block. bool hasColumn(const String & columnName) const { From 854da3b6a241951638cf442f1d08f0b27efc24b0 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 15 Aug 2019 19:09:43 +0300 Subject: [PATCH 118/161] ms column added --- dbms/src/Interpreters/MetricLog.cpp | 17 ++++++++++++++++- dbms/src/Interpreters/MetricLog.h | 1 + 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/MetricLog.cpp b/dbms/src/Interpreters/MetricLog.cpp index c1423a5c11c0..03cf820a2ee2 100644 --- a/dbms/src/Interpreters/MetricLog.cpp +++ b/dbms/src/Interpreters/MetricLog.cpp @@ -13,6 +13,7 @@ Block MetricLogElement::createBlock() columns_with_type_and_name.emplace_back(std::make_shared(), "event_date"); columns_with_type_and_name.emplace_back(std::make_shared(), "event_time"); + columns_with_type_and_name.emplace_back(std::make_shared(), "milliseconds"); //ProfileEvents for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) @@ -43,6 +44,7 @@ void MetricLogElement::appendToBlock(Block & block) const columns[iter++]->insert(DateLUT::instance().toDayNum(event_time)); columns[iter++]->insert(event_time); + columns[iter++]->insert(milliseconds); //ProfileEvents for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) @@ -74,6 +76,16 @@ void MetricLog::stopCollectMetric() metric_flush_thread.join(); } +inline UInt64 time_in_milliseconds(std::chrono::time_point timepoint) +{ + return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); +} + +inline UInt64 time_in_seconds(std::chrono::time_point timepoint) +{ + return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); +} + void MetricLog::metricThreadFunction() { auto desired_timepoint = std::chrono::system_clock::now(); @@ -88,9 +100,12 @@ void MetricLog::metricThreadFunction() MetricLogElement elem; elem.event_time = std::chrono::system_clock::to_time_t(prev_timepoint); + elem.milliseconds = time_in_milliseconds(prev_timepoint) - time_in_seconds(prev_timepoint) * 1000; this->add(elem); - desired_timepoint = prev_timepoint + std::chrono::milliseconds(collect_interval_milliseconds); + while (desired_timepoint <= std::chrono::system_clock::now()) + desired_timepoint += std::chrono::milliseconds(collect_interval_milliseconds); + std::this_thread::sleep_until(desired_timepoint); } catch (...) diff --git a/dbms/src/Interpreters/MetricLog.h b/dbms/src/Interpreters/MetricLog.h index 7c041a19d7d3..032700600f11 100644 --- a/dbms/src/Interpreters/MetricLog.h +++ b/dbms/src/Interpreters/MetricLog.h @@ -10,6 +10,7 @@ using Poco::Message; struct MetricLogElement { time_t event_time{}; + UInt64 milliseconds{}; static std::string name() { return "MetricLog"; } static Block createBlock(); From 2cfbd1e194b46df0ce07567c2eb9f87411577c04 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 15 Aug 2019 19:39:18 +0300 Subject: [PATCH 119/161] reused prev_timepoint --- dbms/src/Interpreters/MetricLog.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/MetricLog.cpp b/dbms/src/Interpreters/MetricLog.cpp index 03cf820a2ee2..00d486c8788e 100644 --- a/dbms/src/Interpreters/MetricLog.cpp +++ b/dbms/src/Interpreters/MetricLog.cpp @@ -93,19 +93,18 @@ void MetricLog::metricThreadFunction() { try { - const auto prev_timepoint = desired_timepoint; - if (is_shutdown_metric_thread) break; MetricLogElement elem; + const auto prev_timepoint = std::chrono::system_clock::now(); elem.event_time = std::chrono::system_clock::to_time_t(prev_timepoint); elem.milliseconds = time_in_milliseconds(prev_timepoint) - time_in_seconds(prev_timepoint) * 1000; + this->add(elem); while (desired_timepoint <= std::chrono::system_clock::now()) desired_timepoint += std::chrono::milliseconds(collect_interval_milliseconds); - std::this_thread::sleep_until(desired_timepoint); } catch (...) From 053f0ee78d863654e23d5709d2a582808728744a Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 15 Aug 2019 20:46:35 +0300 Subject: [PATCH 120/161] fix compilation --- dbms/src/Interpreters/MutationsInterpreter.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index dff891607a5b..160d0bc8023d 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -199,8 +199,7 @@ void MutationsInterpreter::prepare(bool dry_run) { auto query = index->expr->clone(); auto syntax_result = SyntaxAnalyzer(context).analyze(query, all_columns); - ExpressionAnalyzer analyzer(query, syntax_result, context); - const auto required_columns = analyzer.getRequiredSourceColumns(); + const auto required_columns = syntax_result->requiredSourceColumns(); for (const String & dependency : required_columns) { @@ -283,8 +282,7 @@ void MutationsInterpreter::prepare(bool dry_run) auto query = (*it)->expr->clone(); auto syntax_result = SyntaxAnalyzer(context).analyze(query, all_columns); - ExpressionAnalyzer analyzer(query, syntax_result, context); - const auto required_columns = analyzer.getRequiredSourceColumns(); + const auto required_columns = syntax_result->requiredSourceColumns(); affected_indices_columns.insert(std::cbegin(required_columns), std::cend(required_columns)); } else From f59fa67050cd5407789824626826837e3cb6b107 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 15 Aug 2019 21:46:16 +0300 Subject: [PATCH 121/161] added type checks for set index functions --- dbms/src/Functions/bitBoolMaskAnd.cpp | 6 ++++++ dbms/src/Functions/bitBoolMaskOr.cpp | 6 ++++++ dbms/src/Functions/bitSwapLastTwo.cpp | 6 ++++++ dbms/src/Functions/bitWrapperFunc.cpp | 8 +++++++- 4 files changed, 25 insertions(+), 1 deletion(-) diff --git a/dbms/src/Functions/bitBoolMaskAnd.cpp b/dbms/src/Functions/bitBoolMaskAnd.cpp index f7cf7efa25fb..02e681b59b85 100644 --- a/dbms/src/Functions/bitBoolMaskAnd.cpp +++ b/dbms/src/Functions/bitBoolMaskAnd.cpp @@ -4,6 +4,10 @@ namespace DB { + namespace ErrorCodes + { + extern const int BAD_CAST; + } /// Working with UInt8: last bit = can be true, previous = can be false (Like dbms/src/Storages/MergeTree/BoolMask.h). /// This function provides "AND" operation for BoolMasks. @@ -17,6 +21,8 @@ namespace DB template static inline Result apply(A left, B right) { + if constexpr (!std::is_same_v || !std::is_same_v) + throw DB::Exception("Only UInt8 type is supported by __bitBoolMaskAnd.", ErrorCodes::BAD_CAST); return static_cast( ((static_cast(left) & static_cast(right)) & 1) | ((((static_cast(left) >> 1) | (static_cast(right) >> 1)) & 1) << 1)); diff --git a/dbms/src/Functions/bitBoolMaskOr.cpp b/dbms/src/Functions/bitBoolMaskOr.cpp index 0c34b8e5bdbd..d1261f4fe14e 100644 --- a/dbms/src/Functions/bitBoolMaskOr.cpp +++ b/dbms/src/Functions/bitBoolMaskOr.cpp @@ -4,6 +4,10 @@ namespace DB { + namespace ErrorCodes + { + extern const int BAD_CAST; + } /// Working with UInt8: last bit = can be true, previous = can be false (Like dbms/src/Storages/MergeTree/BoolMask.h). /// This function provides "OR" operation for BoolMasks. @@ -17,6 +21,8 @@ namespace DB template static inline Result apply(A left, B right) { + if constexpr (!std::is_same_v || !std::is_same_v) + throw DB::Exception("Only UInt8 type is supported by __bitBoolMaskOr.", ErrorCodes::BAD_CAST); return static_cast( ((static_cast(left) | static_cast(right)) & 1) | ((((static_cast(left) >> 1) & (static_cast(right) >> 1)) & 1) << 1)); diff --git a/dbms/src/Functions/bitSwapLastTwo.cpp b/dbms/src/Functions/bitSwapLastTwo.cpp index 4a3b0cd304a8..7b3f92a07240 100644 --- a/dbms/src/Functions/bitSwapLastTwo.cpp +++ b/dbms/src/Functions/bitSwapLastTwo.cpp @@ -4,6 +4,10 @@ namespace DB { + namespace ErrorCodes + { + extern const int BAD_CAST; + } /// Working with UInt8: last bit = can be true, previous = can be false (Like dbms/src/Storages/MergeTree/BoolMask.h). /// This function provides "NOT" operation for BoolMasks by swapping last two bits ("can be true" <-> "can be false"). @@ -14,6 +18,8 @@ namespace DB static inline ResultType NO_SANITIZE_UNDEFINED apply(A a) { + if constexpr (!std::is_same_v) + throw DB::Exception("Only UInt8 type is supported by __bitSwapLastTwo.", ErrorCodes::BAD_CAST); return static_cast( ((static_cast(a) & 1) << 1) | ((static_cast(a) >> 1) & 1)); } diff --git a/dbms/src/Functions/bitWrapperFunc.cpp b/dbms/src/Functions/bitWrapperFunc.cpp index b9e378388754..c8951de66d11 100644 --- a/dbms/src/Functions/bitWrapperFunc.cpp +++ b/dbms/src/Functions/bitWrapperFunc.cpp @@ -4,6 +4,10 @@ namespace DB { + namespace ErrorCodes + { + extern const int BAD_CAST; + } /// Working with UInt8: last bit = can be true, previous = can be false (Like dbms/src/Storages/MergeTree/BoolMask.h). /// This function wraps bool atomic functions @@ -15,7 +19,9 @@ namespace DB static inline ResultType NO_SANITIZE_UNDEFINED apply(A a) { - return a == static_cast(0) ? static_cast(0b10) : static_cast(0b1); + if constexpr (!std::is_integral_v) + throw DB::Exception("It's a bug! Only integer types are supported by __bitWrapperFunc.", ErrorCodes::BAD_CAST); + return a == 0 ? static_cast(0b10) : static_cast(0b1); } #if USE_EMBEDDED_COMPILER From a051009d2896780942e10014021822faf0859107 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 15 Aug 2019 21:48:48 +0300 Subject: [PATCH 122/161] add it's a bug mes --- dbms/src/Functions/bitBoolMaskAnd.cpp | 2 +- dbms/src/Functions/bitBoolMaskOr.cpp | 2 +- dbms/src/Functions/bitSwapLastTwo.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/bitBoolMaskAnd.cpp b/dbms/src/Functions/bitBoolMaskAnd.cpp index 02e681b59b85..eaa1a2a33434 100644 --- a/dbms/src/Functions/bitBoolMaskAnd.cpp +++ b/dbms/src/Functions/bitBoolMaskAnd.cpp @@ -22,7 +22,7 @@ namespace DB static inline Result apply(A left, B right) { if constexpr (!std::is_same_v || !std::is_same_v) - throw DB::Exception("Only UInt8 type is supported by __bitBoolMaskAnd.", ErrorCodes::BAD_CAST); + throw DB::Exception("It's a bug! Only UInt8 type is supported by __bitBoolMaskAnd.", ErrorCodes::BAD_CAST); return static_cast( ((static_cast(left) & static_cast(right)) & 1) | ((((static_cast(left) >> 1) | (static_cast(right) >> 1)) & 1) << 1)); diff --git a/dbms/src/Functions/bitBoolMaskOr.cpp b/dbms/src/Functions/bitBoolMaskOr.cpp index d1261f4fe14e..903c3582375f 100644 --- a/dbms/src/Functions/bitBoolMaskOr.cpp +++ b/dbms/src/Functions/bitBoolMaskOr.cpp @@ -22,7 +22,7 @@ namespace DB static inline Result apply(A left, B right) { if constexpr (!std::is_same_v || !std::is_same_v) - throw DB::Exception("Only UInt8 type is supported by __bitBoolMaskOr.", ErrorCodes::BAD_CAST); + throw DB::Exception("It's a bug! Only UInt8 type is supported by __bitBoolMaskOr.", ErrorCodes::BAD_CAST); return static_cast( ((static_cast(left) | static_cast(right)) & 1) | ((((static_cast(left) >> 1) & (static_cast(right) >> 1)) & 1) << 1)); diff --git a/dbms/src/Functions/bitSwapLastTwo.cpp b/dbms/src/Functions/bitSwapLastTwo.cpp index 7b3f92a07240..22b7b889d8b4 100644 --- a/dbms/src/Functions/bitSwapLastTwo.cpp +++ b/dbms/src/Functions/bitSwapLastTwo.cpp @@ -19,7 +19,7 @@ namespace DB static inline ResultType NO_SANITIZE_UNDEFINED apply(A a) { if constexpr (!std::is_same_v) - throw DB::Exception("Only UInt8 type is supported by __bitSwapLastTwo.", ErrorCodes::BAD_CAST); + throw DB::Exception("It's a bug! Only UInt8 type is supported by __bitSwapLastTwo.", ErrorCodes::BAD_CAST); return static_cast( ((static_cast(a) & 1) << 1) | ((static_cast(a) >> 1) & 1)); } From 25c882e3cc2e8aae7b0fd1aab5b8e4ec36871537 Mon Sep 17 00:00:00 2001 From: Yuriy Date: Fri, 16 Aug 2019 00:22:54 +0300 Subject: [PATCH 123/161] fixed unbundled build --- dbms/src/Core/MySQLProtocol.h | 6 ++++-- dbms/src/Interpreters/UsersManager.cpp | 28 ++++++++++++++++++++++---- 2 files changed, 28 insertions(+), 6 deletions(-) diff --git a/dbms/src/Core/MySQLProtocol.h b/dbms/src/Core/MySQLProtocol.h index 7e29dc9676df..029d7ded18a7 100644 --- a/dbms/src/Core/MySQLProtocol.h +++ b/dbms/src/Core/MySQLProtocol.h @@ -23,7 +23,6 @@ #include #include #include -#include /// Implementation of MySQL wire protocol. /// Works only on little-endian architecture. @@ -943,7 +942,10 @@ class Native41 : public IPlugin class Sha256Password : public IPlugin { public: - Sha256Password(RSA & public_key, RSA & private_key, Logger * log) : public_key(public_key), private_key(private_key), log(log) + Sha256Password(RSA & public_key_, RSA & private_key_, Logger * log_) + : public_key(public_key_) + , private_key(private_key_) + , log(log_) { /** Native authentication sent 20 bytes + '\0' character = 21 bytes. * This plugin must do the same to stay consistent with historical behavior if it is set to operate as a default plugin. [1] diff --git a/dbms/src/Interpreters/UsersManager.cpp b/dbms/src/Interpreters/UsersManager.cpp index 6d1f7152b9e2..123d4bd87043 100644 --- a/dbms/src/Interpreters/UsersManager.cpp +++ b/dbms/src/Interpreters/UsersManager.cpp @@ -8,9 +8,11 @@ #include #include #include -#include #include #include +#if USE_SSL +# include +#endif namespace DB @@ -69,10 +71,28 @@ UserPtr UsersManager::authorizeAndGetUser( if (!it->second->password_sha256_hex.empty()) { - Poco::SHA2Engine engine; - engine.update(password); - if (Poco::SHA2Engine::digestToHex(engine.digest()) != it->second->password_sha256_hex) +#if USE_SSL + unsigned char hash[32]; + + SHA256_CTX ctx; + SHA256_Init(&ctx); + SHA256_Update(&ctx, reinterpret_cast(password.data()), password.size()); + SHA256_Final(hash, &ctx); + + String hash_hex; + { + WriteBufferFromString buf(hash_hex); + HexWriteBuffer hex_buf(buf); + hex_buf.write(reinterpret_cast(hash), sizeof(hash)); + } + + Poco::toLowerInPlace(hash_hex); + + if (hash_hex != it->second->password_sha256_hex) on_wrong_password(); +#else + throw DB::Exception("SHA256 passwords support is disabled, because ClickHouse was built without SSL library", DB::ErrorCodes::SUPPORT_IS_DISABLED); +#endif } else if (!it->second->password_double_sha1_hex.empty()) { From b28861147268910e6c8777d40496c31fac28a81b Mon Sep 17 00:00:00 2001 From: Yuriy Date: Fri, 16 Aug 2019 00:54:53 +0300 Subject: [PATCH 124/161] static linking of sha256_password in mariadb-connector-c --- .../linux_x86_64/libmariadb/ma_client_plugin.c | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/contrib/mariadb-connector-c-cmake/linux_x86_64/libmariadb/ma_client_plugin.c b/contrib/mariadb-connector-c-cmake/linux_x86_64/libmariadb/ma_client_plugin.c index b7fdcdbcb850..fefba7944ef4 100644 --- a/contrib/mariadb-connector-c-cmake/linux_x86_64/libmariadb/ma_client_plugin.c +++ b/contrib/mariadb-connector-c-cmake/linux_x86_64/libmariadb/ma_client_plugin.c @@ -76,17 +76,18 @@ struct st_client_plugin_int *plugin_list[MYSQL_CLIENT_MAX_PLUGINS + MARIADB_CLIE static pthread_mutex_t LOCK_load_client_plugin; #endif - extern struct st_mysql_client_plugin mysql_native_password_client_plugin; - extern struct st_mysql_client_plugin mysql_old_password_client_plugin; - extern struct st_mysql_client_plugin pvio_socket_client_plugin; +extern struct st_mysql_client_plugin mysql_native_password_client_plugin; +extern struct st_mysql_client_plugin mysql_old_password_client_plugin; +extern struct st_mysql_client_plugin pvio_socket_client_plugin; +extern struct st_mysql_client_plugin sha256_password_client_plugin; struct st_mysql_client_plugin *mysql_client_builtins[]= { - (struct st_mysql_client_plugin *)&mysql_native_password_client_plugin, - (struct st_mysql_client_plugin *)&mysql_old_password_client_plugin, - (struct st_mysql_client_plugin *)&pvio_socket_client_plugin, - + (struct st_mysql_client_plugin *)&mysql_native_password_client_plugin, + (struct st_mysql_client_plugin *)&mysql_old_password_client_plugin, + (struct st_mysql_client_plugin *)&pvio_socket_client_plugin, + (struct st_mysql_client_plugin *)&sha256_password_client_plugin, 0 }; From c08d01cfb4305cab5afcc1eb39973753d1216008 Mon Sep 17 00:00:00 2001 From: Yuriy Date: Fri, 16 Aug 2019 02:03:58 +0300 Subject: [PATCH 125/161] linking caching_sha2_password plugin statically --- contrib/mariadb-connector-c | 2 +- .../linux_x86_64/libmariadb/ma_client_plugin.c | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/contrib/mariadb-connector-c b/contrib/mariadb-connector-c index d85d0e98999c..c6503d3acc85 160000 --- a/contrib/mariadb-connector-c +++ b/contrib/mariadb-connector-c @@ -1 +1 @@ -Subproject commit d85d0e98999cd9e28ceb66645999b4a9ce85370e +Subproject commit c6503d3acc85ca1a7f5e7e38b605d7c9410aac1e diff --git a/contrib/mariadb-connector-c-cmake/linux_x86_64/libmariadb/ma_client_plugin.c b/contrib/mariadb-connector-c-cmake/linux_x86_64/libmariadb/ma_client_plugin.c index fefba7944ef4..434a4b3f4c3e 100644 --- a/contrib/mariadb-connector-c-cmake/linux_x86_64/libmariadb/ma_client_plugin.c +++ b/contrib/mariadb-connector-c-cmake/linux_x86_64/libmariadb/ma_client_plugin.c @@ -80,6 +80,7 @@ extern struct st_mysql_client_plugin mysql_native_password_client_plugin; extern struct st_mysql_client_plugin mysql_old_password_client_plugin; extern struct st_mysql_client_plugin pvio_socket_client_plugin; extern struct st_mysql_client_plugin sha256_password_client_plugin; +extern struct st_mysql_client_plugin caching_sha2_password_client_plugin; struct st_mysql_client_plugin *mysql_client_builtins[]= @@ -88,6 +89,7 @@ struct st_mysql_client_plugin *mysql_client_builtins[]= (struct st_mysql_client_plugin *)&mysql_old_password_client_plugin, (struct st_mysql_client_plugin *)&pvio_socket_client_plugin, (struct st_mysql_client_plugin *)&sha256_password_client_plugin, + (struct st_mysql_client_plugin *)&caching_sha2_password_client_plugin, 0 }; From cf9b41549d4e390ed8fdef75ab024d9102255b28 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Aug 2019 02:35:54 +0300 Subject: [PATCH 126/161] MetricLog: code cleanups; comments --- dbms/src/Interpreters/MetricLog.cpp | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/dbms/src/Interpreters/MetricLog.cpp b/dbms/src/Interpreters/MetricLog.cpp index 00d486c8788e..b17813ba4016 100644 --- a/dbms/src/Interpreters/MetricLog.cpp +++ b/dbms/src/Interpreters/MetricLog.cpp @@ -89,29 +89,28 @@ inline UInt64 time_in_seconds(std::chrono::time_point void MetricLog::metricThreadFunction() { auto desired_timepoint = std::chrono::system_clock::now(); - while (true) + while (!is_shutdown_metric_thread) { try { - if (is_shutdown_metric_thread) - break; - MetricLogElement elem; - const auto prev_timepoint = std::chrono::system_clock::now(); - elem.event_time = std::chrono::system_clock::to_time_t(prev_timepoint); - elem.milliseconds = time_in_milliseconds(prev_timepoint) - time_in_seconds(prev_timepoint) * 1000; + const auto current_time = std::chrono::system_clock::now(); + elem.event_time = std::chrono::system_clock::to_time_t(current_time); + elem.milliseconds = time_in_milliseconds(current_time) - time_in_seconds(current_time) * 1000; this->add(elem); - while (desired_timepoint <= std::chrono::system_clock::now()) + /// We will record current time into table but align it to regular time intervals to avoid time drift. + /// We may drop some time points if the server is overloaded and recording took too much time. + while (desired_timepoint <= current_time) desired_timepoint += std::chrono::milliseconds(collect_interval_milliseconds); + std::this_thread::sleep_until(desired_timepoint); } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } - } } From b66719725f17238ee817f6d292e95116c0591a00 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Aug 2019 03:49:33 +0300 Subject: [PATCH 127/161] Fix race condition in system.parts vs. ALTER --- dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp | 2 ++ dbms/src/Storages/System/StorageSystemParts.cpp | 6 +++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp index f64bdcc97402..0b208a13b59e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp @@ -409,6 +409,8 @@ void MergeTreeDataPart::remove() const #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wunused-variable" #endif + std::shared_lock lock(columns_lock); + for (const auto & [file, _] : checksums.files) { String path_to_remove = to + "/" + file; diff --git a/dbms/src/Storages/System/StorageSystemParts.cpp b/dbms/src/Storages/System/StorageSystemParts.cpp index 65d17f096c39..58fe57c27397 100644 --- a/dbms/src/Storages/System/StorageSystemParts.cpp +++ b/dbms/src/Storages/System/StorageSystemParts.cpp @@ -114,7 +114,11 @@ void StorageSystemParts::processNextStorage(MutableColumns & columns_, const Sto columns_[i++]->insert(part->stateString()); MinimalisticDataPartChecksums helper; - helper.computeTotalChecksums(part->checksums); + { + /// TODO MergeTreeDataPart structure is too error-prone. + std::shared_lock lock(part->columns_lock); + helper.computeTotalChecksums(part->checksums); + } auto checksum = helper.hash_of_all_files; columns_[i++]->insert(getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second)); From d8efa5a842562e655d88f39a3b5a29d4043eb8c2 Mon Sep 17 00:00:00 2001 From: Yuriy Date: Fri, 16 Aug 2019 03:59:59 +0300 Subject: [PATCH 128/161] added missing caching_sha2_pw.c --- contrib/mariadb-connector-c-cmake/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/mariadb-connector-c-cmake/CMakeLists.txt b/contrib/mariadb-connector-c-cmake/CMakeLists.txt index a0582d896851..b8d27d144d39 100644 --- a/contrib/mariadb-connector-c-cmake/CMakeLists.txt +++ b/contrib/mariadb-connector-c-cmake/CMakeLists.txt @@ -42,6 +42,7 @@ ${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/mariadb_cleartext.c ${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/my_auth.c ${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/old_password.c ${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/sha256_pw.c +${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/caching_sha2_pw.c #${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/sspi_client.c #${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/sspi_errmsg.c ${MARIADB_CLIENT_SOURCE_DIR}/plugins/connection/aurora.c From b808f2e2e8dca86c534b52c435b5f5a6f76362d2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 16 Aug 2019 11:19:13 +0300 Subject: [PATCH 129/161] Add metric log --- dbms/tests/config/metric_log.xml | 8 ++++++++ 1 file changed, 8 insertions(+) create mode 100644 dbms/tests/config/metric_log.xml diff --git a/dbms/tests/config/metric_log.xml b/dbms/tests/config/metric_log.xml new file mode 100644 index 000000000000..0ca9f1624169 --- /dev/null +++ b/dbms/tests/config/metric_log.xml @@ -0,0 +1,8 @@ + + + system + metric_log
+ 7500 + 1000 +
+
From 69ed5279b50ec16163fc3f9fa85b06bea478bafa Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 16 Aug 2019 15:48:48 +0300 Subject: [PATCH 130/161] Get rid of dynamic allocation in ParsedJson::Iterator. --- contrib/simdjson | 2 +- dbms/tests/queries/0_stateless/00975_json_hang.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/simdjson b/contrib/simdjson index 9dfab9d9a4c1..e9be643db5cf 160000 --- a/contrib/simdjson +++ b/contrib/simdjson @@ -1 +1 @@ -Subproject commit 9dfab9d9a4c111690a101ea0a7506a2b2f3fa414 +Subproject commit e9be643db5cf1c29a69bc80ee72d220124a9c50e diff --git a/dbms/tests/queries/0_stateless/00975_json_hang.sql b/dbms/tests/queries/0_stateless/00975_json_hang.sql index 0618b4ba8f79..d60411cb7965 100644 --- a/dbms/tests/queries/0_stateless/00975_json_hang.sql +++ b/dbms/tests/queries/0_stateless/00975_json_hang.sql @@ -1 +1 @@ -SELECT DISTINCT JSONExtractRaw(concat('{"x":', rand() % 2 ? 'true' : 'false', '}'), 'x') AS res FROM numbers(100000) ORDER BY res; +SELECT DISTINCT JSONExtractRaw(concat('{"x":', rand() % 2 ? 'true' : 'false', '}'), 'x') AS res FROM numbers(1000000) ORDER BY res; From da6925f74fd742f91632ae0531104071a8097cc0 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 16 Aug 2019 16:22:26 +0300 Subject: [PATCH 131/161] [website] take upcoming meetups from README.md (#6520) --- website/index.html | 33 ++++++++++++++++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/website/index.html b/website/index.html index 5ab51289c920..f2ef2f698f08 100644 --- a/website/index.html +++ b/website/index.html @@ -94,7 +94,6 @@

- Upcoming Meetups: Moscow on September 5, Paris on October 3, Hong Kong on October 17, Shenzhen on October 20 and Shanghai on October 27
@@ -495,6 +494,38 @@

Like ClickHouse?