slightly better code

This commit is contained in:
Alexander Tokmakov 2020-09-01 17:41:49 +03:00
parent b6093d9a86
commit 3a9779adb4
48 changed files with 243 additions and 271 deletions

View File

@ -54,9 +54,8 @@ std::pair<String, StoragePtr> createTableFromAST(
if (ast_create_query.as_table_function) if (ast_create_query.as_table_function)
{ {
const auto & ast_table_function = ast_create_query.as_table_function->as<ASTFunction &>();
const auto & factory = TableFunctionFactory::instance(); const auto & factory = TableFunctionFactory::instance();
auto table_function = factory.get(ast_table_function.name, context); auto table_function = factory.get(ast_create_query.as_table_function, context);
ColumnsDescription columns; ColumnsDescription columns;
if (ast_create_query.columns_list && ast_create_query.columns_list->columns) if (ast_create_query.columns_list && ast_create_query.columns_list->columns)
columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, false); columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, false);

View File

@ -154,8 +154,7 @@ void SelectStreamFactory::createForShard(
if (table_func_ptr) if (table_func_ptr)
{ {
const auto * table_function = table_func_ptr->as<ASTFunction>(); TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_func_ptr, context);
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_function->name, context);
main_table_storage = table_function_ptr->execute(table_func_ptr, context, table_function_ptr->getName()); main_table_storage = table_function_ptr->execute(table_func_ptr, context, table_function_ptr->getName());
} }
else else

View File

@ -916,7 +916,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression)
if (!res) if (!res)
{ {
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression->as<ASTFunction>()->name, *this); TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression, *this);
/// Run it and remember the result /// Run it and remember the result
res = table_function_ptr->execute(table_expression, *this, table_function_ptr->getName()); res = table_function_ptr->execute(table_expression, *this, table_function_ptr->getName());

View File

@ -497,9 +497,9 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS
else if (create.as_table_function) else if (create.as_table_function)
{ {
/// Table function without columns list. /// Table function without columns list.
auto table_function = TableFunctionFactory::instance().get(create.as_table_function->as<ASTFunction &>().name, context); auto table_function = TableFunctionFactory::instance().get(create.as_table_function, context);
properties.columns = table_function->getActualTableStructure(create.as_table_function, context); properties.columns = table_function->getActualTableStructure(context);
if (properties.columns.empty()) //FIXME if (properties.columns.empty()) //FIXME TableFunctionFile may return empty structure for Distributed format
return {}; return {};
} }
else else
@ -768,9 +768,8 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
/// NOTE: CREATE query may be rewritten by Storage creator or table function /// NOTE: CREATE query may be rewritten by Storage creator or table function
if (create.as_table_function) if (create.as_table_function)
{ {
const auto & table_function = create.as_table_function->as<ASTFunction &>();
const auto & factory = TableFunctionFactory::instance(); const auto & factory = TableFunctionFactory::instance();
res = factory.get(table_function.name, context)->execute(create.as_table_function, context, create.table, properties.columns); res = factory.get(create.as_table_function, context)->execute(create.as_table_function, context, create.table, properties.columns);
res->renameInMemory({create.database, create.table, create.uuid}); res->renameInMemory({create.database, create.table, create.uuid});
} }
else else

View File

@ -72,23 +72,16 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
table_expression.subquery->children.at(0), context).getNamesAndTypesList(); table_expression.subquery->children.at(0), context).getNamesAndTypesList();
columns = ColumnsDescription(std::move(names_and_types)); columns = ColumnsDescription(std::move(names_and_types));
} }
else if (table_expression.table_function)
{
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression.table_function, context);
columns = table_function_ptr->getActualTableStructure(context);
}
else else
{ {
StoragePtr table; auto table_id = context.resolveStorageID(table_expression.database_and_table_name);
if (table_expression.table_function) context.checkAccess(AccessType::SHOW_COLUMNS, table_id);
{ auto table = DatabaseCatalog::instance().getTable(table_id, context);
const auto & table_function = table_expression.table_function->as<ASTFunction &>();
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_function.name, context);
/// Run the table function and remember the result
table = table_function_ptr->execute(table_expression.table_function, context, table_function_ptr->getName());
}
else
{
auto table_id = context.resolveStorageID(table_expression.database_and_table_name);
context.checkAccess(AccessType::SHOW_COLUMNS, table_id);
table = DatabaseCatalog::instance().getTable(table_id, context);
}
auto table_lock = table->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); auto table_lock = table->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout);
auto metadata_snapshot = table->getInMemoryMetadataPtr(); auto metadata_snapshot = table->getInMemoryMetadataPtr();
columns = metadata_snapshot->getColumns(); columns = metadata_snapshot->getColumns();

View File

@ -59,9 +59,8 @@ StoragePtr InterpreterInsertQuery::getTable(ASTInsertQuery & query)
{ {
if (query.table_function) if (query.table_function)
{ {
const auto * table_function = query.table_function->as<ASTFunction>();
const auto & factory = TableFunctionFactory::instance(); const auto & factory = TableFunctionFactory::instance();
TableFunctionPtr table_function_ptr = factory.get(table_function->name, context); TableFunctionPtr table_function_ptr = factory.get(query.table_function, context);
return table_function_ptr->execute(query.table_function, context, table_function_ptr->getName()); return table_function_ptr->execute(query.table_function, context, table_function_ptr->getName());
} }

View File

@ -0,0 +1,66 @@
#include <Interpreters/getHeaderForProcessingStage.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Storages/IStorage.h>
#include <DataStreams/OneBlockInputStream.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/// Rewrite original query removing joined tables from it
void removeJoin(const ASTSelectQuery & select)
{
const auto & tables = select.tables();
if (!tables || tables->children.size() < 2)
return;
const auto & joined_table = tables->children[1]->as<ASTTablesInSelectQueryElement &>();
if (!joined_table.table_join)
return;
/// The most simple temporary solution: leave only the first table in query.
/// TODO: we also need to remove joined columns and related functions (taking in account aliases if any).
tables->children.resize(1);
}
Block getHeaderForProcessingStage(
const IStorage & storage,
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum processed_stage)
{
switch (processed_stage)
{
case QueryProcessingStage::FetchColumns:
{
Block header = metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID());
if (query_info.prewhere_info)
{
query_info.prewhere_info->prewhere_actions->execute(header);
if (query_info.prewhere_info->remove_prewhere_column)
header.erase(query_info.prewhere_info->prewhere_column_name);
}
return header;
}
case QueryProcessingStage::WithMergeableState:
case QueryProcessingStage::Complete:
{
auto query = query_info.query->clone();
removeJoin(*query->as<ASTSelectQuery>());
auto stream = std::make_shared<OneBlockInputStream>(
metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()));
return InterpreterSelectQuery(query, context, stream, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
}
}
throw Exception("Logical Error: unknown processed stage.", ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -0,0 +1,27 @@
#pragma once
#include <Core/Block.h>
#include <Core/Names.h>
#include <Core/QueryProcessingStage.h>
namespace DB
{
class IStorage;
struct StorageInMemoryMetadata;
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
struct SelectQueryInfo;
class Context;
class ASTSelectQuery;
void removeJoin(const ASTSelectQuery & select);
Block getHeaderForProcessingStage(
const IStorage & storage,
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum processed_stage);
}

View File

@ -18,7 +18,7 @@ namespace DB
class StorageFileBlockInputStream; class StorageFileBlockInputStream;
class StorageFileBlockOutputStream; class StorageFileBlockOutputStream;
class StorageFile : public ext::shared_ptr_helper<StorageFile>, public IStorage class StorageFile final : public ext::shared_ptr_helper<StorageFile>, public IStorage
{ {
friend struct ext::shared_ptr_helper<StorageFile>; friend struct ext::shared_ptr_helper<StorageFile>;
public: public:

View File

@ -9,7 +9,7 @@ namespace DB
{ {
/* Generates random data for given schema. /* Generates random data for given schema.
*/ */
class StorageGenerateRandom : public ext::shared_ptr_helper<StorageGenerateRandom>, public IStorage class StorageGenerateRandom final : public ext::shared_ptr_helper<StorageGenerateRandom>, public IStorage
{ {
friend struct ext::shared_ptr_helper<StorageGenerateRandom>; friend struct ext::shared_ptr_helper<StorageGenerateRandom>;
public: public:

View File

@ -13,7 +13,7 @@ namespace DB
* This class represents table engine for external hdfs files. * This class represents table engine for external hdfs files.
* Read method is supported for now. * Read method is supported for now.
*/ */
class StorageHDFS : public ext::shared_ptr_helper<StorageHDFS>, public IStorage class StorageHDFS final : public ext::shared_ptr_helper<StorageHDFS>, public IStorage
{ {
friend struct ext::shared_ptr_helper<StorageHDFS>; friend struct ext::shared_ptr_helper<StorageHDFS>;
public: public:

View File

@ -8,7 +8,7 @@ namespace DB
/** Internal temporary storage for table function input(...) /** Internal temporary storage for table function input(...)
*/ */
class StorageInput : public ext::shared_ptr_helper<StorageInput>, public IStorage class StorageInput final : public ext::shared_ptr_helper<StorageInput>, public IStorage
{ {
friend struct ext::shared_ptr_helper<StorageInput>; friend struct ext::shared_ptr_helper<StorageInput>;
public: public:

View File

@ -9,6 +9,7 @@
#include <Interpreters/ExpressionActions.h> #include <Interpreters/ExpressionActions.h>
#include <Interpreters/evaluateConstantExpression.h> #include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/InterpreterSelectQuery.h> #include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/getHeaderForProcessingStage.h>
#include <Parsers/ASTSelectQuery.h> #include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
@ -37,27 +38,6 @@ namespace ErrorCodes
extern const int SAMPLING_NOT_SUPPORTED; extern const int SAMPLING_NOT_SUPPORTED;
} }
namespace
{
/// Rewrite original query removing joined tables from it
void removeJoin(const ASTSelectQuery & select)
{
const auto & tables = select.tables();
if (!tables || tables->children.size() < 2)
return;
const auto & joined_table = tables->children[1]->as<ASTTablesInSelectQueryElement &>();
if (!joined_table.table_join)
return;
/// The most simple temporary solution: leave only the first table in query.
/// TODO: we also need to remove joined columns and related functions (taking in account aliases if any).
tables->children.resize(1);
}
}
StorageMerge::StorageMerge( StorageMerge::StorageMerge(
const StorageID & table_id_, const StorageID & table_id_,
@ -178,7 +158,7 @@ Pipe StorageMerge::read(
modified_context->setSetting("optimize_move_to_prewhere", false); modified_context->setSetting("optimize_move_to_prewhere", false);
/// What will be result structure depending on query processed stage in source tables? /// What will be result structure depending on query processed stage in source tables?
Block header = getQueryHeader(*this, column_names, metadata_snapshot, query_info, context, processed_stage); Block header = getHeaderForProcessingStage(*this, column_names, metadata_snapshot, query_info, context, processed_stage);
/** First we make list of selected tables to find out its size. /** First we make list of selected tables to find out its size.
* This is necessary to correctly pass the recommended number of threads to each table. * This is necessary to correctly pass the recommended number of threads to each table.
@ -430,41 +410,6 @@ void StorageMerge::alter(
setInMemoryMetadata(storage_metadata); setInMemoryMetadata(storage_metadata);
} }
Block StorageMerge::getQueryHeader(
const IStorage & storage,
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum processed_stage)
{
switch (processed_stage)
{
case QueryProcessingStage::FetchColumns:
{
Block header = metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID());
if (query_info.prewhere_info)
{
query_info.prewhere_info->prewhere_actions->execute(header);
if (query_info.prewhere_info->remove_prewhere_column)
header.erase(query_info.prewhere_info->prewhere_column_name);
}
return header;
}
case QueryProcessingStage::WithMergeableState:
case QueryProcessingStage::Complete:
{
auto query = query_info.query->clone();
removeJoin(*query->as<ASTSelectQuery>());
auto stream = std::make_shared<OneBlockInputStream>(
metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()));
return InterpreterSelectQuery(query, context, stream, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
}
}
throw Exception("Logical Error: unknown processed stage.", ErrorCodes::LOGICAL_ERROR);
}
void StorageMerge::convertingSourceStream( void StorageMerge::convertingSourceStream(
const Block & header, const Block & header,
const StorageMetadataPtr & metadata_snapshot, const StorageMetadataPtr & metadata_snapshot,

View File

@ -13,7 +13,7 @@ namespace DB
/** A table that represents the union of an arbitrary number of other tables. /** A table that represents the union of an arbitrary number of other tables.
* All tables must have the same structure. * All tables must have the same structure.
*/ */
class StorageMerge : public ext::shared_ptr_helper<StorageMerge>, public IStorage class StorageMerge final : public ext::shared_ptr_helper<StorageMerge>, public IStorage
{ {
friend struct ext::shared_ptr_helper<StorageMerge>; friend struct ext::shared_ptr_helper<StorageMerge>;
public: public:
@ -47,14 +47,6 @@ public:
bool mayBenefitFromIndexForIn( bool mayBenefitFromIndexForIn(
const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & metadata_snapshot) const override; const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & metadata_snapshot) const override;
static Block getQueryHeader(
const IStorage & storage,
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum processed_stage);
private: private:
String source_database; String source_database;
OptimizedRegularExpression table_name_regexp; OptimizedRegularExpression table_name_regexp;

View File

@ -20,7 +20,7 @@ namespace DB
* Use ENGINE = mysql(host_port, database_name, table_name, user_name, password) * Use ENGINE = mysql(host_port, database_name, table_name, user_name, password)
* Read only. * Read only.
*/ */
class StorageMySQL : public ext::shared_ptr_helper<StorageMySQL>, public IStorage class StorageMySQL final : public ext::shared_ptr_helper<StorageMySQL>, public IStorage
{ {
friend struct ext::shared_ptr_helper<StorageMySQL>; friend struct ext::shared_ptr_helper<StorageMySQL>;
public: public:

View File

@ -5,9 +5,8 @@
#include <Storages/StorageProxy.h> #include <Storages/StorageProxy.h>
#include <Common/CurrentThread.h> #include <Common/CurrentThread.h>
#include <Processors/Transforms/ConvertingTransform.h> #include <Processors/Transforms/ConvertingTransform.h>
#include <Interpreters/getHeaderForProcessingStage.h>
//#include <common/logger_useful.h>
#include <Storages/StorageMerge.h>
namespace DB namespace DB
{ {
@ -28,17 +27,14 @@ public:
StorageInMemoryMetadata cached_metadata; StorageInMemoryMetadata cached_metadata;
cached_metadata.setColumns(std::move(cached_columns)); cached_metadata.setColumns(std::move(cached_columns));
setInMemoryMetadata(cached_metadata); setInMemoryMetadata(cached_metadata);
//log = &Poco::Logger::get("TABLE_FUNCTION_PROXY");
} }
StoragePtr getNested() const override StoragePtr getNested() const override
{ {
//LOG_WARNING(log, "getNested()");
std::lock_guard lock{nested_mutex}; std::lock_guard lock{nested_mutex};
if (nested) if (nested)
return nested; return nested;
//LOG_WARNING(log, "getNested() creating");
auto nested_storage = get_nested(); auto nested_storage = get_nested();
nested_storage->startup(); nested_storage->startup();
nested = nested_storage; nested = nested_storage;
@ -46,12 +42,6 @@ public:
return nested; return nested;
} }
StoragePtr maybeGetNested() const
{
std::lock_guard lock{nested_mutex};
return nested;
}
String getName() const override String getName() const override
{ {
std::lock_guard lock{nested_mutex}; std::lock_guard lock{nested_mutex};
@ -63,9 +53,9 @@ public:
void startup() override { } void startup() override { }
void shutdown() override void shutdown() override
{ {
auto storage = maybeGetNested(); std::lock_guard lock{nested_mutex};
if (storage) if (nested)
storage->shutdown(); nested->shutdown();
} }
Pipe read( Pipe read(
@ -80,19 +70,17 @@ public:
String cnames; String cnames;
for (const auto & c : column_names) for (const auto & c : column_names)
cnames += c + " "; cnames += c + " ";
//LOG_WARNING(log, "read() {} cols: {}", QueryProcessingStage::toString(processed_stage), cnames);
auto storage = getNested(); auto storage = getNested();
auto nested_metadata = storage->getInMemoryMetadataPtr(); auto nested_metadata = storage->getInMemoryMetadataPtr();
auto pipe = storage->read(column_names, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams); auto pipe = storage->read(column_names, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams);
if (!pipe.empty()) if (!pipe.empty())
{ {
auto to_header = getHeaderForProcessingStage(*this, column_names, metadata_snapshot, query_info, context, processed_stage);
pipe.addSimpleTransform([&](const Block & header) pipe.addSimpleTransform([&](const Block & header)
{ {
auto to = StorageMerge::getQueryHeader(*this, column_names, metadata_snapshot, query_info, context, processed_stage);
//LOG_WARNING(log, "try convert \n{}\n to \n{}\n", header.dumpStructure(), to.dumpStructure());
return std::make_shared<ConvertingTransform>( return std::make_shared<ConvertingTransform>(
header, header,
to, to_header,
ConvertingTransform::MatchColumnsMode::Name); ConvertingTransform::MatchColumnsMode::Name);
}); });
} }
@ -116,7 +104,8 @@ public:
void renameInMemory(const StorageID & new_table_id) override void renameInMemory(const StorageID & new_table_id) override
{ {
if (maybeGetNested()) std::lock_guard lock{nested_mutex};
if (nested)
StorageProxy::renameInMemory(new_table_id); StorageProxy::renameInMemory(new_table_id);
else else
IStorage::renameInMemory(new_table_id); IStorage::renameInMemory(new_table_id);
@ -126,7 +115,6 @@ private:
mutable std::mutex nested_mutex; mutable std::mutex nested_mutex;
mutable GetNestedStorageFunc get_nested; mutable GetNestedStorageFunc get_nested;
mutable StoragePtr nested; mutable StoragePtr nested;
//mutable Poco::Logger * log;
}; };
} }

View File

@ -93,7 +93,7 @@ private:
BlockOutputStreamPtr writer; BlockOutputStreamPtr writer;
}; };
class StorageURL : public ext::shared_ptr_helper<StorageURL>, public IStorageURLBase class StorageURL final : public ext::shared_ptr_helper<StorageURL>, public IStorageURLBase
{ {
friend struct ext::shared_ptr_helper<StorageURL>; friend struct ext::shared_ptr_helper<StorageURL>;
public: public:

View File

@ -9,7 +9,7 @@ namespace DB
/* One block storage used for values table function /* One block storage used for values table function
* It's structure is similar to IStorageSystemOneBlock * It's structure is similar to IStorageSystemOneBlock
*/ */
class StorageValues : public ext::shared_ptr_helper<StorageValues>, public IStorage class StorageValues final : public ext::shared_ptr_helper<StorageValues>, public IStorage
{ {
friend struct ext::shared_ptr_helper<StorageValues>; friend struct ext::shared_ptr_helper<StorageValues>;
public: public:

View File

@ -23,7 +23,7 @@ class Context;
* In multithreaded case, if even_distributed is False, implementation with atomic is used, * In multithreaded case, if even_distributed is False, implementation with atomic is used,
* and result is always in [0 ... limit - 1] range. * and result is always in [0 ... limit - 1] range.
*/ */
class StorageSystemNumbers : public ext::shared_ptr_helper<StorageSystemNumbers>, public IStorage class StorageSystemNumbers final : public ext::shared_ptr_helper<StorageSystemNumbers>, public IStorage
{ {
friend struct ext::shared_ptr_helper<StorageSystemNumbers>; friend struct ext::shared_ptr_helper<StorageSystemNumbers>;
public: public:

View File

@ -14,7 +14,7 @@ namespace DB
* You could also specify a limit (how many zeros to give). * You could also specify a limit (how many zeros to give).
* If multithreaded is specified, zeros will be generated in several streams. * If multithreaded is specified, zeros will be generated in several streams.
*/ */
class StorageSystemZeros : public ext::shared_ptr_helper<StorageSystemZeros>, public IStorage class StorageSystemZeros final : public ext::shared_ptr_helper<StorageSystemZeros>, public IStorage
{ {
friend struct ext::shared_ptr_helper<StorageSystemZeros>; friend struct ext::shared_ptr_helper<StorageSystemZeros>;
public: public:

View File

@ -73,10 +73,8 @@ ColumnsDescription getStructureOfRemoteTableInShard(
{ {
if (shard_info.isLocal()) if (shard_info.isLocal())
{ {
const auto * table_function = table_func_ptr->as<ASTFunction>(); TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_func_ptr, context);
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_function->name, context); return table_function_ptr->getActualTableStructure(context);
auto storage_ptr = table_function_ptr->execute(table_func_ptr, context, table_function_ptr->getName());
return storage_ptr->getInMemoryMetadataPtr()->getColumns();
} }
auto table_func_name = queryToString(table_func_ptr); auto table_func_name = queryToString(table_func_ptr);

View File

@ -14,13 +14,12 @@ namespace ProfileEvents
namespace DB namespace DB
{ {
StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns_) const StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns) const
{ {
ProfileEvents::increment(ProfileEvents::TableFunctionExecute); ProfileEvents::increment(ProfileEvents::TableFunctionExecute);
context.checkAccess(AccessType::CREATE_TEMPORARY_TABLE | StorageFactory::instance().getSourceAccessType(getStorageTypeName())); context.checkAccess(AccessType::CREATE_TEMPORARY_TABLE | StorageFactory::instance().getSourceAccessType(getStorageTypeName()));
cached_columns = std::move(cached_columns_);
bool no_conversion_required = hasStaticStructure() && cached_columns == getActualTableStructure(ast_function, context); bool no_conversion_required = hasStaticStructure() && cached_columns == getActualTableStructure(context);
if (cached_columns.empty() || no_conversion_required) if (cached_columns.empty() || no_conversion_required)
return executeImpl(ast_function, context, table_name); return executeImpl(ast_function, context, table_name);
@ -29,7 +28,7 @@ StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context &
return tf->executeImpl(ast_function, context, table_name); return tf->executeImpl(ast_function, context, table_name);
}; };
return std::make_shared<StorageTableFunctionProxy>(StorageID(getDatabaseName(), table_name), std::move(get_storage), cached_columns); return std::make_shared<StorageTableFunctionProxy>(StorageID(getDatabaseName(), table_name), std::move(get_storage), std::move(cached_columns));
} }
} }

View File

@ -34,7 +34,9 @@ public:
virtual bool hasStaticStructure() const { return false; } virtual bool hasStaticStructure() const { return false; }
virtual ColumnsDescription getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) const = 0; virtual void parseArguments(const ASTPtr & /*ast_function*/, const Context & /*context*/) {}
virtual ColumnsDescription getActualTableStructure(const Context & /*context*/) const = 0;
/// Create storage according to the query. /// Create storage according to the query.
StoragePtr execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns_ = {}) const; StoragePtr execute(const ASTPtr & ast_function, const Context & context, const std::string & table_name, ColumnsDescription cached_columns_ = {}) const;
@ -44,9 +46,6 @@ public:
private: private:
virtual StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const = 0; virtual StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const = 0;
virtual const char * getStorageTypeName() const = 0; virtual const char * getStorageTypeName() const = 0;
protected:
mutable ColumnsDescription cached_columns;
}; };
using TableFunctionPtr = std::shared_ptr<ITableFunction>; using TableFunctionPtr = std::shared_ptr<ITableFunction>;

View File

@ -22,11 +22,8 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
} }
void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, const Context & context) const void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, const Context & context)
{ {
if (!filename.empty())
return;
/// Parse args /// Parse args
ASTs & args_func = ast_function->children; ASTs & args_func = ast_function->children;
@ -60,18 +57,16 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, const C
compression_method = args[3]->as<ASTLiteral &>().value.safeGet<String>(); compression_method = args[3]->as<ASTLiteral &>().value.safeGet<String>();
} }
StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const
{ {
parseArguments(ast_function, context); auto columns = getActualTableStructure(context);
auto columns = getActualTableStructure(ast_function, context);
StoragePtr storage = getStorage(filename, format, columns, const_cast<Context &>(context), table_name, compression_method); StoragePtr storage = getStorage(filename, format, columns, const_cast<Context &>(context), table_name, compression_method);
storage->startup(); storage->startup();
return storage; return storage;
} }
ColumnsDescription ITableFunctionFileLike::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const ColumnsDescription ITableFunctionFileLike::getActualTableStructure(const Context & context) const
{ {
parseArguments(ast_function, context);
if (structure.empty()) if (structure.empty())
{ {
assert(getName() == "file" && format == "Distributed"); assert(getName() == "file" && format == "Distributed");

View File

@ -20,13 +20,13 @@ private:
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const String & source, const String & format, const ColumnsDescription & columns, Context & global_context,
const std::string & table_name, const String & compression_method) const = 0; const std::string & table_name, const String & compression_method) const = 0;
ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; ColumnsDescription getActualTableStructure(const Context & context) const override;
void parseArguments(const ASTPtr & ast_function, const Context & context) const; void parseArguments(const ASTPtr & ast_function, const Context & context) override;
mutable String filename; String filename;
mutable String format; String format;
mutable String structure; String structure;
mutable String compression_method = "auto"; String compression_method = "auto";
}; };
} }

View File

@ -25,15 +25,11 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int UNKNOWN_EXCEPTION;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
void ITableFunctionXDBC::parseArguments(const ASTPtr & ast_function, const Context & context) const void ITableFunctionXDBC::parseArguments(const ASTPtr & ast_function, const Context & context)
{ {
if (helper)
return;
const auto & args_func = ast_function->as<ASTFunction &>(); const auto & args_func = ast_function->as<ASTFunction &>();
if (!args_func.arguments) if (!args_func.arguments)
@ -65,9 +61,9 @@ void ITableFunctionXDBC::parseArguments(const ASTPtr & ast_function, const Conte
helper->startBridgeSync(); helper->startBridgeSync();
} }
ColumnsDescription ITableFunctionXDBC::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const ColumnsDescription ITableFunctionXDBC::getActualTableStructure(const Context & context) const
{ {
parseArguments(ast_function, context); assert(helper);
/* Infer external table structure */ /* Infer external table structure */
Poco::URI columns_info_uri = helper->getColumnsInfoURI(); Poco::URI columns_info_uri = helper->getColumnsInfoURI();
@ -89,10 +85,10 @@ ColumnsDescription ITableFunctionXDBC::getActualTableStructure(const ASTPtr & as
return ColumnsDescription{columns}; return ColumnsDescription{columns};
} }
StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const
{ {
parseArguments(ast_function, context); assert(helper);
auto columns = getActualTableStructure(ast_function, context); auto columns = getActualTableStructure(context);
auto result = std::make_shared<StorageXDBC>(StorageID(getDatabaseName(), table_name), schema_name, remote_table_name, columns, context, helper); auto result = std::make_shared<StorageXDBC>(StorageID(getDatabaseName(), table_name), schema_name, remote_table_name, columns, context, helper);
result->startup(); result->startup();
return result; return result;

View File

@ -25,14 +25,14 @@ private:
const Poco::Timespan & http_timeout_, const Poco::Timespan & http_timeout_,
const std::string & connection_string_) const = 0; const std::string & connection_string_) const = 0;
ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; ColumnsDescription getActualTableStructure(const Context & context) const override;
void parseArguments(const ASTPtr & ast_function, const Context & context) const; void parseArguments(const ASTPtr & ast_function, const Context & context) override;
mutable String connection_string; String connection_string;
mutable String schema_name; String schema_name;
mutable String remote_table_name; String remote_table_name;
mutable BridgeHelperPtr helper; BridgeHelperPtr helper;
}; };
class TableFunctionJDBC : public ITableFunctionXDBC class TableFunctionJDBC : public ITableFunctionXDBC

View File

@ -3,6 +3,7 @@
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Parsers/ASTFunction.h>
namespace DB namespace DB
@ -28,19 +29,21 @@ void TableFunctionFactory::registerFunction(const std::string & name, Value crea
} }
TableFunctionPtr TableFunctionFactory::get( TableFunctionPtr TableFunctionFactory::get(
const std::string & name, const ASTPtr & ast_function,
const Context & context) const const Context & context) const
{ {
auto res = tryGet(name, context); const auto * table_function = ast_function->as<ASTFunction>();
auto res = tryGet(table_function->name, context);
if (!res) if (!res)
{ {
auto hints = getHints(name); auto hints = getHints(table_function->name);
if (!hints.empty()) if (!hints.empty())
throw Exception("Unknown table function " + name + ". Maybe you meant: " + toString(hints), ErrorCodes::UNKNOWN_FUNCTION); throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}. Maybe you meant: {}", table_function->name , toString(hints));
else else
throw Exception("Unknown table function " + name, ErrorCodes::UNKNOWN_FUNCTION); throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", table_function->name);
} }
res->parseArguments(ast_function, context);
return res; return res;
} }

View File

@ -41,7 +41,7 @@ public:
} }
/// Throws an exception if not found. /// Throws an exception if not found.
TableFunctionPtr get(const std::string & name, const Context & context) const; TableFunctionPtr get(const ASTPtr & ast_function, const Context & context) const;
/// Returns nullptr if not found. /// Returns nullptr if not found.
TableFunctionPtr tryGet(const std::string & name, const Context & context) const; TableFunctionPtr tryGet(const std::string & name, const Context & context) const;

View File

@ -29,11 +29,8 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, const Context & /*context*/) const void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, const Context & /*context*/)
{ {
if (!structure.empty())
return;
ASTs & args_func = ast_function->children; ASTs & args_func = ast_function->children;
if (args_func.size() != 1) if (args_func.size() != 1)
@ -80,16 +77,14 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, co
max_array_length = args[3]->as<const ASTLiteral &>().value.safeGet<UInt64>(); max_array_length = args[3]->as<const ASTLiteral &>().value.safeGet<UInt64>();
} }
ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(const Context & context) const
{ {
parseArguments(ast_function, context);
return parseColumnsListFromString(structure, context); return parseColumnsListFromString(structure, context);
} }
StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const
{ {
parseArguments(ast_function, context); auto columns = getActualTableStructure(context);
auto columns = getActualTableStructure(ast_function, context);
auto res = StorageGenerateRandom::create(StorageID(getDatabaseName(), table_name), columns, max_array_length, max_string_length, random_seed); auto res = StorageGenerateRandom::create(StorageID(getDatabaseName(), table_name), columns, max_array_length, max_string_length, random_seed);
res->startup(); res->startup();
return res; return res;

View File

@ -17,13 +17,13 @@ private:
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
const char * getStorageTypeName() const override { return "GenerateRandom"; } const char * getStorageTypeName() const override { return "GenerateRandom"; }
ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; ColumnsDescription getActualTableStructure(const Context & context) const override;
void parseArguments(const ASTPtr & ast_function, const Context & context) const; void parseArguments(const ASTPtr & ast_function, const Context & context) override;
mutable String structure; String structure;
mutable UInt64 max_string_length = 10; UInt64 max_string_length = 10;
mutable UInt64 max_array_length = 10; UInt64 max_array_length = 10;
mutable std::optional<UInt64> random_seed; std::optional<UInt64> random_seed;
}; };

View File

@ -24,11 +24,8 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
} }
void TableFunctionInput::parseArguments(const ASTPtr & ast_function, const Context & context) const void TableFunctionInput::parseArguments(const ASTPtr & ast_function, const Context & context)
{ {
if (!structure.empty())
return;
const auto * function = ast_function->as<ASTFunction>(); const auto * function = ast_function->as<ASTFunction>();
if (!function->arguments) if (!function->arguments)
@ -43,16 +40,14 @@ void TableFunctionInput::parseArguments(const ASTPtr & ast_function, const Conte
structure = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context)->as<ASTLiteral &>().value.safeGet<String>(); structure = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context)->as<ASTLiteral &>().value.safeGet<String>();
} }
ColumnsDescription TableFunctionInput::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const ColumnsDescription TableFunctionInput::getActualTableStructure(const Context & context) const
{ {
parseArguments(ast_function, context);
return parseColumnsListFromString(structure, context); return parseColumnsListFromString(structure, context);
} }
StoragePtr TableFunctionInput::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const StoragePtr TableFunctionInput::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const
{ {
parseArguments(ast_function, context); auto storage = StorageInput::create(StorageID(getDatabaseName(), table_name), getActualTableStructure(context));
auto storage = StorageInput::create(StorageID(getDatabaseName(), table_name), getActualTableStructure(ast_function, context));
storage->startup(); storage->startup();
return storage; return storage;
} }

View File

@ -20,10 +20,10 @@ private:
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
const char * getStorageTypeName() const override { return "Input"; } const char * getStorageTypeName() const override { return "Input"; }
ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; ColumnsDescription getActualTableStructure(const Context & context) const override;
void parseArguments(const ASTPtr & ast_function, const Context & context) const; void parseArguments(const ASTPtr & ast_function, const Context & context) override;
mutable String structure; String structure;
}; };
} }

View File

@ -46,7 +46,7 @@ static NamesAndTypesList chooseColumns(const String & source_database, const Str
return any_table->getInMemoryMetadataPtr()->getColumns().getAllPhysical(); return any_table->getInMemoryMetadataPtr()->getColumns().getAllPhysical();
} }
void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, const Context & context) const void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, const Context & context)
{ {
ASTs & args_func = ast_function->children; ASTs & args_func = ast_function->children;
@ -69,19 +69,16 @@ void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, const Conte
table_name_regexp = args[1]->as<ASTLiteral &>().value.safeGet<String>(); table_name_regexp = args[1]->as<ASTLiteral &>().value.safeGet<String>();
} }
ColumnsDescription TableFunctionMerge::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const ColumnsDescription TableFunctionMerge::getActualTableStructure(const Context & context) const
{ {
parseArguments(ast_function, context);
return ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)}; return ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)};
} }
StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const
{ {
parseArguments(ast_function, context);
auto res = StorageMerge::create( auto res = StorageMerge::create(
StorageID(getDatabaseName(), table_name), StorageID(getDatabaseName(), table_name),
getActualTableStructure(ast_function, context), getActualTableStructure(context),
source_database, source_database,
table_name_regexp, table_name_regexp,
context); context);

View File

@ -19,11 +19,11 @@ private:
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
const char * getStorageTypeName() const override { return "Merge"; } const char * getStorageTypeName() const override { return "Merge"; }
ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; ColumnsDescription getActualTableStructure(const Context & context) const override;
void parseArguments(const ASTPtr & ast_function, const Context & context) const; void parseArguments(const ASTPtr & ast_function, const Context & context) override;
mutable String source_database; String source_database;
mutable String table_name_regexp; String table_name_regexp;
}; };

View File

@ -34,7 +34,7 @@ namespace ErrorCodes
extern const int UNKNOWN_TABLE; extern const int UNKNOWN_TABLE;
} }
void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, const Context & context) const void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, const Context & context)
{ {
const auto & args_func = ast_function->as<ASTFunction &>(); const auto & args_func = ast_function->as<ASTFunction &>();
@ -70,9 +70,9 @@ void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, const Conte
parsed_host_port = parseAddress(host_port, 3306); parsed_host_port = parseAddress(host_port, 3306);
} }
ColumnsDescription TableFunctionMySQL::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const ColumnsDescription TableFunctionMySQL::getActualTableStructure(const Context & context) const
{ {
parseArguments(ast_function, context); assert(!parsed_host_port.first.empty());
if (!pool) if (!pool)
pool.emplace(remote_database_name, parsed_host_port.first, user_name, password, parsed_host_port.second); pool.emplace(remote_database_name, parsed_host_port.first, user_name, password, parsed_host_port.second);
@ -121,13 +121,13 @@ ColumnsDescription TableFunctionMySQL::getActualTableStructure(const ASTPtr & as
return ColumnsDescription{columns}; return ColumnsDescription{columns};
} }
StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const
{ {
parseArguments(ast_function, context); assert(!parsed_host_port.first.empty());
if (!pool) if (!pool)
pool.emplace(remote_database_name, parsed_host_port.first, user_name, password, parsed_host_port.second); pool.emplace(remote_database_name, parsed_host_port.first, user_name, password, parsed_host_port.second);
auto columns = getActualTableStructure(ast_function, context); auto columns = getActualTableStructure(context);
auto res = StorageMySQL::create( auto res = StorageMySQL::create(
StorageID(getDatabaseName(), table_name), StorageID(getDatabaseName(), table_name),

View File

@ -27,16 +27,16 @@ private:
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
const char * getStorageTypeName() const override { return "MySQL"; } const char * getStorageTypeName() const override { return "MySQL"; }
ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; ColumnsDescription getActualTableStructure(const Context & context) const override;
void parseArguments(const ASTPtr & ast_function, const Context & context) const; void parseArguments(const ASTPtr & ast_function, const Context & context) override;
mutable std::pair<std::string, UInt16> parsed_host_port; std::pair<std::string, UInt16> parsed_host_port;
mutable String remote_database_name; String remote_database_name;
mutable String remote_table_name; String remote_table_name;
mutable String user_name; String user_name;
mutable String password; String password;
mutable bool replace_query = false; bool replace_query = false;
mutable String on_duplicate_clause; String on_duplicate_clause;
mutable std::optional<mysqlxx::Pool> pool; mutable std::optional<mysqlxx::Pool> pool;
}; };

View File

@ -22,7 +22,7 @@ namespace ErrorCodes
template <bool multithreaded> template <bool multithreaded>
ColumnsDescription TableFunctionNumbers<multithreaded>::getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) const ColumnsDescription TableFunctionNumbers<multithreaded>::getActualTableStructure(const Context & /*context*/) const
{ {
return ColumnsDescription({{"number", std::make_shared<DataTypeUInt64>()}}); return ColumnsDescription({{"number", std::make_shared<DataTypeUInt64>()}});
} }

View File

@ -23,7 +23,7 @@ private:
UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const;
ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; ColumnsDescription getActualTableStructure(const Context & context) const override;
}; };

View File

@ -28,11 +28,8 @@ namespace ErrorCodes
} }
void TableFunctionRemote::prepareClusterInfo(const ASTPtr & ast_function, const Context & context) const void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Context & context)
{ {
if (cluster)
return;
ASTs & args_func = ast_function->children; ASTs & args_func = ast_function->children;
if (args_func.size() != 1) if (args_func.size() != 1)
@ -198,14 +195,13 @@ void TableFunctionRemote::prepareClusterInfo(const ASTPtr & ast_function, const
remote_table_id.table_name = remote_table; remote_table_id.table_name = remote_table;
} }
StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const
{ {
prepareClusterInfo(ast_function, context); assert(cluster);
StoragePtr res = remote_table_function_ptr StoragePtr res = remote_table_function_ptr
? StorageDistributed::create( ? StorageDistributed::create(
StorageID(getDatabaseName(), table_name), StorageID(getDatabaseName(), table_name),
getActualTableStructure(ast_function, context), getActualTableStructure(context),
ConstraintsDescription{}, ConstraintsDescription{},
remote_table_function_ptr, remote_table_function_ptr,
String{}, String{},
@ -217,7 +213,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
cluster) cluster)
: StorageDistributed::create( : StorageDistributed::create(
StorageID(getDatabaseName(), table_name), StorageID(getDatabaseName(), table_name),
getActualTableStructure(ast_function, context), getActualTableStructure(context),
ConstraintsDescription{}, ConstraintsDescription{},
remote_table_id.database_name, remote_table_id.database_name,
remote_table_id.table_name, remote_table_id.table_name,
@ -233,9 +229,9 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
return res; return res;
} }
ColumnsDescription TableFunctionRemote::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const ColumnsDescription TableFunctionRemote::getActualTableStructure(const Context & context) const
{ {
prepareClusterInfo(ast_function, context); assert(cluster);
return getStructureOfRemoteTable(*cluster, remote_table_id, context, remote_table_function_ptr); return getStructureOfRemoteTable(*cluster, remote_table_id, context, remote_table_function_ptr);
} }

View File

@ -22,22 +22,22 @@ public:
std::string getName() const override { return name; } std::string getName() const override { return name; }
ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; ColumnsDescription getActualTableStructure(const Context & context) const override;
private: private:
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
const char * getStorageTypeName() const override { return "Distributed"; } const char * getStorageTypeName() const override { return "Distributed"; }
void prepareClusterInfo(const ASTPtr & ast_function, const Context & context) const; void parseArguments(const ASTPtr & ast_function, const Context & context) override;
std::string name; std::string name;
bool is_cluster_function; bool is_cluster_function;
std::string help_message; std::string help_message;
bool secure; bool secure;
mutable ClusterPtr cluster; ClusterPtr cluster;
mutable StorageID remote_table_id = StorageID::createEmpty(); StorageID remote_table_id = StorageID::createEmpty();
mutable ASTPtr remote_table_function_ptr; ASTPtr remote_table_function_ptr;
}; };
} }

View File

@ -22,10 +22,8 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
} }
void TableFunctionS3::parseArguments(const ASTPtr & ast_function, const Context & context) const void TableFunctionS3::parseArguments(const ASTPtr & ast_function, const Context & context)
{ {
/// Parse args /// Parse args
ASTs & args_func = ast_function->children; ASTs & args_func = ast_function->children;
@ -60,16 +58,13 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, const Context
compression_method = args.back()->as<ASTLiteral &>().value.safeGet<String>(); compression_method = args.back()->as<ASTLiteral &>().value.safeGet<String>();
} }
ColumnsDescription TableFunctionS3::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const ColumnsDescription TableFunctionS3::getActualTableStructure(const Context & context) const
{ {
parseArguments(ast_function, context);
return parseColumnsListFromString(structure, context); return parseColumnsListFromString(structure, context);
} }
StoragePtr TableFunctionS3::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name) const
{ {
parseArguments(ast_function, context);
Poco::URI uri (filename); Poco::URI uri (filename);
S3::URI s3_uri (uri); S3::URI s3_uri (uri);
UInt64 min_upload_part_size = context.getSettingsRef().s3_min_upload_part_size; UInt64 min_upload_part_size = context.getSettingsRef().s3_min_upload_part_size;
@ -81,7 +76,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & ast_function, const Conte
StorageID(getDatabaseName(), table_name), StorageID(getDatabaseName(), table_name),
format, format,
min_upload_part_size, min_upload_part_size,
getActualTableStructure(ast_function, context), getActualTableStructure(context),
ConstraintsDescription{}, ConstraintsDescription{},
const_cast<Context &>(context), const_cast<Context &>(context),
compression_method); compression_method);

View File

@ -31,15 +31,15 @@ protected:
const char * getStorageTypeName() const override { return "S3"; } const char * getStorageTypeName() const override { return "S3"; }
ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; ColumnsDescription getActualTableStructure(const Context & context) const override;
void parseArguments(const ASTPtr & ast_function, const Context & context) const; void parseArguments(const ASTPtr & ast_function, const Context & context) override;
mutable String filename; String filename;
mutable String format; String format;
mutable String structure; String structure;
mutable String access_key_id; String access_key_id;
mutable String secret_access_key; String secret_access_key;
mutable String compression_method = "auto"; String compression_method = "auto";
}; };
class TableFunctionCOS : public TableFunctionS3 class TableFunctionCOS : public TableFunctionS3

View File

@ -14,7 +14,7 @@ StoragePtr TableFunctionURL::getStorage(
const std::string & table_name, const String & compression_method_) const const std::string & table_name, const String & compression_method_) const
{ {
Poco::URI uri(source); Poco::URI uri(source);
return StorageURL::create( uri, StorageID(getDatabaseName(), table_name), format_, columns, ConstraintsDescription{}, return StorageURL::create(uri, StorageID(getDatabaseName(), table_name), format_, columns, ConstraintsDescription{},
global_context, compression_method_); global_context, compression_method_);
} }

View File

@ -63,7 +63,7 @@ static void parseAndInsertValues(MutableColumns & res_columns, const ASTs & args
} }
} }
void TableFunctionValues::parseArguments(const ASTPtr & ast_function, const Context & /*context*/) const void TableFunctionValues::parseArguments(const ASTPtr & ast_function, const Context & /*context*/)
{ {
@ -90,17 +90,14 @@ void TableFunctionValues::parseArguments(const ASTPtr & ast_function, const Cont
structure = args[0]->as<ASTLiteral &>().value.safeGet<String>(); structure = args[0]->as<ASTLiteral &>().value.safeGet<String>();
} }
ColumnsDescription TableFunctionValues::getActualTableStructure(const ASTPtr & ast_function, const Context & context) const ColumnsDescription TableFunctionValues::getActualTableStructure(const Context & context) const
{ {
parseArguments(ast_function, context);
return parseColumnsListFromString(structure, context); return parseColumnsListFromString(structure, context);
} }
StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const
{ {
parseArguments(ast_function, context); auto columns = getActualTableStructure(context);
auto columns = getActualTableStructure(ast_function, context);
Block sample_block; Block sample_block;
for (const auto & name_type : columns.getOrdinary()) for (const auto & name_type : columns.getOrdinary())

View File

@ -16,10 +16,10 @@ private:
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
const char * getStorageTypeName() const override { return "Values"; } const char * getStorageTypeName() const override { return "Values"; }
ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; ColumnsDescription getActualTableStructure(const Context & context) const override;
void parseArguments(const ASTPtr & ast_function, const Context & context) const; void parseArguments(const ASTPtr & ast_function, const Context & context) override;
mutable String structure; String structure;
}; };

View File

@ -21,7 +21,7 @@ extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
template <bool multithreaded> template <bool multithreaded>
ColumnsDescription TableFunctionZeros<multithreaded>::getActualTableStructure(const ASTPtr & /*ast_function*/, const Context & /*context*/) const ColumnsDescription TableFunctionZeros<multithreaded>::getActualTableStructure(const Context & /*context*/) const
{ {
return ColumnsDescription({{"zero", std::make_shared<DataTypeUInt8>()}}); return ColumnsDescription({{"zero", std::make_shared<DataTypeUInt8>()}});
} }

View File

@ -23,7 +23,7 @@ private:
UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const;
ColumnsDescription getActualTableStructure(const ASTPtr & ast_function, const Context & context) const override; ColumnsDescription getActualTableStructure(const Context & context) const override;
}; };