From 93c49dfeaa802ea13143a52e85056fdc230afe8b Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Sep 2023 19:31:12 +0000 Subject: [PATCH 1/3] Fix using structure from insertion tables in case of defaults and explicit insert columns --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 36 +++++++++--------- src/Interpreters/Context.cpp | 37 ++++++++++--------- src/Interpreters/Context.h | 20 ++++++++-- src/Interpreters/InterpreterInsertQuery.cpp | 18 +++++++++ src/Interpreters/InterpreterInsertQuery.h | 4 ++ src/Interpreters/executeQuery.cpp | 2 +- ...ble_with_explicit_insert_columns.reference | 4 ++ ...tion_table_with_explicit_insert_columns.sh | 17 +++++++++ ...om_insertion_table_with_defaults.reference | 1 + ...ture_from_insertion_table_with_defaults.sh | 14 +++++++ 10 files changed, 114 insertions(+), 39 deletions(-) create mode 100644 tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.reference create mode 100755 tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.sh create mode 100644 tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.reference create mode 100755 tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 9bb9308a609d..e8dbbc1be15e 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6243,11 +6243,11 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, const auto & insertion_table = scope_context->getInsertionTable(); if (!insertion_table.empty()) { - const auto & insert_structure = DatabaseCatalog::instance() - .getTable(insertion_table, scope_context) - ->getInMemoryMetadataPtr() - ->getColumns() - .getInsertable(); + const auto & insert_columns = DatabaseCatalog::instance() + .getTable(insertion_table, scope_context) + ->getInMemoryMetadataPtr() + ->getColumns(); + const auto & insert_column_names = insert_columns.getInsertable().getNames(); DB::ColumnsDescription structure_hint; bool use_columns_from_insert_query = true; @@ -6255,8 +6255,8 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, /// Insert table matches columns against SELECT expression by position, so we want to map /// insert table columns to table function columns through names from SELECT expression. - auto insert_column = insert_structure.begin(); - auto insert_structure_end = insert_structure.end(); /// end iterator of the range covered by possible asterisk + auto insert_column_name_it = insert_column_names.begin(); + auto insert_column_names_end = insert_column_names.end(); /// end iterator of the range covered by possible asterisk auto virtual_column_names = table_function_ptr->getVirtualsToCheckBeforeUsingStructureHint(); bool asterisk = false; const auto & expression_list = scope.scope_node->as().getProjection(); @@ -6264,7 +6264,7 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, /// We want to go through SELECT expression list and correspond each expression to column in insert table /// which type will be used as a hint for the file structure inference. - for (; expression != expression_list.end() && insert_column != insert_structure_end; ++expression) + for (; expression != expression_list.end() && insert_column_name_it != insert_column_names_end; ++expression) { if (auto * identifier_node = (*expression)->as()) { @@ -6280,15 +6280,17 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, break; } - structure_hint.add({ identifier_node->getIdentifier().getFullName(), insert_column->type }); + ColumnDescription column = insert_columns.get(*insert_column_name_it); + column.name = identifier_node->getIdentifier().getFullName(); + structure_hint.add(std::move(column)); } /// Once we hit asterisk we want to find end of the range covered by asterisk /// contributing every further SELECT expression to the tail of insert structure if (asterisk) - --insert_structure_end; + --insert_column_names_end; else - ++insert_column; + ++insert_column_name_it; } else if (auto * matcher_node = (*expression)->as(); matcher_node && matcher_node->getMatcherType() == MatcherNodeType::ASTERISK) { @@ -6322,18 +6324,18 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, /// Once we hit asterisk we want to find end of the range covered by asterisk /// contributing every further SELECT expression to the tail of insert structure if (asterisk) - --insert_structure_end; + --insert_column_names_end; else - ++insert_column; + ++insert_column_name_it; } else { /// Once we hit asterisk we want to find end of the range covered by asterisk /// contributing every further SELECT expression to the tail of insert structure if (asterisk) - --insert_structure_end; + --insert_column_names_end; else - ++insert_column; + ++insert_column_name_it; } } @@ -6353,8 +6355,8 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, /// Append tail of insert structure to the hint if (asterisk) { - for (; insert_column != insert_structure_end; ++insert_column) - structure_hint.add({ insert_column->name, insert_column->type }); + for (; insert_column_name_it != insert_column_names_end; ++insert_column_name_it) + structure_hint.add(insert_columns.get(*insert_column_name_it)); } if (!structure_hint.empty()) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 8695669a7de1..fa408b36f555 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1587,11 +1587,12 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const uint64_t use_structure_from_insertion_table_in_table_functions = getSettingsRef().use_structure_from_insertion_table_in_table_functions; if (use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint() && hasInsertionTable()) { - const auto & insert_structure = DatabaseCatalog::instance() - .getTable(getInsertionTable(), shared_from_this()) - ->getInMemoryMetadataPtr() - ->getColumns() - .getInsertable(); + const auto & insert_columns = DatabaseCatalog::instance() + .getTable(getInsertionTable(), shared_from_this()) + ->getInMemoryMetadataPtr() + ->getColumns(); + + const auto & insert_column_names = hasInsertionTableColumnNames() ? getInsertionTableColumnNames() : insert_columns.getInsertable().getNames(); DB::ColumnsDescription structure_hint; bool use_columns_from_insert_query = true; @@ -1599,8 +1600,8 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const /// Insert table matches columns against SELECT expression by position, so we want to map /// insert table columns to table function columns through names from SELECT expression. - auto insert_column = insert_structure.begin(); - auto insert_structure_end = insert_structure.end(); /// end iterator of the range covered by possible asterisk + auto insert_column_name_it = insert_column_names.begin(); + auto insert_column_names_end = insert_column_names.end(); /// end iterator of the range covered by possible asterisk auto virtual_column_names = table_function_ptr->getVirtualsToCheckBeforeUsingStructureHint(); bool asterisk = false; const auto & expression_list = select_query_hint->select()->as()->children; @@ -1608,7 +1609,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const /// We want to go through SELECT expression list and correspond each expression to column in insert table /// which type will be used as a hint for the file structure inference. - for (; expression != expression_list.end() && insert_column != insert_structure_end; ++expression) + for (; expression != expression_list.end() && insert_column_name_it != insert_column_names_end; ++expression) { if (auto * identifier = (*expression)->as()) { @@ -1623,15 +1624,17 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const break; } - structure_hint.add({ identifier->name(), insert_column->type }); + ColumnDescription column = insert_columns.get(*insert_column_name_it); + column.name = identifier->name(); + structure_hint.add(std::move(column)); } /// Once we hit asterisk we want to find end of the range covered by asterisk /// contributing every further SELECT expression to the tail of insert structure if (asterisk) - --insert_structure_end; + --insert_column_names_end; else - ++insert_column; + ++insert_column_name_it; } else if ((*expression)->as()) { @@ -1665,18 +1668,18 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const /// Once we hit asterisk we want to find end of the range covered by asterisk /// contributing every further SELECT expression to the tail of insert structure if (asterisk) - --insert_structure_end; + --insert_column_names_end; else - ++insert_column; + ++insert_column_name_it; } else { /// Once we hit asterisk we want to find end of the range covered by asterisk /// contributing every further SELECT expression to the tail of insert structure if (asterisk) - --insert_structure_end; + --insert_column_names_end; else - ++insert_column; + ++insert_column_name_it; } } @@ -1696,8 +1699,8 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const /// Append tail of insert structure to the hint if (asterisk) { - for (; insert_column != insert_structure_end; ++insert_column) - structure_hint.add({ insert_column->name, insert_column->type }); + for (; insert_column_name_it != insert_column_names_end; ++insert_column_name_it) + structure_hint.add(insert_columns.get(*insert_column_name_it)); } if (!structure_hint.empty()) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b4a5b3d8c85c..19cbf5be54b4 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -265,7 +265,13 @@ class Context: public std::enable_shared_from_this std::weak_ptr process_list_elem; /// For tracking total resource usage for query. bool has_process_list_elem = false; /// It's impossible to check if weak_ptr was initialized or not - StorageID insertion_table = StorageID::createEmpty(); /// Saved insertion table in query context + struct InsertionTableInfo + { + StorageID table = StorageID::createEmpty(); + std::optional column_names; + }; + + InsertionTableInfo insertion_table_info; /// Saved information about insertion table in query context bool is_distributed = false; /// Whether the current context it used for distributed query String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification. @@ -713,9 +719,15 @@ class Context: public std::enable_shared_from_this void killCurrentQuery() const; - bool hasInsertionTable() const { return !insertion_table.empty(); } - void setInsertionTable(StorageID db_and_table) { insertion_table = std::move(db_and_table); } - const StorageID & getInsertionTable() const { return insertion_table; } + bool hasInsertionTable() const { return !insertion_table_info.table.empty(); } + bool hasInsertionTableColumnNames() const { return insertion_table_info.column_names.has_value(); } + void setInsertionTable(StorageID db_and_table, const std::optional & column_names = std::nullopt) { insertion_table_info = {std::move(db_and_table), column_names}; } + const StorageID & getInsertionTable() const { return insertion_table_info.table; } + const Names & getInsertionTableColumnNames() const + { + assert(insertion_table_info.column_names.has_value()); + return *insertion_table_info.column_names; + } void setDistributed(bool is_distributed_) { is_distributed = is_distributed_; } bool isDistributed() const { return is_distributed; } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 1f6f2336ab85..dedbd56d5649 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -149,6 +149,24 @@ Block InterpreterInsertQuery::getSampleBlock( return getSampleBlock(names, table, metadata_snapshot); } +std::optional InterpreterInsertQuery::getInsertColumnNames() const +{ + auto const * insert_query = query_ptr->as(); + if (!insert_query || !insert_query->columns) + return std::nullopt; + + auto table = DatabaseCatalog::instance().getTable(getDatabaseTable(), getContext()); + Names names; + const auto columns_ast = processColumnTransformers(getContext()->getCurrentDatabase(), table, table->getInMemoryMetadataPtr(), insert_query->columns); + for (const auto & identifier : columns_ast->children) + { + std::string current_name = identifier->getColumnName(); + names.emplace_back(std::move(current_name)); + } + + return names; +} + Block InterpreterInsertQuery::getSampleBlock( const Names & names, const StoragePtr & table, diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index b9a146e5338c..845cb6b730bf 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -37,6 +37,10 @@ class InterpreterInsertQuery : public IInterpreter, WithContext StorageID getDatabaseTable() const; + /// Return explicitly specified column names to insert. + /// It not explicit names were specified, return nullopt. + std::optional getInsertColumnNames() const; + Chain buildChain( const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 1bfeeaa8ad49..310af2f98121 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1074,7 +1074,7 @@ static std::tuple executeQueryImpl( /// Save insertion table (not table function). TODO: support remote() table function. auto table_id = insert_interpreter->getDatabaseTable(); if (!table_id.empty()) - context->setInsertionTable(std::move(table_id)); + context->setInsertionTable(std::move(table_id), insert_interpreter->getInsertColumnNames()); if (insert_data_buffer_holder) insert_interpreter->addBuffer(std::move(insert_data_buffer_holder)); diff --git a/tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.reference b/tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.reference new file mode 100644 index 000000000000..06f682a236f5 --- /dev/null +++ b/tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.reference @@ -0,0 +1,4 @@ +0 0 +0 0 +42 0 +42 0 diff --git a/tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.sh b/tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.sh new file mode 100755 index 000000000000..8bdaa47c1118 --- /dev/null +++ b/tests/queries/0_stateless/02878_use_structure_from_insertion_table_with_explicit_insert_columns.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select 42 as x format Native" > $CLICKHOUSE_TEST_UNIQUE_NAME.native +$CLICKHOUSE_LOCAL -n -q " +create table test (x UInt64, y UInt64) engine=Memory; +insert into test (x) select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.native'); +insert into test (y) select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.native'); +insert into test (* except(x)) select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.native'); +insert into test (* except(y)) select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.native'); +select * from test order by x; +" + +rm $CLICKHOUSE_TEST_UNIQUE_NAME.native diff --git a/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.reference b/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.reference new file mode 100644 index 000000000000..2f35ae532c86 --- /dev/null +++ b/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.reference @@ -0,0 +1 @@ +42 42 diff --git a/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh b/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh new file mode 100755 index 000000000000..a792fc9e66c0 --- /dev/null +++ b/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select 42 as x format Native" > $CLICKHOUSE_TEST_UNIQUE_NAME.native +$CLICKHOUSE_LOCAL -n -q " +create table test (x UInt64, y UInt64 default 42) engine=Memory; +insert into test select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.native'); +select * from test; +" + +rm $CLICKHOUSE_TEST_UNIQUE_NAME.native From b9c28ef1f7f9ae696e13d5ad8cdea7acea12a4ac Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Sep 2023 20:10:07 +0000 Subject: [PATCH 2/3] Fix for analyzer --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Interpreters/Context.cpp | 2 +- src/Interpreters/Context.h | 6 +----- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- 4 files changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index e8dbbc1be15e..12d9051ac57f 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6247,7 +6247,7 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, .getTable(insertion_table, scope_context) ->getInMemoryMetadataPtr() ->getColumns(); - const auto & insert_column_names = insert_columns.getInsertable().getNames(); + const auto & insert_column_names = scope_context->hasInsertionTableColumnNames() ? *scope_context->getInsertionTableColumnNames() : insert_columns.getInsertable().getNames(); DB::ColumnsDescription structure_hint; bool use_columns_from_insert_query = true; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fa408b36f555..8f5ad7744783 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1592,7 +1592,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const ->getInMemoryMetadataPtr() ->getColumns(); - const auto & insert_column_names = hasInsertionTableColumnNames() ? getInsertionTableColumnNames() : insert_columns.getInsertable().getNames(); + const auto & insert_column_names = hasInsertionTableColumnNames() ? *getInsertionTableColumnNames() : insert_columns.getInsertable().getNames(); DB::ColumnsDescription structure_hint; bool use_columns_from_insert_query = true; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 19cbf5be54b4..446b6619fcab 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -723,11 +723,7 @@ class Context: public std::enable_shared_from_this bool hasInsertionTableColumnNames() const { return insertion_table_info.column_names.has_value(); } void setInsertionTable(StorageID db_and_table, const std::optional & column_names = std::nullopt) { insertion_table_info = {std::move(db_and_table), column_names}; } const StorageID & getInsertionTable() const { return insertion_table_info.table; } - const Names & getInsertionTableColumnNames() const - { - assert(insertion_table_info.column_names.has_value()); - return *insertion_table_info.column_names; - } + const std::optional & getInsertionTableColumnNames() const{ return insertion_table_info.column_names; } void setDistributed(bool is_distributed_) { is_distributed = is_distributed_; } bool isDistributed() const { return is_distributed; } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index dedbd56d5649..4096204ba403 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -474,7 +474,7 @@ BlockIO InterpreterInsertQuery::execute() auto new_context = Context::createCopy(context); new_context->setSettings(new_settings); - new_context->setInsertionTable(getContext()->getInsertionTable()); + new_context->setInsertionTable(getContext()->getInsertionTable(), getContext()->getInsertionTableColumnNames()); auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1); From 7cfdd88ccf23b569739b7c59a529dbbeedc6924c Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 15 Sep 2023 09:32:41 +0000 Subject: [PATCH 3/3] Address comments --- src/Interpreters/Context.h | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 6 ++++-- ...e_structure_from_insertion_table_with_defaults.reference | 2 +- ...2879_use_structure_from_insertion_table_with_defaults.sh | 2 +- 4 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 446b6619fcab..16c533af86aa 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -721,7 +721,7 @@ class Context: public std::enable_shared_from_this bool hasInsertionTable() const { return !insertion_table_info.table.empty(); } bool hasInsertionTableColumnNames() const { return insertion_table_info.column_names.has_value(); } - void setInsertionTable(StorageID db_and_table, const std::optional & column_names = std::nullopt) { insertion_table_info = {std::move(db_and_table), column_names}; } + void setInsertionTable(StorageID db_and_table, std::optional column_names = std::nullopt) { insertion_table_info = {std::move(db_and_table), std::move(column_names)}; } const StorageID & getInsertionTable() const { return insertion_table_info.table; } const std::optional & getInsertionTableColumnNames() const{ return insertion_table_info.column_names; } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 4096204ba403..22ffc5f965c0 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -138,8 +138,9 @@ Block InterpreterInsertQuery::getSampleBlock( } /// Form the block based on the column names from the query - Names names; const auto columns_ast = processColumnTransformers(getContext()->getCurrentDatabase(), table, metadata_snapshot, query.columns); + Names names; + names.reserve(columns_ast->children.size()); for (const auto & identifier : columns_ast->children) { std::string current_name = identifier->getColumnName(); @@ -156,8 +157,9 @@ std::optional InterpreterInsertQuery::getInsertColumnNames() const return std::nullopt; auto table = DatabaseCatalog::instance().getTable(getDatabaseTable(), getContext()); - Names names; const auto columns_ast = processColumnTransformers(getContext()->getCurrentDatabase(), table, table->getInMemoryMetadataPtr(), insert_query->columns); + Names names; + names.reserve(columns_ast->children.size()); for (const auto & identifier : columns_ast->children) { std::string current_name = identifier->getColumnName(); diff --git a/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.reference b/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.reference index 2f35ae532c86..0e291f3ac0d1 100644 --- a/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.reference +++ b/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.reference @@ -1 +1 @@ -42 42 +1 42 diff --git a/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh b/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh index a792fc9e66c0..315bbcd544ff 100755 --- a/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh +++ b/tests/queries/0_stateless/02879_use_structure_from_insertion_table_with_defaults.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL -q "select 42 as x format Native" > $CLICKHOUSE_TEST_UNIQUE_NAME.native +$CLICKHOUSE_LOCAL -q "select 1 as x format Native" > $CLICKHOUSE_TEST_UNIQUE_NAME.native $CLICKHOUSE_LOCAL -n -q " create table test (x UInt64, y UInt64 default 42) engine=Memory; insert into test select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.native');