mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-12 02:23:14 +00:00
Table function and storage.
This commit is contained in:
parent
47985cf8a7
commit
ff09934219
@ -1,126 +1,176 @@
|
||||
#include <Storages/StorageS3.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/StorageS3.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
||||
#include <IO/ReadWriteBufferFromHTTP.h>
|
||||
#include <IO/WriteBufferFromHTTP.h>
|
||||
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <DataStreams/AddingDefaultsBlockInputStream.h>
|
||||
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <Poco/Path.h>
|
||||
#include <Poco/File.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR;
|
||||
extern const int CANNOT_SEEK_THROUGH_FILE;
|
||||
extern const int DATABASE_ACCESS_DENIED;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int UNKNOWN_IDENTIFIER;
|
||||
extern const int INCORRECT_FILE_NAME;
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
||||
}
|
||||
|
||||
|
||||
StorageS3::StorageS3(
|
||||
const std::string & table_uri_,
|
||||
const std::string & table_name_,
|
||||
const std::string & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
Context & context_)
|
||||
: IStorage(columns_)
|
||||
, table_name(table_name_)
|
||||
, format_name(format_name_)
|
||||
, context_global(context_)
|
||||
, uri(table_uri_)
|
||||
IStorageS3Base::IStorageS3Base(const Poco::URI & uri_,
|
||||
const Context & context_,
|
||||
const std::string & table_name_,
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_)
|
||||
: IStorage(columns_), uri(uri_), context_global(context_), format_name(format_name_), table_name(table_name_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
class StorageS3BlockInputStream : public IBlockInputStream
|
||||
namespace
|
||||
{
|
||||
public:
|
||||
StorageS3BlockInputStream(const Poco::URI & uri,
|
||||
const std::string & method,
|
||||
std::function<void(std::ostream &)> callback,
|
||||
const String & format,
|
||||
const String & name_,
|
||||
const Block & sample_block,
|
||||
const Context & context,
|
||||
UInt64 max_block_size,
|
||||
const ConnectionTimeouts & timeouts)
|
||||
: name(name_)
|
||||
class StorageS3BlockInputStream : public IBlockInputStream
|
||||
{
|
||||
read_buf = std::make_unique<ReadWriteBufferFromHTTP>(uri, method, callback, timeouts);
|
||||
public:
|
||||
StorageS3BlockInputStream(const Poco::URI & uri,
|
||||
const std::string & method,
|
||||
std::function<void(std::ostream &)> callback,
|
||||
const String & format,
|
||||
const String & name_,
|
||||
const Block & sample_block,
|
||||
const Context & context,
|
||||
UInt64 max_block_size,
|
||||
const ConnectionTimeouts & timeouts)
|
||||
: name(name_)
|
||||
{
|
||||
read_buf = std::make_unique<ReadWriteBufferFromHTTP>(uri, method, callback, timeouts);
|
||||
|
||||
reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size);
|
||||
}
|
||||
reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size);
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
Block readImpl() override
|
||||
{
|
||||
return reader->read();
|
||||
}
|
||||
|
||||
Block getHeader() const override
|
||||
{
|
||||
return reader->getHeader();
|
||||
}
|
||||
|
||||
void readPrefixImpl() override
|
||||
{
|
||||
reader->readPrefix();
|
||||
}
|
||||
|
||||
void readSuffixImpl() override
|
||||
{
|
||||
reader->readSuffix();
|
||||
}
|
||||
|
||||
private:
|
||||
String name;
|
||||
std::unique_ptr<ReadWriteBufferFromHTTP> read_buf;
|
||||
BlockInputStreamPtr reader;
|
||||
};
|
||||
|
||||
class StorageS3BlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
return name;
|
||||
}
|
||||
public:
|
||||
StorageS3BlockOutputStream(const Poco::URI & uri,
|
||||
const String & format,
|
||||
const Block & sample_block_,
|
||||
const Context & context,
|
||||
const ConnectionTimeouts & timeouts)
|
||||
: sample_block(sample_block_)
|
||||
{
|
||||
write_buf = std::make_unique<WriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts);
|
||||
writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context);
|
||||
}
|
||||
|
||||
Block readImpl() override
|
||||
{
|
||||
return reader->read();
|
||||
}
|
||||
Block getHeader() const override
|
||||
{
|
||||
return sample_block;
|
||||
}
|
||||
|
||||
Block getHeader() const override
|
||||
{
|
||||
return reader->getHeader();
|
||||
}
|
||||
void write(const Block & block) override
|
||||
{
|
||||
writer->write(block);
|
||||
}
|
||||
|
||||
void readPrefixImpl() override
|
||||
{
|
||||
reader->readPrefix();
|
||||
}
|
||||
void writePrefix() override
|
||||
{
|
||||
writer->writePrefix();
|
||||
}
|
||||
|
||||
void readSuffixImpl() override
|
||||
{
|
||||
reader->readSuffix();
|
||||
}
|
||||
void writeSuffix() override
|
||||
{
|
||||
writer->writeSuffix();
|
||||
writer->flush();
|
||||
write_buf->finalize();
|
||||
}
|
||||
|
||||
private:
|
||||
String name;
|
||||
std::unique_ptr<ReadWriteBufferFromHTTP> read_buf;
|
||||
BlockInputStreamPtr reader;
|
||||
};
|
||||
private:
|
||||
Block sample_block;
|
||||
std::unique_ptr<WriteBufferFromHTTP> write_buf;
|
||||
BlockOutputStreamPtr writer;
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
BlockInputStreams StorageS3::read(
|
||||
const Names & column_names,
|
||||
std::string IStorageS3Base::getReadMethod() const
|
||||
{
|
||||
return Poco::Net::HTTPRequest::HTTP_GET;
|
||||
}
|
||||
|
||||
std::vector<std::pair<std::string, std::string>> IStorageS3Base::getReadURIParams(const Names & /*column_names*/,
|
||||
const SelectQueryInfo & /*query_info*/,
|
||||
const Context & /*context*/,
|
||||
QueryProcessingStage::Enum & /*processed_stage*/,
|
||||
size_t /*max_block_size*/) const
|
||||
{
|
||||
return {};
|
||||
}
|
||||
|
||||
std::function<void(std::ostream &)> IStorageS3Base::getReadPOSTDataCallback(const Names & /*column_names*/,
|
||||
const SelectQueryInfo & /*query_info*/,
|
||||
const Context & /*context*/,
|
||||
QueryProcessingStage::Enum & /*processed_stage*/,
|
||||
size_t /*max_block_size*/) const
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
|
||||
BlockInputStreams IStorageS3Base::read(const Names & column_names,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum /*processed_stage*/,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t max_block_size,
|
||||
unsigned /*num_streams*/)
|
||||
{
|
||||
auto request_uri = uri;
|
||||
auto params = getReadURIParams(column_names, query_info, context, processed_stage, max_block_size);
|
||||
for (const auto & [param, value] : params)
|
||||
request_uri.addQueryParameter(param, value);
|
||||
|
||||
BlockInputStreamPtr block_input = std::make_shared<StorageS3BlockInputStream>(request_uri,
|
||||
Poco::Net::HTTPRequest::HTTP_GET,
|
||||
nullptr,
|
||||
//getReadPOSTDataCallback(column_names, query_info, context, processed_stage, max_block_size),
|
||||
getReadMethod(),
|
||||
getReadPOSTDataCallback(column_names, query_info, context, processed_stage, max_block_size),
|
||||
format_name,
|
||||
getName(),
|
||||
getSampleBlockForColumns(column_names),
|
||||
getHeaderBlock(column_names),
|
||||
context,
|
||||
max_block_size,
|
||||
ConnectionTimeouts::getHTTPTimeouts(context));
|
||||
@ -132,98 +182,16 @@ BlockInputStreams StorageS3::read(
|
||||
return {std::make_shared<AddingDefaultsBlockInputStream>(block_input, column_defaults, context)};
|
||||
}
|
||||
|
||||
void IStorageS3Base::rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/) {}
|
||||
|
||||
class StorageS3BlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
StorageS3BlockOutputStream(const Poco::URI & uri,
|
||||
const String & format,
|
||||
const Block & sample_block_,
|
||||
const Context & context,
|
||||
const ConnectionTimeouts & timeouts)
|
||||
: sample_block(sample_block_)
|
||||
{
|
||||
write_buf = std::make_unique<WriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts);
|
||||
writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context);
|
||||
}
|
||||
|
||||
Block getHeader() const override
|
||||
{
|
||||
return sample_block;
|
||||
}
|
||||
|
||||
void write(const Block & block) override
|
||||
{
|
||||
writer->write(block);
|
||||
}
|
||||
|
||||
void writePrefix() override
|
||||
{
|
||||
writer->writePrefix();
|
||||
}
|
||||
|
||||
void writeSuffix() override
|
||||
{
|
||||
writer->writeSuffix();
|
||||
writer->flush();
|
||||
write_buf->finalize();
|
||||
}
|
||||
|
||||
private:
|
||||
Block sample_block;
|
||||
std::unique_ptr<WriteBufferFromHTTP> write_buf;
|
||||
BlockOutputStreamPtr writer;
|
||||
};
|
||||
|
||||
BlockOutputStreamPtr StorageS3::write(
|
||||
const ASTPtr & /*query*/,
|
||||
const Context & /*context*/)
|
||||
BlockOutputStreamPtr IStorageS3Base::write(const ASTPtr & /*query*/, const Context & /*context*/)
|
||||
{
|
||||
return std::make_shared<StorageS3BlockOutputStream>(
|
||||
uri, format_name, getSampleBlock(), context_global, ConnectionTimeouts::getHTTPTimeouts(context_global));
|
||||
}
|
||||
|
||||
|
||||
void StorageS3::drop()
|
||||
void registerStorageS3(StorageFactory & /*factory*/)
|
||||
{
|
||||
/// Extra actions are not required.
|
||||
// TODO. See #1394.
|
||||
}
|
||||
|
||||
|
||||
void StorageS3::rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/) {}
|
||||
|
||||
|
||||
void registerStorageS3(StorageFactory & factory)
|
||||
{
|
||||
factory.registerStorage("S3", [](const StorageFactory::Arguments & args)
|
||||
{
|
||||
ASTs & engine_args = args.engine_args;
|
||||
|
||||
if (!(engine_args.size() == 2))
|
||||
throw Exception(
|
||||
"Storage S3 requires 2 arguments: name of used format and source.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context);
|
||||
String format_name = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
String source_path;
|
||||
if (const auto * literal = engine_args[1]->as<ASTLiteral>())
|
||||
{
|
||||
auto type = literal->value.getType();
|
||||
if (type == Field::Types::String)
|
||||
{
|
||||
source_path = literal->value.get<String>();
|
||||
return StorageS3::create(
|
||||
source_path,
|
||||
args.table_name, format_name, args.columns,
|
||||
args.context);
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception("Unknown entity in first arg of S3 storage constructor, String expected.",
|
||||
ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,82 +1,88 @@
|
||||
#pragma once
|
||||
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
#include <Poco/File.h>
|
||||
#include <Poco/Path.h>
|
||||
#include <Poco/URI.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <shared_mutex>
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class StorageS3BlockInputStream;
|
||||
class StorageS3BlockOutputStream;
|
||||
|
||||
class StorageS3 : public ext::shared_ptr_helper<StorageS3>, public IStorage
|
||||
/**
|
||||
* This class represents table engine for external urls.
|
||||
* It sends HTTP GET to server when select is called and
|
||||
* HTTP POST when insert is called. In POST request the data is send
|
||||
* using Chunked transfer encoding, so server have to support it.
|
||||
*/
|
||||
class IStorageS3Base : public IStorage
|
||||
{
|
||||
public:
|
||||
std::string getName() const override
|
||||
{
|
||||
return "S3";
|
||||
}
|
||||
|
||||
std::string getTableName() const override
|
||||
String getTableName() const override
|
||||
{
|
||||
return table_name;
|
||||
}
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
BlockInputStreams read(const Names & column_names,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t max_block_size,
|
||||
unsigned num_streams) override;
|
||||
|
||||
BlockOutputStreamPtr write(
|
||||
const ASTPtr & query,
|
||||
const Context & context) override;
|
||||
|
||||
void drop() override;
|
||||
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
||||
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override;
|
||||
|
||||
protected:
|
||||
friend class StorageS3BlockInputStream;
|
||||
friend class StorageS3BlockOutputStream;
|
||||
|
||||
/** there are three options (ordered by priority):
|
||||
- use specified file descriptor if (fd >= 0)
|
||||
- use specified table_path if it isn't empty
|
||||
- create own table inside data/db/table/
|
||||
*/
|
||||
StorageS3(
|
||||
const std::string & table_uri_,
|
||||
IStorageS3Base(const Poco::URI & uri_,
|
||||
const Context & context_,
|
||||
const std::string & table_name_,
|
||||
const std::string & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
Context & context_);
|
||||
|
||||
private:
|
||||
|
||||
std::string table_name;
|
||||
std::string format_name;
|
||||
Context & context_global;
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_);
|
||||
|
||||
Poco::URI uri;
|
||||
const Context & context_global;
|
||||
|
||||
bool is_db_table = true; /// Table is stored in real database, not user's file
|
||||
private:
|
||||
String format_name;
|
||||
String table_name;
|
||||
|
||||
mutable std::shared_mutex rwlock;
|
||||
virtual std::string getReadMethod() const;
|
||||
|
||||
Logger * log = &Logger::get("StorageS3");
|
||||
virtual std::vector<std::pair<std::string, std::string>> getReadURIParams(const Names & column_names,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum & processed_stage,
|
||||
size_t max_block_size) const;
|
||||
|
||||
virtual std::function<void(std::ostream &)> getReadPOSTDataCallback(const Names & column_names,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum & processed_stage,
|
||||
size_t max_block_size) const;
|
||||
|
||||
virtual Block getHeaderBlock(const Names & column_names) const = 0;
|
||||
};
|
||||
|
||||
class StorageS3 : public ext::shared_ptr_helper<StorageS3>, public IStorageS3Base
|
||||
{
|
||||
public:
|
||||
StorageS3(const Poco::URI & uri_,
|
||||
const std::string & table_name_,
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
Context & context_)
|
||||
: IStorageS3Base(uri_, context_, table_name_, format_name_, columns_)
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return "S3";
|
||||
}
|
||||
|
||||
Block getHeaderBlock(const Names & /*column_names*/) const override
|
||||
{
|
||||
return getSampleBlock();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -19,6 +19,7 @@ void registerStorageDistributed(StorageFactory & factory);
|
||||
void registerStorageMemory(StorageFactory & factory);
|
||||
void registerStorageFile(StorageFactory & factory);
|
||||
void registerStorageURL(StorageFactory & factory);
|
||||
void registerStorageS3(StorageFactory & factory);
|
||||
void registerStorageDictionary(StorageFactory & factory);
|
||||
void registerStorageSet(StorageFactory & factory);
|
||||
void registerStorageJoin(StorageFactory & factory);
|
||||
@ -60,6 +61,7 @@ void registerStorages()
|
||||
registerStorageMemory(factory);
|
||||
registerStorageFile(factory);
|
||||
registerStorageURL(factory);
|
||||
registerStorageS3(factory);
|
||||
registerStorageDictionary(factory);
|
||||
registerStorageSet(factory);
|
||||
registerStorageJoin(factory);
|
||||
|
19
dbms/src/TableFunctions/TableFunctionS3.cpp
Normal file
19
dbms/src/TableFunctions/TableFunctionS3.cpp
Normal file
@ -0,0 +1,19 @@
|
||||
#include <Storages/StorageS3.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/TableFunctionS3.h>
|
||||
#include <Poco/URI.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
StoragePtr TableFunctionS3::getStorage(
|
||||
const String & source, const String & format, const Block & sample_block, Context & global_context) const
|
||||
{
|
||||
Poco::URI uri(source);
|
||||
return StorageS3::create(uri, getName(), format, ColumnsDescription{sample_block.getNamesAndTypesList()}, global_context);
|
||||
}
|
||||
|
||||
void registerTableFunctionS3(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionS3>();
|
||||
}
|
||||
}
|
25
dbms/src/TableFunctions/TableFunctionS3.h
Normal file
25
dbms/src/TableFunctions/TableFunctionS3.h
Normal file
@ -0,0 +1,25 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunctionFileLike.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Core/Block.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/* url(source, format, structure) - creates a temporary storage from url
|
||||
*/
|
||||
class TableFunctionS3 : public ITableFunctionFileLike
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "s3";
|
||||
std::string getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
private:
|
||||
StoragePtr getStorage(
|
||||
const String & source, const String & format, const Block & sample_block, Context & global_context) const override;
|
||||
};
|
||||
}
|
@ -11,6 +11,7 @@ void registerTableFunctionMerge(TableFunctionFactory & factory);
|
||||
void registerTableFunctionRemote(TableFunctionFactory & factory);
|
||||
void registerTableFunctionNumbers(TableFunctionFactory & factory);
|
||||
void registerTableFunctionFile(TableFunctionFactory & factory);
|
||||
void registerTableFunctionS3(TableFunctionFactory & factory);
|
||||
void registerTableFunctionURL(TableFunctionFactory & factory);
|
||||
void registerTableFunctionValues(TableFunctionFactory & factory);
|
||||
|
||||
@ -37,6 +38,7 @@ void registerTableFunctions()
|
||||
registerTableFunctionRemote(factory);
|
||||
registerTableFunctionNumbers(factory);
|
||||
registerTableFunctionFile(factory);
|
||||
registerTableFunctionS3(factory);
|
||||
registerTableFunctionURL(factory);
|
||||
registerTableFunctionValues(factory);
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user