This commit is contained in:
Alexander Tokmakov 2021-03-23 14:29:29 +03:00
parent fc3e11a068
commit c938f4f2fe
4 changed files with 28 additions and 14 deletions

View File

@ -972,9 +972,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
{
const auto & factory = TableFunctionFactory::instance();
auto table_func = factory.get(create.as_table_function, context);
/// We should use global context here because there will be no query context on server startup
/// and because storage lifetime is bigger than query context lifetime.
res = table_func->execute(create.as_table_function, context.getGlobalContext(), create.table, properties.columns);
res = table_func->execute(create.as_table_function, context, create.table, properties.columns);
res->renameInMemory({create.database, create.table, create.uuid});
}
else

View File

@ -33,7 +33,7 @@ namespace ErrorCodes
IStorageURLBase::IStorageURLBase(
const Poco::URI & uri_,
const Context & context_,
const Context & /*context_*/,
const StorageID & table_id_,
const String & format_name_,
const std::optional<FormatSettings> & format_settings_,
@ -46,8 +46,6 @@ IStorageURLBase::IStorageURLBase(
, format_name(format_name_)
, format_settings(format_settings_)
{
context_.getRemoteHostFilter().checkURL(uri);
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
storage_metadata.setConstraints(constraints_);
@ -244,6 +242,20 @@ BlockOutputStreamPtr IStorageURLBase::write(const ASTPtr & /*query*/, const Stor
chooseCompressionMethod(uri.toString(), compression_method));
}
StorageURL::StorageURL(const Poco::URI & uri_,
const StorageID & table_id_,
const String & format_name_,
const std::optional<FormatSettings> & format_settings_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
Context & context_,
const String & compression_method_)
: IStorageURLBase(uri_, context_, table_id_, format_name_,
format_settings_, columns_, constraints_, compression_method_)
{
context_.getRemoteHostFilter().checkURL(uri);
}
void registerStorageURL(StorageFactory & factory)
{
factory.registerStorage("URL", [](const StorageFactory::Arguments & args)

View File

@ -113,11 +113,7 @@ public:
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
Context & context_,
const String & compression_method_)
: IStorageURLBase(uri_, context_, table_id_, format_name_,
format_settings_, columns_, constraints_, compression_method_)
{
}
const String & compression_method_);
String getName() const override
{

View File

@ -20,12 +20,20 @@ StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context &
ProfileEvents::increment(ProfileEvents::TableFunctionExecute);
context.checkAccess(AccessType::CREATE_TEMPORARY_TABLE | StorageFactory::instance().getSourceAccessType(getStorageTypeName()));
if (cached_columns.empty() || (hasStaticStructure() && cached_columns == getActualTableStructure(context)))
if (cached_columns.empty())
return executeImpl(ast_function, context, table_name, std::move(cached_columns));
auto get_storage = [=, tf = shared_from_this()]() -> StoragePtr
/// We have table structure, so it's CREATE AS table_function().
/// We should use global context here because there will be no query context on server startup
/// and because storage lifetime is bigger than query context lifetime.
const Context & global_context = context.getGlobalContext();
if (hasStaticStructure() && cached_columns == getActualTableStructure(context))
return executeImpl(ast_function, global_context, table_name, std::move(cached_columns));
auto this_table_function = shared_from_this();
auto get_storage = [=, &global_context]() -> StoragePtr
{
return tf->executeImpl(ast_function, context, table_name, cached_columns);
return this_table_function->executeImpl(ast_function, global_context, table_name, cached_columns);
};
/// It will request actual table structure and create underlying storage lazily