Improve schema inference for insert select queries

This commit is contained in:
avogar 2022-03-30 12:44:53 +00:00
parent eb787c1ddc
commit 849596c7a2
8 changed files with 47 additions and 0 deletions

View File

@ -31,6 +31,7 @@
#include <Disks/DiskLocal.h>
#include <Disks/IDiskRemote.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/TableFunctionInput.h>
#include <Interpreters/ActionLocksManager.h>
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Core/Settings.h>
@ -1092,6 +1093,17 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression)
if (!res)
{
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression, shared_from_this());
if (table_function_ptr->needStructureHint())
{
const auto & insertion_table = getInsertionTable();
if (!insertion_table.empty())
{
const auto & structure_hint
= DatabaseCatalog::instance().getTable(insertion_table, shared_from_this())->getInMemoryMetadataPtr()->columns;
table_function_ptr->setStructureHint(structure_hint);
}
}
res = table_function_ptr->execute(table_expression, shared_from_this(), table_function_ptr->getName());
/// Since ITableFunction::parseArguments() may change table_expression, i.e.:

View File

@ -15,6 +15,7 @@
#include <Common/isLocalAddress.h>
#include <base/types.h>
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
#include <Storages/ColumnsDescription.h>
#include "config_core.h"

View File

@ -18,6 +18,7 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
}
void TableFunctionInput::parseArguments(const ASTPtr & ast_function, ContextPtr context)
@ -29,6 +30,12 @@ void TableFunctionInput::parseArguments(const ASTPtr & ast_function, ContextPtr
auto args = function->arguments->children;
if (args.empty())
{
structure = "auto";
return;
}
if (args.size() != 1)
throw Exception("Table function '" + getName() + "' requires exactly 1 argument: structure",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -38,6 +45,16 @@ void TableFunctionInput::parseArguments(const ASTPtr & ast_function, ContextPtr
ColumnsDescription TableFunctionInput::getActualTableStructure(ContextPtr context) const
{
if (structure == "auto")
{
if (structure_hint.empty())
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Table function '{}' was used without structure argument but structure could not be determined automatically. Please, "
"provide structure manually",
getName());
return structure_hint;
}
return parseColumnsListFromString(structure, context);
}

View File

@ -16,6 +16,8 @@ public:
static constexpr auto name = "input";
std::string getName() const override { return name; }
bool hasStaticStructure() const override { return true; }
bool needStructureHint() const override { return true; }
void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; }
private:
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override;
@ -25,6 +27,7 @@ private:
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
String structure;
ColumnsDescription structure_hint;
};
}

View File

@ -0,0 +1,2 @@
1 string1
2 string2

View File

@ -0,0 +1,5 @@
drop table if exists test_02249;
create table test_02249 (x UInt32, y String) engine=Memory();
insert into test_02249 select * from input() format JSONEachRow {"x" : 1, "y" : "string1"}, {"y" : "string2", "x" : 2};
select * from test_02249;
drop table test_02249;

View File

@ -0,0 +1,6 @@
insert into table function file('data_02250.jsonl') select NULL as x settings engine_file_truncate_on_insert=1;
drop table if exists test_02250;
create table test_02250 (x Nullable(UInt32)) engine=Memory();
insert into test_02250 select * from file('data_02250.jsonl');
select * from test_02250;
drop table test_02250;