ClickHouse/src/TableFunctions/TableFunctionS3.cpp

102 lines
3.1 KiB
C++
Raw Normal View History

2019-12-06 14:37:21 +00:00
#include <Common/config.h>
#if USE_AWS_S3
2019-12-03 16:23:24 +00:00
#include <IO/S3Common.h>
2019-05-31 07:27:14 +00:00
#include <Storages/StorageS3.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/Context.h>
2019-05-31 07:27:14 +00:00
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/TableFunctionS3.h>
2019-12-11 14:21:48 +00:00
#include <TableFunctions/parseColumnsListForTableFunction.h>
#include <Parsers/ASTLiteral.h>
2019-12-15 06:34:43 +00:00
#include "registerTableFunctions.h"
2019-05-31 07:27:14 +00:00
namespace DB
{
2019-09-22 22:13:42 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int LOGICAL_ERROR;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
2020-09-01 14:41:49 +00:00
void TableFunctionS3::parseArguments(const ASTPtr & ast_function, const Context & context)
{
/// Parse args
ASTs & args_func = ast_function->children;
if (args_func.size() != 1)
throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::LOGICAL_ERROR);
ASTs & args = args_func.at(0)->children;
if (args.size() < 3 || args.size() > 6)
throw Exception("Table function '" + getName() + "' requires 3 to 6 arguments: url, [access_key_id, secret_access_key,] format, structure and [compression_method].",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
2020-03-09 02:31:05 +00:00
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
2020-08-26 20:56:30 +00:00
filename = args[0]->as<ASTLiteral &>().value.safeGet<String>();
if (args.size() < 5)
{
format = args[1]->as<ASTLiteral &>().value.safeGet<String>();
structure = args[2]->as<ASTLiteral &>().value.safeGet<String>();
}
else
{
access_key_id = args[1]->as<ASTLiteral &>().value.safeGet<String>();
secret_access_key = args[2]->as<ASTLiteral &>().value.safeGet<String>();
format = args[3]->as<ASTLiteral &>().value.safeGet<String>();
structure = args[4]->as<ASTLiteral &>().value.safeGet<String>();
}
if (args.size() == 4 || args.size() == 6)
compression_method = args.back()->as<ASTLiteral &>().value.safeGet<String>();
2020-08-26 20:56:30 +00:00
}
2020-09-01 14:41:49 +00:00
ColumnsDescription TableFunctionS3::getActualTableStructure(const Context & context) const
2020-08-26 20:56:30 +00:00
{
return parseColumnsListFromString(structure, context);
}
StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
2020-08-26 20:56:30 +00:00
{
Poco::URI uri (filename);
S3::URI s3_uri (uri);
UInt64 min_upload_part_size = context.getSettingsRef().s3_min_upload_part_size;
StoragePtr storage = StorageS3::create(
2020-08-26 20:56:30 +00:00
s3_uri,
access_key_id,
secret_access_key,
StorageID(getDatabaseName(), table_name),
format,
min_upload_part_size,
2020-09-01 14:41:49 +00:00
getActualTableStructure(context),
2020-08-26 20:56:30 +00:00
ConstraintsDescription{},
const_cast<Context &>(context),
compression_method);
storage->startup();
return storage;
}
2019-05-31 07:27:14 +00:00
void registerTableFunctionS3(TableFunctionFactory & factory)
{
factory.registerFunction<TableFunctionS3>();
}
2019-09-22 22:13:42 +00:00
void registerTableFunctionCOS(TableFunctionFactory & factory)
{
2020-07-17 03:33:29 +00:00
factory.registerFunction<TableFunctionCOS>();
}
2019-05-31 07:27:14 +00:00
}
2019-12-06 14:37:21 +00:00
#endif