From 40504f6a6e9b54bdcdb0c63a5724648bf5bc04f5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 31 Jul 2020 17:57:00 +0300 Subject: [PATCH 01/32] Simpler version of #12999 w/o `pos` changes --- programs/client/Client.cpp | 126 +++++++++++++----- ...06_insert_values_and_expressions.reference | 2 + .../00306_insert_values_and_expressions.sql | 9 ++ 3 files changed, 103 insertions(+), 34 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 797342a1b44..78a6d7fe2d9 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -908,74 +908,127 @@ private: return processMultiQuery(text); } - bool processMultiQuery(const String & text) + bool processMultiQuery(const String & all_queries_text) { const bool test_mode = config().has("testmode"); { /// disable logs if expects errors - TestHint test_hint(test_mode, text); + TestHint test_hint(test_mode, all_queries_text); if (test_hint.clientError() || test_hint.serverError()) processTextAsSingleQuery("SET send_logs_level = 'none'"); } /// Several queries separated by ';'. /// INSERT data is ended by the end of line, not ';'. + /// An exception is VALUES format where we also support semicolon in + /// addition to end of line. - const char * begin = text.data(); - const char * end = begin + text.size(); + const char * this_query_begin = all_queries_text.data(); + const char * all_queries_end = all_queries_text.data() + all_queries_text.size(); - while (begin < end) + while (this_query_begin < all_queries_end) { - const char * pos = begin; - ASTPtr orig_ast = parseQuery(pos, end, true); + // Use the token iterator to skip any whitespace, semicolons and + // comments at the beginning of the query. An example from regression + // tests: + // insert into table t values ('invalid'); -- { serverError 469 } + // select 1 + // Here the test hint comment gets parsed as a part of second query. + // We parse the `INSERT VALUES` up to the semicolon, and the rest + // looks like a two-line query: + // -- { serverError 469 } + // select 1 + // and we expect it to fail with error 469, but this hint is actually + // for the previous query. Test hints should go after the query, so + // we can fix this by skipping leading comments. Token iterator skips + // comments and whitespace by itself, so we only have to check for + // semicolons. + // The code block is to limit visibility of `tokens` because we have + // another such variable further down the code, and get warnings for + // that. + { + Tokens tokens(this_query_begin, all_queries_end); + IParser::Pos token_iterator(tokens, + context.getSettingsRef().max_parser_depth); + while (token_iterator->type == TokenType::Semicolon + && token_iterator.isValid()) + { + ++token_iterator; + } + this_query_begin = token_iterator->begin; + if (this_query_begin >= all_queries_end) + { + break; + } + } - if (!orig_ast) + // Try to parse the query. + const char * this_query_end = this_query_begin; + parsed_query = parseQuery(this_query_end, all_queries_end, true); + + if (!parsed_query) { if (ignore_error) { - Tokens tokens(begin, end); + Tokens tokens(this_query_begin, all_queries_end); IParser::Pos token_iterator(tokens, context.getSettingsRef().max_parser_depth); while (token_iterator->type != TokenType::Semicolon && token_iterator.isValid()) ++token_iterator; - begin = token_iterator->end; + this_query_begin = token_iterator->end; continue; } return true; } - auto * insert = orig_ast->as(); - - if (insert && insert->data) + // INSERT queries may have the inserted data in the query text + // that follow the query itself, e.g. "insert into t format CSV 1;2". + // They need special handling. First of all, here we find where the + // inserted data ends. In multy-query mode, it is delimited by a + // newline. + // The VALUES format needs even more handling -- we also allow the + // data to be delimited by semicolon. This case is handled later by + // the format parser itself. + auto * insert_ast = parsed_query->as(); + if (insert_ast && insert_ast->data) { - pos = find_first_symbols<'\n'>(insert->data, end); - insert->end = pos; + this_query_end = find_first_symbols<'\n'>(insert_ast->data, all_queries_end); + insert_ast->end = this_query_end; + query_to_send = all_queries_text.substr( + this_query_begin - all_queries_text.data(), + insert_ast->data - this_query_begin); + } + else + { + query_to_send = all_queries_text.substr( + this_query_begin - all_queries_text.data(), + this_query_end - this_query_begin); } - String str = text.substr(begin - text.data(), pos - begin); + // full_query is the query + inline INSERT data. + full_query = all_queries_text.substr( + this_query_begin - all_queries_text.data(), + this_query_end - this_query_begin); - begin = pos; - while (isWhitespaceASCII(*begin) || *begin == ';') - ++begin; - - TestHint test_hint(test_mode, str); + // Look for the hint in the text of query + insert data, if any. + // e.g. insert into t format CSV 'a' -- { serverError 123 }. + TestHint test_hint(test_mode, full_query); expected_client_error = test_hint.clientError(); expected_server_error = test_hint.serverError(); try { - auto ast_to_process = orig_ast; - if (insert && insert->data) + processParsedSingleQuery(); + + if (insert_ast && insert_ast->data) { - ast_to_process = nullptr; - processTextAsSingleQuery(str); - } - else - { - parsed_query = ast_to_process; - full_query = str; - query_to_send = str; - processParsedSingleQuery(); + // For VALUES format: use the end of inline data as reported + // by the format parser (it is saved in sendData()). This + // allows us to handle queries like: + // insert into t values (1); select 1 + //, where the inline data is delimited by semicolon and not + // by a newline. + this_query_end = parsed_query->as()->end; } } catch (...) @@ -983,7 +1036,7 @@ private: last_exception_received_from_server = std::make_unique(getCurrentExceptionMessage(true), getCurrentExceptionCode()); actual_client_error = last_exception_received_from_server->code(); if (!ignore_error && (!actual_client_error || actual_client_error != expected_client_error)) - std::cerr << "Error on processing query: " << str << std::endl << last_exception_received_from_server->message(); + std::cerr << "Error on processing query: " << full_query << std::endl << last_exception_received_from_server->message(); received_exception_from_server = true; } @@ -997,6 +1050,8 @@ private: else return false; } + + this_query_begin = this_query_end; } return true; @@ -1407,7 +1462,7 @@ private: void sendData(Block & sample, const ColumnsDescription & columns_description) { /// If INSERT data must be sent. - const auto * parsed_insert_query = parsed_query->as(); + auto * parsed_insert_query = parsed_query->as(); if (!parsed_insert_query) return; @@ -1416,6 +1471,9 @@ private: /// Send data contained in the query. ReadBufferFromMemory data_in(parsed_insert_query->data, parsed_insert_query->end - parsed_insert_query->data); sendDataFrom(data_in, sample, columns_description); + // Remember where the data ended. We use this info later to determine + // where the next query begins. + parsed_insert_query->end = data_in.buffer().begin() + data_in.count(); } else if (!is_interactive) { diff --git a/tests/queries/0_stateless/00306_insert_values_and_expressions.reference b/tests/queries/0_stateless/00306_insert_values_and_expressions.reference index 960773dc489..e80a28accf4 100644 --- a/tests/queries/0_stateless/00306_insert_values_and_expressions.reference +++ b/tests/queries/0_stateless/00306_insert_values_and_expressions.reference @@ -2,3 +2,5 @@ 2 Hello, world 00000000-0000-0000-0000-000000000000 2016-01-02 2016-01-02 03:04:00 [0,1] 3 hello, world! ab41bdd6-5cd4-11e7-907b-a6006ad3dba0 2016-01-03 2016-01-02 03:00:00 [] 4 World ab41bdd6-5cd4-11e7-907b-a6006ad3dba0 2016-01-04 2016-12-11 10:09:08 [3,2,1] +11111 +1 diff --git a/tests/queries/0_stateless/00306_insert_values_and_expressions.sql b/tests/queries/0_stateless/00306_insert_values_and_expressions.sql index a57e9e69fe6..10a1415f287 100644 --- a/tests/queries/0_stateless/00306_insert_values_and_expressions.sql +++ b/tests/queries/0_stateless/00306_insert_values_and_expressions.sql @@ -5,3 +5,12 @@ INSERT INTO insert VALUES (1, 'Hello', 'ab41bdd6-5cd4-11e7-907b-a6006ad3dba0', ' SELECT * FROM insert ORDER BY i; DROP TABLE insert; + +-- Test the case where the VALUES are delimited by semicolon and a query follows +-- w/o newline. With most formats the query in the same line would be ignored or +-- lead to an error, but VALUES are an exception and support semicolon delimiter, +-- in addition to the newline. +create table if not exists t_306 (a int) engine Memory; +insert into t_306 values (1); select 11111; +select * from t_306; +drop table if exists t_306; From 405a6fb08fa22a9e063dd5e48e7ee6060f718749 Mon Sep 17 00:00:00 2001 From: hexiaoting <“hewenting_ict@163.com”> Date: Mon, 17 Aug 2020 18:20:23 +0800 Subject: [PATCH 02/32] New feature: LineAsString format. #13630 --- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatFactory.h | 1 + .../Impl/LineAsStringRowInputFormat.cpp | 101 ++++++++++++++++++ .../Formats/Impl/LineAsStringRowInputFormat.h | 31 ++++++ src/Processors/ya.make | 1 + 5 files changed, 135 insertions(+) create mode 100644 src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/LineAsStringRowInputFormat.h diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 5256ab2b321..f996e3d8cf2 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -365,6 +365,7 @@ FormatFactory::FormatFactory() registerInputFormatProcessorMsgPack(*this); registerOutputFormatProcessorMsgPack(*this); registerInputFormatProcessorJSONAsString(*this); + registerInputFormatProcessorLineAsString(*this); registerFileSegmentationEngineTabSeparated(*this); registerFileSegmentationEngineCSV(*this); diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index ea4004c191f..610cf8105b8 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -210,5 +210,6 @@ void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory); void registerInputFormatProcessorCapnProto(FormatFactory & factory); void registerInputFormatProcessorRegexp(FormatFactory & factory); void registerInputFormatProcessorJSONAsString(FormatFactory & factory); +void registerInputFormatProcessorLineAsString(FormatFactory & factory); } diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp new file mode 100644 index 00000000000..a28b3903724 --- /dev/null +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -0,0 +1,101 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INCORRECT_DATA; +} + +LineAsStringRowInputFormat::LineAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) : + IRowInputFormat(header_, in_, std::move(params_)), buf(in) +{ + if (header_.columns() > 1 || header_.getDataTypes()[0]->getTypeId() != TypeIndex::String) + { + throw Exception("This input format is only suitable for tables with a single column of type String.", ErrorCodes::LOGICAL_ERROR); + } +} + +void LineAsStringRowInputFormat::resetParser() +{ + IRowInputFormat::resetParser(); + buf.reset(); +} + +void LineAsStringRowInputFormat::readLineObject(IColumn & column) +{ + PeekableReadBufferCheckpoint checkpoint{buf}; + size_t balance = 0; + + if (*buf.position() != '"') + throw Exception("Line object must begin with '\"'.", ErrorCodes::INCORRECT_DATA); + + ++buf.position(); + ++balance; + + char * pos; + + while (balance) + { + if (buf.eof()) + throw Exception("Unexpected end of file while parsing Line object.", ErrorCodes::INCORRECT_DATA); + + pos = find_last_symbols_or_null<'"', '\\'>(buf.position(), buf.buffer().end()); + buf.position() = pos; + if (buf.position() == buf.buffer().end()) + continue; + else if (*buf.position() == '"') + { + --balance; + ++buf.position(); + } + else if (*buf.position() == '\\') + { + ++buf.position(); + if (!buf.eof()) + { + ++buf.position(); + } + } + + } + buf.makeContinuousMemoryFromCheckpointToPos(); + char * end = buf.position(); + buf.rollbackToCheckpoint(); + column.insertData(buf.position(), end - buf.position()); + buf.position() = end; +} + +bool LineAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) +{ + skipWhitespaceIfAny(buf); + + if (!buf.eof()) + readLineObject(*columns[0]); + + skipWhitespaceIfAny(buf); + if (!buf.eof() && *buf.position() == ',') + ++buf.position(); + skipWhitespaceIfAny(buf); + + return !buf.eof(); +} + +void registerInputFormatProcessorLineAsString(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("LineAsString", []( + ReadBuffer & buf, + const Block & sample, + const RowInputFormatParams & params, + const FormatSettings &) + { + return std::make_shared(sample, buf, params); + }); +} + +} diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h new file mode 100644 index 00000000000..a31dce1cc4a --- /dev/null +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ReadBuffer; + +/// This format parses a sequence of Line objects separated by newlines, spaces and/or comma. +/// Each Line object is parsed as a whole to string. +/// This format can only parse a table with single field of type String. + +class LineAsStringRowInputFormat : public IRowInputFormat +{ +public: + LineAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_); + + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + String getName() const override { return "LineAsStringRowInputFormat"; } + void resetParser() override; + +private: + void readLineObject(IColumn & column); + + PeekableReadBuffer buf; +}; + +} diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 4c25ad5bf3f..081b1d5ba1f 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -23,6 +23,7 @@ SRCS( Formats/Impl/ConstantExpressionTemplate.cpp Formats/Impl/CSVRowInputFormat.cpp Formats/Impl/CSVRowOutputFormat.cpp + Formats/Impl/LineAsStringRowInputFormat.cpp Formats/Impl/JSONAsStringRowInputFormat.cpp Formats/Impl/JSONCompactEachRowRowInputFormat.cpp Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp From 0451d5729323b7f46d79336fea4f0982bb1662ae Mon Sep 17 00:00:00 2001 From: hexiaoting <“hewenting_ict@163.com”> Date: Tue, 18 Aug 2020 10:35:08 +0800 Subject: [PATCH 03/32] Add new feature: LineAsString Format --- src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp | 6 ++++++ .../0_stateless/01460_line_as_string_format.reference | 1 + tests/queries/0_stateless/01460_line_as_string_format.sql | 5 +++++ 3 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/01460_line_as_string_format.reference create mode 100644 tests/queries/0_stateless/01460_line_as_string_format.sql diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index a28b3903724..36844fa700b 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -32,6 +32,12 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) PeekableReadBufferCheckpoint checkpoint{buf}; size_t balance = 0; + if (*buf.position() == ';') { + ++buf.position(); + if(buf.eof()) + return; + } + if (*buf.position() != '"') throw Exception("Line object must begin with '\"'.", ErrorCodes::INCORRECT_DATA); diff --git a/tests/queries/0_stateless/01460_line_as_string_format.reference b/tests/queries/0_stateless/01460_line_as_string_format.reference new file mode 100644 index 00000000000..989f8ac0292 --- /dev/null +++ b/tests/queries/0_stateless/01460_line_as_string_format.reference @@ -0,0 +1 @@ +"I love apple","I love banana","I love pear" diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sql b/tests/queries/0_stateless/01460_line_as_string_format.sql new file mode 100644 index 00000000000..e5518a828d0 --- /dev/null +++ b/tests/queries/0_stateless/01460_line_as_string_format.sql @@ -0,0 +1,5 @@ +DROP TABLE IF EXISTS line_as_string; +CREATE TABLE line_as_string (field String) ENGINE = Memory; +INSERT INTO line_as_string FORMAT LineAsString "I love apple","I love banana","I love pear"; +SELECT * FROM line_as_string; +DROP TABLE line_as_string; From e9be2f14ea8ac45f11c7c65b6c36646b64a5b390 Mon Sep 17 00:00:00 2001 From: hexiaoting <“hewenting_ict@163.com”> Date: Wed, 19 Aug 2020 11:50:43 +0800 Subject: [PATCH 04/32] fix implementation for \n separated lines --- .../Impl/LineAsStringRowInputFormat.cpp | 45 ++++++------------- .../01460_line_as_string_format.reference | 7 ++- .../01460_line_as_string_format.sh | 19 ++++++++ .../01460_line_as_string_format.sql | 5 --- 4 files changed, 38 insertions(+), 38 deletions(-) create mode 100755 tests/queries/0_stateless/01460_line_as_string_format.sh delete mode 100644 tests/queries/0_stateless/01460_line_as_string_format.sql diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index 36844fa700b..27bc71d764d 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -30,35 +30,22 @@ void LineAsStringRowInputFormat::resetParser() void LineAsStringRowInputFormat::readLineObject(IColumn & column) { PeekableReadBufferCheckpoint checkpoint{buf}; - size_t balance = 0; - - if (*buf.position() == ';') { - ++buf.position(); - if(buf.eof()) - return; - } - - if (*buf.position() != '"') - throw Exception("Line object must begin with '\"'.", ErrorCodes::INCORRECT_DATA); - - ++buf.position(); - ++balance; + bool newline = true; + bool over = false; char * pos; - while (balance) + while (newline) { - if (buf.eof()) - throw Exception("Unexpected end of file while parsing Line object.", ErrorCodes::INCORRECT_DATA); - - pos = find_last_symbols_or_null<'"', '\\'>(buf.position(), buf.buffer().end()); + pos = find_first_symbols<'\n', '\\'>(buf.position(), buf.buffer().end()); buf.position() = pos; - if (buf.position() == buf.buffer().end()) - continue; - else if (*buf.position() == '"') + if (buf.position() == buf.buffer().end()) { + over = true; + break; + } + else if (*buf.position() == '\n') { - --balance; - ++buf.position(); + newline = false; } else if (*buf.position() == '\\') { @@ -70,25 +57,19 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) } } + buf.makeContinuousMemoryFromCheckpointToPos(); - char * end = buf.position(); + char * end = over ? buf.position(): ++buf.position(); buf.rollbackToCheckpoint(); - column.insertData(buf.position(), end - buf.position()); + column.insertData(buf.position(), end - (over ? 0 : 1) - buf.position()); buf.position() = end; } bool LineAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { - skipWhitespaceIfAny(buf); - if (!buf.eof()) readLineObject(*columns[0]); - skipWhitespaceIfAny(buf); - if (!buf.eof() && *buf.position() == ',') - ++buf.position(); - skipWhitespaceIfAny(buf); - return !buf.eof(); } diff --git a/tests/queries/0_stateless/01460_line_as_string_format.reference b/tests/queries/0_stateless/01460_line_as_string_format.reference index 989f8ac0292..dec67eb2e0a 100644 --- a/tests/queries/0_stateless/01460_line_as_string_format.reference +++ b/tests/queries/0_stateless/01460_line_as_string_format.reference @@ -1 +1,6 @@ -"I love apple","I love banana","I love pear" +"id" : 1, +"date" : "01.01.2020", +"string" : "123{{{\\"\\\\", +"array" : [1, 2, 3], + +Finally implement this new feature. diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sh b/tests/queries/0_stateless/01460_line_as_string_format.sh new file mode 100755 index 00000000000..a985bc207a8 --- /dev/null +++ b/tests/queries/0_stateless/01460_line_as_string_format.sh @@ -0,0 +1,19 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS line_as_string"; + +$CLICKHOUSE_CLIENT --query="CREATE TABLE line_as_string(field String) ENGINE = Memory"; + +echo '"id" : 1, +"date" : "01.01.2020", +"string" : "123{{{\"\\", +"array" : [1, 2, 3], + +Finally implement this new feature.' | $CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string FORMAT LineAsString"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM line_as_string"; +$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string" + diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sql b/tests/queries/0_stateless/01460_line_as_string_format.sql deleted file mode 100644 index e5518a828d0..00000000000 --- a/tests/queries/0_stateless/01460_line_as_string_format.sql +++ /dev/null @@ -1,5 +0,0 @@ -DROP TABLE IF EXISTS line_as_string; -CREATE TABLE line_as_string (field String) ENGINE = Memory; -INSERT INTO line_as_string FORMAT LineAsString "I love apple","I love banana","I love pear"; -SELECT * FROM line_as_string; -DROP TABLE line_as_string; From 25ca5e91bd0f3074c8d7d0874e125d2dcc611889 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 16:09:32 -0700 Subject: [PATCH 05/32] AsynchronousMetricLog - add event_time_microseconds column --- src/Core/Field.h | 3 ++- src/Interpreters/AsynchronousMetricLog.cpp | 17 +++++++++++++---- src/Interpreters/AsynchronousMetricLog.h | 1 + 3 files changed, 16 insertions(+), 5 deletions(-) diff --git a/src/Core/Field.h b/src/Core/Field.h index 0bfdf597543..8973d106c0b 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -768,7 +768,8 @@ T & Field::get() // Disregard signedness when converting between int64 types. constexpr Field::Types::Which target = TypeToEnum>::value; assert(target == which - || (isInt64FieldType(target) && isInt64FieldType(which))); + || (isInt64FieldType(target) && isInt64FieldType(which)) + || target == Field::Types::Decimal64 /* DateTime64 fields */); #endif ValueType * MAY_ALIAS ptr = reinterpret_cast(&storage); diff --git a/src/Interpreters/AsynchronousMetricLog.cpp b/src/Interpreters/AsynchronousMetricLog.cpp index e4415773655..d2c81c9dfc5 100644 --- a/src/Interpreters/AsynchronousMetricLog.cpp +++ b/src/Interpreters/AsynchronousMetricLog.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -13,10 +14,11 @@ Block AsynchronousMetricLogElement::createBlock() { ColumnsWithTypeAndName columns; - columns.emplace_back(std::make_shared(), "event_date"); - columns.emplace_back(std::make_shared(), "event_time"); - columns.emplace_back(std::make_shared(), "name"); - columns.emplace_back(std::make_shared(), "value"); + columns.emplace_back(std::make_shared(), "event_date"); + columns.emplace_back(std::make_shared(), "event_time"); + columns.emplace_back(std::make_shared(6), "event_time_microseconds"); + columns.emplace_back(std::make_shared(), "name"); + columns.emplace_back(std::make_shared(), "value"); return Block(columns); } @@ -28,6 +30,7 @@ void AsynchronousMetricLogElement::appendToBlock(MutableColumns & columns) const columns[column_idx++]->insert(event_date); columns[column_idx++]->insert(event_time); + columns[column_idx++]->insert(event_time_microseconds); columns[column_idx++]->insert(metric_name); columns[column_idx++]->insert(value); } @@ -38,6 +41,11 @@ inline UInt64 time_in_milliseconds(std::chrono::time_point(timepoint.time_since_epoch()).count(); } +inline UInt64 time_in_microseconds(std::chrono::time_point timepoint) +{ + return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); +} + inline UInt64 time_in_seconds(std::chrono::time_point timepoint) { @@ -50,6 +58,7 @@ void AsynchronousMetricLog::addValues(const AsynchronousMetricValues & values) const auto now = std::chrono::system_clock::now(); element.event_time = time_in_seconds(now); + element.event_time_microseconds = time_in_microseconds(now); element.event_date = DateLUT::instance().toDayNum(element.event_time); for (const auto & [key, value] : values) diff --git a/src/Interpreters/AsynchronousMetricLog.h b/src/Interpreters/AsynchronousMetricLog.h index b7d6aab95b6..0c02244246e 100644 --- a/src/Interpreters/AsynchronousMetricLog.h +++ b/src/Interpreters/AsynchronousMetricLog.h @@ -22,6 +22,7 @@ struct AsynchronousMetricLogElement { UInt16 event_date; time_t event_time; + UInt64 event_time_microseconds; std::string metric_name; double value; From 6a5b885ac1167dd8bba4a0c8b091289dd4c1e79e Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 16:17:28 -0700 Subject: [PATCH 06/32] AsynchronousMetricLog - add tests for event_time_microseconds column --- .../0_stateless/01473_event_time_microseconds.reference | 2 ++ tests/queries/0_stateless/01473_event_time_microseconds.sql | 5 +++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/01473_event_time_microseconds.reference create mode 100644 tests/queries/0_stateless/01473_event_time_microseconds.sql diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference new file mode 100644 index 00000000000..6c3b6ec5c6c --- /dev/null +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -0,0 +1,2 @@ +'01473_asynchronous_metric_log_event_start_time_milliseconds_test' +ok \ No newline at end of file diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql new file mode 100644 index 00000000000..af38cf4ca70 --- /dev/null +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -0,0 +1,5 @@ +set log_queries = 1; + +select '01473_asynchronous_metric_log_event_start_time_milliseconds_test'; +system flush logs; +SELECT If((select count(event_time_microseconds) from system.asynchronous_metric_log) > 0, 'ok', 'fail'); -- success From 1c1f50c6b665b96d7ba5742a60c770081299213d Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 16:19:38 -0700 Subject: [PATCH 07/32] MetricLog - add event_time_microseconds column --- src/Interpreters/MetricLog.cpp | 14 +++++++++++--- src/Interpreters/MetricLog.h | 1 + 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/MetricLog.cpp b/src/Interpreters/MetricLog.cpp index 69fcc4917b9..ce5d5793b87 100644 --- a/src/Interpreters/MetricLog.cpp +++ b/src/Interpreters/MetricLog.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -11,9 +12,10 @@ Block MetricLogElement::createBlock() { ColumnsWithTypeAndName columns_with_type_and_name; - columns_with_type_and_name.emplace_back(std::make_shared(), "event_date"); - columns_with_type_and_name.emplace_back(std::make_shared(), "event_time"); - columns_with_type_and_name.emplace_back(std::make_shared(), "milliseconds"); + columns_with_type_and_name.emplace_back(std::make_shared(), "event_date"); + columns_with_type_and_name.emplace_back(std::make_shared(), "event_time"); + columns_with_type_and_name.emplace_back(std::make_shared(6), "event_time_microseconds"); + columns_with_type_and_name.emplace_back(std::make_shared(), "milliseconds"); for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) { @@ -41,6 +43,7 @@ void MetricLogElement::appendToBlock(MutableColumns & columns) const columns[column_idx++]->insert(DateLUT::instance().toDayNum(event_time)); columns[column_idx++]->insert(event_time); + columns[column_idx++]->insert(event_time_microseconds); columns[column_idx++]->insert(milliseconds); for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) @@ -80,6 +83,10 @@ inline UInt64 time_in_milliseconds(std::chrono::time_point(timepoint.time_since_epoch()).count(); } +inline UInt64 time_in_microseconds(std::chrono::time_point timepoint) +{ + return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); +} inline UInt64 time_in_seconds(std::chrono::time_point timepoint) { @@ -102,6 +109,7 @@ void MetricLog::metricThreadFunction() MetricLogElement elem; elem.event_time = std::chrono::system_clock::to_time_t(current_time); + elem.event_time_microseconds = time_in_microseconds(current_time); elem.milliseconds = time_in_milliseconds(current_time) - time_in_seconds(current_time) * 1000; elem.profile_events.resize(ProfileEvents::end()); diff --git a/src/Interpreters/MetricLog.h b/src/Interpreters/MetricLog.h index 7774a45d7e1..f52d078bdc9 100644 --- a/src/Interpreters/MetricLog.h +++ b/src/Interpreters/MetricLog.h @@ -18,6 +18,7 @@ namespace DB struct MetricLogElement { time_t event_time{}; + UInt64 event_time_microseconds{}; UInt64 milliseconds{}; std::vector profile_events; From ec0d0243cc994f91fcc8a6a0fd36bde22f853af5 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 16:21:51 -0700 Subject: [PATCH 08/32] MetricLog - add tests for event_time_microseconds field --- .../0_stateless/01473_event_time_microseconds.reference | 6 ++++-- tests/queries/0_stateless/01473_event_time_microseconds.sql | 4 ++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference index 6c3b6ec5c6c..cac87f32a29 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.reference +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -1,2 +1,4 @@ -'01473_asynchronous_metric_log_event_start_time_milliseconds_test' -ok \ No newline at end of file +01473_asynchronous_metric_log_event_start_time_milliseconds_test +ok +01473_metric_log_event_start_time_milliseconds_test +ok diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index af38cf4ca70..6a13d6e1543 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -3,3 +3,7 @@ set log_queries = 1; select '01473_asynchronous_metric_log_event_start_time_milliseconds_test'; system flush logs; SELECT If((select count(event_time_microseconds) from system.asynchronous_metric_log) > 0, 'ok', 'fail'); -- success + +select '01473_metric_log_event_start_time_milliseconds_test'; +system flush logs; +SELECT If((select count(event_time_microseconds) from system.metric_log) > 0, 'ok', 'fail'); -- success From 9de49d130f733c0fcc00d8f7bd85c4fb2eecbfc2 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 16:36:38 -0700 Subject: [PATCH 09/32] AsynchronousMetricLog & MetricLog - update docs with examples --- .../system-tables/asynchronous_metric_log.md | 25 ++++++------ .../en/operations/system-tables/metric_log.md | 40 +++++++++---------- 2 files changed, 33 insertions(+), 32 deletions(-) diff --git a/docs/en/operations/system-tables/asynchronous_metric_log.md b/docs/en/operations/system-tables/asynchronous_metric_log.md index 6b1d71e1ca6..75607cc30b0 100644 --- a/docs/en/operations/system-tables/asynchronous_metric_log.md +++ b/docs/en/operations/system-tables/asynchronous_metric_log.md @@ -6,6 +6,7 @@ Columns: - `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time. +- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Event time with microseconds resolution. - `name` ([String](../../sql-reference/data-types/string.md)) — Metric name. - `value` ([Float64](../../sql-reference/data-types/float.md)) — Metric value. @@ -16,18 +17,18 @@ SELECT * FROM system.asynchronous_metric_log LIMIT 10 ``` ``` text -┌─event_date─┬──────────event_time─┬─name─────────────────────────────────────┬────value─┐ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.arenas.all.pmuzzy │ 0 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.arenas.all.pdirty │ 4214 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.background_thread.run_intervals │ 0 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.background_thread.num_runs │ 0 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.retained │ 17657856 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.mapped │ 71471104 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.resident │ 61538304 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.metadata │ 6199264 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.allocated │ 38074336 │ -│ 2020-06-22 │ 2020-06-22 06:57:30 │ jemalloc.epoch │ 2 │ -└────────────┴─────────────────────┴──────────────────────────────────────────┴──────────┘ +┌─event_date─┬──────────event_time─┬────event_time_microseconds─┬─name─────────────────────────────────────┬─────value─┐ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ CPUFrequencyMHz_0 │ 2120.9 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.arenas.all.pmuzzy │ 743 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.arenas.all.pdirty │ 26288 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.background_thread.run_intervals │ 0 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.background_thread.num_runs │ 0 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.retained │ 60694528 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.mapped │ 303161344 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.resident │ 260931584 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.metadata │ 12079488 │ +│ 2020-09-05 │ 2020-09-05 15:56:30 │ 2020-09-05 15:56:30.025227 │ jemalloc.allocated │ 133756128 │ +└────────────┴─────────────────────┴────────────────────────────┴──────────────────────────────────────────┴───────────┘ ``` **See Also** diff --git a/docs/en/operations/system-tables/metric_log.md b/docs/en/operations/system-tables/metric_log.md index 9ccf61291d2..063fe81923b 100644 --- a/docs/en/operations/system-tables/metric_log.md +++ b/docs/en/operations/system-tables/metric_log.md @@ -23,28 +23,28 @@ SELECT * FROM system.metric_log LIMIT 1 FORMAT Vertical; ``` text Row 1: ────── -event_date: 2020-02-18 -event_time: 2020-02-18 07:15:33 -milliseconds: 554 -ProfileEvent_Query: 0 -ProfileEvent_SelectQuery: 0 -ProfileEvent_InsertQuery: 0 -ProfileEvent_FileOpen: 0 -ProfileEvent_Seek: 0 -ProfileEvent_ReadBufferFromFileDescriptorRead: 1 -ProfileEvent_ReadBufferFromFileDescriptorReadFailed: 0 -ProfileEvent_ReadBufferFromFileDescriptorReadBytes: 0 -ProfileEvent_WriteBufferFromFileDescriptorWrite: 1 -ProfileEvent_WriteBufferFromFileDescriptorWriteFailed: 0 -ProfileEvent_WriteBufferFromFileDescriptorWriteBytes: 56 +event_date: 2020-09-05 +event_time: 2020-09-05 16:22:33 +event_time_microseconds: 2020-09-05 16:22:33.196807 +milliseconds: 196 +ProfileEvent_Query: 0 +ProfileEvent_SelectQuery: 0 +ProfileEvent_InsertQuery: 0 +ProfileEvent_FailedQuery: 0 +ProfileEvent_FailedSelectQuery: 0 ... -CurrentMetric_Query: 0 -CurrentMetric_Merge: 0 -CurrentMetric_PartMutation: 0 -CurrentMetric_ReplicatedFetch: 0 -CurrentMetric_ReplicatedSend: 0 -CurrentMetric_ReplicatedChecks: 0 ... +CurrentMetric_Revision: 54439 +CurrentMetric_VersionInteger: 20009001 +CurrentMetric_RWLockWaitingReaders: 0 +CurrentMetric_RWLockWaitingWriters: 0 +CurrentMetric_RWLockActiveReaders: 0 +CurrentMetric_RWLockActiveWriters: 0 +CurrentMetric_GlobalThread: 74 +CurrentMetric_GlobalThreadActive: 26 +CurrentMetric_LocalThread: 0 +CurrentMetric_LocalThreadActive: 0 +CurrentMetric_DistributedFilesToInsert: 0 ``` **See also** From 11ffbda7cc48082ec0c3a44dc34946cf03a9dc4a Mon Sep 17 00:00:00 2001 From: myrrc Date: Wed, 9 Sep 2020 21:17:01 +0300 Subject: [PATCH 10/32] added the debugger info option --- src/Functions/CMakeLists.txt | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 78caabb6941..08eefec84d4 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -53,8 +53,18 @@ endif() target_include_directories(clickhouse_functions SYSTEM PRIVATE ${SPARSEHASH_INCLUDE_DIR}) -# Won't generate debug info for files with heavy template instantiation to achieve faster linking and lower size. -target_compile_options(clickhouse_functions PRIVATE "-g0") +option(STRIP_DEBUG_SYMBOLS_FUNCTIONS + "Do not generate debugger info for ClickHouse functions. + Provides faster linking and lower binary size. + Tradeoff is the inability to debug some source files with e.g. gdb + (empty stack frames and no local variables)." OFF) + +if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) + message(WARNING "Not generating debugger info for ClickHouse functions") + target_compile_options(clickhouse_functions PRIVATE "-g0") +else() + message(STATUS "Generating debugger info for ClickHouse functions") +endif() if (USE_ICU) target_link_libraries (clickhouse_functions PRIVATE ${ICU_LIBRARIES}) From 0a3f7d0fccc8b960d59b415b02313673d1bdc698 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 19:49:47 -0700 Subject: [PATCH 11/32] AsynchronousMetricsLog - add improved tests This adds integration tests to test the asynchronous_metric_log table for event_time_microseconds field. Asynchronous metrics are updated once every 60s, so this setting needs to be overridden for having those metrics available in time so that the tests can run. So the `asynchronous_metric_update_period_s` setting is being overriden to be 2s. --- .../__init__.py | 0 .../asynchronous_metrics_update_period_s.xml | 3 ++ .../test.py | 32 +++++++++++++++++++ .../01473_event_time_microseconds.reference | 4 +-- .../01473_event_time_microseconds.sql | 12 ++++--- 5 files changed, 43 insertions(+), 8 deletions(-) create mode 100644 tests/integration/test_asynchronous_metric_log_table/__init__.py create mode 100644 tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml create mode 100644 tests/integration/test_asynchronous_metric_log_table/test.py diff --git a/tests/integration/test_asynchronous_metric_log_table/__init__.py b/tests/integration/test_asynchronous_metric_log_table/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml b/tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml new file mode 100644 index 00000000000..840c19f03a6 --- /dev/null +++ b/tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml @@ -0,0 +1,3 @@ + + 2 + diff --git a/tests/integration/test_asynchronous_metric_log_table/test.py b/tests/integration/test_asynchronous_metric_log_table/test.py new file mode 100644 index 00000000000..10face9abc6 --- /dev/null +++ b/tests/integration/test_asynchronous_metric_log_table/test.py @@ -0,0 +1,32 @@ +import time + +from helpers.cluster import ClickHouseCluster + +# Tests that the event_time_microseconds field in system.asynchronous_metric_log table gets populated. +# asynchronous metrics are updated once every 60s by default. To make the test run faster, the setting +# asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and +# are available for querying during the test. +def test_asynchronous_metric_log(): + cluster = ClickHouseCluster(__file__) + node1 = cluster.add_instance('node1', with_zookeeper=True, main_configs=['configs/asynchronous_metrics_update_period_s.xml']) + try: + cluster.start() + node1.query("SET log_queries = 1;") + node1.query("CREATE DATABASE replica;") + query_create = '''CREATE TABLE replica.test + ( + id Int64, + event_time DateTime + ) + Engine=MergeTree() + PARTITION BY toYYYYMMDD(event_time) + ORDER BY id;''' + time.sleep(2) + node1.query(query_create) + node1.query('''INSERT INTO replica.test VALUES (1, now())''') + node1.query("SYSTEM FLUSH LOGS;") + node1.query("SELECT * FROM system.asynchronous_metrics LIMIT 10") + assert "1\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') + assert "ok\n" in node1.query("SELECT If((select count(event_time_microseconds) from system.asynchronous_metric_log) > 0, 'ok', 'fail');") + finally: + cluster.shutdown() diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference index cac87f32a29..8aa31f9ab6a 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.reference +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -1,4 +1,2 @@ -01473_asynchronous_metric_log_event_start_time_milliseconds_test -ok -01473_metric_log_event_start_time_milliseconds_test +01473_metric_log_table_event_start_time_microseconds_test ok diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index 6a13d6e1543..9bafd3bdbda 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -1,9 +1,11 @@ +-- This file contains tests for the event_time_microseconds field for various tables. +-- Note: Only event_time_microseconds for asynchronous_metric_log table is tested via +-- an integration test as those metrics take 60s by default to be updated. +-- Refer: tests/integration/test_asynchronous_metric_log_table. + set log_queries = 1; -select '01473_asynchronous_metric_log_event_start_time_milliseconds_test'; -system flush logs; -SELECT If((select count(event_time_microseconds) from system.asynchronous_metric_log) > 0, 'ok', 'fail'); -- success - -select '01473_metric_log_event_start_time_milliseconds_test'; +select '01473_metric_log_table_event_start_time_microseconds_test'; system flush logs; +SELECT sleep(3) Format Null; SELECT If((select count(event_time_microseconds) from system.metric_log) > 0, 'ok', 'fail'); -- success From 2ad63d2b4de47ca598712f9404a26ea53ba35e70 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Wed, 9 Sep 2020 17:56:21 -0700 Subject: [PATCH 12/32] AsynchronousMetricLog - refactor integration test --- .../test.py | 23 +++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_asynchronous_metric_log_table/test.py b/tests/integration/test_asynchronous_metric_log_table/test.py index 10face9abc6..9910a188e2c 100644 --- a/tests/integration/test_asynchronous_metric_log_table/test.py +++ b/tests/integration/test_asynchronous_metric_log_table/test.py @@ -1,14 +1,28 @@ import time +import pytest from helpers.cluster import ClickHouseCluster +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', with_zookeeper=True, + main_configs=['configs/asynchronous_metrics_update_period_s.xml']) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + # Tests that the event_time_microseconds field in system.asynchronous_metric_log table gets populated. # asynchronous metrics are updated once every 60s by default. To make the test run faster, the setting # asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and # are available for querying during the test. -def test_asynchronous_metric_log(): - cluster = ClickHouseCluster(__file__) - node1 = cluster.add_instance('node1', with_zookeeper=True, main_configs=['configs/asynchronous_metrics_update_period_s.xml']) +def test_event_time_microseconds_field(started_cluster): try: cluster.start() node1.query("SET log_queries = 1;") @@ -27,6 +41,7 @@ def test_asynchronous_metric_log(): node1.query("SYSTEM FLUSH LOGS;") node1.query("SELECT * FROM system.asynchronous_metrics LIMIT 10") assert "1\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') - assert "ok\n" in node1.query("SELECT If((select count(event_time_microseconds) from system.asynchronous_metric_log) > 0, 'ok', 'fail');") + assert "ok\n" in node1.query( + "SELECT If((select count(event_time_microseconds) from system.asynchronous_metric_log) > 0, 'ok', 'fail');") finally: cluster.shutdown() From 31b0bd481c687160e2217a1443b244fad2733a04 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Thu, 10 Sep 2020 00:17:20 -0700 Subject: [PATCH 13/32] Integration tests - more accurate tests --- .../test.py | 20 +++++++++++++++---- .../01473_event_time_microseconds.sql | 16 ++++++++++++++- 2 files changed, 31 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_asynchronous_metric_log_table/test.py b/tests/integration/test_asynchronous_metric_log_table/test.py index 9910a188e2c..0eb72c20376 100644 --- a/tests/integration/test_asynchronous_metric_log_table/test.py +++ b/tests/integration/test_asynchronous_metric_log_table/test.py @@ -39,9 +39,21 @@ def test_event_time_microseconds_field(started_cluster): node1.query(query_create) node1.query('''INSERT INTO replica.test VALUES (1, now())''') node1.query("SYSTEM FLUSH LOGS;") - node1.query("SELECT * FROM system.asynchronous_metrics LIMIT 10") - assert "1\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') - assert "ok\n" in node1.query( - "SELECT If((select count(event_time_microseconds) from system.asynchronous_metric_log) > 0, 'ok', 'fail');") + #query assumes that the event_time field is accurate + equals_query = '''WITH ( + ( + SELECT event_time_microseconds + FROM system.asynchronous_metric_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.asynchronous_metric_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time) + SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail')''' + assert "ok\n" in node1.query(equals_query) finally: cluster.shutdown() diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index 9bafd3bdbda..dd79a735d8f 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -8,4 +8,18 @@ set log_queries = 1; select '01473_metric_log_table_event_start_time_microseconds_test'; system flush logs; SELECT sleep(3) Format Null; -SELECT If((select count(event_time_microseconds) from system.metric_log) > 0, 'ok', 'fail'); -- success +-- query assumes that the event_time field is accurate. +WITH ( + ( + SELECT event_time_microseconds + FROM system.metric_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.metric_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time) +SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail') From c309f55c207d5e864f1b05200e2a706627163bac Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 10 Sep 2020 14:02:52 +0300 Subject: [PATCH 14/32] updated setting and added default value --- src/Functions/CMakeLists.txt | 5 ++++- src/Functions/GatherUtils/CMakeLists.txt | 5 +++-- src/Functions/URL/CMakeLists.txt | 5 +++-- src/Functions/array/CMakeLists.txt | 5 +++-- 4 files changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 08eefec84d4..b19266726ae 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -57,7 +57,10 @@ option(STRIP_DEBUG_SYMBOLS_FUNCTIONS "Do not generate debugger info for ClickHouse functions. Provides faster linking and lower binary size. Tradeoff is the inability to debug some source files with e.g. gdb - (empty stack frames and no local variables)." OFF) + (empty stack frames and no local variables)." + CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" + OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" + OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL") if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) message(WARNING "Not generating debugger info for ClickHouse functions") diff --git a/src/Functions/GatherUtils/CMakeLists.txt b/src/Functions/GatherUtils/CMakeLists.txt index 921b06fb1c2..3f7f08621a1 100644 --- a/src/Functions/GatherUtils/CMakeLists.txt +++ b/src/Functions/GatherUtils/CMakeLists.txt @@ -3,5 +3,6 @@ add_headers_and_sources(clickhouse_functions_gatherutils .) add_library(clickhouse_functions_gatherutils ${clickhouse_functions_gatherutils_sources} ${clickhouse_functions_gatherutils_headers}) target_link_libraries(clickhouse_functions_gatherutils PRIVATE dbms) -# Won't generate debug info for files with heavy template instantiation to achieve faster linking and lower size. -target_compile_options(clickhouse_functions_gatherutils PRIVATE "-g0") +if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) + target_compile_options(clickhouse_functions_gatherutils PRIVATE "-g0") +endif() diff --git a/src/Functions/URL/CMakeLists.txt b/src/Functions/URL/CMakeLists.txt index 72fdc3174fc..0cf02fd524f 100644 --- a/src/Functions/URL/CMakeLists.txt +++ b/src/Functions/URL/CMakeLists.txt @@ -3,8 +3,9 @@ add_headers_and_sources(clickhouse_functions_url .) add_library(clickhouse_functions_url ${clickhouse_functions_url_sources} ${clickhouse_functions_url_headers}) target_link_libraries(clickhouse_functions_url PRIVATE dbms) -# Won't generate debug info for files with heavy template instantiation to achieve faster linking and lower size. -target_compile_options(clickhouse_functions_url PRIVATE "-g0") +if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) + target_compile_options(clickhouse_functions_url PRIVATE "-g0") +endif() # TODO: move Functions/Regexps.h to some lib and use here target_link_libraries(clickhouse_functions_url PRIVATE hyperscan) diff --git a/src/Functions/array/CMakeLists.txt b/src/Functions/array/CMakeLists.txt index 32ef0216caa..1457e671261 100644 --- a/src/Functions/array/CMakeLists.txt +++ b/src/Functions/array/CMakeLists.txt @@ -3,5 +3,6 @@ add_headers_and_sources(clickhouse_functions_array .) add_library(clickhouse_functions_array ${clickhouse_functions_array_sources} ${clickhouse_functions_array_headers}) target_link_libraries(clickhouse_functions_array PRIVATE dbms clickhouse_functions_gatherutils) -# Won't generate debug info for files with heavy template instantiation to achieve faster linking and lower size. -target_compile_options(clickhouse_functions_array PRIVATE "-g0") +if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) + target_compile_options(clickhouse_functions_array PRIVATE "-g0") +endif() From 598c81ed0ac1e127136b54d167e9209e252818be Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 10 Sep 2020 14:07:08 +0300 Subject: [PATCH 15/32] fix: default value --- src/Functions/CMakeLists.txt | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index b19266726ae..4f6952aad8f 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -53,14 +53,21 @@ endif() target_include_directories(clickhouse_functions SYSTEM PRIVATE ${SPARSEHASH_INCLUDE_DIR}) +if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" + OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" + OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL") + set (STRIP_DSF_DEFAULT ON) +else() + set (STRIP_DSF_DEFAULT OFF) +endif() + + option(STRIP_DEBUG_SYMBOLS_FUNCTIONS "Do not generate debugger info for ClickHouse functions. Provides faster linking and lower binary size. Tradeoff is the inability to debug some source files with e.g. gdb (empty stack frames and no local variables)." - CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" - OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" - OR CMAKE_BUILD_TYPE_UC STREQUAL "MINSIZEREL") + STRIP_DSF_DEFAULT) if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) message(WARNING "Not generating debugger info for ClickHouse functions") From 00b8d91eb859a2642a0583107498d4642c7d4348 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Thu, 10 Sep 2020 19:46:59 +0800 Subject: [PATCH 16/32] Fix mysql replication TableMapEvent optional metadata ignore --- src/Core/MySQL/IMySQLReadPacket.cpp | 29 ++++++++++--------- src/Core/MySQL/MySQLReplication.cpp | 6 +++- src/Core/tests/mysql_protocol.cpp | 1 + .../MySQL/MaterializeMySQLSyncThread.cpp | 2 ++ 4 files changed, 23 insertions(+), 15 deletions(-) diff --git a/src/Core/MySQL/IMySQLReadPacket.cpp b/src/Core/MySQL/IMySQLReadPacket.cpp index 8fc8855c8a4..5f6bbc7bceb 100644 --- a/src/Core/MySQL/IMySQLReadPacket.cpp +++ b/src/Core/MySQL/IMySQLReadPacket.cpp @@ -50,21 +50,22 @@ uint64_t readLengthEncodedNumber(ReadBuffer & buffer) uint64_t buf = 0; buffer.readStrict(c); auto cc = static_cast(c); - if (cc < 0xfc) + switch (cc) { - return cc; - } - else if (cc < 0xfd) - { - buffer.readStrict(reinterpret_cast(&buf), 2); - } - else if (cc < 0xfe) - { - buffer.readStrict(reinterpret_cast(&buf), 3); - } - else - { - buffer.readStrict(reinterpret_cast(&buf), 8); + /// NULL + case 0xfb: + break; + case 0xfc: + buffer.readStrict(reinterpret_cast(&buf), 2); + break; + case 0xfd: + buffer.readStrict(reinterpret_cast(&buf), 3); + break; + case 0xfe: + buffer.readStrict(reinterpret_cast(&buf), 8); + break; + default: + return cc; } return buf; } diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 42d077260f8..808e9d1a247 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -171,7 +171,9 @@ namespace MySQLReplication /// Ignore MySQL 8.0 optional metadata fields. /// https://mysqlhighavailability.com/more-metadata-is-written-into-binary-log/ - payload.ignore(payload.available() - CHECKSUM_CRC32_SIGNATURE_LENGTH); + auto remain = payload.available(); + if (remain > CHECKSUM_CRC32_SIGNATURE_LENGTH) + payload.ignore(remain); } /// Types that do not used in the binlog event: @@ -221,6 +223,7 @@ namespace MySQLReplication } case MYSQL_TYPE_NEWDECIMAL: case MYSQL_TYPE_STRING: { + /// Big-Endian auto b0 = UInt16(meta[pos] << 8); auto b1 = UInt8(meta[pos + 1]); column_meta.emplace_back(UInt16(b0 + b1)); @@ -231,6 +234,7 @@ namespace MySQLReplication case MYSQL_TYPE_BIT: case MYSQL_TYPE_VARCHAR: case MYSQL_TYPE_VAR_STRING: { + /// Little-Endian auto b0 = UInt8(meta[pos]); auto b1 = UInt16(meta[pos + 1] << 8); column_meta.emplace_back(UInt16(b0 + b1)); diff --git a/src/Core/tests/mysql_protocol.cpp b/src/Core/tests/mysql_protocol.cpp index acae8603c40..6cad095fc85 100644 --- a/src/Core/tests/mysql_protocol.cpp +++ b/src/Core/tests/mysql_protocol.cpp @@ -283,6 +283,7 @@ int main(int argc, char ** argv) } { + /// mysql_protocol --host=172.17.0.3 --user=root --password=123 --db=sbtest try { boost::program_options::options_description desc("Allowed options"); diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index 851ea351876..465a7cb912a 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -195,6 +195,7 @@ void MaterializeMySQLSyncThread::synchronization(const String & mysql_version) } catch (...) { + client.disconnect(); tryLogCurrentException(log); getDatabase(database_name).setException(std::current_exception()); } @@ -206,6 +207,7 @@ void MaterializeMySQLSyncThread::stopSynchronization() { sync_quit = true; background_thread_pool->join(); + client.disconnect(); } } From e0081980230e636545a43a67238dc25b44f7777c Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 10 Sep 2020 15:44:49 +0300 Subject: [PATCH 17/32] fix: default value --- src/Functions/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 4f6952aad8f..0a99a034a33 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -67,7 +67,7 @@ option(STRIP_DEBUG_SYMBOLS_FUNCTIONS Provides faster linking and lower binary size. Tradeoff is the inability to debug some source files with e.g. gdb (empty stack frames and no local variables)." - STRIP_DSF_DEFAULT) + ${STRIP_DSF_DEFAULT}) if (STRIP_DEBUG_SYMBOLS_FUNCTIONS) message(WARNING "Not generating debugger info for ClickHouse functions") From f19f02028b7dca3844127a550c8703a8f40fc383 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Sep 2020 17:50:10 +0300 Subject: [PATCH 18/32] fix misleading exception message about uuid macro --- src/Common/Macros.cpp | 8 +++++++- src/Storages/MergeTree/registerStorageMergeTree.cpp | 9 +++++---- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index 7b5a896015b..a4981fa5be3 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -68,8 +68,14 @@ String Macros::expand(const String & s, res += database_name; else if (macro_name == "table" && !table_name.empty()) res += table_name; - else if (macro_name == "uuid" && uuid != UUIDHelpers::Nil) + else if (macro_name == "uuid") + { + if (uuid == UUIDHelpers::Nil) + throw Exception("Macro 'uuid' and empty arguments of ReplicatedMergeTree " + "are supported only for ON CLUSTER queries with Atomic database engine", + ErrorCodes::SYNTAX_ERROR); res += toString(uuid); + } else throw Exception("No macro '" + macro_name + "' in config while processing substitutions in '" + s + "' at '" diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index b0c422bd79f..d3af3942428 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -395,9 +395,10 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) { - bool has_arguments = arg_num + 2 <= arg_cnt && engine_args[arg_num]->as() && engine_args[arg_num + 1]->as(); + bool has_arguments = arg_num + 2 <= arg_cnt; + bool has_valid_arguments = has_arguments && engine_args[arg_num]->as() && engine_args[arg_num + 1]->as(); - if (has_arguments) + if (has_valid_arguments) { const auto * ast = engine_args[arg_num]->as(); if (ast && ast->value.getType() == Field::Types::String) @@ -420,7 +421,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) "No replica name in config" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::NO_REPLICA_NAME_GIVEN); ++arg_num; } - else if (is_extended_storage_def) + else if (is_extended_storage_def && !has_arguments) { /// Try use default values if arguments are not specified. /// It works for ON CLUSTER queries when database engine is Atomic and there are {shard} and {replica} in config. @@ -428,7 +429,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) replica_name = "{replica}"; /// TODO maybe use hostname if {replica} is not defined? } else - throw Exception("Expected zookeper_path and replica_name arguments", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Expected two string literal arguments: zookeper_path and replica_name", ErrorCodes::BAD_ARGUMENTS); /// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries bool is_on_cluster = args.local_context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; From 397ebdb855c75be3df9f6a007a4e2cf3d0321bb5 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Thu, 10 Sep 2020 08:55:39 -0700 Subject: [PATCH 19/32] remove sleep from unit test --- .../configs/asynchronous_metrics_update_period_s.xml | 2 +- tests/queries/0_stateless/01473_event_time_microseconds.sql | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml b/tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml index 840c19f03a6..b658b0436b1 100644 --- a/tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml +++ b/tests/integration/test_asynchronous_metric_log_table/configs/asynchronous_metrics_update_period_s.xml @@ -1,3 +1,3 @@ - 2 + 1 diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index dd79a735d8f..138df77ffec 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -7,7 +7,6 @@ set log_queries = 1; select '01473_metric_log_table_event_start_time_microseconds_test'; system flush logs; -SELECT sleep(3) Format Null; -- query assumes that the event_time field is accurate. WITH ( ( From 964c18241cabe7566f2e639190c143d8f9c7ca2b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 10 Sep 2020 19:04:53 +0300 Subject: [PATCH 20/32] Added test from #13829 --- tests/queries/0_stateless/01492_array_join_crash_13829.reference | 0 tests/queries/0_stateless/01492_array_join_crash_13829.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/01492_array_join_crash_13829.reference create mode 100644 tests/queries/0_stateless/01492_array_join_crash_13829.sql diff --git a/tests/queries/0_stateless/01492_array_join_crash_13829.reference b/tests/queries/0_stateless/01492_array_join_crash_13829.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01492_array_join_crash_13829.sql b/tests/queries/0_stateless/01492_array_join_crash_13829.sql new file mode 100644 index 00000000000..9e11c3b69d4 --- /dev/null +++ b/tests/queries/0_stateless/01492_array_join_crash_13829.sql @@ -0,0 +1 @@ +SELECT NULL = countEqual(materialize([arrayJoin([NULL, NULL, NULL]), NULL AS x, arrayJoin([255, 1025, NULL, NULL]), arrayJoin([2, 1048576, NULL, NULL])]), materialize(x)) format Null; From 15bdb6048e610fb5787e581b7fa346bd7512e109 Mon Sep 17 00:00:00 2001 From: Artem Hnilov Date: Thu, 10 Sep 2020 16:25:07 +0000 Subject: [PATCH 21/32] Function formatReadableQuantity() added. --- src/Functions/FunctionsFormatting.cpp | 1 + src/Functions/FunctionsFormatting.h | 76 +++++++++++++++++++++++++++ 2 files changed, 77 insertions(+) diff --git a/src/Functions/FunctionsFormatting.cpp b/src/Functions/FunctionsFormatting.cpp index 7582e234622..1187749aa2d 100644 --- a/src/Functions/FunctionsFormatting.cpp +++ b/src/Functions/FunctionsFormatting.cpp @@ -9,6 +9,7 @@ void registerFunctionsFormatting(FunctionFactory & factory) { factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/FunctionsFormatting.h b/src/Functions/FunctionsFormatting.h index e11254e2775..da22babec38 100644 --- a/src/Functions/FunctionsFormatting.h +++ b/src/Functions/FunctionsFormatting.h @@ -202,4 +202,80 @@ private: } }; + +class FunctionFormatReadableQuantity : public IFunction +{ +public: + static constexpr auto name = "formatReadableQuantity"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 1; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const IDataType & type = *arguments[0]; + + if (!isNativeNumber(type)) + throw Exception("Cannot format " + type.getName() + " as quantity", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override + { + if (!(executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result))) + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + +private: + template + bool executeType(Block & block, const ColumnNumbers & arguments, size_t result) const + { + if (const ColumnVector * col_from = checkAndGetColumn>(block.getByPosition(arguments[0]).column.get())) + { + auto col_to = ColumnString::create(); + + const typename ColumnVector::Container & vec_from = col_from->getData(); + ColumnString::Chars & data_to = col_to->getChars(); + ColumnString::Offsets & offsets_to = col_to->getOffsets(); + size_t size = vec_from.size(); + data_to.resize(size * 2); + offsets_to.resize(size); + + WriteBufferFromVector buf_to(data_to); + + for (size_t i = 0; i < size; ++i) + { + formatReadableQuantity(static_cast(vec_from[i]), buf_to); + writeChar(0, buf_to); + offsets_to[i] = buf_to.count(); + } + + buf_to.finalize(); + block.getByPosition(result).column = std::move(col_to); + return true; + } + + return false; + } +}; + } From 6e0d2a427c4e1a3145d2ac33d6cc94040ffd95dc Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Thu, 10 Sep 2020 19:30:11 +0300 Subject: [PATCH 22/32] add another test --- .../01460_line_as_string_format.reference | 2 ++ .../01460_line_as_string_format.sh | 24 ++++++++++++++----- 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01460_line_as_string_format.reference b/tests/queries/0_stateless/01460_line_as_string_format.reference index dec67eb2e0a..c795fba4309 100644 --- a/tests/queries/0_stateless/01460_line_as_string_format.reference +++ b/tests/queries/0_stateless/01460_line_as_string_format.reference @@ -4,3 +4,5 @@ "array" : [1, 2, 3], Finally implement this new feature. +42 ClickHouse +42 ClickHouse is a `fast` #open-source# (OLAP) database "management" :system: diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sh b/tests/queries/0_stateless/01460_line_as_string_format.sh index a985bc207a8..60a960d2bf8 100755 --- a/tests/queries/0_stateless/01460_line_as_string_format.sh +++ b/tests/queries/0_stateless/01460_line_as_string_format.sh @@ -3,17 +3,29 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS line_as_string"; - -$CLICKHOUSE_CLIENT --query="CREATE TABLE line_as_string(field String) ENGINE = Memory"; +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS line_as_string1"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE line_as_string1(field String) ENGINE = Memory"; echo '"id" : 1, "date" : "01.01.2020", "string" : "123{{{\"\\", "array" : [1, 2, 3], -Finally implement this new feature.' | $CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string FORMAT LineAsString"; +Finally implement this new feature.' | $CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string1 FORMAT LineAsString"; -$CLICKHOUSE_CLIENT --query="SELECT * FROM line_as_string"; -$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string" +$CLICKHOUSE_CLIENT --query="SELECT * FROM line_as_string1"; +$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string1" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS line_as_string2"; +$CLICKHOUSE_CLIENT --query="create table line_as_string2( + a UInt64 default 42, + b String materialized toString(a), + c String +) engine=MergeTree() order by tuple();"; + +$CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string2(c) values ('ClickHouse')"; + +echo 'ClickHouse is a `fast` #open-source# (OLAP) 'database' "management" :system:' | $CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string2(c) FORMAT LineAsString"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM line_as_string2 order by c"; +$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string2" From d0c259974d1486dc90ae29958a3f1dd196052961 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Thu, 10 Sep 2020 19:35:18 +0300 Subject: [PATCH 23/32] style check --- .../Formats/Impl/LineAsStringRowInputFormat.cpp | 15 ++++++--------- src/Processors/ya.make | 2 +- 2 files changed, 7 insertions(+), 10 deletions(-) diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index 27bc71d764d..befc635386f 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -8,8 +8,7 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; - extern const int INCORRECT_DATA; + extern const int INCORRECT_QUERY; } LineAsStringRowInputFormat::LineAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) : @@ -17,7 +16,7 @@ LineAsStringRowInputFormat::LineAsStringRowInputFormat(const Block & header_, Re { if (header_.columns() > 1 || header_.getDataTypes()[0]->getTypeId() != TypeIndex::String) { - throw Exception("This input format is only suitable for tables with a single column of type String.", ErrorCodes::LOGICAL_ERROR); + throw Exception("This input format is only suitable for tables with a single column of type String.", ErrorCodes::INCORRECT_QUERY); } } @@ -39,7 +38,8 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) { pos = find_first_symbols<'\n', '\\'>(buf.position(), buf.buffer().end()); buf.position() = pos; - if (buf.position() == buf.buffer().end()) { + if (buf.position() == buf.buffer().end()) + { over = true; break; } @@ -48,14 +48,11 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) newline = false; } else if (*buf.position() == '\\') - { + { ++buf.position(); if (!buf.eof()) - { - ++buf.position(); - } + ++buf.position(); } - } buf.makeContinuousMemoryFromCheckpointToPos(); diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 8fe4e651ade..c69d274a70e 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -25,7 +25,6 @@ SRCS( Formats/Impl/ConstantExpressionTemplate.cpp Formats/Impl/CSVRowInputFormat.cpp Formats/Impl/CSVRowOutputFormat.cpp - Formats/Impl/LineAsStringRowInputFormat.cpp Formats/Impl/JSONAsStringRowInputFormat.cpp Formats/Impl/JSONCompactEachRowRowInputFormat.cpp Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -34,6 +33,7 @@ SRCS( Formats/Impl/JSONEachRowRowOutputFormat.cpp Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp Formats/Impl/JSONRowOutputFormat.cpp + Formats/Impl/LineAsStringRowInputFormat.cpp Formats/Impl/MarkdownRowOutputFormat.cpp Formats/Impl/MsgPackRowInputFormat.cpp Formats/Impl/MsgPackRowOutputFormat.cpp From 4e197cf069c18cafb55b5fa22499483d8f19666d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Sep 2020 19:41:46 +0300 Subject: [PATCH 24/32] Remove unused headers --- base/common/types.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/base/common/types.h b/base/common/types.h index 682fe94366c..a02398a3365 100644 --- a/base/common/types.h +++ b/base/common/types.h @@ -1,8 +1,6 @@ #pragma once -#include #include -#include #include #include From 142026f4abc810426d4ebf195bb8316a83b6adff Mon Sep 17 00:00:00 2001 From: Artem Hnilov Date: Thu, 10 Sep 2020 16:54:52 +0000 Subject: [PATCH 25/32] 01492_format_readable_quantity test added --- .../01492_format_readable_quantity.reference | 50 +++++++++++++++++++ .../01492_format_readable_quantity.sql | 4 ++ 2 files changed, 54 insertions(+) create mode 100644 tests/queries/0_stateless/01492_format_readable_quantity.reference create mode 100644 tests/queries/0_stateless/01492_format_readable_quantity.sql diff --git a/tests/queries/0_stateless/01492_format_readable_quantity.reference b/tests/queries/0_stateless/01492_format_readable_quantity.reference new file mode 100644 index 00000000000..3736933b5ee --- /dev/null +++ b/tests/queries/0_stateless/01492_format_readable_quantity.reference @@ -0,0 +1,50 @@ +1.00 1.00 1.00 +2.72 2.00 2.00 +7.39 7.00 7.00 +20.09 20.00 20.00 +54.60 54.00 54.00 +148.41 148.00 148.00 +403.43 403.00 403.00 +1.10 thousand 1.10 thousand 1.10 thousand +2.98 thousand 2.98 thousand 2.98 thousand +8.10 thousand 8.10 thousand 8.10 thousand +22.03 thousand 22.03 thousand 22.03 thousand +59.87 thousand 59.87 thousand 59.87 thousand +162.75 thousand 162.75 thousand 162.75 thousand +442.41 thousand 442.41 thousand 442.41 thousand +1.20 million 1.20 million 1.20 million +3.27 million 3.27 million 3.27 million +8.89 million 8.89 million 8.89 million +24.15 million 24.15 million 24.15 million +65.66 million 65.66 million 65.66 million +178.48 million 178.48 million 178.48 million +485.17 million 485.17 million 485.17 million +1.32 billion 1.32 billion 1.32 billion +3.58 billion 3.58 billion -2.15 billion +9.74 billion 9.74 billion -2.15 billion +26.49 billion 26.49 billion -2.15 billion +72.00 billion 72.00 billion -2.15 billion +195.73 billion 195.73 billion -2.15 billion +532.05 billion 532.05 billion -2.15 billion +1.45 trillion 1.45 trillion -2.15 billion +3.93 trillion 3.93 trillion -2.15 billion +10.69 trillion 10.69 trillion -2.15 billion +29.05 trillion 29.05 trillion -2.15 billion +78.96 trillion 78.96 trillion -2.15 billion +214.64 trillion 214.64 trillion -2.15 billion +583.46 trillion 583.46 trillion -2.15 billion +1.59 quadrillion 1.59 quadrillion -2.15 billion +4.31 quadrillion 4.31 quadrillion -2.15 billion +11.72 quadrillion 11.72 quadrillion -2.15 billion +31.86 quadrillion 31.86 quadrillion -2.15 billion +86.59 quadrillion 86.59 quadrillion -2.15 billion +235.39 quadrillion 235.39 quadrillion -2.15 billion +639.84 quadrillion 639.84 quadrillion -2.15 billion +1739.27 quadrillion 1739.27 quadrillion -2.15 billion +4727.84 quadrillion 4727.84 quadrillion -2.15 billion +12851.60 quadrillion 12851.60 quadrillion -2.15 billion +34934.27 quadrillion 0.00 -2.15 billion +94961.19 quadrillion 0.00 -2.15 billion +258131.29 quadrillion 0.00 -2.15 billion +701673.59 quadrillion 0.00 -2.15 billion +1907346.57 quadrillion 0.00 -2.15 billion diff --git a/tests/queries/0_stateless/01492_format_readable_quantity.sql b/tests/queries/0_stateless/01492_format_readable_quantity.sql new file mode 100644 index 00000000000..c2ebcfc1a97 --- /dev/null +++ b/tests/queries/0_stateless/01492_format_readable_quantity.sql @@ -0,0 +1,4 @@ +WITH round(exp(number), 6) AS x, toUInt64(x) AS y, toInt32(x) AS z +SELECT formatReadableQuantity(x), formatReadableQuantity(y), formatReadableQuantity(z) +FROM system.numbers +LIMIT 50; From bbe379d58e96968ae24c7dee5fef9e872aa2f272 Mon Sep 17 00:00:00 2001 From: Artem Hnilov Date: Thu, 10 Sep 2020 17:06:32 +0000 Subject: [PATCH 26/32] 00534_filimonov.data was updated --- tests/queries/0_stateless/00534_filimonov.data | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/00534_filimonov.data b/tests/queries/0_stateless/00534_filimonov.data index b4c15b01ef4..393ff5b7155 100644 --- a/tests/queries/0_stateless/00534_filimonov.data +++ b/tests/queries/0_stateless/00534_filimonov.data @@ -174,6 +174,7 @@ SELECT sipHash64(NULL); SELECT protocol(NULL); SELECT toInt16OrZero(NULL); SELECT formatReadableSize(NULL); +SELECT formatReadableQuantity(NULL); SELECT concatAssumeInjective(NULL); SELECT toString(NULL); SELECT MACStringToNum(NULL); From 294af54007b761fbab57f006b4e074f9f996944a Mon Sep 17 00:00:00 2001 From: Artem Hnilov Date: Thu, 10 Sep 2020 20:27:37 +0300 Subject: [PATCH 27/32] Update other-functions.md --- .../functions/other-functions.md | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 468e15e7d57..7b9dacf21cd 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -508,6 +508,29 @@ SELECT └────────────────┴────────────┘ ``` +## formatReadableQuantity(x) {#formatreadablequantityx} + +Принимает число. Возвращает округленное число с суффиксом (thousand, million, billion и т.д.) в виде строки. + +Облегчает визуальное восприятие больших чисел живым человеком. + +Пример: + +``` sql +SELECT + arrayJoin([1024, 1234 * 1000, (4567 * 1000) * 1000, 98765432101234]) AS number, + formatReadableQuantity(number) AS number_for_humans +``` + +``` text +┌─────────number─┬─number_for_humans─┐ +│ 1024 │ 1.02 thousand │ +│ 1234000 │ 1.23 million │ +│ 4567000000 │ 4.57 billion │ +│ 98765432101234 │ 98.77 trillion │ +└────────────────┴───────────────────┘ +``` + ## least(a, b) {#leasta-b} Возвращает наименьшее значение из a и b. From 726277adec1da5924da59883f17945a8e29e0195 Mon Sep 17 00:00:00 2001 From: Artem Hnilov Date: Thu, 10 Sep 2020 20:34:23 +0300 Subject: [PATCH 28/32] Update other-functions.md --- .../functions/other-functions.md | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 05247b6db7d..1c059e9f97b 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -515,6 +515,29 @@ SELECT └────────────────┴────────────┘ ``` +## formatReadableQuantity(x) {#formatreadablequantityx} + +Accepts the number. Returns a rounded number with a suffix (thousand, million, billion, etc.) as a string. + +It is useful for reading big numbers by human. + +Example: + +``` sql +SELECT + arrayJoin([1024, 1234 * 1000, (4567 * 1000) * 1000, 98765432101234]) AS number, + formatReadableQuantity(number) AS number_for_humans +``` + +``` text +┌─────────number─┬─number_for_humans─┐ +│ 1024 │ 1.02 thousand │ +│ 1234000 │ 1.23 million │ +│ 4567000000 │ 4.57 billion │ +│ 98765432101234 │ 98.77 trillion │ +└────────────────┴───────────────────┘ +``` + ## least(a, b) {#leasta-b} Returns the smallest value from a and b. From 54491b8608043871634973b997042864ca29bc6f Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 10 Sep 2020 22:23:23 +0300 Subject: [PATCH 29/32] Update README.md --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 7f6a102a2dd..f14f2e88886 100644 --- a/README.md +++ b/README.md @@ -17,5 +17,4 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events -* [ClickHouse Data Integration Virtual Meetup](https://www.eventbrite.com/e/clickhouse-september-virtual-meetup-data-integration-tickets-117421895049) on September 10, 2020. * [ClickHouse talk at Ya.Subbotnik (in Russian)](https://ya.cc/t/cIBI-3yECj5JF) on September 12, 2020. From 2873777857351a939564e7fac0a285bf8222f19c Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Fri, 11 Sep 2020 12:54:53 +0800 Subject: [PATCH 30/32] Ignore unprocessed replication payload --- src/Core/MySQL/MySQLReplication.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 808e9d1a247..e7f113ba7af 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -171,9 +171,7 @@ namespace MySQLReplication /// Ignore MySQL 8.0 optional metadata fields. /// https://mysqlhighavailability.com/more-metadata-is-written-into-binary-log/ - auto remain = payload.available(); - if (remain > CHECKSUM_CRC32_SIGNATURE_LENGTH) - payload.ignore(remain); + payload.ignoreAll(); } /// Types that do not used in the binlog event: @@ -915,7 +913,7 @@ namespace MySQLReplication break; } } - payload.tryIgnore(CHECKSUM_CRC32_SIGNATURE_LENGTH); + payload.ignoreAll(); } } From 5de3d9c03298101876df60eed11ca2499676faf5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 11 Sep 2020 12:58:04 +0300 Subject: [PATCH 31/32] Update version_date.tsv after release 20.6.6.7 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index d97fdbfa080..3915000cec3 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,5 +1,6 @@ v20.8.2.3-stable 2020-09-08 v20.7.2.30-stable 2020-08-31 +v20.6.6.7-stable 2020-09-11 v20.6.5.8-stable 2020-09-03 v20.6.4.44-stable 2020-08-20 v20.6.3.28-stable 2020-08-07 From eb7e480d399788dae32dc83bd4b32dfdc6ec1aa3 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 11 Sep 2020 15:47:36 +0300 Subject: [PATCH 32/32] DOCSUP-2040: Translate PR to RU (#14551) * DOCSUP-2040: Update by PR#1127. * DOCSUP-2040: Update by PR#11088. * DOCSUP-2040: Update by PR#10923. * DOCSUP-2040: Minor fix. * Update docs/ru/sql-reference/functions/random-functions.md Co-authored-by: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> * Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> * Update translation. * Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: BayoNet * Update docs/ru/sql-reference/functions/random-functions.md Co-authored-by: BayoNet * Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: BayoNet * Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: BayoNet Co-authored-by: romanzhukov Co-authored-by: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Co-authored-by: BayoNet --- docs/ru/interfaces/formats.md | 20 +++- .../functions/random-functions.md | 46 ++++++++++ .../functions/type-conversion-functions.md | 91 +++++++++++++++++++ 3 files changed, 156 insertions(+), 1 deletion(-) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 054f75e8da8..04bca115974 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -28,6 +28,8 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT | [PrettySpace](#prettyspace) | ✗ | ✔ | | [Protobuf](#protobuf) | ✔ | ✔ | | [Parquet](#data-format-parquet) | ✔ | ✔ | +| [Arrow](#data-format-arrow) | ✔ | ✔ | +| [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ | | [ORC](#data-format-orc) | ✔ | ✗ | | [RowBinary](#rowbinary) | ✔ | ✔ | | [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ | @@ -947,6 +949,12 @@ ClickHouse пишет и читает сообщения `Protocol Buffers` в ## Avro {#data-format-avro} +[Apache Avro](https://avro.apache.org/) — это ориентированный на строки фреймворк для сериализации данных. Разработан в рамках проекта Apache Hadoop. + +В ClickHouse формат Avro поддерживает чтение и запись [файлов данных Avro](https://avro.apache.org/docs/current/spec.html#Object+Container+Files). + +[Логические типы Avro](https://avro.apache.org/docs/current/spec.html#Logical+Types) + ## AvroConfluent {#data-format-avro-confluent} Для формата `AvroConfluent` ClickHouse поддерживает декодирование сообщений `Avro` с одним объектом. Такие сообщения используются с [Kafka] (http://kafka.apache.org/) и реестром схем [Confluent](https://docs.confluent.io/current/schema-registry/index.html). @@ -996,7 +1004,7 @@ SELECT * FROM topic1_stream; ## Parquet {#data-format-parquet} -[Apache Parquet](http://parquet.apache.org/) — формат поколоночного хранения данных, который распространён в экосистеме Hadoop. Для формата `Parquet` ClickHouse поддерживает операции чтения и записи. +[Apache Parquet](https://parquet.apache.org/) — формат поколоночного хранения данных, который распространён в экосистеме Hadoop. Для формата `Parquet` ClickHouse поддерживает операции чтения и записи. ### Соответствие типов данных {#sootvetstvie-tipov-dannykh} @@ -1042,6 +1050,16 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Parquet" > {some_ Для обмена данными с экосистемой Hadoop можно использовать движки таблиц [HDFS](../engines/table-engines/integrations/hdfs.md). +## Arrow {data-format-arrow} + +[Apache Arrow](https://arrow.apache.org/) поставляется с двумя встроенными поколоночнами форматами хранения. ClickHouse поддерживает операции чтения и записи для этих форматов. + +`Arrow` — это Apache Arrow's "file mode" формат. Он предназначен для произвольного доступа в памяти. + +## ArrowStream {data-format-arrow-stream} + +`ArrowStream` — это Apache Arrow's "stream mode" формат. Он предназначен для обработки потоков в памяти. + ## ORC {#data-format-orc} [Apache ORC](https://orc.apache.org/) - это column-oriented формат данных, распространённый в экосистеме Hadoop. Вы можете только вставлять данные этого формата в ClickHouse. diff --git a/docs/ru/sql-reference/functions/random-functions.md b/docs/ru/sql-reference/functions/random-functions.md index b425505b69d..4aaaef5cb5d 100644 --- a/docs/ru/sql-reference/functions/random-functions.md +++ b/docs/ru/sql-reference/functions/random-functions.md @@ -55,4 +55,50 @@ FROM numbers(3) └────────────┴────────────┴──────────────┴────────────────┴─────────────────┴──────────────────────┘ ``` +# Случайные функции для работы со строками {#random-functions-for-working-with-strings} + +## randomString {#random-string} + +## randomFixedString {#random-fixed-string} + +## randomPrintableASCII {#random-printable-ascii} + +## randomStringUTF8 {#random-string-utf8} + +## fuzzBits {#fuzzbits} + +**Синтаксис** + +``` sql +fuzzBits([s], [prob]) +``` +Инвертирует каждый бит `s` с вероятностью `prob`. + +**Параметры** + +- `s` — `String` or `FixedString` +- `prob` — constant `Float32/64` + +**Возвращаемое значение** + +Измененная случайным образом строка с тем же типом, что и `s`. + +**Пример** + +Запрос: + +``` sql +SELECT fuzzBits(materialize('abacaba'), 0.1) +FROM numbers(3) +``` + +Результат: + +``` text +┌─fuzzBits(materialize('abacaba'), 0.1)─┐ +│ abaaaja │ +│ a*cjab+ │ +│ aeca2A │ +└───────────────────────────────────────┘ + [Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/random_functions/) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 41ded78055c..c7d74a9d881 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -513,4 +513,95 @@ SELECT parseDateTimeBestEffort('10 20:19') - [toDate](#todate) - [toDateTime](#todatetime) +## toUnixTimestamp64Milli +## toUnixTimestamp64Micro +## toUnixTimestamp64Nano + +Преобразует значение `DateTime64` в значение `Int64` с фиксированной точностью менее одной секунды. +Входное значение округляется соответствующим образом вверх или вниз в зависимости от его точности. Обратите внимание, что возвращаемое значение - это временная метка в UTC, а не в часовом поясе `DateTime64`. + +**Синтаксис** + +``` sql +toUnixTimestamp64Milli(value) +``` + +**Параметры** + +- `value` — значение `DateTime64` с любой точностью. + +**Возвращаемое значение** + +- Значение `value`, преобразованное в тип данных `Int64`. + +**Примеры** + +Запрос: + +``` sql +WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 +SELECT toUnixTimestamp64Milli(dt64) +``` + +Ответ: + +``` text +┌─toUnixTimestamp64Milli(dt64)─┐ +│ 1568650812345 │ +└──────────────────────────────┘ +``` + +Запрос: + +``` sql +WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 +SELECT toUnixTimestamp64Nano(dt64) +``` + +Ответ: + +``` text +┌─toUnixTimestamp64Nano(dt64)─┐ +│ 1568650812345678000 │ +└─────────────────────────────┘ +``` + +## fromUnixTimestamp64Milli +## fromUnixTimestamp64Micro +## fromUnixTimestamp64Nano + +Преобразует значение `Int64` в значение `DateTime64` с фиксированной точностью менее одной секунды и дополнительным часовым поясом. Входное значение округляется соответствующим образом вверх или вниз в зависимости от его точности. Обратите внимание, что входное значение обрабатывается как метка времени UTC, а не метка времени в заданном (или неявном) часовом поясе. + +**Синтаксис** + +``` sql +fromUnixTimestamp64Milli(value [, ti]) +``` + +**Параметры** + +- `value` — значение типы `Int64` с любой точностью. +- `timezone` — (не обязательный параметр) часовой пояс в формате `String` для возвращаемого результата. + +**Возвращаемое значение** + +- Значение `value`, преобразованное в тип данных `DateTime64`. + +**Пример** + +Запрос: + +``` sql +WITH CAST(1234567891011, 'Int64') AS i64 +SELECT fromUnixTimestamp64Milli(i64, 'UTC') +``` + +Ответ: + +``` text +┌─fromUnixTimestamp64Milli(i64, 'UTC')─┐ +│ 2009-02-13 23:31:31.011 │ +└──────────────────────────────────────┘ +``` + [Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/type_conversion_functions/)