From 230938d3a3082fbf241c9d873571231a69a5f450 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 11 Jul 2020 15:12:42 +0800 Subject: [PATCH 001/123] Refactor joinGet and implement multi-key lookup. --- src/Functions/FunctionJoinGet.cpp | 83 +++++++++---------- src/Functions/FunctionJoinGet.h | 11 +-- src/Interpreters/HashJoin.cpp | 69 ++++++++------- src/Interpreters/HashJoin.h | 10 +-- src/Interpreters/misc.h | 2 +- .../0_stateless/01080_join_get_null.reference | 2 +- .../0_stateless/01080_join_get_null.sql | 12 +-- .../01400_join_get_with_multi_keys.reference | 1 + .../01400_join_get_with_multi_keys.sql | 9 ++ 9 files changed, 104 insertions(+), 95 deletions(-) create mode 100644 tests/queries/0_stateless/01400_join_get_with_multi_keys.reference create mode 100644 tests/queries/0_stateless/01400_join_get_with_multi_keys.sql diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index a33b70684a5..1badc689c6a 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -1,10 +1,10 @@ #include +#include #include #include #include #include -#include #include @@ -16,19 +16,35 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +template +void ExecutableFunctionJoinGet::execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t) +{ + Block keys; + for (size_t i = 2; i < arguments.size(); ++i) + { + auto key = block.getByPosition(arguments[i]); + keys.insert(std::move(key)); + } + block.getByPosition(result) = join->joinGet(keys, result_block); +} + +template +ExecutableFunctionImplPtr FunctionJoinGet::prepare(const Block &, const ColumnNumbers &, size_t) const +{ + return std::make_unique>(join, Block{{return_type->createColumn(), return_type, attr_name}}); +} + static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & context) { - if (arguments.size() != 3) - throw Exception{"Function joinGet takes 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - String join_name; if (const auto * name_col = checkAndGetColumnConst(arguments[0].column.get())) { join_name = name_col->getValue(); } else - throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function joinGet, expected a const string.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception( + "Illegal type " + arguments[0].type->getName() + " of first argument of function joinGet, expected a const string.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); size_t dot = join_name.find('.'); String database_name; @@ -43,10 +59,12 @@ static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & co ++dot; } String table_name = join_name.substr(dot); + if (table_name.empty()) + throw Exception("joinGet does not allow empty table name", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); auto table = DatabaseCatalog::instance().getTable({database_name, table_name}, context); auto storage_join = std::dynamic_pointer_cast(table); if (!storage_join) - throw Exception{"Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception("Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); String attr_name; if (const auto * name_col = checkAndGetColumnConst(arguments[1].column.get())) @@ -54,57 +72,30 @@ static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & co attr_name = name_col->getValue(); } else - throw Exception{"Illegal type " + arguments[1].type->getName() - + " of second argument of function joinGet, expected a const string.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception( + "Illegal type " + arguments[1].type->getName() + " of second argument of function joinGet, expected a const string.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_pair(storage_join, attr_name); } template FunctionBaseImplPtr JoinGetOverloadResolver::build(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const { + if (arguments.size() < 3) + throw Exception( + "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + + ", should be greater or equal to 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); auto [storage_join, attr_name] = getJoin(arguments, context); auto join = storage_join->getJoin(); - DataTypes data_types(arguments.size()); - + DataTypes data_types(arguments.size() - 2); + for (size_t i = 2; i < arguments.size(); ++i) + data_types[i - 2] = arguments[i].type; + auto return_type = join->joinGetCheckAndGetReturnType(data_types, attr_name, or_null); auto table_lock = storage_join->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); - for (size_t i = 0; i < arguments.size(); ++i) - data_types[i] = arguments[i].type; - - auto return_type = join->joinGetReturnType(attr_name, or_null); return std::make_unique>(table_lock, storage_join, join, attr_name, data_types, return_type); } -template -DataTypePtr JoinGetOverloadResolver::getReturnType(const ColumnsWithTypeAndName & arguments) const -{ - auto [storage_join, attr_name] = getJoin(arguments, context); - auto join = storage_join->getJoin(); - return join->joinGetReturnType(attr_name, or_null); -} - - -template -void ExecutableFunctionJoinGet::execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) -{ - auto ctn = block.getByPosition(arguments[2]); - if (isColumnConst(*ctn.column)) - ctn.column = ctn.column->cloneResized(1); - ctn.name = ""; // make sure the key name never collide with the join columns - Block key_block = {ctn}; - join->joinGet(key_block, attr_name, or_null); - auto & result_ctn = key_block.getByPosition(1); - if (isColumnConst(*ctn.column)) - result_ctn.column = ColumnConst::create(result_ctn.column, input_rows_count); - block.getByPosition(result) = result_ctn; -} - -template -ExecutableFunctionImplPtr FunctionJoinGet::prepare(const Block &, const ColumnNumbers &, size_t) const -{ - return std::make_unique>(join, attr_name); -} - void registerFunctionJoinGet(FunctionFactory & factory) { // joinGet diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h index a82da589960..6b3b1202f60 100644 --- a/src/Functions/FunctionJoinGet.h +++ b/src/Functions/FunctionJoinGet.h @@ -13,14 +13,14 @@ template class ExecutableFunctionJoinGet final : public IExecutableFunctionImpl { public: - ExecutableFunctionJoinGet(HashJoinPtr join_, String attr_name_) - : join(std::move(join_)), attr_name(std::move(attr_name_)) {} + ExecutableFunctionJoinGet(HashJoinPtr join_, const Block & result_block_) + : join(std::move(join_)), result_block(result_block_) {} static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet"; bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForLowCardinalityColumns() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; @@ -28,7 +28,7 @@ public: private: HashJoinPtr join; - const String attr_name; + Block result_block; }; template @@ -77,13 +77,14 @@ public: String getName() const override { return name; } FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override; - DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const override; + DataTypePtr getReturnType(const ColumnsWithTypeAndName &) const override { return {}; } // Not used bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForLowCardinalityColumns() const override { return true; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } private: const Context & context; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 27294a57675..ffc806b9e88 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -42,6 +42,7 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; extern const int SET_SIZE_LIMIT_EXCEEDED; extern const int TYPE_MISMATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } namespace @@ -1109,27 +1110,34 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) block = block.cloneWithColumns(std::move(dst_columns)); } -static void checkTypeOfKey(const Block & block_left, const Block & block_right) -{ - const auto & [c1, left_type_origin, left_name] = block_left.safeGetByPosition(0); - const auto & [c2, right_type_origin, right_name] = block_right.safeGetByPosition(0); - auto left_type = removeNullable(left_type_origin); - auto right_type = removeNullable(right_type_origin); - if (!left_type->equals(*right_type)) - throw Exception("Type mismatch of columns to joinGet by: " - + left_name + " " + left_type->getName() + " at left, " - + right_name + " " + right_type->getName() + " at right", - ErrorCodes::TYPE_MISMATCH); -} - - -DataTypePtr HashJoin::joinGetReturnType(const String & column_name, bool or_null) const +DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const { std::shared_lock lock(data->rwlock); + size_t num_keys = data_types.size(); + if (right_table_keys.columns() != num_keys) + throw Exception( + "Number of arguments for function joinGet" + toString(or_null ? "OrNull" : "") + + " doesn't match: passed, should be equal to " + toString(num_keys), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (size_t i = 0; i < num_keys; ++i) + { + const auto & left_type_origin = data_types[i]; + const auto & [c2, right_type_origin, right_name] = right_table_keys.safeGetByPosition(i); + auto left_type = removeNullable(left_type_origin); + auto right_type = removeNullable(right_type_origin); + if (!left_type->equals(*right_type)) + throw Exception( + "Type mismatch in joinGet key " + toString(i) + ": found type " + left_type->getName() + ", while the needed type is " + + right_type->getName(), + ErrorCodes::TYPE_MISMATCH); + } + if (!sample_block_with_columns_to_add.has(column_name)) throw Exception("StorageJoin doesn't contain column " + column_name, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + auto elem = sample_block_with_columns_to_add.getByName(column_name); if (or_null) elem.type = makeNullable(elem.type); @@ -1138,34 +1146,33 @@ DataTypePtr HashJoin::joinGetReturnType(const String & column_name, bool or_null template -void HashJoin::joinGetImpl(Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const +ColumnWithTypeAndName HashJoin::joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const { - joinBlockImpl( - block, {block.getByPosition(0).name}, block_with_columns_to_add, maps_); + // Assemble the key block with correct names. + Block keys; + for (size_t i = 0; i < block.columns(); ++i) + { + auto key = block.getByPosition(i); + key.name = key_names_right[i]; + keys.insert(std::move(key)); + } + + joinBlockImpl( + keys, key_names_right, block_with_columns_to_add, maps_); + return keys.getByPosition(keys.columns() - 1); } -// TODO: support composite key // TODO: return multiple columns as named tuple // TODO: return array of values when strictness == ASTTableJoin::Strictness::All -void HashJoin::joinGet(Block & block, const String & column_name, bool or_null) const +ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const { std::shared_lock lock(data->rwlock); - if (key_names_right.size() != 1) - throw Exception("joinGet only supports StorageJoin containing exactly one key", ErrorCodes::UNSUPPORTED_JOIN_KEYS); - - checkTypeOfKey(block, right_table_keys); - - auto elem = sample_block_with_columns_to_add.getByName(column_name); - if (or_null) - elem.type = makeNullable(elem.type); - elem.column = elem.type->createColumn(); - if ((strictness == ASTTableJoin::Strictness::Any || strictness == ASTTableJoin::Strictness::RightAny) && kind == ASTTableJoin::Kind::Left) { - joinGetImpl(block, {elem}, std::get(data->maps)); + return joinGetImpl(block, block_with_columns_to_add, std::get(data->maps)); } else throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN); diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 67d83d27a6d..025f41ac28f 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -162,11 +162,11 @@ public: */ void joinBlock(Block & block, ExtraBlockPtr & not_processed) override; - /// Infer the return type for joinGet function - DataTypePtr joinGetReturnType(const String & column_name, bool or_null) const; + /// Check joinGet arguments and infer the return type. + DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const; - /// Used by joinGet function that turns StorageJoin into a dictionary - void joinGet(Block & block, const String & column_name, bool or_null) const; + /// Used by joinGet function that turns StorageJoin into a dictionary. + ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const; /** Keep "totals" (separate part of dataset, see WITH TOTALS) to use later. */ @@ -383,7 +383,7 @@ private: void joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const; template - void joinGetImpl(Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const; + ColumnWithTypeAndName joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const; static Type chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes); }; diff --git a/src/Interpreters/misc.h b/src/Interpreters/misc.h index 094dfbbbb81..cae2691ca1f 100644 --- a/src/Interpreters/misc.h +++ b/src/Interpreters/misc.h @@ -28,7 +28,7 @@ inline bool functionIsLikeOperator(const std::string & name) inline bool functionIsJoinGet(const std::string & name) { - return name == "joinGet" || startsWith(name, "dictGet"); + return startsWith(name, "joinGet"); } inline bool functionIsDictGet(const std::string & name) diff --git a/tests/queries/0_stateless/01080_join_get_null.reference b/tests/queries/0_stateless/01080_join_get_null.reference index bfde072a796..0cfbf08886f 100644 --- a/tests/queries/0_stateless/01080_join_get_null.reference +++ b/tests/queries/0_stateless/01080_join_get_null.reference @@ -1 +1 @@ -2 2 +2 diff --git a/tests/queries/0_stateless/01080_join_get_null.sql b/tests/queries/0_stateless/01080_join_get_null.sql index 71e7ddf8e75..9f782452d34 100644 --- a/tests/queries/0_stateless/01080_join_get_null.sql +++ b/tests/queries/0_stateless/01080_join_get_null.sql @@ -1,12 +1,12 @@ DROP TABLE IF EXISTS test_joinGet; -DROP TABLE IF EXISTS test_join_joinGet; -CREATE TABLE test_joinGet(id Int32, user_id Nullable(Int32)) Engine = Memory(); -CREATE TABLE test_join_joinGet(user_id Int32, name String) Engine = Join(ANY, LEFT, user_id); +CREATE TABLE test_joinGet(user_id Nullable(Int32), name String) Engine = Join(ANY, LEFT, user_id); -INSERT INTO test_join_joinGet VALUES (2, 'a'), (6, 'b'), (10, 'c'); +INSERT INTO test_joinGet VALUES (2, 'a'), (6, 'b'), (10, 'c'), (null, 'd'); -SELECT 2 id, toNullable(toInt32(2)) user_id WHERE joinGet(test_join_joinGet, 'name', user_id) != ''; +SELECT toNullable(toInt32(2)) user_id WHERE joinGet(test_joinGet, 'name', user_id) != ''; + +-- If the JOIN keys are Nullable fields, the rows where at least one of the keys has the value NULL are not joined. +SELECT cast(null AS Nullable(Int32)) user_id WHERE joinGet(test_joinGet, 'name', user_id) != ''; DROP TABLE test_joinGet; -DROP TABLE test_join_joinGet; diff --git a/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference b/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference new file mode 100644 index 00000000000..49d59571fbf --- /dev/null +++ b/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference @@ -0,0 +1 @@ +0.1 diff --git a/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql b/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql new file mode 100644 index 00000000000..73068270762 --- /dev/null +++ b/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS test_joinGet; + +CREATE TABLE test_joinGet(a String, b String, c Float64) ENGINE = Join(any, left, a, b); + +INSERT INTO test_joinGet VALUES ('ab', '1', 0.1), ('ab', '2', 0.2), ('cd', '3', 0.3); + +SELECT joinGet(test_joinGet, 'c', 'ab', '1'); + +DROP TABLE test_joinGet; From 40504f6a6e9b54bdcdb0c63a5724648bf5bc04f5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 31 Jul 2020 17:57:00 +0300 Subject: [PATCH 002/123] 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 003/123] 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 004/123] 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 005/123] 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 602535396d0ac58c1885ef1d1d3e7c085335f059 Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 16:36:27 +0800 Subject: [PATCH 006/123] Refactor, move function declarations --- src/Formats/FormatFactory.cpp | 100 +++++++++++++++++++++++++++++----- src/Formats/FormatFactory.h | 69 ----------------------- 2 files changed, 85 insertions(+), 84 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 935d31d6541..871098e00c0 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -323,13 +323,85 @@ void FormatFactory::registerFileSegmentationEngine(const String & name, FileSegm target = std::move(file_segmentation_engine); } +/// File Segmentation Engines for parallel reading + +void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); +void registerFileSegmentationEngineCSV(FormatFactory & factory); +void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); +void registerFileSegmentationEngineRegexp(FormatFactory & factory); +void registerFileSegmentationEngineJSONAsString(FormatFactory & factory); + +/// Formats for both input/output. + +void registerInputFormatNative(FormatFactory & factory); +void registerOutputFormatNative(FormatFactory & factory); + +void registerInputFormatProcessorNative(FormatFactory & factory); +void registerOutputFormatProcessorNative(FormatFactory & factory); +void registerInputFormatProcessorRowBinary(FormatFactory & factory); +void registerOutputFormatProcessorRowBinary(FormatFactory & factory); +void registerInputFormatProcessorTabSeparated(FormatFactory & factory); +void registerOutputFormatProcessorTabSeparated(FormatFactory & factory); +void registerInputFormatProcessorValues(FormatFactory & factory); +void registerOutputFormatProcessorValues(FormatFactory & factory); +void registerInputFormatProcessorCSV(FormatFactory & factory); +void registerOutputFormatProcessorCSV(FormatFactory & factory); +void registerInputFormatProcessorTSKV(FormatFactory & factory); +void registerOutputFormatProcessorTSKV(FormatFactory & factory); +void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); +void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); +void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); +void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); +void registerInputFormatProcessorProtobuf(FormatFactory & factory); +void registerOutputFormatProcessorProtobuf(FormatFactory & factory); +void registerInputFormatProcessorTemplate(FormatFactory & factory); +void registerOutputFormatProcessorTemplate(FormatFactory & factory); +void registerInputFormatProcessorMsgPack(FormatFactory & factory); +void registerOutputFormatProcessorMsgPack(FormatFactory & factory); +void registerInputFormatProcessorORC(FormatFactory & factory); +void registerOutputFormatProcessorORC(FormatFactory & factory); +void registerInputFormatProcessorParquet(FormatFactory & factory); +void registerOutputFormatProcessorParquet(FormatFactory & factory); +void registerInputFormatProcessorArrow(FormatFactory & factory); +void registerOutputFormatProcessorArrow(FormatFactory & factory); +void registerInputFormatProcessorAvro(FormatFactory & factory); +void registerOutputFormatProcessorAvro(FormatFactory & factory); + +/// Output only (presentational) formats. + +void registerOutputFormatNull(FormatFactory & factory); + +void registerOutputFormatProcessorPretty(FormatFactory & factory); +void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory); +void registerOutputFormatProcessorPrettySpace(FormatFactory & factory); +void registerOutputFormatProcessorVertical(FormatFactory & factory); +void registerOutputFormatProcessorJSON(FormatFactory & factory); +void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); +void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); +void registerOutputFormatProcessorXML(FormatFactory & factory); +void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); +void registerOutputFormatProcessorNull(FormatFactory & factory); +void registerOutputFormatProcessorMySQLWire(FormatFactory & factory); +void registerOutputFormatProcessorMarkdown(FormatFactory & factory); +void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory); + +/// Input only formats. + +void registerInputFormatProcessorRegexp(FormatFactory & factory); +void registerInputFormatProcessorJSONAsString(FormatFactory & factory); +void registerInputFormatProcessorCapnProto(FormatFactory & factory); + FormatFactory::FormatFactory() { + registerFileSegmentationEngineTabSeparated(*this); + registerFileSegmentationEngineCSV(*this); + registerFileSegmentationEngineJSONEachRow(*this); + registerFileSegmentationEngineRegexp(*this); + registerFileSegmentationEngineJSONAsString(*this); + registerInputFormatNative(*this); registerOutputFormatNative(*this); - registerOutputFormatProcessorJSONEachRowWithProgress(*this); - registerInputFormatProcessorNative(*this); registerOutputFormatProcessorNative(*this); registerInputFormatProcessorRowBinary(*this); @@ -348,8 +420,11 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSONCompactEachRow(*this); registerInputFormatProcessorProtobuf(*this); registerOutputFormatProcessorProtobuf(*this); + registerInputFormatProcessorTemplate(*this); + registerOutputFormatProcessorTemplate(*this); + registerInputFormatProcessorMsgPack(*this); + registerOutputFormatProcessorMsgPack(*this); #if !defined(ARCADIA_BUILD) - registerInputFormatProcessorCapnProto(*this); registerInputFormatProcessorORC(*this); registerOutputFormatProcessorORC(*this); registerInputFormatProcessorParquet(*this); @@ -359,18 +434,6 @@ FormatFactory::FormatFactory() registerInputFormatProcessorAvro(*this); registerOutputFormatProcessorAvro(*this); #endif - registerInputFormatProcessorTemplate(*this); - registerOutputFormatProcessorTemplate(*this); - registerInputFormatProcessorRegexp(*this); - registerInputFormatProcessorMsgPack(*this); - registerOutputFormatProcessorMsgPack(*this); - registerInputFormatProcessorJSONAsString(*this); - - registerFileSegmentationEngineTabSeparated(*this); - registerFileSegmentationEngineCSV(*this); - registerFileSegmentationEngineJSONEachRow(*this); - registerFileSegmentationEngineRegexp(*this); - registerFileSegmentationEngineJSONAsString(*this); registerOutputFormatNull(*this); @@ -380,12 +443,19 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorVertical(*this); registerOutputFormatProcessorJSON(*this); registerOutputFormatProcessorJSONCompact(*this); + registerOutputFormatProcessorJSONEachRowWithProgress(*this); registerOutputFormatProcessorXML(*this); registerOutputFormatProcessorODBCDriver2(*this); registerOutputFormatProcessorNull(*this); registerOutputFormatProcessorMySQLWire(*this); registerOutputFormatProcessorMarkdown(*this); registerOutputFormatProcessorPostgreSQLWire(*this); + + registerInputFormatProcessorRegexp(*this); + registerInputFormatProcessorJSONAsString(*this); +#if !defined(ARCADIA_BUILD) + registerInputFormatProcessorCapnProto(*this); +#endif } FormatFactory & FormatFactory::instance() diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index f0d2b7826a0..54bff1eefc6 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -141,73 +141,4 @@ private: const Creators & getCreators(const String & name) const; }; -/// Formats for both input/output. - -void registerInputFormatNative(FormatFactory & factory); -void registerOutputFormatNative(FormatFactory & factory); - -void registerInputFormatProcessorNative(FormatFactory & factory); -void registerOutputFormatProcessorNative(FormatFactory & factory); -void registerInputFormatProcessorRowBinary(FormatFactory & factory); -void registerOutputFormatProcessorRowBinary(FormatFactory & factory); -void registerInputFormatProcessorTabSeparated(FormatFactory & factory); -void registerOutputFormatProcessorTabSeparated(FormatFactory & factory); -void registerInputFormatProcessorValues(FormatFactory & factory); -void registerOutputFormatProcessorValues(FormatFactory & factory); -void registerInputFormatProcessorCSV(FormatFactory & factory); -void registerOutputFormatProcessorCSV(FormatFactory & factory); -void registerInputFormatProcessorTSKV(FormatFactory & factory); -void registerOutputFormatProcessorTSKV(FormatFactory & factory); -void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); -void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); -void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); -void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); -void registerInputFormatProcessorParquet(FormatFactory & factory); -void registerOutputFormatProcessorParquet(FormatFactory & factory); -void registerInputFormatProcessorArrow(FormatFactory & factory); -void registerOutputFormatProcessorArrow(FormatFactory & factory); -void registerInputFormatProcessorProtobuf(FormatFactory & factory); -void registerOutputFormatProcessorProtobuf(FormatFactory & factory); -void registerInputFormatProcessorAvro(FormatFactory & factory); -void registerOutputFormatProcessorAvro(FormatFactory & factory); -void registerInputFormatProcessorTemplate(FormatFactory & factory); -void registerOutputFormatProcessorTemplate(FormatFactory & factory); -void registerInputFormatProcessorMsgPack(FormatFactory & factory); -void registerOutputFormatProcessorMsgPack(FormatFactory & factory); -void registerInputFormatProcessorORC(FormatFactory & factory); -void registerOutputFormatProcessorORC(FormatFactory & factory); - - -/// File Segmentation Engines for parallel reading - -void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); -void registerFileSegmentationEngineCSV(FormatFactory & factory); -void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); -void registerFileSegmentationEngineRegexp(FormatFactory & factory); -void registerFileSegmentationEngineJSONAsString(FormatFactory & factory); - -/// Output only (presentational) formats. - -void registerOutputFormatNull(FormatFactory & factory); - -void registerOutputFormatProcessorPretty(FormatFactory & factory); -void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory); -void registerOutputFormatProcessorPrettySpace(FormatFactory & factory); -void registerOutputFormatProcessorPrettyASCII(FormatFactory & factory); -void registerOutputFormatProcessorVertical(FormatFactory & factory); -void registerOutputFormatProcessorJSON(FormatFactory & factory); -void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); -void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); -void registerOutputFormatProcessorXML(FormatFactory & factory); -void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); -void registerOutputFormatProcessorNull(FormatFactory & factory); -void registerOutputFormatProcessorMySQLWire(FormatFactory & factory); -void registerOutputFormatProcessorMarkdown(FormatFactory & factory); -void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory); - -/// Input only formats. -void registerInputFormatProcessorCapnProto(FormatFactory & factory); -void registerInputFormatProcessorRegexp(FormatFactory & factory); -void registerInputFormatProcessorJSONAsString(FormatFactory & factory); - } From 86fa185bb6fbf8e1e6bc6044a7f4e523477e84db Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 17:06:28 +0800 Subject: [PATCH 007/123] Add JSONStrings formats --- src/Formats/FormatFactory.cpp | 6 + .../Impl/JSONStringsEachRowRowInputFormat.cpp | 245 ++++++++++++++++++ .../Impl/JSONStringsEachRowRowInputFormat.h | 54 ++++ .../JSONStringsEachRowRowOutputFormat.cpp | 117 +++++++++ .../Impl/JSONStringsEachRowRowOutputFormat.h | 45 ++++ .../Impl/JSONStringsRowOutputFormat.cpp | 93 +++++++ .../Formats/Impl/JSONStringsRowOutputFormat.h | 43 +++ src/Processors/ya.make | 3 + 8 files changed, 606 insertions(+) create mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h create mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp create mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h create mode 100644 src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp create mode 100644 src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 871098e00c0..cb378fbea96 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -352,6 +352,8 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); +void registerInputFormatProcessorJSONStringsEachRow(FormatFactory & factory); +void registerOutputFormatProcessorJSONStringsEachRow(FormatFactory & factory); void registerInputFormatProcessorProtobuf(FormatFactory & factory); void registerOutputFormatProcessorProtobuf(FormatFactory & factory); void registerInputFormatProcessorTemplate(FormatFactory & factory); @@ -378,6 +380,7 @@ void registerOutputFormatProcessorVertical(FormatFactory & factory); void registerOutputFormatProcessorJSON(FormatFactory & factory); void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); +void registerOutputFormatProcessorJSONStrings(FormatFactory & factory); void registerOutputFormatProcessorXML(FormatFactory & factory); void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); void registerOutputFormatProcessorNull(FormatFactory & factory); @@ -418,6 +421,8 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSONEachRow(*this); registerInputFormatProcessorJSONCompactEachRow(*this); registerOutputFormatProcessorJSONCompactEachRow(*this); + registerInputFormatProcessorJSONStringsEachRow(*this); + registerOutputFormatProcessorJSONStringsEachRow(*this); registerInputFormatProcessorProtobuf(*this); registerOutputFormatProcessorProtobuf(*this); registerInputFormatProcessorTemplate(*this); @@ -444,6 +449,7 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSON(*this); registerOutputFormatProcessorJSONCompact(*this); registerOutputFormatProcessorJSONEachRowWithProgress(*this); + registerOutputFormatProcessorJSONStrings(*this); registerOutputFormatProcessorXML(*this); registerOutputFormatProcessorODBCDriver2(*this); registerOutputFormatProcessorNull(*this); diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp new file mode 100644 index 00000000000..fff44a204fb --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp @@ -0,0 +1,245 @@ +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; + extern const int CANNOT_READ_ALL_DATA; +} + + +JSONStringsEachRowRowInputFormat::JSONStringsEachRowRowInputFormat(ReadBuffer & in_, + const Block & header_, + Params params_, + const FormatSettings & format_settings_, + bool with_names_) + : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_) +{ + const auto & sample = getPort().getHeader(); + size_t num_columns = sample.columns(); + + data_types.resize(num_columns); + column_indexes_by_names.reserve(num_columns); + + for (size_t i = 0; i < num_columns; ++i) + { + const auto & column_info = sample.getByPosition(i); + + data_types[i] = column_info.type; + column_indexes_by_names.emplace(column_info.name, i); + } +} + +void JSONStringsEachRowRowInputFormat::resetParser() +{ + IRowInputFormat::resetParser(); + column_indexes_for_input_fields.clear(); + not_seen_columns.clear(); +} + +void JSONStringsEachRowRowInputFormat::readPrefix() +{ + /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. + skipBOMIfExists(in); + + if (with_names) + { + size_t num_columns = getPort().getHeader().columns(); + read_columns.assign(num_columns, false); + + assertChar('[', in); + do + { + skipWhitespaceIfAny(in); + String column_name; + readJSONString(column_name, in); + addInputColumn(column_name); + skipWhitespaceIfAny(in); + } + while (checkChar(',', in)); + assertChar(']', in); + skipEndOfLine(); + + /// Type checking + assertChar('[', in); + for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i) + { + skipWhitespaceIfAny(in); + String data_type; + readJSONString(data_type, in); + + if (column_indexes_for_input_fields[i] && + data_types[*column_indexes_for_input_fields[i]]->getName() != data_type) + { + throw Exception( + "Type of '" + getPort().getHeader().getByPosition(*column_indexes_for_input_fields[i]).name + + "' must be " + data_types[*column_indexes_for_input_fields[i]]->getName() + + ", not " + data_type, + ErrorCodes::INCORRECT_DATA + ); + } + + if (i != column_indexes_for_input_fields.size() - 1) + assertChar(',', in); + skipWhitespaceIfAny(in); + } + assertChar(']', in); + } + else + { + size_t num_columns = getPort().getHeader().columns(); + read_columns.assign(num_columns, true); + column_indexes_for_input_fields.resize(num_columns); + + for (size_t i = 0; i < num_columns; ++i) + { + column_indexes_for_input_fields[i] = i; + } + } + + for (size_t i = 0; i < read_columns.size(); ++i) + { + if (!read_columns[i]) + { + not_seen_columns.emplace_back(i); + } + } +} + +void JSONStringsEachRowRowInputFormat::addInputColumn(const String & column_name) +{ + names_of_columns.emplace_back(column_name); + + const auto column_it = column_indexes_by_names.find(column_name); + if (column_it == column_indexes_by_names.end()) + { + if (format_settings.skip_unknown_fields) + { + column_indexes_for_input_fields.push_back(std::nullopt); + return; + } + + throw Exception( + "Unknown field found in JSONStringsEachRow header: '" + column_name + "' " + + "at position " + std::to_string(column_indexes_for_input_fields.size()) + + "\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed", + ErrorCodes::INCORRECT_DATA + ); + } + + const auto column_index = column_it->second; + + if (read_columns[column_index]) + throw Exception("Duplicate field found while parsing JSONStringsEachRow header: " + column_name, ErrorCodes::INCORRECT_DATA); + + read_columns[column_index] = true; + column_indexes_for_input_fields.emplace_back(column_index); +} + +bool JSONStringsEachRowRowInputFormat::readRow(DB::MutableColumns &columns, DB::RowReadExtension &ext) +{ + skipEndOfLine(); + + if (in.eof()) + return false; + + size_t num_columns = columns.size(); + + read_columns.assign(num_columns, false); + + assertChar('[', in); + for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column) + { + const auto & table_column = column_indexes_for_input_fields[file_column]; + if (table_column) + { + readField(*table_column, columns); + } + else + { + skipJSONField(in, StringRef(names_of_columns[file_column])); + } + + skipWhitespaceIfAny(in); + if (in.eof()) + throw Exception("Unexpected end of stream while parsing JSONStringsEachRow format", ErrorCodes::CANNOT_READ_ALL_DATA); + if (file_column + 1 != column_indexes_for_input_fields.size()) + { + assertChar(',', in); + skipWhitespaceIfAny(in); + } + } + assertChar(']', in); + + for (const auto & name : not_seen_columns) + columns[name]->insertDefault(); + + ext.read_columns = read_columns; + return true; +} + +void JSONStringsEachRowRowInputFormat::skipEndOfLine() +{ + skipWhitespaceIfAny(in); + if (!in.eof() && (*in.position() == ',' || *in.position() == ';')) + ++in.position(); + + skipWhitespaceIfAny(in); +} + +void JSONStringsEachRowRowInputFormat::readField(size_t index, MutableColumns & columns) +{ + try + { + read_columns[index] = true; + const auto & type = data_types[index]; + + String str; + readJSONString(str, in); + + ReadBufferFromString buf(str); + + type->deserializeAsWholeText(*columns[index], buf, format_settings); + } + catch (Exception & e) + { + e.addMessage("(while read the value of key " + getPort().getHeader().getByPosition(index).name + ")"); + throw; + } +} + +void JSONStringsEachRowRowInputFormat::syncAfterError() +{ + skipToUnescapedNextLineOrEOF(in); +} + +void registerInputFormatProcessorJSONStringsEachRow(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("JSONStringsEachRow", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, false); + }); + + factory.registerInputFormatProcessor("JSONStringsEachRowWithNamesAndTypes", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, true); + }); +} + +} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h new file mode 100644 index 00000000000..ec0a0f7bad9 --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h @@ -0,0 +1,54 @@ +#pragma once + +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class ReadBuffer; + +/** A stream for reading data in JSONStringsEachRow and JSONStringsEachRowWithNamesAndTypes formats +*/ +class JSONStringsEachRowRowInputFormat : public IRowInputFormat +{ +public: + JSONStringsEachRowRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_, bool with_names_); + + String getName() const override { return "JSONStringsEachRowRowInputFormat"; } + + + void readPrefix() override; + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + bool allowSyncAfterError() const override { return true; } + void syncAfterError() override; + void resetParser() override; + +private: + void addInputColumn(const String & column_name); + void skipEndOfLine(); + void readField(size_t index, MutableColumns & columns); + + const FormatSettings format_settings; + + using IndexesMap = std::unordered_map; + IndexesMap column_indexes_by_names; + + using OptionalIndexes = std::vector>; + OptionalIndexes column_indexes_for_input_fields; + + DataTypes data_types; + std::vector read_columns; + std::vector not_seen_columns; + + /// This is for the correct exceptions in skipping unknown fields. + std::vector names_of_columns; + + bool with_names; +}; + +} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp new file mode 100644 index 00000000000..75007ea236e --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp @@ -0,0 +1,117 @@ +#include +#include +#include +#include + + +namespace DB +{ + + +JSONStringsEachRowRowOutputFormat::JSONStringsEachRowRowOutputFormat(WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool with_names_) + : IRowOutputFormat(header_, out_, callback), settings(settings_), with_names(with_names_) +{ + const auto & sample = getPort(PortKind::Main).getHeader(); + NamesAndTypesList columns(sample.getNamesAndTypesList()); + fields.assign(columns.begin(), columns.end()); +} + + +void JSONStringsEachRowRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), out, settings); +} + + +void JSONStringsEachRowRowOutputFormat::writeFieldDelimiter() +{ + writeCString(", ", out); +} + + +void JSONStringsEachRowRowOutputFormat::writeRowStartDelimiter() +{ + writeChar('[', out); +} + + +void JSONStringsEachRowRowOutputFormat::writeRowEndDelimiter() +{ + writeCString("]\n", out); +} + +void JSONStringsEachRowRowOutputFormat::writeTotals(const Columns & columns, size_t row_num) +{ + writeChar('\n', out); + size_t num_columns = columns.size(); + writeChar('[', out); + for (size_t i = 0; i < num_columns; ++i) + { + if (i != 0) + JSONStringsEachRowRowOutputFormat::writeFieldDelimiter(); + + JSONStringsEachRowRowOutputFormat::writeField(*columns[i], *types[i], row_num); + } + writeCString("]\n", out); +} + +void JSONStringsEachRowRowOutputFormat::writePrefix() +{ + if (with_names) + { + writeChar('[', out); + for (size_t i = 0; i < fields.size(); ++i) + { + writeChar('\"', out); + writeString(fields[i].name, out); + writeChar('\"', out); + if (i != fields.size() - 1) + writeCString(", ", out); + } + writeCString("]\n[", out); + for (size_t i = 0; i < fields.size(); ++i) + { + writeJSONString(fields[i].type->getName(), out, settings); + if (i != fields.size() - 1) + writeCString(", ", out); + } + writeCString("]\n", out); + } +} + +void JSONStringsEachRowRowOutputFormat::consumeTotals(DB::Chunk chunk) +{ + if (with_names) + IRowOutputFormat::consumeTotals(std::move(chunk)); +} + +void registerOutputFormatProcessorJSONStringsEachRow(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("JSONStringsEachRow", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONStringsEachRowWithNamesAndTypes", []( + WriteBuffer &buf, + const Block &sample, + FormatFactory::WriteCallback callback, + const FormatSettings &format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); + }); +} + + +} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h new file mode 100644 index 00000000000..1d43a333da1 --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +/** The stream for outputting data in JSON format, by object per line. + * Does not validate UTF-8. + */ +class JSONStringsEachRowRowOutputFormat : public IRowOutputFormat +{ +public: + JSONStringsEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_, bool with_names); + + String getName() const override { return "JSONStringsEachRowRowOutputFormat"; } + + void writePrefix() override; + + void writeBeforeTotals() override {} + void writeTotals(const Columns & columns, size_t row_num) override; + void writeAfterTotals() override {} + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + +protected: + void consumeTotals(Chunk) override; + /// No extremes. + void consumeExtremes(Chunk) override {} + +private: + FormatSettings settings; + + NamesAndTypes fields; + + bool with_names; +}; +} diff --git a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp new file mode 100644 index 00000000000..6ccb315f73f --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp @@ -0,0 +1,93 @@ +#include +#include + +#include + + +namespace DB +{ + +JSONStringsRowOutputFormat::JSONStringsRowOutputFormat( + WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_) + : JSONRowOutputFormat(out_, header, callback, settings_) +{ +} + + +void JSONStringsRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), *ostr, settings); + ++field_number; +} + + +void JSONStringsRowOutputFormat::writeFieldDelimiter() +{ + writeCString(", ", *ostr); +} + +void JSONStringsRowOutputFormat::writeTotalsFieldDelimiter() +{ + writeCString(",", *ostr); +} + + +void JSONStringsRowOutputFormat::writeRowStartDelimiter() +{ + writeCString("\t\t[", *ostr); +} + + +void JSONStringsRowOutputFormat::writeRowEndDelimiter() +{ + writeChar(']', *ostr); + field_number = 0; + ++row_count; +} + +void JSONStringsRowOutputFormat::writeBeforeTotals() +{ + writeCString(",\n", *ostr); + writeChar('\n', *ostr); + writeCString("\t\"totals\": [", *ostr); +} + +void JSONStringsRowOutputFormat::writeAfterTotals() +{ + writeChar(']', *ostr); +} + +void JSONStringsRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num) +{ + writeCString("\t\t\"", *ostr); + writeCString(title, *ostr); + writeCString("\": [", *ostr); + + size_t extremes_columns = columns.size(); + for (size_t i = 0; i < extremes_columns; ++i) + { + if (i != 0) + writeTotalsFieldDelimiter(); + + writeField(*columns[i], *types[i], row_num); + } + + writeChar(']', *ostr); +} + +void registerOutputFormatProcessorJSONStrings(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("JSONStrings", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings); + }); +} + +} diff --git a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h new file mode 100644 index 00000000000..b221bc9ee36 --- /dev/null +++ b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h @@ -0,0 +1,43 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +struct FormatSettings; + +/** The stream for outputting data in the JSONStrings format. + */ +class JSONStringsRowOutputFormat : public JSONRowOutputFormat +{ +public: + JSONStringsRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_); + + String getName() const override { return "JSONStringsRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + + void writeBeforeTotals() override; + void writeAfterTotals() override; + +protected: + void writeExtremesElement(const char * title, const Columns & columns, size_t row_num) override; + + void writeTotalsField(const IColumn & column, const IDataType & type, size_t row_num) override + { + return writeField(column, type, row_num); + } + + void writeTotalsFieldDelimiter() override; + +}; + +} diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 30de38fedbd..27893674859 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -31,6 +31,9 @@ SRCS( Formats/Impl/JSONEachRowRowOutputFormat.cpp Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp Formats/Impl/JSONRowOutputFormat.cpp + Formats/Impl/JSONStringsEachRowRowInputFormat.cpp + Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp + Formats/Impl/JSONStringsRowOutputFormat.cpp Formats/Impl/MarkdownRowOutputFormat.cpp Formats/Impl/MsgPackRowInputFormat.cpp Formats/Impl/MsgPackRowOutputFormat.cpp From c25a99aaf58108651149930db5ef86e1313120c4 Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 17:47:37 +0800 Subject: [PATCH 008/123] Add tests --- .../01446_JSONStringsEachRow.reference | 47 ++++++++++++++ .../0_stateless/01446_JSONStringsEachRow.sql | 63 +++++++++++++++++++ .../0_stateless/01447_JSONStrings.reference | 43 +++++++++++++ .../queries/0_stateless/01447_JSONStrings.sql | 8 +++ 4 files changed, 161 insertions(+) create mode 100644 tests/queries/0_stateless/01446_JSONStringsEachRow.reference create mode 100644 tests/queries/0_stateless/01446_JSONStringsEachRow.sql create mode 100644 tests/queries/0_stateless/01447_JSONStrings.reference create mode 100644 tests/queries/0_stateless/01447_JSONStrings.sql diff --git a/tests/queries/0_stateless/01446_JSONStringsEachRow.reference b/tests/queries/0_stateless/01446_JSONStringsEachRow.reference new file mode 100644 index 00000000000..0b05f050b29 --- /dev/null +++ b/tests/queries/0_stateless/01446_JSONStringsEachRow.reference @@ -0,0 +1,47 @@ +1 +["1", "a"] +["2", "b"] +["3", "c"] +2 +["a", "1"] +["b", "1"] +["c", "1"] +3 +["value", "name"] +["UInt8", "String"] +["1", "a"] +["2", "b"] +["3", "c"] +4 +["name", "c"] +["String", "UInt64"] +["a", "1"] +["b", "1"] +["c", "1"] + +["", "3"] +5 +["first", "1", "2", "0"] +["second", "2", "0", "6"] +6 +["first", "1", "2", "0"] +["second", "2", "0", "6"] +7 +["16", "[15,16,17]", "['first','second','third']"] +8 +["first", "1", "2", "0"] +["second", "2", "0", "6"] +9 +["first", "1", "2", "0"] +["second", "2", "0", "6"] +10 +["first", "1", "16", "8"] +["second", "2", "32", "8"] +11 +["v1", "v2", "v3", "v4"] +["String", "UInt8", "UInt16", "UInt8"] +["", "2", "3", "1"] +12 +["v1", "n.id", "n.name"] +["UInt8", "Array(UInt8)", "Array(String)"] +["16", "[15,16,17]", "['first','second','third']"] diff --git a/tests/queries/0_stateless/01446_JSONStringsEachRow.sql b/tests/queries/0_stateless/01446_JSONStringsEachRow.sql new file mode 100644 index 00000000000..f461b217fe4 --- /dev/null +++ b/tests/queries/0_stateless/01446_JSONStringsEachRow.sql @@ -0,0 +1,63 @@ +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; +SELECT 1; +/* Check JSONStringsEachRow Output */ +CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value; +INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c'); +SELECT * FROM test_table FORMAT JSONStringsEachRow; +SELECT 2; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRow; +SELECT 3; +/* Check JSONStringsEachRowWithNamesAndTypes Output */ +SELECT * FROM test_table FORMAT JSONStringsEachRowWithNamesAndTypes; +SELECT 4; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRowWithNamesAndTypes; +DROP TABLE IF EXISTS test_table; +SELECT 5; +/* Check JSONStringsEachRow Input */ +CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; +INSERT INTO test_table FORMAT JSONStringsEachRow ["first", "1", "2", "NULL"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 6; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONStringsEachRow ["first", "1", "2", "ᴺᵁᴸᴸ"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 7; +/* Check Nested */ +CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; +INSERT INTO test_table_2 FORMAT JSONStringsEachRow ["16", "[15, 16, 17]", "['first', 'second', 'third']"]; +SELECT * FROM test_table_2 FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table_2; +SELECT 8; +/* Check JSONStringsEachRowWithNamesAndTypes Output */ +SET input_format_null_as_default = 0; +INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"]["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 9; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +SELECT 10; +/* Check Header */ +TRUNCATE TABLE test_table; +SET input_format_skip_unknown_fields = 1; +INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", "1", "32"]["second", "2", "64"]; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +SELECT 11; +TRUNCATE TABLE test_table; +INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"]["1", "2", "3"] +SELECT * FROM test_table FORMAT JSONStringsEachRowWithNamesAndTypes; +SELECT 12; +/* Check Nested */ +INSERT INTO test_table_2 FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"]["16", "[15, 16, 17]", "['first', 'second', 'third']"]; +SELECT * FROM test_table_2 FORMAT JSONStringsEachRowWithNamesAndTypes; + +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; diff --git a/tests/queries/0_stateless/01447_JSONStrings.reference b/tests/queries/0_stateless/01447_JSONStrings.reference new file mode 100644 index 00000000000..58af593dc77 --- /dev/null +++ b/tests/queries/0_stateless/01447_JSONStrings.reference @@ -0,0 +1,43 @@ +{ + "meta": + [ + { + "name": "1", + "type": "UInt8" + }, + { + "name": "'a'", + "type": "String" + }, + { + "name": "[1, 2, 3]", + "type": "Array(UInt8)" + }, + { + "name": "tuple(1, 'a')", + "type": "Tuple(UInt8, String)" + }, + { + "name": "NULL", + "type": "Nullable(Nothing)" + }, + { + "name": "nan", + "type": "Float64" + } + ], + + "data": + [ + ["1", "a", "[1,2,3]", "(1,'a')", "ᴺᵁᴸᴸ", "nan"] + ], + + "rows": 1, + + "statistics": + { + "elapsed": 0.00068988, + "rows_read": 1, + "bytes_read": 1 + } +} diff --git a/tests/queries/0_stateless/01447_JSONStrings.sql b/tests/queries/0_stateless/01447_JSONStrings.sql new file mode 100644 index 00000000000..7d89f0f5087 --- /dev/null +++ b/tests/queries/0_stateless/01447_JSONStrings.sql @@ -0,0 +1,8 @@ +SELECT + 1, + 'a', + [1, 2, 3], + (1, 'a'), + null, + nan +FORMAT JSONStrings; From 6d37c9d2a68ca3f6ae39c9bc0bb99424d7fc236e Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 17:48:03 +0800 Subject: [PATCH 009/123] Update docs about formats --- docs/en/interfaces/formats.md | 26 +++++++++++++++++++------- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 9d3965b4a9c..9c7c2dda8dc 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -26,7 +26,10 @@ The supported formats are: | [VerticalRaw](#verticalraw) | ✗ | ✔ | | [JSON](#json) | ✗ | ✔ | | [JSONCompact](#jsoncompact) | ✗ | ✔ | +| [JSONStrings](#jsonstrings) | ✗ | ✔ | | [JSONEachRow](#jsoneachrow) | ✔ | ✔ | +| [JSONCompactEachRow](#jsoncompacteachrow) | ✔ | ✔ | +| [JSONStringsEachRow](#jsonstringseachrow) | ✔ | ✔ | | [TSKV](#tskv) | ✔ | ✔ | | [Pretty](#pretty) | ✗ | ✔ | | [PrettyCompact](#prettycompact) | ✗ | ✔ | @@ -470,7 +473,7 @@ See also the [JSONEachRow](#jsoneachrow) format. ## JSONCompact {#jsoncompact} -Differs from JSON only in that data rows are output in arrays, not in objects. +Differs from JSON only in that data rows are output in arrays of any element type, not in objects. Example: @@ -514,17 +517,26 @@ Example: This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). See also the `JSONEachRow` format. -## JSONEachRow {#jsoneachrow} +## JSONStrings {#jsonstrings} -When using this format, ClickHouse outputs rows as separated, newline-delimited JSON objects, but the data as a whole is not valid JSON. +Differs from JSON and JSONCompact only in that data rows are output in arrays of strings. + +This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). +See also the `JSONEachRow` format. + +## JSONEachRow {#jsoneachrow} +## JSONCompactEachRow {#jsoncompacteachrow} +## JSONStringsEachRow {#jsonstringseachrow} + +When using these formats, ClickHouse outputs rows as separated, newline-delimited JSON values, but the data as a whole is not valid JSON. ``` json -{"SearchPhrase":"curtain designs","count()":"1064"} -{"SearchPhrase":"baku","count()":"1000"} -{"SearchPhrase":"","count()":"8267016"} +{"some_int":42,"some_str":"hello","some_tuple":[1,"a"]} // JSONEachRow +[42,"hello",[1,"a"]] // JSONCompactEachRow +["42","hello","(2,'a')"] // JSONStringsEachRow ``` -When inserting the data, you should provide a separate JSON object for each row. +When inserting the data, you should provide a separate JSON value for each row. ### Inserting Data {#inserting-data} From babd3beec09054d1dc4b1b8a35cf30da013f05af Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 1 Sep 2020 17:48:19 +0800 Subject: [PATCH 010/123] Fix nullable data parsing --- src/DataTypes/DataTypeNullable.cpp | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 847047850fd..3318196b951 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -308,7 +308,10 @@ ReturnType DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer const DataTypePtr & nested_data_type) { return safeDeserialize(column, *nested_data_type, - [&istr] { return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); }, + [&istr] + { + return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); + }, [&nested_data_type, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsTextQuoted(nested, istr, settings); }); } @@ -316,7 +319,11 @@ ReturnType DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer void DataTypeNullable::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { safeDeserialize(column, *nested_data_type, - [&istr] { return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); }, + [&istr] + { + return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr) + || checkStringByFirstCharacterAndAssertTheRest("ᴺᵁᴸᴸ", istr); + }, [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsWholeText(nested, istr, settings); }); } From ac5877e601714450a369062abbf80f84485bc6f5 Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 2 Sep 2020 00:58:39 +0800 Subject: [PATCH 011/123] Fix tests --- tests/queries/0_stateless/01447_JSONStrings.reference | 9 +-------- tests/queries/0_stateless/01447_JSONStrings.sql | 2 ++ 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01447_JSONStrings.reference b/tests/queries/0_stateless/01447_JSONStrings.reference index 58af593dc77..1c6f073c0d0 100644 --- a/tests/queries/0_stateless/01447_JSONStrings.reference +++ b/tests/queries/0_stateless/01447_JSONStrings.reference @@ -32,12 +32,5 @@ ["1", "a", "[1,2,3]", "(1,'a')", "ᴺᵁᴸᴸ", "nan"] ], - "rows": 1, - - "statistics": - { - "elapsed": 0.00068988, - "rows_read": 1, - "bytes_read": 1 - } + "rows": 1 } diff --git a/tests/queries/0_stateless/01447_JSONStrings.sql b/tests/queries/0_stateless/01447_JSONStrings.sql index 7d89f0f5087..45fc4a56d7a 100644 --- a/tests/queries/0_stateless/01447_JSONStrings.sql +++ b/tests/queries/0_stateless/01447_JSONStrings.sql @@ -1,3 +1,5 @@ +SET output_format_write_statistics = 0; + SELECT 1, 'a', From a80c1adee81631f770f642ad4430a8ff44ff46af Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 2 Sep 2020 12:05:02 +0800 Subject: [PATCH 012/123] Add JSONCompactStrings formats --- docs/en/interfaces/formats.md | 288 +++++++++++------- src/Formats/FormatFactory.cpp | 6 - .../Impl/JSONCompactEachRowRowInputFormat.cpp | 49 ++- .../Impl/JSONCompactEachRowRowInputFormat.h | 11 +- .../JSONCompactEachRowRowOutputFormat.cpp | 37 ++- .../Impl/JSONCompactEachRowRowOutputFormat.h | 9 +- .../Impl/JSONCompactRowOutputFormat.cpp | 30 +- .../Formats/Impl/JSONCompactRowOutputFormat.h | 10 +- .../Impl/JSONEachRowRowInputFormat.cpp | 42 ++- .../Formats/Impl/JSONEachRowRowInputFormat.h | 9 +- .../Impl/JSONEachRowRowOutputFormat.cpp | 32 +- .../Formats/Impl/JSONEachRowRowOutputFormat.h | 10 +- ...JSONEachRowWithProgressRowOutputFormat.cpp | 11 +- .../Formats/Impl/JSONRowOutputFormat.cpp | 44 ++- .../Formats/Impl/JSONRowOutputFormat.h | 9 +- .../Impl/JSONStringsEachRowRowInputFormat.cpp | 245 --------------- .../Impl/JSONStringsEachRowRowInputFormat.h | 54 ---- .../JSONStringsEachRowRowOutputFormat.cpp | 117 ------- .../Impl/JSONStringsEachRowRowOutputFormat.h | 45 --- .../Impl/JSONStringsRowOutputFormat.cpp | 93 ------ .../Formats/Impl/JSONStringsRowOutputFormat.h | 43 --- .../0_stateless/01446_JSONStringsEachRow.sql | 63 ---- .../01446_json_strings_each_row.reference | 22 ++ .../01446_json_strings_each_row.sql | 38 +++ .../0_stateless/01447_json_strings.reference | 43 +++ ...JSONStrings.sql => 01447_json_strings.sql} | 0 ...8_json_compact_strings_each_row.reference} | 0 .../01448_json_compact_strings_each_row.sql | 63 ++++ ...e => 01449_json_compact_strings.reference} | 0 .../01449_json_compact_strings.sql | 10 + 30 files changed, 621 insertions(+), 812 deletions(-) delete mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp delete mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h delete mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp delete mode 100644 src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h delete mode 100644 src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp delete mode 100644 src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h delete mode 100644 tests/queries/0_stateless/01446_JSONStringsEachRow.sql create mode 100644 tests/queries/0_stateless/01446_json_strings_each_row.reference create mode 100644 tests/queries/0_stateless/01446_json_strings_each_row.sql create mode 100644 tests/queries/0_stateless/01447_json_strings.reference rename tests/queries/0_stateless/{01447_JSONStrings.sql => 01447_json_strings.sql} (100%) rename tests/queries/0_stateless/{01446_JSONStringsEachRow.reference => 01448_json_compact_strings_each_row.reference} (100%) create mode 100644 tests/queries/0_stateless/01448_json_compact_strings_each_row.sql rename tests/queries/0_stateless/{01447_JSONStrings.reference => 01449_json_compact_strings.reference} (100%) create mode 100644 tests/queries/0_stateless/01449_json_compact_strings.sql diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 9c7c2dda8dc..bfe5b6218e4 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -10,45 +10,51 @@ results of a `SELECT`, and to perform `INSERT`s into a file-backed table. The supported formats are: -| Format | Input | Output | -|-----------------------------------------------------------------|-------|--------| -| [TabSeparated](#tabseparated) | ✔ | ✔ | -| [TabSeparatedRaw](#tabseparatedraw) | ✔ | ✔ | -| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | -| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | -| [Template](#format-template) | ✔ | ✔ | -| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | -| [CSV](#csv) | ✔ | ✔ | -| [CSVWithNames](#csvwithnames) | ✔ | ✔ | -| [CustomSeparated](#format-customseparated) | ✔ | ✔ | -| [Values](#data-format-values) | ✔ | ✔ | -| [Vertical](#vertical) | ✗ | ✔ | -| [VerticalRaw](#verticalraw) | ✗ | ✔ | -| [JSON](#json) | ✗ | ✔ | -| [JSONCompact](#jsoncompact) | ✗ | ✔ | -| [JSONStrings](#jsonstrings) | ✗ | ✔ | -| [JSONEachRow](#jsoneachrow) | ✔ | ✔ | -| [JSONCompactEachRow](#jsoncompacteachrow) | ✔ | ✔ | -| [JSONStringsEachRow](#jsonstringseachrow) | ✔ | ✔ | -| [TSKV](#tskv) | ✔ | ✔ | -| [Pretty](#pretty) | ✗ | ✔ | -| [PrettyCompact](#prettycompact) | ✗ | ✔ | -| [PrettyCompactMonoBlock](#prettycompactmonoblock) | ✗ | ✔ | -| [PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ | -| [PrettySpace](#prettyspace) | ✗ | ✔ | -| [Protobuf](#protobuf) | ✔ | ✔ | -| [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) | ✔ | ✔ | -| [Native](#native) | ✔ | ✔ | -| [Null](#null) | ✗ | ✔ | -| [XML](#xml) | ✗ | ✔ | -| [CapnProto](#capnproto) | ✔ | ✗ | +| Format | Input | Output | +|-----------------------------------------------------------------------------------------|-------|--------| +| [TabSeparated](#tabseparated) | ✔ | ✔ | +| [TabSeparatedRaw](#tabseparatedraw) | ✔ | ✔ | +| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | +| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | +| [Template](#format-template) | ✔ | ✔ | +| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | +| [CSV](#csv) | ✔ | ✔ | +| [CSVWithNames](#csvwithnames) | ✔ | ✔ | +| [CustomSeparated](#format-customseparated) | ✔ | ✔ | +| [Values](#data-format-values) | ✔ | ✔ | +| [Vertical](#vertical) | ✗ | ✔ | +| [VerticalRaw](#verticalraw) | ✗ | ✔ | +| [JSON](#json) | ✗ | ✔ | +| [JSONString](#jsonstring) | ✗ | ✔ | +| [JSONCompact](#jsoncompact) | ✗ | ✔ | +| [JSONCompactString](#jsoncompactstring) | ✗ | ✔ | +| [JSONEachRow](#jsoneachrow) | ✔ | ✔ | +| [JSONEachRowWithProgress](#jsoneachrowwithprogress) | ✗ | ✔ | +| [JSONStringEachRow](#jsonstringeachrow) | ✔ | ✔ | +| [JSONStringEachRowWithProgress](#jsonstringeachrowwithprogress) | ✗ | ✔ | +| [JSONCompactEachRow](#jsoncompacteachrow) | ✔ | ✔ | +| [JSONCompactEachRowWithNamesAndTypes](#jsoncompacteachrowwithnamesandtypes) | ✔ | ✔ | +| [JSONCompactStringEachRow](#jsoncompactstringeachrow) | ✔ | ✔ | +| [JSONCompactStringEachRowWithNamesAndTypes](#jsoncompactstringeachrowwithnamesandtypes) | ✔ | ✔ | +| [TSKV](#tskv) | ✔ | ✔ | +| [Pretty](#pretty) | ✗ | ✔ | +| [PrettyCompact](#prettycompact) | ✗ | ✔ | +| [PrettyCompactMonoBlock](#prettycompactmonoblock) | ✗ | ✔ | +| [PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ | +| [PrettySpace](#prettyspace) | ✗ | ✔ | +| [Protobuf](#protobuf) | ✔ | ✔ | +| [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) | ✔ | ✔ | +| [Native](#native) | ✔ | ✔ | +| [Null](#null) | ✗ | ✔ | +| [XML](#xml) | ✗ | ✔ | +| [CapnProto](#capnproto) | ✔ | ✗ | You can control some format processing parameters with the ClickHouse settings. For more information read the [Settings](../operations/settings/settings.md) section. @@ -395,62 +401,41 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA "meta": [ { - "name": "SearchPhrase", + "name": "'hello'", "type": "String" }, { - "name": "c", + "name": "multiply(42, number)", "type": "UInt64" + }, + { + "name": "range(5)", + "type": "Array(UInt8)" } ], "data": [ { - "SearchPhrase": "", - "c": "8267016" + "'hello'": "hello", + "multiply(42, number)": "0", + "range(5)": [0,1,2,3,4] }, { - "SearchPhrase": "bathroom interior design", - "c": "2166" + "'hello'": "hello", + "multiply(42, number)": "42", + "range(5)": [0,1,2,3,4] }, { - "SearchPhrase": "yandex", - "c": "1655" - }, - { - "SearchPhrase": "spring 2014 fashion", - "c": "1549" - }, - { - "SearchPhrase": "freeform photos", - "c": "1480" + "'hello'": "hello", + "multiply(42, number)": "84", + "range(5)": [0,1,2,3,4] } ], - "totals": - { - "SearchPhrase": "", - "c": "8873898" - }, + "rows": 3, - "extremes": - { - "min": - { - "SearchPhrase": "", - "c": "1480" - }, - "max": - { - "SearchPhrase": "", - "c": "8267016" - } - }, - - "rows": 5, - - "rows_before_limit_at_least": 141137 + "rows_before_limit_at_least": 3 } ``` @@ -471,73 +456,166 @@ ClickHouse supports [NULL](../sql-reference/syntax.md), which is displayed as `n See also the [JSONEachRow](#jsoneachrow) format. -## JSONCompact {#jsoncompact} +## JSONString {#jsonstring} -Differs from JSON only in that data rows are output in arrays of any element type, not in objects. +Differs from JSON only in that data fields are output in strings, not in typed json values. Example: -``` json +```json { "meta": [ { - "name": "SearchPhrase", + "name": "'hello'", "type": "String" }, { - "name": "c", + "name": "multiply(42, number)", "type": "UInt64" + }, + { + "name": "range(5)", + "type": "Array(UInt8)" } ], "data": [ - ["", "8267016"], - ["bathroom interior design", "2166"], - ["yandex", "1655"], - ["fashion trends spring 2014", "1549"], - ["freeform photo", "1480"] + { + "'hello'": "hello", + "multiply(42, number)": "0", + "range(5)": "[0,1,2,3,4]" + }, + { + "'hello'": "hello", + "multiply(42, number)": "42", + "range(5)": "[0,1,2,3,4]" + }, + { + "'hello'": "hello", + "multiply(42, number)": "84", + "range(5)": "[0,1,2,3,4]" + } ], - "totals": ["","8873898"], + "rows": 3, - "extremes": - { - "min": ["","1480"], - "max": ["","8267016"] - }, - - "rows": 5, - - "rows_before_limit_at_least": 141137 + "rows_before_limit_at_least": 3 } ``` -This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). -See also the `JSONEachRow` format. +## JSONCompact {#jsoncompact} +## JSONCompactString {#jsoncompactstring} -## JSONStrings {#jsonstrings} +Differs from JSON only in that data rows are output in arrays, not in objects. -Differs from JSON and JSONCompact only in that data rows are output in arrays of strings. +Example: -This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). -See also the `JSONEachRow` format. +``` json +// JSONCompact +{ + "meta": + [ + { + "name": "'hello'", + "type": "String" + }, + { + "name": "multiply(42, number)", + "type": "UInt64" + }, + { + "name": "range(5)", + "type": "Array(UInt8)" + } + ], + + "data": + [ + ["hello", "0", [0,1,2,3,4]], + ["hello", "42", [0,1,2,3,4]], + ["hello", "84", [0,1,2,3,4]] + ], + + "rows": 3, + + "rows_before_limit_at_least": 3 +} +``` + +```json +// JSONCompactString +{ + "meta": + [ + { + "name": "'hello'", + "type": "String" + }, + { + "name": "multiply(42, number)", + "type": "UInt64" + }, + { + "name": "range(5)", + "type": "Array(UInt8)" + } + ], + + "data": + [ + ["hello", "0", "[0,1,2,3,4]"], + ["hello", "42", "[0,1,2,3,4]"], + ["hello", "84", "[0,1,2,3,4]"] + ], + + "rows": 3, + + "rows_before_limit_at_least": 3 +} +``` ## JSONEachRow {#jsoneachrow} +## JSONStringEachRow {#jsonstringeachrow} ## JSONCompactEachRow {#jsoncompacteachrow} -## JSONStringsEachRow {#jsonstringseachrow} +## JSONCompactStringEachRow {#jsoncompactstringeachrow} When using these formats, ClickHouse outputs rows as separated, newline-delimited JSON values, but the data as a whole is not valid JSON. ``` json {"some_int":42,"some_str":"hello","some_tuple":[1,"a"]} // JSONEachRow [42,"hello",[1,"a"]] // JSONCompactEachRow -["42","hello","(2,'a')"] // JSONStringsEachRow +["42","hello","(2,'a')"] // JSONCompactStringsEachRow ``` When inserting the data, you should provide a separate JSON value for each row. +## JSONEachRowWithProgress {#jsoneachrowwithprogress} +## JSONStringEachRowWithProgress {#jsonstringeachrowwithprogress} + +Differs from JSONEachRow/JSONStringEachRow in that ClickHouse will also yield progress information as JSON objects. + +```json +{"row":{"'hello'":"hello","multiply(42, number)":"0","range(5)":[0,1,2,3,4]}} +{"row":{"'hello'":"hello","multiply(42, number)":"42","range(5)":[0,1,2,3,4]}} +{"row":{"'hello'":"hello","multiply(42, number)":"84","range(5)":[0,1,2,3,4]}} +{"progress":{"read_rows":"3","read_bytes":"24","written_rows":"0","written_bytes":"0","total_rows_to_read":"3"}} +``` + +## JSONCompactEachRowWithNamesAndTypes {#jsoncompacteachrowwithnamesandtypes} +## JSONCompactStringEachRowWithNamesAndTypes {#jsoncompactstringeachrowwithnamesandtypes} + +Differs from JSONCompactEachRow/JSONCompactStringEachRow in that the column names and types are written as the first two rows. + +```json +["'hello'", "multiply(42, number)", "range(5)"] +["String", "UInt64", "Array(UInt8)"] +["hello", "0", [0,1,2,3,4]] +["hello", "42", [0,1,2,3,4]] +["hello", "84", [0,1,2,3,4]] +``` + ### Inserting Data {#inserting-data} ``` sql diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index cb378fbea96..871098e00c0 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -352,8 +352,6 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); -void registerInputFormatProcessorJSONStringsEachRow(FormatFactory & factory); -void registerOutputFormatProcessorJSONStringsEachRow(FormatFactory & factory); void registerInputFormatProcessorProtobuf(FormatFactory & factory); void registerOutputFormatProcessorProtobuf(FormatFactory & factory); void registerInputFormatProcessorTemplate(FormatFactory & factory); @@ -380,7 +378,6 @@ void registerOutputFormatProcessorVertical(FormatFactory & factory); void registerOutputFormatProcessorJSON(FormatFactory & factory); void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); -void registerOutputFormatProcessorJSONStrings(FormatFactory & factory); void registerOutputFormatProcessorXML(FormatFactory & factory); void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); void registerOutputFormatProcessorNull(FormatFactory & factory); @@ -421,8 +418,6 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSONEachRow(*this); registerInputFormatProcessorJSONCompactEachRow(*this); registerOutputFormatProcessorJSONCompactEachRow(*this); - registerInputFormatProcessorJSONStringsEachRow(*this); - registerOutputFormatProcessorJSONStringsEachRow(*this); registerInputFormatProcessorProtobuf(*this); registerOutputFormatProcessorProtobuf(*this); registerInputFormatProcessorTemplate(*this); @@ -449,7 +444,6 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorJSON(*this); registerOutputFormatProcessorJSONCompact(*this); registerOutputFormatProcessorJSONEachRowWithProgress(*this); - registerOutputFormatProcessorJSONStrings(*this); registerOutputFormatProcessorXML(*this); registerOutputFormatProcessorODBCDriver2(*this); registerOutputFormatProcessorNull(*this); diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index 82e3cb795bf..eb697ce5318 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -19,8 +20,9 @@ JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat(ReadBuffer & const Block & header_, Params params_, const FormatSettings & format_settings_, - bool with_names_) - : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_) + bool with_names_, + bool yield_strings_) + : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_), yield_strings(yield_strings_) { const auto & sample = getPort().getHeader(); size_t num_columns = sample.columns(); @@ -200,10 +202,25 @@ void JSONCompactEachRowRowInputFormat::readField(size_t index, MutableColumns & { read_columns[index] = true; const auto & type = data_types[index]; - if (format_settings.null_as_default && !type->isNullable()) - read_columns[index] = DataTypeNullable::deserializeTextJSON(*columns[index], in, format_settings, type); + + if (yield_strings) + { + // notice: null_as_default on "null" strings is not supported + + String str; + readJSONString(str, in); + + ReadBufferFromString buf(str); + + type->deserializeAsWholeText(*columns[index], buf, format_settings); + } else - type->deserializeAsTextJSON(*columns[index], in, format_settings); + { + if (format_settings.null_as_default && !type->isNullable()) + read_columns[index] = DataTypeNullable::deserializeTextJSON(*columns[index], in, format_settings, type); + else + type->deserializeAsTextJSON(*columns[index], in, format_settings); + } } catch (Exception & e) { @@ -225,7 +242,7 @@ void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, std::move(params), settings, false); + return std::make_shared(buf, sample, std::move(params), settings, false, false); }); factory.registerInputFormatProcessor("JSONCompactEachRowWithNamesAndTypes", []( @@ -234,7 +251,25 @@ void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, std::move(params), settings, true); + return std::make_shared(buf, sample, std::move(params), settings, true, false); + }); + + factory.registerInputFormatProcessor("JSONCompactStringsEachRow", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, false, true); + }); + + factory.registerInputFormatProcessor("JSONCompactStringsEachRowWithNamesAndTypes", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, true, true); }); } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index 5c864ebc751..593f297108c 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -12,12 +12,18 @@ namespace DB class ReadBuffer; -/** A stream for reading data in JSONCompactEachRow and JSONCompactEachRowWithNamesAndTypes formats +/** A stream for reading data in JSONCompactEachRow- formats */ class JSONCompactEachRowRowInputFormat : public IRowInputFormat { public: - JSONCompactEachRowRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_, bool with_names_); + JSONCompactEachRowRowInputFormat( + ReadBuffer & in_, + const Block & header_, + Params params_, + const FormatSettings & format_settings_, + bool with_names_, + bool yield_strings_); String getName() const override { return "JSONCompactEachRowRowInputFormat"; } @@ -49,6 +55,7 @@ private: std::vector names_of_columns; bool with_names; + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp index e155dcb4247..ab8fd164c3c 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -12,8 +12,9 @@ JSONCompactEachRowRowOutputFormat::JSONCompactEachRowRowOutputFormat(WriteBuffer const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_, - bool with_names_) - : IRowOutputFormat(header_, out_, callback), settings(settings_), with_names(with_names_) + bool with_names_, + bool yield_strings_) + : IRowOutputFormat(header_, out_, callback), settings(settings_), with_names(with_names_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); @@ -23,7 +24,15 @@ JSONCompactEachRowRowOutputFormat::JSONCompactEachRowRowOutputFormat(WriteBuffer void JSONCompactEachRowRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) { - type.serializeAsTextJSON(column, row_num, out, settings); + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), out, settings); + } + else + type.serializeAsTextJSON(column, row_num, out, settings); } @@ -97,7 +106,7 @@ void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings, false); + return std::make_shared(buf, sample, callback, format_settings, false, false); }); factory.registerOutputFormatProcessor("JSONCompactEachRowWithNamesAndTypes", []( @@ -106,7 +115,25 @@ void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings &format_settings) { - return std::make_shared(buf, sample, callback, format_settings, true); + return std::make_shared(buf, sample, callback, format_settings, true, false); + }); + + factory.registerOutputFormatProcessor("JSONCompactStringsEachRow", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, false, true); + }); + + factory.registerOutputFormatProcessor("JSONCompactStringsEachRowWithNamesAndTypes", []( + WriteBuffer &buf, + const Block &sample, + FormatFactory::WriteCallback callback, + const FormatSettings &format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true, true); }); } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h index a7857a82d2d..56936783e78 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h @@ -15,7 +15,13 @@ namespace DB class JSONCompactEachRowRowOutputFormat : public IRowOutputFormat { public: - JSONCompactEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_, bool with_names); + JSONCompactEachRowRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool with_names_, + bool yield_strings_); String getName() const override { return "JSONCompactEachRowRowOutputFormat"; } @@ -41,5 +47,6 @@ private: NamesAndTypes fields; bool with_names; + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp index 7e56a4643da..c36942cff09 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp @@ -8,15 +8,28 @@ namespace DB { JSONCompactRowOutputFormat::JSONCompactRowOutputFormat( - WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_) - : JSONRowOutputFormat(out_, header, callback, settings_) + WriteBuffer & out_, + const Block & header, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_) + : JSONRowOutputFormat(out_, header, callback, settings_, yield_strings_) { } void JSONCompactRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) { - type.serializeAsTextJSON(column, row_num, *ostr, settings); + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), *ostr, settings); + } + else + type.serializeAsTextJSON(column, row_num, *ostr, settings); + ++field_number; } @@ -83,7 +96,16 @@ void registerOutputFormatProcessorJSONCompact(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings); + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONCompactStrings", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h index f4002f74287..6585016c44f 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h @@ -11,12 +11,17 @@ namespace DB struct FormatSettings; -/** The stream for outputting data in the JSONCompact format. +/** The stream for outputting data in the JSONCompact- formats. */ class JSONCompactRowOutputFormat : public JSONRowOutputFormat { public: - JSONCompactRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_); + JSONCompactRowOutputFormat( + WriteBuffer & out_, + const Block & header, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_); String getName() const override { return "JSONCompactRowOutputFormat"; } @@ -37,7 +42,6 @@ protected: } void writeTotalsFieldDelimiter() override; - }; } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 6350db3b211..9ba82fbb009 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -29,8 +30,12 @@ enum JSONEachRowRowInputFormat::JSONEachRowRowInputFormat( - ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) - : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), name_map(header_.columns()) + ReadBuffer & in_, + const Block & header_, + Params params_, + const FormatSettings & format_settings_, + bool yield_strings_) + : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), name_map(header_.columns()), yield_strings(yield_strings_) { /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. skipBOMIfExists(in); @@ -138,10 +143,25 @@ void JSONEachRowRowInputFormat::readField(size_t index, MutableColumns & columns { seen_columns[index] = read_columns[index] = true; const auto & type = getPort().getHeader().getByPosition(index).type; - if (format_settings.null_as_default && !type->isNullable()) - read_columns[index] = DataTypeNullable::deserializeTextJSON(*columns[index], in, format_settings, type); + + if (yield_strings) + { + // notice: null_as_default on "null" strings is not supported + + String str; + readJSONString(str, in); + + ReadBufferFromString buf(str); + + type->deserializeAsWholeText(*columns[index], buf, format_settings); + } else - type->deserializeAsTextJSON(*columns[index], in, format_settings); + { + if (format_settings.null_as_default && !type->isNullable()) + read_columns[index] = DataTypeNullable::deserializeTextJSON(*columns[index], in, format_settings, type); + else + type->deserializeAsTextJSON(*columns[index], in, format_settings); + } } catch (Exception & e) { @@ -318,13 +338,23 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, std::move(params), settings); + return std::make_shared(buf, sample, std::move(params), settings, false); + }); + + factory.registerInputFormatProcessor("JSONStringsEachRow", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, true); }); } void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory) { factory.registerFileSegmentationEngine("JSONEachRow", &fileSegmentationEngineJSONEachRowImpl); + factory.registerFileSegmentationEngine("JSONStringsEachRow", &fileSegmentationEngineJSONEachRowImpl); } } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h index a0a4b735a3e..29a6ce6ecb8 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h @@ -20,7 +20,12 @@ class ReadBuffer; class JSONEachRowRowInputFormat : public IRowInputFormat { public: - JSONEachRowRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_); + JSONEachRowRowInputFormat( + ReadBuffer & in_, + const Block & header_, + Params params_, + const FormatSettings & format_settings_, + bool yield_strings_); String getName() const override { return "JSONEachRowRowInputFormat"; } @@ -75,6 +80,8 @@ private: bool data_in_square_brackets = false; bool allow_new_rows = true; + + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp index 910a9710de3..069499d99c1 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp @@ -8,8 +8,13 @@ namespace DB { -JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_) - : IRowOutputFormat(header_, out_, callback), settings(settings_) +JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_) + : IRowOutputFormat(header_, out_, callback), settings(settings_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -27,7 +32,17 @@ void JSONEachRowRowOutputFormat::writeField(const IColumn & column, const IDataT { writeString(fields[field_number], out); writeChar(':', out); - type.serializeAsTextJSON(column, row_num, out, settings); + + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), out, settings); + } + else + type.serializeAsTextJSON(column, row_num, out, settings); + ++field_number; } @@ -59,7 +74,16 @@ void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings); + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONStringsEachRow", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h index d2b6937cd01..5346a1ab19f 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h @@ -15,7 +15,12 @@ namespace DB class JSONEachRowRowOutputFormat : public IRowOutputFormat { public: - JSONEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_); + JSONEachRowRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_); String getName() const override { return "JSONEachRowRowOutputFormat"; } @@ -35,6 +40,9 @@ private: Names fields; FormatSettings settings; + +protected: + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp index a611b5a129b..35720df9672 100644 --- a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp @@ -36,7 +36,16 @@ void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factor FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings); + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONStringsEachRowWithProgress", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index b3255f2894e..7dd7eb9953a 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -7,8 +7,13 @@ namespace DB { -JSONRowOutputFormat::JSONRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_) - : IRowOutputFormat(header, out_, callback), settings(settings_) +JSONRowOutputFormat::JSONRowOutputFormat( + WriteBuffer & out_, + const Block & header, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_) + : IRowOutputFormat(header, out_, callback), settings(settings_), yield_strings(yield_strings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); @@ -71,7 +76,17 @@ void JSONRowOutputFormat::writeField(const IColumn & column, const IDataType & t writeCString("\t\t\t", *ostr); writeString(fields[field_number].name, *ostr); writeCString(": ", *ostr); - type.serializeAsTextJSON(column, row_num, *ostr, settings); + + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), *ostr, settings); + } + else + type.serializeAsTextJSON(column, row_num, *ostr, settings); + ++field_number; } @@ -80,7 +95,17 @@ void JSONRowOutputFormat::writeTotalsField(const IColumn & column, const IDataTy writeCString("\t\t", *ostr); writeString(fields[field_number].name, *ostr); writeCString(": ", *ostr); - type.serializeAsTextJSON(column, row_num, *ostr, settings); + + if (yield_strings) + { + WriteBufferFromOwnString buf; + + type.serializeAsText(column, row_num, buf, settings); + writeJSONString(buf.str(), *ostr, settings); + } + else + type.serializeAsTextJSON(column, row_num, *ostr, settings); + ++field_number; } @@ -249,7 +274,16 @@ void registerOutputFormatProcessorJSON(FormatFactory & factory) FormatFactory::WriteCallback callback, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, callback, format_settings); + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONStrings", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); }); } diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.h b/src/Processors/Formats/Impl/JSONRowOutputFormat.h index f9aea3a3e8b..4e9cceb717e 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.h @@ -16,7 +16,12 @@ namespace DB class JSONRowOutputFormat : public IRowOutputFormat { public: - JSONRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_); + JSONRowOutputFormat( + WriteBuffer & out_, + const Block & header, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool yield_strings_); String getName() const override { return "JSONRowOutputFormat"; } @@ -78,6 +83,8 @@ protected: Progress progress; Stopwatch watch; FormatSettings settings; + + bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp deleted file mode 100644 index fff44a204fb..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.cpp +++ /dev/null @@ -1,245 +0,0 @@ -#include -#include - -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INCORRECT_DATA; - extern const int CANNOT_READ_ALL_DATA; -} - - -JSONStringsEachRowRowInputFormat::JSONStringsEachRowRowInputFormat(ReadBuffer & in_, - const Block & header_, - Params params_, - const FormatSettings & format_settings_, - bool with_names_) - : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_) -{ - const auto & sample = getPort().getHeader(); - size_t num_columns = sample.columns(); - - data_types.resize(num_columns); - column_indexes_by_names.reserve(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - const auto & column_info = sample.getByPosition(i); - - data_types[i] = column_info.type; - column_indexes_by_names.emplace(column_info.name, i); - } -} - -void JSONStringsEachRowRowInputFormat::resetParser() -{ - IRowInputFormat::resetParser(); - column_indexes_for_input_fields.clear(); - not_seen_columns.clear(); -} - -void JSONStringsEachRowRowInputFormat::readPrefix() -{ - /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. - skipBOMIfExists(in); - - if (with_names) - { - size_t num_columns = getPort().getHeader().columns(); - read_columns.assign(num_columns, false); - - assertChar('[', in); - do - { - skipWhitespaceIfAny(in); - String column_name; - readJSONString(column_name, in); - addInputColumn(column_name); - skipWhitespaceIfAny(in); - } - while (checkChar(',', in)); - assertChar(']', in); - skipEndOfLine(); - - /// Type checking - assertChar('[', in); - for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i) - { - skipWhitespaceIfAny(in); - String data_type; - readJSONString(data_type, in); - - if (column_indexes_for_input_fields[i] && - data_types[*column_indexes_for_input_fields[i]]->getName() != data_type) - { - throw Exception( - "Type of '" + getPort().getHeader().getByPosition(*column_indexes_for_input_fields[i]).name - + "' must be " + data_types[*column_indexes_for_input_fields[i]]->getName() + - ", not " + data_type, - ErrorCodes::INCORRECT_DATA - ); - } - - if (i != column_indexes_for_input_fields.size() - 1) - assertChar(',', in); - skipWhitespaceIfAny(in); - } - assertChar(']', in); - } - else - { - size_t num_columns = getPort().getHeader().columns(); - read_columns.assign(num_columns, true); - column_indexes_for_input_fields.resize(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - column_indexes_for_input_fields[i] = i; - } - } - - for (size_t i = 0; i < read_columns.size(); ++i) - { - if (!read_columns[i]) - { - not_seen_columns.emplace_back(i); - } - } -} - -void JSONStringsEachRowRowInputFormat::addInputColumn(const String & column_name) -{ - names_of_columns.emplace_back(column_name); - - const auto column_it = column_indexes_by_names.find(column_name); - if (column_it == column_indexes_by_names.end()) - { - if (format_settings.skip_unknown_fields) - { - column_indexes_for_input_fields.push_back(std::nullopt); - return; - } - - throw Exception( - "Unknown field found in JSONStringsEachRow header: '" + column_name + "' " + - "at position " + std::to_string(column_indexes_for_input_fields.size()) + - "\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed", - ErrorCodes::INCORRECT_DATA - ); - } - - const auto column_index = column_it->second; - - if (read_columns[column_index]) - throw Exception("Duplicate field found while parsing JSONStringsEachRow header: " + column_name, ErrorCodes::INCORRECT_DATA); - - read_columns[column_index] = true; - column_indexes_for_input_fields.emplace_back(column_index); -} - -bool JSONStringsEachRowRowInputFormat::readRow(DB::MutableColumns &columns, DB::RowReadExtension &ext) -{ - skipEndOfLine(); - - if (in.eof()) - return false; - - size_t num_columns = columns.size(); - - read_columns.assign(num_columns, false); - - assertChar('[', in); - for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column) - { - const auto & table_column = column_indexes_for_input_fields[file_column]; - if (table_column) - { - readField(*table_column, columns); - } - else - { - skipJSONField(in, StringRef(names_of_columns[file_column])); - } - - skipWhitespaceIfAny(in); - if (in.eof()) - throw Exception("Unexpected end of stream while parsing JSONStringsEachRow format", ErrorCodes::CANNOT_READ_ALL_DATA); - if (file_column + 1 != column_indexes_for_input_fields.size()) - { - assertChar(',', in); - skipWhitespaceIfAny(in); - } - } - assertChar(']', in); - - for (const auto & name : not_seen_columns) - columns[name]->insertDefault(); - - ext.read_columns = read_columns; - return true; -} - -void JSONStringsEachRowRowInputFormat::skipEndOfLine() -{ - skipWhitespaceIfAny(in); - if (!in.eof() && (*in.position() == ',' || *in.position() == ';')) - ++in.position(); - - skipWhitespaceIfAny(in); -} - -void JSONStringsEachRowRowInputFormat::readField(size_t index, MutableColumns & columns) -{ - try - { - read_columns[index] = true; - const auto & type = data_types[index]; - - String str; - readJSONString(str, in); - - ReadBufferFromString buf(str); - - type->deserializeAsWholeText(*columns[index], buf, format_settings); - } - catch (Exception & e) - { - e.addMessage("(while read the value of key " + getPort().getHeader().getByPosition(index).name + ")"); - throw; - } -} - -void JSONStringsEachRowRowInputFormat::syncAfterError() -{ - skipToUnescapedNextLineOrEOF(in); -} - -void registerInputFormatProcessorJSONStringsEachRow(FormatFactory & factory) -{ - factory.registerInputFormatProcessor("JSONStringsEachRow", []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, std::move(params), settings, false); - }); - - factory.registerInputFormatProcessor("JSONStringsEachRowWithNamesAndTypes", []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, std::move(params), settings, true); - }); -} - -} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h deleted file mode 100644 index ec0a0f7bad9..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsEachRowRowInputFormat.h +++ /dev/null @@ -1,54 +0,0 @@ -#pragma once - -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -class ReadBuffer; - -/** A stream for reading data in JSONStringsEachRow and JSONStringsEachRowWithNamesAndTypes formats -*/ -class JSONStringsEachRowRowInputFormat : public IRowInputFormat -{ -public: - JSONStringsEachRowRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_, bool with_names_); - - String getName() const override { return "JSONStringsEachRowRowInputFormat"; } - - - void readPrefix() override; - bool readRow(MutableColumns & columns, RowReadExtension & ext) override; - bool allowSyncAfterError() const override { return true; } - void syncAfterError() override; - void resetParser() override; - -private: - void addInputColumn(const String & column_name); - void skipEndOfLine(); - void readField(size_t index, MutableColumns & columns); - - const FormatSettings format_settings; - - using IndexesMap = std::unordered_map; - IndexesMap column_indexes_by_names; - - using OptionalIndexes = std::vector>; - OptionalIndexes column_indexes_for_input_fields; - - DataTypes data_types; - std::vector read_columns; - std::vector not_seen_columns; - - /// This is for the correct exceptions in skipping unknown fields. - std::vector names_of_columns; - - bool with_names; -}; - -} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp deleted file mode 100644 index 75007ea236e..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp +++ /dev/null @@ -1,117 +0,0 @@ -#include -#include -#include -#include - - -namespace DB -{ - - -JSONStringsEachRowRowOutputFormat::JSONStringsEachRowRowOutputFormat(WriteBuffer & out_, - const Block & header_, - FormatFactory::WriteCallback callback, - const FormatSettings & settings_, - bool with_names_) - : IRowOutputFormat(header_, out_, callback), settings(settings_), with_names(with_names_) -{ - const auto & sample = getPort(PortKind::Main).getHeader(); - NamesAndTypesList columns(sample.getNamesAndTypesList()); - fields.assign(columns.begin(), columns.end()); -} - - -void JSONStringsEachRowRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) -{ - WriteBufferFromOwnString buf; - - type.serializeAsText(column, row_num, buf, settings); - writeJSONString(buf.str(), out, settings); -} - - -void JSONStringsEachRowRowOutputFormat::writeFieldDelimiter() -{ - writeCString(", ", out); -} - - -void JSONStringsEachRowRowOutputFormat::writeRowStartDelimiter() -{ - writeChar('[', out); -} - - -void JSONStringsEachRowRowOutputFormat::writeRowEndDelimiter() -{ - writeCString("]\n", out); -} - -void JSONStringsEachRowRowOutputFormat::writeTotals(const Columns & columns, size_t row_num) -{ - writeChar('\n', out); - size_t num_columns = columns.size(); - writeChar('[', out); - for (size_t i = 0; i < num_columns; ++i) - { - if (i != 0) - JSONStringsEachRowRowOutputFormat::writeFieldDelimiter(); - - JSONStringsEachRowRowOutputFormat::writeField(*columns[i], *types[i], row_num); - } - writeCString("]\n", out); -} - -void JSONStringsEachRowRowOutputFormat::writePrefix() -{ - if (with_names) - { - writeChar('[', out); - for (size_t i = 0; i < fields.size(); ++i) - { - writeChar('\"', out); - writeString(fields[i].name, out); - writeChar('\"', out); - if (i != fields.size() - 1) - writeCString(", ", out); - } - writeCString("]\n[", out); - for (size_t i = 0; i < fields.size(); ++i) - { - writeJSONString(fields[i].type->getName(), out, settings); - if (i != fields.size() - 1) - writeCString(", ", out); - } - writeCString("]\n", out); - } -} - -void JSONStringsEachRowRowOutputFormat::consumeTotals(DB::Chunk chunk) -{ - if (with_names) - IRowOutputFormat::consumeTotals(std::move(chunk)); -} - -void registerOutputFormatProcessorJSONStringsEachRow(FormatFactory & factory) -{ - factory.registerOutputFormatProcessor("JSONStringsEachRow", []( - WriteBuffer & buf, - const Block & sample, - FormatFactory::WriteCallback callback, - const FormatSettings & format_settings) - { - return std::make_shared(buf, sample, callback, format_settings, false); - }); - - factory.registerOutputFormatProcessor("JSONStringsEachRowWithNamesAndTypes", []( - WriteBuffer &buf, - const Block &sample, - FormatFactory::WriteCallback callback, - const FormatSettings &format_settings) - { - return std::make_shared(buf, sample, callback, format_settings, true); - }); -} - - -} diff --git a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h deleted file mode 100644 index 1d43a333da1..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsEachRowRowOutputFormat.h +++ /dev/null @@ -1,45 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -/** The stream for outputting data in JSON format, by object per line. - * Does not validate UTF-8. - */ -class JSONStringsEachRowRowOutputFormat : public IRowOutputFormat -{ -public: - JSONStringsEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_, bool with_names); - - String getName() const override { return "JSONStringsEachRowRowOutputFormat"; } - - void writePrefix() override; - - void writeBeforeTotals() override {} - void writeTotals(const Columns & columns, size_t row_num) override; - void writeAfterTotals() override {} - - void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; - void writeFieldDelimiter() override; - void writeRowStartDelimiter() override; - void writeRowEndDelimiter() override; - -protected: - void consumeTotals(Chunk) override; - /// No extremes. - void consumeExtremes(Chunk) override {} - -private: - FormatSettings settings; - - NamesAndTypes fields; - - bool with_names; -}; -} diff --git a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp deleted file mode 100644 index 6ccb315f73f..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.cpp +++ /dev/null @@ -1,93 +0,0 @@ -#include -#include - -#include - - -namespace DB -{ - -JSONStringsRowOutputFormat::JSONStringsRowOutputFormat( - WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_) - : JSONRowOutputFormat(out_, header, callback, settings_) -{ -} - - -void JSONStringsRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) -{ - WriteBufferFromOwnString buf; - - type.serializeAsText(column, row_num, buf, settings); - writeJSONString(buf.str(), *ostr, settings); - ++field_number; -} - - -void JSONStringsRowOutputFormat::writeFieldDelimiter() -{ - writeCString(", ", *ostr); -} - -void JSONStringsRowOutputFormat::writeTotalsFieldDelimiter() -{ - writeCString(",", *ostr); -} - - -void JSONStringsRowOutputFormat::writeRowStartDelimiter() -{ - writeCString("\t\t[", *ostr); -} - - -void JSONStringsRowOutputFormat::writeRowEndDelimiter() -{ - writeChar(']', *ostr); - field_number = 0; - ++row_count; -} - -void JSONStringsRowOutputFormat::writeBeforeTotals() -{ - writeCString(",\n", *ostr); - writeChar('\n', *ostr); - writeCString("\t\"totals\": [", *ostr); -} - -void JSONStringsRowOutputFormat::writeAfterTotals() -{ - writeChar(']', *ostr); -} - -void JSONStringsRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num) -{ - writeCString("\t\t\"", *ostr); - writeCString(title, *ostr); - writeCString("\": [", *ostr); - - size_t extremes_columns = columns.size(); - for (size_t i = 0; i < extremes_columns; ++i) - { - if (i != 0) - writeTotalsFieldDelimiter(); - - writeField(*columns[i], *types[i], row_num); - } - - writeChar(']', *ostr); -} - -void registerOutputFormatProcessorJSONStrings(FormatFactory & factory) -{ - factory.registerOutputFormatProcessor("JSONStrings", []( - WriteBuffer & buf, - const Block & sample, - FormatFactory::WriteCallback callback, - const FormatSettings & format_settings) - { - return std::make_shared(buf, sample, callback, format_settings); - }); -} - -} diff --git a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h b/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h deleted file mode 100644 index b221bc9ee36..00000000000 --- a/src/Processors/Formats/Impl/JSONStringsRowOutputFormat.h +++ /dev/null @@ -1,43 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -struct FormatSettings; - -/** The stream for outputting data in the JSONStrings format. - */ -class JSONStringsRowOutputFormat : public JSONRowOutputFormat -{ -public: - JSONStringsRowOutputFormat(WriteBuffer & out_, const Block & header, FormatFactory::WriteCallback callback, const FormatSettings & settings_); - - String getName() const override { return "JSONStringsRowOutputFormat"; } - - void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; - void writeFieldDelimiter() override; - void writeRowStartDelimiter() override; - void writeRowEndDelimiter() override; - - void writeBeforeTotals() override; - void writeAfterTotals() override; - -protected: - void writeExtremesElement(const char * title, const Columns & columns, size_t row_num) override; - - void writeTotalsField(const IColumn & column, const IDataType & type, size_t row_num) override - { - return writeField(column, type, row_num); - } - - void writeTotalsFieldDelimiter() override; - -}; - -} diff --git a/tests/queries/0_stateless/01446_JSONStringsEachRow.sql b/tests/queries/0_stateless/01446_JSONStringsEachRow.sql deleted file mode 100644 index f461b217fe4..00000000000 --- a/tests/queries/0_stateless/01446_JSONStringsEachRow.sql +++ /dev/null @@ -1,63 +0,0 @@ -DROP TABLE IF EXISTS test_table; -DROP TABLE IF EXISTS test_table_2; -SELECT 1; -/* Check JSONStringsEachRow Output */ -CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value; -INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c'); -SELECT * FROM test_table FORMAT JSONStringsEachRow; -SELECT 2; -/* Check Totals */ -SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRow; -SELECT 3; -/* Check JSONStringsEachRowWithNamesAndTypes Output */ -SELECT * FROM test_table FORMAT JSONStringsEachRowWithNamesAndTypes; -SELECT 4; -/* Check Totals */ -SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRowWithNamesAndTypes; -DROP TABLE IF EXISTS test_table; -SELECT 5; -/* Check JSONStringsEachRow Input */ -CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; -INSERT INTO test_table FORMAT JSONStringsEachRow ["first", "1", "2", "NULL"] ["second", "2", "null", "6"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -TRUNCATE TABLE test_table; -SELECT 6; -/* Check input_format_null_as_default = 1 */ -SET input_format_null_as_default = 1; -INSERT INTO test_table FORMAT JSONStringsEachRow ["first", "1", "2", "ᴺᵁᴸᴸ"] ["second", "2", "null", "6"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -TRUNCATE TABLE test_table; -SELECT 7; -/* Check Nested */ -CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; -INSERT INTO test_table_2 FORMAT JSONStringsEachRow ["16", "[15, 16, 17]", "['first', 'second', 'third']"]; -SELECT * FROM test_table_2 FORMAT JSONStringsEachRow; -TRUNCATE TABLE test_table_2; -SELECT 8; -/* Check JSONStringsEachRowWithNamesAndTypes Output */ -SET input_format_null_as_default = 0; -INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"]["second", "2", "null", "6"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -TRUNCATE TABLE test_table; -SELECT 9; -/* Check input_format_null_as_default = 1 */ -SET input_format_null_as_default = 1; -INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"] ["second", "2", "null", "6"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -SELECT 10; -/* Check Header */ -TRUNCATE TABLE test_table; -SET input_format_skip_unknown_fields = 1; -INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", "1", "32"]["second", "2", "64"]; -SELECT * FROM test_table FORMAT JSONStringsEachRow; -SELECT 11; -TRUNCATE TABLE test_table; -INSERT INTO test_table FORMAT JSONStringsEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"]["1", "2", "3"] -SELECT * FROM test_table FORMAT JSONStringsEachRowWithNamesAndTypes; -SELECT 12; -/* Check Nested */ -INSERT INTO test_table_2 FORMAT JSONStringsEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"]["16", "[15, 16, 17]", "['first', 'second', 'third']"]; -SELECT * FROM test_table_2 FORMAT JSONStringsEachRowWithNamesAndTypes; - -DROP TABLE IF EXISTS test_table; -DROP TABLE IF EXISTS test_table_2; diff --git a/tests/queries/0_stateless/01446_json_strings_each_row.reference b/tests/queries/0_stateless/01446_json_strings_each_row.reference new file mode 100644 index 00000000000..84d41095b77 --- /dev/null +++ b/tests/queries/0_stateless/01446_json_strings_each_row.reference @@ -0,0 +1,22 @@ +1 +{"value":"1","name":"a"} +{"value":"2","name":"b"} +{"value":"3","name":"c"} +2 +{"name":"a","c":"1"} +{"name":"b","c":"1"} +{"name":"c","c":"1"} +3 +{"row":{"a":"1"}} +{"progress":{"read_rows":"1","read_bytes":"1","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} +4 +{"row":{"a":"1"}} +{"progress":{"read_rows":"1","read_bytes":"1","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} +5 +{"v1":"first","v2":"1","v3":"2","v4":"0"} +{"v1":"second","v2":"2","v3":"0","v4":"6"} +6 +{"v1":"first","v2":"1","v3":"2","v4":"0"} +{"v1":"second","v2":"2","v3":"0","v4":"6"} +7 +{"v1":"16","n.id":"[15,16,17]","n.name":"['first','second','third']"} diff --git a/tests/queries/0_stateless/01446_json_strings_each_row.sql b/tests/queries/0_stateless/01446_json_strings_each_row.sql new file mode 100644 index 00000000000..98bd3e3ab47 --- /dev/null +++ b/tests/queries/0_stateless/01446_json_strings_each_row.sql @@ -0,0 +1,38 @@ +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; +SELECT 1; +/* Check JSONStringsEachRow Output */ +CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value; +INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c'); +SELECT * FROM test_table FORMAT JSONStringsEachRow; +SELECT 2; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONStringsEachRow; +SELECT 3; +/* Check JSONStringsEachRowWithProgress Output */ +SELECT 1 as a FROM system.one FORMAT JSONStringsEachRowWithProgress; +SELECT 4; +/* Check Totals */ +SELECT 1 as a FROM system.one GROUP BY a WITH TOTALS ORDER BY a FORMAT JSONStringsEachRowWithProgress; +DROP TABLE IF EXISTS test_table; +SELECT 5; +/* Check JSONStringsEachRow Input */ +CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; +INSERT INTO test_table FORMAT JSONStringsEachRow {"v1": "first", "v2": "1", "v3": "2", "v4": "NULL"} {"v1": "second", "v2": "2", "v3": "null", "v4": "6"}; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 6; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONStringsEachRow {"v1": "first", "v2": "1", "v3": "2", "v4": "ᴺᵁᴸᴸ"} {"v1": "second", "v2": "2", "v3": "null", "v4": "6"}; +SELECT * FROM test_table FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 7; +/* Check Nested */ +CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; +INSERT INTO test_table_2 FORMAT JSONStringsEachRow {"v1": "16", "n.id": "[15, 16, 17]", "n.name": "['first', 'second', 'third']"}; +SELECT * FROM test_table_2 FORMAT JSONStringsEachRow; +TRUNCATE TABLE test_table_2; + +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; diff --git a/tests/queries/0_stateless/01447_json_strings.reference b/tests/queries/0_stateless/01447_json_strings.reference new file mode 100644 index 00000000000..ab88e2f3696 --- /dev/null +++ b/tests/queries/0_stateless/01447_json_strings.reference @@ -0,0 +1,43 @@ +{ + "meta": + [ + { + "name": "1", + "type": "UInt8" + }, + { + "name": "'a'", + "type": "String" + }, + { + "name": "[1, 2, 3]", + "type": "Array(UInt8)" + }, + { + "name": "tuple(1, 'a')", + "type": "Tuple(UInt8, String)" + }, + { + "name": "NULL", + "type": "Nullable(Nothing)" + }, + { + "name": "nan", + "type": "Float64" + } + ], + + "data": + [ + { + "1": "1", + "'a'": "a", + "[1, 2, 3]": "[1,2,3]", + "tuple(1, 'a')": "(1,'a')", + "NULL": "ᴺᵁᴸᴸ", + "nan": "nan" + } + ], + + "rows": 1 +} diff --git a/tests/queries/0_stateless/01447_JSONStrings.sql b/tests/queries/0_stateless/01447_json_strings.sql similarity index 100% rename from tests/queries/0_stateless/01447_JSONStrings.sql rename to tests/queries/0_stateless/01447_json_strings.sql diff --git a/tests/queries/0_stateless/01446_JSONStringsEachRow.reference b/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference similarity index 100% rename from tests/queries/0_stateless/01446_JSONStringsEachRow.reference rename to tests/queries/0_stateless/01448_json_compact_strings_each_row.reference diff --git a/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql b/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql new file mode 100644 index 00000000000..c271de88434 --- /dev/null +++ b/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql @@ -0,0 +1,63 @@ +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; +SELECT 1; +/* Check JSONCompactStringsEachRow Output */ +CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value; +INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c'); +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +SELECT 2; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactStringsEachRow; +SELECT 3; +/* Check JSONCompactStringsEachRowWithNamesAndTypes Output */ +SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes; +SELECT 4; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactStringsEachRowWithNamesAndTypes; +DROP TABLE IF EXISTS test_table; +SELECT 5; +/* Check JSONCompactStringsEachRow Input */ +CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; +INSERT INTO test_table FORMAT JSONCompactStringsEachRow ["first", "1", "2", "NULL"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 6; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONCompactStringsEachRow ["first", "1", "2", "ᴺᵁᴸᴸ"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 7; +/* Check Nested */ +CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; +INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRow ["16", "[15, 16, 17]", "['first', 'second', 'third']"]; +SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRow; +TRUNCATE TABLE test_table_2; +SELECT 8; +/* Check JSONCompactStringsEachRowWithNamesAndTypes Output */ +SET input_format_null_as_default = 0; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"]["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +TRUNCATE TABLE test_table; +SELECT 9; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"] ["second", "2", "null", "6"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +SELECT 10; +/* Check Header */ +TRUNCATE TABLE test_table; +SET input_format_skip_unknown_fields = 1; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", "1", "32"]["second", "2", "64"]; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; +SELECT 11; +TRUNCATE TABLE test_table; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"]["1", "2", "3"] +SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes; +SELECT 12; +/* Check Nested */ +INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"]["16", "[15, 16, 17]", "['first', 'second', 'third']"]; +SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRowWithNamesAndTypes; + +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; diff --git a/tests/queries/0_stateless/01447_JSONStrings.reference b/tests/queries/0_stateless/01449_json_compact_strings.reference similarity index 100% rename from tests/queries/0_stateless/01447_JSONStrings.reference rename to tests/queries/0_stateless/01449_json_compact_strings.reference diff --git a/tests/queries/0_stateless/01449_json_compact_strings.sql b/tests/queries/0_stateless/01449_json_compact_strings.sql new file mode 100644 index 00000000000..5b676e30347 --- /dev/null +++ b/tests/queries/0_stateless/01449_json_compact_strings.sql @@ -0,0 +1,10 @@ +SET output_format_write_statistics = 0; + +SELECT + 1, + 'a', + [1, 2, 3], + (1, 'a'), + null, + nan +FORMAT JSONCompactStrings; From 23b9677879a2a0618b35032439650ec08e760c57 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 4 Sep 2020 08:46:58 +0300 Subject: [PATCH 013/123] Added a script to import git repository to ClickHouse --- src/Common/ShellCommand.cpp | 4 + src/IO/ReadBufferFromFile.cpp | 3 + src/IO/WriteBufferFromFile.cpp | 3 + utils/CMakeLists.txt | 1 + utils/git-to-clickhouse/CMakeLists.txt | 2 + utils/git-to-clickhouse/git-to-clickhouse.cpp | 638 ++++++++++++++++++ 6 files changed, 651 insertions(+) create mode 100644 utils/git-to-clickhouse/CMakeLists.txt create mode 100644 utils/git-to-clickhouse/git-to-clickhouse.cpp diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 53ab2301a0a..127f95fef06 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -186,6 +186,10 @@ int ShellCommand::tryWait() { wait_called = true; + in.close(); + out.close(); + err.close(); + LOG_TRACE(getLogger(), "Will wait for shell command pid {}", pid); int status = 0; diff --git a/src/IO/ReadBufferFromFile.cpp b/src/IO/ReadBufferFromFile.cpp index 40f69625e68..226615c757e 100644 --- a/src/IO/ReadBufferFromFile.cpp +++ b/src/IO/ReadBufferFromFile.cpp @@ -77,6 +77,9 @@ ReadBufferFromFile::~ReadBufferFromFile() void ReadBufferFromFile::close() { + if (fd < 0) + return; + if (0 != ::close(fd)) throw Exception("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE); diff --git a/src/IO/WriteBufferFromFile.cpp b/src/IO/WriteBufferFromFile.cpp index b59a110edb4..4ade2e2c971 100644 --- a/src/IO/WriteBufferFromFile.cpp +++ b/src/IO/WriteBufferFromFile.cpp @@ -92,6 +92,9 @@ WriteBufferFromFile::~WriteBufferFromFile() /// Close file before destruction of object. void WriteBufferFromFile::close() { + if (fd < 0) + return; + next(); if (0 != ::close(fd)) diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 0dd95388e7d..dd03afe9fb8 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -29,6 +29,7 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) add_subdirectory (convert-month-partitioned-parts) add_subdirectory (checksum-for-compressed-block) add_subdirectory (wal-dump) + add_subdirectory (git-to-clickhouse) endif () if (ENABLE_CODE_QUALITY) diff --git a/utils/git-to-clickhouse/CMakeLists.txt b/utils/git-to-clickhouse/CMakeLists.txt new file mode 100644 index 00000000000..0e46b68d471 --- /dev/null +++ b/utils/git-to-clickhouse/CMakeLists.txt @@ -0,0 +1,2 @@ +add_executable (git-to-clickhouse git-to-clickhouse.cpp) +target_link_libraries(git-to-clickhouse PRIVATE dbms boost::program_options) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp new file mode 100644 index 00000000000..42920328ad7 --- /dev/null +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -0,0 +1,638 @@ +#include +#include +#include + +#include + +#include +#include +#include +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; +} + +enum class LineType +{ + Empty, + Comment, + Punct, + Code, +}; + +void writeText(LineType type, WriteBuffer & out) +{ + switch (type) + { + case LineType::Empty: writeString("Empty", out); break; + case LineType::Comment: writeString("Comment", out); break; + case LineType::Punct: writeString("Punct", out); break; + case LineType::Code: writeString("Code", out); break; + } +} + +struct LineChange +{ + int8_t sign{}; /// 1 if added, -1 if deleted + uint16_t line_number_old{}; + uint16_t line_number_new{}; + uint16_t hunk_num{}; /// ordinal number of hunk in diff, starting with 0 + uint16_t hunk_start_line_number_old{}; + uint16_t hunk_start_line_number_new{}; + std::string hunk_context; /// The context (like a line with function name) as it is calculated by git + std::string line; /// Line content without leading whitespaces + uint8_t indent{}; /// The number of leading whitespaces or tabs * 4 + LineType line_type{}; + + void setLineInfo(std::string full_line) + { + indent = 0; + + const char * pos = full_line.data(); + const char * end = pos + full_line.size(); + + while (pos < end) + { + if (*pos == ' ') + ++indent; + else if (*pos == '\t') + indent += 4; + else + break; + ++pos; + } + + line.assign(pos, end); + + if (pos == end) + { + line_type = LineType::Empty; + } + else if (pos + 1 < end + && ((pos[0] == '/' && pos[1] == '/') + || (pos[0] == '*' && pos[1] == ' '))) /// This is not precise. + { + line_type = LineType::Comment; + } + else + { + while (pos < end) + { + if (isAlphaNumericASCII(*pos)) + { + line_type = LineType::Code; + break; + } + ++pos; + } + if (pos == end) + line_type = LineType::Punct; + } + } + + void writeTextWithoutNewline(WriteBuffer & out) const + { + writeText(sign, out); + writeChar('\t', out); + writeText(line_number_old, out); + writeChar('\t', out); + writeText(line_number_new, out); + writeChar('\t', out); + writeText(hunk_num, out); + writeChar('\t', out); + writeText(hunk_start_line_number_old, out); + writeChar('\t', out); + writeText(hunk_start_line_number_new, out); + writeChar('\t', out); + writeText(hunk_context, out); + writeChar('\t', out); + writeText(line, out); + writeChar('\t', out); + writeText(indent, out); + writeChar('\t', out); + writeText(line_type, out); + } +}; + +using LineChanges = std::vector; + +enum class FileChangeType +{ + Add, + Delete, + Modify, + Rename, + Copy, + Type, +}; + +void writeText(FileChangeType type, WriteBuffer & out) +{ + switch (type) + { + case FileChangeType::Add: writeString("Add", out); break; + case FileChangeType::Delete: writeString("Delete", out); break; + case FileChangeType::Modify: writeString("Modify", out); break; + case FileChangeType::Rename: writeString("Rename", out); break; + case FileChangeType::Copy: writeString("Copy", out); break; + case FileChangeType::Type: writeString("Type", out); break; + } +} + +struct FileChange +{ + FileChangeType change_type{}; + std::string new_file_path; + std::string old_file_path; + uint16_t lines_added{}; + uint16_t lines_deleted{}; + uint16_t hunks_added{}; + uint16_t hunks_removed{}; + uint16_t hunks_changed{}; + + void writeTextWithoutNewline(WriteBuffer & out) const + { + writeText(change_type, out); + writeChar('\t', out); + writeText(new_file_path, out); + writeChar('\t', out); + writeText(old_file_path, out); + writeChar('\t', out); + writeText(lines_added, out); + writeChar('\t', out); + writeText(lines_deleted, out); + writeChar('\t', out); + writeText(hunks_added, out); + writeChar('\t', out); + writeText(hunks_removed, out); + writeChar('\t', out); + writeText(hunks_changed, out); + } +}; + +struct FileChangeAndLineChanges +{ + FileChange file_change; + LineChanges line_changes; +}; + +struct Commit +{ + std::string hash; + std::string author_name; + std::string author_email; + time_t time{}; + std::string message; + uint32_t files_added{}; + uint32_t files_deleted{}; + uint32_t files_renamed{}; + uint32_t files_modified{}; + uint32_t lines_added{}; + uint32_t lines_deleted{}; + uint32_t hunks_added{}; + uint32_t hunks_removed{}; + uint32_t hunks_changed{}; + + void writeTextWithoutNewline(WriteBuffer & out) const + { + writeText(hash, out); + writeChar('\t', out); + writeText(author_name, out); + writeChar('\t', out); + writeText(author_email, out); + writeChar('\t', out); + writeText(time, out); + writeChar('\t', out); + writeText(message, out); + writeChar('\t', out); + writeText(files_added, out); + writeChar('\t', out); + writeText(files_deleted, out); + writeChar('\t', out); + writeText(files_renamed, out); + writeChar('\t', out); + writeText(files_modified, out); + writeChar('\t', out); + writeText(lines_added, out); + writeChar('\t', out); + writeText(lines_deleted, out); + writeChar('\t', out); + writeText(hunks_added, out); + writeChar('\t', out); + writeText(hunks_removed, out); + writeChar('\t', out); + writeText(hunks_changed, out); + } +}; + + +void skipUntilWhitespace(ReadBuffer & buf) +{ + while (!buf.eof()) + { + char * next_pos = find_first_symbols<'\t', '\n', ' '>(buf.position(), buf.buffer().end()); + buf.position() = next_pos; + + if (!buf.hasPendingData()) + continue; + + if (*buf.position() == '\t' || *buf.position() == '\n' || *buf.position() == ' ') + return; + } +} + +void skipUntilNextLine(ReadBuffer & buf) +{ + while (!buf.eof()) + { + char * next_pos = find_first_symbols<'\n'>(buf.position(), buf.buffer().end()); + buf.position() = next_pos; + + if (!buf.hasPendingData()) + continue; + + if (*buf.position() == '\n') + { + ++buf.position(); + return; + } + } +} + +void readStringUntilNextLine(std::string & s, ReadBuffer & buf) +{ + s.clear(); + while (!buf.eof()) + { + char * next_pos = find_first_symbols<'\n'>(buf.position(), buf.buffer().end()); + s.append(buf.position(), next_pos - buf.position()); + buf.position() = next_pos; + + if (!buf.hasPendingData()) + continue; + + if (*buf.position() == '\n') + { + ++buf.position(); + return; + } + } +} + + +struct Result +{ + WriteBufferFromFile commits{"commits.tsv"}; + WriteBufferFromFile file_changes{"file_changes.tsv"}; + WriteBufferFromFile line_changes{"line_changes.tsv"}; +}; + + +void processCommit(std::string hash, Result & result) +{ + std::string command = fmt::format( + "git show --raw --pretty='format:%at%x09%aN%x09%aE%x0A%s%x00' --patch --unified=0 {}", + hash); + + std::cerr << command << "\n"; + + auto commit_info = ShellCommand::execute(command); + auto & in = commit_info->out; + + Commit commit; + commit.hash = hash; + + readText(commit.time, in); + assertChar('\t', in); + readText(commit.author_name, in); + assertChar('\t', in); + readText(commit.author_email, in); + assertChar('\n', in); + readNullTerminated(commit.message, in); + + std::cerr << fmt::format("{}\t{}\n", toString(LocalDateTime(commit.time)), commit.message); + + if (!in.eof()) + assertChar('\n', in); + + /// File changes in form + /// :100644 100644 b90fe6bb94 3ffe4c380f M src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp + /// :100644 100644 828dedf6b5 828dedf6b5 R100 dbms/src/Functions/GeoUtils.h dbms/src/Functions/PolygonUtils.h + + std::map file_changes; + + while (checkChar(':', in)) + { + FileChange file_change; + + for (size_t i = 0; i < 4; ++i) + { + skipUntilWhitespace(in); + skipWhitespaceIfAny(in); + } + + char change_type; + readChar(change_type, in); + + int confidence; + switch (change_type) + { + case 'A': + file_change.change_type = FileChangeType::Add; + ++commit.files_added; + break; + case 'D': + file_change.change_type = FileChangeType::Delete; + ++commit.files_deleted; + break; + case 'M': + file_change.change_type = FileChangeType::Modify; + ++commit.files_modified; + break; + case 'R': + file_change.change_type = FileChangeType::Rename; + ++commit.files_renamed; + readText(confidence, in); + break; + case 'C': + file_change.change_type = FileChangeType::Copy; + readText(confidence, in); + break; + case 'T': + file_change.change_type = FileChangeType::Type; + break; + default: + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected file change type: {}", change_type); + } + + skipWhitespaceIfAny(in); + + if (change_type == 'R' || change_type == 'C') + { + readText(file_change.old_file_path, in); + skipWhitespaceIfAny(in); + readText(file_change.new_file_path, in); + } + else + { + readText(file_change.new_file_path, in); + } + + assertChar('\n', in); + + file_changes.emplace( + file_change.new_file_path, + FileChangeAndLineChanges{ file_change, {} }); + } + + if (!in.eof()) + { + assertChar('\n', in); + + /// Diffs for every file in form of + /// --- a/src/Storages/StorageReplicatedMergeTree.cpp + /// +++ b/src/Storages/StorageReplicatedMergeTree.cpp + /// @@ -1387,2 +1387 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) + /// - table_lock, entry.create_time, reserved_space, entry.deduplicate, + /// - entry.force_ttl); + /// + table_lock, entry.create_time, reserved_space, entry.deduplicate); + + std::string old_file_path; + std::string new_file_path; + FileChangeAndLineChanges * file_change_and_line_changes = nullptr; + LineChange line_change; + + while (!in.eof()) + { + if (checkString("@@ ", in)) + { + if (!file_change_and_line_changes) + { + auto file_name = new_file_path.empty() ? old_file_path : new_file_path; + auto it = file_changes.find(file_name); + if (file_changes.end() == it) + std::cerr << fmt::format("Warning: skipping bad file name {}\n", file_name); + else + file_change_and_line_changes = &it->second; + } + + if (file_change_and_line_changes) + { + uint16_t old_lines = 1; + uint16_t new_lines = 1; + + assertChar('-', in); + readText(line_change.hunk_start_line_number_old, in); + if (checkChar(',', in)) + readText(old_lines, in); + + assertString(" +", in); + readText(line_change.hunk_start_line_number_new, in); + if (checkChar(',', in)) + readText(new_lines, in); + + assertString(" @@", in); + if (checkChar(' ', in)) + readStringUntilNextLine(line_change.hunk_context, in); + else + assertChar('\n', in); + + ++line_change.hunk_num; + line_change.line_number_old = line_change.hunk_start_line_number_old; + line_change.line_number_new = line_change.hunk_start_line_number_new; + + if (old_lines && new_lines) + { + ++commit.hunks_changed; + ++file_change_and_line_changes->file_change.hunks_changed; + } + else if (old_lines) + { + ++commit.hunks_removed; + ++file_change_and_line_changes->file_change.hunks_removed; + } + else if (new_lines) + { + ++commit.hunks_added; + ++file_change_and_line_changes->file_change.hunks_added; + } + } + } + else if (checkChar('-', in)) + { + if (checkString("-- ", in)) + { + if (checkString("a/", in)) + { + readStringUntilNextLine(old_file_path, in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else if (checkString("/dev/null", in)) + { + old_file_path.clear(); + assertChar('\n', in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else + skipUntilNextLine(in); /// Actually it can be the line in diff. Skip it for simplicity. + } + else + { + if (file_change_and_line_changes) + { + ++commit.lines_deleted; + + line_change.sign = -1; + readStringUntilNextLine(line_change.line, in); + line_change.setLineInfo(line_change.line); + + file_change_and_line_changes->line_changes.push_back(line_change); + ++line_change.line_number_old; + } + } + } + else if (checkChar('+', in)) + { + if (checkString("++ ", in)) + { + if (checkString("b/", in)) + { + readStringUntilNextLine(new_file_path, in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else if (checkString("/dev/null", in)) + { + new_file_path.clear(); + assertChar('\n', in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else + skipUntilNextLine(in); /// Actually it can be the line in diff. Skip it for simplicity. + } + else + { + if (file_change_and_line_changes) + { + ++commit.lines_added; + + line_change.sign = 1; + readStringUntilNextLine(line_change.line, in); + line_change.setLineInfo(line_change.line); + + file_change_and_line_changes->line_changes.push_back(line_change); + ++line_change.line_number_new; + } + } + } + else + { + skipUntilNextLine(in); + } + } + } + + /// Write the result + + /// commits table + { + auto & out = result.commits; + + commit.writeTextWithoutNewline(out); + writeChar('\n', out); + } + + for (const auto & elem : file_changes) + { + const FileChange & file_change = elem.second.file_change; + + /// file_changes table + { + auto & out = result.file_changes; + + file_change.writeTextWithoutNewline(out); + writeChar('\t', out); + commit.writeTextWithoutNewline(out); + writeChar('\n', out); + } + + /// line_changes table + for (const auto & line_change : elem.second.line_changes) + { + auto & out = result.line_changes; + + line_change.writeTextWithoutNewline(out); + writeChar('\t', out); + file_change.writeTextWithoutNewline(out); + writeChar('\t', out); + commit.writeTextWithoutNewline(out); + writeChar('\n', out); + } + } +} + + +void processLog() +{ + Result result; + + std::string command = "git log --no-merges --pretty=%H"; + std::cerr << command << "\n"; + auto git_log = ShellCommand::execute(command); + + auto & in = git_log->out; + while (!in.eof()) + { + std::string hash; + readString(hash, in); + assertChar('\n', in); + + std::cerr << fmt::format("Processing commit {}\n", hash); + processCommit(std::move(hash), result); + } +} + + +} + +int main(int /*argc*/, char ** /*argv*/) +try +{ + using namespace DB; + +/* boost::program_options::options_description desc("Allowed options"); + desc.add_options()("help,h", "produce help message"); + + boost::program_options::variables_map options; + boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); + + if (options.count("help") || argc != 2) + { + std::cout << "Usage: " << argv[0] << std::endl; + std::cout << desc << std::endl; + return 1; + }*/ + + processLog(); + return 0; +} +catch (...) +{ + std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; + throw; +} From 338a6e20f60bb21c99ee2c4f261d96bc55ec4b97 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 4 Sep 2020 09:12:16 +0300 Subject: [PATCH 014/123] Added a script to import git repository to ClickHouse --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 97 +++++++++++++++++++ 1 file changed, 97 insertions(+) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 42920328ad7..314bba0d5b4 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -16,6 +16,101 @@ #include +/** How to use: + +DROP DATABASE IF EXISTS git; +CREATE DATABASE git; + +CREATE TABLE git.commits +( + hash String, + author_name LowCardinality(String), + author_email LowCardinality(String), + time DateTime, + message String, + files_added UInt32, + files_deleted UInt32, + files_renamed UInt32, + files_modified UInt32, + lines_added UInt32, + lines_deleted UInt32, + hunks_added UInt32, + hunks_removed UInt32, + hunks_changed UInt32 +) ENGINE = MergeTree ORDER BY time; + +CREATE TABLE git.file_changes +( + change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), + new_file_path LowCardinality(String), + old_file_path LowCardinality(String), + lines_added UInt16, + lines_deleted UInt16, + hunks_added UInt16, + hunks_removed UInt16, + hunks_changed UInt16, + + commit_hash String, + author_name LowCardinality(String), + author_email LowCardinality(String), + time DateTime, + commit_message String, + commit_files_added UInt32, + commit_files_deleted UInt32, + commit_files_renamed UInt32, + commit_files_modified UInt32, + commit_lines_added UInt32, + commit_lines_deleted UInt32, + commit_hunks_added UInt32, + commit_hunks_removed UInt32, + commit_hunks_changed UInt32 +) ENGINE = MergeTree ORDER BY time; + +CREATE TABLE git.line_changes +( + sign Int8, + line_number_old UInt16, + line_number_new UInt16, + hunk_num UInt16, + hunk_start_line_number_old UInt16, + hunk_start_line_number_new UInt16, + hunk_context LowCardinality(String), + line LowCardinality(String), + indent UInt8, + line_type Enum('Empty' = 0, 'Comment' = 1, 'Punct' = 2, 'Code' = 3), + + file_change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), + new_file_path LowCardinality(String), + old_file_path LowCardinality(String), + file_lines_added UInt16, + file_lines_deleted UInt16, + file_hunks_added UInt16, + file_hunks_removed UInt16, + file_hunks_changed UInt16, + + commit_hash String, + author_name LowCardinality(String), + author_email LowCardinality(String), + time DateTime, + commit_message String, + commit_files_added UInt32, + commit_files_deleted UInt32, + commit_files_renamed UInt32, + commit_files_modified UInt32, + commit_lines_added UInt32, + commit_lines_deleted UInt32, + commit_hunks_added UInt32, + commit_hunks_removed UInt32, + commit_hunks_changed UInt32 +) ENGINE = MergeTree ORDER BY time; + +clickhouse-client --query "INSERT INTO git.commits FORMAT TSV" < commits.tsv +clickhouse-client --query "INSERT INTO git.file_changes FORMAT TSV" < file_changes.tsv +clickhouse-client --query "INSERT INTO git.line_changes FORMAT TSV" < line_changes.tsv + + */ + + namespace DB { @@ -495,6 +590,7 @@ void processCommit(std::string hash, Result & result) if (file_change_and_line_changes) { ++commit.lines_deleted; + ++file_change_and_line_changes->file_change.lines_deleted; line_change.sign = -1; readStringUntilNextLine(line_change.line, in); @@ -530,6 +626,7 @@ void processCommit(std::string hash, Result & result) if (file_change_and_line_changes) { ++commit.lines_added; + ++file_change_and_line_changes->file_change.lines_added; line_change.sign = 1; readStringUntilNextLine(line_change.line, in); From 7b95e56e8c902578f8fcebc5d9edeccce1eb35ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 03:09:40 +0300 Subject: [PATCH 015/123] Advancements --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 187 +++++++++++++----- 1 file changed, 133 insertions(+), 54 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 314bba0d5b4..d6264a63978 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -1,6 +1,11 @@ #include #include #include +#include +#include +#include + +#include #include @@ -16,7 +21,8 @@ #include -/** How to use: +static constexpr auto documentation = R"( +Prepare the database by executing the following queries: DROP DATABASE IF EXISTS git; CREATE DATABASE git; @@ -44,11 +50,11 @@ CREATE TABLE git.file_changes change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), new_file_path LowCardinality(String), old_file_path LowCardinality(String), - lines_added UInt16, - lines_deleted UInt16, - hunks_added UInt16, - hunks_removed UInt16, - hunks_changed UInt16, + lines_added UInt32, + lines_deleted UInt32, + hunks_added UInt32, + hunks_removed UInt32, + hunks_changed UInt32, commit_hash String, author_name LowCardinality(String), @@ -69,11 +75,11 @@ CREATE TABLE git.file_changes CREATE TABLE git.line_changes ( sign Int8, - line_number_old UInt16, - line_number_new UInt16, - hunk_num UInt16, - hunk_start_line_number_old UInt16, - hunk_start_line_number_new UInt16, + line_number_old UInt32, + line_number_new UInt32, + hunk_num UInt32, + hunk_start_line_number_old UInt32, + hunk_start_line_number_new UInt32, hunk_context LowCardinality(String), line LowCardinality(String), indent UInt8, @@ -82,11 +88,11 @@ CREATE TABLE git.line_changes file_change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), new_file_path LowCardinality(String), old_file_path LowCardinality(String), - file_lines_added UInt16, - file_lines_deleted UInt16, - file_hunks_added UInt16, - file_hunks_removed UInt16, - file_hunks_changed UInt16, + file_lines_added UInt32, + file_lines_deleted UInt32, + file_hunks_added UInt32, + file_hunks_removed UInt32, + file_hunks_changed UInt32, commit_hash String, author_name LowCardinality(String), @@ -104,12 +110,15 @@ CREATE TABLE git.line_changes commit_hunks_changed UInt32 ) ENGINE = MergeTree ORDER BY time; +Insert the data with the following commands: + clickhouse-client --query "INSERT INTO git.commits FORMAT TSV" < commits.tsv clickhouse-client --query "INSERT INTO git.file_changes FORMAT TSV" < file_changes.tsv clickhouse-client --query "INSERT INTO git.line_changes FORMAT TSV" < line_changes.tsv - */ +)"; +namespace po = boost::program_options; namespace DB { @@ -141,11 +150,11 @@ void writeText(LineType type, WriteBuffer & out) struct LineChange { int8_t sign{}; /// 1 if added, -1 if deleted - uint16_t line_number_old{}; - uint16_t line_number_new{}; - uint16_t hunk_num{}; /// ordinal number of hunk in diff, starting with 0 - uint16_t hunk_start_line_number_old{}; - uint16_t hunk_start_line_number_new{}; + uint32_t line_number_old{}; + uint32_t line_number_new{}; + uint32_t hunk_num{}; /// ordinal number of hunk in diff, starting with 0 + uint32_t hunk_start_line_number_old{}; + uint32_t hunk_start_line_number_new{}; std::string hunk_context; /// The context (like a line with function name) as it is calculated by git std::string line; /// Line content without leading whitespaces uint8_t indent{}; /// The number of leading whitespaces or tabs * 4 @@ -251,11 +260,11 @@ struct FileChange FileChangeType change_type{}; std::string new_file_path; std::string old_file_path; - uint16_t lines_added{}; - uint16_t lines_deleted{}; - uint16_t hunks_added{}; - uint16_t hunks_removed{}; - uint16_t hunks_changed{}; + uint32_t lines_added{}; + uint32_t lines_deleted{}; + uint32_t hunks_added{}; + uint32_t hunks_removed{}; + uint32_t hunks_changed{}; void writeTextWithoutNewline(WriteBuffer & out) const { @@ -395,13 +404,38 @@ struct Result }; -void processCommit(std::string hash, Result & result) +struct Options +{ + bool skip_commits_without_parents = true; + std::optional skip_paths; + std::unordered_set skip_commits; + size_t diff_size_limit = 0; + + Options(const po::variables_map & options) + { + skip_commits_without_parents = options["skip-commits-without-parents"].as(); + if (options.count("skip-paths")) + { + skip_paths.emplace(options["skip-paths"].as()); + } + if (options.count("skip-commit")) + { + auto vec = options["skip-commit"].as>(); + skip_commits.insert(vec.begin(), vec.end()); + } + diff_size_limit = options["diff-size-limit"].as(); + } +}; + + +void processCommit( + const Options & options, size_t commit_num, size_t total_commits, std::string hash, Result & result) { std::string command = fmt::format( - "git show --raw --pretty='format:%at%x09%aN%x09%aE%x0A%s%x00' --patch --unified=0 {}", + "git show --raw --pretty='format:%at%x09%aN%x09%aE%x09%P%x0A%s%x00' --patch --unified=0 {}", hash); - std::cerr << command << "\n"; + //std::cerr << command << "\n"; auto commit_info = ShellCommand::execute(command); auto & in = commit_info->out; @@ -414,10 +448,23 @@ void processCommit(std::string hash, Result & result) readText(commit.author_name, in); assertChar('\t', in); readText(commit.author_email, in); + assertChar('\t', in); + std::string parent_hash; + readString(parent_hash, in); assertChar('\n', in); readNullTerminated(commit.message, in); - std::cerr << fmt::format("{}\t{}\n", toString(LocalDateTime(commit.time)), commit.message); + std::string message_to_print = commit.message; + std::replace_if(message_to_print.begin(), message_to_print.end(), [](char c){ return std::iscntrl(c); }, ' '); + + fmt::print("{}% {} {} {}\n", + commit_num * 100 / total_commits, toString(LocalDateTime(commit.time)), hash, message_to_print); + + if (options.skip_commits_without_parents && commit_num != 0 && parent_hash.empty()) + { + std::cerr << "Warning: skipping commit without parents\n"; + return; + } if (!in.eof()) assertChar('\n', in); @@ -487,9 +534,12 @@ void processCommit(std::string hash, Result & result) assertChar('\n', in); - file_changes.emplace( - file_change.new_file_path, - FileChangeAndLineChanges{ file_change, {} }); + if (!(options.skip_paths && re2_st::RE2::PartialMatch(file_change.new_file_path, *options.skip_paths))) + { + file_changes.emplace( + file_change.new_file_path, + FileChangeAndLineChanges{ file_change, {} }); + } } if (!in.eof()) @@ -517,16 +567,14 @@ void processCommit(std::string hash, Result & result) { auto file_name = new_file_path.empty() ? old_file_path : new_file_path; auto it = file_changes.find(file_name); - if (file_changes.end() == it) - std::cerr << fmt::format("Warning: skipping bad file name {}\n", file_name); - else + if (file_changes.end() != it) file_change_and_line_changes = &it->second; } if (file_change_and_line_changes) { - uint16_t old_lines = 1; - uint16_t new_lines = 1; + uint32_t old_lines = 1; + uint32_t new_lines = 1; assertChar('-', in); readText(line_change.hunk_start_line_number_old, in); @@ -644,6 +692,9 @@ void processCommit(std::string hash, Result & result) } } + if (commit.lines_added + commit.lines_deleted > options.diff_size_limit) + return; + /// Write the result /// commits table @@ -684,14 +735,20 @@ void processCommit(std::string hash, Result & result) } -void processLog() +void processLog(const Options & options) { Result result; - std::string command = "git log --no-merges --pretty=%H"; - std::cerr << command << "\n"; + std::string command = "git log --reverse --no-merges --pretty=%H"; + fmt::print("{}\n", command); auto git_log = ShellCommand::execute(command); + /// Collect hashes in memory. This is inefficient but allows to display beautiful progress. + /// The number of commits is in order of single millions for the largest repositories, + /// so don't care about potential waste of ~100 MB of memory. + + std::vector hashes; + auto & in = git_log->out; while (!in.eof()) { @@ -699,33 +756,55 @@ void processLog() readString(hash, in); assertChar('\n', in); - std::cerr << fmt::format("Processing commit {}\n", hash); - processCommit(std::move(hash), result); + if (!options.skip_commits.count(hash)) + hashes.emplace_back(std::move(hash)); + } + + size_t num_commits = hashes.size(); + fmt::print("Total {} commits to process.\n", num_commits); + + for (size_t i = 0; i < num_commits; ++i) + { + processCommit(options, i, num_commits, hashes[i], result); } } } -int main(int /*argc*/, char ** /*argv*/) +int main(int argc, char ** argv) try { using namespace DB; -/* boost::program_options::options_description desc("Allowed options"); - desc.add_options()("help,h", "produce help message"); + po::options_description desc("Allowed options"); + desc.add_options() + ("help,h", "produce help message") + ("skip-commits-without-parents", po::value()->default_value(true), + "Skip commits without parents (except the initial commit)." + " These commits are usually erroneous but they can make sense in very rare cases.") + ("skip-paths", po::value(), + "Skip paths that matches regular expression (re2 syntax).") + ("skip-commit", po::value>(), + "Skip commit with specified hash. The option can be specified multiple times.") + ("diff-size-limit", po::value()->default_value(0), + "Skip commits whose diff size (number of added + removed lines) is larger than specified threshold") + ; - boost::program_options::variables_map options; - boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); + po::variables_map options; + po::store(boost::program_options::parse_command_line(argc, argv, desc), options); - if (options.count("help") || argc != 2) + if (options.count("help")) { - std::cout << "Usage: " << argv[0] << std::endl; - std::cout << desc << std::endl; + std::cout << documentation << '\n' + << "Usage: " << argv[0] << '\n' + << desc << '\n' + << "\nExample:\n" + << "\n./git-to-clickhouse --diff-size-limit 100000 --skip-paths '^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/'\n"; return 1; - }*/ + } - processLog(); + processLog(options); return 0; } catch (...) From abe836a584aeaf71b0ba04b8c8cc670385519e94 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 03:13:39 +0300 Subject: [PATCH 016/123] Remove emails as they are mostly useless --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 20 ++++++------------- 1 file changed, 6 insertions(+), 14 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index d6264a63978..9203efb0043 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -30,8 +30,7 @@ CREATE DATABASE git; CREATE TABLE git.commits ( hash String, - author_name LowCardinality(String), - author_email LowCardinality(String), + author LowCardinality(String), time DateTime, message String, files_added UInt32, @@ -57,8 +56,7 @@ CREATE TABLE git.file_changes hunks_changed UInt32, commit_hash String, - author_name LowCardinality(String), - author_email LowCardinality(String), + author LowCardinality(String), time DateTime, commit_message String, commit_files_added UInt32, @@ -95,8 +93,7 @@ CREATE TABLE git.line_changes file_hunks_changed UInt32, commit_hash String, - author_name LowCardinality(String), - author_email LowCardinality(String), + author LowCardinality(String), time DateTime, commit_message String, commit_files_added UInt32, @@ -295,8 +292,7 @@ struct FileChangeAndLineChanges struct Commit { std::string hash; - std::string author_name; - std::string author_email; + std::string author; time_t time{}; std::string message; uint32_t files_added{}; @@ -313,9 +309,7 @@ struct Commit { writeText(hash, out); writeChar('\t', out); - writeText(author_name, out); - writeChar('\t', out); - writeText(author_email, out); + writeText(author, out); writeChar('\t', out); writeText(time, out); writeChar('\t', out); @@ -445,9 +439,7 @@ void processCommit( readText(commit.time, in); assertChar('\t', in); - readText(commit.author_name, in); - assertChar('\t', in); - readText(commit.author_email, in); + readText(commit.author, in); assertChar('\t', in); std::string parent_hash; readString(parent_hash, in); From 09978decbdf40c95e7cd8855ad804a2ad31cc09d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 03:47:00 +0300 Subject: [PATCH 017/123] Adjustments --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 52 +++++++++++++------ 1 file changed, 35 insertions(+), 17 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 9203efb0043..a81bc6679a7 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -47,8 +48,9 @@ CREATE TABLE git.commits CREATE TABLE git.file_changes ( change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), - new_file_path LowCardinality(String), - old_file_path LowCardinality(String), + path LowCardinality(String), + old_path LowCardinality(String), + file_extension LowCardinality(String), lines_added UInt32, lines_deleted UInt32, hunks_added UInt32, @@ -84,8 +86,9 @@ CREATE TABLE git.line_changes line_type Enum('Empty' = 0, 'Comment' = 1, 'Punct' = 2, 'Code' = 3), file_change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), - new_file_path LowCardinality(String), - old_file_path LowCardinality(String), + path LowCardinality(String), + old_path LowCardinality(String), + file_extension LowCardinality(String), file_lines_added UInt32, file_lines_deleted UInt32, file_hunks_added UInt32, @@ -255,8 +258,9 @@ void writeText(FileChangeType type, WriteBuffer & out) struct FileChange { FileChangeType change_type{}; - std::string new_file_path; - std::string old_file_path; + std::string path; + std::string old_path; + std::string file_extension; uint32_t lines_added{}; uint32_t lines_deleted{}; uint32_t hunks_added{}; @@ -267,9 +271,11 @@ struct FileChange { writeText(change_type, out); writeChar('\t', out); - writeText(new_file_path, out); + writeText(path, out); writeChar('\t', out); - writeText(old_file_path, out); + writeText(old_path, out); + writeChar('\t', out); + writeText(file_extension, out); writeChar('\t', out); writeText(lines_added, out); writeChar('\t', out); @@ -422,11 +428,20 @@ struct Options }; +/// Rough snapshot of repository calculated by application of diffs. It's used to calculate blame info. +struct File +{ + std::vector lines; +}; + +using Snapshot = std::map; + + void processCommit( - const Options & options, size_t commit_num, size_t total_commits, std::string hash, Result & result) + const Options & options, size_t commit_num, size_t total_commits, std::string hash, Snapshot & /*snapshot*/, Result & result) { std::string command = fmt::format( - "git show --raw --pretty='format:%at%x09%aN%x09%aE%x09%P%x0A%s%x00' --patch --unified=0 {}", + "git show --raw --pretty='format:%at%x09%aN%x09%P%x0A%s%x00' --patch --unified=0 {}", hash); //std::cerr << command << "\n"; @@ -515,21 +530,23 @@ void processCommit( if (change_type == 'R' || change_type == 'C') { - readText(file_change.old_file_path, in); + readText(file_change.old_path, in); skipWhitespaceIfAny(in); - readText(file_change.new_file_path, in); + readText(file_change.path, in); } else { - readText(file_change.new_file_path, in); + readText(file_change.path, in); } + file_change.file_extension = std::filesystem::path(file_change.path).extension(); + assertChar('\n', in); - if (!(options.skip_paths && re2_st::RE2::PartialMatch(file_change.new_file_path, *options.skip_paths))) + if (!(options.skip_paths && re2_st::RE2::PartialMatch(file_change.path, *options.skip_paths))) { file_changes.emplace( - file_change.new_file_path, + file_change.path, FileChangeAndLineChanges{ file_change, {} }); } } @@ -755,9 +772,10 @@ void processLog(const Options & options) size_t num_commits = hashes.size(); fmt::print("Total {} commits to process.\n", num_commits); + Snapshot snapshot; for (size_t i = 0; i < num_commits; ++i) { - processCommit(options, i, num_commits, hashes[i], result); + processCommit(options, i, num_commits, hashes[i], snapshot, result); } } @@ -792,7 +810,7 @@ try << "Usage: " << argv[0] << '\n' << desc << '\n' << "\nExample:\n" - << "\n./git-to-clickhouse --diff-size-limit 100000 --skip-paths '^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/'\n"; + << "\n./git-to-clickhouse --diff-size-limit 100000 --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/'\n"; return 1; } From d1f1326a1370abd5d837864d02851ef1b3b20745 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 07:02:17 +0300 Subject: [PATCH 018/123] Concurrent processing + history --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 411 ++++++++++++------ 1 file changed, 283 insertions(+), 128 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index a81bc6679a7..6686c1ac480 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include @@ -80,11 +82,17 @@ CREATE TABLE git.line_changes hunk_num UInt32, hunk_start_line_number_old UInt32, hunk_start_line_number_new UInt32, + hunk_lines_added UInt32, + hunk_lines_deleted UInt32, hunk_context LowCardinality(String), line LowCardinality(String), indent UInt8, line_type Enum('Empty' = 0, 'Comment' = 1, 'Punct' = 2, 'Code' = 3), + prev_commit_hash String, + prev_author LowCardinality(String), + prev_time DateTime, + file_change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), path LowCardinality(String), old_path LowCardinality(String), @@ -128,6 +136,112 @@ namespace ErrorCodes extern const int INCORRECT_DATA; } + +struct Commit +{ + std::string hash; + std::string author; + LocalDateTime time{}; + std::string message; + uint32_t files_added{}; + uint32_t files_deleted{}; + uint32_t files_renamed{}; + uint32_t files_modified{}; + uint32_t lines_added{}; + uint32_t lines_deleted{}; + uint32_t hunks_added{}; + uint32_t hunks_removed{}; + uint32_t hunks_changed{}; + + void writeTextWithoutNewline(WriteBuffer & out) const + { + writeText(hash, out); + writeChar('\t', out); + writeText(author, out); + writeChar('\t', out); + writeText(time, out); + writeChar('\t', out); + writeText(message, out); + writeChar('\t', out); + writeText(files_added, out); + writeChar('\t', out); + writeText(files_deleted, out); + writeChar('\t', out); + writeText(files_renamed, out); + writeChar('\t', out); + writeText(files_modified, out); + writeChar('\t', out); + writeText(lines_added, out); + writeChar('\t', out); + writeText(lines_deleted, out); + writeChar('\t', out); + writeText(hunks_added, out); + writeChar('\t', out); + writeText(hunks_removed, out); + writeChar('\t', out); + writeText(hunks_changed, out); + } +}; + + +enum class FileChangeType +{ + Add, + Delete, + Modify, + Rename, + Copy, + Type, +}; + +void writeText(FileChangeType type, WriteBuffer & out) +{ + switch (type) + { + case FileChangeType::Add: writeString("Add", out); break; + case FileChangeType::Delete: writeString("Delete", out); break; + case FileChangeType::Modify: writeString("Modify", out); break; + case FileChangeType::Rename: writeString("Rename", out); break; + case FileChangeType::Copy: writeString("Copy", out); break; + case FileChangeType::Type: writeString("Type", out); break; + } +} + +struct FileChange +{ + FileChangeType change_type{}; + std::string path; + std::string old_path; + std::string file_extension; + uint32_t lines_added{}; + uint32_t lines_deleted{}; + uint32_t hunks_added{}; + uint32_t hunks_removed{}; + uint32_t hunks_changed{}; + + void writeTextWithoutNewline(WriteBuffer & out) const + { + writeText(change_type, out); + writeChar('\t', out); + writeText(path, out); + writeChar('\t', out); + writeText(old_path, out); + writeChar('\t', out); + writeText(file_extension, out); + writeChar('\t', out); + writeText(lines_added, out); + writeChar('\t', out); + writeText(lines_deleted, out); + writeChar('\t', out); + writeText(hunks_added, out); + writeChar('\t', out); + writeText(hunks_removed, out); + writeChar('\t', out); + writeText(hunks_changed, out); + } +}; + + enum class LineType { Empty, @@ -155,10 +269,15 @@ struct LineChange uint32_t hunk_num{}; /// ordinal number of hunk in diff, starting with 0 uint32_t hunk_start_line_number_old{}; uint32_t hunk_start_line_number_new{}; + uint32_t hunk_lines_added{}; + uint32_t hunk_lines_deleted{}; std::string hunk_context; /// The context (like a line with function name) as it is calculated by git std::string line; /// Line content without leading whitespaces uint8_t indent{}; /// The number of leading whitespaces or tabs * 4 LineType line_type{}; + std::string prev_commit_hash; + std::string prev_author; + LocalDateTime prev_time{}; void setLineInfo(std::string full_line) { @@ -220,6 +339,10 @@ struct LineChange writeChar('\t', out); writeText(hunk_start_line_number_new, out); writeChar('\t', out); + writeText(hunk_lines_added, out); + writeChar('\t', out); + writeText(hunk_lines_deleted, out); + writeChar('\t', out); writeText(hunk_context, out); writeChar('\t', out); writeText(line, out); @@ -227,120 +350,17 @@ struct LineChange writeText(indent, out); writeChar('\t', out); writeText(line_type, out); + writeChar('\t', out); + writeText(prev_commit_hash, out); + writeChar('\t', out); + writeText(prev_author, out); + writeChar('\t', out); + writeText(prev_time, out); } }; using LineChanges = std::vector; -enum class FileChangeType -{ - Add, - Delete, - Modify, - Rename, - Copy, - Type, -}; - -void writeText(FileChangeType type, WriteBuffer & out) -{ - switch (type) - { - case FileChangeType::Add: writeString("Add", out); break; - case FileChangeType::Delete: writeString("Delete", out); break; - case FileChangeType::Modify: writeString("Modify", out); break; - case FileChangeType::Rename: writeString("Rename", out); break; - case FileChangeType::Copy: writeString("Copy", out); break; - case FileChangeType::Type: writeString("Type", out); break; - } -} - -struct FileChange -{ - FileChangeType change_type{}; - std::string path; - std::string old_path; - std::string file_extension; - uint32_t lines_added{}; - uint32_t lines_deleted{}; - uint32_t hunks_added{}; - uint32_t hunks_removed{}; - uint32_t hunks_changed{}; - - void writeTextWithoutNewline(WriteBuffer & out) const - { - writeText(change_type, out); - writeChar('\t', out); - writeText(path, out); - writeChar('\t', out); - writeText(old_path, out); - writeChar('\t', out); - writeText(file_extension, out); - writeChar('\t', out); - writeText(lines_added, out); - writeChar('\t', out); - writeText(lines_deleted, out); - writeChar('\t', out); - writeText(hunks_added, out); - writeChar('\t', out); - writeText(hunks_removed, out); - writeChar('\t', out); - writeText(hunks_changed, out); - } -}; - -struct FileChangeAndLineChanges -{ - FileChange file_change; - LineChanges line_changes; -}; - -struct Commit -{ - std::string hash; - std::string author; - time_t time{}; - std::string message; - uint32_t files_added{}; - uint32_t files_deleted{}; - uint32_t files_renamed{}; - uint32_t files_modified{}; - uint32_t lines_added{}; - uint32_t lines_deleted{}; - uint32_t hunks_added{}; - uint32_t hunks_removed{}; - uint32_t hunks_changed{}; - - void writeTextWithoutNewline(WriteBuffer & out) const - { - writeText(hash, out); - writeChar('\t', out); - writeText(author, out); - writeChar('\t', out); - writeText(time, out); - writeChar('\t', out); - writeText(message, out); - writeChar('\t', out); - writeText(files_added, out); - writeChar('\t', out); - writeText(files_deleted, out); - writeChar('\t', out); - writeText(files_renamed, out); - writeChar('\t', out); - writeText(files_modified, out); - writeChar('\t', out); - writeText(lines_added, out); - writeChar('\t', out); - writeText(lines_deleted, out); - writeChar('\t', out); - writeText(hunks_added, out); - writeChar('\t', out); - writeText(hunks_removed, out); - writeChar('\t', out); - writeText(hunks_changed, out); - } -}; - void skipUntilWhitespace(ReadBuffer & buf) { @@ -407,13 +427,15 @@ struct Result struct Options { bool skip_commits_without_parents = true; + size_t threads = 1; std::optional skip_paths; std::unordered_set skip_commits; - size_t diff_size_limit = 0; + std::optional diff_size_limit; Options(const po::variables_map & options) { skip_commits_without_parents = options["skip-commits-without-parents"].as(); + threads = options["threads"].as(); if (options.count("skip-paths")) { skip_paths.emplace(options["skip-paths"].as()); @@ -423,36 +445,123 @@ struct Options auto vec = options["skip-commit"].as>(); skip_commits.insert(vec.begin(), vec.end()); } - diff_size_limit = options["diff-size-limit"].as(); + if (options.count("diff-size-limit")) + { + diff_size_limit = options["diff-size-limit"].as(); + } } }; /// Rough snapshot of repository calculated by application of diffs. It's used to calculate blame info. -struct File +struct FileBlame { - std::vector lines; + using Lines = std::list; + Lines lines; + Lines::iterator it; + size_t current_idx = 1; + + FileBlame() + { + it = lines.begin(); + } + + FileBlame & operator=(const FileBlame & rhs) + { + lines = rhs.lines; + it = lines.begin(); + current_idx = 1; + return *this; + } + + FileBlame(const FileBlame & rhs) + { + *this = rhs; + } + + void walk(uint32_t num) + { + if (current_idx < num) + { + while (current_idx < num && it != lines.end()) + { + ++current_idx; + ++it; + } + } + else if (current_idx > num) + { + --current_idx; + --it; + } + } + + const Commit * find(uint32_t num) + { + walk(num); + + if (current_idx == num && it != lines.end()) + return &*it; + return {}; + } + + void addLine(uint32_t num, Commit commit) + { + walk(num); + + while (it == lines.end() && current_idx < num) + { + lines.emplace_back(); + ++current_idx; + } + if (it == lines.end()) + { + lines.emplace_back(); + --it; + } + + lines.insert(it, commit); + } + + void removeLine(uint32_t num) + { + walk(num); + + if (current_idx == num) + it = lines.erase(it); + } }; -using Snapshot = std::map; +using Snapshot = std::map; + +struct FileChangeAndLineChanges +{ + FileChangeAndLineChanges(FileChange file_change_) : file_change(file_change_) {} + + FileChange file_change; + LineChanges line_changes; + + std::map deleted_lines; +}; void processCommit( - const Options & options, size_t commit_num, size_t total_commits, std::string hash, Snapshot & /*snapshot*/, Result & result) + std::unique_ptr & commit_info, + const Options & options, + size_t commit_num, + size_t total_commits, + std::string hash, + Snapshot & snapshot, + Result & result) { - std::string command = fmt::format( - "git show --raw --pretty='format:%at%x09%aN%x09%P%x0A%s%x00' --patch --unified=0 {}", - hash); - - //std::cerr << command << "\n"; - - auto commit_info = ShellCommand::execute(command); auto & in = commit_info->out; Commit commit; commit.hash = hash; - readText(commit.time, in); + time_t commit_time; + readText(commit_time, in); + commit.time = commit_time; assertChar('\t', in); readText(commit.author, in); assertChar('\t', in); @@ -465,7 +574,7 @@ void processCommit( std::replace_if(message_to_print.begin(), message_to_print.end(), [](char c){ return std::iscntrl(c); }, ' '); fmt::print("{}% {} {} {}\n", - commit_num * 100 / total_commits, toString(LocalDateTime(commit.time)), hash, message_to_print); + commit_num * 100 / total_commits, toString(commit.time), hash, message_to_print); if (options.skip_commits_without_parents && commit_num != 0 && parent_hash.empty()) { @@ -533,6 +642,8 @@ void processCommit( readText(file_change.old_path, in); skipWhitespaceIfAny(in); readText(file_change.path, in); + + snapshot[file_change.path] = snapshot[file_change.old_path]; } else { @@ -547,7 +658,7 @@ void processCommit( { file_changes.emplace( file_change.path, - FileChangeAndLineChanges{ file_change, {} }); + FileChangeAndLineChanges(file_change)); } } @@ -601,6 +712,9 @@ void processCommit( else assertChar('\n', in); + line_change.hunk_lines_added = new_lines; + line_change.hunk_lines_deleted = old_lines; + ++line_change.hunk_num; line_change.line_number_old = line_change.hunk_start_line_number_old; line_change.line_number_new = line_change.hunk_start_line_number_new; @@ -653,6 +767,16 @@ void processCommit( readStringUntilNextLine(line_change.line, in); line_change.setLineInfo(line_change.line); + FileBlame & file_snapshot = snapshot[old_file_path]; + if (const Commit * prev_commit = file_snapshot.find(line_change.line_number_old)) + { + line_change.prev_commit_hash = prev_commit->hash; + line_change.prev_author = prev_commit->author; + line_change.prev_time = prev_commit->time; + file_change_and_line_changes->deleted_lines[line_change.line_number_old] = *prev_commit; + file_snapshot.removeLine(line_change.line_number_old); + } + file_change_and_line_changes->line_changes.push_back(line_change); ++line_change.line_number_old; } @@ -689,6 +813,16 @@ void processCommit( readStringUntilNextLine(line_change.line, in); line_change.setLineInfo(line_change.line); + FileBlame & file_snapshot = snapshot[new_file_path]; + if (file_change_and_line_changes->deleted_lines.count(line_change.line_number_new)) + { + const auto & prev_commit = file_change_and_line_changes->deleted_lines[line_change.line_number_new]; + line_change.prev_commit_hash = prev_commit.hash; + line_change.prev_author = prev_commit.author; + line_change.prev_time = prev_commit.time; + } + file_snapshot.addLine(line_change.line_number_new, commit); + file_change_and_line_changes->line_changes.push_back(line_change); ++line_change.line_number_new; } @@ -701,7 +835,7 @@ void processCommit( } } - if (commit.lines_added + commit.lines_deleted > options.diff_size_limit) + if (options.diff_size_limit && commit.lines_added + commit.lines_deleted > *options.diff_size_limit) return; /// Write the result @@ -744,6 +878,16 @@ void processCommit( } +auto gitShow(const std::string & hash) +{ + std::string command = fmt::format( + "git show --raw --pretty='format:%at%x09%aN%x09%P%x0A%s%x00' --patch --unified=0 {}", + hash); + + return ShellCommand::execute(command); +} + + void processLog(const Options & options) { Result result; @@ -772,10 +916,19 @@ void processLog(const Options & options) size_t num_commits = hashes.size(); fmt::print("Total {} commits to process.\n", num_commits); + /// Will run multiple processes in parallel + size_t num_threads = options.threads; + + std::vector> show_commands(num_threads); + for (size_t i = 0; i < num_commits && i < num_threads; ++i) + show_commands[i] = gitShow(hashes[i]); + Snapshot snapshot; for (size_t i = 0; i < num_commits; ++i) { - processCommit(options, i, num_commits, hashes[i], snapshot, result); + processCommit(show_commands[i % num_threads], options, i, num_commits, hashes[i], snapshot, result); + if (i + num_threads < num_commits) + show_commands[i % num_threads] = gitShow(hashes[i + num_threads]); } } @@ -797,8 +950,10 @@ try "Skip paths that matches regular expression (re2 syntax).") ("skip-commit", po::value>(), "Skip commit with specified hash. The option can be specified multiple times.") - ("diff-size-limit", po::value()->default_value(0), + ("diff-size-limit", po::value(), "Skip commits whose diff size (number of added + removed lines) is larger than specified threshold") + ("threads", po::value()->default_value(std::thread::hardware_concurrency()), + "Number of threads to interact with git") ; po::variables_map options; From 3f29453c02ef3d3716927d81258218516b183d7b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 09:38:48 +0300 Subject: [PATCH 019/123] Roughly working blame --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 179 ++++++++++++++---- 1 file changed, 137 insertions(+), 42 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 6686c1ac480..c1c27a82812 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -13,6 +14,7 @@ #include #include +#include #include #include #include @@ -427,19 +429,26 @@ struct Result struct Options { bool skip_commits_without_parents = true; + bool skip_commits_with_duplicate_diffs = true; size_t threads = 1; std::optional skip_paths; + std::optional skip_commits_with_messages; std::unordered_set skip_commits; std::optional diff_size_limit; Options(const po::variables_map & options) { skip_commits_without_parents = options["skip-commits-without-parents"].as(); + skip_commits_with_duplicate_diffs = options["skip-commits-with-duplicate-diffs"].as(); threads = options["threads"].as(); if (options.count("skip-paths")) { skip_paths.emplace(options["skip-paths"].as()); } + if (options.count("skip-commits-with-messages")) + { + skip_commits_with_messages.emplace(options["skip-commits-with-messages"].as()); + } if (options.count("skip-commit")) { auto vec = options["skip-commit"].as>(); @@ -481,15 +490,12 @@ struct FileBlame void walk(uint32_t num) { - if (current_idx < num) + while (current_idx < num && it != lines.end()) { - while (current_idx < num && it != lines.end()) - { - ++current_idx; - ++it; - } + ++current_idx; + ++it; } - else if (current_idx > num) + while (current_idx > num) { --current_idx; --it; @@ -500,6 +506,8 @@ struct FileBlame { walk(num); +// std::cerr << "current_idx: " << current_idx << ", num: " << num << "\n"; + if (current_idx == num && it != lines.end()) return &*it; return {}; @@ -514,20 +522,17 @@ struct FileBlame lines.emplace_back(); ++current_idx; } - if (it == lines.end()) - { - lines.emplace_back(); - --it; - } - lines.insert(it, commit); + it = lines.insert(it, commit); } void removeLine(uint32_t num) { +// std::cerr << "Removing line " << num << ", current_idx: " << current_idx << "\n"; + walk(num); - if (current_idx == num) + if (current_idx == num && it != lines.end()) it = lines.erase(it); } }; @@ -540,10 +545,10 @@ struct FileChangeAndLineChanges FileChange file_change; LineChanges line_changes; - - std::map deleted_lines; }; +using DiffHashes = std::unordered_set; + void processCommit( std::unique_ptr & commit_info, @@ -552,6 +557,7 @@ void processCommit( size_t total_commits, std::string hash, Snapshot & snapshot, + DiffHashes & diff_hashes, Result & result) { auto & in = commit_info->out; @@ -570,6 +576,9 @@ void processCommit( assertChar('\n', in); readNullTerminated(commit.message, in); + if (options.skip_commits_with_messages && re2_st::RE2::PartialMatch(commit.message, *options.skip_commits_with_messages)) + return; + std::string message_to_print = commit.message; std::replace_if(message_to_print.begin(), message_to_print.end(), [](char c){ return std::iscntrl(c); }, ' '); @@ -643,7 +652,10 @@ void processCommit( skipWhitespaceIfAny(in); readText(file_change.path, in); - snapshot[file_change.path] = snapshot[file_change.old_path]; +// std::cerr << "Move from " << file_change.old_path << " to " << file_change.path << "\n"; + + if (file_change.path != file_change.old_path) + snapshot[file_change.path] = snapshot[file_change.old_path]; } else { @@ -706,6 +718,9 @@ void processCommit( if (checkChar(',', in)) readText(new_lines, in); + if (line_change.hunk_start_line_number_new == 0) + line_change.hunk_start_line_number_new = 1; + assertString(" @@", in); if (checkChar(' ', in)) readStringUntilNextLine(line_change.hunk_context, in); @@ -767,16 +782,6 @@ void processCommit( readStringUntilNextLine(line_change.line, in); line_change.setLineInfo(line_change.line); - FileBlame & file_snapshot = snapshot[old_file_path]; - if (const Commit * prev_commit = file_snapshot.find(line_change.line_number_old)) - { - line_change.prev_commit_hash = prev_commit->hash; - line_change.prev_author = prev_commit->author; - line_change.prev_time = prev_commit->time; - file_change_and_line_changes->deleted_lines[line_change.line_number_old] = *prev_commit; - file_snapshot.removeLine(line_change.line_number_old); - } - file_change_and_line_changes->line_changes.push_back(line_change); ++line_change.line_number_old; } @@ -813,16 +818,6 @@ void processCommit( readStringUntilNextLine(line_change.line, in); line_change.setLineInfo(line_change.line); - FileBlame & file_snapshot = snapshot[new_file_path]; - if (file_change_and_line_changes->deleted_lines.count(line_change.line_number_new)) - { - const auto & prev_commit = file_change_and_line_changes->deleted_lines[line_change.line_number_new]; - line_change.prev_commit_hash = prev_commit.hash; - line_change.prev_author = prev_commit.author; - line_change.prev_time = prev_commit.time; - } - file_snapshot.addLine(line_change.line_number_new, commit); - file_change_and_line_changes->line_changes.push_back(line_change); ++line_change.line_number_new; } @@ -838,6 +833,99 @@ void processCommit( if (options.diff_size_limit && commit.lines_added + commit.lines_deleted > *options.diff_size_limit) return; + /// Calculate hash of diff and skip duplicates + if (options.skip_commits_with_duplicate_diffs) + { + SipHash hasher; + + for (auto & elem : file_changes) + { + hasher.update(elem.second.file_change.change_type); + hasher.update(elem.second.file_change.old_path.size()); + hasher.update(elem.second.file_change.old_path); + hasher.update(elem.second.file_change.path.size()); + hasher.update(elem.second.file_change.path); + + hasher.update(elem.second.line_changes.size()); + for (auto & line_change : elem.second.line_changes) + { + hasher.update(line_change.sign); + hasher.update(line_change.line_number_old); + hasher.update(line_change.line_number_new); + hasher.update(line_change.indent); + hasher.update(line_change.line.size()); + hasher.update(line_change.line); + } + } + + UInt128 hash_of_diff; + hasher.get128(hash_of_diff.low, hash_of_diff.high); + + if (!diff_hashes.insert(hash_of_diff).second) + return; + } + + /// Update snapshot and blame info + + for (auto & elem : file_changes) + { +// std::cerr << elem.first << "\n"; + + FileBlame & file_snapshot = snapshot[elem.first]; + std::unordered_map deleted_lines; + + /// Obtain blame info from previous state of the snapshot + + for (auto & line_change : elem.second.line_changes) + { + if (line_change.sign == -1) + { + if (const Commit * prev_commit = file_snapshot.find(line_change.line_number_old); + prev_commit && prev_commit->time <= commit.time) + { + line_change.prev_commit_hash = prev_commit->hash; + line_change.prev_author = prev_commit->author; + line_change.prev_time = prev_commit->time; + deleted_lines[line_change.line_number_old] = *prev_commit; + } + else + { + // std::cerr << "Did not find line " << line_change.line_number_old << " from file " << elem.first << ": " << line_change.line << "\n"; + } + } + else if (line_change.sign == 1) + { + uint32_t this_line_in_prev_commit = line_change.hunk_start_line_number_old + + (line_change.line_number_new - line_change.hunk_start_line_number_new); + + if (deleted_lines.count(this_line_in_prev_commit)) + { + const auto & prev_commit = deleted_lines[this_line_in_prev_commit]; + if (prev_commit.time <= commit.time) + { + line_change.prev_commit_hash = prev_commit.hash; + line_change.prev_author = prev_commit.author; + line_change.prev_time = prev_commit.time; + } + } + } + } + + /// Update the snapshot + + for (const auto & line_change : elem.second.line_changes) + { + if (line_change.sign == -1) + { + file_snapshot.removeLine(line_change.line_number_new); + } + else if (line_change.sign == 1) + { + file_snapshot.addLine(line_change.line_number_new, commit); + } + } + } + /// Write the result /// commits table @@ -881,7 +969,7 @@ void processCommit( auto gitShow(const std::string & hash) { std::string command = fmt::format( - "git show --raw --pretty='format:%at%x09%aN%x09%P%x0A%s%x00' --patch --unified=0 {}", + "git show --raw --pretty='format:%ct%x09%aN%x09%P%x0A%s%x00' --patch --unified=0 {}", hash); return ShellCommand::execute(command); @@ -924,9 +1012,11 @@ void processLog(const Options & options) show_commands[i] = gitShow(hashes[i]); Snapshot snapshot; + DiffHashes diff_hashes; + for (size_t i = 0; i < num_commits; ++i) { - processCommit(show_commands[i % num_threads], options, i, num_commits, hashes[i], snapshot, result); + processCommit(show_commands[i % num_threads], options, i, num_commits, hashes[i], snapshot, diff_hashes, result); if (i + num_threads < num_commits) show_commands[i % num_threads] = gitShow(hashes[i + num_threads]); } @@ -946,10 +1036,15 @@ try ("skip-commits-without-parents", po::value()->default_value(true), "Skip commits without parents (except the initial commit)." " These commits are usually erroneous but they can make sense in very rare cases.") - ("skip-paths", po::value(), - "Skip paths that matches regular expression (re2 syntax).") + ("skip-commits-with-duplicate-diffs", po::value()->default_value(true), + "Skip commits with duplicate diffs." + " These commits are usually results of cherry-pick or merge after rebase.") ("skip-commit", po::value>(), "Skip commit with specified hash. The option can be specified multiple times.") + ("skip-paths", po::value(), + "Skip paths that matches regular expression (re2 syntax).") + ("skip-commits-with-messages", po::value(), + "Skip commits whose messages matches regular expression (re2 syntax).") ("diff-size-limit", po::value(), "Skip commits whose diff size (number of added + removed lines) is larger than specified threshold") ("threads", po::value()->default_value(std::thread::hardware_concurrency()), @@ -965,7 +1060,7 @@ try << "Usage: " << argv[0] << '\n' << desc << '\n' << "\nExample:\n" - << "\n./git-to-clickhouse --diff-size-limit 100000 --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/'\n"; + << "\n./git-to-clickhouse --diff-size-limit 100000 --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/' --skip-commits-with-messages '^Merge branch '\n"; return 1; } From 99c33612d65c627bbb9fc31d9d97906195d3cf53 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 10:29:58 +0300 Subject: [PATCH 020/123] Better diagnostics --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index c1c27a82812..6b29708ead3 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -582,7 +582,7 @@ void processCommit( std::string message_to_print = commit.message; std::replace_if(message_to_print.begin(), message_to_print.end(), [](char c){ return std::iscntrl(c); }, ' '); - fmt::print("{}% {} {} {}\n", + std::cerr << fmt::format("{}% {} {} {}\n", commit_num * 100 / total_commits, toString(commit.time), hash, message_to_print); if (options.skip_commits_without_parents && commit_num != 0 && parent_hash.empty()) From 3ec9656aa21a3142d2898b7d259a4740a6691fd2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Sep 2020 10:38:39 +0300 Subject: [PATCH 021/123] Slightly more robust --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 6b29708ead3..f3653bb282f 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -568,12 +568,10 @@ void processCommit( time_t commit_time; readText(commit_time, in); commit.time = commit_time; - assertChar('\t', in); - readText(commit.author, in); - assertChar('\t', in); + assertChar('\0', in); + readNullTerminated(commit.author, in); std::string parent_hash; - readString(parent_hash, in); - assertChar('\n', in); + readNullTerminated(parent_hash, in); readNullTerminated(commit.message, in); if (options.skip_commits_with_messages && re2_st::RE2::PartialMatch(commit.message, *options.skip_commits_with_messages)) @@ -969,7 +967,7 @@ void processCommit( auto gitShow(const std::string & hash) { std::string command = fmt::format( - "git show --raw --pretty='format:%ct%x09%aN%x09%P%x0A%s%x00' --patch --unified=0 {}", + "git show --raw --pretty='format:%ct%x00%aN%x00%P%x00%s%x00' --patch --unified=0 {}", hash); return ShellCommand::execute(command); From 25ca5e91bd0f3074c8d7d0874e125d2dcc611889 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 16:09:32 -0700 Subject: [PATCH 022/123] 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 023/123] 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 024/123] 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 025/123] 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 026/123] 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 db58fa15aaf202318e043549440589797b51aa0a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 02:24:31 +0300 Subject: [PATCH 027/123] Some tweaks --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index f3653bb282f..9e1ef14fcbf 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -435,6 +435,7 @@ struct Options std::optional skip_commits_with_messages; std::unordered_set skip_commits; std::optional diff_size_limit; + std::string stop_after_commit; Options(const po::variables_map & options) { @@ -458,6 +459,10 @@ struct Options { diff_size_limit = options["diff-size-limit"].as(); } + if (options.count("stop-after-commit")) + { + stop_after_commit = options["stop-after-commit"].as(); + } } }; @@ -828,7 +833,7 @@ void processCommit( } } - if (options.diff_size_limit && commit.lines_added + commit.lines_deleted > *options.diff_size_limit) + if (options.diff_size_limit && commit_num != 0 && commit.lines_added + commit.lines_deleted > *options.diff_size_limit) return; /// Calculate hash of diff and skip duplicates @@ -1015,6 +1020,10 @@ void processLog(const Options & options) for (size_t i = 0; i < num_commits; ++i) { processCommit(show_commands[i % num_threads], options, i, num_commits, hashes[i], snapshot, diff_hashes, result); + + if (!options.stop_after_commit.empty() && hashes[i] == options.stop_after_commit) + break; + if (i + num_threads < num_commits) show_commands[i % num_threads] = gitShow(hashes[i + num_threads]); } @@ -1043,10 +1052,12 @@ try "Skip paths that matches regular expression (re2 syntax).") ("skip-commits-with-messages", po::value(), "Skip commits whose messages matches regular expression (re2 syntax).") - ("diff-size-limit", po::value(), - "Skip commits whose diff size (number of added + removed lines) is larger than specified threshold") + ("diff-size-limit", po::value()->default_value(100000), + "Skip commits whose diff size (number of added + removed lines) is larger than specified threshold. Does not apply for initial commit.") + ("stop-after-commit", po::value(), + "Stop processing after specified commit hash.") ("threads", po::value()->default_value(std::thread::hardware_concurrency()), - "Number of threads to interact with git") + "Number of concurrent git subprocesses to spawn") ; po::variables_map options; @@ -1058,7 +1069,7 @@ try << "Usage: " << argv[0] << '\n' << desc << '\n' << "\nExample:\n" - << "\n./git-to-clickhouse --diff-size-limit 100000 --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/' --skip-commits-with-messages '^Merge branch '\n"; + << "\n./git-to-clickhouse --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/' --skip-commits-with-messages '^Merge branch '\n"; return 1; } From 684a910395cc37203453d1faa09ab839d3a4f32a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 03:17:26 +0300 Subject: [PATCH 028/123] Polish --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 803 ++++++++++-------- 1 file changed, 451 insertions(+), 352 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 9e1ef14fcbf..6e43853d6ba 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -277,10 +277,14 @@ struct LineChange std::string line; /// Line content without leading whitespaces uint8_t indent{}; /// The number of leading whitespaces or tabs * 4 LineType line_type{}; + /// Information from the history (blame). std::string prev_commit_hash; std::string prev_author; LocalDateTime prev_time{}; + /** Classify line to empty / code / comment / single punctuation char. + * Very rough and mostly suitable for our C++ style. + */ void setLineInfo(std::string full_line) { indent = 0; @@ -306,8 +310,9 @@ struct LineChange line_type = LineType::Empty; } else if (pos + 1 < end - && ((pos[0] == '/' && pos[1] == '/') - || (pos[0] == '*' && pos[1] == ' '))) /// This is not precise. + && ((pos[0] == '/' && (pos[1] == '/' || pos[1] == '*')) + || (pos[0] == '*' && pos[1] == ' ') /// This is not precise. + || (pos[0] == '#' && pos[1] == ' '))) { line_type = LineType::Comment; } @@ -363,6 +368,18 @@ struct LineChange using LineChanges = std::vector; +struct FileDiff +{ + FileDiff(FileChange file_change_) : file_change(file_change_) {} + + FileChange file_change; + LineChanges line_changes; +}; + +using CommitDiff = std::map; + + +/** Parsing helpers */ void skipUntilWhitespace(ReadBuffer & buf) { @@ -418,14 +435,57 @@ void readStringUntilNextLine(std::string & s, ReadBuffer & buf) } -struct Result +/** Writes the resulting tables to files that can be imported to ClickHouse. + */ +struct ResultWriter { WriteBufferFromFile commits{"commits.tsv"}; WriteBufferFromFile file_changes{"file_changes.tsv"}; WriteBufferFromFile line_changes{"line_changes.tsv"}; + + void appendCommit(const Commit & commit, const CommitDiff & files) + { + /// commits table + { + auto & out = commits; + + commit.writeTextWithoutNewline(out); + writeChar('\n', out); + } + + for (const auto & elem : files) + { + const FileChange & file_change = elem.second.file_change; + + /// file_changes table + { + auto & out = file_changes; + + file_change.writeTextWithoutNewline(out); + writeChar('\t', out); + commit.writeTextWithoutNewline(out); + writeChar('\n', out); + } + + /// line_changes table + for (const auto & line_change : elem.second.line_changes) + { + auto & out = line_changes; + + line_change.writeTextWithoutNewline(out); + writeChar('\t', out); + file_change.writeTextWithoutNewline(out); + writeChar('\t', out); + commit.writeTextWithoutNewline(out); + writeChar('\n', out); + } + } + } }; +/** See description in "main". + */ struct Options { bool skip_commits_without_parents = true; @@ -467,11 +527,23 @@ struct Options }; -/// Rough snapshot of repository calculated by application of diffs. It's used to calculate blame info. +/** Rough snapshot of repository calculated by application of diffs. It's used to calculate blame info. + * Represented by a list of lines. For every line it contains information about commit that modified this line the last time. + * + * Note that there are many cases when this info may become incorrect. + * The first reason is that git history is non-linear but we form this snapshot by application of commit diffs in some order + * that cannot give us correct results even theoretically. + * The second reason is that we don't process merge commits. But merge commits may contain differences for conflict resolution. + * + * We expect that the information will be mostly correct for the purpose of analytics. + * So, it can provide the expected "blame" info for the most of the lines. + */ struct FileBlame { using Lines = std::list; Lines lines; + + /// We walk through this list adding or removing lines. Lines::iterator it; size_t current_idx = 1; @@ -480,6 +552,7 @@ struct FileBlame it = lines.begin(); } + /// This is important when file was copied or renamed. FileBlame & operator=(const FileBlame & rhs) { lines = rhs.lines; @@ -493,6 +566,7 @@ struct FileBlame *this = rhs; } + /// Move iterator to requested line or stop at the end. void walk(uint32_t num) { while (current_idx < num && it != lines.end()) @@ -522,6 +596,7 @@ struct FileBlame { walk(num); + /// If the inserted line is over the end of file, we insert empty lines before it. while (it == lines.end() && current_idx < num) { lines.emplace_back(); @@ -542,334 +617,24 @@ struct FileBlame } }; +/// All files with their blame info. When file is renamed, we also rename it in snapshot. using Snapshot = std::map; -struct FileChangeAndLineChanges + +/** Enrich the line changes data with the history info from the snapshot + * - the author, time and commit of the previous change to every found line (blame). + * And update the snapshot. + */ +void updateSnapshot(Snapshot & snapshot, const Commit & commit, CommitDiff & file_changes) { - FileChangeAndLineChanges(FileChange file_change_) : file_change(file_change_) {} - - FileChange file_change; - LineChanges line_changes; -}; - -using DiffHashes = std::unordered_set; - - -void processCommit( - std::unique_ptr & commit_info, - const Options & options, - size_t commit_num, - size_t total_commits, - std::string hash, - Snapshot & snapshot, - DiffHashes & diff_hashes, - Result & result) -{ - auto & in = commit_info->out; - - Commit commit; - commit.hash = hash; - - time_t commit_time; - readText(commit_time, in); - commit.time = commit_time; - assertChar('\0', in); - readNullTerminated(commit.author, in); - std::string parent_hash; - readNullTerminated(parent_hash, in); - readNullTerminated(commit.message, in); - - if (options.skip_commits_with_messages && re2_st::RE2::PartialMatch(commit.message, *options.skip_commits_with_messages)) - return; - - std::string message_to_print = commit.message; - std::replace_if(message_to_print.begin(), message_to_print.end(), [](char c){ return std::iscntrl(c); }, ' '); - - std::cerr << fmt::format("{}% {} {} {}\n", - commit_num * 100 / total_commits, toString(commit.time), hash, message_to_print); - - if (options.skip_commits_without_parents && commit_num != 0 && parent_hash.empty()) + /// Renames and copies. + for (auto & elem : file_changes) { - std::cerr << "Warning: skipping commit without parents\n"; - return; + auto & file = elem.second.file_change; + if (file.path != file.old_path) + snapshot[file.path] = snapshot[file.old_path]; } - if (!in.eof()) - assertChar('\n', in); - - /// File changes in form - /// :100644 100644 b90fe6bb94 3ffe4c380f M src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp - /// :100644 100644 828dedf6b5 828dedf6b5 R100 dbms/src/Functions/GeoUtils.h dbms/src/Functions/PolygonUtils.h - - std::map file_changes; - - while (checkChar(':', in)) - { - FileChange file_change; - - for (size_t i = 0; i < 4; ++i) - { - skipUntilWhitespace(in); - skipWhitespaceIfAny(in); - } - - char change_type; - readChar(change_type, in); - - int confidence; - switch (change_type) - { - case 'A': - file_change.change_type = FileChangeType::Add; - ++commit.files_added; - break; - case 'D': - file_change.change_type = FileChangeType::Delete; - ++commit.files_deleted; - break; - case 'M': - file_change.change_type = FileChangeType::Modify; - ++commit.files_modified; - break; - case 'R': - file_change.change_type = FileChangeType::Rename; - ++commit.files_renamed; - readText(confidence, in); - break; - case 'C': - file_change.change_type = FileChangeType::Copy; - readText(confidence, in); - break; - case 'T': - file_change.change_type = FileChangeType::Type; - break; - default: - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected file change type: {}", change_type); - } - - skipWhitespaceIfAny(in); - - if (change_type == 'R' || change_type == 'C') - { - readText(file_change.old_path, in); - skipWhitespaceIfAny(in); - readText(file_change.path, in); - -// std::cerr << "Move from " << file_change.old_path << " to " << file_change.path << "\n"; - - if (file_change.path != file_change.old_path) - snapshot[file_change.path] = snapshot[file_change.old_path]; - } - else - { - readText(file_change.path, in); - } - - file_change.file_extension = std::filesystem::path(file_change.path).extension(); - - assertChar('\n', in); - - if (!(options.skip_paths && re2_st::RE2::PartialMatch(file_change.path, *options.skip_paths))) - { - file_changes.emplace( - file_change.path, - FileChangeAndLineChanges(file_change)); - } - } - - if (!in.eof()) - { - assertChar('\n', in); - - /// Diffs for every file in form of - /// --- a/src/Storages/StorageReplicatedMergeTree.cpp - /// +++ b/src/Storages/StorageReplicatedMergeTree.cpp - /// @@ -1387,2 +1387 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) - /// - table_lock, entry.create_time, reserved_space, entry.deduplicate, - /// - entry.force_ttl); - /// + table_lock, entry.create_time, reserved_space, entry.deduplicate); - - std::string old_file_path; - std::string new_file_path; - FileChangeAndLineChanges * file_change_and_line_changes = nullptr; - LineChange line_change; - - while (!in.eof()) - { - if (checkString("@@ ", in)) - { - if (!file_change_and_line_changes) - { - auto file_name = new_file_path.empty() ? old_file_path : new_file_path; - auto it = file_changes.find(file_name); - if (file_changes.end() != it) - file_change_and_line_changes = &it->second; - } - - if (file_change_and_line_changes) - { - uint32_t old_lines = 1; - uint32_t new_lines = 1; - - assertChar('-', in); - readText(line_change.hunk_start_line_number_old, in); - if (checkChar(',', in)) - readText(old_lines, in); - - assertString(" +", in); - readText(line_change.hunk_start_line_number_new, in); - if (checkChar(',', in)) - readText(new_lines, in); - - if (line_change.hunk_start_line_number_new == 0) - line_change.hunk_start_line_number_new = 1; - - assertString(" @@", in); - if (checkChar(' ', in)) - readStringUntilNextLine(line_change.hunk_context, in); - else - assertChar('\n', in); - - line_change.hunk_lines_added = new_lines; - line_change.hunk_lines_deleted = old_lines; - - ++line_change.hunk_num; - line_change.line_number_old = line_change.hunk_start_line_number_old; - line_change.line_number_new = line_change.hunk_start_line_number_new; - - if (old_lines && new_lines) - { - ++commit.hunks_changed; - ++file_change_and_line_changes->file_change.hunks_changed; - } - else if (old_lines) - { - ++commit.hunks_removed; - ++file_change_and_line_changes->file_change.hunks_removed; - } - else if (new_lines) - { - ++commit.hunks_added; - ++file_change_and_line_changes->file_change.hunks_added; - } - } - } - else if (checkChar('-', in)) - { - if (checkString("-- ", in)) - { - if (checkString("a/", in)) - { - readStringUntilNextLine(old_file_path, in); - line_change = LineChange{}; - file_change_and_line_changes = nullptr; - } - else if (checkString("/dev/null", in)) - { - old_file_path.clear(); - assertChar('\n', in); - line_change = LineChange{}; - file_change_and_line_changes = nullptr; - } - else - skipUntilNextLine(in); /// Actually it can be the line in diff. Skip it for simplicity. - } - else - { - if (file_change_and_line_changes) - { - ++commit.lines_deleted; - ++file_change_and_line_changes->file_change.lines_deleted; - - line_change.sign = -1; - readStringUntilNextLine(line_change.line, in); - line_change.setLineInfo(line_change.line); - - file_change_and_line_changes->line_changes.push_back(line_change); - ++line_change.line_number_old; - } - } - } - else if (checkChar('+', in)) - { - if (checkString("++ ", in)) - { - if (checkString("b/", in)) - { - readStringUntilNextLine(new_file_path, in); - line_change = LineChange{}; - file_change_and_line_changes = nullptr; - } - else if (checkString("/dev/null", in)) - { - new_file_path.clear(); - assertChar('\n', in); - line_change = LineChange{}; - file_change_and_line_changes = nullptr; - } - else - skipUntilNextLine(in); /// Actually it can be the line in diff. Skip it for simplicity. - } - else - { - if (file_change_and_line_changes) - { - ++commit.lines_added; - ++file_change_and_line_changes->file_change.lines_added; - - line_change.sign = 1; - readStringUntilNextLine(line_change.line, in); - line_change.setLineInfo(line_change.line); - - file_change_and_line_changes->line_changes.push_back(line_change); - ++line_change.line_number_new; - } - } - } - else - { - skipUntilNextLine(in); - } - } - } - - if (options.diff_size_limit && commit_num != 0 && commit.lines_added + commit.lines_deleted > *options.diff_size_limit) - return; - - /// Calculate hash of diff and skip duplicates - if (options.skip_commits_with_duplicate_diffs) - { - SipHash hasher; - - for (auto & elem : file_changes) - { - hasher.update(elem.second.file_change.change_type); - hasher.update(elem.second.file_change.old_path.size()); - hasher.update(elem.second.file_change.old_path); - hasher.update(elem.second.file_change.path.size()); - hasher.update(elem.second.file_change.path); - - hasher.update(elem.second.line_changes.size()); - for (auto & line_change : elem.second.line_changes) - { - hasher.update(line_change.sign); - hasher.update(line_change.line_number_old); - hasher.update(line_change.line_number_new); - hasher.update(line_change.indent); - hasher.update(line_change.line.size()); - hasher.update(line_change.line); - } - } - - UInt128 hash_of_diff; - hasher.get128(hash_of_diff.low, hash_of_diff.high); - - if (!diff_hashes.insert(hash_of_diff).second) - return; - } - - /// Update snapshot and blame info - for (auto & elem : file_changes) { // std::cerr << elem.first << "\n"; @@ -928,47 +693,379 @@ void processCommit( } } } +} - /// Write the result - /// commits table +/** Deduplication of commits with identical diffs. + */ +using DiffHashes = std::unordered_set; + +UInt128 diffHash(const CommitDiff & file_changes) +{ + SipHash hasher; + + for (auto & elem : file_changes) { - auto & out = result.commits; + hasher.update(elem.second.file_change.change_type); + hasher.update(elem.second.file_change.old_path.size()); + hasher.update(elem.second.file_change.old_path); + hasher.update(elem.second.file_change.path.size()); + hasher.update(elem.second.file_change.path); - commit.writeTextWithoutNewline(out); - writeChar('\n', out); + hasher.update(elem.second.line_changes.size()); + for (auto & line_change : elem.second.line_changes) + { + hasher.update(line_change.sign); + hasher.update(line_change.line_number_old); + hasher.update(line_change.line_number_new); + hasher.update(line_change.indent); + hasher.update(line_change.line.size()); + hasher.update(line_change.line); + } } - for (const auto & elem : file_changes) + UInt128 hash_of_diff; + hasher.get128(hash_of_diff.low, hash_of_diff.high); + + return hash_of_diff; +} + + +/** File changes in form + * :100644 100644 b90fe6bb94 3ffe4c380f M src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp + * :100644 100644 828dedf6b5 828dedf6b5 R100 dbms/src/Functions/GeoUtils.h dbms/src/Functions/PolygonUtils.h + * according to the output of 'git show --raw' + */ +void processFileChanges( + ReadBuffer & in, + const Options & options, + Commit & commit, + CommitDiff & file_changes) +{ + while (checkChar(':', in)) { - const FileChange & file_change = elem.second.file_change; + FileChange file_change; - /// file_changes table + /// We don't care about file mode and content hashes. + for (size_t i = 0; i < 4; ++i) { - auto & out = result.file_changes; - - file_change.writeTextWithoutNewline(out); - writeChar('\t', out); - commit.writeTextWithoutNewline(out); - writeChar('\n', out); + skipUntilWhitespace(in); + skipWhitespaceIfAny(in); } - /// line_changes table - for (const auto & line_change : elem.second.line_changes) - { - auto & out = result.line_changes; + char change_type; + readChar(change_type, in); - line_change.writeTextWithoutNewline(out); - writeChar('\t', out); - file_change.writeTextWithoutNewline(out); - writeChar('\t', out); - commit.writeTextWithoutNewline(out); - writeChar('\n', out); + /// For rename and copy there is a number called "score". We ignore it. + int score; + + switch (change_type) + { + case 'A': + file_change.change_type = FileChangeType::Add; + ++commit.files_added; + break; + case 'D': + file_change.change_type = FileChangeType::Delete; + ++commit.files_deleted; + break; + case 'M': + file_change.change_type = FileChangeType::Modify; + ++commit.files_modified; + break; + case 'R': + file_change.change_type = FileChangeType::Rename; + ++commit.files_renamed; + readText(score, in); + break; + case 'C': + file_change.change_type = FileChangeType::Copy; + readText(score, in); + break; + case 'T': + file_change.change_type = FileChangeType::Type; + break; + default: + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected file change type: {}", change_type); + } + + skipWhitespaceIfAny(in); + + if (change_type == 'R' || change_type == 'C') + { + readText(file_change.old_path, in); + skipWhitespaceIfAny(in); + readText(file_change.path, in); + } + else + { + readText(file_change.path, in); + } + + file_change.file_extension = std::filesystem::path(file_change.path).extension(); + /// It gives us extension in form of '.cpp'. There is a reason for it but we remove initial dot for simplicity. + if (!file_change.file_extension.empty() && file_change.file_extension.front() == '.') + file_change.file_extension = file_change.file_extension.substr(1, std::string::npos); + + assertChar('\n', in); + + if (!(options.skip_paths && re2_st::RE2::PartialMatch(file_change.path, *options.skip_paths))) + { + file_changes.emplace( + file_change.path, + FileDiff(file_change)); } } } +/** Process the list of diffs for every file from the result of "git show". + * Caveats: + * - changes in binary files can be ignored; + * - if a line content begins with '+' or '-' it will be skipped + * it means that if you store diffs in repository and "git show" will display diff-of-diff for you, + * it won't be processed correctly; + * - we expect some specific format of the diff; but it may actually depend on git config; + * - non-ASCII file names are not processed correctly (they will not be found and will be ignored). + */ +void processDiffs( + ReadBuffer & in, + std::optional size_limit, + Commit & commit, + CommitDiff & file_changes) +{ + std::string old_file_path; + std::string new_file_path; + FileDiff * file_change_and_line_changes = nullptr; + LineChange line_change; + + /// Diffs for every file in form of + /// --- a/src/Storages/StorageReplicatedMergeTree.cpp + /// +++ b/src/Storages/StorageReplicatedMergeTree.cpp + /// @@ -1387,2 +1387 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) + /// - table_lock, entry.create_time, reserved_space, entry.deduplicate, + /// - entry.force_ttl); + /// + table_lock, entry.create_time, reserved_space, entry.deduplicate); + + size_t diff_size = 0; + while (!in.eof()) + { + if (checkString("@@ ", in)) + { + if (!file_change_and_line_changes) + { + auto file_name = new_file_path.empty() ? old_file_path : new_file_path; + auto it = file_changes.find(file_name); + if (file_changes.end() != it) + file_change_and_line_changes = &it->second; + } + + if (file_change_and_line_changes) + { + uint32_t old_lines = 1; + uint32_t new_lines = 1; + + assertChar('-', in); + readText(line_change.hunk_start_line_number_old, in); + if (checkChar(',', in)) + readText(old_lines, in); + + assertString(" +", in); + readText(line_change.hunk_start_line_number_new, in); + if (checkChar(',', in)) + readText(new_lines, in); + + /// This is needed to simplify the logic of updating snapshot: + /// When all lines are removed we can treat it as repeated removal of line with number 1. + if (line_change.hunk_start_line_number_new == 0) + line_change.hunk_start_line_number_new = 1; + + assertString(" @@", in); + if (checkChar(' ', in)) + readStringUntilNextLine(line_change.hunk_context, in); + else + assertChar('\n', in); + + line_change.hunk_lines_added = new_lines; + line_change.hunk_lines_deleted = old_lines; + + ++line_change.hunk_num; + line_change.line_number_old = line_change.hunk_start_line_number_old; + line_change.line_number_new = line_change.hunk_start_line_number_new; + + if (old_lines && new_lines) + { + ++commit.hunks_changed; + ++file_change_and_line_changes->file_change.hunks_changed; + } + else if (old_lines) + { + ++commit.hunks_removed; + ++file_change_and_line_changes->file_change.hunks_removed; + } + else if (new_lines) + { + ++commit.hunks_added; + ++file_change_and_line_changes->file_change.hunks_added; + } + } + } + else if (checkChar('-', in)) + { + if (checkString("-- ", in)) + { + if (checkString("a/", in)) + { + readStringUntilNextLine(old_file_path, in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else if (checkString("/dev/null", in)) + { + old_file_path.clear(); + assertChar('\n', in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else + skipUntilNextLine(in); /// Actually it can be the line in diff. Skip it for simplicity. + } + else + { + ++diff_size; + if (file_change_and_line_changes) + { + ++commit.lines_deleted; + ++file_change_and_line_changes->file_change.lines_deleted; + + line_change.sign = -1; + readStringUntilNextLine(line_change.line, in); + line_change.setLineInfo(line_change.line); + + file_change_and_line_changes->line_changes.push_back(line_change); + ++line_change.line_number_old; + } + } + } + else if (checkChar('+', in)) + { + if (checkString("++ ", in)) + { + if (checkString("b/", in)) + { + readStringUntilNextLine(new_file_path, in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else if (checkString("/dev/null", in)) + { + new_file_path.clear(); + assertChar('\n', in); + line_change = LineChange{}; + file_change_and_line_changes = nullptr; + } + else + skipUntilNextLine(in); /// Actually it can be the line in diff. Skip it for simplicity. + } + else + { + ++diff_size; + if (file_change_and_line_changes) + { + ++commit.lines_added; + ++file_change_and_line_changes->file_change.lines_added; + + line_change.sign = 1; + readStringUntilNextLine(line_change.line, in); + line_change.setLineInfo(line_change.line); + + file_change_and_line_changes->line_changes.push_back(line_change); + ++line_change.line_number_new; + } + } + } + else + { + /// Unknown lines are ignored. + skipUntilNextLine(in); + } + + if (size_limit && diff_size > *size_limit) + return; + } +} + + +/** Process the "git show" result for a single commit. Append the result to tables. + */ +void processCommit( + ReadBuffer & in, + const Options & options, + size_t commit_num, + size_t total_commits, + std::string hash, + Snapshot & snapshot, + DiffHashes & diff_hashes, + ResultWriter & result) +{ + Commit commit; + commit.hash = hash; + + time_t commit_time; + readText(commit_time, in); + commit.time = commit_time; + assertChar('\0', in); + readNullTerminated(commit.author, in); + std::string parent_hash; + readNullTerminated(parent_hash, in); + readNullTerminated(commit.message, in); + + if (options.skip_commits_with_messages && re2_st::RE2::PartialMatch(commit.message, *options.skip_commits_with_messages)) + return; + + std::string message_to_print = commit.message; + std::replace_if(message_to_print.begin(), message_to_print.end(), [](char c){ return std::iscntrl(c); }, ' '); + + std::cerr << fmt::format("{}% {} {} {}\n", + commit_num * 100 / total_commits, toString(commit.time), hash, message_to_print); + + if (options.skip_commits_without_parents && commit_num != 0 && parent_hash.empty()) + { + std::cerr << "Warning: skipping commit without parents\n"; + return; + } + + if (!in.eof()) + assertChar('\n', in); + + CommitDiff file_changes; + processFileChanges(in, options, commit, file_changes); + + if (!in.eof()) + { + assertChar('\n', in); + processDiffs(in, commit_num != 0 ? options.diff_size_limit : std::nullopt, commit, file_changes); + } + + /// Skip commits with too large diffs. + if (options.diff_size_limit && commit_num != 0 && commit.lines_added + commit.lines_deleted > *options.diff_size_limit) + return; + + /// Calculate hash of diff and skip duplicates + if (options.skip_commits_with_duplicate_diffs && !diff_hashes.insert(diffHash(file_changes)).second) + return; + + /// Update snapshot and blame info + updateSnapshot(snapshot, commit, file_changes); + + /// Write the result + result.appendCommit(commit, file_changes); +} + + +/** Runs child process and allows to read the result. + * Multiple processes can be run for parallel processing. + */ auto gitShow(const std::string & hash) { std::string command = fmt::format( @@ -979,9 +1076,11 @@ auto gitShow(const std::string & hash) } +/** Obtain the list of commits and process them. + */ void processLog(const Options & options) { - Result result; + ResultWriter result; std::string command = "git log --reverse --no-merges --pretty=%H"; fmt::print("{}\n", command); @@ -1019,7 +1118,7 @@ void processLog(const Options & options) for (size_t i = 0; i < num_commits; ++i) { - processCommit(show_commands[i % num_threads], options, i, num_commits, hashes[i], snapshot, diff_hashes, result); + processCommit(show_commands[i % num_threads]->out, options, i, num_commits, hashes[i], snapshot, diff_hashes, result); if (!options.stop_after_commit.empty() && hashes[i] == options.stop_after_commit) break; From 94d49e4197b443a6bced0ac0d137ad646c1c1946 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 03:18:15 +0300 Subject: [PATCH 029/123] Minor modifications --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 6e43853d6ba..2add6813008 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -120,7 +120,9 @@ CREATE TABLE git.line_changes commit_hunks_changed UInt32 ) ENGINE = MergeTree ORDER BY time; -Insert the data with the following commands: +Run the tool. + +Then insert the data with the following commands: clickhouse-client --query "INSERT INTO git.commits FORMAT TSV" < commits.tsv clickhouse-client --query "INSERT INTO git.file_changes FORMAT TSV" < file_changes.tsv From 47ca6211604c6fcb7b2c4e137d739ebff88da975 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 03:25:06 +0300 Subject: [PATCH 030/123] Minor modifications --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 2add6813008..875da3ba0ac 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -993,7 +993,12 @@ void processDiffs( } if (size_limit && diff_size > *size_limit) + { + /// Drain to avoid "broken pipe" error in child process. + while (!in.eof()) + in.ignore(in.available()); return; + } } } From 6e0afbecf4fd0ccd04e9dbb82bff6a507545e8d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 04:02:35 +0300 Subject: [PATCH 031/123] Minor modifications --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 875da3ba0ac..b5488b0d69a 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -138,6 +138,7 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_DATA; + extern const int CHILD_WAS_NOT_EXITED_NORMALLY; } @@ -994,9 +995,6 @@ void processDiffs( if (size_limit && diff_size > *size_limit) { - /// Drain to avoid "broken pipe" error in child process. - while (!in.eof()) - in.ignore(in.available()); return; } } @@ -1127,6 +1125,19 @@ void processLog(const Options & options) { processCommit(show_commands[i % num_threads]->out, options, i, num_commits, hashes[i], snapshot, diff_hashes, result); + try + { + show_commands[i % num_threads]->wait(); + } + catch (const Exception & e) + { + /// For broken pipe when we stopped reading prematurally. + if (e.code() == ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY) + std::cerr << getCurrentExceptionMessage(false) << "\n"; + else + throw; + } + if (!options.stop_after_commit.empty() && hashes[i] == options.stop_after_commit) break; From 69ce9e1f7020df985d7ea6ee450bf0d4b3438a0d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 05:36:54 +0300 Subject: [PATCH 032/123] More documentation --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index b5488b0d69a..d3b6f77d3d7 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -27,6 +27,51 @@ static constexpr auto documentation = R"( +A tool to extract information from Git repository for analytics. + +It dumps the data for the following tables: +- commits - commits with statistics; +- file_changes - files changed in every commit with the info about the change and statistics; +- line_changes - every changed line in every changed file in every commit with full info about the line and the information about previous change of this line. + +The largest and the most important table is "line_changes". + +Allows to answer questions like: +- list files with maximum number of authors; +- show me the oldest lines of code in the repository; +- show me the files with longest history; +- list favorite files for author; +- list largest files with lowest number of authors; +- at what weekday the code has highest chance to stay in repository; +- the distribution of code age across repository; +- files sorted by average code age; +- quickly show file with blame info (rough); +- commits and lines of code distribution by time; by weekday, by author; for specific subdirectories; +- show history for every subdirectory, file, line of file, the number of changes (lines and commits) across time; how the number of contributors was changed across time; +- list files with most modifications; +- list files that were rewritten most number of time or by most of authors; +- what is percentage of code removal by other authors, across authors; +- the matrix of authors that shows what authors tends to rewrite another authors code; +- what is the worst time to write code in sense that the code has highest chance to be rewritten; +- the average time before code will be rewritten and the median (half-life of code decay); +- comments/code percentage change in time / by author / by location; +- who tend to write more tests / cpp code / comments. + +The data is intended for analytical purposes. It can be imprecise by many reasons but it should be good enough for its purpose. + +The data is not intended to provide any conclusions for managers, it is especially counter-indicative for any kinds of "performance review". Instead you can spend multiple days looking at various interesting statistics. + +Run this tool inside your git repository. It will create .tsv files that can be loaded into ClickHouse (or into other DBMS if you dare). + +The tool can process large enough repositories in a reasonable time. +It has been tested on: +- ClickHouse: 31 seconds; 3 million rows; +- LLVM: 8 minues; 62 million rows; +- Linux - 12 minutes; 85 million rows; +- Chromium - 67 minutes; 343 million rows; +(the numbers as of Sep 2020) + + Prepare the database by executing the following queries: DROP DATABASE IF EXISTS git; From 1dc48f66710c5a93e5376320ea7cf3c4a18046d5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 05:39:08 +0300 Subject: [PATCH 033/123] Better help --- utils/git-to-clickhouse/git-to-clickhouse.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index d3b6f77d3d7..6ef82ac3b6b 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -13,6 +13,7 @@ #include +#include #include #include #include @@ -1199,7 +1200,7 @@ try { using namespace DB; - po::options_description desc("Allowed options"); + po::options_description desc("Allowed options", getTerminalWidth()); desc.add_options() ("help,h", "produce help message") ("skip-commits-without-parents", po::value()->default_value(true), From 1400bdbf83c9ebf6e63eeda73966b7e7c0210d80 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 06:11:35 +0300 Subject: [PATCH 034/123] Fix unit tests --- src/Common/ShellCommand.cpp | 23 +++++++++++++++---- utils/git-to-clickhouse/git-to-clickhouse.cpp | 13 ----------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 127f95fef06..bbb8801f190 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -57,7 +57,16 @@ ShellCommand::~ShellCommand() LOG_WARNING(getLogger(), "Cannot kill shell command pid {} errno '{}'", pid, errnoToString(retcode)); } else if (!wait_called) - tryWait(); + { + try + { + tryWait(); + } + catch (...) + { + tryLogCurrentException(getLogger()); + } + } } void ShellCommand::logCommand(const char * filename, char * const argv[]) @@ -74,7 +83,8 @@ void ShellCommand::logCommand(const char * filename, char * const argv[]) LOG_TRACE(ShellCommand::getLogger(), "Will start shell command '{}' with arguments {}", filename, args.str()); } -std::unique_ptr ShellCommand::executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only, bool terminate_in_destructor) +std::unique_ptr ShellCommand::executeImpl( + const char * filename, char * const argv[], bool pipe_stdin_only, bool terminate_in_destructor) { logCommand(filename, argv); @@ -130,7 +140,8 @@ std::unique_ptr ShellCommand::executeImpl(const char * filename, c _exit(int(ReturnCodes::CANNOT_EXEC)); } - std::unique_ptr res(new ShellCommand(pid, pipe_stdin.fds_rw[1], pipe_stdout.fds_rw[0], pipe_stderr.fds_rw[0], terminate_in_destructor)); + std::unique_ptr res(new ShellCommand( + pid, pipe_stdin.fds_rw[1], pipe_stdout.fds_rw[0], pipe_stderr.fds_rw[0], terminate_in_destructor)); LOG_TRACE(getLogger(), "Started shell command '{}' with pid {}", filename, pid); @@ -143,7 +154,8 @@ std::unique_ptr ShellCommand::executeImpl(const char * filename, c } -std::unique_ptr ShellCommand::execute(const std::string & command, bool pipe_stdin_only, bool terminate_in_destructor) +std::unique_ptr ShellCommand::execute( + const std::string & command, bool pipe_stdin_only, bool terminate_in_destructor) { /// Arguments in non-constant chunks of memory (as required for `execv`). /// Moreover, their copying must be done before calling `vfork`, so after `vfork` do a minimum of things. @@ -157,7 +169,8 @@ std::unique_ptr ShellCommand::execute(const std::string & command, } -std::unique_ptr ShellCommand::executeDirect(const std::string & path, const std::vector & arguments, bool terminate_in_destructor) +std::unique_ptr ShellCommand::executeDirect( + const std::string & path, const std::vector & arguments, bool terminate_in_destructor) { size_t argv_sum_size = path.size() + 1; for (const auto & arg : arguments) diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/utils/git-to-clickhouse/git-to-clickhouse.cpp index 6ef82ac3b6b..a081efa3f47 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/utils/git-to-clickhouse/git-to-clickhouse.cpp @@ -1171,19 +1171,6 @@ void processLog(const Options & options) { processCommit(show_commands[i % num_threads]->out, options, i, num_commits, hashes[i], snapshot, diff_hashes, result); - try - { - show_commands[i % num_threads]->wait(); - } - catch (const Exception & e) - { - /// For broken pipe when we stopped reading prematurally. - if (e.code() == ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY) - std::cerr << getCurrentExceptionMessage(false) << "\n"; - else - throw; - } - if (!options.stop_after_commit.empty() && hashes[i] == options.stop_after_commit) break; From d18e7adbc03e4e7d7ee268e8f90a14e73be7b021 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 06:22:47 +0300 Subject: [PATCH 035/123] Add git-import as a tool --- programs/CMakeLists.txt | 18 ++++++++++++++---- programs/config_tools.h.in | 1 + programs/git-import/CMakeLists.txt | 10 ++++++++++ programs/git-import/clickhouse-git-import.cpp | 2 ++ .../git-import/git-import.cpp | 4 ++-- programs/install/Install.cpp | 1 + programs/main.cpp | 6 ++++++ utils/CMakeLists.txt | 1 - utils/git-to-clickhouse/CMakeLists.txt | 2 -- 9 files changed, 36 insertions(+), 9 deletions(-) create mode 100644 programs/git-import/CMakeLists.txt create mode 100644 programs/git-import/clickhouse-git-import.cpp rename utils/git-to-clickhouse/git-to-clickhouse.cpp => programs/git-import/git-import.cpp (99%) delete mode 100644 utils/git-to-clickhouse/CMakeLists.txt diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 89220251cda..ae4a72ef62a 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -16,6 +16,7 @@ option (ENABLE_CLICKHOUSE_COMPRESSOR "Enable clickhouse-compressor" ${ENABLE_CLI option (ENABLE_CLICKHOUSE_COPIER "Enable clickhouse-copier" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_FORMAT "Enable clickhouse-format" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_OBFUSCATOR "Enable clickhouse-obfuscator" ${ENABLE_CLICKHOUSE_ALL}) +option (ENABLE_CLICKHOUSE_GIT_IMPORT "Enable clickhouse-git-import" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_ODBC_BRIDGE "Enable clickhouse-odbc-bridge" ${ENABLE_CLICKHOUSE_ALL}) if (CLICKHOUSE_SPLIT_BINARY) @@ -91,21 +92,22 @@ add_subdirectory (copier) add_subdirectory (format) add_subdirectory (obfuscator) add_subdirectory (install) +add_subdirectory (git-import) if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) add_subdirectory (odbc-bridge) endif () if (CLICKHOUSE_ONE_SHARED) - add_library(clickhouse-lib SHARED ${CLICKHOUSE_SERVER_SOURCES} ${CLICKHOUSE_CLIENT_SOURCES} ${CLICKHOUSE_LOCAL_SOURCES} ${CLICKHOUSE_BENCHMARK_SOURCES} ${CLICKHOUSE_COPIER_SOURCES} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_SOURCES} ${CLICKHOUSE_COMPRESSOR_SOURCES} ${CLICKHOUSE_FORMAT_SOURCES} ${CLICKHOUSE_OBFUSCATOR_SOURCES} ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) - target_link_libraries(clickhouse-lib ${CLICKHOUSE_SERVER_LINK} ${CLICKHOUSE_CLIENT_LINK} ${CLICKHOUSE_LOCAL_LINK} ${CLICKHOUSE_BENCHMARK_LINK} ${CLICKHOUSE_COPIER_LINK} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_LINK} ${CLICKHOUSE_COMPRESSOR_LINK} ${CLICKHOUSE_FORMAT_LINK} ${CLICKHOUSE_OBFUSCATOR_LINK} ${CLICKHOUSE_ODBC_BRIDGE_LINK}) - target_include_directories(clickhouse-lib ${CLICKHOUSE_SERVER_INCLUDE} ${CLICKHOUSE_CLIENT_INCLUDE} ${CLICKHOUSE_LOCAL_INCLUDE} ${CLICKHOUSE_BENCHMARK_INCLUDE} ${CLICKHOUSE_COPIER_INCLUDE} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_INCLUDE} ${CLICKHOUSE_COMPRESSOR_INCLUDE} ${CLICKHOUSE_FORMAT_INCLUDE} ${CLICKHOUSE_OBFUSCATOR_INCLUDE} ${CLICKHOUSE_ODBC_BRIDGE_INCLUDE}) + add_library(clickhouse-lib SHARED ${CLICKHOUSE_SERVER_SOURCES} ${CLICKHOUSE_CLIENT_SOURCES} ${CLICKHOUSE_LOCAL_SOURCES} ${CLICKHOUSE_BENCHMARK_SOURCES} ${CLICKHOUSE_COPIER_SOURCES} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_SOURCES} ${CLICKHOUSE_COMPRESSOR_SOURCES} ${CLICKHOUSE_FORMAT_SOURCES} ${CLICKHOUSE_OBFUSCATOR_SOURCES} ${CLICKHOUSE_GIT_IMPORT_SOURCES} ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) + target_link_libraries(clickhouse-lib ${CLICKHOUSE_SERVER_LINK} ${CLICKHOUSE_CLIENT_LINK} ${CLICKHOUSE_LOCAL_LINK} ${CLICKHOUSE_BENCHMARK_LINK} ${CLICKHOUSE_COPIER_LINK} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_LINK} ${CLICKHOUSE_COMPRESSOR_LINK} ${CLICKHOUSE_FORMAT_LINK} ${CLICKHOUSE_OBFUSCATOR_LINK} ${CLICKHOUSE_GIT_IMPORT_LINK} ${CLICKHOUSE_ODBC_BRIDGE_LINK}) + target_include_directories(clickhouse-lib ${CLICKHOUSE_SERVER_INCLUDE} ${CLICKHOUSE_CLIENT_INCLUDE} ${CLICKHOUSE_LOCAL_INCLUDE} ${CLICKHOUSE_BENCHMARK_INCLUDE} ${CLICKHOUSE_COPIER_INCLUDE} ${CLICKHOUSE_EXTRACT_FROM_CONFIG_INCLUDE} ${CLICKHOUSE_COMPRESSOR_INCLUDE} ${CLICKHOUSE_FORMAT_INCLUDE} ${CLICKHOUSE_OBFUSCATOR_INCLUDE} ${CLICKHOUSE_GIT_IMPORT_INCLUDE} ${CLICKHOUSE_ODBC_BRIDGE_INCLUDE}) set_target_properties(clickhouse-lib PROPERTIES SOVERSION ${VERSION_MAJOR}.${VERSION_MINOR} VERSION ${VERSION_SO} OUTPUT_NAME clickhouse DEBUG_POSTFIX "") install (TARGETS clickhouse-lib LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} COMPONENT clickhouse) endif() if (CLICKHOUSE_SPLIT_BINARY) - set (CLICKHOUSE_ALL_TARGETS clickhouse-server clickhouse-client clickhouse-local clickhouse-benchmark clickhouse-extract-from-config clickhouse-compressor clickhouse-format clickhouse-obfuscator clickhouse-copier) + set (CLICKHOUSE_ALL_TARGETS clickhouse-server clickhouse-client clickhouse-local clickhouse-benchmark clickhouse-extract-from-config clickhouse-compressor clickhouse-format clickhouse-obfuscator clickhouse-git-import clickhouse-copier) if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-odbc-bridge) @@ -149,6 +151,9 @@ else () if (ENABLE_CLICKHOUSE_OBFUSCATOR) clickhouse_target_link_split_lib(clickhouse obfuscator) endif () + if (ENABLE_CLICKHOUSE_GIT_IMPORT) + clickhouse_target_link_split_lib(clickhouse git-import) + endif () if (ENABLE_CLICKHOUSE_INSTALL) clickhouse_target_link_split_lib(clickhouse install) endif () @@ -199,6 +204,11 @@ else () install (FILES ${CMAKE_CURRENT_BINARY_DIR}/clickhouse-obfuscator DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) list(APPEND CLICKHOUSE_BUNDLE clickhouse-obfuscator) endif () + if (ENABLE_CLICKHOUSE_GIT_IMPORT) + add_custom_target (clickhouse-git-import ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-git-import DEPENDS clickhouse) + install (FILES ${CMAKE_CURRENT_BINARY_DIR}/clickhouse-git-import DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) + list(APPEND CLICKHOUSE_BUNDLE clickhouse-git-import) + endif () if(ENABLE_CLICKHOUSE_ODBC_BRIDGE) list(APPEND CLICKHOUSE_BUNDLE clickhouse-odbc-bridge) endif() diff --git a/programs/config_tools.h.in b/programs/config_tools.h.in index 11386aca60e..7cb5a6d883a 100644 --- a/programs/config_tools.h.in +++ b/programs/config_tools.h.in @@ -12,5 +12,6 @@ #cmakedefine01 ENABLE_CLICKHOUSE_COMPRESSOR #cmakedefine01 ENABLE_CLICKHOUSE_FORMAT #cmakedefine01 ENABLE_CLICKHOUSE_OBFUSCATOR +#cmakedefine01 ENABLE_CLICKHOUSE_GIT_IMPORT #cmakedefine01 ENABLE_CLICKHOUSE_INSTALL #cmakedefine01 ENABLE_CLICKHOUSE_ODBC_BRIDGE diff --git a/programs/git-import/CMakeLists.txt b/programs/git-import/CMakeLists.txt new file mode 100644 index 00000000000..279bb35a272 --- /dev/null +++ b/programs/git-import/CMakeLists.txt @@ -0,0 +1,10 @@ +set (CLICKHOUSE_GIT_IMPORT_SOURCES git-import.cpp) + +set (CLICKHOUSE_GIT_IMPORT_LINK + PRIVATE + boost::program_options + dbms +) + +clickhouse_program_add(git-import) + diff --git a/programs/git-import/clickhouse-git-import.cpp b/programs/git-import/clickhouse-git-import.cpp new file mode 100644 index 00000000000..cfa06306604 --- /dev/null +++ b/programs/git-import/clickhouse-git-import.cpp @@ -0,0 +1,2 @@ +int mainEntryClickHouseGitImport(int argc, char ** argv); +int main(int argc_, char ** argv_) { return mainEntryClickHouseGitImport(argc_, argv_); } diff --git a/utils/git-to-clickhouse/git-to-clickhouse.cpp b/programs/git-import/git-import.cpp similarity index 99% rename from utils/git-to-clickhouse/git-to-clickhouse.cpp rename to programs/git-import/git-import.cpp index a081efa3f47..f1ed4d28c6e 100644 --- a/utils/git-to-clickhouse/git-to-clickhouse.cpp +++ b/programs/git-import/git-import.cpp @@ -1182,7 +1182,7 @@ void processLog(const Options & options) } -int main(int argc, char ** argv) +int mainEntryClickHouseGitImport(int argc, char ** argv) try { using namespace DB; @@ -1219,7 +1219,7 @@ try << "Usage: " << argv[0] << '\n' << desc << '\n' << "\nExample:\n" - << "\n./git-to-clickhouse --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/' --skip-commits-with-messages '^Merge branch '\n"; + << "\nclickhouse git-import --skip-paths 'generated\\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/' --skip-commits-with-messages '^Merge branch '\n"; return 1; } diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 7b7ab149447..bd60fbb63ba 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -205,6 +205,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) "clickhouse-benchmark", "clickhouse-copier", "clickhouse-obfuscator", + "clickhouse-git-import", "clickhouse-compressor", "clickhouse-format", "clickhouse-extract-from-config" diff --git a/programs/main.cpp b/programs/main.cpp index 3df5f9f683b..b91bd732f21 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -46,6 +46,9 @@ int mainEntryClickHouseClusterCopier(int argc, char ** argv); #if ENABLE_CLICKHOUSE_OBFUSCATOR int mainEntryClickHouseObfuscator(int argc, char ** argv); #endif +#if ENABLE_CLICKHOUSE_GIT_IMPORT +int mainEntryClickHouseGitImport(int argc, char ** argv); +#endif #if ENABLE_CLICKHOUSE_INSTALL int mainEntryClickHouseInstall(int argc, char ** argv); int mainEntryClickHouseStart(int argc, char ** argv); @@ -91,6 +94,9 @@ std::pair clickhouse_applications[] = #if ENABLE_CLICKHOUSE_OBFUSCATOR {"obfuscator", mainEntryClickHouseObfuscator}, #endif +#if ENABLE_CLICKHOUSE_GIT_IMPORT + {"git-import", mainEntryClickHouseGitImport}, +#endif #if ENABLE_CLICKHOUSE_INSTALL {"install", mainEntryClickHouseInstall}, {"start", mainEntryClickHouseStart}, diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 93490fba565..b4408a298c3 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -30,7 +30,6 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) add_subdirectory (checksum-for-compressed-block) add_subdirectory (db-generator) add_subdirectory (wal-dump) - add_subdirectory (git-to-clickhouse) endif () if (ENABLE_CODE_QUALITY) diff --git a/utils/git-to-clickhouse/CMakeLists.txt b/utils/git-to-clickhouse/CMakeLists.txt deleted file mode 100644 index 0e46b68d471..00000000000 --- a/utils/git-to-clickhouse/CMakeLists.txt +++ /dev/null @@ -1,2 +0,0 @@ -add_executable (git-to-clickhouse git-to-clickhouse.cpp) -target_link_libraries(git-to-clickhouse PRIVATE dbms boost::program_options) From ee54971c3d26ca1219da4909bd30f44bee77fd97 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 07:11:03 +0300 Subject: [PATCH 036/123] Fix build --- programs/git-import/git-import.cpp | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/programs/git-import/git-import.cpp b/programs/git-import/git-import.cpp index f1ed4d28c6e..d314969a1a8 100644 --- a/programs/git-import/git-import.cpp +++ b/programs/git-import/git-import.cpp @@ -184,7 +184,6 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_DATA; - extern const int CHILD_WAS_NOT_EXITED_NORMALLY; } @@ -419,7 +418,7 @@ using LineChanges = std::vector; struct FileDiff { - FileDiff(FileChange file_change_) : file_change(file_change_) {} + explicit FileDiff(FileChange file_change_) : file_change(file_change_) {} FileChange file_change; LineChanges line_changes; @@ -546,7 +545,7 @@ struct Options std::optional diff_size_limit; std::string stop_after_commit; - Options(const po::variables_map & options) + explicit Options(const po::variables_map & options) { skip_commits_without_parents = options["skip-commits-without-parents"].as(); skip_commits_with_duplicate_diffs = options["skip-commits-with-duplicate-diffs"].as(); @@ -753,7 +752,7 @@ UInt128 diffHash(const CommitDiff & file_changes) { SipHash hasher; - for (auto & elem : file_changes) + for (const auto & elem : file_changes) { hasher.update(elem.second.file_change.change_type); hasher.update(elem.second.file_change.old_path.size()); @@ -762,7 +761,7 @@ UInt128 diffHash(const CommitDiff & file_changes) hasher.update(elem.second.file_change.path); hasher.update(elem.second.line_changes.size()); - for (auto & line_change : elem.second.line_changes) + for (const auto & line_change : elem.second.line_changes) { hasher.update(line_change.sign); hasher.update(line_change.line_number_old); @@ -1159,6 +1158,8 @@ void processLog(const Options & options) /// Will run multiple processes in parallel size_t num_threads = options.threads; + if (num_threads == 0) + throw Exception("num-threads cannot be zero", ErrorCodes::INCORRECT_DATA); std::vector> show_commands(num_threads); for (size_t i = 0; i < num_commits && i < num_threads; ++i) @@ -1223,7 +1224,7 @@ try return 1; } - processLog(options); + processLog(Options(options)); return 0; } catch (...) From 04a69650068c3ff5967f3639c55082dbd34017cf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 09:40:01 +0300 Subject: [PATCH 037/123] Fix error with executable dictionary source --- docker/test/fasttest/run.sh | 3 +- docker/test/stateless/run.sh | 1 + docker/test/stateless_unbundled/run.sh | 1 + docker/test/stateless_with_coverage/run.sh | 1 + src/Common/tests/CMakeLists.txt | 3 + src/Common/tests/shell_command_inout.cpp | 47 +++++++ .../ExecutableDictionarySource.cpp | 119 ++++++++++-------- src/Dictionaries/ExecutableDictionarySource.h | 1 + tests/config/executable_dictionary.xml | 108 ++++++++++++++++ .../01474_executable_dictionary.reference | 3 + .../01474_executable_dictionary.sql | 3 + 11 files changed, 240 insertions(+), 50 deletions(-) create mode 100644 src/Common/tests/shell_command_inout.cpp create mode 100644 tests/config/executable_dictionary.xml create mode 100644 tests/queries/0_stateless/01474_executable_dictionary.reference create mode 100644 tests/queries/0_stateless/01474_executable_dictionary.sql diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 1f8d612a125..9f5a9b05219 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -11,7 +11,7 @@ stage=${stage:-} # A variable to pass additional flags to CMake. # Here we explicitly default it to nothing so that bash doesn't complain about -# it being undefined. Also read it as array so that we can pass an empty list +# it being undefined. Also read it as array so that we can pass an empty list # of additional variable to cmake properly, and it doesn't generate an extra # empty parameter. read -ra FASTTEST_CMAKE_FLAGS <<< "${FASTTEST_CMAKE_FLAGS:-}" @@ -128,6 +128,7 @@ ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-se ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/ #ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/ diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 2ff15ca9c6a..4a9ad891883 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -24,6 +24,7 @@ ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-se ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/ diff --git a/docker/test/stateless_unbundled/run.sh b/docker/test/stateless_unbundled/run.sh index 2ff15ca9c6a..4a9ad891883 100755 --- a/docker/test/stateless_unbundled/run.sh +++ b/docker/test/stateless_unbundled/run.sh @@ -24,6 +24,7 @@ ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-se ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/ diff --git a/docker/test/stateless_with_coverage/run.sh b/docker/test/stateless_with_coverage/run.sh index 64317ee62fd..c3ccb18659b 100755 --- a/docker/test/stateless_with_coverage/run.sh +++ b/docker/test/stateless_with_coverage/run.sh @@ -57,6 +57,7 @@ ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-se ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/ diff --git a/src/Common/tests/CMakeLists.txt b/src/Common/tests/CMakeLists.txt index f6c232cdd22..8de9424e044 100644 --- a/src/Common/tests/CMakeLists.txt +++ b/src/Common/tests/CMakeLists.txt @@ -84,3 +84,6 @@ target_link_libraries (procfs_metrics_provider_perf PRIVATE clickhouse_common_io add_executable (average average.cpp) target_link_libraries (average PRIVATE clickhouse_common_io) + +add_executable (shell_command_inout shell_command_inout.cpp) +target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io) diff --git a/src/Common/tests/shell_command_inout.cpp b/src/Common/tests/shell_command_inout.cpp new file mode 100644 index 00000000000..615700cd042 --- /dev/null +++ b/src/Common/tests/shell_command_inout.cpp @@ -0,0 +1,47 @@ +#include + +#include +#include + +#include +#include +#include + +/** This example shows how we can proxy stdin to ShellCommand and obtain stdout in streaming fashion. */ + +int main(int argc, char ** argv) +try +{ + using namespace DB; + + if (argc < 2) + { + std::cerr << "Usage: shell_command_inout 'command...' < in > out\n"; + return 1; + } + + auto command = ShellCommand::execute(argv[1]); + + ReadBufferFromFileDescriptor in(STDIN_FILENO); + WriteBufferFromFileDescriptor out(STDOUT_FILENO); + WriteBufferFromFileDescriptor err(STDERR_FILENO); + + /// Background thread sends data and foreground thread receives result. + + std::thread thread([&] + { + copyData(in, command->in); + command->in.close(); + }); + + copyData(command->out, out); + copyData(command->err, err); + + thread.join(); + return 0; +} +catch (...) +{ + std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; + throw; +} diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 918cf0732ab..74aab610e0d 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -1,12 +1,13 @@ #include "ExecutableDictionarySource.h" -#include -#include +#include #include #include #include #include #include +#include +#include #include #include #include @@ -16,6 +17,7 @@ #include "DictionaryStructure.h" #include "registerDictionaries.h" + namespace DB { static const UInt64 max_block_size = 8192; @@ -31,15 +33,23 @@ namespace /// Owns ShellCommand and calls wait for it. class ShellCommandOwningBlockInputStream : public OwningBlockInputStream { + private: + Poco::Logger * log; public: - ShellCommandOwningBlockInputStream(const BlockInputStreamPtr & impl, std::unique_ptr own_) - : OwningBlockInputStream(std::move(impl), std::move(own_)) + ShellCommandOwningBlockInputStream(Poco::Logger * log_, const BlockInputStreamPtr & impl, std::unique_ptr command_) + : OwningBlockInputStream(std::move(impl), std::move(command_)), log(log_) { } void readSuffix() override { OwningBlockInputStream::readSuffix(); + + std::string err; + readStringUntilEOF(err, own->err); + if (!err.empty()) + LOG_ERROR(log, "Having stderr: {}", err); + own->wait(); } }; @@ -80,7 +90,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadAll() LOG_TRACE(log, "loadAll {}", toString()); auto process = ShellCommand::execute(command); auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size); - return std::make_shared(input_stream, std::move(process)); + return std::make_shared(log, input_stream, std::move(process)); } BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll() @@ -95,67 +105,77 @@ BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll() LOG_TRACE(log, "loadUpdatedAll {}", command_with_update_field); auto process = ShellCommand::execute(command_with_update_field); auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size); - return std::make_shared(input_stream, std::move(process)); + return std::make_shared(log, input_stream, std::move(process)); } namespace { - /** A stream, that also runs and waits for background thread - * (that will feed data into pipe to be read from the other side of the pipe). + /** A stream, that runs child process and sends data to its stdin in background thread, + * and receives data from its stdout. */ class BlockInputStreamWithBackgroundThread final : public IBlockInputStream { public: BlockInputStreamWithBackgroundThread( - const BlockInputStreamPtr & stream_, std::unique_ptr && command_, std::packaged_task && task_) - : stream{stream_}, command{std::move(command_)}, task(std::move(task_)), thread([this] { - task(); - command->in.close(); - }) + const Context & context, + const std::string & format, + const Block & sample_block, + const std::string & command_str, + Poco::Logger * log_, + std::function && send_data_) + : log(log_), + command(ShellCommand::execute(command_str)), + send_data(std::move(send_data_)), + thread([this] { send_data(command->in); }) { - children.push_back(stream); + //WriteBufferFromFileDescriptor err(STDERR_FILENO); + //copyData(command->out, err); + //err.next(); + //thread.join(); + stream = context.getInputFormat(format, command->out, sample_block, max_block_size); } ~BlockInputStreamWithBackgroundThread() override { if (thread.joinable()) - { - try - { - readSuffix(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } + thread.join(); } - Block getHeader() const override { return stream->getHeader(); } + Block getHeader() const override + { + return stream->getHeader(); + } private: - Block readImpl() override { return stream->read(); } + Block readImpl() override + { + return stream->read(); + } + + void readPrefix() override + { + stream->readPrefix(); + } void readSuffix() override { - IBlockInputStream::readSuffix(); - if (!wait_called) - { - wait_called = true; - command->wait(); - } - thread.join(); - /// To rethrow an exception, if any. - task.get_future().get(); + stream->readSuffix(); + + std::string err; + readStringUntilEOF(err, command->err); + if (!err.empty()) + LOG_ERROR(log, "Having stderr: {}", err); + + command->wait(); } String getName() const override { return "WithBackgroundThread"; } + Poco::Logger * log; BlockInputStreamPtr stream; std::unique_ptr command; - std::packaged_task task; - ThreadFromGlobalPool thread; - bool wait_called = false; + std::function send_data; + mutable ThreadFromGlobalPool thread; }; } @@ -164,28 +184,29 @@ namespace BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector & ids) { LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size()); - auto process = ShellCommand::execute(command); - - auto output_stream = context.getOutputFormat(format, process->in, sample_block); - auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size); return std::make_shared( - input_stream, std::move(process), std::packaged_task([output_stream, &ids]() mutable { formatIDs(output_stream, ids); })); + context, format, sample_block, command, log, + [&ids, this](WriteBufferFromFile & out) mutable + { + auto output_stream = context.getOutputFormat(format, out, sample_block); + formatIDs(output_stream, ids); + out.close(); + }); } BlockInputStreamPtr ExecutableDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size()); - auto process = ShellCommand::execute(command); - - auto output_stream = context.getOutputFormat(format, process->in, sample_block); - auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size); return std::make_shared( - input_stream, std::move(process), std::packaged_task([output_stream, key_columns, &requested_rows, this]() mutable + context, format, sample_block, command, log, + [key_columns, &requested_rows, this](WriteBufferFromFile & out) mutable { + auto output_stream = context.getOutputFormat(format, out, sample_block); formatKeys(dict_struct, output_stream, key_columns, requested_rows); - })); + out.close(); + }); } bool ExecutableDictionarySource::isModified() const diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index f28d71ca5e3..b2aabf26323 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -14,6 +14,7 @@ namespace DB /// Allows loading dictionaries from executable class ExecutableDictionarySource final : public IDictionarySource { + friend class BlockInputStreamWithBackgroundThread; public: ExecutableDictionarySource( const DictionaryStructure & dict_struct_, diff --git a/tests/config/executable_dictionary.xml b/tests/config/executable_dictionary.xml new file mode 100644 index 00000000000..50df32e2ec6 --- /dev/null +++ b/tests/config/executable_dictionary.xml @@ -0,0 +1,108 @@ + + + + executable_complex + + + JSONEachRow + cd /; clickhouse-local --input-format JSONEachRow --output-format JSONEachRow --structure 'x UInt64, y UInt64' --query "SELECT x, y, x + y AS a, x * y AS b FROM table" + + + 0 + + + 1000 + + + + + + x + UInt64 + + + y + UInt64 + + + + a + UInt64 + 0 + + + b + UInt64 + 0 + + + + + + executable_simple + + + JSONEachRow + cd /; clickhouse-local --input-format JSONEachRow --output-format JSONEachRow --structure 'x UInt64' --query "SELECT x, x + x AS a, x * x AS b FROM table" + + + 0 + + + 1000 + + + + + x + + + a + UInt64 + 0 + + + b + UInt64 + 0 + + + + + + executable_complex_direct + + + JSONEachRow + cd /; clickhouse-local --input-format JSONEachRow --output-format JSONEachRow --structure 'x UInt64, y UInt64' --query "SELECT x, y, x + y AS a, x * y AS b FROM table" + + + 0 + + + + + + + x + UInt64 + + + y + UInt64 + + + + a + UInt64 + 0 + + + b + UInt64 + 0 + + + + + diff --git a/tests/queries/0_stateless/01474_executable_dictionary.reference b/tests/queries/0_stateless/01474_executable_dictionary.reference new file mode 100644 index 00000000000..4d0994b08c3 --- /dev/null +++ b/tests/queries/0_stateless/01474_executable_dictionary.reference @@ -0,0 +1,3 @@ +999999 1999998 999998000001 +999999 1999998 999998000001 +999999 1999998 999998000001 diff --git a/tests/queries/0_stateless/01474_executable_dictionary.sql b/tests/queries/0_stateless/01474_executable_dictionary.sql new file mode 100644 index 00000000000..727cf47f79f --- /dev/null +++ b/tests/queries/0_stateless/01474_executable_dictionary.sql @@ -0,0 +1,3 @@ +SELECT number, dictGet('executable_complex', 'a', (number, number)) AS a, dictGet('executable_complex', 'b', (number, number)) AS b FROM numbers(1000000) WHERE number = 999999; +SELECT number, dictGet('executable_complex_direct', 'a', (number, number)) AS a, dictGet('executable_complex_direct', 'b', (number, number)) AS b FROM numbers(1000000) WHERE number = 999999; +SELECT number, dictGet('executable_simple', 'a', number) AS a, dictGet('executable_simple', 'b', number) AS b FROM numbers(1000000) WHERE number = 999999; From 8dd98f74a5a5cd5c5cba804f96b3349c5f9a2e25 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 09:43:28 +0300 Subject: [PATCH 038/123] Remove debug output --- src/Dictionaries/ExecutableDictionarySource.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 74aab610e0d..0709be2420a 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -128,10 +128,6 @@ namespace send_data(std::move(send_data_)), thread([this] { send_data(command->in); }) { - //WriteBufferFromFileDescriptor err(STDERR_FILENO); - //copyData(command->out, err); - //err.next(); - //thread.join(); stream = context.getInputFormat(format, command->out, sample_block, max_block_size); } From 1f0d2be17adbc292fef91d4b7703d654871cb815 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 7 Sep 2020 09:44:32 +0300 Subject: [PATCH 039/123] Update ExecutableDictionarySource.h --- src/Dictionaries/ExecutableDictionarySource.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index b2aabf26323..f28d71ca5e3 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -14,7 +14,6 @@ namespace DB /// Allows loading dictionaries from executable class ExecutableDictionarySource final : public IDictionarySource { - friend class BlockInputStreamWithBackgroundThread; public: ExecutableDictionarySource( const DictionaryStructure & dict_struct_, From 3903794386c32d1894fa266d760eed07419a1d54 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 7 Sep 2020 09:45:01 +0300 Subject: [PATCH 040/123] Update ExecutableDictionarySource.cpp --- src/Dictionaries/ExecutableDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 0709be2420a..cc250727261 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -171,7 +171,7 @@ namespace BlockInputStreamPtr stream; std::unique_ptr command; std::function send_data; - mutable ThreadFromGlobalPool thread; + ThreadFromGlobalPool thread; }; } From 3942cc615f03ecb8e5b9e7437fdc5c57613c245d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 7 Sep 2020 10:09:42 +0300 Subject: [PATCH 041/123] Update git-import.cpp --- programs/git-import/git-import.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/programs/git-import/git-import.cpp b/programs/git-import/git-import.cpp index d314969a1a8..45bc47348e7 100644 --- a/programs/git-import/git-import.cpp +++ b/programs/git-import/git-import.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include @@ -539,8 +539,8 @@ struct Options bool skip_commits_without_parents = true; bool skip_commits_with_duplicate_diffs = true; size_t threads = 1; - std::optional skip_paths; - std::optional skip_commits_with_messages; + std::optional skip_paths; + std::optional skip_commits_with_messages; std::unordered_set skip_commits; std::optional diff_size_limit; std::string stop_after_commit; @@ -857,7 +857,7 @@ void processFileChanges( assertChar('\n', in); - if (!(options.skip_paths && re2_st::RE2::PartialMatch(file_change.path, *options.skip_paths))) + if (!(options.skip_paths && re2::RE2::PartialMatch(file_change.path, *options.skip_paths))) { file_changes.emplace( file_change.path, @@ -1070,7 +1070,7 @@ void processCommit( readNullTerminated(parent_hash, in); readNullTerminated(commit.message, in); - if (options.skip_commits_with_messages && re2_st::RE2::PartialMatch(commit.message, *options.skip_commits_with_messages)) + if (options.skip_commits_with_messages && re2::RE2::PartialMatch(commit.message, *options.skip_commits_with_messages)) return; std::string message_to_print = commit.message; From 99ebab706cf70a286a5a6b5b2ac6070085f1ebf0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 12:02:22 +0300 Subject: [PATCH 042/123] Fix "Arcadia" --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 707f91b0c93..16450efb26e 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -139,3 +139,4 @@ 01455_time_zones 01456_ast_optimizations_over_distributed 01460_DistributedFilesToInsert +01474_executable_dictionary From 661d9bdb4c1489d6a9c5c8f0ae6d06bb5480a2b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Sep 2020 19:03:36 +0300 Subject: [PATCH 043/123] Skip test --- tests/queries/skip_list.json | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index adfc5f0e582..0aa98499d42 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -18,7 +18,8 @@ "00152_insert_different_granularity", "00151_replace_partition_with_different_granularity", "00157_cache_dictionary", - "01193_metadata_loading" + "01193_metadata_loading", + "01474_executable_dictionary" /// informational stderr from sanitizer at start ], "address-sanitizer": [ "00281", From 2c04b0a8e67d1aeefdbb523ac4f8cd321b83a347 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 7 Sep 2020 19:07:34 +0300 Subject: [PATCH 044/123] comment added --- .../Formats/Impl/JSONCompactEachRowRowInputFormat.h | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index 593f297108c..6845b2974ab 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -12,7 +12,12 @@ namespace DB class ReadBuffer; -/** A stream for reading data in JSONCompactEachRow- formats +/** A stream for reading data in a bunch of formats: + * - JSONCompactEachRow + * - JSONCompactEachRowWithNamesAndTypes + * - JSONCompactStringsEachRow + * - JSONCompactStringsEachRowWithNamesAndTypes + * */ class JSONCompactEachRowRowInputFormat : public IRowInputFormat { @@ -54,7 +59,9 @@ private: /// This is for the correct exceptions in skipping unknown fields. std::vector names_of_columns; + /// For *WithNamesAndTypes formats. bool with_names; + /// For JSONCompactString* formats. bool yield_strings; }; From a56d42de67496404fb507d05c2d399012fd479ce Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 7 Sep 2020 20:55:06 +0300 Subject: [PATCH 045/123] fix arcadia --- src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h | 2 -- src/Processors/ya.make | 3 --- 2 files changed, 5 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index 6845b2974ab..4077eb6e008 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -1,7 +1,5 @@ #pragma once -#pragma once - #include #include #include diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 27893674859..30de38fedbd 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -31,9 +31,6 @@ SRCS( Formats/Impl/JSONEachRowRowOutputFormat.cpp Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp Formats/Impl/JSONRowOutputFormat.cpp - Formats/Impl/JSONStringsEachRowRowInputFormat.cpp - Formats/Impl/JSONStringsEachRowRowOutputFormat.cpp - Formats/Impl/JSONStringsRowOutputFormat.cpp Formats/Impl/MarkdownRowOutputFormat.cpp Formats/Impl/MsgPackRowInputFormat.cpp Formats/Impl/MsgPackRowOutputFormat.cpp From 0c00b992d5741f7d9750567afd3cbda92b019273 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 7 Sep 2020 16:01:49 -0700 Subject: [PATCH 046/123] StorageReplicatedMergeTree - cleanup data dir after Zk exceptions It's possible for `getZooKeeper()` to timeout if zookeeper host(s) can't be reached. In such cases `Poco::Exception` is thrown after a connection timeout - refer to `src/Common/ZooKeeper/ZooKeeperImpl.cpp:866` for more info. Side effect of this is that the CreateQuery gets interrupted and it exits. But the data Directories for the tables being created aren't cleaned up. This unclean state will hinder table creation on any retries and will complain that the Directory for table already exists. To achieve a clean state on failed table creations, catch this error if the exception is of type Poco::Exception and call `dropIfEmpty()` method, then proceed throwing the exception. Without this, the Directory for the tables need to be manually deleted before retrying the CreateQuery. --- src/Storages/StorageReplicatedMergeTree.cpp | 25 ++++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6058632d220..64eb1c8d4cd 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -216,7 +216,30 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mutationsFinalizingTask)", [this] { mutationsFinalizingTask(); }); if (global_context.hasZooKeeper()) - current_zookeeper = global_context.getZooKeeper(); + { + /// It's possible for getZooKeeper() to timeout if zookeeper host(s) can't + /// be reached. In such cases Poco::Exception is thrown after a connection + /// timeout - refer to src/Common/ZooKeeper/ZooKeeperImpl.cpp:866 for more info. + /// + /// Side effect of this is that the CreateQuery gets interrupted and it exits. + /// But the data Directories for the tables being created aren't cleaned up. + /// This unclean state will hinder table creation on any retries and will + /// complain that the Directory for table already exists. + /// + /// To acheive a clean state on failed table creations, catch this error if + /// the excaption is of type Poco::Exception and call dropIfEmpty() method, + /// then proceed throwing the exception. Without this, the Directory for the + /// tables need to be manually deleted before retrying the CreateQuery. + try + { + current_zookeeper = global_context.getZooKeeper(); + } + catch (Poco::Exception & e) + { + dropIfEmpty(); + throw e; + } + } bool skip_sanity_checks = false; From 806334a642a87c09b77a2b69c765ea10ecbcaad7 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 7 Sep 2020 22:27:49 -0700 Subject: [PATCH 047/123] StorageReplicatedMergeTree - add integration tests that test dir cleanup This adds a integration test that tests if table directory is cleaned up after a ZooKeeper connection failure for ReplicatedMergeTree tables. --- .../__init__.py | 0 .../configs/remote_servers.xml | 14 ++++++ .../test_replicated_zk_conn_failure/test.py | 45 +++++++++++++++++++ 3 files changed, 59 insertions(+) create mode 100644 tests/integration/test_replicated_zk_conn_failure/__init__.py create mode 100644 tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml create mode 100644 tests/integration/test_replicated_zk_conn_failure/test.py diff --git a/tests/integration/test_replicated_zk_conn_failure/__init__.py b/tests/integration/test_replicated_zk_conn_failure/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml b/tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml new file mode 100644 index 00000000000..538aa72d386 --- /dev/null +++ b/tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml @@ -0,0 +1,14 @@ + + + + + true + + shard_0 + node1 + 9000 + + + + + diff --git a/tests/integration/test_replicated_zk_conn_failure/test.py b/tests/integration/test_replicated_zk_conn_failure/test.py new file mode 100644 index 00000000000..3f106bd2981 --- /dev/null +++ b/tests/integration/test_replicated_zk_conn_failure/test.py @@ -0,0 +1,45 @@ +import time + +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager + + +# This tests if the data directory for a table is cleaned up if there is a Zookeeper +# connection exception during a CreateQuery operation involving ReplicatedMergeTree tables. +# Test flow is as follows: +# 1. Configure cluster with ZooKeeper and create a database. +# 2. Drop all connections to ZooKeeper. +# 3. Try creating the table and there would be a Poco:Exception. +# 4. Try creating the table again and there should not be any error +# that indicates that the Directory for table already exists. + + +def test_replicated_zk_conn_failure(): + cluster = ClickHouseCluster(__file__) + node1 = cluster.add_instance('node1', main_configs=["configs/remote_servers.xml"], with_zookeeper=True) + try: + cluster.start() + node1.query("CREATE DATABASE replica;") + query_create = '''CREATE TABLE replica.test + ( + id Int64, + event_time DateTime + ) + Engine=ReplicatedMergeTree('/clickhouse/tables/replica/test', 'node1') + PARTITION BY toYYYYMMDD(event_time) + ORDER BY id;'''.format(replica=node1.name) + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node1) + time.sleep(5) + error = node1.query_and_get_error(query_create) + # Assert that there was net exception. + assert "Poco::Exception. Code: 1000" in error + # Assert that the exception was due to ZooKeeper connectivity. + assert "All connection tries failed while connecting to ZooKeeper" in error + # retry table creation + error = node1.query_and_get_error(query_create) + # Should not expect any errors related to directory already existing + # and those should have been already cleaned up during the previous retry. + assert "Directory for table data data/replica/test/ already exists" not in error + finally: + cluster.shutdown() From bee629c971d8f5add8fe4f205aa30f8f4e66375f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 8 Sep 2020 02:08:42 +0300 Subject: [PATCH 048/123] Use join() instead of detach() for the lists_writing_thread in DiskAccessStorage. --- src/Access/DiskAccessStorage.cpp | 47 ++++++++++++++------------------ src/Access/DiskAccessStorage.h | 5 ++-- 2 files changed, 22 insertions(+), 30 deletions(-) diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index fc80859885d..6162e4aacc2 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -426,33 +426,41 @@ bool DiskAccessStorage::writeLists() void DiskAccessStorage::scheduleWriteLists(EntityType type) { if (failed_to_write_lists) - return; + return; /// We don't try to write list files after the first fail. + /// The next restart of the server will invoke rebuilding of the list files. - bool already_scheduled = !types_of_lists_to_write.empty(); types_of_lists_to_write.insert(type); - if (already_scheduled) - return; + if (lists_writing_thread_is_waiting) + return; /// If the lists' writing thread is still waiting we can update `types_of_lists_to_write` easily, + /// without restarting that thread. + + if (lists_writing_thread.joinable()) + lists_writing_thread.join(); /// Create the 'need_rebuild_lists.mark' file. /// This file will be used later to find out if writing lists is successful or not. std::ofstream{getNeedRebuildListsMarkFilePath(directory_path)}; - startListsWritingThread(); + lists_writing_thread = ThreadFromGlobalPool{&DiskAccessStorage::listsWritingThreadFunc, this}; + lists_writing_thread_is_waiting = true; } -void DiskAccessStorage::startListsWritingThread() +void DiskAccessStorage::listsWritingThreadFunc() { - if (lists_writing_thread.joinable()) + std::unique_lock lock{mutex}; + { - if (!lists_writing_thread_exited) - return; - lists_writing_thread.detach(); + /// It's better not to write the lists files too often, that's why we need + /// the following timeout. + const auto timeout = std::chrono::minutes(1); + SCOPE_EXIT({ lists_writing_thread_is_waiting = false; }); + if (lists_writing_thread_should_exit.wait_for(lock, timeout) != std::cv_status::timeout) + return; /// The destructor requires us to exit. } - lists_writing_thread_exited = false; - lists_writing_thread = ThreadFromGlobalPool{&DiskAccessStorage::listsWritingThreadFunc, this}; + writeLists(); } @@ -466,21 +474,6 @@ void DiskAccessStorage::stopListsWritingThread() } -void DiskAccessStorage::listsWritingThreadFunc() -{ - std::unique_lock lock{mutex}; - SCOPE_EXIT({ lists_writing_thread_exited = true; }); - - /// It's better not to write the lists files too often, that's why we need - /// the following timeout. - const auto timeout = std::chrono::minutes(1); - if (lists_writing_thread_should_exit.wait_for(lock, timeout) != std::cv_status::timeout) - return; /// The destructor requires us to exit. - - writeLists(); -} - - /// Reads and parses all the ".sql" files from a specified directory /// and then saves the files "users.list", "roles.list", etc. to the same directory. bool DiskAccessStorage::rebuildLists() diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index 11eb1c3b1ad..ed2dc8b1242 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -42,9 +42,8 @@ private: void scheduleWriteLists(EntityType type); bool rebuildLists(); - void startListsWritingThread(); - void stopListsWritingThread(); void listsWritingThreadFunc(); + void stopListsWritingThread(); void insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, Notifications & notifications); void removeNoLock(const UUID & id, Notifications & notifications); @@ -74,7 +73,7 @@ private: bool failed_to_write_lists = false; /// Whether writing of the list files has been failed since the recent restart of the server. ThreadFromGlobalPool lists_writing_thread; /// List files are written in a separate thread. std::condition_variable lists_writing_thread_should_exit; /// Signals `lists_writing_thread` to exit. - std::atomic lists_writing_thread_exited = false; + bool lists_writing_thread_is_waiting = false; mutable std::list handlers_by_type[static_cast(EntityType::MAX)]; mutable std::mutex mutex; }; From cce970e40cdf1eba81a1d34c6e692ec883d544e2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 8 Sep 2020 02:09:03 +0300 Subject: [PATCH 049/123] Use join() instead of detach() for loading threads in ExternalLoader. --- src/Interpreters/ExternalLoader.cpp | 29 +++++++++++++++++++++++------ 1 file changed, 23 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index e8df205760a..dcef36de175 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -893,6 +893,8 @@ private: cancelLoading(info); } + putBackFinishedThreadsToPool(); + /// All loadings have unique loading IDs. size_t loading_id = next_id_counter++; info.loading_id = loading_id; @@ -914,6 +916,21 @@ private: } } + void putBackFinishedThreadsToPool() + { + for (auto loading_id : recently_finished_loadings) + { + auto it = loading_threads.find(loading_id); + if (it != loading_threads.end()) + { + auto thread = std::move(it->second); + loading_threads.erase(it); + thread.join(); /// It's very likely that `thread` has already finished. + } + } + recently_finished_loadings.clear(); + } + static void cancelLoading(Info & info) { if (!info.isLoading()) @@ -1095,12 +1112,11 @@ private: } min_id_to_finish_loading_dependencies.erase(std::this_thread::get_id()); - auto it = loading_threads.find(loading_id); - if (it != loading_threads.end()) - { - it->second.detach(); - loading_threads.erase(it); - } + /// Add `loading_id` to the list of recently finished loadings. + /// This list is used to later put the threads which finished loading back to the thread pool. + /// (We can't put the loading thread back to the thread pool immediately here because at this point + /// the loading thread is about to finish but it's not finished yet right now.) + recently_finished_loadings.push_back(loading_id); } /// Calculate next update time for loaded_object. Can be called without mutex locking, @@ -1158,6 +1174,7 @@ private: bool always_load_everything = false; std::atomic enable_async_loading = false; std::unordered_map loading_threads; + std::vector recently_finished_loadings; std::unordered_map min_id_to_finish_loading_dependencies; size_t next_id_counter = 1; /// should always be > 0 mutable pcg64 rnd_engine{randomSeed()}; From 48bf65d63de2cacab0742f79fcfbab499dae384e Mon Sep 17 00:00:00 2001 From: bharatnc Date: Tue, 8 Sep 2020 23:45:04 -0700 Subject: [PATCH 050/123] StorageReplicatedMergeTree - improve integration test --- .../configs/remote_servers.xml | 14 -------------- .../test_replicated_zk_conn_failure/test.py | 13 +++++++++---- 2 files changed, 9 insertions(+), 18 deletions(-) delete mode 100644 tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml diff --git a/tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml b/tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml deleted file mode 100644 index 538aa72d386..00000000000 --- a/tests/integration/test_replicated_zk_conn_failure/configs/remote_servers.xml +++ /dev/null @@ -1,14 +0,0 @@ - - - - - true - - shard_0 - node1 - 9000 - - - - - diff --git a/tests/integration/test_replicated_zk_conn_failure/test.py b/tests/integration/test_replicated_zk_conn_failure/test.py index 3f106bd2981..a860716ee82 100644 --- a/tests/integration/test_replicated_zk_conn_failure/test.py +++ b/tests/integration/test_replicated_zk_conn_failure/test.py @@ -12,11 +12,11 @@ from helpers.network import PartitionManager # 3. Try creating the table and there would be a Poco:Exception. # 4. Try creating the table again and there should not be any error # that indicates that the Directory for table already exists. - - +# 5. Final step is to restore ZooKeeper connection and verify that +# the table creation and queries work. def test_replicated_zk_conn_failure(): cluster = ClickHouseCluster(__file__) - node1 = cluster.add_instance('node1', main_configs=["configs/remote_servers.xml"], with_zookeeper=True) + node1 = cluster.add_instance('node1', with_zookeeper=True) try: cluster.start() node1.query("CREATE DATABASE replica;") @@ -27,7 +27,7 @@ def test_replicated_zk_conn_failure(): ) Engine=ReplicatedMergeTree('/clickhouse/tables/replica/test', 'node1') PARTITION BY toYYYYMMDD(event_time) - ORDER BY id;'''.format(replica=node1.name) + ORDER BY id;''' with PartitionManager() as pm: pm.drop_instance_zk_connections(node1) time.sleep(5) @@ -41,5 +41,10 @@ def test_replicated_zk_conn_failure(): # Should not expect any errors related to directory already existing # and those should have been already cleaned up during the previous retry. assert "Directory for table data data/replica/test/ already exists" not in error + # restore ZooKeeper connections. + pm.restore_instance_zk_connections(node1) + # retry create query and query the table created. + node1.query(query_create) + assert "0\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') finally: cluster.shutdown() From b40998ca007afbe702768ede9bf5776274347040 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 9 Sep 2020 15:41:38 +0800 Subject: [PATCH 051/123] Treat query as function argument. --- src/Interpreters/QueryNormalizer.cpp | 2 +- src/Parsers/ASTFunction.cpp | 13 +++++++++++-- src/Parsers/ASTFunction.h | 4 +++- src/Parsers/ExpressionElementParsers.cpp | 6 ++++-- src/TableFunctions/TableFunctionView.cpp | 19 ++++++++----------- 5 files changed, 27 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 59233218a50..07d4888b555 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -152,7 +152,7 @@ void QueryNormalizer::visitChildren(const ASTPtr & node, Data & data) { if (const auto * func_node = node->as()) { - if (func_node->query) + if (func_node->tryGetQueryArgument()) { if (func_node->name != "view") throw Exception("Query argument can only be used in the `view` TableFunction", ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 07429c8104f..bbd910ae875 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -48,7 +48,6 @@ ASTPtr ASTFunction::clone() const auto res = std::make_shared(*this); res->children.clear(); - if (query) { res->query = query->clone(); res->children.push_back(res->query); } if (arguments) { res->arguments = arguments->clone(); res->children.push_back(res->arguments); } if (parameters) { res->parameters = parameters->clone(); res->children.push_back(res->parameters); } @@ -112,6 +111,16 @@ static bool highlightStringLiteralWithMetacharacters(const ASTPtr & node, const } +ASTSelectWithUnionQuery * ASTFunction::tryGetQueryArgument() const +{ + if (arguments && arguments->children.size() == 1) + { + return arguments->children[0]->as(); + } + return nullptr; +} + + void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { FormatStateStacked nested_need_parens = frame; @@ -119,7 +128,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format nested_need_parens.need_parens = true; nested_dont_need_parens.need_parens = false; - if (query) + if (auto * query = tryGetQueryArgument()) { std::string nl_or_nothing = settings.one_line ? "" : "\n"; std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' '); diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index b94614426d8..3b87ab68282 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -13,7 +14,6 @@ class ASTFunction : public ASTWithAlias { public: String name; - ASTPtr query; // It's possible for a function to accept a query as its only argument. ASTPtr arguments; /// parameters - for parametric aggregate function. Example: quantile(0.9)(x) - what in first parens are 'parameters'. ASTPtr parameters; @@ -26,6 +26,8 @@ public: void updateTreeHashImpl(SipHash & hash_state) const override; + ASTSelectWithUnionQuery * tryGetQueryArgument() const; + protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 985507071be..64e3a0363d1 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -260,8 +260,10 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ++pos; auto function_node = std::make_shared(); tryGetIdentifierNameInto(identifier, function_node->name); - function_node->query = query; - function_node->children.push_back(function_node->query); + auto expr_list_with_single_query = std::make_shared(); + expr_list_with_single_query->children.push_back(query); + function_node->arguments = expr_list_with_single_query; + function_node->children.push_back(function_node->arguments); node = function_node; return true; } diff --git a/src/TableFunctions/TableFunctionView.cpp b/src/TableFunctions/TableFunctionView.cpp index 6166fa56f47..8d3f7b06fa3 100644 --- a/src/TableFunctions/TableFunctionView.cpp +++ b/src/TableFunctions/TableFunctionView.cpp @@ -20,18 +20,15 @@ StoragePtr TableFunctionView::executeImpl(const ASTPtr & ast_function, const Con { if (const auto * function = ast_function->as()) { - if (function->query) + if (auto * select = function->tryGetQueryArgument()) { - if (auto * select = function->query->as()) - { - auto sample = InterpreterSelectWithUnionQuery::getSampleBlock(function->query, context); - auto columns = ColumnsDescription(sample.getNamesAndTypesList()); - ASTCreateQuery create; - create.select = select; - auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns); - res->startup(); - return res; - } + auto sample = InterpreterSelectWithUnionQuery::getSampleBlock(function->arguments->children[0] /* ASTPtr */, context); + auto columns = ColumnsDescription(sample.getNamesAndTypesList()); + ASTCreateQuery create; + create.select = select; + auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns); + res->startup(); + return res; } } throw Exception("Table function '" + getName() + "' requires a query argument.", ErrorCodes::BAD_ARGUMENTS); From d8fce448a29eecff6e1dc77299f63c3e75f0fbbc Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 9 Sep 2020 14:20:14 +0800 Subject: [PATCH 052/123] Implement null_as_default for JSONStrings formats --- src/DataTypes/DataTypeNullable.cpp | 12 ++++++++++-- src/DataTypes/DataTypeNullable.h | 2 ++ .../Impl/JSONCompactEachRowRowInputFormat.cpp | 7 ++++--- .../Formats/Impl/JSONEachRowRowInputFormat.cpp | 7 ++++--- .../01016_input_null_as_default.reference | 5 +++++ .../0_stateless/01016_input_null_as_default.sh | 8 ++++++++ 6 files changed, 33 insertions(+), 8 deletions(-) diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 3318196b951..6e452b2759b 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -318,13 +318,20 @@ ReturnType DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer void DataTypeNullable::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - safeDeserialize(column, *nested_data_type, + deserializeWholeText(column, istr, settings, nested_data_type); +} + +template +ReturnType DataTypeNullable::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, + const DataTypePtr & nested_data_type) +{ + return safeDeserialize(column, *nested_data_type, [&istr] { return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr) || checkStringByFirstCharacterAndAssertTheRest("ᴺᵁᴸᴸ", istr); }, - [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsWholeText(nested, istr, settings); }); + [&nested_data_type, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeAsWholeText(nested, istr, settings); }); } @@ -551,6 +558,7 @@ DataTypePtr removeNullable(const DataTypePtr & type) } +template bool DataTypeNullable::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const DataTypePtr & nested); template bool DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const DataTypePtr & nested); template bool DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &, const DataTypePtr & nested); template bool DataTypeNullable::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const DataTypePtr & nested); diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index 22d403da6c4..587eecdf32e 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -103,6 +103,8 @@ public: /// If ReturnType is bool, check for NULL and deserialize value into non-nullable column (and return true) or insert default value of nested type (and return false) /// If ReturnType is void, deserialize Nullable(T) template + static ReturnType deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const DataTypePtr & nested); + template static ReturnType deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const DataTypePtr & nested); template static ReturnType deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &, const DataTypePtr & nested); diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index eb697ce5318..49c8d29ca2f 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -205,14 +205,15 @@ void JSONCompactEachRowRowInputFormat::readField(size_t index, MutableColumns & if (yield_strings) { - // notice: null_as_default on "null" strings is not supported - String str; readJSONString(str, in); ReadBufferFromString buf(str); - type->deserializeAsWholeText(*columns[index], buf, format_settings); + if (format_settings.null_as_default && !type->isNullable()) + read_columns[index] = DataTypeNullable::deserializeWholeText(*columns[index], buf, format_settings, type); + else + type->deserializeAsWholeText(*columns[index], buf, format_settings); } else { diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 9ba82fbb009..ab775a3e7aa 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -146,14 +146,15 @@ void JSONEachRowRowInputFormat::readField(size_t index, MutableColumns & columns if (yield_strings) { - // notice: null_as_default on "null" strings is not supported - String str; readJSONString(str, in); ReadBufferFromString buf(str); - type->deserializeAsWholeText(*columns[index], buf, format_settings); + if (format_settings.null_as_default && !type->isNullable()) + read_columns[index] = DataTypeNullable::deserializeWholeText(*columns[index], buf, format_settings, type); + else + type->deserializeAsWholeText(*columns[index], buf, format_settings); } else { diff --git a/tests/queries/0_stateless/01016_input_null_as_default.reference b/tests/queries/0_stateless/01016_input_null_as_default.reference index ba9657bf16e..d7010f42d4e 100644 --- a/tests/queries/0_stateless/01016_input_null_as_default.reference +++ b/tests/queries/0_stateless/01016_input_null_as_default.reference @@ -18,6 +18,11 @@ JSONEachRow 1 world 3 2019-07-23 [1,2,3] ('tuple',3.14) 2 Hello 123 2019-06-19 [] ('test',2.71828) 3 Hello 42 2019-06-19 [1,2,3] ('default',0.75) +JSONStringsEachRow +0 1 42 2019-07-22 [10,20,30] ('default',0) +1 world 3 2019-07-23 [1,2,3] ('tuple',3.14) +2 Hello 123 2019-06-19 [] ('test',2.71828) +3 Hello 42 2019-06-19 [1,2,3] ('default',0.75) Template (Quoted) 0 1 42 2019-07-22 [10,20,30] ('default',0) 1 world 3 2019-07-23 [1,2,3] ('tuple',3.14) diff --git a/tests/queries/0_stateless/01016_input_null_as_default.sh b/tests/queries/0_stateless/01016_input_null_as_default.sh index a40287eaba8..f31e6591e97 100755 --- a/tests/queries/0_stateless/01016_input_null_as_default.sh +++ b/tests/queries/0_stateless/01016_input_null_as_default.sh @@ -38,6 +38,14 @@ echo '{"i": null, "s": "1", "n": null, "d": "2019-07-22", "a": [10, 20, 30], "t" $CLICKHOUSE_CLIENT --query="SELECT * FROM null_as_default ORDER BY i"; $CLICKHOUSE_CLIENT --query="TRUNCATE TABLE null_as_default"; +echo 'JSONStringsEachRow' +echo '{"i": "null", "s": "1", "n": "ᴺᵁᴸᴸ", "d": "2019-07-22", "a": "[10, 20, 30]", "t": "NULL"} +{"i": "1", "s": "world", "n": "3", "d": "2019-07-23", "a": "null", "t": "('\''tuple'\'', 3.14)"} +{"i": "2", "s": "null", "n": "123", "d": "null", "a": "[]", "t": "('\''test'\'', 2.71828)"} +{"i": "3", "s": "null", "n": "null", "d": "null", "a": "null", "t": "null"}' | $CLICKHOUSE_CLIENT --input_format_null_as_default=1 --query="INSERT INTO null_as_default FORMAT JSONStringsEachRow"; +$CLICKHOUSE_CLIENT --query="SELECT * FROM null_as_default ORDER BY i"; +$CLICKHOUSE_CLIENT --query="TRUNCATE TABLE null_as_default"; + echo 'Template (Quoted)' echo 'NULL, '\''1'\'', null, '\''2019-07-22'\'', [10, 20, 30], NuLl 1, '\''world'\'', 3, '\''2019-07-23'\'', NULL, ('\''tuple'\'', 3.14) From 2ea59cb0c2fc9ea25cb6029f910952903e1d0bbd Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 9 Sep 2020 17:54:41 +0800 Subject: [PATCH 053/123] Fix tests --- .../0_stateless/01446_json_strings_each_row.reference | 4 ++-- .../01448_json_compact_strings_each_row.reference | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01446_json_strings_each_row.reference b/tests/queries/0_stateless/01446_json_strings_each_row.reference index 84d41095b77..812026534ea 100644 --- a/tests/queries/0_stateless/01446_json_strings_each_row.reference +++ b/tests/queries/0_stateless/01446_json_strings_each_row.reference @@ -16,7 +16,7 @@ {"v1":"first","v2":"1","v3":"2","v4":"0"} {"v1":"second","v2":"2","v3":"0","v4":"6"} 6 -{"v1":"first","v2":"1","v3":"2","v4":"0"} -{"v1":"second","v2":"2","v3":"0","v4":"6"} +{"v1":"first","v2":"1","v3":"2","v4":"8"} +{"v1":"second","v2":"2","v3":"32","v4":"6"} 7 {"v1":"16","n.id":"[15,16,17]","n.name":"['first','second','third']"} diff --git a/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference b/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference index 0b05f050b29..fb1a066f272 100644 --- a/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference +++ b/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference @@ -24,16 +24,16 @@ ["first", "1", "2", "0"] ["second", "2", "0", "6"] 6 -["first", "1", "2", "0"] -["second", "2", "0", "6"] +["first", "1", "2", "8"] +["second", "2", "32", "6"] 7 ["16", "[15,16,17]", "['first','second','third']"] 8 ["first", "1", "2", "0"] ["second", "2", "0", "6"] 9 -["first", "1", "2", "0"] -["second", "2", "0", "6"] +["first", "1", "2", "8"] +["second", "2", "32", "6"] 10 ["first", "1", "16", "8"] ["second", "2", "32", "8"] From 83ec93dec868a1e17950f6298b5ec1ce0d5352db Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 9 Sep 2020 16:46:59 +0300 Subject: [PATCH 054/123] Fuzzer: reset default database before reconnect --- programs/client/Client.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index c9701950dc5..99598c70397 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -866,6 +866,8 @@ private: // will exit. The ping() would be the best match here, but it's // private, probably for a good reason that the protocol doesn't allow // pings at any possible moment. + // Don't forget to reset the default database which might have changed. + connection->setDefaultDatabase(""); connection->forceConnected(connection_parameters.timeouts); if (text.size() > 4 * 1024) @@ -1103,7 +1105,9 @@ private: { last_exception_received_from_server = std::make_unique(getCurrentExceptionMessage(true), getCurrentExceptionCode()); received_exception_from_server = true; - std::cerr << "Error on processing query: " << ast_to_process->formatForErrorMessage() << std::endl << last_exception_received_from_server->message(); + fmt::print(stderr, "Error on processing query '{}': {}\n", + ast_to_process->formatForErrorMessage(), + last_exception_received_from_server->message()); } if (!connection->isConnected()) From 673244876ea15008b54bf93e9d535a0a65e68696 Mon Sep 17 00:00:00 2001 From: Mikhail Cheshkov Date: Wed, 9 Sep 2020 17:57:15 +0300 Subject: [PATCH 055/123] Use global ICU ADDINCL for Arcadia build --- src/Columns/ya.make | 2 -- src/Functions/ya.make | 1 - src/Functions/ya.make.in | 1 - 3 files changed, 4 deletions(-) diff --git a/src/Columns/ya.make b/src/Columns/ya.make index 78c0e1b992d..910c479c2a9 100644 --- a/src/Columns/ya.make +++ b/src/Columns/ya.make @@ -2,8 +2,6 @@ LIBRARY() ADDINCL( - contrib/libs/icu/common - contrib/libs/icu/i18n contrib/libs/pdqsort ) diff --git a/src/Functions/ya.make b/src/Functions/ya.make index b9a7b5b64ea..f48b4d607ed 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -10,7 +10,6 @@ ADDINCL( contrib/libs/farmhash contrib/libs/h3/h3lib/include contrib/libs/hyperscan/src - contrib/libs/icu/common contrib/libs/libdivide contrib/libs/rapidjson/include contrib/libs/xxhash diff --git a/src/Functions/ya.make.in b/src/Functions/ya.make.in index a5a54d94c6c..2a66aa5553e 100644 --- a/src/Functions/ya.make.in +++ b/src/Functions/ya.make.in @@ -9,7 +9,6 @@ ADDINCL( contrib/libs/farmhash contrib/libs/h3/h3lib/include contrib/libs/hyperscan/src - contrib/libs/icu/common contrib/libs/libdivide contrib/libs/rapidjson/include contrib/libs/xxhash From d1d3af7501a35629d48b046387dfb95e0731f657 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Wed, 9 Sep 2020 00:22:24 -0700 Subject: [PATCH 056/123] StorageReplicatedMergeTree - more improvements to integration test --- .../__init__.py | 0 .../test.py | 24 +++++++------------ 2 files changed, 9 insertions(+), 15 deletions(-) rename tests/integration/{test_replicated_zk_conn_failure => test_cleanup_dir_after_bad_zk_conn}/__init__.py (100%) rename tests/integration/{test_replicated_zk_conn_failure => test_cleanup_dir_after_bad_zk_conn}/test.py (63%) diff --git a/tests/integration/test_replicated_zk_conn_failure/__init__.py b/tests/integration/test_cleanup_dir_after_bad_zk_conn/__init__.py similarity index 100% rename from tests/integration/test_replicated_zk_conn_failure/__init__.py rename to tests/integration/test_cleanup_dir_after_bad_zk_conn/__init__.py diff --git a/tests/integration/test_replicated_zk_conn_failure/test.py b/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py similarity index 63% rename from tests/integration/test_replicated_zk_conn_failure/test.py rename to tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py index a860716ee82..ef31e8487be 100644 --- a/tests/integration/test_replicated_zk_conn_failure/test.py +++ b/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py @@ -9,12 +9,12 @@ from helpers.network import PartitionManager # Test flow is as follows: # 1. Configure cluster with ZooKeeper and create a database. # 2. Drop all connections to ZooKeeper. -# 3. Try creating the table and there would be a Poco:Exception. +# 3. Try creating the table and there will be a Poco:Exception. # 4. Try creating the table again and there should not be any error -# that indicates that the Directory for table already exists. +# that indicates that the directory for table already exists. # 5. Final step is to restore ZooKeeper connection and verify that -# the table creation and queries work. -def test_replicated_zk_conn_failure(): +# the table creation works. +def test_cleanup_dir_after_bad_zk_conn(): cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', with_zookeeper=True) try: @@ -30,21 +30,15 @@ def test_replicated_zk_conn_failure(): ORDER BY id;''' with PartitionManager() as pm: pm.drop_instance_zk_connections(node1) - time.sleep(5) + time.sleep(3) error = node1.query_and_get_error(query_create) - # Assert that there was net exception. - assert "Poco::Exception. Code: 1000" in error - # Assert that the exception was due to ZooKeeper connectivity. - assert "All connection tries failed while connecting to ZooKeeper" in error - # retry table creation + assert "Poco::Exception. Code: 1000" and \ + "All connection tries failed while connecting to ZooKeeper" in error error = node1.query_and_get_error(query_create) - # Should not expect any errors related to directory already existing - # and those should have been already cleaned up during the previous retry. assert "Directory for table data data/replica/test/ already exists" not in error - # restore ZooKeeper connections. pm.restore_instance_zk_connections(node1) - # retry create query and query the table created. node1.query(query_create) - assert "0\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') + node1.query('''INSERT INTO replica.test VALUES (1, now())''') + assert "1\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') finally: cluster.shutdown() From 62428845a0fdcaaa19ecc5fd33f3ecd849104cf5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Sep 2020 19:47:06 +0300 Subject: [PATCH 057/123] Bug in mutation --- src/Columns/ColumnVector.h | 7 ++++--- .../0_stateless/01475_mutation_with_if.reference | 1 + .../0_stateless/01475_mutation_with_if.sql | 16 ++++++++++++++++ 3 files changed, 21 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01475_mutation_with_if.reference create mode 100644 tests/queries/0_stateless/01475_mutation_with_if.sql diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 1090de556a0..55ab67d6214 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -130,7 +131,7 @@ public: void insertFrom(const IColumn & src, size_t n) override { - data.push_back(static_cast(src).getData()[n]); + data.push_back(assert_cast(src).getData()[n]); } void insertData(const char * pos, size_t) override @@ -205,14 +206,14 @@ public: /// This method implemented in header because it could be possibly devirtualized. int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override { - return CompareHelper::compare(data[n], static_cast(rhs_).data[m], nan_direction_hint); + return CompareHelper::compare(data[n], assert_cast(rhs_).data[m], nan_direction_hint); } void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override { - return this->template doCompareColumn(static_cast(rhs), rhs_row_num, row_indexes, + return this->template doCompareColumn(assert_cast(rhs), rhs_row_num, row_indexes, compare_results, direction, nan_direction_hint); } diff --git a/tests/queries/0_stateless/01475_mutation_with_if.reference b/tests/queries/0_stateless/01475_mutation_with_if.reference new file mode 100644 index 00000000000..2874a18147f --- /dev/null +++ b/tests/queries/0_stateless/01475_mutation_with_if.reference @@ -0,0 +1 @@ +1 150 diff --git a/tests/queries/0_stateless/01475_mutation_with_if.sql b/tests/queries/0_stateless/01475_mutation_with_if.sql new file mode 100644 index 00000000000..6f0ef8924be --- /dev/null +++ b/tests/queries/0_stateless/01475_mutation_with_if.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS mutation_table; +CREATE TABLE mutation_table ( + id int, + price Nullable(Int32) +) +ENGINE = MergeTree() +PARTITION BY id +ORDER BY id; + +INSERT INTO mutation_table (id, price) VALUES (1, 100); + +ALTER TABLE mutation_table UPDATE price = 150 WHERE id = 1 SETTINGS mutations_sync = 2; + +SELECT * FROM mutation_table; + +DROP TABLE IF EXISTS mutation_table; From 9fa04cf48b1f2aa9288dbd026e37b7aae9e8a6f3 Mon Sep 17 00:00:00 2001 From: Peng Jian Date: Thu, 10 Sep 2020 00:59:38 +0800 Subject: [PATCH 058/123] Add QueryMemoryLimitExceeded event --- src/Common/MemoryTracker.cpp | 7 +++++++ src/Common/ProfileEvents.cpp | 1 + 2 files changed, 8 insertions(+) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 9d073cf8dd8..5d51fc9f301 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -22,6 +23,10 @@ namespace DB } } +namespace ProfileEvents +{ + extern const Event QueryMemoryLimitExceeded; +} static constexpr size_t log_peak_memory_usage_every = 1ULL << 30; @@ -104,6 +109,7 @@ void MemoryTracker::alloc(Int64 size) /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc auto untrack_lock = blocker.cancel(); // NOLINT + ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); std::stringstream message; message << "Memory tracker"; if (const auto * description = description_ptr.load(std::memory_order_relaxed)) @@ -136,6 +142,7 @@ void MemoryTracker::alloc(Int64 size) /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc auto no_track = blocker.cancel(); // NOLINT + ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); std::stringstream message; message << "Memory limit"; if (const auto * description = description_ptr.load(std::memory_order_relaxed)) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 475e073d253..c9ff9642361 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -233,6 +233,7 @@ M(S3WriteRequestsErrors, "Number of non-throttling errors in POST, DELETE, PUT and PATCH requests to S3 storage.") \ M(S3WriteRequestsThrottling, "Number of 429 and 503 errors in POST, DELETE, PUT and PATCH requests to S3 storage.") \ M(S3WriteRequestsRedirects, "Number of redirects in POST, DELETE, PUT and PATCH requests to S3 storage.") \ + M(QueryMemoryLimitExceeded, "Number of times of memory limit exceeded for query.") \ namespace ProfileEvents From dee1fefeb472bebdfa4e15eb1b7ec7e428069f8a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 9 Sep 2020 20:39:49 +0300 Subject: [PATCH 059/123] add more tests --- src/Storages/StorageReplicatedMergeTree.cpp | 5 +- .../test.py | 74 ++++++++++++------- 2 files changed, 50 insertions(+), 29 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a1027a8be53..00dcc7aeb08 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -612,7 +612,10 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr return true; } - throw Exception("Cannot create table, because it is created concurrently every time or because of logical error", ErrorCodes::LOGICAL_ERROR); + /// Do not use LOGICAL_ERROR code, because it may happen if user has specified wrong zookeeper_path + throw Exception("Cannot create table, because it is created concurrently every time " + "or because of wrong zookeeper_path " + "or because of logical error", ErrorCodes::REPLICA_IS_ALREADY_EXIST); } void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metadata_snapshot) diff --git a/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py b/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py index ef31e8487be..4cb243160a0 100644 --- a/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py +++ b/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py @@ -1,9 +1,21 @@ import time +import pytest from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', with_zookeeper=True) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + # This tests if the data directory for a table is cleaned up if there is a Zookeeper # connection exception during a CreateQuery operation involving ReplicatedMergeTree tables. # Test flow is as follows: @@ -14,31 +26,37 @@ from helpers.network import PartitionManager # that indicates that the directory for table already exists. # 5. Final step is to restore ZooKeeper connection and verify that # the table creation works. -def test_cleanup_dir_after_bad_zk_conn(): - cluster = ClickHouseCluster(__file__) - node1 = cluster.add_instance('node1', with_zookeeper=True) - try: - cluster.start() - node1.query("CREATE DATABASE replica;") - query_create = '''CREATE TABLE replica.test - ( - id Int64, - event_time DateTime - ) - Engine=ReplicatedMergeTree('/clickhouse/tables/replica/test', 'node1') - PARTITION BY toYYYYMMDD(event_time) - ORDER BY id;''' - with PartitionManager() as pm: - pm.drop_instance_zk_connections(node1) - time.sleep(3) - error = node1.query_and_get_error(query_create) - assert "Poco::Exception. Code: 1000" and \ - "All connection tries failed while connecting to ZooKeeper" in error - error = node1.query_and_get_error(query_create) - assert "Directory for table data data/replica/test/ already exists" not in error - pm.restore_instance_zk_connections(node1) - node1.query(query_create) - node1.query('''INSERT INTO replica.test VALUES (1, now())''') - assert "1\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') - finally: - cluster.shutdown() +def test_cleanup_dir_after_bad_zk_conn(start_cluster): + node1.query("CREATE DATABASE replica;") + query_create = '''CREATE TABLE replica.test + ( + id Int64, + event_time DateTime + ) + Engine=ReplicatedMergeTree('/clickhouse/tables/replica/test', 'node1') + PARTITION BY toYYYYMMDD(event_time) + ORDER BY id;''' + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node1) + time.sleep(3) + error = node1.query_and_get_error(query_create) + assert "Poco::Exception. Code: 1000" and \ + "All connection tries failed while connecting to ZooKeeper" in error + error = node1.query_and_get_error(query_create) + assert "Directory for table data data/replica/test/ already exists" not in error + node1.query(query_create) + node1.query('''INSERT INTO replica.test VALUES (1, now())''') + assert "1\n" in node1.query('''SELECT count() from replica.test FORMAT TSV''') + +def test_cleanup_dir_after_wrong_replica_name(start_cluster): + node1.query("CREATE TABLE test2_r1 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test2/', 'r1') ORDER BY n") + error = node1.query_and_get_error("CREATE TABLE test2_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test2/', 'r1') ORDER BY n") + assert "already exists" in error + node1.query("CREATE TABLE test_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test2/', 'r2') ORDER BY n") + + +def test_cleanup_dir_after_wrong_zk_path(start_cluster): + node1.query("CREATE TABLE test3_r1 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test3/', 'r1') ORDER BY n") + error = node1.query_and_get_error("CREATE TABLE test3_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/', 'r2') ORDER BY n") + assert "Cannot create" in error + node1.query("CREATE TABLE test3_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test3/', 'r2') ORDER BY n") From 453914b6b860dc558719269ab9af35828376fac7 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 9 Sep 2020 21:02:53 +0300 Subject: [PATCH 060/123] Update ProfileEvents.cpp --- src/Common/ProfileEvents.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index c9ff9642361..486cb7e1a6e 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -233,7 +233,7 @@ M(S3WriteRequestsErrors, "Number of non-throttling errors in POST, DELETE, PUT and PATCH requests to S3 storage.") \ M(S3WriteRequestsThrottling, "Number of 429 and 503 errors in POST, DELETE, PUT and PATCH requests to S3 storage.") \ M(S3WriteRequestsRedirects, "Number of redirects in POST, DELETE, PUT and PATCH requests to S3 storage.") \ - M(QueryMemoryLimitExceeded, "Number of times of memory limit exceeded for query.") \ + M(QueryMemoryLimitExceeded, "Number of times when memory limit exceeded for query.") \ namespace ProfileEvents From 11ffbda7cc48082ec0c3a44dc34946cf03a9dc4a Mon Sep 17 00:00:00 2001 From: myrrc Date: Wed, 9 Sep 2020 21:17:01 +0300 Subject: [PATCH 061/123] 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 2a9ab482792cdadf0d4e2365c3d11494a3e38230 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 8 Sep 2020 02:08:17 +0300 Subject: [PATCH 062/123] Use join() instead of detach() for the no_users_thread in StorageLiveView. --- src/Interpreters/Context.cpp | 8 + src/Interpreters/Context.h | 8 +- src/Interpreters/InterpreterDropQuery.h | 1 + .../LiveView/LiveViewBlockInputStream.h | 15 +- .../LiveView/LiveViewEventsBlockInputStream.h | 14 +- src/Storages/LiveView/StorageLiveView.cpp | 144 +---------------- src/Storages/LiveView/StorageLiveView.h | 23 ++- .../LiveView/TemporaryLiveViewCleaner.cpp | 148 ++++++++++++++++++ .../LiveView/TemporaryLiveViewCleaner.h | 51 ++++++ src/Storages/ya.make | 1 + 10 files changed, 233 insertions(+), 180 deletions(-) create mode 100644 src/Storages/LiveView/TemporaryLiveViewCleaner.cpp create mode 100644 src/Storages/LiveView/TemporaryLiveViewCleaner.h diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 70cf41a679c..3c4c095cc26 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -423,6 +424,7 @@ struct ContextShared if (system_logs) system_logs->shutdown(); + TemporaryLiveViewCleaner::shutdown(); DatabaseCatalog::shutdown(); /// Preemptive destruction is important, because these objects may have a refcount to ContextShared (cyclic reference). @@ -479,6 +481,12 @@ Context Context::createGlobal(ContextShared * shared) return res; } +void Context::initGlobal() +{ + DatabaseCatalog::init(this); + TemporaryLiveViewCleaner::init(*this); +} + SharedContextHolder Context::createShared() { return SharedContextHolder(std::make_unique()); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c8d13baa9ae..743c92d56b5 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -445,11 +445,7 @@ public: void makeQueryContext() { query_context = this; } void makeSessionContext() { session_context = this; } - void makeGlobalContext() - { - global_context = this; - DatabaseCatalog::init(this); - } + void makeGlobalContext() { initGlobal(); global_context = this; } const Settings & getSettingsRef() const { return settings; } @@ -622,6 +618,8 @@ public: private: std::unique_lock getLock() const; + void initGlobal(); + /// Compute and set actual user settings, client_info.current_user should be set void calculateAccessRights(); diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index 80bd6c6531a..b54736b5c21 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -10,6 +10,7 @@ namespace DB { class Context; using DatabaseAndTable = std::pair; +class AccessRightsElements; /** Allow to either drop table with all its data (DROP), * or remove information about table (just forget) from server (DETACH), diff --git a/src/Storages/LiveView/LiveViewBlockInputStream.h b/src/Storages/LiveView/LiveViewBlockInputStream.h index 7cab2cb41ed..737e76754c5 100644 --- a/src/Storages/LiveView/LiveViewBlockInputStream.h +++ b/src/Storages/LiveView/LiveViewBlockInputStream.h @@ -16,27 +16,17 @@ class LiveViewBlockInputStream : public IBlockInputStream using NonBlockingResult = std::pair; public: - ~LiveViewBlockInputStream() override - { - /// Start storage no users thread - /// if we are the last active user - if (!storage->is_dropped && blocks_ptr.use_count() < 3) - storage->startNoUsersThread(temporary_live_view_timeout_sec); - } - LiveViewBlockInputStream(std::shared_ptr storage_, std::shared_ptr blocks_ptr_, std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, const bool has_limit_, const UInt64 limit_, - const UInt64 heartbeat_interval_sec_, - const UInt64 temporary_live_view_timeout_sec_) + const UInt64 heartbeat_interval_sec_) : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), active_ptr(std::move(active_ptr_)), has_limit(has_limit_), limit(limit_), - heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000), - temporary_live_view_timeout_sec(temporary_live_view_timeout_sec_) + heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000) { /// grab active pointer active = active_ptr.lock(); @@ -205,7 +195,6 @@ private: Int64 num_updates = -1; bool end_of_blocks = false; UInt64 heartbeat_interval_usec; - UInt64 temporary_live_view_timeout_sec; UInt64 last_event_timestamp_usec = 0; }; diff --git a/src/Storages/LiveView/LiveViewEventsBlockInputStream.h b/src/Storages/LiveView/LiveViewEventsBlockInputStream.h index ac5e7e3d6fd..4060b17c1ed 100644 --- a/src/Storages/LiveView/LiveViewEventsBlockInputStream.h +++ b/src/Storages/LiveView/LiveViewEventsBlockInputStream.h @@ -34,13 +34,6 @@ class LiveViewEventsBlockInputStream : public IBlockInputStream using NonBlockingResult = std::pair; public: - ~LiveViewEventsBlockInputStream() override - { - /// Start storage no users thread - /// if we are the last active user - if (!storage->is_dropped && blocks_ptr.use_count() < 3) - storage->startNoUsersThread(temporary_live_view_timeout_sec); - } /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update /// and LIMIT 0 just returns data without waiting for any updates LiveViewEventsBlockInputStream(std::shared_ptr storage_, @@ -48,14 +41,12 @@ public: std::shared_ptr blocks_metadata_ptr_, std::shared_ptr active_ptr_, const bool has_limit_, const UInt64 limit_, - const UInt64 heartbeat_interval_sec_, - const UInt64 temporary_live_view_timeout_sec_) + const UInt64 heartbeat_interval_sec_) : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), active_ptr(std::move(active_ptr_)), has_limit(has_limit_), limit(limit_), - heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000), - temporary_live_view_timeout_sec(temporary_live_view_timeout_sec_) + heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000) { /// grab active pointer active = active_ptr.lock(); @@ -236,7 +227,6 @@ private: Int64 num_updates = -1; bool end_of_blocks = false; UInt64 heartbeat_interval_usec; - UInt64 temporary_live_view_timeout_sec; UInt64 last_event_timestamp_usec = 0; Poco::Timestamp timestamp; }; diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 54ac5bcc791..b16c02eec6b 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -12,10 +12,8 @@ limitations under the License. */ #include #include #include -#include #include #include -#include #include #include #include @@ -31,6 +29,7 @@ limitations under the License. */ #include #include #include +#include #include #include @@ -276,7 +275,7 @@ StorageLiveView::StorageLiveView( if (query.live_view_timeout) { is_temporary = true; - temporary_live_view_timeout = *query.live_view_timeout; + temporary_live_view_timeout = std::chrono::seconds{*query.live_view_timeout}; } blocks_ptr = std::make_shared(); @@ -384,128 +383,21 @@ void StorageLiveView::checkTableCanBeDropped() const } } -void StorageLiveView::noUsersThread(std::shared_ptr storage, const UInt64 & timeout) -{ - bool drop_table = false; - - if (storage->shutdown_called) - return; - - auto table_id = storage->getStorageID(); - { - while (true) - { - std::unique_lock lock(storage->no_users_thread_wakeup_mutex); - if (!storage->no_users_thread_condition.wait_for(lock, std::chrono::seconds(timeout), [&] { return storage->no_users_thread_wakeup; })) - { - storage->no_users_thread_wakeup = false; - if (storage->shutdown_called) - return; - if (storage->hasUsers()) - return; - if (!DatabaseCatalog::instance().getDependencies(table_id).empty()) - continue; - drop_table = true; - } - break; - } - } - - if (drop_table) - { - if (DatabaseCatalog::instance().tryGetTable(table_id, storage->global_context)) - { - try - { - /// We create and execute `drop` query for this table - auto drop_query = std::make_shared(); - drop_query->database = table_id.database_name; - drop_query->table = table_id.table_name; - drop_query->kind = ASTDropQuery::Kind::Drop; - ASTPtr ast_drop_query = drop_query; - InterpreterDropQuery drop_interpreter(ast_drop_query, storage->global_context); - drop_interpreter.execute(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - } -} - -void StorageLiveView::startNoUsersThread(const UInt64 & timeout) -{ - bool expected = false; - if (!start_no_users_thread_called.compare_exchange_strong(expected, true)) - return; - - if (is_temporary) - { - std::lock_guard no_users_thread_lock(no_users_thread_mutex); - - if (shutdown_called) - return; - - if (no_users_thread.joinable()) - { - { - std::lock_guard lock(no_users_thread_wakeup_mutex); - no_users_thread_wakeup = true; - no_users_thread_condition.notify_one(); - } - no_users_thread.join(); - } - { - std::lock_guard lock(no_users_thread_wakeup_mutex); - no_users_thread_wakeup = false; - } - if (!is_dropped) - no_users_thread = std::thread(&StorageLiveView::noUsersThread, - std::static_pointer_cast(shared_from_this()), timeout); - } - - start_no_users_thread_called = false; -} - void StorageLiveView::startup() { - startNoUsersThread(temporary_live_view_timeout); + if (is_temporary) + TemporaryLiveViewCleaner::instance().addView(std::static_pointer_cast(shared_from_this())); } void StorageLiveView::shutdown() { + shutdown_called = true; DatabaseCatalog::instance().removeDependency(select_table_id, getStorageID()); - bool expected = false; - if (!shutdown_called.compare_exchange_strong(expected, true)) - return; - - /// WATCH queries should be stopped after setting shutdown_called to true. - /// Otherwise livelock is possible for LiveView table in Atomic database: - /// WATCH query will wait for table to be dropped and DatabaseCatalog will wait for queries to finish - - { - std::lock_guard no_users_thread_lock(no_users_thread_mutex); - if (no_users_thread.joinable()) - { - { - std::lock_guard lock(no_users_thread_wakeup_mutex); - no_users_thread_wakeup = true; - no_users_thread_condition.notify_one(); - } - } - } } StorageLiveView::~StorageLiveView() { shutdown(); - - { - std::lock_guard lock(no_users_thread_mutex); - if (no_users_thread.joinable()) - no_users_thread.detach(); - } } void StorageLiveView::drop() @@ -572,18 +464,7 @@ BlockInputStreams StorageLiveView::watch( auto reader = std::make_shared( std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, - context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), - temporary_live_view_timeout); - - { - std::lock_guard no_users_thread_lock(no_users_thread_mutex); - if (no_users_thread.joinable()) - { - std::lock_guard lock(no_users_thread_wakeup_mutex); - no_users_thread_wakeup = true; - no_users_thread_condition.notify_one(); - } - } + context.getSettingsRef().live_view_heartbeat_interval.totalSeconds()); { std::lock_guard lock(mutex); @@ -603,18 +484,7 @@ BlockInputStreams StorageLiveView::watch( auto reader = std::make_shared( std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, - context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), - temporary_live_view_timeout); - - { - std::lock_guard no_users_thread_lock(no_users_thread_mutex); - if (no_users_thread.joinable()) - { - std::lock_guard lock(no_users_thread_wakeup_mutex); - no_users_thread_wakeup = true; - no_users_thread_condition.notify_one(); - } - } + context.getSettingsRef().live_view_heartbeat_interval.totalSeconds()); { std::lock_guard lock(mutex); diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 43afd169a92..32e18ef6092 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -38,6 +38,10 @@ using ASTPtr = std::shared_ptr; using BlocksMetadataPtr = std::shared_ptr; using MergeableBlocksPtr = std::shared_ptr; +class Pipe; +using Pipes = std::vector; + + class StorageLiveView final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; @@ -70,7 +74,9 @@ public: NamesAndTypesList getVirtuals() const override; - bool isTemporary() { return is_temporary; } + bool isTemporary() const { return is_temporary; } + std::chrono::seconds getTimeout() const { return temporary_live_view_timeout; } + /// Check if we have any readers /// must be called with mutex locked @@ -85,11 +91,7 @@ public: { return active_ptr.use_count() > 1; } - /// No users thread mutex, predicate and wake up condition - void startNoUsersThread(const UInt64 & timeout); - std::mutex no_users_thread_wakeup_mutex; - bool no_users_thread_wakeup = false; - std::condition_variable no_users_thread_condition; + /// Get blocks hash /// must be called with mutex locked String getBlocksHashKey() @@ -175,6 +177,8 @@ private: std::unique_ptr live_view_context; bool is_temporary = false; + std::chrono::seconds temporary_live_view_timeout; + /// Mutex to protect access to sample block and inner_blocks_query mutable std::mutex sample_block_lock; mutable Block sample_block; @@ -193,14 +197,7 @@ private: std::shared_ptr blocks_metadata_ptr; MergeableBlocksPtr mergeable_blocks; - /// Background thread for temporary tables - /// which drops this table if there are no users - static void noUsersThread(std::shared_ptr storage, const UInt64 & timeout); - std::mutex no_users_thread_mutex; - std::thread no_users_thread; std::atomic shutdown_called = false; - std::atomic start_no_users_thread_called = false; - UInt64 temporary_live_view_timeout; StorageLiveView( const StorageID & table_id_, diff --git a/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp b/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp new file mode 100644 index 00000000000..0f7c1039d72 --- /dev/null +++ b/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp @@ -0,0 +1,148 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + + +namespace +{ + void executeDropQuery(const StorageID & storage_id, Context & context) + { + if (!DatabaseCatalog::instance().isTableExist(storage_id, context)) + return; + try + { + /// We create and execute `drop` query for this table + auto drop_query = std::make_shared(); + drop_query->database = storage_id.database_name; + drop_query->table = storage_id.table_name; + drop_query->kind = ASTDropQuery::Kind::Drop; + ASTPtr ast_drop_query = drop_query; + InterpreterDropQuery drop_interpreter(ast_drop_query, context); + drop_interpreter.execute(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } +} + + +std::unique_ptr TemporaryLiveViewCleaner::the_instance; + + +void TemporaryLiveViewCleaner::init(Context & global_context_) +{ + if (the_instance) + throw Exception("TemporaryLiveViewCleaner already initialized", ErrorCodes::LOGICAL_ERROR); + the_instance.reset(new TemporaryLiveViewCleaner(global_context_)); +} + + +void TemporaryLiveViewCleaner::shutdown() +{ + the_instance.reset(); +} + + +TemporaryLiveViewCleaner::TemporaryLiveViewCleaner(Context & global_context_) + : global_context(global_context_) +{ +} + + +TemporaryLiveViewCleaner::~TemporaryLiveViewCleaner() +{ + stopBackgroundThread(); +} + + +void TemporaryLiveViewCleaner::addView(const std::shared_ptr & view) +{ + if (!view->isTemporary()) + return; + + auto current_time = std::chrono::system_clock::now(); + auto time_of_next_check = current_time + view->getTimeout(); + + std::lock_guard lock{mutex}; + + /// Keep the vector `views` sorted by time of next check. + StorageAndTimeOfCheck storage_and_time_of_check{view, time_of_next_check}; + views.insert(std::upper_bound(views.begin(), views.end(), storage_and_time_of_check), storage_and_time_of_check); + + if (!background_thread.joinable()) + background_thread = ThreadFromGlobalPool{&TemporaryLiveViewCleaner::backgroundThreadFunc, this}; + + background_thread_wake_up.notify_one(); +} + + +void TemporaryLiveViewCleaner::backgroundThreadFunc() +{ + std::unique_lock lock{mutex}; + while (!background_thread_should_exit && !views.empty()) + { + background_thread_wake_up.wait_until(lock, views.front().time_of_check); + if (background_thread_should_exit) + return; + + auto current_time = std::chrono::system_clock::now(); + std::vector storages_to_drop; + + auto it = views.begin(); + while (it != views.end()) + { + std::shared_ptr storage = it->storage.lock(); + auto & time_of_check = it->time_of_check; + if (!storage) + { + /// Storage has been already removed. + it = views.erase(it); + continue; + } + + ++it; + + if (current_time < time_of_check) + break; /// It's not the time to check it yet. + + time_of_check = current_time + storage->getTimeout(); + + auto storage_id = storage->getStorageID(); + if (storage->hasUsers() || !DatabaseCatalog::instance().getDependencies(storage_id).empty()) + continue; + + storages_to_drop.emplace_back(storage_id); + } + + lock.unlock(); + for (const auto & storage_id : storages_to_drop) + executeDropQuery(storage_id, global_context); + lock.lock(); + } +} + + +void TemporaryLiveViewCleaner::stopBackgroundThread() +{ + std::lock_guard lock{mutex}; + if (background_thread.joinable()) + { + background_thread_should_exit = true; + background_thread_wake_up.notify_one(); + background_thread.join(); + } +} + +} diff --git a/src/Storages/LiveView/TemporaryLiveViewCleaner.h b/src/Storages/LiveView/TemporaryLiveViewCleaner.h new file mode 100644 index 00000000000..57c12bd1c07 --- /dev/null +++ b/src/Storages/LiveView/TemporaryLiveViewCleaner.h @@ -0,0 +1,51 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class StorageLiveView; +struct StorageID; + +/// This class removes temporary live views in the background thread when it's possible. +/// There should only a single instance of this class. +class TemporaryLiveViewCleaner +{ +public: + static TemporaryLiveViewCleaner & instance() { return *the_instance; } + + /// Drops a specified live view after a while if it's temporary. + void addView(const std::shared_ptr & view); + + /// Should be called once. + static void init(Context & global_context_); + static void shutdown(); + +private: + friend std::unique_ptr::deleter_type; + + TemporaryLiveViewCleaner(Context & global_context_); + ~TemporaryLiveViewCleaner(); + + void backgroundThreadFunc(); + void stopBackgroundThread(); + + struct StorageAndTimeOfCheck + { + std::weak_ptr storage; + std::chrono::system_clock::time_point time_of_check; + bool operator <(const StorageAndTimeOfCheck & other) const { return time_of_check < other.time_of_check; } + }; + + static std::unique_ptr the_instance; + Context & global_context; + std::mutex mutex; + std::vector views; + ThreadFromGlobalPool background_thread; + std::atomic background_thread_should_exit = false; + std::condition_variable background_thread_wake_up; +}; + +} diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 1ddb8c77072..fed961ed2bb 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -20,6 +20,7 @@ SRCS( IStorage.cpp KeyDescription.cpp LiveView/StorageLiveView.cpp + LiveView/TemporaryLiveViewCleaner.cpp MergeTree/ActiveDataPartSet.cpp MergeTree/AllMergeSelector.cpp MergeTree/BackgroundProcessingPool.cpp From 7080d56470a9bfed57dde63ef2e40c9832599c72 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 9 Sep 2020 22:59:34 +0300 Subject: [PATCH 063/123] fix missed database name when altering mv --- src/Storages/SelectQueryDescription.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/SelectQueryDescription.cpp b/src/Storages/SelectQueryDescription.cpp index bb8295df6f3..0935a5be5ca 100644 --- a/src/Storages/SelectQueryDescription.cpp +++ b/src/Storages/SelectQueryDescription.cpp @@ -105,14 +105,14 @@ SelectQueryDescription SelectQueryDescription::getSelectQueryFromASTForMatView(c if (new_select.list_of_selects->children.size() != 1) throw Exception("UNION is not supported for MATERIALIZED VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW); - SelectQueryDescription result; - - result.inner_query = new_select.list_of_selects->children.at(0)->clone(); - - auto & select_query = result.inner_query->as(); + auto & new_inner_query = new_select.list_of_selects->children.at(0); + auto & select_query = new_inner_query->as(); checkAllowedQueries(select_query); + + SelectQueryDescription result; result.select_table_id = extractDependentTableFromSelectQuery(select_query, context); - result.select_query = select->clone(); + result.select_query = new_select.clone(); + result.inner_query = new_inner_query->clone(); return result; } From 27258c8e70213cf57e1bbf36176cda961d56e12f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 9 Sep 2020 23:47:42 +0300 Subject: [PATCH 064/123] utils/list-licenses/list-licenses.sh: ignore more files - *.rtf They can have NULL byte, and StorageSystemLicenses.sh will warn: ./StorageSystemLicenses.sh: line 11: warning: command substitution: ignored null byte in input Found with: find contrib/ -type f -and '(' -iname 'LICENSE*' -or -iname 'COPYING*' -or -iname 'COPYRIGHT*' ')' -and -not -iname '*.html' | xargs grep -Pa '\x00' - *.h - *.cpp - *.htm And after verified with: $ find contrib/ -type f -and '(' -iname 'LICENSE*' -or -iname 'COPYING*' -or -iname 'COPYRIGHT*' ')' -and -not '(' -iname '*.html' -or -iname '*.htm' -or -iname '*.rtf' -or -name '*.cpp' -or -name '*.h' -or -iname '*.json' ')' | xargs file -b | sort -u ASCII text ASCII text, with CR line terminators ASCII text, with very long lines empty UTF-8 Unicode text --- utils/list-licenses/list-licenses.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/list-licenses/list-licenses.sh b/utils/list-licenses/list-licenses.sh index 987179e26a8..8eee3f97253 100755 --- a/utils/list-licenses/list-licenses.sh +++ b/utils/list-licenses/list-licenses.sh @@ -7,7 +7,7 @@ ls -1 -d ${LIBS_PATH}/*/ | grep -F -v -- '-cmake' | while read LIB; do LIB_NAME=$(basename $LIB) LIB_LICENSE=$( - LC_ALL=C find "$LIB" -type f -and '(' -iname 'LICENSE*' -or -iname 'COPYING*' -or -iname 'COPYRIGHT*' ')' -and -not -iname '*.html' -printf "%d\t%p\n" | + LC_ALL=C find "$LIB" -type f -and '(' -iname 'LICENSE*' -or -iname 'COPYING*' -or -iname 'COPYRIGHT*' ')' -and -not '(' -iname '*.html' -or -iname '*.htm' -or -iname '*.rtf' -or -name '*.cpp' -or -name '*.h' -or -iname '*.json' ')' -printf "%d\t%p\n" | awk ' BEGIN { IGNORECASE=1; min_depth = 0 } /LICENSE/ { if (!min_depth || $1 <= min_depth) { min_depth = $1; license = $2 } } From 0f4fdcbf389909ed2e642263b0d6a65a3580d8e0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 10 Sep 2020 02:05:41 +0300 Subject: [PATCH 065/123] Pass -fsanitize-blacklist for TSAN only under clang (gcc does not support this) And no such check for -fsnaitize=memory, since gcc does not support it anyway. --- cmake/sanitize.cmake | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 32443ed78c3..7c7e9c388a0 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -36,7 +36,15 @@ if (SANITIZE) endif () elseif (SANITIZE STREQUAL "thread") - set (TSAN_FLAGS "-fsanitize=thread -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/tsan_suppressions.txt") + set (TSAN_FLAGS "-fsanitize=thread") + if (COMPILER_CLANG) + set (TSAN_FLAGS "${TSAN_FLAGS} -fsanitize-blacklist=${CMAKE_SOURCE_DIR}/tests/tsan_suppressions.txt") + else() + message (WARNING "TSAN suppressions was not passed to the compiler (since the compiler is not clang)") + message (WARNING "Use the following command to pass them manually:") + message (WARNING " export TSAN_OPTIONS=\"$TSAN_OPTIONS suppressions=${CMAKE_SOURCE_DIR}/tests/tsan_suppressions.txt\"") + endif() + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${TSAN_FLAGS}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${TSAN_FLAGS}") From 0a3f7d0fccc8b960d59b415b02313673d1bdc698 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 19:49:47 -0700 Subject: [PATCH 066/123] 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 067/123] 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 068/123] 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 7fb45461d174c3ae122c9d11ba5a4414fe0d3fd6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 10 Sep 2020 10:36:29 +0300 Subject: [PATCH 069/123] Try fix totals port for IAccumulatingTransform. --- src/Processors/IAccumulatingTransform.cpp | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Processors/IAccumulatingTransform.cpp b/src/Processors/IAccumulatingTransform.cpp index 2905d185df2..64bdbe2410f 100644 --- a/src/Processors/IAccumulatingTransform.cpp +++ b/src/Processors/IAccumulatingTransform.cpp @@ -49,13 +49,15 @@ IAccumulatingTransform::Status IAccumulatingTransform::prepare() return Status::Finished; } - /// Close input if flag was set manually. + if (input.isFinished()) + finished_input = true; + if (finished_input) + { + /// Close input if flag was set manually. input.close(); - /// Read from totals port if has it. - if (input.isFinished()) - { + /// Read from totals port if has it. if (inputs.size() > 1) { auto & totals_input = inputs.back(); @@ -69,12 +71,8 @@ IAccumulatingTransform::Status IAccumulatingTransform::prepare() totals_input.close(); } } - } - /// Generate output block. - if (input.isFinished()) - { - finished_input = true; + /// Generate output block. return Status::Ready; } From 20b3823770f60feefcd73306141bc25875d87f9a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Sep 2020 13:21:13 +0300 Subject: [PATCH 070/123] do not remove directory on attach --- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++++--- .../test_cleanup_dir_after_bad_zk_conn/test.py | 13 ++++++++++++- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 00dcc7aeb08..908112c17ff 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -234,10 +234,11 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( { current_zookeeper = global_context.getZooKeeper(); } - catch (Poco::Exception & e) + catch (...) { - dropIfEmpty(); - throw e; + if (!attach) + dropIfEmpty(); + throw; } } @@ -262,7 +263,10 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( if (!current_zookeeper) { if (!attach) + { + dropIfEmpty(); throw Exception("Can't create replicated table without ZooKeeper", ErrorCodes::NO_ZOOKEEPER); + } /// Do not activate the replica. It will be readonly. LOG_ERROR(log, "No ZooKeeper: table will be in readonly mode."); diff --git a/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py b/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py index 4cb243160a0..ab2db469157 100644 --- a/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py +++ b/tests/integration/test_cleanup_dir_after_bad_zk_conn/test.py @@ -54,9 +54,20 @@ def test_cleanup_dir_after_wrong_replica_name(start_cluster): assert "already exists" in error node1.query("CREATE TABLE test_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test2/', 'r2') ORDER BY n") - def test_cleanup_dir_after_wrong_zk_path(start_cluster): node1.query("CREATE TABLE test3_r1 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test3/', 'r1') ORDER BY n") error = node1.query_and_get_error("CREATE TABLE test3_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/', 'r2') ORDER BY n") assert "Cannot create" in error node1.query("CREATE TABLE test3_r2 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test3/', 'r2') ORDER BY n") + +def test_attach_without_zk(start_cluster): + node1.query("CREATE TABLE test4_r1 (n UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test4/', 'r1') ORDER BY n") + node1.query("DETACH TABLE test4_r1") + with PartitionManager() as pm: + pm._add_rule({'probability': 0.5, 'source': node1.ip_address, 'destination_port': 2181, 'action': 'DROP'}) + try: + node1.query("ATTACH TABLE test4_r1") + except: + pass + node1.query("ATTACH TABLE IF NOT EXISTS test4_r1") + node1.query("SELECT * FROM test4_r1") From c309f55c207d5e864f1b05200e2a706627163bac Mon Sep 17 00:00:00 2001 From: myrrc Date: Thu, 10 Sep 2020 14:02:52 +0300 Subject: [PATCH 071/123] 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 072/123] 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 073/123] 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 074/123] 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 075/123] 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 ca2a33008b291bc5d1507b568ac31d588a6aa3d8 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 2 Sep 2020 19:42:24 +0300 Subject: [PATCH 076/123] faster --- docker/test/performance-comparison/eqmed.sql | 4 ++-- docker/test/performance-comparison/perf.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/performance-comparison/eqmed.sql b/docker/test/performance-comparison/eqmed.sql index f7f8d6ac40d..139f0758798 100644 --- a/docker/test/performance-comparison/eqmed.sql +++ b/docker/test/performance-comparison/eqmed.sql @@ -8,7 +8,7 @@ select from ( -- quantiles of randomization distributions - select quantileExactForEach(0.999)( + select quantileExactForEach(0.99)( arrayMap(x, y -> abs(x - y), metrics_by_label[1], metrics_by_label[2]) as d ) threshold ---- uncomment to see what the distribution is really like @@ -33,7 +33,7 @@ from -- strip the query away before the join -- it might be several kB long; (select metrics, run, version from table) no_query, -- duplicate input measurements into many virtual runs - numbers(1, 100000) nn + numbers(1, 10000) nn -- for each virtual run, randomly reorder measurements order by virtual_run, rand() ) virtual_runs diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index e1476d9aeb4..05e89c9e44c 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -20,7 +20,7 @@ parser = argparse.ArgumentParser(description='Run performance test.') parser.add_argument('file', metavar='FILE', type=argparse.FileType('r', encoding='utf-8'), nargs=1, help='test description file') parser.add_argument('--host', nargs='*', default=['localhost'], help="Server hostname(s). Corresponds to '--port' options.") parser.add_argument('--port', nargs='*', default=[9000], help="Server port(s). Corresponds to '--host' options.") -parser.add_argument('--runs', type=int, default=int(os.environ.get('CHPC_RUNS', 13)), help='Number of query runs per server. Defaults to CHPC_RUNS environment variable.') +parser.add_argument('--runs', type=int, default=int(os.environ.get('CHPC_RUNS', 7)), help='Number of query runs per server. Defaults to CHPC_RUNS environment variable.') parser.add_argument('--long', action='store_true', help='Do not skip the tests tagged as long.') parser.add_argument('--print-queries', action='store_true', help='Print test queries and exit.') parser.add_argument('--print-settings', action='store_true', help='Print test settings and exit.') From 26348ad0143f881c8d14e41e0c80d706614ab110 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 10 Sep 2020 18:48:39 +0300 Subject: [PATCH 077/123] fixup --- docker/test/performance-comparison/report.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 1003a6d0e1a..b3f8ef01138 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -372,7 +372,7 @@ if args.report == 'main': 'New, s', # 1 'Ratio of speedup (-) or slowdown (+)', # 2 'Relative difference (new − old) / old', # 3 - 'p < 0.001 threshold', # 4 + 'p < 0.01 threshold', # 4 # Failed # 5 'Test', # 6 '#', # 7 @@ -416,7 +416,7 @@ if args.report == 'main': 'Old, s', #0 'New, s', #1 'Relative difference (new - old)/old', #2 - 'p < 0.001 threshold', #3 + 'p < 0.01 threshold', #3 # Failed #4 'Test', #5 '#', #6 @@ -649,7 +649,7 @@ elif args.report == 'all-queries': 'New, s', #3 'Ratio of speedup (-) or slowdown (+)', #4 'Relative difference (new − old) / old', #5 - 'p < 0.001 threshold', #6 + 'p < 0.01 threshold', #6 'Test', #7 '#', #8 'Query', #9 From 397ebdb855c75be3df9f6a007a4e2cf3d0321bb5 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Thu, 10 Sep 2020 08:55:39 -0700 Subject: [PATCH 078/123] 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 079/123] 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 3b3b7bd5e790ec6ef0dff61793c2790167ea3469 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Thu, 10 Sep 2020 09:09:48 -0700 Subject: [PATCH 080/123] minor changes to comment --- src/Storages/StorageReplicatedMergeTree.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 908112c17ff..feb2f95849c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -226,10 +226,10 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( /// This unclean state will hinder table creation on any retries and will /// complain that the Directory for table already exists. /// - /// To acheive a clean state on failed table creations, catch this error if - /// the excaption is of type Poco::Exception and call dropIfEmpty() method, - /// then proceed throwing the exception. Without this, the Directory for the - /// tables need to be manually deleted before retrying the CreateQuery. + /// To achieve a clean state on failed table creations, catch this error and + /// call dropIfEmpty() method only if the operation isn't ATTACH then proceed + /// throwing the exception. Without this, the Directory for the tables need + /// to be manually deleted before retrying the CreateQuery. try { current_zookeeper = global_context.getZooKeeper(); From 15bdb6048e610fb5787e581b7fa346bd7512e109 Mon Sep 17 00:00:00 2001 From: Artem Hnilov Date: Thu, 10 Sep 2020 16:25:07 +0000 Subject: [PATCH 081/123] 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 082/123] 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 083/123] 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 084/123] 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 085/123] 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 086/123] 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 087/123] 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 088/123] 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 089/123] 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 090/123] 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 da2bb4e0d3d2e642993f070b923401a9db470d81 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 11 Sep 2020 15:46:14 +0800 Subject: [PATCH 091/123] Fix missing clone in replace column transformer --- src/Parsers/ASTColumnsTransformers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index 2625a03830b..43d54f07ab8 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -110,7 +110,7 @@ void ASTColumnsReplaceTransformer::replaceChildren(ASTPtr & node, const ASTPtr & if (const auto * id = child->as()) { if (id->shortName() == name) - child = replacement; + child = replacement->clone(); } else replaceChildren(child, replacement, name); From 3b9ab3f1be330b5ae7ffd7c68fd629ad3ebc9f6b Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 11 Sep 2020 12:23:31 +0300 Subject: [PATCH 092/123] Fix if --- src/Functions/if.cpp | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 20848bede32..584bed3f8c5 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -604,7 +604,6 @@ private: const ColumnUInt8 * cond_col, Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) { /// Convert both columns to the common type (if needed). - const ColumnWithTypeAndName & arg1 = block.getByPosition(arguments[1]); const ColumnWithTypeAndName & arg2 = block.getByPosition(arguments[2]); @@ -765,10 +764,22 @@ private: return ColumnNullable::create(materialized, ColumnUInt8::create(column->size(), 0)); } - static ColumnPtr getNestedColumn(const ColumnPtr & column) + /// Return nested column recursively removing Nullable, examples: + /// Nullable(size = 1, Int32(size = 1), UInt8(size = 1)) -> Int32(size = 1) + /// Const(size = 0, Nullable(size = 1, Int32(size = 1), UInt8(size = 1))) -> + /// Const(size = 0, Int32(size = 1)) + static ColumnPtr recursiveGetNestedColumnWithoutNullable(const ColumnPtr & column) { if (const auto * nullable = checkAndGetColumn(*column)) + { + /// Nullable cannot contain Nullable return nullable->getNestedColumnPtr(); + } + else if (const auto * column_const = checkAndGetColumn(*column)) + { + /// Save Constant, but remove Nullable + return ColumnConst::create(recursiveGetNestedColumnWithoutNullable(column_const->getDataColumnPtr()), column->size()); + } return column; } @@ -826,12 +837,12 @@ private: { arg_cond, { - getNestedColumn(arg_then.column), + recursiveGetNestedColumnWithoutNullable(arg_then.column), removeNullable(arg_then.type), "" }, { - getNestedColumn(arg_else.column), + recursiveGetNestedColumnWithoutNullable(arg_else.column), removeNullable(arg_else.type), "" }, From 5de3d9c03298101876df60eed11ca2499676faf5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 11 Sep 2020 12:58:04 +0300 Subject: [PATCH 093/123] 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 094/123] 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/) From 6dd764bcfe5b28d3ccb89ca950558bd82c911847 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 11 Sep 2020 18:12:08 +0300 Subject: [PATCH 095/123] FunctionBinaryArithmetic refactoring (#14712) --- src/Functions/FunctionBinaryArithmetic.h | 165 ++++++++++------------- src/Functions/divide.cpp | 1 - src/Functions/minus.cpp | 1 - src/Functions/multiply.cpp | 1 - src/Functions/plus.cpp | 2 +- 5 files changed, 72 insertions(+), 98 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 15b6ea6ca5d..ca0cc876035 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -561,6 +561,8 @@ public: template