From e11fa03bddeb637255d19da1a2fbe5bf6f4a6a0e Mon Sep 17 00:00:00 2001 From: hcz Date: Thu, 21 May 2020 12:07:47 +0800 Subject: [PATCH 01/79] Add ArrowStream input and output format --- .../Formats/Impl/ArrowBlockInputFormat.cpp | 70 ++++++++++++++----- .../Formats/Impl/ArrowBlockInputFormat.h | 10 +-- .../Formats/Impl/ArrowBlockOutputFormat.cpp | 37 +++++++--- .../Formats/Impl/ArrowBlockOutputFormat.h | 6 +- 4 files changed, 91 insertions(+), 32 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index df73c93fd55..c8a098418b7 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -16,12 +16,12 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_EXCEPTION; extern const int CANNOT_READ_ALL_DATA; } -ArrowBlockInputFormat::ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_) - : IInputFormat(header_, in_) +ArrowBlockInputFormat::ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_, bool stream_) + : IInputFormat(header_, in_), stream{stream_} { prepareReader(); } @@ -31,11 +31,21 @@ Chunk ArrowBlockInputFormat::generate() Chunk res; const Block & header = getPort().getHeader(); - if (record_batch_current >= record_batch_total) + if (!stream && record_batch_current >= record_batch_total) return res; std::vector> single_batch(1); - arrow::Status read_status = file_reader->ReadRecordBatch(record_batch_current, &single_batch[0]); + arrow::Status read_status; + if (stream) + { + read_status = stream_reader->ReadNext(&single_batch[0]); + } + else + { + read_status = file_reader->ReadRecordBatch(record_batch_current, &single_batch[0]); + if (!single_batch[0]) + return res; + } if (!read_status.ok()) throw Exception{"Error while reading batch of Arrow data: " + read_status.ToString(), ErrorCodes::CANNOT_READ_ALL_DATA}; @@ -57,30 +67,54 @@ void ArrowBlockInputFormat::resetParser() { IInputFormat::resetParser(); - file_reader.reset(); + if (stream) + stream_reader.reset(); + else + file_reader.reset(); prepareReader(); } void ArrowBlockInputFormat::prepareReader() { - arrow::Status open_status = arrow::ipc::RecordBatchFileReader::Open(asArrowFile(in), &file_reader); - if (!open_status.ok()) - throw Exception(open_status.ToString(), ErrorCodes::BAD_ARGUMENTS); - record_batch_total = file_reader->num_record_batches(); + arrow::Status status; + + if (stream) + status = arrow::ipc::RecordBatchStreamReader::Open(asArrowFile(in), &stream_reader); + else + status = arrow::ipc::RecordBatchFileReader::Open(asArrowFile(in), &file_reader); + + if (!status.ok()) + throw Exception{"Error while opening a table: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION}; + + if (stream) + record_batch_total = -1; + else + record_batch_total = file_reader->num_record_batches(); + record_batch_current = 0; } void registerInputFormatProcessorArrow(FormatFactory &factory) { factory.registerInputFormatProcessor( - "Arrow", - [](ReadBuffer & buf, - const Block & sample, - const RowInputFormatParams & /* params */, - const FormatSettings & /* format_settings */) - { - return std::make_shared(buf, sample); - }); + "Arrow", + [](ReadBuffer & buf, + const Block & sample, + const RowInputFormatParams & /* params */, + const FormatSettings & /* format_settings */) + { + return std::make_shared(buf, sample, false); + }); + + factory.registerInputFormatProcessor( + "ArrowStream", + [](ReadBuffer & buf, + const Block & sample, + const RowInputFormatParams & /* params */, + const FormatSettings & /* format_settings */) + { + return std::make_shared(buf, sample, true); + }); } } diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index 28ca033dd13..2e7f647b4f7 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -4,6 +4,7 @@ #include +namespace arrow { class RecordBatchReader; } namespace arrow::ipc { class RecordBatchFileReader; } namespace DB @@ -14,7 +15,7 @@ class ReadBuffer; class ArrowBlockInputFormat : public IInputFormat { public: - ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_); + ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_, bool stream_); void resetParser() override; @@ -24,12 +25,13 @@ protected: Chunk generate() override; private: - void prepareReader(); - -private: + bool stream; + std::shared_ptr stream_reader; std::shared_ptr file_reader; int record_batch_total = 0; int record_batch_current = 0; + + void prepareReader(); }; } diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index ed33eb08728..ef6bd315319 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -15,8 +15,8 @@ namespace ErrorCodes extern const int UNKNOWN_EXCEPTION; } -ArrowBlockOutputFormat::ArrowBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) - : IOutputFormat(header_, out_), format_settings{format_settings_}, arrow_ostream{std::make_shared(out_)} +ArrowBlockOutputFormat::ArrowBlockOutputFormat(WriteBuffer & out_, const Block & header_, bool stream_, const FormatSettings & format_settings_) + : IOutputFormat(header_, out_), stream{stream_}, format_settings{format_settings_}, arrow_ostream{std::make_shared(out_)} { } @@ -29,12 +29,7 @@ void ArrowBlockOutputFormat::consume(Chunk chunk) CHColumnToArrowColumn::chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Arrow"); if (!writer) - { - // TODO: should we use arrow::ipc::IpcOptions::alignment? - auto status = arrow::ipc::RecordBatchFileWriter::Open(arrow_ostream.get(), arrow_table->schema(), &writer); - if (!status.ok()) - throw Exception{"Error while opening a table: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION}; - } + prepareWriter(arrow_table->schema()); // TODO: calculate row_group_size depending on a number of rows and table size auto status = writer->WriteTable(*arrow_table, format_settings.arrow.row_group_size); @@ -53,6 +48,20 @@ void ArrowBlockOutputFormat::finalize() } } +void ArrowBlockOutputFormat::prepareWriter(const std::shared_ptr & schema) +{ + arrow::Status status; + + // TODO: should we use arrow::ipc::IpcOptions::alignment? + if (stream) + status = arrow::ipc::RecordBatchStreamWriter::Open(arrow_ostream.get(), schema, &writer); + else + status = arrow::ipc::RecordBatchFileWriter::Open(arrow_ostream.get(), schema, &writer); + + if (!status.ok()) + throw Exception{"Error while opening a table writer: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION}; +} + void registerOutputFormatProcessorArrow(FormatFactory & factory) { factory.registerOutputFormatProcessor( @@ -62,7 +71,17 @@ void registerOutputFormatProcessorArrow(FormatFactory & factory) FormatFactory::WriteCallback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, format_settings); + return std::make_shared(buf, sample, false, format_settings); + }); + + factory.registerOutputFormatProcessor( + "ArrowStream", + [](WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, true, format_settings); }); } diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h index 9ea84f83a0d..0cc6804705b 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h @@ -6,6 +6,7 @@ #include #include "ArrowBufferedStreams.h" +namespace arrow { class Schema; } namespace arrow::ipc { class RecordBatchWriter; } namespace DB @@ -14,7 +15,7 @@ namespace DB class ArrowBlockOutputFormat : public IOutputFormat { public: - ArrowBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_); + ArrowBlockOutputFormat(WriteBuffer & out_, const Block & header_, bool stream_, const FormatSettings & format_settings_); String getName() const override { return "ArrowBlockOutputFormat"; } void consume(Chunk) override; @@ -23,9 +24,12 @@ public: String getContentType() const override { return "application/octet-stream"; } private: + bool stream; const FormatSettings format_settings; std::shared_ptr arrow_ostream; std::shared_ptr writer; + + void prepareWriter(const std::shared_ptr & schema); }; } From 4effdeaa0e2a07902a286d74a3d13e95d8a40689 Mon Sep 17 00:00:00 2001 From: hcz Date: Thu, 21 May 2020 12:13:49 +0800 Subject: [PATCH 02/79] Add tests --- .../0_stateless/01273_arrow_stream.reference | 62 ++++++++++ .../queries/0_stateless/01273_arrow_stream.sh | 115 ++++++++++++++++++ 2 files changed, 177 insertions(+) create mode 100644 tests/queries/0_stateless/01273_arrow_stream.reference create mode 100755 tests/queries/0_stateless/01273_arrow_stream.sh diff --git a/tests/queries/0_stateless/01273_arrow_stream.reference b/tests/queries/0_stateless/01273_arrow_stream.reference new file mode 100644 index 00000000000..0f4be2c74a0 --- /dev/null +++ b/tests/queries/0_stateless/01273_arrow_stream.reference @@ -0,0 +1,62 @@ +9999 +9998 +9997 +9996 +9995 +9994 +9993 +9992 +9991 +9990 +99999 +99998 +99997 +99996 +99995 +99994 +99993 +99992 +99991 +99990 +2 +1 +0 +999 +998 +997 +996 +995 +994 +993 +992 +991 +990 +ContextLock Number of times the lock of Context was acquired or tried to acquire. This is global lock. +Query Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries. +original: +-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06 +-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 +127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06 +converted: +-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06 +-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 +127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06 +diff: +dest: +79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 1970-01-01 06:29:04 +80 81 82 83 84 85 86 87 88 89 str02 fstr2\0\0\0\0\0\0\0\0\0\0 2005-03-04 2006-08-09 10:11:12 +min: +-128 0 0 0 0 0 0 0 -1 -1 string-1\0\0\0\0\0\0\0 fixedstring-1\0\0 2003-04-05 2003-02-03 +-108 108 8 92 -8 108 -40 -116 -1 -1 string-0\0\0\0\0\0\0\0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 +79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 +127 -1 -1 -1 -1 -1 -1 -1 -1 -1 string-2\0\0\0\0\0\0\0 fixedstring-2\0\0 2004-06-07 2004-02-03 +max: +-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 1970-01-01 06:22:27 2003-02-03 04:05:06 +-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 1970-01-01 06:09:16 2002-02-03 04:05:06 +80 81 82 83 84 85 86 87 88 89 str02 fstr2 2005-03-04 05:06:07 2006-08-09 10:11:12 +127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2\0\0 1970-01-01 06:29:36 2004-02-03 04:05:06 +dest from null: +-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06 +-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 +127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06 +\N \N \N \N \N \N \N \N \N \N \N \N \N \N diff --git a/tests/queries/0_stateless/01273_arrow_stream.sh b/tests/queries/0_stateless/01273_arrow_stream.sh new file mode 100755 index 00000000000..2fc630a3a8b --- /dev/null +++ b/tests/queries/0_stateless/01273_arrow_stream.sh @@ -0,0 +1,115 @@ +#!/usr/bin/env bash + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CUR_DIR/../shell_config.sh + + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS contributors" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE contributors (name String) ENGINE = Memory" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.contributors ORDER BY name DESC FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO contributors FORMAT ArrowStream" +# random results +${CLICKHOUSE_CLIENT} --query="SELECT * FROM contributors LIMIT 10" > /dev/null +${CLICKHOUSE_CLIENT} --query="DROP TABLE contributors" + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_numbers" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_numbers (number UInt64) ENGINE = Memory" +# less than default block size (65k) +${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 10000 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_numbers FORMAT ArrowStream" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_numbers ORDER BY number DESC LIMIT 10" +${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE arrow_numbers" + +# More than default block size +${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 100000 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_numbers FORMAT ArrowStream" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_numbers ORDER BY number DESC LIMIT 10" +${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE arrow_numbers" + +${CLICKHOUSE_CLIENT} --max_block_size=2 --query="SELECT * FROM system.numbers LIMIT 3 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_numbers FORMAT ArrowStream" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_numbers ORDER BY number DESC LIMIT 10" + +${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE arrow_numbers" +${CLICKHOUSE_CLIENT} --max_block_size=1 --query="SELECT * FROM system.numbers LIMIT 1000 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_numbers FORMAT ArrowStream" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_numbers ORDER BY number DESC LIMIT 10" + +${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_numbers" + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_events" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_events (event String, value UInt64, description String) ENGINE = Memory" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.events FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_events FORMAT ArrowStream" +${CLICKHOUSE_CLIENT} --query="SELECT event, description FROM arrow_events WHERE event IN ('ContextLock', 'Query') ORDER BY event" +${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_events" + + + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types1" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types3" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types4" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types1 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime) ENGINE = Memory" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types2 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime) ENGINE = Memory" +# convert min type +${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types3 (int8 Int8, uint8 Int8, int16 Int8, uint16 Int8, int32 Int8, uint32 Int8, int64 Int8, uint64 Int8, float32 Int8, float64 Int8, string FixedString(15), fixedstring FixedString(15), date Date, datetime Date) ENGINE = Memory" +# convert max type +${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types4 (int8 Int64, uint8 Int64, int16 Int64, uint16 Int64, int32 Int64, uint32 Int64, int64 Int64, uint64 Int64, float32 Int64, float64 Int64, string String, fixedstring String, date DateTime, datetime DateTime) ENGINE = Memory" + +${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types1 values ( -108, 108, -1016, 1116, -1032, 1132, -1064, 1164, -1.032, -1.064, 'string-0', 'fixedstring', '2001-02-03', '2002-02-03 04:05:06')" + +# min +${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types1 values ( -128, 0, -32768, 0, -2147483648, 0, -9223372036854775808, 0, -1.032, -1.064, 'string-1', 'fixedstring-1', '2003-04-05', '2003-02-03 04:05:06')" + +# max +${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06')" + +# 'SELECT -127,-128,-129,126,127,128,255,256,257,-32767,-32768,-32769,32766,32767,32768,65535,65536,65537, -2147483647,-2147483648,-2147483649,2147483646,2147483647,2147483648,4294967295,4294967296,4294967297, -9223372036854775807,-9223372036854775808,9223372036854775806,9223372036854775807,9223372036854775808,18446744073709551615'; + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types2 FORMAT ArrowStream" + +echo original: +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8" | tee ${CLICKHOUSE_TMP}/arrow_all_types_1.dump +echo converted: +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types2 ORDER BY int8" | tee ${CLICKHOUSE_TMP}/arrow_all_types_2.dump +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8 FORMAT ArrowStream" > ${CLICKHOUSE_TMP}/arrow_all_types_1.arrow +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types2 ORDER BY int8 FORMAT ArrowStream" > ${CLICKHOUSE_TMP}/arrow_all_types_2.arrow +echo diff: +diff ${CLICKHOUSE_TMP}/arrow_all_types_1.dump ${CLICKHOUSE_TMP}/arrow_all_types_2.dump + +${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE arrow_types2" +${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types3 values ( 79, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str01', 'fstr1', '2003-03-04', '2004-05-06')" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types3 ORDER BY int8 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types2 FORMAT ArrowStream" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types3 FORMAT ArrowStream" + +${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types4 values ( 80, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str02', 'fstr2', '2005-03-04 05:06:07', '2006-08-09 10:11:12')" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types4 ORDER BY int8 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types2 FORMAT ArrowStream" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types4 FORMAT ArrowStream" + +echo dest: +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types2 ORDER BY int8" +echo min: +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types3 ORDER BY int8" +echo max: +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types4 ORDER BY int8" + + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types5" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types6" +${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE arrow_types2" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types5 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime)) ENGINE = Memory" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime)) ENGINE = Memory" +${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types5 ORDER BY int8 FORMAT ArrowStream" > ${CLICKHOUSE_TMP}/arrow_all_types_5.arrow +#${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types5 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types6 FORMAT ArrowStream" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types5 ORDER BY int8 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types6 FORMAT ArrowStream" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types6 FORMAT ArrowStream" +echo dest from null: +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types6 ORDER BY int8" + +${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_types5" +${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_types6" + + +${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_types1" +${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_types2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_types3" +${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_types4" + From ab37791134b7317c99679ef5e4cb738c2ff9d9a4 Mon Sep 17 00:00:00 2001 From: hcz Date: Thu, 21 May 2020 15:28:10 +0800 Subject: [PATCH 03/79] Fix arrow stream termination --- src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index c8a098418b7..5468fbcac16 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -39,12 +39,12 @@ Chunk ArrowBlockInputFormat::generate() if (stream) { read_status = stream_reader->ReadNext(&single_batch[0]); + if (!single_batch[0]) + return res; } else { read_status = file_reader->ReadRecordBatch(record_batch_current, &single_batch[0]); - if (!single_batch[0]) - return res; } if (!read_status.ok()) throw Exception{"Error while reading batch of Arrow data: " + read_status.ToString(), From 95e2f0fa0769e87cbf77cb8d264c75a6fb198a8d Mon Sep 17 00:00:00 2001 From: hcz Date: Thu, 21 May 2020 18:20:41 +0800 Subject: [PATCH 04/79] Add brief docs about apache arrow --- docs/en/interfaces/formats.md | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 4697fbca7e1..ed411a1b0d4 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -37,6 +37,8 @@ The supported formats are: | [Avro](#data-format-avro) | ✔ | ✔ | | [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ | | [Parquet](#data-format-parquet) | ✔ | ✔ | +| [Arrow](#data-format-arrow) | ✔ | ✔ | +| [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ | | [ORC](#data-format-orc) | ✔ | ✗ | | [RowBinary](#rowbinary) | ✔ | ✔ | | [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ | @@ -985,9 +987,9 @@ See also [how to read/write length-delimited protobuf messages in popular langua ## Avro {#data-format-avro} -[Apache Avro](http://avro.apache.org/) is a row-oriented data serialization framework developed within Apache’s Hadoop project. +[Apache Avro](https://avro.apache.org/) is a row-oriented data serialization framework developed within Apache’s Hadoop project. -ClickHouse Avro format supports reading and writing [Avro data files](http://avro.apache.org/docs/current/spec.html#Object+Container+Files). +ClickHouse Avro format supports reading and writing [Avro data files](https://avro.apache.org/docs/current/spec.html#Object+Container+Files). ### Data Types Matching {#data_types-matching} @@ -1009,7 +1011,7 @@ The table below shows supported data types and how they match ClickHouse [data t | `long (timestamp-millis)` \* | [DateTime64(3)](../sql-reference/data-types/datetime.md) | `long (timestamp-millis)` \* | | `long (timestamp-micros)` \* | [DateTime64(6)](../sql-reference/data-types/datetime.md) | `long (timestamp-micros)` \* | -\* [Avro logical types](http://avro.apache.org/docs/current/spec.html#Logical+Types) +\* [Avro logical types](https://avro.apache.org/docs/current/spec.html#Logical+Types) Unsupported Avro data types: `record` (non-root), `map` @@ -1095,7 +1097,7 @@ SELECT * FROM topic1_stream; ## Parquet {#data-format-parquet} -[Apache Parquet](http://parquet.apache.org/) is a columnar storage format widespread in the Hadoop ecosystem. ClickHouse supports read and write operations for this format. +[Apache Parquet](https://parquet.apache.org/) is a columnar storage format widespread in the Hadoop ecosystem. ClickHouse supports read and write operations for this format. ### Data Types Matching {#data_types-matching-2} @@ -1141,6 +1143,16 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Parquet" > {some_ To exchange data with Hadoop, you can use [HDFS table engine](../engines/table-engines/integrations/hdfs.md). +## Arrow {data-format-arrow} + +[Apache Arrow](https://arrow.apache.org/) comes with two built-in columnar storage formats. ClickHouse supports read and write operations for these formats. + +`Arrow` is Apache Arrow's "file mode" format. It is designed for in-memory random access. + +## ArrowStream {data-format-arrow-stream} + +`ArrowStream` is Apache Arrow's "stream mode" format. It is designed for in-memory stream processing. + ## ORC {#data-format-orc} [Apache ORC](https://orc.apache.org/) is a columnar storage format widespread in the Hadoop ecosystem. You can only insert data in this format to ClickHouse. From 409cff11df96b810f3dbe8adb75c62d71894d10f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 22 May 2020 00:37:47 +0300 Subject: [PATCH 05/79] Add port() function It uses domain() (getURLHost()) internally so it has the same limitations: - no ipv6 support - no host-without-dots support (i.e. "foobar") --- .../sql-reference/functions/url-functions.md | 4 + src/Functions/URL/port.cpp | 95 +++++++++++++++++++ src/Functions/URL/registerFunctionsURL.cpp | 2 + .../queries/0_stateless/01284_port.reference | 21 ++++ tests/queries/0_stateless/01284_port.sql | 25 +++++ 5 files changed, 147 insertions(+) create mode 100644 src/Functions/URL/port.cpp create mode 100644 tests/queries/0_stateless/01284_port.reference create mode 100644 tests/queries/0_stateless/01284_port.sql diff --git a/docs/en/sql-reference/functions/url-functions.md b/docs/en/sql-reference/functions/url-functions.md index 143bd42b08e..f35c621a6e2 100644 --- a/docs/en/sql-reference/functions/url-functions.md +++ b/docs/en/sql-reference/functions/url-functions.md @@ -117,6 +117,10 @@ Returns the part of the domain that includes top-level subdomains up to the “f For example, `cutToFirstSignificantSubdomain('https://news.yandex.com.tr/') = 'yandex.com.tr'`. +### port(URL) {#port} + +Returns the port or zero if there is no port in the URL (or in case of validation error). + ### path {#path} Returns the path. Example: `/top/news.html` The path does not include the query string. diff --git a/src/Functions/URL/port.cpp b/src/Functions/URL/port.cpp new file mode 100644 index 00000000000..d4876e9f647 --- /dev/null +++ b/src/Functions/URL/port.cpp @@ -0,0 +1,95 @@ +#include +#include +#include +#include "domain.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +struct ExtractPort +{ + static constexpr auto name = "port"; + static constexpr auto is_fixed_to_constant = true; + + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray & res) + { + size_t size = offsets.size(); + + ColumnString::Offset prev_offset = 0; + for (size_t i = 0; i < size; ++i) + { + res[i] = parse(data, prev_offset, offsets[i] - prev_offset - 1); + prev_offset = offsets[i]; + } + } + + static void vectorFixedToConstant(const ColumnString::Chars & data, size_t n, UInt16 & res) { res = parse(data, 0, n); } + + static void vectorFixedToVector(const ColumnString::Chars & data, size_t n, PaddedPODArray & res) + { + size_t size = data.size() / n; + + for (size_t i = 0; i < size; ++i) + { + res[i] = parse(data, i * n, n); + } + } + + [[noreturn]] static void array(const ColumnString::Offsets &, PaddedPODArray &) + { + throw Exception("Cannot apply function port to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + +private: + static UInt16 parse(const ColumnString::Chars & buf, size_t offset, size_t size) + { + const char * p = reinterpret_cast(&buf[0]) + offset; + const char * end = p + size; + + StringRef host = getURLHost(p, size); + if (!host.size) + return 0; + if (host.size == size) + return 0; + + p = host.data + host.size; + if (*p++ != ':') + return 0; + + Int64 port = 0; + while (p < end) + { + if (*p == '/') + break; + if (!isNumericASCII(*p)) + return 0; + + port = (port * 10) + (*p - '0'); + if (port < 0 || port > UInt16(-1)) + return 0; + ++p; + } + return port; + } +}; + +struct NamePort +{ + static constexpr auto name = "port"; +}; + +using FunctionPort = FunctionStringOrArrayToT; + +void registerFunctionPort(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + diff --git a/src/Functions/URL/registerFunctionsURL.cpp b/src/Functions/URL/registerFunctionsURL.cpp index 66a847185f3..9ba5261f728 100644 --- a/src/Functions/URL/registerFunctionsURL.cpp +++ b/src/Functions/URL/registerFunctionsURL.cpp @@ -8,6 +8,7 @@ void registerFunctionDomain(FunctionFactory & factory); void registerFunctionDomainWithoutWWW(FunctionFactory & factory); void registerFunctionFirstSignificantSubdomain(FunctionFactory & factory); void registerFunctionTopLevelDomain(FunctionFactory & factory); +void registerFunctionPort(FunctionFactory & factory); void registerFunctionPath(FunctionFactory & factory); void registerFunctionPathFull(FunctionFactory & factory); void registerFunctionQueryString(FunctionFactory & factory); @@ -33,6 +34,7 @@ void registerFunctionsURL(FunctionFactory & factory) registerFunctionDomainWithoutWWW(factory); registerFunctionFirstSignificantSubdomain(factory); registerFunctionTopLevelDomain(factory); + registerFunctionPort(factory); registerFunctionPath(factory); registerFunctionPathFull(factory); registerFunctionQueryString(factory); diff --git a/tests/queries/0_stateless/01284_port.reference b/tests/queries/0_stateless/01284_port.reference new file mode 100644 index 00000000000..5bf1c035ca8 --- /dev/null +++ b/tests/queries/0_stateless/01284_port.reference @@ -0,0 +1,21 @@ +ipv4 +0 +80 +80 +80 +80 +hostname +0 +80 +80 +80 +80 +ipv6 +0 +0 +0 +0 +0 +0 +host-no-dot +0 diff --git a/tests/queries/0_stateless/01284_port.sql b/tests/queries/0_stateless/01284_port.sql new file mode 100644 index 00000000000..3ba5aecd691 --- /dev/null +++ b/tests/queries/0_stateless/01284_port.sql @@ -0,0 +1,25 @@ +select 'ipv4'; +select port('http://127.0.0.1/'); +select port('http://127.0.0.1:80'); +select port('http://127.0.0.1:80/'); +select port('//127.0.0.1:80/'); +select port('127.0.0.1:80'); +select 'hostname'; +select port('http://foobar.com/'); +select port('http://foobar.com:80'); +select port('http://foobar.com:80/'); +select port('//foobar.com:80/'); +select port('foobar.com:80'); + +-- +-- Known limitations of domain() (getURLHost()) +-- +select 'ipv6'; +select port('http://[2001:db8::8a2e:370:7334]/'); +select port('http://[2001:db8::8a2e:370:7334]:80'); +select port('http://[2001:db8::8a2e:370:7334]:80/'); +select port('//[2001:db8::8a2e:370:7334]:80/'); +select port('[2001:db8::8a2e:370:7334]:80'); +select port('2001:db8::8a2e:370:7334:80'); +select 'host-no-dot'; +select port('//foobar:80/'); From aa5247f1fed85e64479985dc0de65621118a249d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 21 May 2020 23:58:55 +0300 Subject: [PATCH 06/79] Get back 01281_group_by_limit_memory_tracking (without flackiness) --- ...1_group_by_limit_memory_tracking.reference | 0 .../01281_group_by_limit_memory_tracking.sh | 34 +++++++++++++++++++ 2 files changed, 34 insertions(+) create mode 100644 tests/queries/0_stateless/01281_group_by_limit_memory_tracking.reference create mode 100755 tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh diff --git a/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.reference b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh new file mode 100755 index 00000000000..79a731b9273 --- /dev/null +++ b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh @@ -0,0 +1,34 @@ +#!/usr/bin/env bash + +# Regression for MemoryTracker that had been incorrectly accounted +# (it was reseted before deallocation) +# +# For this will be used: +# - two-level group by +# - max_memory_usage_for_user +# - one users' query in background (to avoid reseting max_memory_usage_for_user) + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +set -o pipefail + +function execute_null() +{ + ${CLICKHOUSE_CLIENT} --format Null -n "$@" +} + +function execute_group_by() +{ + local opts=( + --max_memory_usage_for_user=$((150<<20)) + --max_threads=2 + ) + execute_null "${opts[@]}" <<<'SELECT uniq(number) FROM numbers_mt(toUInt64(1e6)) GROUP BY number % 5e5' +} + +execute_null <<<'SELECT sleep(3)' & +execute_group_by +# if memory accounting will be incorrect, the second query will be failed with MEMORY_LIMIT_EXCEEDED +execute_group_by +wait From 9de1a231c9b003c306570cedbf380b4bff922294 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 23 May 2020 21:14:52 +0300 Subject: [PATCH 07/79] Add optional default_port argument for the port() function --- .../sql-reference/functions/url-functions.md | 4 +- src/Functions/URL/port.cpp | 110 ++++++++++++------ .../queries/0_stateless/01284_port.reference | 3 + tests/queries/0_stateless/01284_port.sql | 9 ++ 4 files changed, 86 insertions(+), 40 deletions(-) diff --git a/docs/en/sql-reference/functions/url-functions.md b/docs/en/sql-reference/functions/url-functions.md index f35c621a6e2..34477529649 100644 --- a/docs/en/sql-reference/functions/url-functions.md +++ b/docs/en/sql-reference/functions/url-functions.md @@ -117,9 +117,9 @@ Returns the part of the domain that includes top-level subdomains up to the “f For example, `cutToFirstSignificantSubdomain('https://news.yandex.com.tr/') = 'yandex.com.tr'`. -### port(URL) {#port} +### port(URL[, default_port = 0]) {#port} -Returns the port or zero if there is no port in the URL (or in case of validation error). +Returns the port or `default_port` if there is no port in the URL (or in case of validation error). ### path {#path} diff --git a/src/Functions/URL/port.cpp b/src/Functions/URL/port.cpp index d4876e9f647..e703da2602b 100644 --- a/src/Functions/URL/port.cpp +++ b/src/Functions/URL/port.cpp @@ -1,6 +1,10 @@ #include -#include +#include #include +#include +#include +#include +#include #include "domain.h" @@ -9,83 +13,113 @@ namespace DB namespace ErrorCodes { + extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -struct ExtractPort +struct FunctionPort : public IFunction { static constexpr auto name = "port"; - static constexpr auto is_fixed_to_constant = true; + static FunctionPtr create(const Context &) { return std::make_shared(); } - static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray & res) + String getName() const override { return name; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 1 && arguments.size() != 2) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + std::to_string(arguments.size()) + ", should be 1 or 2", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!WhichDataType(arguments[0].type).isString()) + throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + ". Must be String.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (arguments.size() == 2 && !WhichDataType(arguments[1].type).isUInt16()) + throw Exception("Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() + ". Must be UInt16.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override + { + UInt16 default_port = 0; + if (arguments.size() == 2) + { + const auto * port_column = checkAndGetColumn(block.getByPosition(arguments[1]).column.get()); + if (!port_column) + throw Exception("Second argument for function " + getName() + " must be constant UInt16", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + default_port = port_column->getValue(); + } + + const ColumnPtr url_column = block.getByPosition(arguments[0]).column; + if (const ColumnString * url_strs = checkAndGetColumn(url_column.get())) + { + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_res = col_res->getData(); + vec_res.resize(url_column->size()); + + vector(default_port, url_strs->getChars(), url_strs->getOffsets(), vec_res); + block.getByPosition(result).column = std::move(col_res); + } + else + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); +} + +private: + static void vector(UInt16 default_port, const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray & res) { size_t size = offsets.size(); ColumnString::Offset prev_offset = 0; for (size_t i = 0; i < size; ++i) { - res[i] = parse(data, prev_offset, offsets[i] - prev_offset - 1); + res[i] = extractPort(default_port, data, prev_offset, offsets[i] - prev_offset - 1); prev_offset = offsets[i]; } - } +} - static void vectorFixedToConstant(const ColumnString::Chars & data, size_t n, UInt16 & res) { res = parse(data, 0, n); } - - static void vectorFixedToVector(const ColumnString::Chars & data, size_t n, PaddedPODArray & res) - { - size_t size = data.size() / n; - - for (size_t i = 0; i < size; ++i) - { - res[i] = parse(data, i * n, n); - } - } - - [[noreturn]] static void array(const ColumnString::Offsets &, PaddedPODArray &) - { - throw Exception("Cannot apply function port to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - -private: - static UInt16 parse(const ColumnString::Chars & buf, size_t offset, size_t size) + static UInt16 extractPort(UInt16 default_port, const ColumnString::Chars & buf, size_t offset, size_t size) { const char * p = reinterpret_cast(&buf[0]) + offset; const char * end = p + size; StringRef host = getURLHost(p, size); if (!host.size) - return 0; + return default_port; if (host.size == size) - return 0; + return default_port; p = host.data + host.size; if (*p++ != ':') - return 0; + return default_port; - Int64 port = 0; + Int64 port = default_port; while (p < end) { if (*p == '/') break; if (!isNumericASCII(*p)) - return 0; + return default_port; port = (port * 10) + (*p - '0'); if (port < 0 || port > UInt16(-1)) - return 0; + return default_port; ++p; } return port; } }; -struct NamePort -{ - static constexpr auto name = "port"; -}; - -using FunctionPort = FunctionStringOrArrayToT; - void registerFunctionPort(FunctionFactory & factory) { factory.registerFunction(); diff --git a/tests/queries/0_stateless/01284_port.reference b/tests/queries/0_stateless/01284_port.reference index 5bf1c035ca8..7e776595065 100644 --- a/tests/queries/0_stateless/01284_port.reference +++ b/tests/queries/0_stateless/01284_port.reference @@ -10,6 +10,9 @@ hostname 80 80 80 +default-port +80 +80 ipv6 0 0 diff --git a/tests/queries/0_stateless/01284_port.sql b/tests/queries/0_stateless/01284_port.sql index 3ba5aecd691..9c31a5d42ad 100644 --- a/tests/queries/0_stateless/01284_port.sql +++ b/tests/queries/0_stateless/01284_port.sql @@ -11,6 +11,15 @@ select port('http://foobar.com:80/'); select port('//foobar.com:80/'); select port('foobar.com:80'); +select 'default-port'; +select port('http://127.0.0.1/', toUInt16(80)); +select port('http://foobar.com/', toUInt16(80)); + +-- unsupported +/* ILLEGAL_TYPE_OF_ARGUMENT */ select port(toFixedString('', 1)); -- { serverError 43; } +/* ILLEGAL_TYPE_OF_ARGUMENT */ select port('', 1); -- { serverError 43; } +/* NUMBER_OF_ARGUMENTS_DOESNT_MATCH */ select port('', 1, 1); -- { serverError 42; } + -- -- Known limitations of domain() (getURLHost()) -- From 96260cf33a12e906e8db3303c057713db954f1b9 Mon Sep 17 00:00:00 2001 From: Albert Kidrachev Date: Mon, 25 May 2020 02:55:00 +0300 Subject: [PATCH 08/79] init --- src/Common/SharedBlockRowRef.h | 30 +++++++ src/Interpreters/InterpreterSelectQuery.cpp | 1 + .../OptimizedPartialSortingTransform.cpp | 83 +++++++++++++++++++ .../OptimizedPartialSortingTransform.h | 34 ++++++++ src/Processors/ya.make | 1 + 5 files changed, 149 insertions(+) create mode 100644 src/Processors/Transforms/OptimizedPartialSortingTransform.cpp create mode 100644 src/Processors/Transforms/OptimizedPartialSortingTransform.h diff --git a/src/Common/SharedBlockRowRef.h b/src/Common/SharedBlockRowRef.h index 193f7e4dd05..e9fd076da07 100644 --- a/src/Common/SharedBlockRowRef.h +++ b/src/Common/SharedBlockRowRef.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -86,4 +87,33 @@ struct SharedBlockRowRef } }; +struct SharedBlockRowWithSortDescriptionRef : SharedBlockRowRef +{ + SortDescription * description = nullptr; + + void set(SharedBlockPtr & shared_block_, ColumnRawPtrs * columns_, size_t row_num_) = delete; + + bool operator< (const SharedBlockRowRef & other) const + { + size_t size = columns->size(); + for (size_t i = 0; i < size; ++i) + { + int res = (*description)[i].direction * (*columns)[i]->compareAt(row_num, other.row_num, *(*other.columns)[i], 1); + if (res < 0) + return true; + else if (res > 0) + return false; + } + return false; + } + + void set(SharedBlockPtr & shared_block_, ColumnRawPtrs * columns_, size_t row_num_, SortDescription * description_) + { + shared_block = shared_block_; + columns = columns_; + row_num = row_num_; + description = description_; + } +}; + } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c8d842dfa53..3d95265cb27 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -84,6 +84,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Processors/Transforms/OptimizedPartialSortingTransform.cpp b/src/Processors/Transforms/OptimizedPartialSortingTransform.cpp new file mode 100644 index 00000000000..e753052a2b3 --- /dev/null +++ b/src/Processors/Transforms/OptimizedPartialSortingTransform.cpp @@ -0,0 +1,83 @@ +#include +#include +#include + +namespace DB +{ + +OptimizedPartialSortingTransform::OptimizedPartialSortingTransform( + const Block & header_, SortDescription & description_, UInt64 limit_) + : ISimpleTransform(header_, header_, false) + , description(description_), limit(limit_) + , threshold_shared_block(nullptr) +{ +} + +static ColumnRawPtrs extractColumns(const Block & block, const SortDescription& description) +{ + size_t size = description.size(); + ColumnRawPtrs res; + res.reserve(size); + + for (size_t i = 0; i < size; ++i) + { + const IColumn * column = !description[i].column_name.empty() + ? block.getByName(description[i].column_name).column.get() + : block.safeGetByPosition(description[i].column_number).column.get(); + res.emplace_back(column); + } + + return res; +} + +void OptimizedPartialSortingTransform::transform(Chunk & chunk) +{ + if (read_rows) + read_rows->add(chunk.getNumRows()); + + auto block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); + chunk.clear(); + + SharedBlockPtr shared_block = new detail::SharedBlock(std::move(block)); + UInt64 rows_num = shared_block->rows(); + + + if (threshold_shared_block) { + SharedBlockRowWithSortDescriptionRef row; + IColumn::Filter filter(rows_num); + ColumnRawPtrs shared_block_columns = extractColumns(*shared_block, description); + size_t filtered_count = 0; + + for (UInt64 i = 0; i < rows_num; ++i) { + row.set(shared_block, &shared_block_columns, i, &description); + + if (threshold_row < row) + { + ++filtered_count; + filter[i] = 1; + } + } + + if (filtered_count) + { + for (auto & column : shared_block->getColumns()) + { + column = column->filter(filter, filtered_count); + } + } + } + + sortBlock(*shared_block, description, limit); + + if (!threshold_shared_block && limit && limit < rows_num) + { + Block threshold_block = shared_block->cloneWithColumns(shared_block->getColumns()); + threshold_shared_block = new detail::SharedBlock(std::move(threshold_block)); + threshold_block_columns = extractColumns(*threshold_shared_block, description); + threshold_row.set(threshold_shared_block, &threshold_block_columns, limit - 1, &description); + } + + chunk.setColumns(shared_block->getColumns(), shared_block->rows()); +} + +} diff --git a/src/Processors/Transforms/OptimizedPartialSortingTransform.h b/src/Processors/Transforms/OptimizedPartialSortingTransform.h new file mode 100644 index 00000000000..20e72bd836f --- /dev/null +++ b/src/Processors/Transforms/OptimizedPartialSortingTransform.h @@ -0,0 +1,34 @@ +#pragma once +#include +#include +#include +#include + + +namespace DB +{ +class OptimizedPartialSortingTransform : public ISimpleTransform +{ +public: + OptimizedPartialSortingTransform( + const Block & header_, + SortDescription & description_, + UInt64 limit_ = 0); + + String getName() const override { return "OptimizedPartialSortingTransform"; } + + void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) { read_rows.swap(counter); } + +protected: + void transform(Chunk & chunk) override; + +private: + SortDescription description; + UInt64 limit; + RowsBeforeLimitCounterPtr read_rows; + SharedBlockRowWithSortDescriptionRef threshold_row; + SharedBlockPtr threshold_shared_block; + ColumnRawPtrs threshold_block_columns; +}; + +} diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 62320f1c147..5952341527b 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -129,6 +129,7 @@ SRCS( Transforms/MergeSortingTransform.cpp Transforms/MergingAggregatedMemoryEfficientTransform.cpp Transforms/MergingAggregatedTransform.cpp + Transforms/OptimizedPartialSortingTransform.cpp Transforms/PartialSortingTransform.cpp Transforms/ReverseTransform.cpp Transforms/RollupTransform.cpp From 3abf7b7dcdc26acda9aae9663e9e026f1b5cb631 Mon Sep 17 00:00:00 2001 From: Albert Kidrachev Date: Mon, 25 May 2020 03:32:08 +0300 Subject: [PATCH 09/79] fix --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 3d95265cb27..c3fce45ac61 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2109,7 +2109,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting if (stream_type != QueryPipeline::StreamType::Main) return nullptr; - return std::make_shared(header, output_order_descr, limit); + return std::make_shared(header, output_order_descr, limit); }); pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr From f2e3bcdd15d5436ea2c8866457964f6c3572978f Mon Sep 17 00:00:00 2001 From: hcz Date: Mon, 25 May 2020 10:50:55 +0800 Subject: [PATCH 10/79] Improve code style in src and test files of Arrow formats --- src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp | 9 +++++++-- src/Processors/Formats/Impl/ArrowBlockInputFormat.h | 4 ++++ tests/queries/0_stateless/01273_arrow.reference | 2 -- tests/queries/0_stateless/01273_arrow.sh | 11 ----------- .../queries/0_stateless/01273_arrow_stream.reference | 2 -- tests/queries/0_stateless/01273_arrow_stream.sh | 11 ----------- 6 files changed, 11 insertions(+), 28 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index 5468fbcac16..e0eb0e21567 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -31,11 +31,15 @@ Chunk ArrowBlockInputFormat::generate() Chunk res; const Block & header = getPort().getHeader(); - if (!stream && record_batch_current >= record_batch_total) - return res; + if (!stream) + { + if (record_batch_current >= record_batch_total) + return res; + } std::vector> single_batch(1); arrow::Status read_status; + if (stream) { read_status = stream_reader->ReadNext(&single_batch[0]); @@ -46,6 +50,7 @@ Chunk ArrowBlockInputFormat::generate() { read_status = file_reader->ReadRecordBatch(record_batch_current, &single_batch[0]); } + if (!read_status.ok()) throw Exception{"Error while reading batch of Arrow data: " + read_status.ToString(), ErrorCodes::CANNOT_READ_ALL_DATA}; diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index 2e7f647b4f7..5ad112efde9 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -25,9 +25,13 @@ protected: Chunk generate() override; private: + // Whether to use ArrowStream format bool stream; + // This field is only used for ArrowStream format std::shared_ptr stream_reader; + // The following fields are used only for Arrow format std::shared_ptr file_reader; + int record_batch_total = 0; int record_batch_current = 0; diff --git a/tests/queries/0_stateless/01273_arrow.reference b/tests/queries/0_stateless/01273_arrow.reference index 0f4be2c74a0..0dc503f65e4 100644 --- a/tests/queries/0_stateless/01273_arrow.reference +++ b/tests/queries/0_stateless/01273_arrow.reference @@ -31,8 +31,6 @@ 992 991 990 -ContextLock Number of times the lock of Context was acquired or tried to acquire. This is global lock. -Query Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries. original: -128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06 -108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 diff --git a/tests/queries/0_stateless/01273_arrow.sh b/tests/queries/0_stateless/01273_arrow.sh index f659a81f118..deb0aa20ccc 100755 --- a/tests/queries/0_stateless/01273_arrow.sh +++ b/tests/queries/0_stateless/01273_arrow.sh @@ -34,14 +34,6 @@ ${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_numbers ORDER BY number DESC L ${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_numbers" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_events" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_events (event String, value UInt64, description String) ENGINE = Memory" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.events FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_events FORMAT Arrow" -${CLICKHOUSE_CLIENT} --query="SELECT event, description FROM arrow_events WHERE event IN ('ContextLock', 'Query') ORDER BY event" -${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_events" - - - ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types1" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types2" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types3" @@ -61,8 +53,6 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types1 values ( -128, # max ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06')" -# 'SELECT -127,-128,-129,126,127,128,255,256,257,-32767,-32768,-32769,32766,32767,32768,65535,65536,65537, -2147483647,-2147483648,-2147483649,2147483646,2147483647,2147483648,4294967295,4294967296,4294967297, -9223372036854775807,-9223372036854775808,9223372036854775806,9223372036854775807,9223372036854775808,18446744073709551615'; - ${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types2 FORMAT Arrow" echo original: @@ -98,7 +88,6 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types5 (int8 Nullable(Int ${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime)) ENGINE = Memory" ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types5 ORDER BY int8 FORMAT Arrow" > ${CLICKHOUSE_TMP}/arrow_all_types_5.arrow -#${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types5 FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types6 FORMAT Arrow" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types5 ORDER BY int8 FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types6 FORMAT Arrow" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8 FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types6 FORMAT Arrow" echo dest from null: diff --git a/tests/queries/0_stateless/01273_arrow_stream.reference b/tests/queries/0_stateless/01273_arrow_stream.reference index 0f4be2c74a0..0dc503f65e4 100644 --- a/tests/queries/0_stateless/01273_arrow_stream.reference +++ b/tests/queries/0_stateless/01273_arrow_stream.reference @@ -31,8 +31,6 @@ 992 991 990 -ContextLock Number of times the lock of Context was acquired or tried to acquire. This is global lock. -Query Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries. original: -128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06 -108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 diff --git a/tests/queries/0_stateless/01273_arrow_stream.sh b/tests/queries/0_stateless/01273_arrow_stream.sh index 2fc630a3a8b..a4bb403cb5a 100755 --- a/tests/queries/0_stateless/01273_arrow_stream.sh +++ b/tests/queries/0_stateless/01273_arrow_stream.sh @@ -34,14 +34,6 @@ ${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_numbers ORDER BY number DESC L ${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_numbers" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_events" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_events (event String, value UInt64, description String) ENGINE = Memory" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.events FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_events FORMAT ArrowStream" -${CLICKHOUSE_CLIENT} --query="SELECT event, description FROM arrow_events WHERE event IN ('ContextLock', 'Query') ORDER BY event" -${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_events" - - - ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types1" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types2" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_types3" @@ -61,8 +53,6 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types1 values ( -128, # max ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06')" -# 'SELECT -127,-128,-129,126,127,128,255,256,257,-32767,-32768,-32769,32766,32767,32768,65535,65536,65537, -2147483647,-2147483648,-2147483649,2147483646,2147483647,2147483648,4294967295,4294967296,4294967297, -9223372036854775807,-9223372036854775808,9223372036854775806,9223372036854775807,9223372036854775808,18446744073709551615'; - ${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types2 FORMAT ArrowStream" echo original: @@ -98,7 +88,6 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types5 (int8 Nullable(Int ${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime)) ENGINE = Memory" ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types5 ORDER BY int8 FORMAT ArrowStream" > ${CLICKHOUSE_TMP}/arrow_all_types_5.arrow -#${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types5 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types6 FORMAT ArrowStream" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types5 ORDER BY int8 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types6 FORMAT ArrowStream" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_types1 ORDER BY int8 FORMAT ArrowStream" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_types6 FORMAT ArrowStream" echo dest from null: From 548fc7141a47583ae33ab02e716cd338a0b7300e Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 25 May 2020 21:12:03 +0300 Subject: [PATCH 11/79] Enable log_queries by default --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index be7c09ec6c2..142e0872d72 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -156,7 +156,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, priority, 0, "Priority of the query. 1 - the highest, higher value - lower priority; 0 - do not use priorities.", 0) \ M(SettingInt64, os_thread_priority, 0, "If non zero - set corresponding 'nice' value for query processing threads. Can be used to adjust query priority for OS scheduler.", 0) \ \ - M(SettingBool, log_queries, 0, "Log requests and write the log to the system table.", 0) \ + M(SettingBool, log_queries, 1, "Log requests and write the log to the system table.", 0) \ M(SettingLogQueriesType, log_queries_min_type, QueryLogElementType::QUERY_START, "query_log minimal type to log, possible values (from low to high): QUERY_START, QUERY_FINISH, EXCEPTION_BEFORE_START, EXCEPTION_WHILE_PROCESSING.", 0) \ M(SettingUInt64, log_queries_cut_to_length, 100000, "If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.", 0) \ \ From 12384551da5fd4d280eb95c356da449c31761d01 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 May 2020 16:19:24 +0300 Subject: [PATCH 12/79] Fix totals and extremes header for views. --- src/Processors/Pipe.h | 2 +- src/Processors/QueryPipeline.cpp | 13 ++++++++++- src/Processors/QueryPipeline.h | 3 ++- src/Storages/StorageView.cpp | 37 ++++++++++++++++++++++---------- 4 files changed, 41 insertions(+), 14 deletions(-) diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 984fa7605c6..ec5514915a7 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -23,7 +23,7 @@ public: /// Will connect pipes outputs with transform inputs automatically. Pipe(Pipes && pipes, ProcessorPtr transform); /// Create pipe from output port. If pipe was created that way, it possibly will not have tree shape. - Pipe(OutputPort * port); + explicit Pipe(OutputPort * port); Pipe(const Pipe & other) = delete; Pipe(Pipe && other) = default; diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 13787a3fd3b..26736672412 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -703,6 +703,11 @@ void QueryPipeline::initRowsBeforeLimit() Pipe QueryPipeline::getPipe() && { resize(1); + return std::move(std::move(*this).getPipes()[0]); +} + +Pipes QueryPipeline::getPipes() && +{ Pipe pipe(std::move(processors), streams.at(0), totals_having_port, extremes_port); pipe.max_parallel_streams = streams.maxParallelStreams(); @@ -721,7 +726,13 @@ Pipe QueryPipeline::getPipe() && if (extremes_port) pipe.setExtremesPort(extremes_port); - return pipe; + Pipes pipes; + pipes.emplace_back(std::move(pipe)); + + for (size_t i = 1; i < streams.size(); ++i) + pipes.emplace_back(Pipe(streams[i])); + + return pipes; } PipelineExecutorPtr QueryPipeline::execute() diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 45e38ffa715..d248853131c 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -155,8 +155,9 @@ public: /// Set upper limit for the recommend number of threads void setMaxThreads(size_t max_threads_) { max_threads = max_threads_; } - /// Convert query pipeline to single pipe. + /// Convert query pipeline to single or several pipes. Pipe getPipe() &&; + Pipes getPipes() &&; private: /// Destruction order: processors, header, locks, temporary storages, local contexts diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 636c7f9d64d..01c85c5238a 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -19,6 +19,8 @@ #include #include #include +#include +#include namespace DB @@ -62,29 +64,42 @@ Pipes StorageView::read( if (context.getSettings().enable_optimize_predicate_expression) current_inner_query = getRuntimeViewQuery(*query_info.query->as(), context); - QueryPipeline pipeline; InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, {}, column_names); /// FIXME res may implicitly use some objects owned be pipeline, but them will be destructed after return if (query_info.force_tree_shaped_pipeline) { + QueryPipeline pipeline; BlockInputStreams streams = interpreter.executeWithMultipleStreams(pipeline); + + for (auto & stream : streams) + { + stream = std::make_shared(stream); + stream = std::make_shared(stream, getSampleBlockForColumns(column_names), + ConvertingBlockInputStream::MatchColumnsMode::Name); + } + for (auto & stream : streams) pipes.emplace_back(std::make_shared(std::move(stream))); } else - /// TODO: support multiple streams here. Need more general interface than pipes. - pipes.emplace_back(interpreter.executeWithProcessors().getPipe()); - - /// It's expected that the columns read from storage are not constant. - /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. - for (auto & pipe : pipes) { - pipe.addSimpleTransform(std::make_shared(pipe.getHeader())); + auto pipeline = interpreter.executeWithProcessors(); + + /// It's expected that the columns read from storage are not constant. + /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. + pipeline.addSimpleTransform([](const Block & header) + { + return std::make_shared(header); + }); /// And also convert to expected structure. - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), getSampleBlockForColumns(column_names), - ConvertingTransform::MatchColumnsMode::Name)); + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, getSampleBlockForColumns(column_names), + ConvertingTransform::MatchColumnsMode::Name); + }); + + pipes = std::move(pipeline).getPipes(); } return pipes; From 8074e2027fd76e03725598fbdf5e15565301441b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 May 2020 16:32:44 +0300 Subject: [PATCH 13/79] Added test. --- .../queries/0_stateless/01284_view_and_extremes_bug.reference | 1 + tests/queries/0_stateless/01284_view_and_extremes_bug.sql | 4 ++++ 2 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/01284_view_and_extremes_bug.reference create mode 100644 tests/queries/0_stateless/01284_view_and_extremes_bug.sql diff --git a/tests/queries/0_stateless/01284_view_and_extremes_bug.reference b/tests/queries/0_stateless/01284_view_and_extremes_bug.reference new file mode 100644 index 00000000000..216e97ce082 --- /dev/null +++ b/tests/queries/0_stateless/01284_view_and_extremes_bug.reference @@ -0,0 +1 @@ +World diff --git a/tests/queries/0_stateless/01284_view_and_extremes_bug.sql b/tests/queries/0_stateless/01284_view_and_extremes_bug.sql new file mode 100644 index 00000000000..c444441a258 --- /dev/null +++ b/tests/queries/0_stateless/01284_view_and_extremes_bug.sql @@ -0,0 +1,4 @@ +drop table if exists view_bug_const; +CREATE VIEW view_bug_const AS SELECT 'World' AS hello FROM (SELECT number FROM system.numbers LIMIT 1) AS n1 JOIN (SELECT number FROM system.numbers LIMIT 1) AS n2 USING (number); +select * from view_bug_const; +drop table if exists view_bug_const; From 90f49c0c17ff5ce8182a5d45543c5c3aab9985f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 May 2020 21:37:20 +0300 Subject: [PATCH 14/79] Fix error in performance test --- tests/performance/agg_functions_min_max_any.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/performance/agg_functions_min_max_any.xml b/tests/performance/agg_functions_min_max_any.xml index 4591182cdfc..9c16cb88970 100644 --- a/tests/performance/agg_functions_min_max_any.xml +++ b/tests/performance/agg_functions_min_max_any.xml @@ -35,10 +35,6 @@ select max(PageCharset) from test.hits where PageCharset != '' group by intHash32(UserID) % 1000000 FORMAT Null select any(PageCharset) from test.hits where PageCharset != '' group by intHash32(UserID) % 1000000 FORMAT Null select anyHeavy(PageCharset) from test.hits where PageCharset != '' group by intHash32(UserID) % 1000000 FORMAT Null -select min(OriginalURL) from test.hits where OriginalURL != '' group by intHash32(UserID) % 1000000 FORMAT Null -select max(OriginalURL) from test.hits where OriginalURL != '' group by intHash32(UserID) % 1000000 FORMAT Null -select any(OriginalURL) from test.hits where OriginalURL != '' group by intHash32(UserID) % 1000000 FORMAT Null -select anyHeavy(OriginalURL) from test.hits where OriginalURL != '' group by intHash32(UserID) % 1000000 FORMAT Null select min(SocialNetwork) from test.hits where SocialNetwork != '' group by intHash32(UserID) % 1000000 FORMAT Null select max(SocialNetwork) from test.hits where SocialNetwork != '' group by intHash32(UserID) % 1000000 FORMAT Null select any(SocialNetwork) from test.hits where SocialNetwork != '' group by intHash32(UserID) % 1000000 FORMAT Null From 15e7c618bb5023ed2d63465e2e3eed7e99503799 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 May 2020 22:21:07 +0300 Subject: [PATCH 15/79] Fix cctz linking with dirty hack --- contrib/cctz-cmake/CMakeLists.txt | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index df9fd6aa61c..d353fc9d4de 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -618,7 +618,11 @@ if (USE_INTERNAL_CCTZ) add_library(tzdata STATIC ${TZ_OBJS}) set_target_properties(tzdata PROPERTIES LINKER_LANGUAGE C) - target_link_libraries(cctz -Wl,--whole-archive tzdata -Wl,--no-whole-archive) # whole-archive prevents symbols from being discarded + # whole-archive prevents symbols from being discarded for unknown reason + # CMake can shuffle each of target_link_libraries arguments with other + # libraries in linker command. To avoid this we hardcode whole-archive + # library into single string. + target_link_libraries(cctz PUBLIC "-Wl,--whole-archive contrib/cctz-cmake/libtzdata.a -Wl,--no-whole-archive") endif () else () From 4a92ad11ec02a4a94b7b3567cd02a62df4dd37ea Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 26 May 2020 23:27:31 +0300 Subject: [PATCH 16/79] Update test.py --- tests/integration/test_system_queries/test.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_system_queries/test.py b/tests/integration/test_system_queries/test.py index 1761017362a..9397cd20ec7 100644 --- a/tests/integration/test_system_queries/test.py +++ b/tests/integration/test_system_queries/test.py @@ -95,6 +95,10 @@ def test_RELOAD_CONFIG_AND_MACROS(started_cluster): def test_SYSTEM_FLUSH_LOGS(started_cluster): instance = cluster.instances['ch1'] + instance.query(''' + SET log_queries = 0; + TRUNCATE TABLE system.query_log; + ''') for i in range(4): # Sleep to execute flushing from background thread at first query # by expiration of flush_interval_millisecond and test probable race condition. @@ -105,7 +109,10 @@ def test_SYSTEM_FLUSH_LOGS(started_cluster): SET log_queries = 0; SYSTEM FLUSH LOGS; SELECT count() FROM system.query_log;''') - instance.query('TRUNCATE TABLE system.query_log') + instance.query(''' + SET log_queries = 0; + TRUNCATE TABLE system.query_log; + ''') assert TSV(result) == TSV('4') From 734cc78e5838bafc35a3abc544fd7609c09902ca Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 May 2020 13:16:34 +0300 Subject: [PATCH 17/79] jemalloc: make include_linux_*/jemalloc/*.h common for aarch64 and x86_64 This patch can go upstream, regardless of the jemalloc update eventually. Also I noticed that header with jemalloc version differs (someone forgot to update?) --- .../jemalloc/jemalloc_defs.h | 0 .../jemalloc/jemalloc_macros.h | 0 .../jemalloc/jemalloc_protos.h | 0 .../jemalloc/jemalloc_typedefs.h | 18 +-- .../jemalloc/jemalloc_macros.h | 123 ------------------ .../jemalloc/jemalloc_defs.h | 43 ------ .../jemalloc/jemalloc_protos.h | 66 ---------- .../jemalloc/jemalloc_typedefs.h | 77 ----------- 8 files changed, 9 insertions(+), 318 deletions(-) rename contrib/jemalloc-cmake/{include_linux_aarch64 => include}/jemalloc/jemalloc_defs.h (100%) rename contrib/jemalloc-cmake/{include_linux_x86_64 => include}/jemalloc/jemalloc_macros.h (100%) rename contrib/jemalloc-cmake/{include_linux_aarch64 => include}/jemalloc/jemalloc_protos.h (100%) rename contrib/jemalloc-cmake/{include_linux_aarch64 => include}/jemalloc/jemalloc_typedefs.h (88%) delete mode 100644 contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_macros.h delete mode 100644 contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_defs.h delete mode 100644 contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_protos.h delete mode 100644 contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_typedefs.h diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_defs.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h similarity index 100% rename from contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_defs.h rename to contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_macros.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_macros.h similarity index 100% rename from contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_macros.h rename to contrib/jemalloc-cmake/include/jemalloc/jemalloc_macros.h diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_protos.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h similarity index 100% rename from contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_protos.h rename to contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_typedefs.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_typedefs.h similarity index 88% rename from contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_typedefs.h rename to contrib/jemalloc-cmake/include/jemalloc/jemalloc_typedefs.h index 1a58874306e..5f94f16f937 100644 --- a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_typedefs.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_typedefs.h @@ -65,13 +65,13 @@ typedef bool (extent_merge_t)(extent_hooks_t *, void *, size_t, void *, size_t, bool, unsigned); struct extent_hooks_s { - extent_alloc_t *alloc; - extent_dalloc_t *dalloc; - extent_destroy_t *destroy; - extent_commit_t *commit; - extent_decommit_t *decommit; - extent_purge_t *purge_lazy; - extent_purge_t *purge_forced; - extent_split_t *split; - extent_merge_t *merge; + extent_alloc_t *alloc; + extent_dalloc_t *dalloc; + extent_destroy_t *destroy; + extent_commit_t *commit; + extent_decommit_t *decommit; + extent_purge_t *purge_lazy; + extent_purge_t *purge_forced; + extent_split_t *split; + extent_merge_t *merge; }; diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_macros.h b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_macros.h deleted file mode 100644 index 79b13337fbb..00000000000 --- a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_macros.h +++ /dev/null @@ -1,123 +0,0 @@ -#include -#include -#include -#include -#include - -#define JEMALLOC_VERSION "5.1.0-97-gcd2931ad9bbd78208565716ab102e86d858c2fff" -#define JEMALLOC_VERSION_MAJOR 5 -#define JEMALLOC_VERSION_MINOR 1 -#define JEMALLOC_VERSION_BUGFIX 0 -#define JEMALLOC_VERSION_NREV 97 -#define JEMALLOC_VERSION_GID "cd2931ad9bbd78208565716ab102e86d858c2fff" -#define JEMALLOC_VERSION_GID_IDENT cd2931ad9bbd78208565716ab102e86d858c2fff - -#define MALLOCX_LG_ALIGN(la) ((int)(la)) -#if LG_SIZEOF_PTR == 2 -# define MALLOCX_ALIGN(a) ((int)(ffs((int)(a))-1)) -#else -# define MALLOCX_ALIGN(a) \ - ((int)(((size_t)(a) < (size_t)INT_MAX) ? ffs((int)(a))-1 : \ - ffs((int)(((size_t)(a))>>32))+31)) -#endif -#define MALLOCX_ZERO ((int)0x40) -/* - * Bias tcache index bits so that 0 encodes "automatic tcache management", and 1 - * encodes MALLOCX_TCACHE_NONE. - */ -#define MALLOCX_TCACHE(tc) ((int)(((tc)+2) << 8)) -#define MALLOCX_TCACHE_NONE MALLOCX_TCACHE(-1) -/* - * Bias arena index bits so that 0 encodes "use an automatically chosen arena". - */ -#define MALLOCX_ARENA(a) ((((int)(a))+1) << 20) - -/* - * Use as arena index in "arena..{purge,decay,dss}" and - * "stats.arenas..*" mallctl interfaces to select all arenas. This - * definition is intentionally specified in raw decimal format to support - * cpp-based string concatenation, e.g. - * - * #define STRINGIFY_HELPER(x) #x - * #define STRINGIFY(x) STRINGIFY_HELPER(x) - * - * mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", NULL, NULL, NULL, - * 0); - */ -#define MALLCTL_ARENAS_ALL 4096 -/* - * Use as arena index in "stats.arenas..*" mallctl interfaces to select - * destroyed arenas. - */ -#define MALLCTL_ARENAS_DESTROYED 4097 - -#if defined(__cplusplus) && defined(JEMALLOC_USE_CXX_THROW) -# define JEMALLOC_CXX_THROW throw() -#else -# define JEMALLOC_CXX_THROW -#endif - -#if defined(_MSC_VER) -# define JEMALLOC_ATTR(s) -# define JEMALLOC_ALIGNED(s) __declspec(align(s)) -# define JEMALLOC_ALLOC_SIZE(s) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) -# ifndef JEMALLOC_EXPORT -# ifdef DLLEXPORT -# define JEMALLOC_EXPORT __declspec(dllexport) -# else -# define JEMALLOC_EXPORT __declspec(dllimport) -# endif -# endif -# define JEMALLOC_FORMAT_PRINTF(s, i) -# define JEMALLOC_NOINLINE __declspec(noinline) -# ifdef __cplusplus -# define JEMALLOC_NOTHROW __declspec(nothrow) -# else -# define JEMALLOC_NOTHROW -# endif -# define JEMALLOC_SECTION(s) __declspec(allocate(s)) -# define JEMALLOC_RESTRICT_RETURN __declspec(restrict) -# if _MSC_VER >= 1900 && !defined(__EDG__) -# define JEMALLOC_ALLOCATOR __declspec(allocator) -# else -# define JEMALLOC_ALLOCATOR -# endif -#elif defined(JEMALLOC_HAVE_ATTR) -# define JEMALLOC_ATTR(s) __attribute__((s)) -# define JEMALLOC_ALIGNED(s) JEMALLOC_ATTR(aligned(s)) -# ifdef JEMALLOC_HAVE_ATTR_ALLOC_SIZE -# define JEMALLOC_ALLOC_SIZE(s) JEMALLOC_ATTR(alloc_size(s)) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) JEMALLOC_ATTR(alloc_size(s1, s2)) -# else -# define JEMALLOC_ALLOC_SIZE(s) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) -# endif -# ifndef JEMALLOC_EXPORT -# define JEMALLOC_EXPORT JEMALLOC_ATTR(visibility("default")) -# endif -# ifdef JEMALLOC_HAVE_ATTR_FORMAT_GNU_PRINTF -# define JEMALLOC_FORMAT_PRINTF(s, i) JEMALLOC_ATTR(format(gnu_printf, s, i)) -# elif defined(JEMALLOC_HAVE_ATTR_FORMAT_PRINTF) -# define JEMALLOC_FORMAT_PRINTF(s, i) JEMALLOC_ATTR(format(printf, s, i)) -# else -# define JEMALLOC_FORMAT_PRINTF(s, i) -# endif -# define JEMALLOC_NOINLINE JEMALLOC_ATTR(noinline) -# define JEMALLOC_NOTHROW JEMALLOC_ATTR(nothrow) -# define JEMALLOC_SECTION(s) JEMALLOC_ATTR(section(s)) -# define JEMALLOC_RESTRICT_RETURN -# define JEMALLOC_ALLOCATOR -#else -# define JEMALLOC_ATTR(s) -# define JEMALLOC_ALIGNED(s) -# define JEMALLOC_ALLOC_SIZE(s) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) -# define JEMALLOC_EXPORT -# define JEMALLOC_FORMAT_PRINTF(s, i) -# define JEMALLOC_NOINLINE -# define JEMALLOC_NOTHROW -# define JEMALLOC_SECTION(s) -# define JEMALLOC_RESTRICT_RETURN -# define JEMALLOC_ALLOCATOR -#endif diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_defs.h b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_defs.h deleted file mode 100644 index d1389237a77..00000000000 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_defs.h +++ /dev/null @@ -1,43 +0,0 @@ -/* include/jemalloc/jemalloc_defs.h. Generated from jemalloc_defs.h.in by configure. */ -/* Defined if __attribute__((...)) syntax is supported. */ -#define JEMALLOC_HAVE_ATTR - -/* Defined if alloc_size attribute is supported. */ -#define JEMALLOC_HAVE_ATTR_ALLOC_SIZE - -/* Defined if format(printf, ...) attribute is supported. */ -#define JEMALLOC_HAVE_ATTR_FORMAT_PRINTF - -/* - * Define overrides for non-standard allocator-related functions if they are - * present on the system. - */ -#define JEMALLOC_OVERRIDE_MEMALIGN -#define JEMALLOC_OVERRIDE_VALLOC - -/* - * At least Linux omits the "const" in: - * - * size_t malloc_usable_size(const void *ptr); - * - * Match the operating system's prototype. - */ -#define JEMALLOC_USABLE_SIZE_CONST - -/* - * If defined, specify throw() for the public function prototypes when compiling - * with C++. The only justification for this is to match the prototypes that - * glibc defines. - */ -#define JEMALLOC_USE_CXX_THROW - -#ifdef _MSC_VER -# ifdef _WIN64 -# define LG_SIZEOF_PTR_WIN 3 -# else -# define LG_SIZEOF_PTR_WIN 2 -# endif -#endif - -/* sizeof(void *) == 2^LG_SIZEOF_PTR. */ -#define LG_SIZEOF_PTR 3 diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_protos.h b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_protos.h deleted file mode 100644 index ff025e30fa7..00000000000 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_protos.h +++ /dev/null @@ -1,66 +0,0 @@ -/* - * The je_ prefix on the following public symbol declarations is an artifact - * of namespace management, and should be omitted in application code unless - * JEMALLOC_NO_DEMANGLE is defined (see jemalloc_mangle.h). - */ -extern JEMALLOC_EXPORT const char *je_malloc_conf; -extern JEMALLOC_EXPORT void (*je_malloc_message)(void *cbopaque, - const char *s); - -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_malloc(size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE(1); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_calloc(size_t num, size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE2(1, 2); -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_posix_memalign(void **memptr, - size_t alignment, size_t size) JEMALLOC_CXX_THROW JEMALLOC_ATTR(nonnull(1)); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_aligned_alloc(size_t alignment, - size_t size) JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) - JEMALLOC_ALLOC_SIZE(2); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_realloc(void *ptr, size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ALLOC_SIZE(2); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_free(void *ptr) - JEMALLOC_CXX_THROW; - -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_mallocx(size_t size, int flags) - JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE(1); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_rallocx(void *ptr, size_t size, - int flags) JEMALLOC_ALLOC_SIZE(2); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_xallocx(void *ptr, size_t size, - size_t extra, int flags); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_sallocx(const void *ptr, - int flags) JEMALLOC_ATTR(pure); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_dallocx(void *ptr, int flags); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_sdallocx(void *ptr, size_t size, - int flags); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_nallocx(size_t size, int flags) - JEMALLOC_ATTR(pure); - -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctl(const char *name, - void *oldp, size_t *oldlenp, void *newp, size_t newlen); -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctlnametomib(const char *name, - size_t *mibp, size_t *miblenp); -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctlbymib(const size_t *mib, - size_t miblen, void *oldp, size_t *oldlenp, void *newp, size_t newlen); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_malloc_stats_print( - void (*write_cb)(void *, const char *), void *je_cbopaque, - const char *opts); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_malloc_usable_size( - JEMALLOC_USABLE_SIZE_CONST void *ptr) JEMALLOC_CXX_THROW; - -#ifdef JEMALLOC_OVERRIDE_MEMALIGN -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_memalign(size_t alignment, size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc); -#endif - -#ifdef JEMALLOC_OVERRIDE_VALLOC -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_valloc(size_t size) JEMALLOC_CXX_THROW - JEMALLOC_ATTR(malloc); -#endif diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_typedefs.h b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_typedefs.h deleted file mode 100644 index 1a58874306e..00000000000 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_typedefs.h +++ /dev/null @@ -1,77 +0,0 @@ -typedef struct extent_hooks_s extent_hooks_t; - -/* - * void * - * extent_alloc(extent_hooks_t *extent_hooks, void *new_addr, size_t size, - * size_t alignment, bool *zero, bool *commit, unsigned arena_ind); - */ -typedef void *(extent_alloc_t)(extent_hooks_t *, void *, size_t, size_t, bool *, - bool *, unsigned); - -/* - * bool - * extent_dalloc(extent_hooks_t *extent_hooks, void *addr, size_t size, - * bool committed, unsigned arena_ind); - */ -typedef bool (extent_dalloc_t)(extent_hooks_t *, void *, size_t, bool, - unsigned); - -/* - * void - * extent_destroy(extent_hooks_t *extent_hooks, void *addr, size_t size, - * bool committed, unsigned arena_ind); - */ -typedef void (extent_destroy_t)(extent_hooks_t *, void *, size_t, bool, - unsigned); - -/* - * bool - * extent_commit(extent_hooks_t *extent_hooks, void *addr, size_t size, - * size_t offset, size_t length, unsigned arena_ind); - */ -typedef bool (extent_commit_t)(extent_hooks_t *, void *, size_t, size_t, size_t, - unsigned); - -/* - * bool - * extent_decommit(extent_hooks_t *extent_hooks, void *addr, size_t size, - * size_t offset, size_t length, unsigned arena_ind); - */ -typedef bool (extent_decommit_t)(extent_hooks_t *, void *, size_t, size_t, - size_t, unsigned); - -/* - * bool - * extent_purge(extent_hooks_t *extent_hooks, void *addr, size_t size, - * size_t offset, size_t length, unsigned arena_ind); - */ -typedef bool (extent_purge_t)(extent_hooks_t *, void *, size_t, size_t, size_t, - unsigned); - -/* - * bool - * extent_split(extent_hooks_t *extent_hooks, void *addr, size_t size, - * size_t size_a, size_t size_b, bool committed, unsigned arena_ind); - */ -typedef bool (extent_split_t)(extent_hooks_t *, void *, size_t, size_t, size_t, - bool, unsigned); - -/* - * bool - * extent_merge(extent_hooks_t *extent_hooks, void *addr_a, size_t size_a, - * void *addr_b, size_t size_b, bool committed, unsigned arena_ind); - */ -typedef bool (extent_merge_t)(extent_hooks_t *, void *, size_t, void *, size_t, - bool, unsigned); - -struct extent_hooks_s { - extent_alloc_t *alloc; - extent_dalloc_t *dalloc; - extent_destroy_t *destroy; - extent_commit_t *commit; - extent_decommit_t *decommit; - extent_purge_t *purge_lazy; - extent_purge_t *purge_forced; - extent_split_t *split; - extent_merge_t *merge; -}; From b53c6ddfd9a9960fc94486b85db1d1d07f86fbc7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 May 2020 02:37:11 +0300 Subject: [PATCH 18/79] jemalloc: bump to 5.2.1 --- contrib/jemalloc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/jemalloc b/contrib/jemalloc index cd2931ad9bb..ea6b3e973b4 160000 --- a/contrib/jemalloc +++ b/contrib/jemalloc @@ -1 +1 @@ -Subproject commit cd2931ad9bbd78208565716ab102e86d858c2fff +Subproject commit ea6b3e973b477b8061e0076bb257dbd7f3faa756 From bbb38664c10cbed66c82922595102ff28d8b3f71 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 May 2020 12:55:21 +0300 Subject: [PATCH 19/79] jemalloc: add missing safety_check.c --- contrib/jemalloc-cmake/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 9081de593b8..e187ca2926d 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -55,6 +55,7 @@ if (ENABLE_JEMALLOC) ${LIBRARY_DIR}/src/ticker.c ${LIBRARY_DIR}/src/tsd.c ${LIBRARY_DIR}/src/witness.c + ${LIBRARY_DIR}/src/safety_check.c ) if (OS_DARWIN) list(APPEND SRCS ${LIBRARY_DIR}/src/zone.c) From b2ff1b3f20d15722e5097fd8c7622572eb809d01 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 May 2020 13:05:20 +0300 Subject: [PATCH 20/79] jemalloc: merge jemalloc_preamble.h for x86_64 --- .../jemalloc/internal/jemalloc_preamble.h | 23 +++++++++++++++++-- 1 file changed, 21 insertions(+), 2 deletions(-) diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_preamble.h b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_preamble.h index c150785fb4a..e5e34925b55 100644 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_preamble.h +++ b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_preamble.h @@ -21,7 +21,7 @@ # include "jemalloc/jemalloc.h" #endif -#if (defined(JEMALLOC_OSATOMIC) || defined(JEMALLOC_OSSPIN)) +#if defined(JEMALLOC_OSATOMIC) #include #endif @@ -161,7 +161,26 @@ static const bool config_log = false #endif ; -#ifdef JEMALLOC_HAVE_SCHED_GETCPU +/* + * Are extra safety checks enabled; things like checking the size of sized + * deallocations, double-frees, etc. + */ +static const bool config_opt_safety_checks = +#ifdef JEMALLOC_OPT_SAFETY_CHECKS + true +#elif defined(JEMALLOC_DEBUG) + /* + * This lets us only guard safety checks by one flag instead of two; fast + * checks can guard solely by config_opt_safety_checks and run in debug mode + * too. + */ + true +#else + false +#endif + ; + +#if defined(_WIN32) || defined(JEMALLOC_HAVE_SCHED_GETCPU) /* Currently percpu_arena depends on sched_getcpu. */ #define JEMALLOC_PERCPU_ARENA #endif From 2590ee64d902cf18be3ba5ad715803bfe94370b3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 May 2020 12:55:55 +0300 Subject: [PATCH 21/79] jemalloc: merge jemalloc_preamble.h for aarch64 --- .../jemalloc/internal/jemalloc_preamble.h | 23 +++++++++++++++++-- 1 file changed, 21 insertions(+), 2 deletions(-) diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_preamble.h b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_preamble.h index c150785fb4a..e5e34925b55 100644 --- a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_preamble.h +++ b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_preamble.h @@ -21,7 +21,7 @@ # include "jemalloc/jemalloc.h" #endif -#if (defined(JEMALLOC_OSATOMIC) || defined(JEMALLOC_OSSPIN)) +#if defined(JEMALLOC_OSATOMIC) #include #endif @@ -161,7 +161,26 @@ static const bool config_log = false #endif ; -#ifdef JEMALLOC_HAVE_SCHED_GETCPU +/* + * Are extra safety checks enabled; things like checking the size of sized + * deallocations, double-frees, etc. + */ +static const bool config_opt_safety_checks = +#ifdef JEMALLOC_OPT_SAFETY_CHECKS + true +#elif defined(JEMALLOC_DEBUG) + /* + * This lets us only guard safety checks by one flag instead of two; fast + * checks can guard solely by config_opt_safety_checks and run in debug mode + * too. + */ + true +#else + false +#endif + ; + +#if defined(_WIN32) || defined(JEMALLOC_HAVE_SCHED_GETCPU) /* Currently percpu_arena depends on sched_getcpu. */ #define JEMALLOC_PERCPU_ARENA #endif From 2f256b0d2de001d064efa5c755b5a10da4ddb013 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 May 2020 13:00:41 +0300 Subject: [PATCH 22/79] jemalloc: merge jemalloc_internal_defs.h.in for x86_64 - Use upstream JEMALLOC_PURGE_MADVISE_FREE --- .../internal/jemalloc_internal_defs.h.in | 56 +++++++++---------- 1 file changed, 25 insertions(+), 31 deletions(-) diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in index 63f7f765023..71a93253441 100644 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -1,11 +1,6 @@ /* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ #ifndef JEMALLOC_INTERNAL_DEFS_H_ #define JEMALLOC_INTERNAL_DEFS_H_ - -#ifndef _GNU_SOURCE - #define _GNU_SOURCE -#endif - /* * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all * public APIs to be prefixed. This makes it possible, with some care, to use @@ -24,7 +19,7 @@ #define JEMALLOC_OVERRIDE___LIBC_MEMALIGN #define JEMALLOC_OVERRIDE___LIBC_REALLOC #define JEMALLOC_OVERRIDE___LIBC_VALLOC -#define JEMALLOC_OVERRIDE___POSIX_MEMALIGN +/* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ /* * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. @@ -54,25 +49,13 @@ /* Defined if GCC __atomic atomics are available. */ #define JEMALLOC_GCC_ATOMIC_ATOMICS 1 +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS 1 /* Defined if GCC __sync atomics are available. */ #define JEMALLOC_GCC_SYNC_ATOMICS 1 - -/* - * Defined if __sync_add_and_fetch(uint32_t *, uint32_t) and - * __sync_sub_and_fetch(uint32_t *, uint32_t) are available, despite - * __GCC_HAVE_SYNC_COMPARE_AND_SWAP_4 not being defined (which means the - * functions are defined in libgcc instead of being inlines). - */ -/* #undef JE_FORCE_SYNC_COMPARE_AND_SWAP_4 */ - -/* - * Defined if __sync_add_and_fetch(uint64_t *, uint64_t) and - * __sync_sub_and_fetch(uint64_t *, uint64_t) are available, despite - * __GCC_HAVE_SYNC_COMPARE_AND_SWAP_8 not being defined (which means the - * functions are defined in libgcc instead of being inlines). - */ -/* #undef JE_FORCE_SYNC_COMPARE_AND_SWAP_8 */ +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS 1 /* * Defined if __builtin_clz() and __builtin_clzl() are available. @@ -84,20 +67,13 @@ */ /* #undef JEMALLOC_OS_UNFAIR_LOCK */ -/* - * Defined if OSSpin*() functions are available, as provided by Darwin, and - * documented in the spinlock(3) manual page. - */ -/* #undef JEMALLOC_OSSPIN */ - /* Defined if syscall(2) is usable. */ #define JEMALLOC_USE_SYSCALL /* * Defined if secure_getenv(3) is available. */ -// Don't want dependency on newer GLIBC -//#define JEMALLOC_HAVE_SECURE_GETENV +#define JEMALLOC_HAVE_SECURE_GETENV /* * Defined if issetugid(2) is available. @@ -160,6 +136,9 @@ /* JEMALLOC_STATS enables statistics calculation. */ #define JEMALLOC_STATS +/* JEMALLOC_EXPERIMENTAL_SMALLOCX_API enables experimental smallocx API. */ +/* #undef JEMALLOC_EXPERIMENTAL_SMALLOCX_API */ + /* JEMALLOC_PROF enables allocation profiling. */ /* #undef JEMALLOC_PROF */ @@ -240,6 +219,12 @@ #define JEMALLOC_INTERNAL_FFSL __builtin_ffsl #define JEMALLOC_INTERNAL_FFS __builtin_ffs +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + /* * If defined, explicitly attempt to more uniformly distribute large allocation * pointer alignments across all cache indices. @@ -252,6 +237,12 @@ */ /* #undef JEMALLOC_LOG */ +/* + * If defined, use readlinkat() (instead of readlink()) to follow + * /etc/malloc_conf. + */ +/* #undef JEMALLOC_READLINKAT */ + /* * Darwin (OS X) uses zones to work around Mach-O symbol override shortcomings. */ @@ -288,7 +279,7 @@ * MADV_FREE, though typically with higher * system overhead. */ -//#define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_FREE #define JEMALLOC_PURGE_MADVISE_DONTNEED #define JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS @@ -370,4 +361,7 @@ */ #define JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + #endif /* JEMALLOC_INTERNAL_DEFS_H_ */ From 82f2b1a986965fc850188460f4873566101692a0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 May 2020 13:02:23 +0300 Subject: [PATCH 23/79] jemalloc: merge jemalloc_internal_defs.h.in for aarch64 - keep LG_PAGE/LG_HUGEPAGE - use upstream JEMALLOC_PURGE_MADVISE_FREE --- .../internal/jemalloc_internal_defs.h.in | 49 +++++++------------ 1 file changed, 17 insertions(+), 32 deletions(-) diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in index 089f1a773aa..231d80ceb5c 100644 --- a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -1,12 +1,6 @@ /* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ #ifndef JEMALLOC_INTERNAL_DEFS_H_ #define JEMALLOC_INTERNAL_DEFS_H_ - - -#ifndef _GNU_SOURCE - #define _GNU_SOURCE -#endif - /* * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all * public APIs to be prefixed. This makes it possible, with some care, to use @@ -25,7 +19,7 @@ #define JEMALLOC_OVERRIDE___LIBC_MEMALIGN #define JEMALLOC_OVERRIDE___LIBC_REALLOC #define JEMALLOC_OVERRIDE___LIBC_VALLOC -#define JEMALLOC_OVERRIDE___POSIX_MEMALIGN +/* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ /* * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. @@ -39,9 +33,9 @@ * Hyper-threaded CPUs may need a special instruction inside spin loops in * order to yield to another virtual CPU. */ -#define CPU_SPINWAIT +#define CPU_SPINWAIT __asm__ volatile("pause") /* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ -#define HAVE_CPU_SPINWAIT 0 +#define HAVE_CPU_SPINWAIT 1 /* * Number of significant bits in virtual addresses. This may be less than the @@ -55,25 +49,13 @@ /* Defined if GCC __atomic atomics are available. */ #define JEMALLOC_GCC_ATOMIC_ATOMICS 1 +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS 1 /* Defined if GCC __sync atomics are available. */ #define JEMALLOC_GCC_SYNC_ATOMICS 1 - -/* - * Defined if __sync_add_and_fetch(uint32_t *, uint32_t) and - * __sync_sub_and_fetch(uint32_t *, uint32_t) are available, despite - * __GCC_HAVE_SYNC_COMPARE_AND_SWAP_4 not being defined (which means the - * functions are defined in libgcc instead of being inlines). - */ -/* #undef JE_FORCE_SYNC_COMPARE_AND_SWAP_4 */ - -/* - * Defined if __sync_add_and_fetch(uint64_t *, uint64_t) and - * __sync_sub_and_fetch(uint64_t *, uint64_t) are available, despite - * __GCC_HAVE_SYNC_COMPARE_AND_SWAP_8 not being defined (which means the - * functions are defined in libgcc instead of being inlines). - */ -/* #undef JE_FORCE_SYNC_COMPARE_AND_SWAP_8 */ +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS 1 /* * Defined if __builtin_clz() and __builtin_clzl() are available. @@ -85,12 +67,6 @@ */ /* #undef JEMALLOC_OS_UNFAIR_LOCK */ -/* - * Defined if OSSpin*() functions are available, as provided by Darwin, and - * documented in the spinlock(3) manual page. - */ -/* #undef JEMALLOC_OSSPIN */ - /* Defined if syscall(2) is usable. */ #define JEMALLOC_USE_SYSCALL @@ -243,6 +219,12 @@ #define JEMALLOC_INTERNAL_FFSL __builtin_ffsl #define JEMALLOC_INTERNAL_FFS __builtin_ffs +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + /* * If defined, explicitly attempt to more uniformly distribute large allocation * pointer alignments across all cache indices. @@ -297,7 +279,7 @@ * MADV_FREE, though typically with higher * system overhead. */ -// #define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_FREE #define JEMALLOC_PURGE_MADVISE_DONTNEED #define JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS @@ -379,4 +361,7 @@ */ #define JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + #endif /* JEMALLOC_INTERNAL_DEFS_H_ */ From 9f1cefd8dfea9e28b1d1b2a8ae7eb64dedb8ceaf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 May 2020 13:09:10 +0300 Subject: [PATCH 24/79] jemalloc: disable secure_getenv() --- .../jemalloc/internal/jemalloc_internal_defs.h.in | 2 +- .../jemalloc/internal/jemalloc_internal_defs.h.in | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in index 231d80ceb5c..3852f8b6382 100644 --- a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -73,7 +73,7 @@ /* * Defined if secure_getenv(3) is available. */ -#define JEMALLOC_HAVE_SECURE_GETENV +// #define JEMALLOC_HAVE_SECURE_GETENV /* * Defined if issetugid(2) is available. diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in index 71a93253441..d5cf0e719ef 100644 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -73,7 +73,7 @@ /* * Defined if secure_getenv(3) is available. */ -#define JEMALLOC_HAVE_SECURE_GETENV +// #define JEMALLOC_HAVE_SECURE_GETENV /* * Defined if issetugid(2) is available. From 5cd31dbf0deda64dadcf6a3f6b8f5ad33eb9726a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 May 2020 13:12:00 +0300 Subject: [PATCH 25/79] jemalloc: add _GNU_SOURCE for RTLD_NEXT --- contrib/jemalloc-cmake/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index e187ca2926d..e5f52f4dc8e 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -90,6 +90,8 @@ if (ENABLE_JEMALLOC) endif () target_compile_options(jemalloc PRIVATE -Wno-redundant-decls) + # for RTLD_NEXT + target_compile_options(jemalloc PRIVATE -D_GNU_SOURCE) else () find_library(LIBRARY_JEMALLOC jemalloc) find_path(INCLUDE_JEMALLOC jemalloc/jemalloc.h) From bc572dc5f22b139a60a9a0dc188f2e38ed0ba9f9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 May 2020 13:20:07 +0300 Subject: [PATCH 26/79] jemalloc: sync common (non per-arch) pre-generated headers $ for f in include/jemalloc/jemalloc_*.h; do cp /src/oss/jemalloc/$f $f; done --- .../include/jemalloc/jemalloc_defs.h | 6 ++++++ .../include/jemalloc/jemalloc_macros.h | 17 ++++++++++++----- .../include/jemalloc/jemalloc_rename.h | 1 + 3 files changed, 19 insertions(+), 5 deletions(-) diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h index d1389237a77..6a03a231a0e 100644 --- a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h @@ -5,6 +5,12 @@ /* Defined if alloc_size attribute is supported. */ #define JEMALLOC_HAVE_ATTR_ALLOC_SIZE +/* Defined if format_arg(...) attribute is supported. */ +#define JEMALLOC_HAVE_ATTR_FORMAT_ARG + +/* Defined if format(gnu_printf, ...) attribute is supported. */ +#define JEMALLOC_HAVE_ATTR_FORMAT_GNU_PRINTF + /* Defined if format(printf, ...) attribute is supported. */ #define JEMALLOC_HAVE_ATTR_FORMAT_PRINTF diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_macros.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_macros.h index 7432f1cda53..34235894285 100644 --- a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_macros.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_macros.h @@ -4,12 +4,13 @@ #include #include -#define JEMALLOC_VERSION "5.1.0-56-g41b7372eadee941b9164751b8d4963f915d3ceae" +#define JEMALLOC_VERSION "5.2.1-0-gea6b3e973b477b8061e0076bb257dbd7f3faa756" #define JEMALLOC_VERSION_MAJOR 5 -#define JEMALLOC_VERSION_MINOR 1 -#define JEMALLOC_VERSION_BUGFIX 0 -#define JEMALLOC_VERSION_NREV 56 -#define JEMALLOC_VERSION_GID "41b7372eadee941b9164751b8d4963f915d3ceae" +#define JEMALLOC_VERSION_MINOR 2 +#define JEMALLOC_VERSION_BUGFIX 1 +#define JEMALLOC_VERSION_NREV 0 +#define JEMALLOC_VERSION_GID "ea6b3e973b477b8061e0076bb257dbd7f3faa756" +#define JEMALLOC_VERSION_GID_IDENT ea6b3e973b477b8061e0076bb257dbd7f3faa756 #define MALLOCX_LG_ALIGN(la) ((int)(la)) #if LG_SIZEOF_PTR == 2 @@ -68,6 +69,7 @@ # define JEMALLOC_EXPORT __declspec(dllimport) # endif # endif +# define JEMALLOC_FORMAT_ARG(i) # define JEMALLOC_FORMAT_PRINTF(s, i) # define JEMALLOC_NOINLINE __declspec(noinline) # ifdef __cplusplus @@ -95,6 +97,11 @@ # ifndef JEMALLOC_EXPORT # define JEMALLOC_EXPORT JEMALLOC_ATTR(visibility("default")) # endif +# ifdef JEMALLOC_HAVE_ATTR_FORMAT_ARG +# define JEMALLOC_FORMAT_ARG(i) JEMALLOC_ATTR(__format_arg__(3)) +# else +# define JEMALLOC_FORMAT_ARG(i) +# endif # ifdef JEMALLOC_HAVE_ATTR_FORMAT_GNU_PRINTF # define JEMALLOC_FORMAT_PRINTF(s, i) JEMALLOC_ATTR(format(gnu_printf, s, i)) # elif defined(JEMALLOC_HAVE_ATTR_FORMAT_PRINTF) diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_rename.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_rename.h index a2ea2dd3533..2e94f7a0cc3 100644 --- a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_rename.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_rename.h @@ -17,6 +17,7 @@ # define je_malloc_stats_print malloc_stats_print # define je_malloc_usable_size malloc_usable_size # define je_mallocx mallocx +# define je_smallocx_ea6b3e973b477b8061e0076bb257dbd7f3faa756 smallocx_ea6b3e973b477b8061e0076bb257dbd7f3faa756 # define je_nallocx nallocx # define je_posix_memalign posix_memalign # define je_rallocx rallocx From a8749c5d5c668a079239112f96a1fafe072a9ad5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 May 2020 19:17:50 +0300 Subject: [PATCH 27/79] jemalloc: aarch64 does not have pause --- .../jemalloc/internal/jemalloc_internal_defs.h.in | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in index 3852f8b6382..cbd2740e1f1 100644 --- a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -33,9 +33,9 @@ * Hyper-threaded CPUs may need a special instruction inside spin loops in * order to yield to another virtual CPU. */ -#define CPU_SPINWAIT __asm__ volatile("pause") +#define CPU_SPINWAIT /* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ -#define HAVE_CPU_SPINWAIT 1 +#define HAVE_CPU_SPINWAIT 9 /* * Number of significant bits in virtual addresses. This may be less than the From b694d6ed8c78eacb248a0cd20255e81433f88d27 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 26 May 2020 21:45:51 +0300 Subject: [PATCH 28/79] jemalloc: disable oversize_threshold (slower) --- contrib/jemalloc-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index e5f52f4dc8e..a02a63ba999 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -17,7 +17,7 @@ if (ENABLE_JEMALLOC) # # By enabling percpu_arena number of arenas limited to number of CPUs and hence # this problem should go away. - set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu" CACHE STRING "Change default configuration string of JEMalloc" ) + set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0" CACHE STRING "Change default configuration string of JEMalloc" ) message (STATUS "jemalloc malloc_conf: ${JEMALLOC_CONFIG_MALLOC_CONF}") set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/jemalloc") From 210e67d8317252252402aeba6ef6d05051f7fc7e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 26 May 2020 21:51:06 +0300 Subject: [PATCH 29/79] jemalloc: allow changing default malloc_conf without purging cmake cache --- contrib/jemalloc-cmake/CMakeLists.txt | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index a02a63ba999..63c5a5e66ea 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -17,7 +17,13 @@ if (ENABLE_JEMALLOC) # # By enabling percpu_arena number of arenas limited to number of CPUs and hence # this problem should go away. - set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0" CACHE STRING "Change default configuration string of JEMalloc" ) + set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0") + # CACHE variable is empty, to allow changing defaults without necessity + # to purge cache + set (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE "" CACHE STRING "Change default configuration string of JEMalloc" ) + if (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE) + set (JEMALLOC_CONFIG_MALLOC_CONF "${JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE}") + endif() message (STATUS "jemalloc malloc_conf: ${JEMALLOC_CONFIG_MALLOC_CONF}") set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/jemalloc") From ec894e7cb3e0e3f9f7c880b00d44ee3b83e028ad Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 May 2020 19:39:31 +0300 Subject: [PATCH 30/79] Check for POPCNT instruction --- programs/main.cpp | 19 ++++++++++++++++--- ...heck_cpu_instructions_at_startup.reference | 1 + ...01103_check_cpu_instructions_at_startup.sh | 13 +++++++++---- 3 files changed, 26 insertions(+), 7 deletions(-) diff --git a/programs/main.cpp b/programs/main.cpp index 2eb226d3c00..b1bff1de1b0 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -127,9 +127,10 @@ enum class InstructionFail SSSE3 = 2, SSE4_1 = 3, SSE4_2 = 4, - AVX = 5, - AVX2 = 6, - AVX512 = 7 + POPCNT = 5, + AVX = 6, + AVX2 = 7, + AVX512 = 8 }; const char * instructionFailToString(InstructionFail fail) @@ -146,6 +147,8 @@ const char * instructionFailToString(InstructionFail fail) return "SSE4.1"; case InstructionFail::SSE4_2: return "SSE4.2"; + case InstructionFail::POPCNT: + return "POPCNT"; case InstructionFail::AVX: return "AVX"; case InstructionFail::AVX2: @@ -189,6 +192,16 @@ void checkRequiredInstructionsImpl(volatile InstructionFail & fail) __asm__ volatile ("pcmpgtq %%xmm0, %%xmm0" : : : "xmm0"); #endif + /// Defined by -msse4.2 +#if defined(__POPCNT__) + fail = InstructionFail::POPCNT; + { + uint64_t a = 0; + uint64_t b = 0; + __asm__ volatile ("popcnt %1, %0" : "=r"(a) :"r"(b) : ); + } +#endif + #if defined(__AVX__) fail = InstructionFail::AVX; __asm__ volatile ("vaddpd %%ymm0, %%ymm0, %%ymm0" : : : "ymm0"); diff --git a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference index 59c88f013dd..03ed07cf1a4 100644 --- a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference +++ b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference @@ -1,4 +1,5 @@ Instruction check fail. The CPU does not support SSSE3 instruction set. Instruction check fail. The CPU does not support SSE4.1 instruction set. Instruction check fail. The CPU does not support SSE4.2 instruction set. +Instruction check fail. The CPU does not support POPCNT instruction set. 1 diff --git a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh index 15466e1889a..5ae4f8b3dd2 100755 --- a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh +++ b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh @@ -9,8 +9,13 @@ ${CLICKHOUSE_LOCAL} --query "SELECT max(value LIKE '%sanitize%') FROM system.bui command=$(command -v ${CLICKHOUSE_LOCAL}) -qemu-x86_64-static -cpu qemu64 $command --query "SELECT 1" 2>&1 | grep -v -F "warning: TCG doesn't support requested feature" ||: -qemu-x86_64-static -cpu qemu64,+ssse3 $command --query "SELECT 1" 2>&1 | grep -v -F "warning: TCG doesn't support requested feature" ||: -qemu-x86_64-static -cpu qemu64,+ssse3,+sse4.1 $command --query "SELECT 1" 2>&1 | grep -v -F "warning: TCG doesn't support requested feature" ||: -qemu-x86_64-static -cpu qemu64,+ssse3,+sse4.1,+sse4.2 $command --query "SELECT 1" 2>&1 | grep -v -F "warning: TCG doesn't support requested feature" ||: +function run_with_cpu() +{ + qemu-x86_64-static -cpu "$@" $command --query "SELECT 1" 2>&1 | grep -v -F "warning: TCG doesn't support requested feature" ||: +} +run_with_cpu qemu64 +run_with_cpu qemu64,+ssse3 +run_with_cpu qemu64,+ssse3,+sse4.1 +run_with_cpu qemu64,+ssse3,+sse4.1,+sse4.2 +run_with_cpu qemu64,+ssse3,+sse4.1,+sse4.2,+popcnt From efc0b977078af8a80a2afd6a2849efeb78148201 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 22 May 2020 18:58:08 +0200 Subject: [PATCH 31/79] Fix Kafka performance issue related to reschedules based on limits, which were always applied. --- src/Storages/Kafka/KafkaBlockInputStream.h | 1 + .../Kafka/ReadBufferFromKafkaConsumer.h | 1 + src/Storages/Kafka/StorageKafka.cpp | 31 ++++++++++---- tests/integration/test_storage_kafka/test.py | 40 ++++++++++++++++++- 4 files changed, 64 insertions(+), 9 deletions(-) diff --git a/src/Storages/Kafka/KafkaBlockInputStream.h b/src/Storages/Kafka/KafkaBlockInputStream.h index 1f94ee332d3..e3052122894 100644 --- a/src/Storages/Kafka/KafkaBlockInputStream.h +++ b/src/Storages/Kafka/KafkaBlockInputStream.h @@ -25,6 +25,7 @@ public: void readSuffixImpl() override; void commit(); + bool isStalled() const { return buffer->isStalled(); } private: StorageKafka & storage; diff --git a/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h b/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h index 12da701a55d..e90e3b48881 100644 --- a/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h +++ b/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h @@ -38,6 +38,7 @@ public: bool hasMorePolledMessages() const; bool polledDataUnusable() const { return (was_stopped || rebalance_happened); } + bool isStalled() const { return stalled; } void storeLastReadMessageOffset(); void resetToLastCommitted(const char * msg); diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index d1dfbe9d875..7731cf3c06a 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -51,6 +51,7 @@ namespace { const auto RESCHEDULE_MS = 500; const auto CLEANUP_TIMEOUT_MS = 3000; + const auto MAX_THREAD_WORK_DURATION_MS = 60000; // once per minute leave do reschedule (we can't lock threads in pool forever) /// Configuration prefix const String CONFIG_PREFIX = "kafka"; @@ -386,6 +387,8 @@ void StorageKafka::threadFunc() size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); if (dependencies_count) { + auto start_time = std::chrono::steady_clock::now(); + // Keep streaming as long as there are attached views and streaming is not cancelled while (!stream_cancelled && num_created_consumers > 0) { @@ -394,9 +397,21 @@ void StorageKafka::threadFunc() LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - // Reschedule if not limited - if (!streamToViews()) + // Exit the loop & reschedule if some stream stalled + auto some_stream_is_stalled = streamToViews(); + if (some_stream_is_stalled) + { + LOG_TRACE(log, "Stream(s) stalled. Reschedule."); break; + } + + auto ts = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(ts-start_time); + if (duration.count() > MAX_THREAD_WORK_DURATION_MS) + { + LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); + break; + } } } } @@ -459,15 +474,15 @@ bool StorageKafka::streamToViews() // It will be cancelled on underlying layer (kafka buffer) std::atomic stub = {false}; copyData(*in, *block_io.out, &stub); + + bool some_stream_is_stalled = false; for (auto & stream : streams) + { + some_stream_is_stalled = some_stream_is_stalled || stream->as()->isStalled(); stream->as()->commit(); + } - // Check whether the limits were applied during query execution - bool limits_applied = false; - const BlockStreamProfileInfo & info = in->getProfileInfo(); - limits_applied = info.hasAppliedLimit(); - - return limits_applied; + return some_stream_is_stalled; } void registerStorageKafka(StorageFactory & factory) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 8e329a1cb60..d71347e83f1 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1017,7 +1017,10 @@ def test_kafka_flush_by_block_size(kafka_cluster): time.sleep(1) - result = instance.query('SELECT count() FROM test.view') + # TODO: due to https://github.com/ClickHouse/ClickHouse/pull/11149 + # second flush happens earlier than expected, so we have 2 parts here instead of one + # flush by block size works correctly, so the feature checked by the test is working correctly + result = instance.query("SELECT count() FROM test.view WHERE _part='all_1_1_0'") # print(result) # kafka_cluster.open_bash_shell('instance') @@ -1390,6 +1393,41 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): assert TSV(result) == TSV('{0}\t{0}\t{0}'.format(i[0]-1)), 'Missing data!' + +@pytest.mark.timeout(120) +def test_bad_reschedule(kafka_cluster): + messages = [json.dumps({'key': j+1, 'value': j+1}) for j in range(20000)] + kafka_produce('test_bad_reschedule', messages) + + instance.query(''' + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'test_bad_reschedule', + kafka_group_name = 'test_bad_reschedule', + kafka_format = 'JSONEachRow', + kafka_max_block_size = 1000; + + CREATE MATERIALIZED VIEW test.destination Engine=Log AS + SELECT + key, + now() as consume_ts, + value, + _topic, + _key, + _offset, + _partition, + _timestamp + FROM test.kafka; + ''') + + while int(instance.query("SELECT count() FROM test.destination")) < 20000: + print("Waiting for consume") + time.sleep(1) + + assert int(instance.query("SELECT max(consume_ts) - min(consume_ts) FROM test.destination")) < 8 + + @pytest.mark.timeout(1200) def test_kafka_duplicates_when_commit_failed(kafka_cluster): messages = [json.dumps({'key': j+1, 'value': 'x' * 300}) for j in range(22)] From c73b8372e786ba6de958e8be222bf20081c97afd Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Wed, 27 May 2020 07:44:13 +0200 Subject: [PATCH 32/79] Update test.py --- tests/integration/test_storage_kafka/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index d71347e83f1..13577864870 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1017,7 +1017,7 @@ def test_kafka_flush_by_block_size(kafka_cluster): time.sleep(1) - # TODO: due to https://github.com/ClickHouse/ClickHouse/pull/11149 + # TODO: due to https://github.com/ClickHouse/ClickHouse/issues/11216 # second flush happens earlier than expected, so we have 2 parts here instead of one # flush by block size works correctly, so the feature checked by the test is working correctly result = instance.query("SELECT count() FROM test.view WHERE _part='all_1_1_0'") From d941c483b2923c378536631fc479c06f557a7e44 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 27 May 2020 11:25:36 +0300 Subject: [PATCH 33/79] Update test.py --- tests/integration/test_system_queries/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_system_queries/test.py b/tests/integration/test_system_queries/test.py index 9397cd20ec7..6f36a13b184 100644 --- a/tests/integration/test_system_queries/test.py +++ b/tests/integration/test_system_queries/test.py @@ -97,6 +97,7 @@ def test_SYSTEM_FLUSH_LOGS(started_cluster): instance = cluster.instances['ch1'] instance.query(''' SET log_queries = 0; + SYSTEM FLUSH LOGS; TRUNCATE TABLE system.query_log; ''') for i in range(4): @@ -104,13 +105,13 @@ def test_SYSTEM_FLUSH_LOGS(started_cluster): # by expiration of flush_interval_millisecond and test probable race condition. time.sleep(0.5) result = instance.query(''' - SET log_queries = 1; SELECT 1 FORMAT Null; SET log_queries = 0; SYSTEM FLUSH LOGS; SELECT count() FROM system.query_log;''') instance.query(''' SET log_queries = 0; + SYSTEM FLUSH LOGS; TRUNCATE TABLE system.query_log; ''') assert TSV(result) == TSV('4') From f29d3b8cc916d9ff750186f89e4932b076481193 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 27 May 2020 11:58:11 +0300 Subject: [PATCH 34/79] Fix cmake one more time --- contrib/cctz-cmake/CMakeLists.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index d353fc9d4de..76a1fe6e8ac 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -622,7 +622,8 @@ if (USE_INTERNAL_CCTZ) # CMake can shuffle each of target_link_libraries arguments with other # libraries in linker command. To avoid this we hardcode whole-archive # library into single string. - target_link_libraries(cctz PUBLIC "-Wl,--whole-archive contrib/cctz-cmake/libtzdata.a -Wl,--no-whole-archive") + add_dependencies(cctz tzdata) + target_link_libraries(cctz PRIVATE "-Wl,--whole-archive contrib/cctz-cmake/libtzdata.a -Wl,--no-whole-archive") endif () else () From f452c50d86f897e40bc167abf56674d41b77c7c5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 27 May 2020 12:00:33 +0300 Subject: [PATCH 35/79] Replace private with public --- contrib/cctz-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 76a1fe6e8ac..707ebe30a13 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -623,7 +623,7 @@ if (USE_INTERNAL_CCTZ) # libraries in linker command. To avoid this we hardcode whole-archive # library into single string. add_dependencies(cctz tzdata) - target_link_libraries(cctz PRIVATE "-Wl,--whole-archive contrib/cctz-cmake/libtzdata.a -Wl,--no-whole-archive") + target_link_libraries(cctz PUBLIC "-Wl,--whole-archive contrib/cctz-cmake/libtzdata.a -Wl,--no-whole-archive") endif () else () From d04962808e1f0c4fe53db55aab765741f8b6b945 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 27 May 2020 12:14:40 +0300 Subject: [PATCH 36/79] Actually it's INTERFACE linking --- contrib/cctz-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 707ebe30a13..2c44b25b17a 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -623,7 +623,7 @@ if (USE_INTERNAL_CCTZ) # libraries in linker command. To avoid this we hardcode whole-archive # library into single string. add_dependencies(cctz tzdata) - target_link_libraries(cctz PUBLIC "-Wl,--whole-archive contrib/cctz-cmake/libtzdata.a -Wl,--no-whole-archive") + target_link_libraries(cctz INTERFACE "-Wl,--whole-archive contrib/cctz-cmake/libtzdata.a -Wl,--no-whole-archive") endif () else () From 61974e0047e9826ab84d5d4b07f93ec065b285dc Mon Sep 17 00:00:00 2001 From: Nikolai Sorokin Date: Mon, 27 Apr 2020 17:47:59 +0300 Subject: [PATCH 37/79] TTL Expressions WHERE and GROUP BY draft --- src/DataStreams/TTLBlockInputStream.cpp | 225 +++++++++++++++++++++-- src/DataStreams/TTLBlockInputStream.h | 8 + src/Interpreters/ExpressionAnalyzer.cpp | 9 +- src/Interpreters/SyntaxAnalyzer.cpp | 17 +- src/Interpreters/SyntaxAnalyzer.h | 2 +- src/Parsers/ASTTTLElement.cpp | 67 ++++++- src/Parsers/ASTTTLElement.h | 38 +++- src/Parsers/ExpressionElementParsers.cpp | 84 ++++++++- src/Parsers/ExpressionListParsers.cpp | 9 + src/Parsers/ExpressionListParsers.h | 11 +- src/Storages/MergeTree/MergeTreeData.cpp | 44 ++++- src/Storages/MergeTree/MergeTreeData.h | 9 + 12 files changed, 479 insertions(+), 44 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index d2d5d6a92f9..d816cba4b2f 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -67,6 +67,30 @@ TTLBlockInputStream::TTLBlockInputStream( default_expr_list, storage.getColumns().getAllPhysical()); defaults_expression = ExpressionAnalyzer{default_expr_list, syntax_result, storage.global_context}.getActions(true); } + + if (storage.hasRowsTTL() && !storage.rows_ttl_entry.group_by_keys.empty()) + { + ColumnNumbers keys; + for (const auto & key : storage.rows_ttl_entry.group_by_keys) + keys.push_back(header.getPositionByName(key)); + agg_key_columns.resize(storage.rows_ttl_entry.group_by_keys.size()); + + AggregateDescriptions aggregates = storage.rows_ttl_entry.aggregate_descriptions; + for (auto & descr : aggregates) + if (descr.arguments.empty()) + for (const auto & name : descr.argument_names) + descr.arguments.push_back(header.getPositionByName(name)); + agg_aggregate_columns.resize(storage.rows_ttl_entry.aggregate_descriptions.size()); + + const Settings & settings = storage.global_context.getSettingsRef(); + + Aggregator::Params params(header, keys, aggregates, + false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, + SettingUInt64(0), SettingUInt64(0), + settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, + storage.global_context.getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); + aggregator = std::make_unique(params); + } } bool TTLBlockInputStream::isTTLExpired(time_t ttl) const @@ -77,7 +101,8 @@ bool TTLBlockInputStream::isTTLExpired(time_t ttl) const Block TTLBlockInputStream::readImpl() { /// Skip all data if table ttl is expired for part - if (storage.hasRowsTTL() && isTTLExpired(old_ttl_infos.table_ttl.max)) + if (storage.hasRowsTTL() && !storage.rows_ttl_entry.where_expression && + storage.rows_ttl_entry.group_by_keys.empty() && isTTLExpired(old_ttl_infos.table_ttl.max)) { rows_removed = data_part->rows_count; return {}; @@ -85,7 +110,43 @@ Block TTLBlockInputStream::readImpl() Block block = children.at(0)->read(); if (!block) + { + if (aggregator && !agg_result.empty()) + { + MutableColumns result_columns; + const auto & column_names = header.getNames(); + for (const auto & column_name : column_names) + { + const IColumn * values_column = header.getByName(column_name).column.get(); + MutableColumnPtr result_column = values_column->cloneEmpty(); + result_columns.emplace_back(std::move(result_column)); + } + + auto aggregated_res = aggregator->convertToBlocks(agg_result, true, 1); + for (auto & agg_block : aggregated_res) + { + for (const auto & [name, actions] : storage.rows_ttl_entry.group_by_aggregations) + actions->execute(agg_block); + for (const auto & name : storage.rows_ttl_entry.group_by_keys) + { + const IColumn * values_column = agg_block.getByName(name).column.get(); + auto & result_column = result_columns[header.getPositionByName(name)]; + result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); + } + for (const auto & [name, res_column] : storage.rows_ttl_entry.group_by_aggregations_res_column) + { + const IColumn * values_column = agg_block.getByName(res_column).column.get(); + auto & result_column = result_columns[header.getPositionByName(name)]; + result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); + } + } + + block = header.cloneWithColumns(std::move(result_columns)); + agg_result.invalidate(); + } + return block; + } if (storage.hasRowsTTL() && (force || isTTLExpired(old_ttl_infos.table_ttl.min))) removeRowsWithExpiredTableTTL(block); @@ -114,35 +175,171 @@ void TTLBlockInputStream::readSuffixImpl() void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) { storage.rows_ttl_entry.expression->execute(block); + if (storage.rows_ttl_entry.where_expression) + storage.rows_ttl_entry.where_expression->execute(block); const IColumn * ttl_column = block.getByName(storage.rows_ttl_entry.result_column).column.get(); + const IColumn * where_filter_column = storage.rows_ttl_entry.where_expression ? + block.getByName(storage.rows_ttl_entry.where_filter_column).column.get() : nullptr; + const auto & column_names = header.getNames(); - MutableColumns result_columns; - result_columns.reserve(column_names.size()); - for (auto it = column_names.begin(); it != column_names.end(); ++it) + if (!aggregator) { - const IColumn * values_column = block.getByName(*it).column.get(); - MutableColumnPtr result_column = values_column->cloneEmpty(); - result_column->reserve(block.rows()); + MutableColumns result_columns; + result_columns.reserve(column_names.size()); + for (auto it = column_names.begin(); it != column_names.end(); ++it) + { + const IColumn * values_column = block.getByName(*it).column.get(); + MutableColumnPtr result_column = values_column->cloneEmpty(); + result_column->reserve(block.rows()); + for (size_t i = 0; i < block.rows(); ++i) + { + UInt32 cur_ttl = getTimestampByIndex(ttl_column, i); + bool where_filter_passed = !where_filter_column || where_filter_column->getBool(i); + if (!isTTLExpired(cur_ttl) || !where_filter_passed) + { + new_ttl_infos.table_ttl.update(cur_ttl); + result_column->insertFrom(*values_column, i); + } + else if (it == column_names.begin()) + ++rows_removed; + } + result_columns.emplace_back(std::move(result_column)); + } + block = header.cloneWithColumns(std::move(result_columns)); + } + else + { + MutableColumns result_columns; + MutableColumns aggregate_columns; + + for (const auto & column_name : column_names) + { + const IColumn * values_column = block.getByName(column_name).column.get(); + MutableColumnPtr result_column = values_column->cloneEmpty(); + result_column->reserve(block.rows()); + result_columns.emplace_back(std::move(result_column)); + + MutableColumnPtr aggregate_column = values_column->cloneEmpty(); + aggregate_column->reserve(block.rows()); + aggregate_columns.emplace_back(std::move(aggregate_column)); + } + + size_t rows_aggregated = 0; + size_t current_key_start = 0; + size_t rows_with_current_key = 0; for (size_t i = 0; i < block.rows(); ++i) { UInt32 cur_ttl = getTimestampByIndex(ttl_column, i); - if (!isTTLExpired(cur_ttl)) + bool where_filter_passed = !where_filter_column || where_filter_column->getBool(i); + bool ttl_expired = isTTLExpired(cur_ttl) && where_filter_passed; + + bool same_as_current = true; + if (current_key_value.empty()) + { + same_as_current = false; + current_key_value.resize(storage.rows_ttl_entry.group_by_keys.size()); + } + for (size_t j = 0; j < storage.rows_ttl_entry.group_by_keys.size(); ++j) + { + const String & key_column = storage.rows_ttl_entry.group_by_keys[j]; + const IColumn * values_column = block.getByName(key_column).column.get(); + if (!same_as_current) + values_column->get(i, current_key_value[j]); + else + { + Field value; + values_column->get(i, value); + if (value != current_key_value[j]) + { + current_key_value[j] = value; + same_as_current = false; + } + } + } + if (!same_as_current) + { + if (rows_with_current_key) + { + Columns aggregate_chunk; + aggregate_chunk.reserve(aggregate_columns.size()); + for (const auto & name : column_names) + { + const auto & column = aggregate_columns[header.getPositionByName(name)]; + ColumnPtr chunk_column = column->cut(current_key_start, rows_with_current_key); + aggregate_chunk.emplace_back(std::move(chunk_column)); + } + aggregator->executeOnBlock(aggregate_chunk, rows_with_current_key, agg_result, agg_key_columns, + agg_aggregate_columns, agg_no_more_keys); + } + if (!agg_result.empty()) + { + auto aggregated_res = aggregator->convertToBlocks(agg_result, true, 1); + for (auto & agg_block : aggregated_res) + { + for (const auto & [name, actions] : storage.rows_ttl_entry.group_by_aggregations) + actions->execute(agg_block); + for (const auto & name : storage.rows_ttl_entry.group_by_keys) + { + const IColumn * values_column = agg_block.getByName(name).column.get(); + auto & result_column = result_columns[header.getPositionByName(name)]; + result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); + } + for (const auto & [name, res_column] : storage.rows_ttl_entry.group_by_aggregations_res_column) + { + const IColumn * values_column = agg_block.getByName(res_column).column.get(); + auto & result_column = result_columns[header.getPositionByName(name)]; + result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); + } + } + } + agg_result.invalidate(); + current_key_start = rows_aggregated; + rows_with_current_key = 0; + } + if (ttl_expired) + { + ++rows_with_current_key; + ++rows_aggregated; + for (const auto & name : column_names) + { + const IColumn * values_column = block.getByName(name).column.get(); + auto & column = aggregate_columns[header.getPositionByName(name)]; + column->insertFrom(*values_column, i); + } + } + else { new_ttl_infos.table_ttl.update(cur_ttl); - result_column->insertFrom(*values_column, i); + for (const auto & name : column_names) + { + const IColumn * values_column = block.getByName(name).column.get(); + auto & column = result_columns[header.getPositionByName(name)]; + column->insertFrom(*values_column, i); + } } - else if (it == column_names.begin()) - ++rows_removed; } - result_columns.emplace_back(std::move(result_column)); - } - block = header.cloneWithColumns(std::move(result_columns)); + if (rows_with_current_key) + { + Columns aggregate_chunk; + aggregate_chunk.reserve(aggregate_columns.size()); + for (const auto & name : column_names) + { + const auto & column = aggregate_columns[header.getPositionByName(name)]; + ColumnPtr chunk_column = column->cut(current_key_start, rows_with_current_key); + aggregate_chunk.emplace_back(std::move(chunk_column)); + } + aggregator->executeOnBlock(aggregate_chunk, rows_with_current_key, agg_result, agg_key_columns, + agg_aggregate_columns, agg_no_more_keys); + } + + block = header.cloneWithColumns(std::move(result_columns)); + } } void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/DataStreams/TTLBlockInputStream.h index 3896e5232f8..821f8dd9284 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/DataStreams/TTLBlockInputStream.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -39,6 +40,13 @@ private: time_t current_time; bool force; + std::unique_ptr aggregator; + std::vector current_key_value; + AggregatedDataVariants agg_result; + ColumnRawPtrs agg_key_columns; + Aggregator::AggregateColumns agg_aggregate_columns; + bool agg_no_more_keys; + IMergeTreeDataPart::TTLInfos old_ttl_infos; IMergeTreeDataPart::TTLInfos new_ttl_infos; NameSet empty_columns; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 3341855b8c6..c432b920a9f 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -192,10 +192,10 @@ void ExpressionAnalyzer::analyzeAggregation() if (has_aggregation) { - getSelectQuery(); /// assertSelect() + // getSelectQuery(); /// assertSelect() /// Find out aggregation keys. - if (select_query->groupBy()) + if (select_query && select_query->groupBy()) { NameSet unique_keys; ASTs & group_asts = select_query->groupBy()->children; @@ -926,7 +926,10 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result) { - ExpressionActionsPtr actions = std::make_shared(sourceColumns(), context); + NamesAndTypesList columns(sourceColumns()); + for (const auto & col : aggregated_columns) + columns.push_back(col); + ExpressionActionsPtr actions = std::make_shared(columns, context); NamesWithAliases result_columns; Names result_names; diff --git a/src/Interpreters/SyntaxAnalyzer.cpp b/src/Interpreters/SyntaxAnalyzer.cpp index b3d566dbdc8..ec7f2154dad 100644 --- a/src/Interpreters/SyntaxAnalyzer.cpp +++ b/src/Interpreters/SyntaxAnalyzer.cpp @@ -839,7 +839,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyzeSelect( return std::make_shared(result); } -SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(ASTPtr & query, const NamesAndTypesList & source_columns, ConstStoragePtr storage) const +SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(ASTPtr & query, const NamesAndTypesList & source_columns, ConstStoragePtr storage, bool allow_aggregates) const { if (query->as()) throw Exception("Not select analyze for select asts.", ErrorCodes::LOGICAL_ERROR); @@ -855,7 +855,20 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(ASTPtr & query, const NamesAndTy optimizeIf(query, result.aliases, settings.optimize_if_chain_to_miltiif); - assertNoAggregates(query, "in wrong place"); + if (allow_aggregates) + { + 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"); + result.aggregates = data.aggregates; + } + else + assertNoAggregates(query, "in wrong place"); + result.collectUsedColumns(query); return std::make_shared(result); } diff --git a/src/Interpreters/SyntaxAnalyzer.h b/src/Interpreters/SyntaxAnalyzer.h index dda0add38db..abacb25ac4d 100644 --- a/src/Interpreters/SyntaxAnalyzer.h +++ b/src/Interpreters/SyntaxAnalyzer.h @@ -86,7 +86,7 @@ public: {} /// Analyze and rewrite not select query - SyntaxAnalyzerResultPtr analyze(ASTPtr & query, const NamesAndTypesList & source_columns_, ConstStoragePtr storage = {}) const; + SyntaxAnalyzerResultPtr analyze(ASTPtr & query, const NamesAndTypesList & source_columns_, ConstStoragePtr storage = {}, bool allow_aggregations = false) const; /// Analyze and rewrite select query SyntaxAnalyzerResultPtr analyzeSelect( diff --git a/src/Parsers/ASTTTLElement.cpp b/src/Parsers/ASTTTLElement.cpp index 7e03a73e36d..fb35266465b 100644 --- a/src/Parsers/ASTTTLElement.cpp +++ b/src/Parsers/ASTTTLElement.cpp @@ -7,21 +7,80 @@ namespace DB { +ASTPtr ASTTTLElement::clone() const +{ + auto clone = std::make_shared(*this); + clone->children.clear(); + clone->positions.clear(); + + for (auto expr : {Expression::TTL, Expression::WHERE}) + clone->setExpression(expr, getExpression(expr, true)); + + for (auto & [name, expr] : clone->group_by_aggregations) + expr = expr->clone(); + + return clone; +} + void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - children.front()->formatImpl(settings, state, frame); - if (destination_type == PartDestinationType::DISK) + ttl()->formatImpl(settings, state, frame); + if (mode == Mode::MOVE && destination_type == PartDestinationType::DISK) { settings.ostr << " TO DISK " << quoteString(destination_name); } - else if (destination_type == PartDestinationType::VOLUME) + else if (mode == Mode::MOVE && destination_type == PartDestinationType::VOLUME) { settings.ostr << " TO VOLUME " << quoteString(destination_name); } - else if (destination_type == PartDestinationType::DELETE) + else if (mode == Mode::GROUP_BY) + { + settings.ostr << " GROUP BY "; + for (size_t i = 0; i < group_by_key_columns.size(); ++i) + { + settings.ostr << group_by_key_columns[i]; + if (i + 1 != group_by_key_columns.size()) + settings.ostr << ", "; + } + settings.ostr << " SET "; + for (size_t i = 0; i < group_by_aggregations.size(); ++i) + { + settings.ostr << group_by_aggregations[i].first << " = "; + group_by_aggregations[i].second->formatImpl(settings, state, frame); + if (i + 1 != group_by_aggregations.size()) + settings.ostr << ", "; + } + } + else if (mode == Mode::DELETE) { /// It would be better to output "DELETE" here but that will break compatibility with earlier versions. } + + if (where()) + { + settings.ostr << " WHERE "; + where()->formatImpl(settings, state, frame); + } +} + +void ASTTTLElement::setExpression(Expression expr, ASTPtr && ast) +{ + auto it = positions.find(expr); + if (it == positions.end()) + { + positions[expr] = children.size(); + children.emplace_back(ast); + } + else + children[it->second] = ast; +} + +ASTPtr ASTTTLElement::getExpression(Expression expr, bool clone) const +{ + auto it = positions.find(expr); + if (it != positions.end()) + return clone ? children[it->second]->clone() : children[it->second]; + return {}; } } diff --git a/src/Parsers/ASTTTLElement.h b/src/Parsers/ASTTTLElement.h index 02f70094e04..1a2d7d3c723 100644 --- a/src/Parsers/ASTTTLElement.h +++ b/src/Parsers/ASTTTLElement.h @@ -6,31 +6,53 @@ namespace DB { + /** Element of TTL expression. */ class ASTTTLElement : public IAST { public: + enum class Expression : uint8_t + { + TTL, + WHERE + }; + + enum class Mode : uint8_t + { + DELETE, + MOVE, + GROUP_BY + }; + + Mode mode; PartDestinationType destination_type; String destination_name; + std::vector group_by_key_columns; + std::vector> group_by_aggregations; - ASTTTLElement(PartDestinationType destination_type_, const String & destination_name_) - : destination_type(destination_type_) + ASTTTLElement(Mode mode_, PartDestinationType destination_type_, const String & destination_name_) + : mode(mode_) + , destination_type(destination_type_) , destination_name(destination_name_) { } String getID(char) const override { return "TTLElement"; } - ASTPtr clone() const override - { - auto clone = std::make_shared(*this); - clone->cloneChildren(); - return clone; - } + ASTPtr clone() const override; + + const ASTPtr ttl() const { return getExpression(Expression::TTL); } + const ASTPtr where() const { return getExpression(Expression::WHERE); } + + void setExpression(Expression expr, ASTPtr && ast); + ASTPtr getExpression(Expression expr, bool clone = false) const; protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; + +private: + std::unordered_map positions; }; } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 70a8b282a72..2734aa67193 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1455,23 +1455,50 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_to_disk("TO DISK"); ParserKeyword s_to_volume("TO VOLUME"); ParserKeyword s_delete("DELETE"); + ParserKeyword s_where("WHERE"); + ParserKeyword s_group_by("GROUP BY"); + ParserKeyword s_set("SET"); + ParserToken s_comma(TokenType::Comma); + ParserToken s_eq(TokenType::Equals); + ParserIdentifier parser_identifier; ParserStringLiteral parser_string_literal; ParserExpression parser_exp; + ParserIdentifierList parser_identifier_list; - ASTPtr expr_elem; - if (!parser_exp.parse(pos, expr_elem, expected)) + + ASTPtr ttl_expr; + if (!parser_exp.parse(pos, ttl_expr, expected)) return false; + ASTPtr where_expr; + + std::vector group_by_key_columns; + std::vector> group_by_aggregations; + + ASTTTLElement::Mode mode; PartDestinationType destination_type = PartDestinationType::DELETE; String destination_name; if (s_to_disk.ignore(pos)) + { + mode = ASTTTLElement::Mode::MOVE; destination_type = PartDestinationType::DISK; + } else if (s_to_volume.ignore(pos)) + { + mode = ASTTTLElement::Mode::MOVE; destination_type = PartDestinationType::VOLUME; + } + else if (s_group_by.ignore(pos)) + { + mode = ASTTTLElement::Mode::GROUP_BY; + } else + { s_delete.ignore(pos); + mode = ASTTTLElement::Mode::DELETE; + } - if (destination_type == PartDestinationType::DISK || destination_type == PartDestinationType::VOLUME) + if (mode == ASTTTLElement::Mode::MOVE) { ASTPtr ast_space_name; if (!parser_string_literal.parse(pos, ast_space_name, expected)) @@ -1479,10 +1506,57 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) destination_name = ast_space_name->as().value.get(); } + else if (mode == ASTTTLElement::Mode::GROUP_BY) + { + ASTPtr ast_group_by_key_columns; + if (!parser_identifier_list.parse(pos, ast_group_by_key_columns, expected)) + return false; + for (const auto identifier : ast_group_by_key_columns->children) + { + String identifier_str; + if (!tryGetIdentifierNameInto(identifier, identifier_str)) + return false; + group_by_key_columns.emplace_back(std::move(identifier_str)); + } - node = std::make_shared(destination_type, destination_name); - node->children.push_back(expr_elem); + if (!s_set.ignore(pos)) + return false; + while (true) + { + if (!group_by_aggregations.empty() && !s_comma.ignore(pos)) + break; + + ASTPtr name; + ASTPtr value; + if (!parser_identifier.parse(pos, name, expected)) + return false; + if (!s_eq.ignore(pos)) + return false; + if (!parser_exp.parse(pos, value, expected)) + return false; + String name_str; + if (!tryGetIdentifierNameInto(name, name_str)) + return false; + group_by_aggregations.emplace_back(name_str, std::move(value)); + } + } + + if ((mode == ASTTTLElement::Mode::MOVE || mode == ASTTTLElement::Mode::DELETE) && s_where.ignore(pos)) + { + if (!parser_exp.parse(pos, where_expr, expected)) + return false; + } + + auto ttl_element = std::make_shared(mode, destination_type, destination_name); + ttl_element->setExpression(ASTTTLElement::Expression::TTL, std::move(ttl_expr)); + if (where_expr) + ttl_element->setExpression(ASTTTLElement::Expression::WHERE, std::move(where_expr)); + + ttl_element->group_by_key_columns = std::move(group_by_key_columns); + ttl_element->group_by_aggregations = std::move(group_by_aggregations); + + node = ttl_element; return true; } diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index a967ae19691..062d1dd4483 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -742,4 +742,13 @@ bool ParserKeyValuePairsList::parseImpl(Pos & pos, ASTPtr & node, Expected & exp return parser.parse(pos, node, expected); } + +bool ParserIdentifierList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + return ParserList( + std::make_unique(), + std::make_unique(TokenType::Comma)) + .parse(pos, node, expected); } + +} \ No newline at end of file diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 0cef29b6d67..ba939d8d160 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -421,4 +421,13 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; -} + +/** A comma-separated list of identifiers, probably empty. */ +class ParserIdentifierList : public IParserBase +{ +protected: + const char * getName() const override { return "list of identifiers"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} \ No newline at end of file diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 48cf3934820..9eb14ed5475 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -615,19 +615,50 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, auto new_column_ttls = new_columns.getColumnTTLs(); - auto create_ttl_entry = [this, &new_columns](ASTPtr ttl_ast) + auto create_ttl_entry = [this, &new_columns](ASTPtr ttl_expr_ast) { TTLEntry result; - auto syntax_result = SyntaxAnalyzer(global_context).analyze(ttl_ast, new_columns.getAllPhysical()); - result.expression = ExpressionAnalyzer(ttl_ast, syntax_result, global_context).getActions(false); + auto ttl_syntax_result = SyntaxAnalyzer(global_context).analyze(ttl_expr_ast, new_columns.getAllPhysical()); + result.expression = ExpressionAnalyzer(ttl_expr_ast, ttl_syntax_result, global_context).getActions(false); + result.result_column = ttl_expr_ast->getColumnName(); + result.destination_type = PartDestinationType::DELETE; - result.result_column = ttl_ast->getColumnName(); checkTTLExpression(result.expression, result.result_column); return result; }; + auto create_rows_ttl_entry = [this, &new_columns, &create_ttl_entry](const ASTTTLElement * ttl_element) + { + auto result = create_ttl_entry(ttl_element->ttl()); + if (ttl_element->mode == ASTTTLElement::Mode::DELETE || ttl_element->mode ==ASTTTLElement::Mode::MOVE) + { + if (ASTPtr where_expr_ast = ttl_element->where()) + { + auto where_syntax_result = SyntaxAnalyzer(global_context).analyze(where_expr_ast, new_columns.getAllPhysical()); + result.where_expression = ExpressionAnalyzer(where_expr_ast, where_syntax_result, global_context).getActions(false); + result.where_filter_column = where_expr_ast->getColumnName(); + } + } + else if (ttl_element->mode == ASTTTLElement::Mode::GROUP_BY) + { + result.group_by_keys = ttl_element->group_by_key_columns; + for (auto [name, value] : ttl_element->group_by_aggregations) + { + auto syntax_result = SyntaxAnalyzer(global_context).analyze(value, new_columns.getAllPhysical(), {}, true); + auto expr_analyzer = ExpressionAnalyzer(value, syntax_result, global_context); + + result.group_by_aggregations.emplace_back(name, expr_analyzer.getActions(false)); + result.group_by_aggregations_res_column.emplace_back(name, value->getColumnName()); + + for (const auto descr : expr_analyzer.getAnalyzedData().aggregate_descriptions) + result.aggregate_descriptions.push_back(descr); + } + } + return result; + }; + if (!new_column_ttls.empty()) { NameSet columns_ttl_forbidden; @@ -672,7 +703,8 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, throw Exception("More than one DELETE TTL expression is not allowed", ErrorCodes::BAD_TTL_EXPRESSION); } - auto new_rows_ttl_entry = create_ttl_entry(ttl_element->children[0]); + LOG_DEBUG(log, "ttl_element->size is " << ttl_element->size()); + auto new_rows_ttl_entry = create_rows_ttl_entry(ttl_element); if (!only_check) update_rows_ttl_entry = new_rows_ttl_entry; @@ -680,7 +712,7 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, } else { - auto new_ttl_entry = create_ttl_entry(ttl_element->children[0]); + auto new_ttl_entry = create_rows_ttl_entry(ttl_element); new_ttl_entry.entry_ast = ttl_element_ptr; new_ttl_entry.destination_type = ttl_element->destination_type; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index cd1d4912a29..dfcddb40f3c 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -652,6 +653,14 @@ public: ExpressionActionsPtr expression; String result_column; + ExpressionActionsPtr where_expression; + String where_filter_column; + + Names group_by_keys; + std::vector> group_by_aggregations; + std::vector> group_by_aggregations_res_column; + AggregateDescriptions aggregate_descriptions; + /// Name and type of a destination are only valid in table-level context. PartDestinationType destination_type; String destination_name; From eeaf608aa912543540106ec5bc4363b88d3324f3 Mon Sep 17 00:00:00 2001 From: Nikolai Sorokin Date: Tue, 12 May 2020 17:27:00 +0300 Subject: [PATCH 38/79] Refactor code; support prefix of PK as GROUP BY key --- src/DataStreams/TTLBlockInputStream.cpp | 175 +++++++++-------------- src/DataStreams/TTLBlockInputStream.h | 6 + src/Parsers/ASTTTLElement.cpp | 57 ++++---- src/Parsers/ASTTTLElement.h | 38 +++-- src/Parsers/ExpressionElementParsers.cpp | 33 +++-- src/Storages/MergeTree/MergeTreeData.cpp | 32 +++-- src/Storages/MergeTree/MergeTreeData.h | 8 +- src/Storages/MergeTree/TTLMode.h | 14 ++ 8 files changed, 177 insertions(+), 186 deletions(-) create mode 100644 src/Storages/MergeTree/TTLMode.h diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index d816cba4b2f..d5e46a9929d 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -68,8 +69,10 @@ TTLBlockInputStream::TTLBlockInputStream( defaults_expression = ExpressionAnalyzer{default_expr_list, syntax_result, storage.global_context}.getActions(true); } - if (storage.hasRowsTTL() && !storage.rows_ttl_entry.group_by_keys.empty()) + if (storage.hasRowsTTL() && storage.rows_ttl_entry.mode == TTLMode::GROUP_BY) { + current_key_value.resize(storage.rows_ttl_entry.group_by_keys.size()); + ColumnNumbers keys; for (const auto & key : storage.rows_ttl_entry.group_by_keys) keys.push_back(header.getPositionByName(key)); @@ -84,9 +87,11 @@ TTLBlockInputStream::TTLBlockInputStream( const Settings & settings = storage.global_context.getSettingsRef(); + bool allow_to_use_two_level_group_by = false; // settings.max_bytes_before_external_group_by != 0; Aggregator::Params params(header, keys, aggregates, false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, - SettingUInt64(0), SettingUInt64(0), + allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), + allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, storage.global_context.getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); aggregator = std::make_unique(params); @@ -102,7 +107,7 @@ Block TTLBlockInputStream::readImpl() { /// Skip all data if table ttl is expired for part if (storage.hasRowsTTL() && !storage.rows_ttl_entry.where_expression && - storage.rows_ttl_entry.group_by_keys.empty() && isTTLExpired(old_ttl_infos.table_ttl.max)) + storage.rows_ttl_entry.mode != TTLMode::GROUP_BY && isTTLExpired(old_ttl_infos.table_ttl.max)) { rows_removed = data_part->rows_count; return {}; @@ -113,36 +118,9 @@ Block TTLBlockInputStream::readImpl() { if (aggregator && !agg_result.empty()) { - MutableColumns result_columns; - const auto & column_names = header.getNames(); - for (const auto & column_name : column_names) - { - const IColumn * values_column = header.getByName(column_name).column.get(); - MutableColumnPtr result_column = values_column->cloneEmpty(); - result_columns.emplace_back(std::move(result_column)); - } - - auto aggregated_res = aggregator->convertToBlocks(agg_result, true, 1); - for (auto & agg_block : aggregated_res) - { - for (const auto & [name, actions] : storage.rows_ttl_entry.group_by_aggregations) - actions->execute(agg_block); - for (const auto & name : storage.rows_ttl_entry.group_by_keys) - { - const IColumn * values_column = agg_block.getByName(name).column.get(); - auto & result_column = result_columns[header.getPositionByName(name)]; - result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); - } - for (const auto & [name, res_column] : storage.rows_ttl_entry.group_by_aggregations_res_column) - { - const IColumn * values_column = agg_block.getByName(res_column).column.get(); - auto & result_column = result_columns[header.getPositionByName(name)]; - result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); - } - } - + MutableColumns result_columns = header.cloneEmptyColumns(); + finalizeAggregates(result_columns); block = header.cloneWithColumns(std::move(result_columns)); - agg_result.invalidate(); } return block; @@ -181,8 +159,8 @@ void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) const IColumn * ttl_column = block.getByName(storage.rows_ttl_entry.result_column).column.get(); - const IColumn * where_filter_column = storage.rows_ttl_entry.where_expression ? - block.getByName(storage.rows_ttl_entry.where_filter_column).column.get() : nullptr; + const IColumn * where_result_column = storage.rows_ttl_entry.where_expression ? + block.getByName(storage.rows_ttl_entry.where_result_column).column.get() : nullptr; const auto & column_names = header.getNames(); @@ -199,7 +177,7 @@ void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) for (size_t i = 0; i < block.rows(); ++i) { UInt32 cur_ttl = getTimestampByIndex(ttl_column, i); - bool where_filter_passed = !where_filter_column || where_filter_column->getBool(i); + bool where_filter_passed = !where_result_column || where_result_column->getBool(i); if (!isTTLExpired(cur_ttl) || !where_filter_passed) { new_ttl_infos.table_ttl.update(cur_ttl); @@ -214,20 +192,8 @@ void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) } else { - MutableColumns result_columns; - MutableColumns aggregate_columns; - - for (const auto & column_name : column_names) - { - const IColumn * values_column = block.getByName(column_name).column.get(); - MutableColumnPtr result_column = values_column->cloneEmpty(); - result_column->reserve(block.rows()); - result_columns.emplace_back(std::move(result_column)); - - MutableColumnPtr aggregate_column = values_column->cloneEmpty(); - aggregate_column->reserve(block.rows()); - aggregate_columns.emplace_back(std::move(aggregate_column)); - } + MutableColumns result_columns = header.cloneEmptyColumns(); + MutableColumns aggregate_columns = header.cloneEmptyColumns(); size_t rows_aggregated = 0; size_t current_key_start = 0; @@ -235,72 +201,30 @@ void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) for (size_t i = 0; i < block.rows(); ++i) { UInt32 cur_ttl = getTimestampByIndex(ttl_column, i); - bool where_filter_passed = !where_filter_column || where_filter_column->getBool(i); + bool where_filter_passed = !where_result_column || where_result_column->getBool(i); bool ttl_expired = isTTLExpired(cur_ttl) && where_filter_passed; bool same_as_current = true; - if (current_key_value.empty()) - { - same_as_current = false; - current_key_value.resize(storage.rows_ttl_entry.group_by_keys.size()); - } for (size_t j = 0; j < storage.rows_ttl_entry.group_by_keys.size(); ++j) { const String & key_column = storage.rows_ttl_entry.group_by_keys[j]; const IColumn * values_column = block.getByName(key_column).column.get(); - if (!same_as_current) - values_column->get(i, current_key_value[j]); - else + if (!same_as_current || (*values_column)[i] != current_key_value[j]) { - Field value; - values_column->get(i, value); - if (value != current_key_value[j]) - { - current_key_value[j] = value; - same_as_current = false; - } + values_column->get(i, current_key_value[j]); + same_as_current = false; } } if (!same_as_current) { if (rows_with_current_key) - { - Columns aggregate_chunk; - aggregate_chunk.reserve(aggregate_columns.size()); - for (const auto & name : column_names) - { - const auto & column = aggregate_columns[header.getPositionByName(name)]; - ColumnPtr chunk_column = column->cut(current_key_start, rows_with_current_key); - aggregate_chunk.emplace_back(std::move(chunk_column)); - } - aggregator->executeOnBlock(aggregate_chunk, rows_with_current_key, agg_result, agg_key_columns, - agg_aggregate_columns, agg_no_more_keys); - } - if (!agg_result.empty()) - { - auto aggregated_res = aggregator->convertToBlocks(agg_result, true, 1); - for (auto & agg_block : aggregated_res) - { - for (const auto & [name, actions] : storage.rows_ttl_entry.group_by_aggregations) - actions->execute(agg_block); - for (const auto & name : storage.rows_ttl_entry.group_by_keys) - { - const IColumn * values_column = agg_block.getByName(name).column.get(); - auto & result_column = result_columns[header.getPositionByName(name)]; - result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); - } - for (const auto & [name, res_column] : storage.rows_ttl_entry.group_by_aggregations_res_column) - { - const IColumn * values_column = agg_block.getByName(res_column).column.get(); - auto & result_column = result_columns[header.getPositionByName(name)]; - result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); - } - } - } - agg_result.invalidate(); + calculateAggregates(aggregate_columns, current_key_start, rows_with_current_key); + finalizeAggregates(result_columns); + current_key_start = rows_aggregated; rows_with_current_key = 0; } + if (ttl_expired) { ++rows_with_current_key; @@ -325,23 +249,52 @@ void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) } if (rows_with_current_key) - { - Columns aggregate_chunk; - aggregate_chunk.reserve(aggregate_columns.size()); - for (const auto & name : column_names) - { - const auto & column = aggregate_columns[header.getPositionByName(name)]; - ColumnPtr chunk_column = column->cut(current_key_start, rows_with_current_key); - aggregate_chunk.emplace_back(std::move(chunk_column)); - } - aggregator->executeOnBlock(aggregate_chunk, rows_with_current_key, agg_result, agg_key_columns, - agg_aggregate_columns, agg_no_more_keys); - } + calculateAggregates(aggregate_columns, current_key_start, rows_with_current_key); block = header.cloneWithColumns(std::move(result_columns)); } } +void TTLBlockInputStream::calculateAggregates(const MutableColumns & aggregate_columns, size_t start_pos, size_t length) +{ + Columns aggregate_chunk; + aggregate_chunk.reserve(aggregate_columns.size()); + for (const auto & name : header.getNames()) + { + const auto & column = aggregate_columns[header.getPositionByName(name)]; + ColumnPtr chunk_column = column->cut(start_pos, length); + aggregate_chunk.emplace_back(std::move(chunk_column)); + } + aggregator->executeOnBlock(aggregate_chunk, length, agg_result, agg_key_columns, + agg_aggregate_columns, agg_no_more_keys); +} + +void TTLBlockInputStream::finalizeAggregates(MutableColumns & result_columns) +{ + if (!agg_result.empty()) + { + auto aggregated_res = aggregator->convertToBlocks(agg_result, true, 1); + for (auto & agg_block : aggregated_res) + { + for (const auto & it : storage.rows_ttl_entry.group_by_aggregations) + std::get<2>(it)->execute(agg_block); + for (const auto & name : storage.rows_ttl_entry.group_by_keys) + { + const IColumn * values_column = agg_block.getByName(name).column.get(); + auto & result_column = result_columns[header.getPositionByName(name)]; + result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); + } + for (const auto & it : storage.rows_ttl_entry.group_by_aggregations) + { + const IColumn * values_column = agg_block.getByName(get<1>(it)).column.get(); + auto & result_column = result_columns[header.getPositionByName(std::get<0>(it))]; + result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); + } + } + } + agg_result.invalidate(); +} + void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) { Block block_with_defaults; diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/DataStreams/TTLBlockInputStream.h index 821f8dd9284..c5ec6596038 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/DataStreams/TTLBlockInputStream.h @@ -66,6 +66,12 @@ private: /// Removes rows with expired table ttl and computes new ttl_infos for part void removeRowsWithExpiredTableTTL(Block & block); + + // Calculate aggregates of aggregate_columns into agg_result + void calculateAggregates(const MutableColumns & aggregate_columns, size_t start_pos, size_t length); + + /// Finalize agg_result into result_columns + void finalizeAggregates(MutableColumns & result_columns); /// Updates TTL for moves void updateMovesTTL(Block & block); diff --git a/src/Parsers/ASTTTLElement.cpp b/src/Parsers/ASTTTLElement.cpp index fb35266465b..35e9dc89a0d 100644 --- a/src/Parsers/ASTTTLElement.cpp +++ b/src/Parsers/ASTTTLElement.cpp @@ -11,10 +11,11 @@ ASTPtr ASTTTLElement::clone() const { auto clone = std::make_shared(*this); clone->children.clear(); - clone->positions.clear(); + clone->ttl_expr_pos = -1; + clone->where_expr_pos = -1; - for (auto expr : {Expression::TTL, Expression::WHERE}) - clone->setExpression(expr, getExpression(expr, true)); + clone->setExpression(clone->ttl_expr_pos, getExpression(ttl_expr_pos, true)); + clone->setExpression(clone->where_expr_pos, getExpression(where_expr_pos, true)); for (auto & [name, expr] : clone->group_by_aggregations) expr = expr->clone(); @@ -25,33 +26,33 @@ ASTPtr ASTTTLElement::clone() const void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { ttl()->formatImpl(settings, state, frame); - if (mode == Mode::MOVE && destination_type == PartDestinationType::DISK) + if (mode == TTLMode::MOVE && destination_type == PartDestinationType::DISK) { settings.ostr << " TO DISK " << quoteString(destination_name); } - else if (mode == Mode::MOVE && destination_type == PartDestinationType::VOLUME) + else if (mode == TTLMode::MOVE && destination_type == PartDestinationType::VOLUME) { settings.ostr << " TO VOLUME " << quoteString(destination_name); } - else if (mode == Mode::GROUP_BY) + else if (mode == TTLMode::GROUP_BY) { settings.ostr << " GROUP BY "; - for (size_t i = 0; i < group_by_key_columns.size(); ++i) + for (auto it = group_by_key_columns.begin(); it != group_by_key_columns.end(); ++it) { - settings.ostr << group_by_key_columns[i]; - if (i + 1 != group_by_key_columns.size()) + if (it != group_by_key_columns.begin()) settings.ostr << ", "; + settings.ostr << *it; } settings.ostr << " SET "; - for (size_t i = 0; i < group_by_aggregations.size(); ++i) + for (auto it = group_by_aggregations.begin(); it != group_by_aggregations.end(); ++it) { - settings.ostr << group_by_aggregations[i].first << " = "; - group_by_aggregations[i].second->formatImpl(settings, state, frame); - if (i + 1 != group_by_aggregations.size()) + if (it != group_by_aggregations.begin()) settings.ostr << ", "; + settings.ostr << it->first << " = "; + it->second->formatImpl(settings, state, frame); } } - else if (mode == Mode::DELETE) + else if (mode == TTLMode::DELETE) { /// It would be better to output "DELETE" here but that will break compatibility with earlier versions. } @@ -63,24 +64,28 @@ void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & st } } -void ASTTTLElement::setExpression(Expression expr, ASTPtr && ast) +void ASTTTLElement::setExpression(int & pos, ASTPtr && ast) { - auto it = positions.find(expr); - if (it == positions.end()) + if (ast) { - positions[expr] = children.size(); - children.emplace_back(ast); + if (pos == -1) + { + pos = children.size(); + children.emplace_back(ast); + } + else + children[pos] = ast; + } + else if (pos != -1) + { + children[pos] = ASTPtr{}; + pos = -1; } - else - children[it->second] = ast; } -ASTPtr ASTTTLElement::getExpression(Expression expr, bool clone) const +ASTPtr ASTTTLElement::getExpression(int pos, bool clone) const { - auto it = positions.find(expr); - if (it != positions.end()) - return clone ? children[it->second]->clone() : children[it->second]; - return {}; + return pos != -1 ? (clone ? children[pos]->clone() : children[pos]) : ASTPtr{}; } } diff --git a/src/Parsers/ASTTTLElement.h b/src/Parsers/ASTTTLElement.h index 1a2d7d3c723..24b02860f32 100644 --- a/src/Parsers/ASTTTLElement.h +++ b/src/Parsers/ASTTTLElement.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -12,29 +13,19 @@ namespace DB class ASTTTLElement : public IAST { public: - enum class Expression : uint8_t - { - TTL, - WHERE - }; - - enum class Mode : uint8_t - { - DELETE, - MOVE, - GROUP_BY - }; - - Mode mode; + TTLMode mode; PartDestinationType destination_type; String destination_name; - std::vector group_by_key_columns; + + Strings group_by_key_columns; std::vector> group_by_aggregations; - ASTTTLElement(Mode mode_, PartDestinationType destination_type_, const String & destination_name_) + ASTTTLElement(TTLMode mode_, PartDestinationType destination_type_, const String & destination_name_) : mode(mode_) , destination_type(destination_type_) , destination_name(destination_name_) + , ttl_expr_pos(-1) + , where_expr_pos(-1) { } @@ -42,17 +33,22 @@ public: ASTPtr clone() const override; - const ASTPtr ttl() const { return getExpression(Expression::TTL); } - const ASTPtr where() const { return getExpression(Expression::WHERE); } + const ASTPtr ttl() const { return getExpression(ttl_expr_pos); } + const ASTPtr where() const { return getExpression(where_expr_pos); } - void setExpression(Expression expr, ASTPtr && ast); - ASTPtr getExpression(Expression expr, bool clone = false) const; + void setTTL(ASTPtr && ast) { setExpression(ttl_expr_pos, std::forward(ast)); } + void setWhere(ASTPtr && ast) { setExpression(where_expr_pos, std::forward(ast)); } protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +private: + int ttl_expr_pos; + int where_expr_pos; + private: - std::unordered_map positions; + void setExpression(int & pos, ASTPtr && ast); + ASTPtr getExpression(int pos, bool clone = false) const; }; } diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 2734aa67193..afd3814d913 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1460,45 +1460,45 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_set("SET"); ParserToken s_comma(TokenType::Comma); ParserToken s_eq(TokenType::Equals); + ParserIdentifier parser_identifier; ParserStringLiteral parser_string_literal; ParserExpression parser_exp; ParserIdentifierList parser_identifier_list; - ASTPtr ttl_expr; if (!parser_exp.parse(pos, ttl_expr, expected)) return false; - ASTPtr where_expr; - - std::vector group_by_key_columns; - std::vector> group_by_aggregations; - - ASTTTLElement::Mode mode; + TTLMode mode; PartDestinationType destination_type = PartDestinationType::DELETE; String destination_name; + if (s_to_disk.ignore(pos)) { - mode = ASTTTLElement::Mode::MOVE; + mode = TTLMode::MOVE; destination_type = PartDestinationType::DISK; } else if (s_to_volume.ignore(pos)) { - mode = ASTTTLElement::Mode::MOVE; + mode = TTLMode::MOVE; destination_type = PartDestinationType::VOLUME; } else if (s_group_by.ignore(pos)) { - mode = ASTTTLElement::Mode::GROUP_BY; + mode = TTLMode::GROUP_BY; } else { s_delete.ignore(pos); - mode = ASTTTLElement::Mode::DELETE; + mode = TTLMode::DELETE; } - if (mode == ASTTTLElement::Mode::MOVE) + ASTPtr where_expr; + std::vector group_by_key_columns; + std::vector> group_by_aggregations; + + if (mode == TTLMode::MOVE) { ASTPtr ast_space_name; if (!parser_string_literal.parse(pos, ast_space_name, expected)) @@ -1506,7 +1506,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) destination_name = ast_space_name->as().value.get(); } - else if (mode == ASTTTLElement::Mode::GROUP_BY) + else if (mode == TTLMode::GROUP_BY) { ASTPtr ast_group_by_key_columns; if (!parser_identifier_list.parse(pos, ast_group_by_key_columns, expected)) @@ -1541,17 +1541,16 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) group_by_aggregations.emplace_back(name_str, std::move(value)); } } - - if ((mode == ASTTTLElement::Mode::MOVE || mode == ASTTTLElement::Mode::DELETE) && s_where.ignore(pos)) + else if (mode == TTLMode::DELETE && s_where.ignore(pos)) { if (!parser_exp.parse(pos, where_expr, expected)) return false; } auto ttl_element = std::make_shared(mode, destination_type, destination_name); - ttl_element->setExpression(ASTTTLElement::Expression::TTL, std::move(ttl_expr)); + ttl_element->setTTL(std::move(ttl_expr)); if (where_expr) - ttl_element->setExpression(ASTTTLElement::Expression::WHERE, std::move(where_expr)); + ttl_element->setWhere(std::move(where_expr)); ttl_element->group_by_key_columns = std::move(group_by_key_columns); ttl_element->group_by_aggregations = std::move(group_by_aggregations); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9eb14ed5475..1f55f5ecff5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -624,6 +624,7 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, result.result_column = ttl_expr_ast->getColumnName(); result.destination_type = PartDestinationType::DELETE; + result.mode = TTLMode::DELETE; checkTTLExpression(result.expression, result.result_column); return result; @@ -632,27 +633,43 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, auto create_rows_ttl_entry = [this, &new_columns, &create_ttl_entry](const ASTTTLElement * ttl_element) { auto result = create_ttl_entry(ttl_element->ttl()); - if (ttl_element->mode == ASTTTLElement::Mode::DELETE || ttl_element->mode ==ASTTTLElement::Mode::MOVE) + result.mode = ttl_element->mode; + + if (ttl_element->mode == TTLMode::DELETE) { if (ASTPtr where_expr_ast = ttl_element->where()) { auto where_syntax_result = SyntaxAnalyzer(global_context).analyze(where_expr_ast, new_columns.getAllPhysical()); result.where_expression = ExpressionAnalyzer(where_expr_ast, where_syntax_result, global_context).getActions(false); - result.where_filter_column = where_expr_ast->getColumnName(); + result.where_result_column = where_expr_ast->getColumnName(); } } - else if (ttl_element->mode == ASTTTLElement::Mode::GROUP_BY) + else if (ttl_element->mode == TTLMode::GROUP_BY) { + if (ttl_element->group_by_key_columns.size() > this->primary_key_columns.size()) + throw Exception("TTL Expression GROUP BY key should be a prefix of primary key", ErrorCodes::BAD_TTL_EXPRESSION); + for (size_t i = 0; i < ttl_element->group_by_key_columns.size(); ++i) + { + if (ttl_element->group_by_key_columns[i] != this->primary_key_columns[i]) + throw Exception("TTL Expression GROUP BY key should be a prefix of primary key", ErrorCodes::BAD_TTL_EXPRESSION); + } + result.group_by_keys = ttl_element->group_by_key_columns; - for (auto [name, value] : ttl_element->group_by_aggregations) + + auto aggregations = ttl_element->group_by_aggregations; + for (size_t i = ttl_element->group_by_key_columns.size(); i < this->primary_key_columns.size(); ++i) + { + ASTPtr expr = makeASTFunction("max", std::make_shared(this->primary_key_columns[i])); + aggregations.emplace_back(this->primary_key_columns[i], std::move(expr)); + } + for (auto [name, value] : aggregations) { auto syntax_result = SyntaxAnalyzer(global_context).analyze(value, new_columns.getAllPhysical(), {}, true); auto expr_analyzer = ExpressionAnalyzer(value, syntax_result, global_context); - result.group_by_aggregations.emplace_back(name, expr_analyzer.getActions(false)); - result.group_by_aggregations_res_column.emplace_back(name, value->getColumnName()); + result.group_by_aggregations.emplace_back(name, value->getColumnName(), expr_analyzer.getActions(false)); - for (const auto descr : expr_analyzer.getAnalyzedData().aggregate_descriptions) + for (const auto & descr : expr_analyzer.getAnalyzedData().aggregate_descriptions) result.aggregate_descriptions.push_back(descr); } } @@ -703,7 +720,6 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, throw Exception("More than one DELETE TTL expression is not allowed", ErrorCodes::BAD_TTL_EXPRESSION); } - LOG_DEBUG(log, "ttl_element->size is " << ttl_element->size()); auto new_rows_ttl_entry = create_rows_ttl_entry(ttl_element); if (!only_check) update_rows_ttl_entry = new_rows_ttl_entry; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index dfcddb40f3c..5272ee25314 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -650,15 +651,16 @@ public: struct TTLEntry { + TTLMode mode; + ExpressionActionsPtr expression; String result_column; ExpressionActionsPtr where_expression; - String where_filter_column; + String where_result_column; Names group_by_keys; - std::vector> group_by_aggregations; - std::vector> group_by_aggregations_res_column; + std::vector> group_by_aggregations; AggregateDescriptions aggregate_descriptions; /// Name and type of a destination are only valid in table-level context. diff --git a/src/Storages/MergeTree/TTLMode.h b/src/Storages/MergeTree/TTLMode.h new file mode 100644 index 00000000000..99a57f46853 --- /dev/null +++ b/src/Storages/MergeTree/TTLMode.h @@ -0,0 +1,14 @@ +#pragma once + + +namespace DB +{ + +enum class TTLMode +{ + DELETE, + MOVE, + GROUP_BY +}; + +} From 66496dc7e350971da8774f66517a4c3a1b62dbbe Mon Sep 17 00:00:00 2001 From: Nikolai Sorokin Date: Tue, 12 May 2020 23:44:48 +0300 Subject: [PATCH 39/79] Fix trailing spaces --- src/DataStreams/TTLBlockInputStream.cpp | 2 +- src/DataStreams/TTLBlockInputStream.h | 4 ++-- src/Interpreters/SyntaxAnalyzer.cpp | 4 ++-- src/Parsers/ASTTTLElement.cpp | 10 +++++----- src/Parsers/ASTTTLElement.h | 6 +++--- src/Parsers/ExpressionElementParsers.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeData.cpp | 6 +++--- src/Storages/MergeTree/TTLMode.h | 2 +- 8 files changed, 20 insertions(+), 20 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index d5e46a9929d..106ae1fd5a0 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -197,7 +197,7 @@ void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) size_t rows_aggregated = 0; size_t current_key_start = 0; - size_t rows_with_current_key = 0; + size_t rows_with_current_key = 0; for (size_t i = 0; i < block.rows(); ++i) { UInt32 cur_ttl = getTimestampByIndex(ttl_column, i); diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/DataStreams/TTLBlockInputStream.h index c5ec6596038..a8f86477b88 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/DataStreams/TTLBlockInputStream.h @@ -43,7 +43,7 @@ private: std::unique_ptr aggregator; std::vector current_key_value; AggregatedDataVariants agg_result; - ColumnRawPtrs agg_key_columns; + ColumnRawPtrs agg_key_columns; Aggregator::AggregateColumns agg_aggregate_columns; bool agg_no_more_keys; @@ -66,7 +66,7 @@ private: /// Removes rows with expired table ttl and computes new ttl_infos for part void removeRowsWithExpiredTableTTL(Block & block); - + // Calculate aggregates of aggregate_columns into agg_result void calculateAggregates(const MutableColumns & aggregate_columns, size_t start_pos, size_t length); diff --git a/src/Interpreters/SyntaxAnalyzer.cpp b/src/Interpreters/SyntaxAnalyzer.cpp index ec7f2154dad..54f369c9495 100644 --- a/src/Interpreters/SyntaxAnalyzer.cpp +++ b/src/Interpreters/SyntaxAnalyzer.cpp @@ -855,7 +855,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(ASTPtr & query, const NamesAndTy optimizeIf(query, result.aliases, settings.optimize_if_chain_to_miltiif); - if (allow_aggregates) + if (allow_aggregates) { GetAggregatesVisitor::Data data; GetAggregatesVisitor(data).visit(query); @@ -866,7 +866,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(ASTPtr & query, const NamesAndTy assertNoAggregates(arg, "inside another aggregate function"); result.aggregates = data.aggregates; } - else + else assertNoAggregates(query, "in wrong place"); result.collectUsedColumns(query); diff --git a/src/Parsers/ASTTTLElement.cpp b/src/Parsers/ASTTTLElement.cpp index 35e9dc89a0d..991274e8350 100644 --- a/src/Parsers/ASTTTLElement.cpp +++ b/src/Parsers/ASTTTLElement.cpp @@ -7,13 +7,13 @@ namespace DB { -ASTPtr ASTTTLElement::clone() const +ASTPtr ASTTTLElement::clone() const { auto clone = std::make_shared(*this); clone->children.clear(); clone->ttl_expr_pos = -1; clone->where_expr_pos = -1; - + clone->setExpression(clone->ttl_expr_pos, getExpression(ttl_expr_pos, true)); clone->setExpression(clone->where_expr_pos, getExpression(where_expr_pos, true)); @@ -57,14 +57,14 @@ void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & st /// It would be better to output "DELETE" here but that will break compatibility with earlier versions. } - if (where()) + if (where()) { settings.ostr << " WHERE "; where()->formatImpl(settings, state, frame); } } -void ASTTTLElement::setExpression(int & pos, ASTPtr && ast) +void ASTTTLElement::setExpression(int & pos, ASTPtr && ast) { if (ast) { @@ -83,7 +83,7 @@ void ASTTTLElement::setExpression(int & pos, ASTPtr && ast) } } -ASTPtr ASTTTLElement::getExpression(int pos, bool clone) const +ASTPtr ASTTTLElement::getExpression(int pos, bool clone) const { return pos != -1 ? (clone ? children[pos]->clone() : children[pos]) : ASTPtr{}; } diff --git a/src/Parsers/ASTTTLElement.h b/src/Parsers/ASTTTLElement.h index 24b02860f32..9b40adf2be4 100644 --- a/src/Parsers/ASTTTLElement.h +++ b/src/Parsers/ASTTTLElement.h @@ -21,7 +21,7 @@ public: std::vector> group_by_aggregations; ASTTTLElement(TTLMode mode_, PartDestinationType destination_type_, const String & destination_name_) - : mode(mode_) + : mode(mode_) , destination_type(destination_type_) , destination_name(destination_name_) , ttl_expr_pos(-1) @@ -42,12 +42,12 @@ public: protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; -private: +private: int ttl_expr_pos; int where_expr_pos; private: - void setExpression(int & pos, ASTPtr && ast); + void setExpression(int & pos, ASTPtr && ast); ASTPtr getExpression(int pos, bool clone = false) const; }; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index afd3814d913..65ff9e1da94 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1484,7 +1484,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) mode = TTLMode::MOVE; destination_type = PartDestinationType::VOLUME; } - else if (s_group_by.ignore(pos)) + else if (s_group_by.ignore(pos)) { mode = TTLMode::GROUP_BY; } @@ -1517,7 +1517,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!tryGetIdentifierNameInto(identifier, identifier_str)) return false; group_by_key_columns.emplace_back(std::move(identifier_str)); - } + } if (!s_set.ignore(pos)) return false; @@ -1525,7 +1525,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (!group_by_aggregations.empty() && !s_comma.ignore(pos)) break; - + ASTPtr name; ASTPtr value; if (!parser_identifier.parse(pos, name, expected)) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1f55f5ecff5..bb9d8ce186e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -635,16 +635,16 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, auto result = create_ttl_entry(ttl_element->ttl()); result.mode = ttl_element->mode; - if (ttl_element->mode == TTLMode::DELETE) + if (ttl_element->mode == TTLMode::DELETE) { - if (ASTPtr where_expr_ast = ttl_element->where()) + if (ASTPtr where_expr_ast = ttl_element->where()) { auto where_syntax_result = SyntaxAnalyzer(global_context).analyze(where_expr_ast, new_columns.getAllPhysical()); result.where_expression = ExpressionAnalyzer(where_expr_ast, where_syntax_result, global_context).getActions(false); result.where_result_column = where_expr_ast->getColumnName(); } } - else if (ttl_element->mode == TTLMode::GROUP_BY) + else if (ttl_element->mode == TTLMode::GROUP_BY) { if (ttl_element->group_by_key_columns.size() > this->primary_key_columns.size()) throw Exception("TTL Expression GROUP BY key should be a prefix of primary key", ErrorCodes::BAD_TTL_EXPRESSION); diff --git a/src/Storages/MergeTree/TTLMode.h b/src/Storages/MergeTree/TTLMode.h index 99a57f46853..0681f10fc17 100644 --- a/src/Storages/MergeTree/TTLMode.h +++ b/src/Storages/MergeTree/TTLMode.h @@ -4,7 +4,7 @@ namespace DB { -enum class TTLMode +enum class TTLMode { DELETE, MOVE, From a991dcf9f4f7d5552b9406548e414432fd237c41 Mon Sep 17 00:00:00 2001 From: Nikolai Sorokin Date: Thu, 14 May 2020 13:48:45 +0300 Subject: [PATCH 40/79] Add tests --- .../01246_ttl_where_group_by.reference | 15 +++++++ .../0_stateless/01246_ttl_where_group_by.sql | 41 +++++++++++++++++++ 2 files changed, 56 insertions(+) create mode 100644 tests/queries/0_stateless/01246_ttl_where_group_by.reference create mode 100644 tests/queries/0_stateless/01246_ttl_where_group_by.sql diff --git a/tests/queries/0_stateless/01246_ttl_where_group_by.reference b/tests/queries/0_stateless/01246_ttl_where_group_by.reference new file mode 100644 index 00000000000..7e276e2681e --- /dev/null +++ b/tests/queries/0_stateless/01246_ttl_where_group_by.reference @@ -0,0 +1,15 @@ +1 1 0 4 +1 2 3 7 +1 3 0 5 +2 1 20 1 +2 1 0 1 +1 1 0 4 +1 1 10 6 +1 2 3 7 +1 3 0 5 +2 1 10 1 +3 1 0 8 +1 1 0 4 +1 3 10 6 +2 1 20 1 +3 1 0 8 diff --git a/tests/queries/0_stateless/01246_ttl_where_group_by.sql b/tests/queries/0_stateless/01246_ttl_where_group_by.sql new file mode 100644 index 00000000000..fbbd0336e1b --- /dev/null +++ b/tests/queries/0_stateless/01246_ttl_where_group_by.sql @@ -0,0 +1,41 @@ +drop table if exists ttl_01246_1; + +create table ttl_01246_1 (a Int, b Int, x Int, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second delete where x % 10 == 0 and y > 5; +insert into ttl_01246_1 values (1, 1, 0, 4, now() + 10); +insert into ttl_01246_1 values (1, 1, 10, 6, now()); +insert into ttl_01246_1 values (1, 2, 3, 7, now()); +insert into ttl_01246_1 values (1, 3, 0, 5, now()); +insert into ttl_01246_1 values (2, 1, 20, 1, now()); +insert into ttl_01246_1 values (2, 1, 0, 1, now()); +insert into ttl_01246_1 values (3, 1, 0, 8, now()); +select sleep(1.1) format Null; +optimize table ttl_01246_1 final; +select a, b, x, y from ttl_01246_1; + +drop table if exists ttl_01246_1; + +create table ttl_01246_1 (a Int, b Int, x Int32, y Double, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a, b set x = cast(median(x) as Int32), y = avg(y), d = max(d); +insert into ttl_01246_1 values (1, 1, 0, 4, now() + 10); +insert into ttl_01246_1 values (1, 1, 10, 6, now()); +insert into ttl_01246_1 values (1, 2, 3, 7, now()); +insert into ttl_01246_1 values (1, 3, 0, 5, now()); +insert into ttl_01246_1 values (2, 1, 20, 1, now()); +insert into ttl_01246_1 values (2, 1, 0, 1, now()); +insert into ttl_01246_1 values (3, 1, 0, 8, now()); +select sleep(1.1) format Null; +optimize table ttl_01246_1 final; +select a, b, x, y from ttl_01246_1; + +drop table if exists ttl_01246_1; + +create table ttl_01246_1 (a Int, b Int, x Int32, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a set x = max(x), y = cast(round(avg(y)) as Int), d = max(d); +insert into ttl_01246_1 values (1, 1, 0, 4, now() + 10); +insert into ttl_01246_1 values (1, 1, 10, 6, now()); +insert into ttl_01246_1 values (1, 2, 3, 7, now()); +insert into ttl_01246_1 values (1, 3, 0, 5, now()); +insert into ttl_01246_1 values (2, 1, 20, 1, now()); +insert into ttl_01246_1 values (2, 1, 0, 1, now()); +insert into ttl_01246_1 values (3, 1, 0, 8, now()); +select sleep(1.1) format Null; +optimize table ttl_01246_1 final; +select a, b, x, y from ttl_01246_1; \ No newline at end of file From 9269db072af72d5f8c5dbf6483d21b50bdad4ac9 Mon Sep 17 00:00:00 2001 From: Nikolai Sorokin Date: Thu, 14 May 2020 17:07:36 +0300 Subject: [PATCH 41/79] Better tests --- .../0_stateless/01246_ttl_where_group_by.reference | 11 +++++------ .../queries/0_stateless/01246_ttl_where_group_by.sql | 12 ++++++++---- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/01246_ttl_where_group_by.reference b/tests/queries/0_stateless/01246_ttl_where_group_by.reference index 7e276e2681e..69d3d77cd46 100644 --- a/tests/queries/0_stateless/01246_ttl_where_group_by.reference +++ b/tests/queries/0_stateless/01246_ttl_where_group_by.reference @@ -4,12 +4,11 @@ 2 1 20 1 2 1 0 1 1 1 0 4 -1 1 10 6 -1 2 3 7 -1 3 0 5 -2 1 10 1 +1 1 6 6.5 +1 3 4 5.666666666666667 +2 1 10 4 3 1 0 8 1 1 0 4 1 3 10 6 -2 1 20 1 -3 1 0 8 +2 1 20 2 +3 5 8 4 diff --git a/tests/queries/0_stateless/01246_ttl_where_group_by.sql b/tests/queries/0_stateless/01246_ttl_where_group_by.sql index fbbd0336e1b..ad969127eb1 100644 --- a/tests/queries/0_stateless/01246_ttl_where_group_by.sql +++ b/tests/queries/0_stateless/01246_ttl_where_group_by.sql @@ -17,9 +17,11 @@ drop table if exists ttl_01246_1; create table ttl_01246_1 (a Int, b Int, x Int32, y Double, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a, b set x = cast(median(x) as Int32), y = avg(y), d = max(d); insert into ttl_01246_1 values (1, 1, 0, 4, now() + 10); insert into ttl_01246_1 values (1, 1, 10, 6, now()); -insert into ttl_01246_1 values (1, 2, 3, 7, now()); +insert into ttl_01246_1 values (1, 1, 3, 7, now()); insert into ttl_01246_1 values (1, 3, 0, 5, now()); -insert into ttl_01246_1 values (2, 1, 20, 1, now()); +insert into ttl_01246_1 values (1, 3, 4, 9, now()); +insert into ttl_01246_1 values (1, 3, 6, 3, now()); +insert into ttl_01246_1 values (2, 1, 20, 7, now()); insert into ttl_01246_1 values (2, 1, 0, 1, now()); insert into ttl_01246_1 values (3, 1, 0, 8, now()); select sleep(1.1) format Null; @@ -34,8 +36,10 @@ insert into ttl_01246_1 values (1, 1, 10, 6, now()); insert into ttl_01246_1 values (1, 2, 3, 7, now()); insert into ttl_01246_1 values (1, 3, 0, 5, now()); insert into ttl_01246_1 values (2, 1, 20, 1, now()); -insert into ttl_01246_1 values (2, 1, 0, 1, now()); -insert into ttl_01246_1 values (3, 1, 0, 8, now()); +insert into ttl_01246_1 values (2, 1, 0, 3, now()); +insert into ttl_01246_1 values (3, 1, 0, 3, now()); +insert into ttl_01246_1 values (3, 2, 8, 2, now()); +insert into ttl_01246_1 values (3, 5, 5, 8, now()); select sleep(1.1) format Null; optimize table ttl_01246_1 final; select a, b, x, y from ttl_01246_1; \ No newline at end of file From 1f02ba9692b7152dc82c3ce8f0e6f62ec5aa120f Mon Sep 17 00:00:00 2001 From: Nikolai Sorokin Date: Thu, 14 May 2020 21:46:15 +0300 Subject: [PATCH 42/79] Add eof --- src/Parsers/ExpressionListParsers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index ba939d8d160..9cc36967bd7 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -430,4 +430,4 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; -} \ No newline at end of file +} From c6541b1dad1fe95644a9dc1c916dbd162be5d37e Mon Sep 17 00:00:00 2001 From: Nikolai Sorokin Date: Fri, 15 May 2020 02:32:45 +0300 Subject: [PATCH 43/79] Up test id; Fix clang build --- src/Interpreters/SyntaxAnalyzer.cpp | 4 ++-- src/Parsers/ExpressionElementParsers.cpp | 2 +- src/Parsers/ExpressionListParsers.cpp | 2 +- ..._group_by.reference => 01280_ttl_where_group_by.reference} | 0 ...46_ttl_where_group_by.sql => 01280_ttl_where_group_by.sql} | 0 5 files changed, 4 insertions(+), 4 deletions(-) rename tests/queries/0_stateless/{01246_ttl_where_group_by.reference => 01280_ttl_where_group_by.reference} (100%) rename tests/queries/0_stateless/{01246_ttl_where_group_by.sql => 01280_ttl_where_group_by.sql} (100%) diff --git a/src/Interpreters/SyntaxAnalyzer.cpp b/src/Interpreters/SyntaxAnalyzer.cpp index 54f369c9495..b5f86b87fdc 100644 --- a/src/Interpreters/SyntaxAnalyzer.cpp +++ b/src/Interpreters/SyntaxAnalyzer.cpp @@ -839,7 +839,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyzeSelect( return std::make_shared(result); } -SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(ASTPtr & query, const NamesAndTypesList & source_columns, ConstStoragePtr storage, bool allow_aggregates) const +SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(ASTPtr & query, const NamesAndTypesList & source_columns, ConstStoragePtr storage, bool allow_aggregations) const { if (query->as()) throw Exception("Not select analyze for select asts.", ErrorCodes::LOGICAL_ERROR); @@ -855,7 +855,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(ASTPtr & query, const NamesAndTy optimizeIf(query, result.aliases, settings.optimize_if_chain_to_miltiif); - if (allow_aggregates) + if (allow_aggregations) { GetAggregatesVisitor::Data data; GetAggregatesVisitor(data).visit(query); diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 65ff9e1da94..5b5d9f05b22 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1511,7 +1511,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr ast_group_by_key_columns; if (!parser_identifier_list.parse(pos, ast_group_by_key_columns, expected)) return false; - for (const auto identifier : ast_group_by_key_columns->children) + for (const auto & identifier : ast_group_by_key_columns->children) { String identifier_str; if (!tryGetIdentifierNameInto(identifier, identifier_str)) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 062d1dd4483..2561bf7ef83 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -751,4 +751,4 @@ bool ParserIdentifierList::parseImpl(Pos & pos, ASTPtr & node, Expected & expect .parse(pos, node, expected); } -} \ No newline at end of file +} diff --git a/tests/queries/0_stateless/01246_ttl_where_group_by.reference b/tests/queries/0_stateless/01280_ttl_where_group_by.reference similarity index 100% rename from tests/queries/0_stateless/01246_ttl_where_group_by.reference rename to tests/queries/0_stateless/01280_ttl_where_group_by.reference diff --git a/tests/queries/0_stateless/01246_ttl_where_group_by.sql b/tests/queries/0_stateless/01280_ttl_where_group_by.sql similarity index 100% rename from tests/queries/0_stateless/01246_ttl_where_group_by.sql rename to tests/queries/0_stateless/01280_ttl_where_group_by.sql From 6585571ee2e8d8b0a832e4b51f0e40762d096a4b Mon Sep 17 00:00:00 2001 From: Nikolai Sorokin Date: Sat, 16 May 2020 02:53:47 +0300 Subject: [PATCH 44/79] Fix bug --- src/Interpreters/ExpressionAnalyzer.cpp | 89 +++++++++++++------------ 1 file changed, 45 insertions(+), 44 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index c432b920a9f..b38af6feef9 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -192,61 +192,65 @@ void ExpressionAnalyzer::analyzeAggregation() if (has_aggregation) { - // getSelectQuery(); /// assertSelect() /// Find out aggregation keys. - if (select_query && select_query->groupBy()) + if (select_query) { - NameSet unique_keys; - ASTs & group_asts = select_query->groupBy()->children; - for (ssize_t i = 0; i < ssize_t(group_asts.size()); ++i) + if (select_query->groupBy()) { - ssize_t size = group_asts.size(); - getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false); - - const auto & column_name = group_asts[i]->getColumnName(); - const auto & block = temp_actions->getSampleBlock(); - - if (!block.has(column_name)) - throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); - - const auto & col = block.getByName(column_name); - - /// Constant expressions have non-null column pointer at this stage. - if (col.column && isColumnConst(*col.column)) + NameSet unique_keys; + ASTs & group_asts = select_query->groupBy()->children; + for (ssize_t i = 0; i < ssize_t(group_asts.size()); ++i) { - /// But don't remove last key column if no aggregate functions, otherwise aggregation will not work. - if (!aggregate_descriptions.empty() || size > 1) + ssize_t size = group_asts.size(); + getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false); + + const auto & column_name = group_asts[i]->getColumnName(); + const auto & block = temp_actions->getSampleBlock(); + + if (!block.has(column_name)) + throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); + + const auto & col = block.getByName(column_name); + + /// Constant expressions have non-null column pointer at this stage. + if (col.column && isColumnConst(*col.column)) { - if (i + 1 < static_cast(size)) - group_asts[i] = std::move(group_asts.back()); + /// But don't remove last key column if no aggregate functions, otherwise aggregation will not work. + if (!aggregate_descriptions.empty() || size > 1) + { + if (i + 1 < static_cast(size)) + group_asts[i] = std::move(group_asts.back()); - group_asts.pop_back(); + group_asts.pop_back(); - --i; - continue; + --i; + continue; + } + } + + NameAndTypePair key{column_name, col.type}; + + /// Aggregation keys are uniqued. + if (!unique_keys.count(key.name)) + { + unique_keys.insert(key.name); + aggregation_keys.push_back(key); + + /// Key is no longer needed, therefore we can save a little by moving it. + aggregated_columns.push_back(std::move(key)); } } - NameAndTypePair key{column_name, col.type}; - - /// Aggregation keys are uniqued. - if (!unique_keys.count(key.name)) + if (group_asts.empty()) { - unique_keys.insert(key.name); - aggregation_keys.push_back(key); - - /// Key is no longer needed, therefore we can save a little by moving it. - aggregated_columns.push_back(std::move(key)); + select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, {}); + has_aggregation = select_query->having() || !aggregate_descriptions.empty(); } } - - if (group_asts.empty()) - { - select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, {}); - has_aggregation = select_query->having() || !aggregate_descriptions.empty(); - } } + else + aggregated_columns = temp_actions->getSampleBlock().getNamesAndTypesList(); for (const auto & desc : aggregate_descriptions) aggregated_columns.emplace_back(desc.column_name, desc.function->getReturnType()); @@ -926,10 +930,7 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result) { - NamesAndTypesList columns(sourceColumns()); - for (const auto & col : aggregated_columns) - columns.push_back(col); - ExpressionActionsPtr actions = std::make_shared(columns, context); + ExpressionActionsPtr actions = std::make_shared(aggregated_columns, context); NamesWithAliases result_columns; Names result_names; From ed09f68b8dbead260ee3506ff3533616259dca04 Mon Sep 17 00:00:00 2001 From: Nikolai Sorokin Date: Sat, 16 May 2020 19:31:29 +0300 Subject: [PATCH 45/79] Better tests --- .../01280_ttl_where_group_by.reference | 14 ++-- .../0_stateless/01280_ttl_where_group_by.sql | 76 +++++++++---------- 2 files changed, 45 insertions(+), 45 deletions(-) diff --git a/tests/queries/0_stateless/01280_ttl_where_group_by.reference b/tests/queries/0_stateless/01280_ttl_where_group_by.reference index 69d3d77cd46..572fc7731d3 100644 --- a/tests/queries/0_stateless/01280_ttl_where_group_by.reference +++ b/tests/queries/0_stateless/01280_ttl_where_group_by.reference @@ -3,12 +3,12 @@ 1 3 0 5 2 1 20 1 2 1 0 1 -1 1 0 4 -1 1 6 6.5 -1 3 4 5.666666666666667 -2 1 10 4 -3 1 0 8 +1 1 [0,2,3] 4 +1 1 [5,4,1] 13 +1 3 [1,0,1,0] 17 +2 1 [3,1,0,3] 8 +3 1 [2,4,5] 8 1 1 0 4 1 3 10 6 -2 1 20 2 -3 5 8 4 +2 1 0 3 +3 5 8 2 diff --git a/tests/queries/0_stateless/01280_ttl_where_group_by.sql b/tests/queries/0_stateless/01280_ttl_where_group_by.sql index ad969127eb1..0ca06e77001 100644 --- a/tests/queries/0_stateless/01280_ttl_where_group_by.sql +++ b/tests/queries/0_stateless/01280_ttl_where_group_by.sql @@ -1,45 +1,45 @@ -drop table if exists ttl_01246_1; +drop table if exists ttl_01280_1; -create table ttl_01246_1 (a Int, b Int, x Int, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second delete where x % 10 == 0 and y > 5; -insert into ttl_01246_1 values (1, 1, 0, 4, now() + 10); -insert into ttl_01246_1 values (1, 1, 10, 6, now()); -insert into ttl_01246_1 values (1, 2, 3, 7, now()); -insert into ttl_01246_1 values (1, 3, 0, 5, now()); -insert into ttl_01246_1 values (2, 1, 20, 1, now()); -insert into ttl_01246_1 values (2, 1, 0, 1, now()); -insert into ttl_01246_1 values (3, 1, 0, 8, now()); +create table ttl_01280_1 (a Int, b Int, x Int, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second delete where x % 10 == 0 and y > 5; +insert into ttl_01280_1 values (1, 1, 0, 4, now() + 10); +insert into ttl_01280_1 values (1, 1, 10, 6, now()); +insert into ttl_01280_1 values (1, 2, 3, 7, now()); +insert into ttl_01280_1 values (1, 3, 0, 5, now()); +insert into ttl_01280_1 values (2, 1, 20, 1, now()); +insert into ttl_01280_1 values (2, 1, 0, 1, now()); +insert into ttl_01280_1 values (3, 1, 0, 8, now()); select sleep(1.1) format Null; -optimize table ttl_01246_1 final; -select a, b, x, y from ttl_01246_1; +optimize table ttl_01280_1 final; +select a, b, x, y from ttl_01280_1; -drop table if exists ttl_01246_1; +drop table if exists ttl_01280_2; -create table ttl_01246_1 (a Int, b Int, x Int32, y Double, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a, b set x = cast(median(x) as Int32), y = avg(y), d = max(d); -insert into ttl_01246_1 values (1, 1, 0, 4, now() + 10); -insert into ttl_01246_1 values (1, 1, 10, 6, now()); -insert into ttl_01246_1 values (1, 1, 3, 7, now()); -insert into ttl_01246_1 values (1, 3, 0, 5, now()); -insert into ttl_01246_1 values (1, 3, 4, 9, now()); -insert into ttl_01246_1 values (1, 3, 6, 3, now()); -insert into ttl_01246_1 values (2, 1, 20, 7, now()); -insert into ttl_01246_1 values (2, 1, 0, 1, now()); -insert into ttl_01246_1 values (3, 1, 0, 8, now()); +create table ttl_01280_2 (a Int, b Int, x Array(Int32), y Double, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a, b set x = minForEach(x), y = sum(y), d = max(d); +insert into ttl_01280_2 values (1, 1, array(0, 2, 3), 4, now() + 10); +insert into ttl_01280_2 values (1, 1, array(5, 4, 3), 6, now()); +insert into ttl_01280_2 values (1, 1, array(5, 5, 1), 7, now()); +insert into ttl_01280_2 values (1, 3, array(3, 0, 4), 5, now()); +insert into ttl_01280_2 values (1, 3, array(1, 1, 2, 1), 9, now()); +insert into ttl_01280_2 values (1, 3, array(3, 2, 1, 0), 3, now()); +insert into ttl_01280_2 values (2, 1, array(3, 3, 3), 7, now()); +insert into ttl_01280_2 values (2, 1, array(11, 1, 0, 3), 1, now()); +insert into ttl_01280_2 values (3, 1, array(2, 4, 5), 8, now()); select sleep(1.1) format Null; -optimize table ttl_01246_1 final; -select a, b, x, y from ttl_01246_1; +optimize table ttl_01280_2 final; +select a, b, x, y from ttl_01280_2; -drop table if exists ttl_01246_1; +drop table if exists ttl_01280_3; -create table ttl_01246_1 (a Int, b Int, x Int32, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a set x = max(x), y = cast(round(avg(y)) as Int), d = max(d); -insert into ttl_01246_1 values (1, 1, 0, 4, now() + 10); -insert into ttl_01246_1 values (1, 1, 10, 6, now()); -insert into ttl_01246_1 values (1, 2, 3, 7, now()); -insert into ttl_01246_1 values (1, 3, 0, 5, now()); -insert into ttl_01246_1 values (2, 1, 20, 1, now()); -insert into ttl_01246_1 values (2, 1, 0, 3, now()); -insert into ttl_01246_1 values (3, 1, 0, 3, now()); -insert into ttl_01246_1 values (3, 2, 8, 2, now()); -insert into ttl_01246_1 values (3, 5, 5, 8, now()); -select sleep(1.1) format Null; -optimize table ttl_01246_1 final; -select a, b, x, y from ttl_01246_1; \ No newline at end of file +create table ttl_01280_3 (a Int, b Int, x Int64, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a set x = argMax(x, d), y = argMax(y, d), d = max(d); +insert into ttl_01280_3 values (1, 1, 0, 4, now() + 10); +insert into ttl_01280_3 values (1, 1, 10, 6, now() + 1); +insert into ttl_01280_3 values (1, 2, 3, 7, now()); +insert into ttl_01280_3 values (1, 3, 0, 5, now()); +insert into ttl_01280_3 values (2, 1, 20, 1, now()); +insert into ttl_01280_3 values (2, 1, 0, 3, now() + 1); +insert into ttl_01280_3 values (3, 1, 0, 3, now()); +insert into ttl_01280_3 values (3, 2, 8, 2, now() + 1); +insert into ttl_01280_3 values (3, 5, 5, 8, now()); +select sleep(2.1) format Null; +optimize table ttl_01280_3 final; +select a, b, x, y from ttl_01280_3; \ No newline at end of file From 7d937b43c7f0248069bdb7b1686244ce9d0c0f2b Mon Sep 17 00:00:00 2001 From: Nikolai Sorokin Date: Sun, 17 May 2020 16:01:38 +0300 Subject: [PATCH 46/79] Init with correct value --- src/DataStreams/TTLBlockInputStream.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/DataStreams/TTLBlockInputStream.h index a8f86477b88..c6ffa95cd75 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/DataStreams/TTLBlockInputStream.h @@ -45,7 +45,7 @@ private: AggregatedDataVariants agg_result; ColumnRawPtrs agg_key_columns; Aggregator::AggregateColumns agg_aggregate_columns; - bool agg_no_more_keys; + bool agg_no_more_keys = false; IMergeTreeDataPart::TTLInfos old_ttl_infos; IMergeTreeDataPart::TTLInfos new_ttl_infos; From 141ed887515e6523c192690baf66dd1c2ca3a3b8 Mon Sep 17 00:00:00 2001 From: Nikolai Sorokin Date: Sun, 24 May 2020 23:21:23 +0300 Subject: [PATCH 47/79] Allow functions in group by keys; Add default aggregate function; Add more tests --- src/Parsers/ASTTTLElement.cpp | 23 ++++--- src/Parsers/ASTTTLElement.h | 2 +- src/Parsers/ExpressionElementParsers.cpp | 58 ++++++++--------- src/Parsers/ExpressionListParsers.cpp | 9 --- src/Parsers/ExpressionListParsers.h | 9 --- src/Storages/MergeTree/MergeTreeData.cpp | 62 ++++++++++++++++--- .../01280_ttl_where_group_by.reference | 6 ++ .../0_stateless/01280_ttl_where_group_by.sql | 44 ++++++++++++- 8 files changed, 146 insertions(+), 67 deletions(-) diff --git a/src/Parsers/ASTTTLElement.cpp b/src/Parsers/ASTTTLElement.cpp index 991274e8350..33f8a3ac906 100644 --- a/src/Parsers/ASTTTLElement.cpp +++ b/src/Parsers/ASTTTLElement.cpp @@ -17,6 +17,8 @@ ASTPtr ASTTTLElement::clone() const clone->setExpression(clone->ttl_expr_pos, getExpression(ttl_expr_pos, true)); clone->setExpression(clone->where_expr_pos, getExpression(where_expr_pos, true)); + for (auto & expr : clone->group_by_key) + expr = expr->clone(); for (auto & [name, expr] : clone->group_by_aggregations) expr = expr->clone(); @@ -37,19 +39,22 @@ void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & st else if (mode == TTLMode::GROUP_BY) { settings.ostr << " GROUP BY "; - for (auto it = group_by_key_columns.begin(); it != group_by_key_columns.end(); ++it) + for (auto it = group_by_key.begin(); it != group_by_key.end(); ++it) { - if (it != group_by_key_columns.begin()) + if (it != group_by_key.begin()) settings.ostr << ", "; - settings.ostr << *it; + (*it)->formatImpl(settings, state, frame); } - settings.ostr << " SET "; - for (auto it = group_by_aggregations.begin(); it != group_by_aggregations.end(); ++it) + if (!group_by_aggregations.empty()) { - if (it != group_by_aggregations.begin()) - settings.ostr << ", "; - settings.ostr << it->first << " = "; - it->second->formatImpl(settings, state, frame); + settings.ostr << " SET "; + for (auto it = group_by_aggregations.begin(); it != group_by_aggregations.end(); ++it) + { + if (it != group_by_aggregations.begin()) + settings.ostr << ", "; + settings.ostr << it->first << " = "; + it->second->formatImpl(settings, state, frame); + } } } else if (mode == TTLMode::DELETE) diff --git a/src/Parsers/ASTTTLElement.h b/src/Parsers/ASTTTLElement.h index 9b40adf2be4..2004a4f927b 100644 --- a/src/Parsers/ASTTTLElement.h +++ b/src/Parsers/ASTTTLElement.h @@ -17,7 +17,7 @@ public: PartDestinationType destination_type; String destination_name; - Strings group_by_key_columns; + ASTs group_by_key; std::vector> group_by_aggregations; ASTTTLElement(TTLMode mode_, PartDestinationType destination_type_, const String & destination_name_) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 5b5d9f05b22..2ea39946ff7 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1464,7 +1464,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserIdentifier parser_identifier; ParserStringLiteral parser_string_literal; ParserExpression parser_exp; - ParserIdentifierList parser_identifier_list; + ParserExpressionList parser_expression_list(false); ASTPtr ttl_expr; if (!parser_exp.parse(pos, ttl_expr, expected)) @@ -1495,7 +1495,7 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } ASTPtr where_expr; - std::vector group_by_key_columns; + ASTPtr ast_group_by_key; std::vector> group_by_aggregations; if (mode == TTLMode::MOVE) @@ -1508,37 +1508,30 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else if (mode == TTLMode::GROUP_BY) { - ASTPtr ast_group_by_key_columns; - if (!parser_identifier_list.parse(pos, ast_group_by_key_columns, expected)) + if (!parser_expression_list.parse(pos, ast_group_by_key, expected)) return false; - for (const auto & identifier : ast_group_by_key_columns->children) + + if (s_set.ignore(pos)) { - String identifier_str; - if (!tryGetIdentifierNameInto(identifier, identifier_str)) - return false; - group_by_key_columns.emplace_back(std::move(identifier_str)); - } + while (true) + { + if (!group_by_aggregations.empty() && !s_comma.ignore(pos)) + break; - if (!s_set.ignore(pos)) - return false; - while (true) - { - if (!group_by_aggregations.empty() && !s_comma.ignore(pos)) - break; + ASTPtr name; + ASTPtr value; + if (!parser_identifier.parse(pos, name, expected)) + return false; + if (!s_eq.ignore(pos)) + return false; + if (!parser_exp.parse(pos, value, expected)) + return false; - ASTPtr name; - ASTPtr value; - if (!parser_identifier.parse(pos, name, expected)) - return false; - if (!s_eq.ignore(pos)) - return false; - if (!parser_exp.parse(pos, value, expected)) - return false; - - String name_str; - if (!tryGetIdentifierNameInto(name, name_str)) - return false; - group_by_aggregations.emplace_back(name_str, std::move(value)); + String name_str; + if (!tryGetIdentifierNameInto(name, name_str)) + return false; + group_by_aggregations.emplace_back(name_str, std::move(value)); + } } } else if (mode == TTLMode::DELETE && s_where.ignore(pos)) @@ -1552,8 +1545,11 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (where_expr) ttl_element->setWhere(std::move(where_expr)); - ttl_element->group_by_key_columns = std::move(group_by_key_columns); - ttl_element->group_by_aggregations = std::move(group_by_aggregations); + if (mode == TTLMode::GROUP_BY) + { + ttl_element->group_by_key = std::move(ast_group_by_key->children); + ttl_element->group_by_aggregations = std::move(group_by_aggregations); + } node = ttl_element; return true; diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 2561bf7ef83..a967ae19691 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -742,13 +742,4 @@ bool ParserKeyValuePairsList::parseImpl(Pos & pos, ASTPtr & node, Expected & exp return parser.parse(pos, node, expected); } - -bool ParserIdentifierList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - return ParserList( - std::make_unique(), - std::make_unique(TokenType::Comma)) - .parse(pos, node, expected); -} - } diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 9cc36967bd7..0cef29b6d67 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -421,13 +421,4 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; - -/** A comma-separated list of identifiers, probably empty. */ -class ParserIdentifierList : public IParserBase -{ -protected: - const char * getName() const override { return "list of identifiers"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bb9d8ce186e..5de3d204f16 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -646,22 +646,70 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, } else if (ttl_element->mode == TTLMode::GROUP_BY) { - if (ttl_element->group_by_key_columns.size() > this->primary_key_columns.size()) + if (ttl_element->group_by_key.size() > this->primary_key_columns.size()) throw Exception("TTL Expression GROUP BY key should be a prefix of primary key", ErrorCodes::BAD_TTL_EXPRESSION); - for (size_t i = 0; i < ttl_element->group_by_key_columns.size(); ++i) + + NameSet primary_key_columns_set(this->primary_key_columns.begin(), this->primary_key_columns.end()); + NameSet aggregation_columns_set; + + for (const auto & column : this->primary_key_expr->getRequiredColumns()) + primary_key_columns_set.insert(column); + + for (size_t i = 0; i < ttl_element->group_by_key.size(); ++i) { - if (ttl_element->group_by_key_columns[i] != this->primary_key_columns[i]) + if (ttl_element->group_by_key[i]->getColumnName() != this->primary_key_columns[i]) throw Exception("TTL Expression GROUP BY key should be a prefix of primary key", ErrorCodes::BAD_TTL_EXPRESSION); } + for (const auto & [name, value] : ttl_element->group_by_aggregations) + { + if (primary_key_columns_set.contains(name)) + throw Exception("Can not set custom aggregation for column in primary key in TTL Expression", ErrorCodes::BAD_TTL_EXPRESSION); + aggregation_columns_set.insert(name); + } + if (aggregation_columns_set.size() != ttl_element->group_by_aggregations.size()) + throw Exception("Multiple aggregations set for one column in TTL Expression", ErrorCodes::BAD_TTL_EXPRESSION); - result.group_by_keys = ttl_element->group_by_key_columns; + result.group_by_keys = Names(this->primary_key_columns.begin(), this->primary_key_columns.begin() + ttl_element->group_by_key.size()); auto aggregations = ttl_element->group_by_aggregations; - for (size_t i = ttl_element->group_by_key_columns.size(); i < this->primary_key_columns.size(); ++i) + for (size_t i = 0; i < this->primary_key_columns.size(); ++i) { - ASTPtr expr = makeASTFunction("max", std::make_shared(this->primary_key_columns[i])); - aggregations.emplace_back(this->primary_key_columns[i], std::move(expr)); + ASTPtr value = this->primary_key_expr_ast->children[i]->clone(); + + if (i >= ttl_element->group_by_key.size()) + { + ASTPtr value_max = makeASTFunction("max", value->clone()); + aggregations.emplace_back(value->getColumnName(), std::move(value_max)); + } + + if (value->as()) + { + auto syntax_result = SyntaxAnalyzer(global_context).analyze(value, new_columns.getAllPhysical(), {}, true); + auto expr_actions = ExpressionAnalyzer(value, syntax_result, global_context).getActions(false); + for (const auto & column : expr_actions->getRequiredColumns()) + { + if (i < ttl_element->group_by_key.size()) + { + ASTPtr expr = makeASTFunction("any", std::make_shared(column)); + aggregations.emplace_back(column, std::move(expr)); + } + else + { + ASTPtr expr = makeASTFunction("argMax", std::make_shared(column), value->clone()); + aggregations.emplace_back(column, std::move(expr)); + } + } + } } + for (const auto & column : new_columns.getAllPhysical()) + { + if (!primary_key_columns_set.contains(column.name) && !aggregation_columns_set.contains(column.name)) + { + ASTPtr expr = makeASTFunction("any", std::make_shared(column.name)); + aggregations.emplace_back(column.name, std::move(expr)); + } + } + for (auto [name, value] : aggregations) { auto syntax_result = SyntaxAnalyzer(global_context).analyze(value, new_columns.getAllPhysical(), {}, true); diff --git a/tests/queries/0_stateless/01280_ttl_where_group_by.reference b/tests/queries/0_stateless/01280_ttl_where_group_by.reference index 572fc7731d3..dead0a5aac3 100644 --- a/tests/queries/0_stateless/01280_ttl_where_group_by.reference +++ b/tests/queries/0_stateless/01280_ttl_where_group_by.reference @@ -12,3 +12,9 @@ 1 3 10 6 2 1 0 3 3 5 8 2 +1 1 0 4 +3 3 13 9 +1 2 7 5 +2 3 6 5 +1 2 3 5 +2 3 3 5 diff --git a/tests/queries/0_stateless/01280_ttl_where_group_by.sql b/tests/queries/0_stateless/01280_ttl_where_group_by.sql index 0ca06e77001..e61716cfe81 100644 --- a/tests/queries/0_stateless/01280_ttl_where_group_by.sql +++ b/tests/queries/0_stateless/01280_ttl_where_group_by.sql @@ -42,4 +42,46 @@ insert into ttl_01280_3 values (3, 2, 8, 2, now() + 1); insert into ttl_01280_3 values (3, 5, 5, 8, now()); select sleep(2.1) format Null; optimize table ttl_01280_3 final; -select a, b, x, y from ttl_01280_3; \ No newline at end of file +select a, b, x, y from ttl_01280_3; + +drop table if exists ttl_01280_4; + +create table ttl_01280_4 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), -(a + b)) ttl d + interval 1 second group by toDate(d) set x = sum(x), y = max(y); +insert into ttl_01280_4 values (1, 1, 0, 4, now() + 10); +insert into ttl_01280_4 values (10, 2, 3, 3, now()); +insert into ttl_01280_4 values (2, 10, 1, 7, now()); +insert into ttl_01280_4 values (3, 3, 5, 2, now()); +insert into ttl_01280_4 values (1, 5, 4, 9, now()); +select sleep(1.1) format Null; +optimize table ttl_01280_4 final; +select a, b, x, y from ttl_01280_4; + +drop table if exists ttl_01280_5; + +create table ttl_01280_5 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), a, -b) ttl d + interval 1 second group by toDate(d), a set x = sum(x); +insert into ttl_01280_5 values (1, 2, 3, 5, now()); +insert into ttl_01280_5 values (2, 10, 1, 5, now()); +insert into ttl_01280_5 values (2, 3, 5, 5, now()); +insert into ttl_01280_5 values (1, 5, 4, 5, now()); +select sleep(1.1) format Null; +optimize table ttl_01280_5 final; +select a, b, x, y from ttl_01280_5; + +drop table if exists ttl_01280_6; + +create table ttl_01280_6 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), a, -b) ttl d + interval 1 second group by toDate(d), a; +insert into ttl_01280_6 values (1, 2, 3, 5, now()); +insert into ttl_01280_6 values (2, 10, 3, 5, now()); +insert into ttl_01280_6 values (2, 3, 3, 5, now()); +insert into ttl_01280_6 values (1, 5, 3, 5, now()); +select sleep(1.1) format Null; +optimize table ttl_01280_6 final; +select a, b, x, y from ttl_01280_6; + +create table ttl_01280_error (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by x set y = max(y); -- { serverError 450} +create table ttl_01280_error (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by b set y = max(y); -- { serverError 450} +create table ttl_01280_error (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a, b, x set y = max(y); -- { serverError 450} +create table ttl_01280_error (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a set b = min(b), y = max(y); -- { serverError 450} +create table ttl_01280_error (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a, b set y = max(y), y = max(y); -- { serverError 450} +create table ttl_01280_error (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), a) ttl d + interval 1 second group by toDate(d), a set d = min(d), b = max(b); -- { serverError 450} +create table ttl_01280_error (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (d, -(a + b)) ttl d + interval 1 second group by d, -(a + b) set a = sum(a), b = min(b); -- { serverError 450} From e182d4df2ef95c59527bb06b162d92c9af3fd341 Mon Sep 17 00:00:00 2001 From: Nikolai Sorokin Date: Mon, 25 May 2020 01:52:18 +0300 Subject: [PATCH 48/79] Fix context usage --- src/DataStreams/TTLBlockInputStream.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 106ae1fd5a0..f11f6ea38cd 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -87,11 +88,9 @@ TTLBlockInputStream::TTLBlockInputStream( const Settings & settings = storage.global_context.getSettingsRef(); - bool allow_to_use_two_level_group_by = false; // settings.max_bytes_before_external_group_by != 0; Aggregator::Params params(header, keys, aggregates, false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, - allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), - allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), + SettingUInt64(0), SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, storage.global_context.getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); aggregator = std::make_unique(params); From f62adfd588b579e51f76ac78fe19e66df250ff8b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 27 May 2020 04:52:02 +0300 Subject: [PATCH 49/79] Fix Arcadia build --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 5de3d204f16..9a1b4f27872 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -662,7 +662,7 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, } for (const auto & [name, value] : ttl_element->group_by_aggregations) { - if (primary_key_columns_set.contains(name)) + if (primary_key_columns_set.count(name)) throw Exception("Can not set custom aggregation for column in primary key in TTL Expression", ErrorCodes::BAD_TTL_EXPRESSION); aggregation_columns_set.insert(name); } @@ -703,7 +703,7 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, } for (const auto & column : new_columns.getAllPhysical()) { - if (!primary_key_columns_set.contains(column.name) && !aggregation_columns_set.contains(column.name)) + if (!primary_key_columns_set.count(column.name) && !aggregation_columns_set.count(column.name)) { ASTPtr expr = makeASTFunction("any", std::make_shared(column.name)); aggregations.emplace_back(column.name, std::move(expr)); From 146370934a295759f342c310ab74eb660d675423 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 27 May 2020 13:07:38 +0300 Subject: [PATCH 50/79] Keep the value of DistributedFilesToInsert metric on exceptions --- src/Common/CurrentMetrics.h | 6 ++++ src/Storages/Distributed/DirectoryMonitor.cpp | 32 +++++++++++++------ src/Storages/Distributed/DirectoryMonitor.h | 8 +++-- 3 files changed, 34 insertions(+), 12 deletions(-) diff --git a/src/Common/CurrentMetrics.h b/src/Common/CurrentMetrics.h index b87504ef49a..a3bac96a16c 100644 --- a/src/Common/CurrentMetrics.h +++ b/src/Common/CurrentMetrics.h @@ -94,6 +94,12 @@ namespace CurrentMetrics amount = new_amount; } + void sub(Value value = 1) + { + what->fetch_sub(value, std::memory_order_relaxed); + amount -= value; + } + /// Subtract value before destructor. void destroy() { diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index fdcbe8e69f9..4dd62db0965 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -110,8 +110,9 @@ void StorageDistributedDirectoryMonitor::flushAllData() { if (!quit) { + CurrentMetrics::Increment metric_pending_files{CurrentMetrics::DistributedFilesToInsert, 0}; std::unique_lock lock{mutex}; - processFiles(); + processFiles(metric_pending_files); } } @@ -131,6 +132,9 @@ void StorageDistributedDirectoryMonitor::run() { std::unique_lock lock{mutex}; + /// This metric will be updated with the number of pending files later. + CurrentMetrics::Increment metric_pending_files{CurrentMetrics::DistributedFilesToInsert, 0}; + bool do_sleep = false; while (!quit) { @@ -139,7 +143,7 @@ void StorageDistributedDirectoryMonitor::run() { try { - do_sleep = !processFiles(); + do_sleep = !processFiles(metric_pending_files); } catch (...) { @@ -222,7 +226,7 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri } -bool StorageDistributedDirectoryMonitor::processFiles() +bool StorageDistributedDirectoryMonitor::processFiles(CurrentMetrics::Increment & metric_pending_files) { std::map files; @@ -236,14 +240,16 @@ bool StorageDistributedDirectoryMonitor::processFiles() files[parse(file_path.getBaseName())] = file_path_str; } + /// Note: the value of this metric will be kept if this function will throw an exception. + /// This is needed, because in case of exception, files still pending. + metric_pending_files.changeTo(files.size()); + if (files.empty()) return false; - CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedFilesToInsert, CurrentMetrics::Value(files.size())}; - if (should_batch_inserts) { - processFilesWithBatching(files); + processFilesWithBatching(files, metric_pending_files); } else { @@ -252,14 +258,14 @@ bool StorageDistributedDirectoryMonitor::processFiles() if (quit) return true; - processFile(file.second); + processFile(file.second, metric_pending_files); } } return true; } -void StorageDistributedDirectoryMonitor::processFile(const std::string & file_path) +void StorageDistributedDirectoryMonitor::processFile(const std::string & file_path, CurrentMetrics::Increment & metric_pending_files) { LOG_TRACE(log, "Started processing `{}`", file_path); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(storage.global_context->getSettingsRef()); @@ -289,6 +295,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa } Poco::File{file_path}.remove(); + metric_pending_files.sub(); LOG_TRACE(log, "Finished processing `{}`", file_path); } @@ -584,7 +591,9 @@ bool StorageDistributedDirectoryMonitor::scheduleAfter(size_t ms) return task_handle->scheduleAfter(ms, false); } -void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map & files) +void StorageDistributedDirectoryMonitor::processFilesWithBatching( + const std::map & files, + CurrentMetrics::Increment & metric_pending_files) { std::unordered_set file_indices_to_skip; @@ -596,6 +605,7 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map batch.readText(in); file_indices_to_skip.insert(batch.file_indices.begin(), batch.file_indices.end()); batch.send(); + metric_pending_files.sub(batch.file_indices.size()); } std::unordered_map header_to_batch; @@ -656,13 +666,17 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map batch.total_bytes += total_bytes; if (batch.isEnoughSize()) + { batch.send(); + metric_pending_files.sub(batch.file_indices.size()); + } } for (auto & kv : header_to_batch) { Batch & batch = kv.second; batch.send(); + metric_pending_files.sub(batch.file_indices.size()); } /// current_batch.txt will not exist if there was no send diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DirectoryMonitor.h index 77abf35630c..e2a913ee1ef 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DirectoryMonitor.h @@ -9,6 +9,8 @@ #include +namespace CurrentMetrics { class Increment; } + namespace DB { @@ -37,9 +39,9 @@ public: bool scheduleAfter(size_t ms); private: void run(); - bool processFiles(); - void processFile(const std::string & file_path); - void processFilesWithBatching(const std::map & files); + bool processFiles(CurrentMetrics::Increment & metric_pending_files); + void processFile(const std::string & file_path, CurrentMetrics::Increment & metric_pending_files); + void processFilesWithBatching(const std::map & files, CurrentMetrics::Increment & metric_pending_files); static bool isFileBrokenErrorCode(int code); void markAsBroken(const std::string & file_path) const; From 57555dbabf11780ca751e8f89a8e8f3787294d90 Mon Sep 17 00:00:00 2001 From: Nikolai Sorokin Date: Wed, 27 May 2020 14:00:17 +0300 Subject: [PATCH 51/79] Fix after rebase --- src/Storages/MergeTree/MergeTreeData.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9a1b4f27872..da351fc7750 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -646,18 +646,18 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, } else if (ttl_element->mode == TTLMode::GROUP_BY) { - if (ttl_element->group_by_key.size() > this->primary_key_columns.size()) + if (ttl_element->group_by_key.size() > this->getPrimaryKey().column_names.size()) throw Exception("TTL Expression GROUP BY key should be a prefix of primary key", ErrorCodes::BAD_TTL_EXPRESSION); - NameSet primary_key_columns_set(this->primary_key_columns.begin(), this->primary_key_columns.end()); + NameSet primary_key_columns_set(this->getPrimaryKey().column_names.begin(), this->getPrimaryKey().column_names.end()); NameSet aggregation_columns_set; - for (const auto & column : this->primary_key_expr->getRequiredColumns()) + for (const auto & column : this->getPrimaryKey().expression->getRequiredColumns()) primary_key_columns_set.insert(column); for (size_t i = 0; i < ttl_element->group_by_key.size(); ++i) { - if (ttl_element->group_by_key[i]->getColumnName() != this->primary_key_columns[i]) + if (ttl_element->group_by_key[i]->getColumnName() != this->getPrimaryKey().column_names[i]) throw Exception("TTL Expression GROUP BY key should be a prefix of primary key", ErrorCodes::BAD_TTL_EXPRESSION); } for (const auto & [name, value] : ttl_element->group_by_aggregations) @@ -669,12 +669,12 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, if (aggregation_columns_set.size() != ttl_element->group_by_aggregations.size()) throw Exception("Multiple aggregations set for one column in TTL Expression", ErrorCodes::BAD_TTL_EXPRESSION); - result.group_by_keys = Names(this->primary_key_columns.begin(), this->primary_key_columns.begin() + ttl_element->group_by_key.size()); + result.group_by_keys = Names(this->getPrimaryKey().column_names.begin(), this->getPrimaryKey().column_names.begin() + ttl_element->group_by_key.size()); auto aggregations = ttl_element->group_by_aggregations; - for (size_t i = 0; i < this->primary_key_columns.size(); ++i) + for (size_t i = 0; i < this->getPrimaryKey().column_names.size(); ++i) { - ASTPtr value = this->primary_key_expr_ast->children[i]->clone(); + ASTPtr value = this->getPrimaryKey().expression_list_ast->children[i]->clone(); if (i >= ttl_element->group_by_key.size()) { From 8bfe21f10194ae05f4f1fa74409560622f5c0460 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 27 May 2020 14:09:12 +0300 Subject: [PATCH 52/79] Update 01281_group_by_limit_memory_tracking.sh --- .../queries/0_stateless/01281_group_by_limit_memory_tracking.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh index 79a731b9273..5922b8d74d2 100755 --- a/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh +++ b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh @@ -27,6 +27,7 @@ function execute_group_by() execute_null "${opts[@]}" <<<'SELECT uniq(number) FROM numbers_mt(toUInt64(1e6)) GROUP BY number % 5e5' } +# This is needed to keep at least one running query for user for the time of test. execute_null <<<'SELECT sleep(3)' & execute_group_by # if memory accounting will be incorrect, the second query will be failed with MEMORY_LIMIT_EXCEEDED From d24286d5da2ce890872de6f0837b076032673339 Mon Sep 17 00:00:00 2001 From: Albert Kidrachev Date: Wed, 27 May 2020 14:52:45 +0300 Subject: [PATCH 53/79] fix --- src/Common/SharedBlockRowRef.h | 29 ------- src/Interpreters/InterpreterSelectQuery.cpp | 3 +- .../OptimizedPartialSortingTransform.cpp | 83 ------------------- .../OptimizedPartialSortingTransform.h | 34 -------- .../Transforms/PartialSortingTransform.cpp | 66 +++++++++++++++ .../Transforms/PartialSortingTransform.h | 2 + 6 files changed, 69 insertions(+), 148 deletions(-) delete mode 100644 src/Processors/Transforms/OptimizedPartialSortingTransform.cpp delete mode 100644 src/Processors/Transforms/OptimizedPartialSortingTransform.h diff --git a/src/Common/SharedBlockRowRef.h b/src/Common/SharedBlockRowRef.h index e9fd076da07..957d66243dd 100644 --- a/src/Common/SharedBlockRowRef.h +++ b/src/Common/SharedBlockRowRef.h @@ -87,33 +87,4 @@ struct SharedBlockRowRef } }; -struct SharedBlockRowWithSortDescriptionRef : SharedBlockRowRef -{ - SortDescription * description = nullptr; - - void set(SharedBlockPtr & shared_block_, ColumnRawPtrs * columns_, size_t row_num_) = delete; - - bool operator< (const SharedBlockRowRef & other) const - { - size_t size = columns->size(); - for (size_t i = 0; i < size; ++i) - { - int res = (*description)[i].direction * (*columns)[i]->compareAt(row_num, other.row_num, *(*other.columns)[i], 1); - if (res < 0) - return true; - else if (res > 0) - return false; - } - return false; - } - - void set(SharedBlockPtr & shared_block_, ColumnRawPtrs * columns_, size_t row_num_, SortDescription * description_) - { - shared_block = shared_block_; - columns = columns_; - row_num = row_num_; - description = description_; - } -}; - } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c3fce45ac61..c8d842dfa53 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -84,7 +84,6 @@ #include #include #include -#include #include #include #include @@ -2109,7 +2108,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting if (stream_type != QueryPipeline::StreamType::Main) return nullptr; - return std::make_shared(header, output_order_descr, limit); + return std::make_shared(header, output_order_descr, limit); }); pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr diff --git a/src/Processors/Transforms/OptimizedPartialSortingTransform.cpp b/src/Processors/Transforms/OptimizedPartialSortingTransform.cpp deleted file mode 100644 index e753052a2b3..00000000000 --- a/src/Processors/Transforms/OptimizedPartialSortingTransform.cpp +++ /dev/null @@ -1,83 +0,0 @@ -#include -#include -#include - -namespace DB -{ - -OptimizedPartialSortingTransform::OptimizedPartialSortingTransform( - const Block & header_, SortDescription & description_, UInt64 limit_) - : ISimpleTransform(header_, header_, false) - , description(description_), limit(limit_) - , threshold_shared_block(nullptr) -{ -} - -static ColumnRawPtrs extractColumns(const Block & block, const SortDescription& description) -{ - size_t size = description.size(); - ColumnRawPtrs res; - res.reserve(size); - - for (size_t i = 0; i < size; ++i) - { - const IColumn * column = !description[i].column_name.empty() - ? block.getByName(description[i].column_name).column.get() - : block.safeGetByPosition(description[i].column_number).column.get(); - res.emplace_back(column); - } - - return res; -} - -void OptimizedPartialSortingTransform::transform(Chunk & chunk) -{ - if (read_rows) - read_rows->add(chunk.getNumRows()); - - auto block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); - chunk.clear(); - - SharedBlockPtr shared_block = new detail::SharedBlock(std::move(block)); - UInt64 rows_num = shared_block->rows(); - - - if (threshold_shared_block) { - SharedBlockRowWithSortDescriptionRef row; - IColumn::Filter filter(rows_num); - ColumnRawPtrs shared_block_columns = extractColumns(*shared_block, description); - size_t filtered_count = 0; - - for (UInt64 i = 0; i < rows_num; ++i) { - row.set(shared_block, &shared_block_columns, i, &description); - - if (threshold_row < row) - { - ++filtered_count; - filter[i] = 1; - } - } - - if (filtered_count) - { - for (auto & column : shared_block->getColumns()) - { - column = column->filter(filter, filtered_count); - } - } - } - - sortBlock(*shared_block, description, limit); - - if (!threshold_shared_block && limit && limit < rows_num) - { - Block threshold_block = shared_block->cloneWithColumns(shared_block->getColumns()); - threshold_shared_block = new detail::SharedBlock(std::move(threshold_block)); - threshold_block_columns = extractColumns(*threshold_shared_block, description); - threshold_row.set(threshold_shared_block, &threshold_block_columns, limit - 1, &description); - } - - chunk.setColumns(shared_block->getColumns(), shared_block->rows()); -} - -} diff --git a/src/Processors/Transforms/OptimizedPartialSortingTransform.h b/src/Processors/Transforms/OptimizedPartialSortingTransform.h deleted file mode 100644 index 20e72bd836f..00000000000 --- a/src/Processors/Transforms/OptimizedPartialSortingTransform.h +++ /dev/null @@ -1,34 +0,0 @@ -#pragma once -#include -#include -#include -#include - - -namespace DB -{ -class OptimizedPartialSortingTransform : public ISimpleTransform -{ -public: - OptimizedPartialSortingTransform( - const Block & header_, - SortDescription & description_, - UInt64 limit_ = 0); - - String getName() const override { return "OptimizedPartialSortingTransform"; } - - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) { read_rows.swap(counter); } - -protected: - void transform(Chunk & chunk) override; - -private: - SortDescription description; - UInt64 limit; - RowsBeforeLimitCounterPtr read_rows; - SharedBlockRowWithSortDescriptionRef threshold_row; - SharedBlockPtr threshold_shared_block; - ColumnRawPtrs threshold_block_columns; -}; - -} diff --git a/src/Processors/Transforms/PartialSortingTransform.cpp b/src/Processors/Transforms/PartialSortingTransform.cpp index 018614f0165..062064c0fd6 100644 --- a/src/Processors/Transforms/PartialSortingTransform.cpp +++ b/src/Processors/Transforms/PartialSortingTransform.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { @@ -11,6 +12,38 @@ PartialSortingTransform::PartialSortingTransform( { } +static ColumnRawPtrs extractColumns(const Block & block, const SortDescription & description) +{ + size_t size = description.size(); + ColumnRawPtrs res; + res.reserve(size); + + for (size_t i = 0; i < size; ++i) + { + const IColumn * column = !description[i].column_name.empty() + ? block.getByName(description[i].column_name).column.get() + : block.safeGetByPosition(description[i].column_number).column.get(); + res.emplace_back(column); + } + + return res; +} + +bool less(const ColumnRawPtrs & lhs, UInt64 lhs_row_num, + const ColumnRawPtrs & rhs, UInt64 rhs_row_num, const SortDescription & description) +{ + size_t size = description.size(); + for (size_t i = 0; i < size; ++i) + { + int res = description[i].direction * lhs[i]->compareAt(lhs_row_num, rhs_row_num, *rhs[i], 1); + if (res < 0) + return true; + else if (res > 0) + return false; + } + return false; +} + void PartialSortingTransform::transform(Chunk & chunk) { if (read_rows) @@ -19,7 +52,40 @@ void PartialSortingTransform::transform(Chunk & chunk) auto block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); chunk.clear(); + UInt64 rows_num = block.rows(); + + if (!threshold_block_columns.empty()) + { + IColumn::Filter filter(rows_num, 0); + ColumnRawPtrs block_columns = extractColumns(block, description); + size_t filtered_count = 0; + + for (UInt64 i = 0; i < rows_num; ++i) + { + if (less(threshold_block_columns, limit - 1, block_columns, i, description)) + { + ++filtered_count; + filter[i] = 1; + } + } + + if (filtered_count) + { + for (auto & column : block.getColumns()) + { + column = column->filter(filter, filtered_count); + } + } + } + sortBlock(block, description, limit); + + if (threshold_block_columns.empty() && limit && limit < rows_num) + { + threshold_block = block.cloneWithColumns(block.getColumns()); + threshold_block_columns = extractColumns(threshold_block, description); + } + chunk.setColumns(block.getColumns(), block.rows()); } diff --git a/src/Processors/Transforms/PartialSortingTransform.h b/src/Processors/Transforms/PartialSortingTransform.h index 47ac90c6904..d6749e4dfad 100644 --- a/src/Processors/Transforms/PartialSortingTransform.h +++ b/src/Processors/Transforms/PartialSortingTransform.h @@ -29,6 +29,8 @@ private: SortDescription description; UInt64 limit; RowsBeforeLimitCounterPtr read_rows; + Block threshold_block; + ColumnRawPtrs threshold_block_columns; }; } From a0f99fde700d702f29ad2411ee53571498900e41 Mon Sep 17 00:00:00 2001 From: Albert Kidrachev Date: Wed, 27 May 2020 14:56:01 +0300 Subject: [PATCH 54/79] fix --- src/Common/SharedBlockRowRef.h | 1 - src/Processors/ya.make | 1 - 2 files changed, 2 deletions(-) diff --git a/src/Common/SharedBlockRowRef.h b/src/Common/SharedBlockRowRef.h index 957d66243dd..193f7e4dd05 100644 --- a/src/Common/SharedBlockRowRef.h +++ b/src/Common/SharedBlockRowRef.h @@ -2,7 +2,6 @@ #include #include -#include #include #include diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 5952341527b..62320f1c147 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -129,7 +129,6 @@ SRCS( Transforms/MergeSortingTransform.cpp Transforms/MergingAggregatedMemoryEfficientTransform.cpp Transforms/MergingAggregatedTransform.cpp - Transforms/OptimizedPartialSortingTransform.cpp Transforms/PartialSortingTransform.cpp Transforms/ReverseTransform.cpp Transforms/RollupTransform.cpp From 81a5af1938afbd6168500b02579b259393b8a987 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 27 May 2020 15:41:45 +0300 Subject: [PATCH 55/79] Update src/Processors/Formats/Impl/ArrowBlockInputFormat.h --- src/Processors/Formats/Impl/ArrowBlockInputFormat.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index 5ad112efde9..7599182bb3a 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -25,6 +25,7 @@ protected: Chunk generate() override; private: + // Whether to use ArrowStream format // Whether to use ArrowStream format bool stream; // This field is only used for ArrowStream format From 61f833efc11f6dfb109343e86721327bf83cd742 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 27 May 2020 15:41:52 +0300 Subject: [PATCH 56/79] Update src/Processors/Formats/Impl/ArrowBlockInputFormat.h --- src/Processors/Formats/Impl/ArrowBlockInputFormat.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index 7599182bb3a..92cced2ead8 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -31,6 +31,7 @@ private: // This field is only used for ArrowStream format std::shared_ptr stream_reader; // The following fields are used only for Arrow format + // The following fields are used only for Arrow format std::shared_ptr file_reader; int record_batch_total = 0; From f54c892b49a3f1f52135d0ba9008cf2641f19898 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 27 May 2020 15:41:59 +0300 Subject: [PATCH 57/79] Update src/Processors/Formats/Impl/ArrowBlockInputFormat.h --- src/Processors/Formats/Impl/ArrowBlockInputFormat.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index 92cced2ead8..b1ee21a51e4 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -29,6 +29,7 @@ private: // Whether to use ArrowStream format bool stream; // This field is only used for ArrowStream format + // This field is only used for ArrowStream format std::shared_ptr stream_reader; // The following fields are used only for Arrow format // The following fields are used only for Arrow format From 935524096545a1d0ed7dc7980609c2b98dddfc97 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 27 May 2020 15:43:25 +0300 Subject: [PATCH 58/79] Better path detection --- contrib/cctz-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 2c44b25b17a..0837a366f20 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -623,7 +623,7 @@ if (USE_INTERNAL_CCTZ) # libraries in linker command. To avoid this we hardcode whole-archive # library into single string. add_dependencies(cctz tzdata) - target_link_libraries(cctz INTERFACE "-Wl,--whole-archive contrib/cctz-cmake/libtzdata.a -Wl,--no-whole-archive") + target_link_libraries(cctz INTERFACE "-Wl,--whole-archive $ -Wl,--no-whole-archive") endif () else () From c1cfc68cd611bf5aa7f625185867aec18dfa02b4 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 27 May 2020 15:46:49 +0300 Subject: [PATCH 59/79] revert accidental changes --- src/Processors/Formats/Impl/ArrowBlockInputFormat.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index b1ee21a51e4..5ad112efde9 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -25,14 +25,11 @@ protected: Chunk generate() override; private: - // Whether to use ArrowStream format // Whether to use ArrowStream format bool stream; // This field is only used for ArrowStream format - // This field is only used for ArrowStream format std::shared_ptr stream_reader; // The following fields are used only for Arrow format - // The following fields are used only for Arrow format std::shared_ptr file_reader; int record_batch_total = 0; From ad936442547d69ef3d7cf39c708c0f2f78072110 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 27 May 2020 15:50:12 +0300 Subject: [PATCH 60/79] make single if --- src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index e0eb0e21567..2873a5417ea 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -30,13 +30,6 @@ Chunk ArrowBlockInputFormat::generate() { Chunk res; const Block & header = getPort().getHeader(); - - if (!stream) - { - if (record_batch_current >= record_batch_total) - return res; - } - std::vector> single_batch(1); arrow::Status read_status; @@ -48,6 +41,9 @@ Chunk ArrowBlockInputFormat::generate() } else { + if (record_batch_current >= record_batch_total) + return res; + read_status = file_reader->ReadRecordBatch(record_batch_current, &single_batch[0]); } From a727e1fb1b0a0d51caa2c578e0dfdd30dce18140 Mon Sep 17 00:00:00 2001 From: Albert Kidrachev Date: Wed, 27 May 2020 15:57:14 +0300 Subject: [PATCH 61/79] more relaxation --- src/Processors/Transforms/PartialSortingTransform.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/PartialSortingTransform.cpp b/src/Processors/Transforms/PartialSortingTransform.cpp index 062064c0fd6..ad45159f77e 100644 --- a/src/Processors/Transforms/PartialSortingTransform.cpp +++ b/src/Processors/Transforms/PartialSortingTransform.cpp @@ -52,12 +52,13 @@ void PartialSortingTransform::transform(Chunk & chunk) auto block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); chunk.clear(); + ColumnRawPtrs block_columns; UInt64 rows_num = block.rows(); if (!threshold_block_columns.empty()) { IColumn::Filter filter(rows_num, 0); - ColumnRawPtrs block_columns = extractColumns(block, description); + block_columns = extractColumns(block, description); size_t filtered_count = 0; for (UInt64 i = 0; i < rows_num; ++i) @@ -80,7 +81,8 @@ void PartialSortingTransform::transform(Chunk & chunk) sortBlock(block, description, limit); - if (threshold_block_columns.empty() && limit && limit < rows_num) + if (limit && limit < rows_num && + (threshold_block_columns.empty() || less(block_columns, limit - 1, threshold_block_columns, limit - 1, description))) { threshold_block = block.cloneWithColumns(block.getColumns()); threshold_block_columns = extractColumns(threshold_block, description); From c34f1ed6fdbfaffca28aa2d61d8b3271fea9f0a8 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 27 May 2020 17:13:37 +0300 Subject: [PATCH 62/79] DOCS-631: Settings partial_merge_join_optimizations, partial_merge_join_rows_in_right_blocks (#11130) * CLICKHOUSEDOCS-631: Settings partial_merge_join_optimizations, partial_merge_join_rows_in_right_blocks * CLICKHOUSEDOCS-631: Updated by comments. * CLICKHOUSEDOCS-631: Fixed some grammar. Co-authored-by: Sergei Shtykov --- docs/en/operations/settings/settings.md | 29 +++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 909ba7bf876..7103819e499 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -404,6 +404,35 @@ Possible values: Default value: 0. +## partial_merge_join_optimizations {#partial_merge_join_optimizations} + +Disables optimizations in partial merge join algorithm for [JOIN](../../sql-reference/statements/select/join.md) queries. + +By default, this setting enables improvements that could lead to wrong results. If you see suspicious results in your queries, disable optimizations by this setting. Optimizations can be different in different versions of the ClickHouse server. + +Possible values: + +- 0 — Optimizations disabled. +- 1 — Optimizations enabled. + +Default value: 1. + +## partial_merge_join_rows_in_right_blocks {#partial_merge_join_rows_in_right_blocks} + +Limits sizes of right-hand join data blocks in partial merge join algorithm for [JOIN](../../sql-reference/statements/select/join.md) queries. + +ClickHouse server: + +1. Splits right-hand join data into blocks with up to the specified number of rows. +2. Indexes each block with their minimum and maximum values +3. Unloads prepared blocks to disk if possible. + +Possible values: + +- Any positive integer. Recommended range of values: [1000, 100000]. + +Default value: 65536. + ## any_join_distinct_right_table_keys {#any_join_distinct_right_table_keys} Enables legacy ClickHouse server behavior in `ANY INNER|LEFT JOIN` operations. From 4fa3628097180dced023b35a4e83aef10beaa6f5 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Wed, 27 May 2020 17:25:11 +0300 Subject: [PATCH 63/79] test for probably fixed issue (#11221) --- .../01138_join_on_distributed_and_tmp.reference | 0 .../01138_join_on_distributed_and_tmp.sql | 16 ++++++++++++++++ 2 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/01138_join_on_distributed_and_tmp.reference create mode 100644 tests/queries/0_stateless/01138_join_on_distributed_and_tmp.sql diff --git a/tests/queries/0_stateless/01138_join_on_distributed_and_tmp.reference b/tests/queries/0_stateless/01138_join_on_distributed_and_tmp.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01138_join_on_distributed_and_tmp.sql b/tests/queries/0_stateless/01138_join_on_distributed_and_tmp.sql new file mode 100644 index 00000000000..67492e7c683 --- /dev/null +++ b/tests/queries/0_stateless/01138_join_on_distributed_and_tmp.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS foo_local; +DROP TABLE IF EXISTS foo_distributed; + +CREATE TABLE foo_local (bar UInt64) +ENGINE = MergeTree() +ORDER BY tuple(); + +CREATE TABLE foo_distributed AS foo_local +ENGINE = Distributed('test_cluster_two_shards_localhost', currentDatabase(), foo_local); + +CREATE TEMPORARY TABLE _tmp_baz (qux UInt64); + +SELECT * FROM foo_distributed JOIN _tmp_baz ON foo_distributed.bar = _tmp_baz.qux; + +DROP TABLE foo_local; +DROP TABLE foo_distributed; From 1658705f7c82839906b9d9b40afde08f2d37f428 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 27 May 2020 18:29:22 +0300 Subject: [PATCH 64/79] Add URL/port.cpp to ya.make (a follow-up for #11120) --- src/Functions/ya.make | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/ya.make b/src/Functions/ya.make index ddb1d27b398..0b4776853e9 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -425,6 +425,7 @@ SRCS( URL/path.cpp URL/pathFull.cpp URL/protocol.cpp + URL/port.cpp URL/queryStringAndFragment.cpp URL/queryString.cpp URL/registerFunctionsURL.cpp From 4c5324600b6c847009da083d185d97a9c425b0f6 Mon Sep 17 00:00:00 2001 From: Albert Kidrachev Date: Wed, 27 May 2020 18:38:29 +0300 Subject: [PATCH 65/79] fix tests --- src/Processors/Transforms/PartialSortingTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/PartialSortingTransform.cpp b/src/Processors/Transforms/PartialSortingTransform.cpp index ad45159f77e..f68a415117b 100644 --- a/src/Processors/Transforms/PartialSortingTransform.cpp +++ b/src/Processors/Transforms/PartialSortingTransform.cpp @@ -81,7 +81,7 @@ void PartialSortingTransform::transform(Chunk & chunk) sortBlock(block, description, limit); - if (limit && limit < rows_num && + if (limit && limit < block.rows() && (threshold_block_columns.empty() || less(block_columns, limit - 1, threshold_block_columns, limit - 1, description))) { threshold_block = block.cloneWithColumns(block.getColumns()); From 788e32822adc97ebb4538d335049c25174011b2f Mon Sep 17 00:00:00 2001 From: Michael Smitasin <46496263+michaelsmitasin@users.noreply.github.com> Date: Wed, 27 May 2020 09:29:34 -0700 Subject: [PATCH 66/79] Added LBNL (#11229) --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 8868522e977..4daadf32be6 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -41,6 +41,7 @@ toc_title: Adopters | [Integros](https://integros.com){.favicon} | Platform for video services | Analytics | — | — | [Slides in Russian, May 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup22/strategies.pdf) | | [Kodiak Data](https://www.kodiakdata.com/){.favicon} | Clouds | Main product | — | — | [Slides in Engish, April 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup13/kodiak_data.pdf) | | [Kontur](https://kontur.ru){.favicon} | Software Development | Metrics | — | — | [Talk in Russian, November 2018](https://www.youtube.com/watch?v=U4u4Bd0FtrY) | +| [Lawrence Berkeley National Laboratory](https://www.lbl.gov){.favicon} | Research | Traffic analysis | 1 server | 11.8 TiB | [Slides in English, April 2019](https://www.smitasin.com/presentations/2019-04-17_DOE-NSM.pdf) | | [LifeStreet](https://lifestreet.com/){.favicon} | Ad network | Main product | 75 servers (3 replicas) | 5.27 PiB | [Blog post in Russian, February 2017](https://habr.com/en/post/322620/) | | [Mail.ru Cloud Solutions](https://mcs.mail.ru/){.favicon} | Cloud services | Main product | — | — | [Article in Russian](https://mcs.mail.ru/help/db-create/clickhouse#) | | [MessageBird](https://www.messagebird.com){.favicon} | Telecommunications | Statistics | — | — | [Slides in English, November 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup20/messagebird.pdf) | From 7b03e36c2a6c390344ce855eac7b6dd95a6b9e08 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 27 May 2020 17:03:38 +0300 Subject: [PATCH 67/79] Improve build scripts related to protobuf and gRPC a little more. --- cmake/protobuf_generate_cpp.cmake | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/cmake/protobuf_generate_cpp.cmake b/cmake/protobuf_generate_cpp.cmake index 40ec3c8e681..7ee9e8d7c81 100644 --- a/cmake/protobuf_generate_cpp.cmake +++ b/cmake/protobuf_generate_cpp.cmake @@ -112,16 +112,21 @@ if (PROTOBUF_GENERATE_CPP_SCRIPT_MODE) set (intermediate_dir ${DIR}/intermediate) set (intermediate_output "${intermediate_dir}/${FILENAME}") - if (COMPILER_ID STREQUAL "GNU") + if (COMPILER_ID STREQUAL "Clang") + set (pragma_push "#pragma clang diagnostic push\n") + set (pragma_pop "#pragma clang diagnostic pop\n") + set (pragma_disable_warnings "#pragma clang diagnostic ignored \"-Weverything\"\n") + elseif (COMPILER_ID MATCHES "GNU") set (pragma_push "#pragma GCC diagnostic push\n") set (pragma_pop "#pragma GCC diagnostic pop\n") set (pragma_disable_warnings "#pragma GCC diagnostic ignored \"-Wall\"\n" "#pragma GCC diagnostic ignored \"-Wextra\"\n" - "#pragma GCC diagnostic ignored \"-Warray-bounds\"\n") - elseif (COMPILER_ID MATCHES "Clang") - set (pragma_push "#pragma clang diagnostic push\n") - set (pragma_pop "#pragma clang diagnostic pop\n") - set (pragma_disable_warnings "#pragma clang diagnostic ignored \"-Weverything\"\n") + "#pragma GCC diagnostic ignored \"-Warray-bounds\"\n" + "#pragma GCC diagnostic ignored \"-Wold-style-cast\"\n" + "#pragma GCC diagnostic ignored \"-Wshadow\"\n" + "#pragma GCC diagnostic ignored \"-Wsuggest-override\"\n" + "#pragma GCC diagnostic ignored \"-Wcast-qual\"\n" + "#pragma GCC diagnostic ignored \"-Wunused-parameter\"\n") endif() if (${FILENAME} MATCHES ".*\\.h") From fd64d391666a06fe737fde6be951185b5af0c74e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 27 May 2020 20:23:09 +0300 Subject: [PATCH 68/79] Fix style check for POPCNT check --- programs/main.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/main.cpp b/programs/main.cpp index b1bff1de1b0..746b624e065 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -198,7 +198,7 @@ void checkRequiredInstructionsImpl(volatile InstructionFail & fail) { uint64_t a = 0; uint64_t b = 0; - __asm__ volatile ("popcnt %1, %0" : "=r"(a) :"r"(b) : ); + __asm__ volatile ("popcnt %1, %0" : "=r"(a) :"r"(b) :); } #endif From bb0045a242e5f23908ddbaf84706fa10da6bcf0a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 27 May 2020 16:27:51 +0300 Subject: [PATCH 69/79] Add test for row policy that policy defined in users.xml affects only the user it assigned to. --- tests/integration/test_row_policy/test.py | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index a1884d059c7..71496c6dbf2 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -42,8 +42,13 @@ def started_cluster(): CREATE TABLE mydb.`.filtered_table4` (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a; INSERT INTO mydb.`.filtered_table4` values (0, 0), (0, 1), (1, 0), (1, 1); + + CREATE TABLE mydb.local (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a; ''') + node.query("INSERT INTO mydb.local values (2, 0), (2, 1), (1, 0), (1, 1)") + node2.query("INSERT INTO mydb.local values (3, 0), (3, 1), (1, 0), (1, 1)") + yield cluster finally: @@ -122,6 +127,17 @@ def test_single_table_name(): assert node.query("SELECT a + b = 1 FROM mydb.filtered_table3") == TSV([[1], [1]]) +def test_policy_from_users_xml_affects_only_user_assigned(): + assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1,0], [1, 1]]) + assert node.query("SELECT * FROM mydb.filtered_table1", user="another") == TSV([[0, 0], [0, 1], [1, 0], [1, 1]]) + + assert node.query("SELECT * FROM mydb.filtered_table2") == TSV([[0, 0, 0, 0], [0, 0, 6, 0]]) + assert node.query("SELECT * FROM mydb.filtered_table2", user="another") == TSV([[0, 0, 0, 0], [0, 0, 6, 0], [1, 2, 3, 4], [4, 3, 2, 1]]) + + assert node.query("SELECT * FROM mydb.local") == TSV([[1,0], [1, 1], [2, 0], [2, 1]]) + assert node.query("SELECT * FROM mydb.local", user="another") == TSV([[1, 0], [1, 1]]) + + def test_custom_table_name(): copy_policy_xml('multiple_tags_with_table_names.xml') assert node.query("SELECT * FROM mydb.table") == TSV([[1, 0], [1, 1]]) @@ -286,9 +302,5 @@ def test_miscellaneous_engines(): # DistributedMergeTree node.query("DROP TABLE IF EXISTS mydb.not_filtered_table") node.query("CREATE TABLE mydb.not_filtered_table (a UInt8, b UInt8) ENGINE Distributed('test_local_cluster', mydb, local)") - node.query("CREATE TABLE mydb.local (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a") - node2.query("CREATE TABLE mydb.local (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a") - node.query("INSERT INTO mydb.local values (2, 0), (2, 1), (1, 0), (1, 1)") - node2.query("INSERT INTO mydb.local values (3, 0), (3, 1), (1, 0), (1, 1)") assert node.query("SELECT * FROM mydb.not_filtered_table", user="another") == TSV([[1, 0], [1, 1], [1, 0], [1, 1]]) assert node.query("SELECT sum(a), b FROM mydb.not_filtered_table GROUP BY b ORDER BY b", user="another") == TSV([[2, 0], [2, 1]]) From 5b858de374b1de5d4c1a53d64142c70dbf4841ec Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Wed, 27 May 2020 20:52:52 +0300 Subject: [PATCH 70/79] Split programs/server into actual program and library (#11186) * Split programs/server into actual program and library --- programs/server/CMakeLists.txt | 19 ++-------------- programs/server/Server.cpp | 6 ++--- programs/server/Server.h | 2 +- programs/server/ya.make | 11 ---------- src/CMakeLists.txt | 2 ++ src/Server/CMakeLists.txt | 0 .../server => src/Server}/HTTPHandler.cpp | 0 {programs/server => src/Server}/HTTPHandler.h | 3 ++- .../Server}/HTTPHandlerFactory.cpp | 1 + .../Server}/HTTPHandlerFactory.h | 0 .../Server}/HTTPHandlerRequestFilter.h | 1 + {programs/server => src/Server}/IServer.h | 14 +++++++++--- .../Server}/InterserverIOHTTPHandler.cpp | 1 + .../Server}/InterserverIOHTTPHandler.h | 0 .../server => src/Server}/MySQLHandler.cpp | 0 .../server => src/Server}/MySQLHandler.h | 0 .../Server}/MySQLHandlerFactory.cpp | 3 +-- .../Server}/MySQLHandlerFactory.h | 3 ++- .../server => src/Server}/NotFoundHandler.cpp | 0 .../server => src/Server}/NotFoundHandler.h | 0 .../Server}/PrometheusMetricsWriter.cpp | 0 .../Server}/PrometheusMetricsWriter.h | 0 .../Server}/PrometheusRequestHandler.cpp | 1 + .../Server}/PrometheusRequestHandler.h | 0 .../Server}/ReplicasStatusHandler.cpp | 0 .../Server}/ReplicasStatusHandler.h | 0 .../Server}/StaticRequestHandler.cpp | 0 .../Server}/StaticRequestHandler.h | 2 ++ .../server => src/Server}/TCPHandler.cpp | 0 {programs/server => src/Server}/TCPHandler.h | 0 .../server => src/Server}/TCPHandlerFactory.h | 4 ++-- src/Server/ya.make | 22 +++++++++++++++++++ src/ya.make | 1 + 33 files changed, 55 insertions(+), 41 deletions(-) create mode 100644 src/Server/CMakeLists.txt rename {programs/server => src/Server}/HTTPHandler.cpp (100%) rename {programs/server => src/Server}/HTTPHandler.h (97%) rename {programs/server => src/Server}/HTTPHandlerFactory.cpp (99%) rename {programs/server => src/Server}/HTTPHandlerFactory.h (100%) rename {programs/server => src/Server}/HTTPHandlerRequestFilter.h (99%) rename {programs/server => src/Server}/IServer.h (82%) rename {programs/server => src/Server}/InterserverIOHTTPHandler.cpp (99%) rename {programs/server => src/Server}/InterserverIOHTTPHandler.h (100%) rename {programs/server => src/Server}/MySQLHandler.cpp (100%) rename {programs/server => src/Server}/MySQLHandler.h (100%) rename {programs/server => src/Server}/MySQLHandlerFactory.cpp (99%) rename {programs/server => src/Server}/MySQLHandlerFactory.h (95%) rename {programs/server => src/Server}/NotFoundHandler.cpp (100%) rename {programs/server => src/Server}/NotFoundHandler.h (100%) rename {programs/server => src/Server}/PrometheusMetricsWriter.cpp (100%) rename {programs/server => src/Server}/PrometheusMetricsWriter.h (100%) rename {programs/server => src/Server}/PrometheusRequestHandler.cpp (95%) rename {programs/server => src/Server}/PrometheusRequestHandler.h (100%) rename {programs/server => src/Server}/ReplicasStatusHandler.cpp (100%) rename {programs/server => src/Server}/ReplicasStatusHandler.h (100%) rename {programs/server => src/Server}/StaticRequestHandler.cpp (100%) rename {programs/server => src/Server}/StaticRequestHandler.h (93%) rename {programs/server => src/Server}/TCPHandler.cpp (100%) rename {programs/server => src/Server}/TCPHandler.h (100%) rename {programs/server => src/Server}/TCPHandlerFactory.h (95%) create mode 100644 src/Server/ya.make diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index 026bb0bfeb2..1563f5ac51e 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -1,21 +1,6 @@ set(CLICKHOUSE_SERVER_SOURCES - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/HTTPHandlerFactory.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/InterserverIOHTTPHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/MetricsTransmitter.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/NotFoundHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/PrometheusMetricsWriter.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/PrometheusRequestHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/ReplicasStatusHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/StaticRequestHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/Server.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/TCPHandler.cpp -) - -set(CLICKHOUSE_SERVER_SOURCES - ${CLICKHOUSE_SERVER_SOURCES} - ${CMAKE_CURRENT_SOURCE_DIR}/MySQLHandler.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/MySQLHandlerFactory.cpp + MetricsTransmitter.cpp + Server.cpp ) set (CLICKHOUSE_SERVER_LINK diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c1a520030f4..ce1d35e65d4 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -53,13 +53,13 @@ #include #include #include -#include "HTTPHandlerFactory.h" +#include #include "MetricsTransmitter.h" #include -#include "TCPHandlerFactory.h" +#include #include #include -#include "MySQLHandlerFactory.h" +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" diff --git a/programs/server/Server.h b/programs/server/Server.h index ffd89df6af4..ad9e51c881c 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -1,6 +1,6 @@ #pragma once -#include "IServer.h" +#include #include diff --git a/programs/server/ya.make b/programs/server/ya.make index 2c74c01c7cb..2e13267f715 100644 --- a/programs/server/ya.make +++ b/programs/server/ya.make @@ -11,19 +11,8 @@ PEERDIR( SRCS( clickhouse-server.cpp - HTTPHandler.cpp - HTTPHandlerFactory.cpp - InterserverIOHTTPHandler.cpp MetricsTransmitter.cpp - MySQLHandler.cpp - MySQLHandlerFactory.cpp - NotFoundHandler.cpp - PrometheusMetricsWriter.cpp - PrometheusRequestHandler.cpp - ReplicasStatusHandler.cpp - StaticRequestHandler.cpp Server.cpp - TCPHandler.cpp ) END() diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 222a3e486f9..baa0fbcb883 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -58,6 +58,7 @@ add_subdirectory (TableFunctions) add_subdirectory (Processors) add_subdirectory (Formats) add_subdirectory (Compression) +add_subdirectory (Server) set(dbms_headers) @@ -145,6 +146,7 @@ add_object_library(clickhouse_storages_distributed Storages/Distributed) add_object_library(clickhouse_storages_mergetree Storages/MergeTree) add_object_library(clickhouse_storages_liveview Storages/LiveView) add_object_library(clickhouse_client Client) +add_object_library(clickhouse_server Server) add_object_library(clickhouse_formats Formats) add_object_library(clickhouse_processors Processors) add_object_library(clickhouse_processors_executors Processors/Executors) diff --git a/src/Server/CMakeLists.txt b/src/Server/CMakeLists.txt new file mode 100644 index 00000000000..e69de29bb2d diff --git a/programs/server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp similarity index 100% rename from programs/server/HTTPHandler.cpp rename to src/Server/HTTPHandler.cpp diff --git a/programs/server/HTTPHandler.h b/src/Server/HTTPHandler.h similarity index 97% rename from programs/server/HTTPHandler.h rename to src/Server/HTTPHandler.h index 6228523d343..b1a6355d281 100644 --- a/programs/server/HTTPHandler.h +++ b/src/Server/HTTPHandler.h @@ -6,6 +6,7 @@ #include #include +#include #include @@ -21,7 +22,7 @@ namespace DB class WriteBufferFromHTTPServerResponse; -typedef std::shared_ptr CompiledRegexPtr; +using CompiledRegexPtr = std::shared_ptr; class HTTPHandler : public Poco::Net::HTTPRequestHandler { diff --git a/programs/server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp similarity index 99% rename from programs/server/HTTPHandlerFactory.cpp rename to src/Server/HTTPHandlerFactory.cpp index f302216e22b..e916070be22 100644 --- a/programs/server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include "HTTPHandler.h" #include "NotFoundHandler.h" diff --git a/programs/server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h similarity index 100% rename from programs/server/HTTPHandlerFactory.h rename to src/Server/HTTPHandlerFactory.h diff --git a/programs/server/HTTPHandlerRequestFilter.h b/src/Server/HTTPHandlerRequestFilter.h similarity index 99% rename from programs/server/HTTPHandlerRequestFilter.h rename to src/Server/HTTPHandlerRequestFilter.h index b0b748506e5..f952efd7653 100644 --- a/programs/server/HTTPHandlerRequestFilter.h +++ b/src/Server/HTTPHandlerRequestFilter.h @@ -6,6 +6,7 @@ #include #include #include +#include #include diff --git a/programs/server/IServer.h b/src/Server/IServer.h similarity index 82% rename from programs/server/IServer.h rename to src/Server/IServer.h index 29e9bc16a75..131e7443646 100644 --- a/programs/server/IServer.h +++ b/src/Server/IServer.h @@ -1,14 +1,22 @@ #pragma once -#include -#include +namespace Poco +{ -#include +namespace Util +{ +class LayeredConfiguration; +} +class Logger; + +} namespace DB { +class Context; + class IServer { public: diff --git a/programs/server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp similarity index 99% rename from programs/server/InterserverIOHTTPHandler.cpp rename to src/Server/InterserverIOHTTPHandler.cpp index 4b733c7f1fd..062721a01aa 100644 --- a/programs/server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/programs/server/InterserverIOHTTPHandler.h b/src/Server/InterserverIOHTTPHandler.h similarity index 100% rename from programs/server/InterserverIOHTTPHandler.h rename to src/Server/InterserverIOHTTPHandler.h diff --git a/programs/server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp similarity index 100% rename from programs/server/MySQLHandler.cpp rename to src/Server/MySQLHandler.cpp diff --git a/programs/server/MySQLHandler.h b/src/Server/MySQLHandler.h similarity index 100% rename from programs/server/MySQLHandler.h rename to src/Server/MySQLHandler.h diff --git a/programs/server/MySQLHandlerFactory.cpp b/src/Server/MySQLHandlerFactory.cpp similarity index 99% rename from programs/server/MySQLHandlerFactory.cpp rename to src/Server/MySQLHandlerFactory.cpp index 022167fe766..5d78ed81068 100644 --- a/programs/server/MySQLHandlerFactory.cpp +++ b/src/Server/MySQLHandlerFactory.cpp @@ -4,8 +4,7 @@ #include #include #include -#include "IServer.h" -#include "MySQLHandler.h" +#include #if USE_SSL # include diff --git a/programs/server/MySQLHandlerFactory.h b/src/Server/MySQLHandlerFactory.h similarity index 95% rename from programs/server/MySQLHandlerFactory.h rename to src/Server/MySQLHandlerFactory.h index 74f0bb35a40..df7bd794b16 100644 --- a/programs/server/MySQLHandlerFactory.h +++ b/src/Server/MySQLHandlerFactory.h @@ -2,7 +2,8 @@ #include #include -#include "IServer.h" +#include +#include #if !defined(ARCADIA_BUILD) # include diff --git a/programs/server/NotFoundHandler.cpp b/src/Server/NotFoundHandler.cpp similarity index 100% rename from programs/server/NotFoundHandler.cpp rename to src/Server/NotFoundHandler.cpp diff --git a/programs/server/NotFoundHandler.h b/src/Server/NotFoundHandler.h similarity index 100% rename from programs/server/NotFoundHandler.h rename to src/Server/NotFoundHandler.h diff --git a/programs/server/PrometheusMetricsWriter.cpp b/src/Server/PrometheusMetricsWriter.cpp similarity index 100% rename from programs/server/PrometheusMetricsWriter.cpp rename to src/Server/PrometheusMetricsWriter.cpp diff --git a/programs/server/PrometheusMetricsWriter.h b/src/Server/PrometheusMetricsWriter.h similarity index 100% rename from programs/server/PrometheusMetricsWriter.h rename to src/Server/PrometheusMetricsWriter.h diff --git a/programs/server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp similarity index 95% rename from programs/server/PrometheusRequestHandler.cpp rename to src/Server/PrometheusRequestHandler.cpp index b5a48d13b64..43f39e36de8 100644 --- a/programs/server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include diff --git a/programs/server/PrometheusRequestHandler.h b/src/Server/PrometheusRequestHandler.h similarity index 100% rename from programs/server/PrometheusRequestHandler.h rename to src/Server/PrometheusRequestHandler.h diff --git a/programs/server/ReplicasStatusHandler.cpp b/src/Server/ReplicasStatusHandler.cpp similarity index 100% rename from programs/server/ReplicasStatusHandler.cpp rename to src/Server/ReplicasStatusHandler.cpp diff --git a/programs/server/ReplicasStatusHandler.h b/src/Server/ReplicasStatusHandler.h similarity index 100% rename from programs/server/ReplicasStatusHandler.h rename to src/Server/ReplicasStatusHandler.h diff --git a/programs/server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp similarity index 100% rename from programs/server/StaticRequestHandler.cpp rename to src/Server/StaticRequestHandler.cpp diff --git a/programs/server/StaticRequestHandler.h b/src/Server/StaticRequestHandler.h similarity index 93% rename from programs/server/StaticRequestHandler.h rename to src/Server/StaticRequestHandler.h index bdbc17f5e00..707087df24d 100644 --- a/programs/server/StaticRequestHandler.h +++ b/src/Server/StaticRequestHandler.h @@ -4,6 +4,8 @@ #include #include +#include +#include namespace DB diff --git a/programs/server/TCPHandler.cpp b/src/Server/TCPHandler.cpp similarity index 100% rename from programs/server/TCPHandler.cpp rename to src/Server/TCPHandler.cpp diff --git a/programs/server/TCPHandler.h b/src/Server/TCPHandler.h similarity index 100% rename from programs/server/TCPHandler.h rename to src/Server/TCPHandler.h diff --git a/programs/server/TCPHandlerFactory.h b/src/Server/TCPHandlerFactory.h similarity index 95% rename from programs/server/TCPHandlerFactory.h rename to src/Server/TCPHandlerFactory.h index 68652540192..a5532a8dc02 100644 --- a/programs/server/TCPHandlerFactory.h +++ b/src/Server/TCPHandlerFactory.h @@ -3,8 +3,8 @@ #include #include #include -#include "IServer.h" -#include "TCPHandler.h" +#include +#include namespace Poco { class Logger; } diff --git a/src/Server/ya.make b/src/Server/ya.make new file mode 100644 index 00000000000..1d689ee73b8 --- /dev/null +++ b/src/Server/ya.make @@ -0,0 +1,22 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + contrib/libs/poco/Util +) + +SRCS( + HTTPHandler.cpp + HTTPHandlerFactory.cpp + InterserverIOHTTPHandler.cpp + MySQLHandler.cpp + MySQLHandlerFactory.cpp + NotFoundHandler.cpp + PrometheusMetricsWriter.cpp + PrometheusRequestHandler.cpp + ReplicasStatusHandler.cpp + StaticRequestHandler.cpp + TCPHandler.cpp +) + +END() diff --git a/src/ya.make b/src/ya.make index 95e5914bbe7..eedd98cb178 100644 --- a/src/ya.make +++ b/src/ya.make @@ -19,6 +19,7 @@ PEERDIR( clickhouse/src/IO clickhouse/src/Parsers clickhouse/src/Processors + clickhouse/src/Server clickhouse/src/Storages clickhouse/src/TableFunctions ) From 0456302355d6eedc1c502ba60497b80941e75d91 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 27 May 2020 20:58:05 +0300 Subject: [PATCH 71/79] Update formats.md --- docs/en/interfaces/formats.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ed411a1b0d4..a08f2db7149 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1143,13 +1143,13 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Parquet" > {some_ To exchange data with Hadoop, you can use [HDFS table engine](../engines/table-engines/integrations/hdfs.md). -## Arrow {data-format-arrow} +## Arrow {#data-format-arrow} [Apache Arrow](https://arrow.apache.org/) comes with two built-in columnar storage formats. ClickHouse supports read and write operations for these formats. `Arrow` is Apache Arrow's "file mode" format. It is designed for in-memory random access. -## ArrowStream {data-format-arrow-stream} +## ArrowStream {#data-format-arrow-stream} `ArrowStream` is Apache Arrow's "stream mode" format. It is designed for in-memory stream processing. From e93882c977ba53486e6bd06eac2d4b16cb7d3ecc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 27 May 2020 21:20:26 +0300 Subject: [PATCH 72/79] Insert select using PipelineExecutor. --- programs/server/TCPHandler.cpp | 4 +- .../NullAndDoCopyBlockInputStream.h | 24 +--- src/DataStreams/copyData.cpp | 81 ------------ src/DataStreams/copyData.h | 2 - src/Interpreters/InterpreterInsertQuery.cpp | 78 +++++++----- src/Interpreters/executeQuery.cpp | 48 ++++--- .../PullingAsyncPipelineExecutor.cpp | 51 ++++++-- .../Executors/PullingPipelineExecutor.cpp | 2 +- src/Processors/QueryPipeline.cpp | 117 +++++++++++++----- src/Processors/QueryPipeline.h | 16 ++- 10 files changed, 222 insertions(+), 201 deletions(-) diff --git a/programs/server/TCPHandler.cpp b/programs/server/TCPHandler.cpp index e64d7799dcb..21a4a389b9e 100644 --- a/programs/server/TCPHandler.cpp +++ b/programs/server/TCPHandler.cpp @@ -263,8 +263,8 @@ void TCPHandler::runImpl() else if (state.need_receive_data_for_input) { /// It is special case for input(), all works for reading data from client will be done in callbacks. - /// state.io.in is NullAndDoCopyBlockInputStream so read it once. - state.io.in->read(); + auto executor = state.io.pipeline.execute(); + executor->execute(state.io.pipeline.getNumThreads()); state.io.onFinish(); } else if (state.io.pipeline.initialized()) diff --git a/src/DataStreams/NullAndDoCopyBlockInputStream.h b/src/DataStreams/NullAndDoCopyBlockInputStream.h index 8fe05c387a3..8bfb3538f3a 100644 --- a/src/DataStreams/NullAndDoCopyBlockInputStream.h +++ b/src/DataStreams/NullAndDoCopyBlockInputStream.h @@ -21,19 +21,10 @@ class NullAndDoCopyBlockInputStream : public IBlockInputStream { public: NullAndDoCopyBlockInputStream(const BlockInputStreamPtr & input_, BlockOutputStreamPtr output_) + : input(std::move(input_)) + , output(std::move(output_)) { - input_streams.push_back(input_); - output_streams.push_back(output_); - - for (auto & input_stream : input_streams) - children.push_back(input_stream); - } - - NullAndDoCopyBlockInputStream(const BlockInputStreams & input_, BlockOutputStreams & output_) - : input_streams(input_), output_streams(output_) - { - for (auto & input_stream : input_) - children.push_back(input_stream); + children.push_back(input); } /// Suppress readPrefix and readSuffix, because they are called by copyData. @@ -53,16 +44,13 @@ protected: /// If query was cancelled, it will be processed by child streams. /// Part of the data will be processed. - if (input_streams.size() == 1 && output_streams.size() == 1) - copyData(*input_streams.at(0), *output_streams.at(0)); - else - copyData(input_streams, output_streams); + copyData(*input, *output); return Block(); } private: - BlockInputStreams input_streams; - BlockOutputStreams output_streams; + BlockInputStreamPtr input; + BlockOutputStreamPtr output; }; } diff --git a/src/DataStreams/copyData.cpp b/src/DataStreams/copyData.cpp index fd4bfab28d8..a0651999034 100644 --- a/src/DataStreams/copyData.cpp +++ b/src/DataStreams/copyData.cpp @@ -1,9 +1,6 @@ -#include #include #include #include -#include -#include #include @@ -55,79 +52,6 @@ void copyDataImpl(IBlockInputStream & from, IBlockOutputStream & to, TCancelCall inline void doNothing(const Block &) {} -namespace -{ - - -struct ParallelInsertsHandler -{ - using CencellationHook = std::function; - - explicit ParallelInsertsHandler(BlockOutputStreams & output_streams, CencellationHook cancellation_hook_, size_t num_threads) - : outputs(output_streams.size()), cancellation_hook(std::move(cancellation_hook_)) - { - exceptions.resize(num_threads); - - for (auto & output : output_streams) - outputs.push(output.get()); - } - - void onBlock(Block & block, size_t /*thread_num*/) - { - IBlockOutputStream * out = nullptr; - - outputs.pop(out); - out->write(block); - outputs.push(out); - } - - void onFinishThread(size_t /*thread_num*/) {} - void onFinish() {} - - void onException(std::exception_ptr & exception, size_t thread_num) - { - exceptions[thread_num] = exception; - cancellation_hook(); - } - - void rethrowFirstException() - { - for (auto & exception : exceptions) - if (exception) - std::rethrow_exception(exception); - } - - ConcurrentBoundedQueue outputs; - std::vector exceptions; - CencellationHook cancellation_hook; -}; - -} - -static void copyDataImpl(BlockInputStreams & inputs, BlockOutputStreams & outputs) -{ - for (auto & output : outputs) - output->writePrefix(); - - using Processor = ParallelInputsProcessor; - Processor * processor_ptr = nullptr; - - ParallelInsertsHandler handler(outputs, [&processor_ptr]() { processor_ptr->cancel(false); }, inputs.size()); - ParallelInputsProcessor processor(inputs, nullptr, inputs.size(), handler); - processor_ptr = &processor; - - processor.process(); - processor.wait(); - handler.rethrowFirstException(); - - /// readPrefix is called in ParallelInputsProcessor. - for (auto & input : inputs) - input->readSuffix(); - - for (auto & output : outputs) - output->writeSuffix(); -} - void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic * is_cancelled) { auto is_cancelled_pred = [is_cancelled] () @@ -138,11 +62,6 @@ void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic & is_cancelled) { copyDataImpl(from, to, is_cancelled, doNothing); diff --git a/src/DataStreams/copyData.h b/src/DataStreams/copyData.h index ae72dbd2421..f2bce8f411b 100644 --- a/src/DataStreams/copyData.h +++ b/src/DataStreams/copyData.h @@ -16,8 +16,6 @@ class Block; */ void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic * is_cancelled = nullptr); -void copyData(BlockInputStreams & inputs, BlockOutputStreams & outputs); - void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function & is_cancelled); void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function & is_cancelled, diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index f3b116e490c..b6efa5d6d46 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -28,6 +28,11 @@ #include #include #include +#include +#include +#include +#include +#include namespace DB @@ -117,8 +122,6 @@ BlockIO InterpreterInsertQuery::execute() if (!query.table_function) context.checkAccess(AccessType::INSERT, query.table_id, query_sample_block.getNames()); - BlockInputStreams in_streams; - BlockOutputStreams out_streams; bool is_distributed_insert_select = false; if (query.select && table->isRemote() && settings.parallel_distributed_insert_select) @@ -159,6 +162,8 @@ BlockIO InterpreterInsertQuery::execute() const auto & cluster = storage_src->getCluster(); const auto & shards_info = cluster->getShardsInfo(); + std::vector pipelines; + String new_query_str = queryToString(new_query); for (size_t shard_index : ext::range(0, shards_info.size())) { @@ -166,8 +171,7 @@ BlockIO InterpreterInsertQuery::execute() if (shard_info.isLocal()) { InterpreterInsertQuery interpreter(new_query, context); - auto block_io = interpreter.execute(); - in_streams.push_back(block_io.in); + pipelines.emplace_back(interpreter.execute().pipeline); } else { @@ -179,13 +183,20 @@ BlockIO InterpreterInsertQuery::execute() /// INSERT SELECT query returns empty block auto in_stream = std::make_shared(std::move(connections), new_query_str, Block{}, context); - in_streams.push_back(in_stream); + pipelines.emplace_back(); + pipelines.back().init(Pipe(std::make_shared(std::move(in_stream)))); + pipelines.back().setSinks([](const Block & header, QueryPipeline::StreamType) -> ProcessorPtr + { + return std::make_shared(header); + }); } - out_streams.push_back(std::make_shared(Block())); } + + res.pipeline.unitePipelines(std::move(pipelines), {}); } } + BlockOutputStreams out_streams; if (!is_distributed_insert_select || query.watch) { size_t out_streams_size = 1; @@ -193,27 +204,21 @@ BlockIO InterpreterInsertQuery::execute() { /// Passing 1 as subquery_depth will disable limiting size of intermediate result. InterpreterSelectWithUnionQuery interpreter_select{ query.select, context, SelectQueryOptions(QueryProcessingStage::Complete, 1)}; + res.pipeline = interpreter_select.executeWithProcessors(); if (table->supportsParallelInsert() && settings.max_insert_threads > 1) - { - in_streams = interpreter_select.executeWithMultipleStreams(res.pipeline); - out_streams_size = std::min(size_t(settings.max_insert_threads), in_streams.size()); - } + out_streams_size = std::min(size_t(settings.max_insert_threads), res.pipeline.getNumStreams()); + + if (out_streams_size == 1) + res.pipeline.addPipe({std::make_shared(res.pipeline.getHeader(), res.pipeline.getNumStreams())}); else - { - res = interpreter_select.execute(); - in_streams.emplace_back(res.in); - res.in = nullptr; - res.out = nullptr; - } + res.pipeline.resize(out_streams_size); } else if (query.watch) { InterpreterWatchQuery interpreter_watch{ query.watch, context }; res = interpreter_watch.execute(); - in_streams.emplace_back(res.in); - res.in = nullptr; - res.out = nullptr; + res.pipeline.init(Pipe(std::make_shared(std::move(res.in)))); } for (size_t i = 0; i < out_streams_size; i++) @@ -256,27 +261,35 @@ BlockIO InterpreterInsertQuery::execute() } /// What type of query: INSERT or INSERT SELECT or INSERT WATCH? - if (query.select || query.watch) + if (is_distributed_insert_select) { - for (auto & in_stream : in_streams) - { - in_stream = std::make_shared( - in_stream, out_streams.at(0)->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Position); - } + /// Pipeline was already built. + } + else if (query.select || query.watch) + { + const auto & header = out_streams.at(0)->getHeader(); - Block in_header = in_streams.at(0)->getHeader(); - if (in_streams.size() > 1) + res.pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { - for (size_t i = 1; i < in_streams.size(); ++i) - assertBlocksHaveEqualStructure(in_streams[i]->getHeader(), in_header, query.select ? "INSERT SELECT" : "INSERT WATCH"); - } + return std::make_shared(in_header, header, + ConvertingTransform::MatchColumnsMode::Position); + }); - res.in = std::make_shared(in_streams, out_streams); + res.pipeline.setSinks([&](const Block &, QueryPipeline::StreamType type) -> ProcessorPtr + { + if (type != QueryPipeline::StreamType::Main) + return nullptr; + + auto stream = std::move(out_streams.back()); + out_streams.pop_back(); + + return std::make_shared(std::move(stream)); + }); if (!allow_materialized) { for (const auto & column : table->getColumns()) - if (column.default_desc.kind == ColumnDefaultKind::Materialized && in_header.has(column.name)) + if (column.default_desc.kind == ColumnDefaultKind::Materialized && header.has(column.name)) throw Exception("Cannot insert column " + column.name + ", because it is MATERIALIZED column.", ErrorCodes::ILLEGAL_COLUMN); } } @@ -288,6 +301,7 @@ BlockIO InterpreterInsertQuery::execute() } else res.out = std::move(out_streams.at(0)); + res.pipeline.addStorageHolder(table); return res; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 2d8f901d3bb..f90f37b88c3 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -342,6 +342,9 @@ static std::tuple executeQueryImpl( else res = interpreter->execute(); + if (res.pipeline.initialized()) + use_processors = true; + if (const auto * insert_interpreter = typeid_cast(&*interpreter)) { /// Save insertion table (not table function). TODO: support remote() table function. @@ -369,7 +372,7 @@ static std::tuple executeQueryImpl( /// Limits apply only to the final result. pipeline.setProgressCallback(context.getProgressCallback()); pipeline.setProcessListElement(context.getProcessListElement()); - if (stage == QueryProcessingStage::Complete) + if (stage == QueryProcessingStage::Complete && !pipeline.isCompleted()) { pipeline.resize(1); pipeline.addSimpleTransform([&](const Block & header) @@ -740,29 +743,36 @@ void executeQuery( if (ast_query_with_output && ast_query_with_output->settings_ast) InterpreterSetQuery(ast_query_with_output->settings_ast, context).executeForCurrentContext(); - pipeline.addSimpleTransform([](const Block & header) + if (!pipeline.isCompleted()) { - return std::make_shared(header); - }); + pipeline.addSimpleTransform([](const Block & header) + { + return std::make_shared(header); + }); - auto out = context.getOutputFormatProcessor(format_name, *out_buf, pipeline.getHeader()); - out->setAutoFlush(); + auto out = context.getOutputFormatProcessor(format_name, *out_buf, pipeline.getHeader()); + out->setAutoFlush(); - /// Save previous progress callback if any. TODO Do it more conveniently. - auto previous_progress_callback = context.getProgressCallback(); + /// Save previous progress callback if any. TODO Do it more conveniently. + auto previous_progress_callback = context.getProgressCallback(); - /// NOTE Progress callback takes shared ownership of 'out'. - pipeline.setProgressCallback([out, previous_progress_callback] (const Progress & progress) + /// NOTE Progress callback takes shared ownership of 'out'. + pipeline.setProgressCallback([out, previous_progress_callback] (const Progress & progress) + { + if (previous_progress_callback) + previous_progress_callback(progress); + out->onProgress(progress); + }); + + if (set_result_details) + set_result_details(context.getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::instance().getTimeZone()); + + pipeline.setOutputFormat(std::move(out)); + } + else { - if (previous_progress_callback) - previous_progress_callback(progress); - out->onProgress(progress); - }); - - if (set_result_details) - set_result_details(context.getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::instance().getTimeZone()); - - pipeline.setOutput(std::move(out)); + pipeline.setProgressCallback(context.getProgressCallback()); + } { auto executor = pipeline.execute(); diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 003508ab86f..49e85fcc6d3 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -14,9 +14,10 @@ struct PullingAsyncPipelineExecutor::Data { PipelineExecutorPtr executor; std::exception_ptr exception; - std::atomic_bool is_executed = false; + std::atomic_bool is_finished = false; std::atomic_bool has_exception = false; ThreadFromGlobalPool thread; + Poco::Event finish_event; ~Data() { @@ -36,8 +37,11 @@ struct PullingAsyncPipelineExecutor::Data PullingAsyncPipelineExecutor::PullingAsyncPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) { - lazy_format = std::make_shared(pipeline.getHeader()); - pipeline.setOutput(lazy_format); + if (!pipeline.isCompleted()) + { + lazy_format = std::make_shared(pipeline.getHeader()); + pipeline.setOutputFormat(lazy_format); + } } PullingAsyncPipelineExecutor::~PullingAsyncPipelineExecutor() @@ -54,7 +58,8 @@ PullingAsyncPipelineExecutor::~PullingAsyncPipelineExecutor() const Block & PullingAsyncPipelineExecutor::getHeader() const { - return lazy_format->getPort(IOutputFormat::PortKind::Main).getHeader(); + return lazy_format ? lazy_format->getPort(IOutputFormat::PortKind::Main).getHeader() + : pipeline.getHeader(); /// Empty. } static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads) @@ -78,6 +83,9 @@ static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGrou data.exception = std::current_exception(); data.has_exception = true; } + + data.is_finished = true; + data.finish_event.set(); } @@ -99,20 +107,33 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) if (data->has_exception) { /// Finish lazy format in case of exception. Otherwise thread.join() may hung. - lazy_format->finish(); + if (lazy_format) + lazy_format->finish(); + data->has_exception = false; std::rethrow_exception(std::move(data->exception)); } - if (lazy_format->isFinished()) + bool is_execution_finished = lazy_format ? lazy_format->isFinished() + : data->is_finished.load(); + + if (is_execution_finished) { - data->is_executed = true; + /// If lazy format is finished, we don't cancel pipeline but wait for main thread to be finished. + data->is_finished = true; /// Wait thread ant rethrow exception if any. cancel(); return false; } - chunk = lazy_format->getChunk(milliseconds); + if (lazy_format) + { + chunk = lazy_format->getChunk(milliseconds); + return true; + } + + chunk.clear(); + data->finish_event.tryWait(milliseconds); return true; } @@ -147,11 +168,11 @@ bool PullingAsyncPipelineExecutor::pull(Block & block, uint64_t milliseconds) void PullingAsyncPipelineExecutor::cancel() { /// Cancel execution if it wasn't finished. - if (data && !data->is_executed && data->executor) + if (data && !data->is_finished && data->executor) data->executor->cancel(); /// Finish lazy format. Otherwise thread.join() may hung. - if (!lazy_format->isFinished()) + if (lazy_format && !lazy_format->isFinished()) lazy_format->finish(); /// Join thread here to wait for possible exception. @@ -165,12 +186,14 @@ void PullingAsyncPipelineExecutor::cancel() Chunk PullingAsyncPipelineExecutor::getTotals() { - return lazy_format->getTotals(); + return lazy_format ? lazy_format->getTotals() + : Chunk(); } Chunk PullingAsyncPipelineExecutor::getExtremes() { - return lazy_format->getExtremes(); + return lazy_format ? lazy_format->getExtremes() + : Chunk(); } Block PullingAsyncPipelineExecutor::getTotalsBlock() @@ -197,7 +220,9 @@ Block PullingAsyncPipelineExecutor::getExtremesBlock() BlockStreamProfileInfo & PullingAsyncPipelineExecutor::getProfileInfo() { - return lazy_format->getProfileInfo(); + static BlockStreamProfileInfo profile_info; + return lazy_format ? lazy_format->getProfileInfo() + : profile_info; } } diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index 375f6c9ed0e..af061a373cc 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -9,7 +9,7 @@ namespace DB PullingPipelineExecutor::PullingPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) { pulling_format = std::make_shared(pipeline.getHeader(), has_data_flag); - pipeline.setOutput(pulling_format); + pipeline.setOutputFormat(pulling_format); } PullingPipelineExecutor::~PullingPipelineExecutor() diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 13787a3fd3b..146061e8a47 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -34,6 +34,14 @@ void QueryPipeline::checkInitialized() throw Exception("QueryPipeline wasn't initialized.", ErrorCodes::LOGICAL_ERROR); } +void QueryPipeline::checkInitializedAndNotCompleted() +{ + checkInitialized(); + + if (streams.empty()) + throw Exception("QueryPipeline was already completed.", ErrorCodes::LOGICAL_ERROR); +} + void QueryPipeline::checkSource(const ProcessorPtr & source, bool can_have_totals) { if (!source->getInputs().empty()) @@ -194,11 +202,11 @@ static ProcessorPtr callProcessorGetter( template void QueryPipeline::addSimpleTransformImpl(const TProcessorGetter & getter) { - checkInitialized(); + checkInitializedAndNotCompleted(); Block header; - auto add_transform = [&](OutputPort *& stream, StreamType stream_type, size_t stream_num [[maybe_unused]] = IProcessor::NO_STREAM) + auto add_transform = [&](OutputPort *& stream, StreamType stream_type) { if (!stream) return; @@ -231,17 +239,14 @@ void QueryPipeline::addSimpleTransformImpl(const TProcessorGetter & getter) if (transform) { -// if (stream_type == StreamType::Main) -// transform->setStream(stream_num); - connect(*stream, transform->getInputs().front()); stream = &transform->getOutputs().front(); processors.emplace_back(std::move(transform)); } }; - for (size_t stream_num = 0; stream_num < streams.size(); ++stream_num) - add_transform(streams[stream_num], StreamType::Main, stream_num); + for (auto & stream : streams) + add_transform(stream, StreamType::Main); add_transform(totals_having_port, StreamType::Totals); add_transform(extremes_port, StreamType::Extremes); @@ -259,9 +264,50 @@ void QueryPipeline::addSimpleTransform(const ProcessorGetterWithStreamKind & get addSimpleTransformImpl(getter); } +void QueryPipeline::setSinks(const ProcessorGetterWithStreamKind & getter) +{ + checkInitializedAndNotCompleted(); + + auto add_transform = [&](OutputPort *& stream, StreamType stream_type) + { + if (!stream) + return; + + auto transform = getter(stream->getHeader(), stream_type); + + if (transform) + { + if (transform->getInputs().size() != 1) + throw Exception("Sink for query pipeline transform should have single input, " + "but " + transform->getName() + " has " + + toString(transform->getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR); + + if (!transform->getOutputs().empty()) + throw Exception("Sink for query pipeline transform should have no outputs, " + "but " + transform->getName() + " has " + + toString(transform->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); + } + + if (!transform) + transform = std::make_shared(stream->getHeader()); + + connect(*stream, transform->getInputs().front()); + processors.emplace_back(std::move(transform)); + }; + + for (auto & stream : streams) + add_transform(stream, StreamType::Main); + + add_transform(totals_having_port, StreamType::Totals); + add_transform(extremes_port, StreamType::Extremes); + + streams.clear(); + current_header.clear(); +} + void QueryPipeline::addPipe(Processors pipe) { - checkInitialized(); + checkInitializedAndNotCompleted(); if (pipe.empty()) throw Exception("Can't add empty processors list to QueryPipeline.", ErrorCodes::LOGICAL_ERROR); @@ -298,7 +344,7 @@ void QueryPipeline::addPipe(Processors pipe) void QueryPipeline::addDelayedStream(ProcessorPtr source) { - checkInitialized(); + checkInitializedAndNotCompleted(); checkSource(source, false); assertBlocksHaveEqualStructure(current_header, source->getOutputs().front().getHeader(), "QueryPipeline"); @@ -313,7 +359,7 @@ void QueryPipeline::addDelayedStream(ProcessorPtr source) void QueryPipeline::resize(size_t num_streams, bool force, bool strict) { - checkInitialized(); + checkInitializedAndNotCompleted(); if (!force && num_streams == getNumStreams()) return; @@ -347,7 +393,7 @@ void QueryPipeline::enableQuotaForCurrentStreams() void QueryPipeline::addTotalsHavingTransform(ProcessorPtr transform) { - checkInitialized(); + checkInitializedAndNotCompleted(); if (!typeid_cast(transform.get())) throw Exception("TotalsHavingTransform expected for QueryPipeline::addTotalsHavingTransform.", @@ -370,7 +416,7 @@ void QueryPipeline::addTotalsHavingTransform(ProcessorPtr transform) void QueryPipeline::addDefaultTotals() { - checkInitialized(); + checkInitializedAndNotCompleted(); if (totals_having_port) throw Exception("Totals having transform was already added to pipeline.", ErrorCodes::LOGICAL_ERROR); @@ -392,7 +438,7 @@ void QueryPipeline::addDefaultTotals() void QueryPipeline::addTotals(ProcessorPtr source) { - checkInitialized(); + checkInitializedAndNotCompleted(); if (totals_having_port) throw Exception("Totals having transform was already added to pipeline.", ErrorCodes::LOGICAL_ERROR); @@ -423,7 +469,7 @@ void QueryPipeline::dropTotalsAndExtremes() void QueryPipeline::addExtremesTransform() { - checkInitialized(); + checkInitializedAndNotCompleted(); if (extremes_port) throw Exception("Extremes transform was already added to pipeline.", ErrorCodes::LOGICAL_ERROR); @@ -450,7 +496,7 @@ void QueryPipeline::addExtremesTransform() void QueryPipeline::addCreatingSetsTransform(ProcessorPtr transform) { - checkInitialized(); + checkInitializedAndNotCompleted(); if (!typeid_cast(transform.get())) throw Exception("CreatingSetsTransform expected for QueryPipeline::addExtremesTransform.", @@ -467,14 +513,14 @@ void QueryPipeline::addCreatingSetsTransform(ProcessorPtr transform) processors.emplace_back(std::move(concat)); } -void QueryPipeline::setOutput(ProcessorPtr output) +void QueryPipeline::setOutputFormat(ProcessorPtr output) { - checkInitialized(); + checkInitializedAndNotCompleted(); auto * format = dynamic_cast(output.get()); if (!format) - throw Exception("IOutputFormat processor expected for QueryPipeline::setOutput.", ErrorCodes::LOGICAL_ERROR); + throw Exception("IOutputFormat processor expected for QueryPipeline::setOutputFormat.", ErrorCodes::LOGICAL_ERROR); if (output_format) throw Exception("QueryPipeline already has output.", ErrorCodes::LOGICAL_ERROR); @@ -507,19 +553,25 @@ void QueryPipeline::setOutput(ProcessorPtr output) connect(*totals_having_port, totals); connect(*extremes_port, extremes); + streams.clear(); + current_header.clear(); + extremes_port = nullptr; + totals_having_port = nullptr; + initRowsBeforeLimit(); } void QueryPipeline::unitePipelines( std::vector && pipelines, const Block & common_header) { - checkInitialized(); - - addSimpleTransform([&](const Block & header) + if (initialized()) { - return std::make_shared( - header, common_header, ConvertingTransform::MatchColumnsMode::Position); - }); + addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, common_header, ConvertingTransform::MatchColumnsMode::Position); + }); + } std::vector extremes; std::vector totals; @@ -534,11 +586,14 @@ void QueryPipeline::unitePipelines( { pipeline.checkInitialized(); - pipeline.addSimpleTransform([&](const Block & header) + if (!pipeline.isCompleted()) { - return std::make_shared( - header, common_header, ConvertingTransform::MatchColumnsMode::Position); - }); + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, common_header, ConvertingTransform::MatchColumnsMode::Position); + }); + } if (pipeline.extremes_port) { @@ -726,10 +781,8 @@ Pipe QueryPipeline::getPipe() && PipelineExecutorPtr QueryPipeline::execute() { - checkInitialized(); - - if (!output_format) - throw Exception("Cannot execute pipeline because it doesn't have output.", ErrorCodes::LOGICAL_ERROR); + if (!isCompleted()) + throw Exception("Cannot execute pipeline because it is not completed.", ErrorCodes::LOGICAL_ERROR); return std::make_shared(processors, process_list_element); } diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 45e38ffa715..4b2b32c3258 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -28,6 +28,7 @@ private: { public: auto size() const { return data.size(); } + bool empty() const { return size() == 0; } auto begin() { return data.begin(); } auto end() { return data.end(); } auto & front() { return data.front(); } @@ -81,6 +82,7 @@ public: void init(Pipes pipes); void init(Pipe pipe); /// Simple init for single pipe bool initialized() { return !processors.empty(); } + bool isCompleted() { return initialized() && streams.empty(); } /// Type of logical data stream for simple transform. /// Sometimes it's important to know which part of pipeline we are working for. @@ -95,13 +97,23 @@ public: using ProcessorGetter = std::function; using ProcessorGetterWithStreamKind = std::function; + /// Add transform with simple input and simple output for each port. void addSimpleTransform(const ProcessorGetter & getter); void addSimpleTransform(const ProcessorGetterWithStreamKind & getter); + /// Add several processors. They must have same header for inputs and same for outputs. + /// Total number of inputs must be the same as the number of streams. Output ports will become new streams. void addPipe(Processors pipe); + /// Add TotalsHavingTransform. Resize pipeline to single input. Adds totals port. void addTotalsHavingTransform(ProcessorPtr transform); + /// Add transform which calculates extremes. This transform adds extremes port and doesn't change inputs number. void addExtremesTransform(); + /// Adds transform which creates sets. It will be executed before reading any data from input ports. void addCreatingSetsTransform(ProcessorPtr transform); - void setOutput(ProcessorPtr output); + /// Resize pipeline to single output and add IOutputFormat. Pipeline will be completed after this transformation. + void setOutputFormat(ProcessorPtr output); + /// Sink is a processor with single input port and no output ports. Creates sink for each output port. + /// Pipeline will be completed after this transformation. + void setSinks(const ProcessorGetterWithStreamKind & getter); /// Add totals which returns one chunk with single row with defaults. void addDefaultTotals(); @@ -118,6 +130,7 @@ public: /// Check if resize transform was used. (In that case another distinct transform will be added). bool hasMixedStreams() const { return has_resize || hasMoreThanOneStream(); } + /// Changes the number of input ports if needed. Adds ResizeTransform. void resize(size_t num_streams, bool force = false, bool strict = false); void enableQuotaForCurrentStreams(); @@ -193,6 +206,7 @@ private: QueryStatus * process_list_element = nullptr; void checkInitialized(); + void checkInitializedAndNotCompleted(); static void checkSource(const ProcessorPtr & source, bool can_have_totals); template From 2432ea2c4b5a0307c175f0a5dcc7e95639e1da68 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 28 May 2020 00:58:46 +0300 Subject: [PATCH 73/79] Build universal binary using ya make --- programs/client/CMakeLists.txt | 11 ++--- programs/client/Client.cpp | 5 ++- programs/client/ConnectionParameters.cpp | 2 +- programs/client/config_client.h.in | 3 -- programs/client/readpassphrase/CMakeLists.txt | 12 ++---- .../{includes.h.in => includes.h} | 2 +- .../client/readpassphrase/readpassphrase.c | 16 +------ .../client/readpassphrase/readpassphrase.h | 39 +++++------------ programs/client/readpassphrase/ya.make | 7 +++ programs/main.cpp | 43 +++++++++---------- programs/ya.make | 28 +++++++++++- src/Common/ya.make | 3 ++ 12 files changed, 80 insertions(+), 91 deletions(-) delete mode 100644 programs/client/config_client.h.in rename programs/client/readpassphrase/{includes.h.in => includes.h} (79%) create mode 100644 programs/client/readpassphrase/ya.make diff --git a/programs/client/CMakeLists.txt b/programs/client/CMakeLists.txt index 11ade559a8d..e273123afe0 100644 --- a/programs/client/CMakeLists.txt +++ b/programs/client/CMakeLists.txt @@ -6,14 +6,9 @@ set(CLICKHOUSE_CLIENT_SOURCES set(CLICKHOUSE_CLIENT_LINK PRIVATE clickhouse_common_config clickhouse_functions clickhouse_aggregate_functions clickhouse_common_io clickhouse_parsers string_utils ${Boost_PROGRAM_OPTIONS_LIBRARY}) -include(CheckSymbolExists) -check_symbol_exists(readpassphrase readpassphrase.h HAVE_READPASSPHRASE) -configure_file(config_client.h.in ${ConfigIncludePath}/config_client.h) - -if(NOT HAVE_READPASSPHRASE) - add_subdirectory(readpassphrase) - list(APPEND CLICKHOUSE_CLIENT_LINK PRIVATE readpassphrase) -endif() +# Always use internal readpassphrase +add_subdirectory(readpassphrase) +list(APPEND CLICKHOUSE_CLIENT_LINK PRIVATE readpassphrase) clickhouse_program_add(client) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index d6cac7a7b02..afc8f9a72b1 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -39,7 +39,6 @@ #include #include #include -#include #include #include #include @@ -77,6 +76,10 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + #ifndef __clang__ #pragma GCC optimize("-fno-var-tracking-assignments") #endif diff --git a/programs/client/ConnectionParameters.cpp b/programs/client/ConnectionParameters.cpp index f0ef3ae5694..d8b4d0f1add 100644 --- a/programs/client/ConnectionParameters.cpp +++ b/programs/client/ConnectionParameters.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include "readpassphrase/readpassphrase.h" namespace DB { diff --git a/programs/client/config_client.h.in b/programs/client/config_client.h.in deleted file mode 100644 index 5ad788ff54c..00000000000 --- a/programs/client/config_client.h.in +++ /dev/null @@ -1,3 +0,0 @@ -#pragma once - -#cmakedefine HAVE_READPASSPHRASE diff --git a/programs/client/readpassphrase/CMakeLists.txt b/programs/client/readpassphrase/CMakeLists.txt index a10b54c377d..dd1bf2c91b9 100644 --- a/programs/client/readpassphrase/CMakeLists.txt +++ b/programs/client/readpassphrase/CMakeLists.txt @@ -1,13 +1,7 @@ - # wget https://raw.githubusercontent.com/openssh/openssh-portable/master/openbsd-compat/readpassphrase.c # wget https://raw.githubusercontent.com/openssh/openssh-portable/master/openbsd-compat/readpassphrase.h -set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-result -Wno-reserved-id-macro") +add_library(readpassphrase readpassphrase.c) -configure_file(includes.h.in ${CMAKE_CURRENT_BINARY_DIR}/include/includes.h) -add_library(readpassphrase ${CMAKE_CURRENT_SOURCE_DIR}/readpassphrase.c) -set_target_properties(readpassphrase - PROPERTIES LINKER_LANGUAGE C - ) -# . to allow #include -target_include_directories(readpassphrase PUBLIC . ${CMAKE_CURRENT_BINARY_DIR}/include) +set_target_properties(readpassphrase PROPERTIES LINKER_LANGUAGE C) +target_compile_options(readpassphrase PRIVATE -Wno-unused-result -Wno-reserved-id-macro) diff --git a/programs/client/readpassphrase/includes.h.in b/programs/client/readpassphrase/includes.h similarity index 79% rename from programs/client/readpassphrase/includes.h.in rename to programs/client/readpassphrase/includes.h index 44580d1ed95..3ca5eb2bff8 100644 --- a/programs/client/readpassphrase/includes.h.in +++ b/programs/client/readpassphrase/includes.h @@ -1,6 +1,6 @@ #pragma once -#cmakedefine HAVE_READPASSPHRASE +/* #undef HAVE_READPASSPHRASE */ #if !defined(HAVE_READPASSPHRASE) # ifndef _PATH_TTY diff --git a/programs/client/readpassphrase/readpassphrase.c b/programs/client/readpassphrase/readpassphrase.c index 243701239bf..21b48e7efc3 100644 --- a/programs/client/readpassphrase/readpassphrase.c +++ b/programs/client/readpassphrase/readpassphrase.c @@ -25,13 +25,11 @@ #include "includes.h" -#ifndef HAVE_READPASSPHRASE - #include #include #include #include -#include +#include "readpassphrase.h" #include #include #include @@ -193,19 +191,7 @@ restart: } //DEF_WEAK(readpassphrase); -#if 0 -char * -getpass(const char *prompt) -{ - static char buf[_PASSWORD_LEN + 1]; - - return(readpassphrase(prompt, buf, sizeof(buf), RPP_ECHO_OFF)); -} -#endif - static void handler(int s) { - signo[s] = 1; } -#endif /* HAVE_READPASSPHRASE */ diff --git a/programs/client/readpassphrase/readpassphrase.h b/programs/client/readpassphrase/readpassphrase.h index 0782a1773ea..399eb7b062c 100644 --- a/programs/client/readpassphrase/readpassphrase.h +++ b/programs/client/readpassphrase/readpassphrase.h @@ -23,39 +23,22 @@ /* OPENBSD ORIGINAL: include/readpassphrase.h */ #pragma once -// #ifndef _READPASSPHRASE_H_ -// #define _READPASSPHRASE_H_ -//#include "includes.h" -#include "config_client.h" - -// Should not be included on BSD systems, but if it happen... -#ifdef HAVE_READPASSPHRASE -# include_next +#if defined(__cplusplus) +extern "C" { #endif -#ifndef HAVE_READPASSPHRASE -# ifdef __cplusplus -extern "C" { -# endif - - -# define RPP_ECHO_OFF 0x00 /* Turn off echo (default). */ -# define RPP_ECHO_ON 0x01 /* Leave echo on. */ -# define RPP_REQUIRE_TTY 0x02 /* Fail if there is no tty. */ -# define RPP_FORCELOWER 0x04 /* Force input to lower case. */ -# define RPP_FORCEUPPER 0x08 /* Force input to upper case. */ -# define RPP_SEVENBIT 0x10 /* Strip the high bit from input. */ -# define RPP_STDIN 0x20 /* Read from stdin, not /dev/tty */ +#define RPP_ECHO_OFF 0x00 /* Turn off echo (default). */ +#define RPP_ECHO_ON 0x01 /* Leave echo on. */ +#define RPP_REQUIRE_TTY 0x02 /* Fail if there is no tty. */ +#define RPP_FORCELOWER 0x04 /* Force input to lower case. */ +#define RPP_FORCEUPPER 0x08 /* Force input to upper case. */ +#define RPP_SEVENBIT 0x10 /* Strip the high bit from input. */ +#define RPP_STDIN 0x20 /* Read from stdin, not /dev/tty */ char * readpassphrase(const char *, char *, size_t, int); -# ifdef __cplusplus +#if defined(__cplusplus) } -# endif - - -#endif /* HAVE_READPASSPHRASE */ - -// #endif /* !_READPASSPHRASE_H_ */ +#endif diff --git a/programs/client/readpassphrase/ya.make b/programs/client/readpassphrase/ya.make new file mode 100644 index 00000000000..80ad197e5d4 --- /dev/null +++ b/programs/client/readpassphrase/ya.make @@ -0,0 +1,7 @@ +LIBRARY() + +SRCS( + readpassphrase.c +) + +END() diff --git a/programs/main.cpp b/programs/main.cpp index 2eb226d3c00..6b34504ce37 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -8,11 +8,8 @@ #include #include /// pair -#if __has_include("config_tools.h") -#include "config_tools.h" -#endif -#if __has_include("config_core.h") -#include "config_core.h" +#if !defined(ARCADIA_BUILD) +# include "config_tools.h" #endif #include @@ -22,31 +19,31 @@ /// Universal executable for various clickhouse applications -#if ENABLE_CLICKHOUSE_SERVER || !defined(ENABLE_CLICKHOUSE_SERVER) +#if ENABLE_CLICKHOUSE_SERVER int mainEntryClickHouseServer(int argc, char ** argv); #endif -#if ENABLE_CLICKHOUSE_CLIENT || !defined(ENABLE_CLICKHOUSE_CLIENT) +#if ENABLE_CLICKHOUSE_CLIENT int mainEntryClickHouseClient(int argc, char ** argv); #endif -#if ENABLE_CLICKHOUSE_LOCAL || !defined(ENABLE_CLICKHOUSE_LOCAL) +#if ENABLE_CLICKHOUSE_LOCAL int mainEntryClickHouseLocal(int argc, char ** argv); #endif -#if ENABLE_CLICKHOUSE_BENCHMARK || !defined(ENABLE_CLICKHOUSE_BENCHMARK) +#if ENABLE_CLICKHOUSE_BENCHMARK int mainEntryClickHouseBenchmark(int argc, char ** argv); #endif -#if ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG || !defined(ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG) +#if ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG int mainEntryClickHouseExtractFromConfig(int argc, char ** argv); #endif -#if ENABLE_CLICKHOUSE_COMPRESSOR || !defined(ENABLE_CLICKHOUSE_COMPRESSOR) +#if ENABLE_CLICKHOUSE_COMPRESSOR int mainEntryClickHouseCompressor(int argc, char ** argv); #endif -#if ENABLE_CLICKHOUSE_FORMAT || !defined(ENABLE_CLICKHOUSE_FORMAT) +#if ENABLE_CLICKHOUSE_FORMAT int mainEntryClickHouseFormat(int argc, char ** argv); #endif -#if ENABLE_CLICKHOUSE_COPIER || !defined(ENABLE_CLICKHOUSE_COPIER) +#if ENABLE_CLICKHOUSE_COPIER int mainEntryClickHouseClusterCopier(int argc, char ** argv); #endif -#if ENABLE_CLICKHOUSE_OBFUSCATOR || !defined(ENABLE_CLICKHOUSE_OBFUSCATOR) +#if ENABLE_CLICKHOUSE_OBFUSCATOR int mainEntryClickHouseObfuscator(int argc, char ** argv); #endif @@ -60,31 +57,31 @@ using MainFunc = int (*)(int, char**); /// Add an item here to register new application std::pair clickhouse_applications[] = { -#if ENABLE_CLICKHOUSE_LOCAL || !defined(ENABLE_CLICKHOUSE_LOCAL) +#if ENABLE_CLICKHOUSE_LOCAL {"local", mainEntryClickHouseLocal}, #endif -#if ENABLE_CLICKHOUSE_CLIENT || !defined(ENABLE_CLICKHOUSE_CLIENT) +#if ENABLE_CLICKHOUSE_CLIENT {"client", mainEntryClickHouseClient}, #endif -#if ENABLE_CLICKHOUSE_BENCHMARK || !defined(ENABLE_CLICKHOUSE_BENCHMARK) +#if ENABLE_CLICKHOUSE_BENCHMARK {"benchmark", mainEntryClickHouseBenchmark}, #endif -#if ENABLE_CLICKHOUSE_SERVER || !defined(ENABLE_CLICKHOUSE_SERVER) +#if ENABLE_CLICKHOUSE_SERVER {"server", mainEntryClickHouseServer}, #endif -#if ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG || !defined(ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG) +#if ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG {"extract-from-config", mainEntryClickHouseExtractFromConfig}, #endif -#if ENABLE_CLICKHOUSE_COMPRESSOR || !defined(ENABLE_CLICKHOUSE_COMPRESSOR) +#if ENABLE_CLICKHOUSE_COMPRESSOR {"compressor", mainEntryClickHouseCompressor}, #endif -#if ENABLE_CLICKHOUSE_FORMAT || !defined(ENABLE_CLICKHOUSE_FORMAT) +#if ENABLE_CLICKHOUSE_FORMAT {"format", mainEntryClickHouseFormat}, #endif -#if ENABLE_CLICKHOUSE_COPIER || !defined(ENABLE_CLICKHOUSE_COPIER) +#if ENABLE_CLICKHOUSE_COPIER {"copier", mainEntryClickHouseClusterCopier}, #endif -#if ENABLE_CLICKHOUSE_OBFUSCATOR || !defined(ENABLE_CLICKHOUSE_OBFUSCATOR) +#if ENABLE_CLICKHOUSE_OBFUSCATOR {"obfuscator", mainEntryClickHouseObfuscator}, #endif }; diff --git a/programs/ya.make b/programs/ya.make index 6c773c312b8..f4a61850212 100644 --- a/programs/ya.make +++ b/programs/ya.make @@ -1,3 +1,27 @@ -RECURSE( - server +PROGRAM(clickhouse) + +CFLAGS( + -DENABLE_CLICKHOUSE_CLIENT + -DENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG + -DENABLE_CLICKHOUSE_SERVER ) + +PEERDIR( + clickhouse/base/daemon + clickhouse/base/loggers + clickhouse/programs/client/readpassphrase + clickhouse/src +) + +SRCS( + main.cpp + + client/Client.cpp + client/ConnectionParameters.cpp + client/Suggest.cpp + extract-from-config/ExtractFromConfig.cpp + server/Server.cpp + server/MetricsTransmitter.cpp +) + +END() diff --git a/src/Common/ya.make b/src/Common/ya.make index c0178f3d310..99659aa2aa0 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -25,6 +25,9 @@ CFLAGS (GLOBAL -DDBMS_VERSION_MINOR=0) CFLAGS (GLOBAL -DDBMS_VERSION_PATCH=0) CFLAGS (GLOBAL -DVERSION_FULL=\"ClickHouse\") CFLAGS (GLOBAL -DVERSION_INTEGER=0) +CFLAGS (GLOBAL -DVERSION_MAJOR=0) +CFLAGS (GLOBAL -DVERSION_MINOR=0) +CFLAGS (GLOBAL -DVERSION_PATCH=0) CFLAGS (GLOBAL -DVERSION_NAME=\"ClickHouse\") CFLAGS (GLOBAL -DVERSION_OFFICIAL=\"\\\(arcadia\\\)\") CFLAGS (GLOBAL -DVERSION_REVISION=0) From aabf8d479eba263860b814437237a1c77fa34262 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Thu, 28 May 2020 07:33:06 +0000 Subject: [PATCH 74/79] Bump protobuf from 3.12.1 to 3.12.2 in /docs/tools Bumps [protobuf](https://github.com/protocolbuffers/protobuf) from 3.12.1 to 3.12.2. - [Release notes](https://github.com/protocolbuffers/protobuf/releases) - [Changelog](https://github.com/protocolbuffers/protobuf/blob/master/generate_changelog.py) - [Commits](https://github.com/protocolbuffers/protobuf/compare/v3.12.1...v3.12.2) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 4d4e9f98780..d9ea19ff389 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -21,7 +21,7 @@ mkdocs-htmlproofer-plugin==0.0.3 mkdocs-macros-plugin==0.4.9 nltk==3.5 nose==1.3.7 -protobuf==3.12.1 +protobuf==3.12.2 numpy==1.18.4 Pygments==2.5.2 pymdown-extensions==7.1 From ca5d7cf5bb0d06224015881dcf4171be8adaffae Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 28 May 2020 10:45:03 +0300 Subject: [PATCH 75/79] fix some broken performance tests --- docker/test/performance-comparison/compare.sh | 4 ++-- docker/test/performance-comparison/report.py | 2 +- tests/performance/leftpad.xml | 1 - tests/performance/sort_radix_trivial.xml | 6 +++--- 4 files changed, 6 insertions(+), 7 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index bf4599acb9a..209b36f59af 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -317,7 +317,7 @@ function report rm -r report ||: mkdir report report/tmp ||: -rm ./*.{rep,svg} test-times.tsv test-dump.tsv unstable.tsv unstable-query-ids.tsv unstable-query-metrics.tsv changed-perf.tsv unstable-tests.tsv unstable-queries.tsv bad-tests.tsv slow-on-client.tsv all-queries.tsv ||: +rm ./*.{rep,svg} test-times.tsv test-dump.tsv unstable.tsv unstable-query-ids.tsv unstable-query-metrics.tsv changed-perf.tsv unstable-tests.tsv unstable-queries.tsv bad-tests.tsv slow-on-client.tsv all-queries.tsv run-errors.tsv ||: build_log_column_definitions @@ -434,7 +434,7 @@ create table wall_clock engine Memory as select * from file('wall-clock-times.tsv', TSV, 'test text, real float, user float, system float'); create table slow_on_client_tsv engine File(TSV, 'report/slow-on-client.tsv') as - select client, server, floor(client/server, 3) p, query_display_name + select client, server, floor(client/server, 3) p, test, query_display_name from query_time left join query_display_names using (test, query_index) where p > 1.02 order by p desc; diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index b171603700d..866e78da098 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -189,7 +189,7 @@ if args.report == 'main': slow_on_client_rows = tsvRows('report/slow-on-client.tsv') error_tests += len(slow_on_client_rows) printSimpleTable('Slow on client', - ['Client time, s', 'Server time, s', 'Ratio', 'Query'], + ['Client time, s', 'Server time, s', 'Ratio', 'Test', 'Query'], slow_on_client_rows) def print_changes(): diff --git a/tests/performance/leftpad.xml b/tests/performance/leftpad.xml index 4349c6ac0e5..199f9224b89 100644 --- a/tests/performance/leftpad.xml +++ b/tests/performance/leftpad.xml @@ -6,7 +6,6 @@ - diff --git a/tests/performance/sort_radix_trivial.xml b/tests/performance/sort_radix_trivial.xml index fb3dc838a9c..096c3dd0854 100644 --- a/tests/performance/sort_radix_trivial.xml +++ b/tests/performance/sort_radix_trivial.xml @@ -1,5 +1,5 @@ - SELECT rand32() AS x FROM numbers(1000000) ORDER BY x - SELECT rand64() AS x FROM numbers(1000000) ORDER BY x - SELECT 1 / rand64() AS x FROM numbers(1000000) ORDER BY x + SELECT rand32() AS x FROM numbers(1000000) ORDER BY x FORMAT Null + SELECT rand64() AS x FROM numbers(1000000) ORDER BY x FORMAT Null + SELECT 1 / rand64() AS x FROM numbers(1000000) ORDER BY x FORMAT Null From bc773be0e4dab18c45e67e053f82457e5c107534 Mon Sep 17 00:00:00 2001 From: Ilya Date: Thu, 28 May 2020 13:30:29 +0300 Subject: [PATCH 76/79] Fix package name in docker file: tzata -> tzdata --- docker/server/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 3ad20a9479c..93f192c3f3c 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -21,7 +21,7 @@ RUN apt-get update \ locales \ ca-certificates \ wget \ - tzata \ + tzdata \ && rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ From c6f82590ad8a0a4cd56fd7458748d30af1a18fea Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 28 May 2020 18:34:33 +0300 Subject: [PATCH 77/79] Add missed files --- src/Storages/TTLDescription.cpp | 197 ++++++++++++++++++++++++++++++++ src/Storages/TTLDescription.h | 69 +++++++++++ 2 files changed, 266 insertions(+) create mode 100644 src/Storages/TTLDescription.cpp create mode 100644 src/Storages/TTLDescription.h diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp new file mode 100644 index 00000000000..0efe6076174 --- /dev/null +++ b/src/Storages/TTLDescription.cpp @@ -0,0 +1,197 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include + + +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int BAD_TTL_EXPRESSION; +} + +namespace +{ + +void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const String & result_column_name) +{ + for (const auto & action : ttl_expression->getActions()) + { + if (action.type == ExpressionAction::APPLY_FUNCTION) + { + IFunctionBase & func = *action.function_base; + if (!func.isDeterministic()) + throw Exception( + "TTL expression cannot contain non-deterministic functions, " + "but contains function " + + func.getName(), + ErrorCodes::BAD_ARGUMENTS); + } + } + + const auto & result_column = ttl_expression->getSampleBlock().getByName(result_column_name); + + if (!typeid_cast(result_column.type.get()) + && !typeid_cast(result_column.type.get())) + { + throw Exception( + "TTL expression result column should have DateTime or Date type, but has " + result_column.type->getName(), + ErrorCodes::BAD_TTL_EXPRESSION); + } +} + +} + +TTLDescription TTLDescription::getTTLFromAST( + const ASTPtr & definition_ast, + const ColumnsDescription & columns, + const Context & context, + const StorageMetadataKeyField & primary_key) +{ + TTLDescription result; + const auto * ttl_element = definition_ast->as(); + + /// First child is expression: `TTL expr TO DISK` + if (ttl_element != nullptr) + result.expression_ast = ttl_element->children.front()->clone(); + else /// It's columns TTL without any additions, just copy it + result.expression_ast = definition_ast->clone(); + + auto ttl_ast = result.expression_ast->clone(); + auto syntax_analyzer_result = SyntaxAnalyzer(context).analyze(ttl_ast, columns.getAllPhysical()); + result.expression = ExpressionAnalyzer(ttl_ast, syntax_analyzer_result, context).getActions(false); + result.result_column = ttl_ast->getColumnName(); + + if (ttl_element == nullptr) /// columns TTL + { + result.destination_type = DataDestinationType::DELETE; + result.mode = TTLMode::DELETE; + } + else /// rows TTL + { + result.destination_type = ttl_element->destination_type; + result.destination_name = ttl_element->destination_name; + result.mode = ttl_element->mode; + + if (ttl_element->mode == TTLMode::DELETE) + { + if (ASTPtr where_expr_ast = ttl_element->where()) + { + auto where_syntax_result = SyntaxAnalyzer(context).analyze(where_expr_ast, columns.getAllPhysical()); + result.where_expression = ExpressionAnalyzer(where_expr_ast, where_syntax_result, context).getActions(false); + result.where_result_column = where_expr_ast->getColumnName(); + } + } + else if (ttl_element->mode == TTLMode::GROUP_BY) + { + const auto & pk_columns = primary_key.column_names; + + if (ttl_element->group_by_key.size() > pk_columns.size()) + throw Exception("TTL Expression GROUP BY key should be a prefix of primary key", ErrorCodes::BAD_TTL_EXPRESSION); + + NameSet primary_key_columns_set(pk_columns.begin(), pk_columns.end()); + NameSet aggregation_columns_set; + + for (const auto & column : primary_key.expression->getRequiredColumns()) + primary_key_columns_set.insert(column); + + for (size_t i = 0; i < ttl_element->group_by_key.size(); ++i) + { + if (ttl_element->group_by_key[i]->getColumnName() != pk_columns[i]) + throw Exception( + "TTL Expression GROUP BY key should be a prefix of primary key", + ErrorCodes::BAD_TTL_EXPRESSION); + } + + for (const auto & [name, value] : ttl_element->group_by_aggregations) + { + if (primary_key_columns_set.count(name)) + throw Exception( + "Can not set custom aggregation for column in primary key in TTL Expression", + ErrorCodes::BAD_TTL_EXPRESSION); + + aggregation_columns_set.insert(name); + } + + if (aggregation_columns_set.size() != ttl_element->group_by_aggregations.size()) + throw Exception( + "Multiple aggregations set for one column in TTL Expression", + ErrorCodes::BAD_TTL_EXPRESSION); + + + result.group_by_keys = Names(pk_columns.begin(), pk_columns.begin() + ttl_element->group_by_key.size()); + + auto aggregations = ttl_element->group_by_aggregations; + + for (size_t i = 0; i < pk_columns.size(); ++i) + { + ASTPtr value = primary_key.expression_list_ast->children[i]->clone(); + + if (i >= ttl_element->group_by_key.size()) + { + ASTPtr value_max = makeASTFunction("max", value->clone()); + aggregations.emplace_back(value->getColumnName(), std::move(value_max)); + } + + if (value->as()) + { + auto syntax_result = SyntaxAnalyzer(context).analyze(value, columns.getAllPhysical(), {}, true); + auto expr_actions = ExpressionAnalyzer(value, syntax_result, context).getActions(false); + for (const auto & column : expr_actions->getRequiredColumns()) + { + if (i < ttl_element->group_by_key.size()) + { + ASTPtr expr = makeASTFunction("any", std::make_shared(column)); + aggregations.emplace_back(column, std::move(expr)); + } + else + { + ASTPtr expr = makeASTFunction("argMax", std::make_shared(column), value->clone()); + aggregations.emplace_back(column, std::move(expr)); + } + } + } + } + + for (const auto & column : columns.getAllPhysical()) + { + if (!primary_key_columns_set.count(column.name) && !aggregation_columns_set.count(column.name)) + { + ASTPtr expr = makeASTFunction("any", std::make_shared(column.name)); + aggregations.emplace_back(column.name, std::move(expr)); + } + } + + for (auto [name, value] : aggregations) + { + auto syntax_result = SyntaxAnalyzer(context).analyze(value, columns.getAllPhysical(), {}, true); + auto expr_analyzer = ExpressionAnalyzer(value, syntax_result, context); + + result.group_by_aggregations.emplace_back(name, value->getColumnName(), expr_analyzer.getActions(false)); + + for (const auto & descr : expr_analyzer.getAnalyzedData().aggregate_descriptions) + result.aggregate_descriptions.push_back(descr); + } + } + } + + checkTTLExpression(result.expression, result.result_column); + + + return result; +} + +} diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h new file mode 100644 index 00000000000..59829dbecfd --- /dev/null +++ b/src/Storages/TTLDescription.h @@ -0,0 +1,69 @@ +#pragma once +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +/// Common struct for TTL record in storage +struct TTLDescription +{ + TTLMode mode; + + /// Expression part of TTL AST: + /// TTL d + INTERVAL 1 DAY + /// ^~~~~expression~~~~^ + ASTPtr expression_ast; + + /// Expresion actions evaluated from AST + ExpressionActionsPtr expression; + + /// Result column of this TTL expression + String result_column; + + ExpressionActionsPtr where_expression; + + String where_result_column; + + Names group_by_keys; + + std::vector> group_by_aggregations; + AggregateDescriptions aggregate_descriptions; + + /// Destination type, only valid for table TTLs. + /// For example DISK or VOLUME + DataDestinationType destination_type; + + /// Name of destination disk or volume + String destination_name; + + /// Parse TTL structure from definition. Able to parse both column and table + /// TTLs. + static TTLDescription getTTLFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context, const StorageMetadataKeyField & primary_key); +}; + +/// Mapping from column name to column TTL +using TTLColumnsDescription = std::unordered_map; +using TTLDescriptions = std::vector; + +/// Common TTL for all table. Specified after defining the table columns. +struct TTLTableDescription +{ + /// Definition. Include all parts of TTL: + /// TTL d + INTERVAL 1 day TO VOLUME 'disk1' + /// ^~~~~~~~~~~~~~~definition~~~~~~~~~~~~~~~^ + ASTPtr definition_ast; + + /// Rows removing TTL + TTLDescription rows_ttl; + + /// Moving data TTL (to other disks or volumes) + TTLDescriptions move_ttl; +}; + +} From acc3e6b15a82c1ca93eb7e2061c00ae5a5ca72a8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 28 May 2020 19:31:27 +0300 Subject: [PATCH 78/79] Better structure --- src/DataStreams/TTLBlockInputStream.cpp | 10 ++++---- src/Storages/TTLDescription.cpp | 3 ++- src/Storages/TTLDescription.h | 32 +++++++++++++++++++++++-- src/Storages/ya.make | 1 + 4 files changed, 38 insertions(+), 8 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 87cab76ce8c..c6542763533 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -277,18 +277,18 @@ void TTLBlockInputStream::finalizeAggregates(MutableColumns & result_columns) auto aggregated_res = aggregator->convertToBlocks(agg_result, true, 1); for (auto & agg_block : aggregated_res) { - for (const auto & it : storage.getRowsTTL().group_by_aggregations) - std::get<2>(it)->execute(agg_block); + for (const auto & it : storage.getRowsTTL().set_parts) + it.expression->execute(agg_block); for (const auto & name : storage.getRowsTTL().group_by_keys) { const IColumn * values_column = agg_block.getByName(name).column.get(); auto & result_column = result_columns[header.getPositionByName(name)]; result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); } - for (const auto & it : storage.getRowsTTL().group_by_aggregations) + for (const auto & it : storage.getRowsTTL().set_parts) { - const IColumn * values_column = agg_block.getByName(get<1>(it)).column.get(); - auto & result_column = result_columns[header.getPositionByName(std::get<0>(it))]; + const IColumn * values_column = agg_block.getByName(it.expression_result_column_name).column.get(); + auto & result_column = result_columns[header.getPositionByName(it.column_name)]; result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); } } diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 0efe6076174..92fcf400dd3 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -180,7 +180,8 @@ TTLDescription TTLDescription::getTTLFromAST( auto syntax_result = SyntaxAnalyzer(context).analyze(value, columns.getAllPhysical(), {}, true); auto expr_analyzer = ExpressionAnalyzer(value, syntax_result, context); - result.group_by_aggregations.emplace_back(name, value->getColumnName(), expr_analyzer.getActions(false)); + result.set_parts.emplace_back(TTLSetPartDescription{ + name, value->getColumnName(), expr_analyzer.getActions(false)}); for (const auto & descr : expr_analyzer.getAnalyzedData().aggregate_descriptions) result.aggregate_descriptions.push_back(descr); diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index 59829dbecfd..d0e669ef4cf 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -10,6 +10,24 @@ namespace DB { +struct TTLSetPartDescription +{ + /// Name of column in set part of ttl expression + /// x = sum(y) + /// ^ + String column_name; + + /// Name of column on the right hand of the set part of TTL expression + /// x = sum(y) + /// ^~~~~~^ + String expression_result_column_name; + + /// Expressions to calculate the value of set expression + ExpressionActionsPtr expression; +}; + +using TTLSetPartDescriptions = std::vector; + /// Common struct for TTL record in storage struct TTLDescription { @@ -17,7 +35,7 @@ struct TTLDescription /// Expression part of TTL AST: /// TTL d + INTERVAL 1 DAY - /// ^~~~~expression~~~~^ + /// ^~~~~~~~~~~~~~~~~~~^ ASTPtr expression_ast; /// Expresion actions evaluated from AST @@ -26,13 +44,23 @@ struct TTLDescription /// Result column of this TTL expression String result_column; + /// WHERE part in TTL expression + /// TTL ... WHERE x % 10 == 0 and y > 5 + /// ^~~~~~~~~~~~~~~~~~~~~~^ ExpressionActionsPtr where_expression; + /// Name of result column from WHERE expression String where_result_column; + /// Names of key columns in GROUP BY expression + /// TTL ... GROUP BY toDate(d), x SET ... + /// ^~~~~~~~~~~~^ Names group_by_keys; - std::vector> group_by_aggregations; + /// SET parts of TTL expression + TTLSetPartDescriptions set_parts; + + /// Aggregate descriptions for GROUP BY in TTL AggregateDescriptions aggregate_descriptions; /// Destination type, only valid for table TTLs. diff --git a/src/Storages/ya.make b/src/Storages/ya.make index ffa3924d11a..3e6943bac69 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -173,6 +173,7 @@ SRCS( StorageXDBC.cpp transformQueryForExternalDatabase.cpp VirtualColumnUtils.cpp + TTLDescriptions.cpp ) END() From c5fe87e123dca8a5fdcaace0f349209e51c9c11b Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 28 May 2020 19:32:35 +0300 Subject: [PATCH 79/79] Less includes --- src/Storages/StorageInMemoryMetadata.cpp | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 0111c0632c8..6713519151f 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -5,22 +5,11 @@ #include #include #include -#include -#include - -#include -#include namespace DB { -namespace ErrorCodes -{ - extern const int BAD_TTL_EXPRESSION; - extern const int BAD_ARGUMENTS; -}; - StorageInMemoryMetadata::StorageInMemoryMetadata( const ColumnsDescription & columns_, const IndicesDescription & indices_,