2018-06-11 12:13:00 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
|
|
|
#include <Storages/StorageURL.h>
|
|
|
|
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
2020-11-05 11:28:20 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
2018-06-11 12:13:00 +00:00
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
|
2019-11-20 14:48:01 +00:00
|
|
|
#include <IO/ReadHelpers.h>
|
2018-06-11 12:13:00 +00:00
|
|
|
#include <IO/ReadWriteBufferFromHTTP.h>
|
2018-06-14 21:20:39 +00:00
|
|
|
#include <IO/WriteBufferFromHTTP.h>
|
2019-11-20 14:48:01 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
2020-12-10 22:05:02 +00:00
|
|
|
#include <IO/ConnectionTimeouts.h>
|
|
|
|
#include <IO/ConnectionTimeoutsContext.h>
|
2018-06-11 12:13:00 +00:00
|
|
|
|
2018-06-13 07:36:47 +00:00
|
|
|
#include <Formats/FormatFactory.h>
|
2020-05-18 10:00:22 +00:00
|
|
|
#include <Processors/Formats/InputStreamFromInputFormat.h>
|
2018-06-13 07:36:47 +00:00
|
|
|
|
|
|
|
#include <DataStreams/IBlockOutputStream.h>
|
2018-07-11 12:05:04 +00:00
|
|
|
#include <DataStreams/AddingDefaultsBlockInputStream.h>
|
2018-06-11 12:13:00 +00:00
|
|
|
|
|
|
|
#include <Poco/Net/HTTPRequest.h>
|
2020-02-17 15:01:03 +00:00
|
|
|
#include <Processors/Sources/SourceWithProgress.h>
|
|
|
|
#include <Processors/Pipe.h>
|
2018-06-11 12:13:00 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2018-08-10 04:02:56 +00:00
|
|
|
}
|
2018-06-11 12:13:00 +00:00
|
|
|
|
2019-08-24 21:20:20 +00:00
|
|
|
IStorageURLBase::IStorageURLBase(
|
|
|
|
const Poco::URI & uri_,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr /*context_*/,
|
2019-12-04 16:06:55 +00:00
|
|
|
const StorageID & table_id_,
|
2018-06-11 12:13:00 +00:00
|
|
|
const String & format_name_,
|
2020-11-07 08:53:39 +00:00
|
|
|
const std::optional<FormatSettings> & format_settings_,
|
2019-08-24 21:20:20 +00:00
|
|
|
const ColumnsDescription & columns_,
|
2019-11-19 12:46:07 +00:00
|
|
|
const ConstraintsDescription & constraints_,
|
|
|
|
const String & compression_method_)
|
2019-12-04 16:06:55 +00:00
|
|
|
: IStorage(table_id_)
|
|
|
|
, uri(uri_)
|
|
|
|
, compression_method(compression_method_)
|
|
|
|
, format_name(format_name_)
|
2020-11-05 11:28:20 +00:00
|
|
|
, format_settings(format_settings_)
|
2018-06-11 12:13:00 +00:00
|
|
|
{
|
2020-06-19 15:39:41 +00:00
|
|
|
StorageInMemoryMetadata storage_metadata;
|
|
|
|
storage_metadata.setColumns(columns_);
|
|
|
|
storage_metadata.setConstraints(constraints_);
|
|
|
|
setInMemoryMetadata(storage_metadata);
|
2018-06-11 12:13:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
2020-02-17 15:01:03 +00:00
|
|
|
class StorageURLSource : public SourceWithProgress
|
2018-06-11 12:13:00 +00:00
|
|
|
{
|
|
|
|
public:
|
2020-02-17 15:01:03 +00:00
|
|
|
StorageURLSource(const Poco::URI & uri,
|
2018-08-09 18:49:05 +00:00
|
|
|
const std::string & method,
|
|
|
|
std::function<void(std::ostream &)> callback,
|
2018-06-11 12:13:00 +00:00
|
|
|
const String & format,
|
2020-11-07 08:53:39 +00:00
|
|
|
const std::optional<FormatSettings> & format_settings,
|
2020-02-17 15:01:03 +00:00
|
|
|
String name_,
|
2018-06-11 12:13:00 +00:00
|
|
|
const Block & sample_block,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2020-10-02 12:38:50 +00:00
|
|
|
const ColumnsDescription & columns,
|
2019-02-10 16:55:12 +00:00
|
|
|
UInt64 max_block_size,
|
2019-11-19 12:46:07 +00:00
|
|
|
const ConnectionTimeouts & timeouts,
|
|
|
|
const CompressionMethod compression_method)
|
2020-02-17 15:01:03 +00:00
|
|
|
: SourceWithProgress(sample_block), name(std::move(name_))
|
2018-06-11 12:13:00 +00:00
|
|
|
{
|
2020-08-28 01:21:08 +00:00
|
|
|
ReadWriteBufferFromHTTP::HTTPHeaderEntries header;
|
|
|
|
|
|
|
|
// Propagate OpenTelemetry trace context, if any, downstream.
|
2020-11-18 17:43:18 +00:00
|
|
|
if (CurrentThread::isInitialized())
|
2020-08-28 01:21:08 +00:00
|
|
|
{
|
2020-11-18 17:43:18 +00:00
|
|
|
const auto & thread_trace_context = CurrentThread::get().thread_trace_context;
|
2020-11-18 17:49:51 +00:00
|
|
|
if (thread_trace_context.trace_id)
|
2020-08-28 01:21:08 +00:00
|
|
|
{
|
2020-11-18 17:43:18 +00:00
|
|
|
header.emplace_back("traceparent",
|
|
|
|
thread_trace_context.composeTraceparentHeader());
|
|
|
|
|
|
|
|
if (!thread_trace_context.tracestate.empty())
|
|
|
|
{
|
|
|
|
header.emplace_back("tracestate",
|
|
|
|
thread_trace_context.tracestate);
|
|
|
|
}
|
2020-08-28 01:21:08 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-01-04 07:31:00 +00:00
|
|
|
read_buf = wrapReadBufferWithCompressionMethod(
|
|
|
|
std::make_unique<ReadWriteBufferFromHTTP>(
|
|
|
|
uri,
|
|
|
|
method,
|
2020-02-17 15:01:03 +00:00
|
|
|
std::move(callback),
|
2020-01-04 07:31:00 +00:00
|
|
|
timeouts,
|
2021-04-10 23:33:54 +00:00
|
|
|
context->getSettingsRef().max_http_get_redirects,
|
2020-01-04 07:31:00 +00:00
|
|
|
Poco::Net::HTTPBasicCredentials{},
|
|
|
|
DBMS_DEFAULT_BUFFER_SIZE,
|
2020-08-28 01:21:08 +00:00
|
|
|
header,
|
2021-04-10 23:33:54 +00:00
|
|
|
context->getRemoteHostFilter()),
|
2020-01-04 07:31:00 +00:00
|
|
|
compression_method);
|
2019-12-03 17:15:16 +00:00
|
|
|
|
2020-12-07 22:52:51 +00:00
|
|
|
auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size, format_settings);
|
2020-05-18 10:00:22 +00:00
|
|
|
reader = std::make_shared<InputStreamFromInputFormat>(input_format);
|
2020-12-07 22:52:51 +00:00
|
|
|
reader = std::make_shared<AddingDefaultsBlockInputStream>(reader, columns, context);
|
2018-06-11 12:13:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
String getName() const override
|
|
|
|
{
|
|
|
|
return name;
|
|
|
|
}
|
|
|
|
|
2020-02-17 15:01:03 +00:00
|
|
|
Chunk generate() override
|
2018-06-11 12:13:00 +00:00
|
|
|
{
|
2020-02-17 15:01:03 +00:00
|
|
|
if (!reader)
|
|
|
|
return {};
|
2018-06-11 12:13:00 +00:00
|
|
|
|
2020-02-17 15:01:03 +00:00
|
|
|
if (!initialized)
|
|
|
|
reader->readPrefix();
|
2018-06-13 18:51:56 +00:00
|
|
|
|
2020-02-19 10:17:49 +00:00
|
|
|
initialized = true;
|
|
|
|
|
2020-02-17 15:01:03 +00:00
|
|
|
if (auto block = reader->read())
|
|
|
|
return Chunk(block.getColumns(), block.rows());
|
2018-06-11 12:13:00 +00:00
|
|
|
|
|
|
|
reader->readSuffix();
|
2020-02-17 15:01:03 +00:00
|
|
|
reader.reset();
|
|
|
|
|
|
|
|
return {};
|
2018-06-11 12:13:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
String name;
|
2019-11-19 12:46:07 +00:00
|
|
|
std::unique_ptr<ReadBuffer> read_buf;
|
2018-06-11 12:13:00 +00:00
|
|
|
BlockInputStreamPtr reader;
|
2020-02-17 15:01:03 +00:00
|
|
|
bool initialized = false;
|
2018-06-11 12:13:00 +00:00
|
|
|
};
|
|
|
|
}
|
2018-06-16 05:54:06 +00:00
|
|
|
|
2020-04-28 00:56:44 +00:00
|
|
|
StorageURLBlockOutputStream::StorageURLBlockOutputStream(const Poco::URI & uri,
|
|
|
|
const String & format,
|
2020-11-07 08:53:39 +00:00
|
|
|
const std::optional<FormatSettings> & format_settings,
|
2020-04-28 00:56:44 +00:00
|
|
|
const Block & sample_block_,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2020-04-28 00:56:44 +00:00
|
|
|
const ConnectionTimeouts & timeouts,
|
|
|
|
const CompressionMethod compression_method)
|
|
|
|
: sample_block(sample_block_)
|
|
|
|
{
|
|
|
|
write_buf = wrapWriteBufferWithCompressionMethod(
|
|
|
|
std::make_unique<WriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts),
|
|
|
|
compression_method, 3);
|
2020-12-30 03:07:30 +00:00
|
|
|
writer = FormatFactory::instance().getOutputStream(format, *write_buf, sample_block,
|
2020-11-05 11:28:20 +00:00
|
|
|
context, {} /* write callback */, format_settings);
|
2020-04-28 00:56:44 +00:00
|
|
|
}
|
2018-06-16 05:54:06 +00:00
|
|
|
|
2020-07-09 01:00:16 +00:00
|
|
|
|
|
|
|
void StorageURLBlockOutputStream::write(const Block & block)
|
|
|
|
{
|
|
|
|
writer->write(block);
|
|
|
|
}
|
|
|
|
|
|
|
|
void StorageURLBlockOutputStream::writePrefix()
|
|
|
|
{
|
|
|
|
writer->writePrefix();
|
|
|
|
}
|
|
|
|
|
|
|
|
void StorageURLBlockOutputStream::writeSuffix()
|
|
|
|
{
|
|
|
|
writer->writeSuffix();
|
|
|
|
writer->flush();
|
|
|
|
write_buf->finalize();
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-08-09 18:49:05 +00:00
|
|
|
std::string IStorageURLBase::getReadMethod() const
|
|
|
|
{
|
|
|
|
return Poco::Net::HTTPRequest::HTTP_GET;
|
|
|
|
}
|
|
|
|
|
2020-06-17 16:39:58 +00:00
|
|
|
std::vector<std::pair<std::string, std::string>> IStorageURLBase::getReadURIParams(
|
|
|
|
const Names & /*column_names*/,
|
|
|
|
const StorageMetadataPtr & /*metadata_snapshot*/,
|
2018-06-11 12:13:00 +00:00
|
|
|
const SelectQueryInfo & /*query_info*/,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr /*context*/,
|
2018-06-11 12:13:00 +00:00
|
|
|
QueryProcessingStage::Enum & /*processed_stage*/,
|
2018-08-09 18:49:05 +00:00
|
|
|
size_t /*max_block_size*/) const
|
|
|
|
{
|
|
|
|
return {};
|
|
|
|
}
|
|
|
|
|
2020-06-17 16:39:58 +00:00
|
|
|
std::function<void(std::ostream &)> IStorageURLBase::getReadPOSTDataCallback(
|
|
|
|
const Names & /*column_names*/,
|
|
|
|
const StorageMetadataPtr & /*metadata_snapshot*/,
|
2018-06-11 12:13:00 +00:00
|
|
|
const SelectQueryInfo & /*query_info*/,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr /*context*/,
|
2018-08-09 18:49:05 +00:00
|
|
|
QueryProcessingStage::Enum & /*processed_stage*/,
|
|
|
|
size_t /*max_block_size*/) const
|
|
|
|
{
|
|
|
|
return nullptr;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-08-03 13:54:14 +00:00
|
|
|
Pipe IStorageURLBase::read(
|
2020-06-15 19:08:58 +00:00
|
|
|
const Names & column_names,
|
2020-06-16 14:25:08 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2020-09-20 17:52:17 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr local_context,
|
2018-06-25 12:21:54 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
2019-02-18 23:38:44 +00:00
|
|
|
size_t max_block_size,
|
2018-06-11 12:13:00 +00:00
|
|
|
unsigned /*num_streams*/)
|
|
|
|
{
|
2018-08-09 18:49:05 +00:00
|
|
|
auto request_uri = uri;
|
2021-04-10 23:33:54 +00:00
|
|
|
auto params = getReadURIParams(column_names, metadata_snapshot, query_info, local_context, processed_stage, max_block_size);
|
2018-08-09 18:49:05 +00:00
|
|
|
for (const auto & [param, value] : params)
|
|
|
|
request_uri.addQueryParameter(param, value);
|
|
|
|
|
2020-08-03 13:54:14 +00:00
|
|
|
return Pipe(std::make_shared<StorageURLSource>(
|
2020-06-16 14:25:08 +00:00
|
|
|
request_uri,
|
2018-08-09 18:49:05 +00:00
|
|
|
getReadMethod(),
|
2020-06-17 16:39:58 +00:00
|
|
|
getReadPOSTDataCallback(
|
|
|
|
column_names, metadata_snapshot, query_info,
|
2021-04-10 23:33:54 +00:00
|
|
|
local_context, processed_stage, max_block_size),
|
2018-06-11 12:13:00 +00:00
|
|
|
format_name,
|
2020-11-05 11:28:20 +00:00
|
|
|
format_settings,
|
2018-06-11 12:13:00 +00:00
|
|
|
getName(),
|
2020-06-16 14:25:08 +00:00
|
|
|
getHeaderBlock(column_names, metadata_snapshot),
|
2021-04-10 23:33:54 +00:00
|
|
|
local_context,
|
2020-10-02 12:38:50 +00:00
|
|
|
metadata_snapshot->getColumns(),
|
2018-06-11 12:13:00 +00:00
|
|
|
max_block_size,
|
2021-04-10 23:33:54 +00:00
|
|
|
ConnectionTimeouts::getHTTPTimeouts(local_context),
|
2020-02-17 15:01:03 +00:00
|
|
|
chooseCompressionMethod(request_uri.getPath(), compression_method)));
|
2018-06-11 12:13:00 +00:00
|
|
|
}
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
BlockOutputStreamPtr IStorageURLBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context)
|
2018-06-11 12:13:00 +00:00
|
|
|
{
|
2020-11-05 11:28:20 +00:00
|
|
|
return std::make_shared<StorageURLBlockOutputStream>(uri, format_name,
|
2021-03-19 14:02:48 +00:00
|
|
|
format_settings, metadata_snapshot->getSampleBlock(), context,
|
|
|
|
ConnectionTimeouts::getHTTPTimeouts(context),
|
2020-01-04 07:31:00 +00:00
|
|
|
chooseCompressionMethod(uri.toString(), compression_method));
|
2018-06-11 12:13:00 +00:00
|
|
|
}
|
2018-06-16 05:54:06 +00:00
|
|
|
|
2021-03-23 11:29:29 +00:00
|
|
|
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_,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context_,
|
2021-03-23 11:29:29 +00:00
|
|
|
const String & compression_method_)
|
|
|
|
: IStorageURLBase(uri_, context_, table_id_, format_name_,
|
|
|
|
format_settings_, columns_, constraints_, compression_method_)
|
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
context_->getRemoteHostFilter().checkURL(uri);
|
2021-03-23 11:29:29 +00:00
|
|
|
}
|
|
|
|
|
2018-06-11 12:13:00 +00:00
|
|
|
void registerStorageURL(StorageFactory & factory)
|
|
|
|
{
|
2018-08-24 00:07:25 +00:00
|
|
|
factory.registerStorage("URL", [](const StorageFactory::Arguments & args)
|
|
|
|
{
|
2018-06-11 12:13:00 +00:00
|
|
|
ASTs & engine_args = args.engine_args;
|
|
|
|
|
2019-11-19 12:46:07 +00:00
|
|
|
if (engine_args.size() != 2 && engine_args.size() != 3)
|
2018-06-11 12:13:00 +00:00
|
|
|
throw Exception(
|
2019-11-19 12:46:07 +00:00
|
|
|
"Storage URL requires 2 or 3 arguments: url, name of used format and optional compression method.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
2018-06-11 12:13:00 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.getLocalContext());
|
2018-06-11 12:13:00 +00:00
|
|
|
|
2019-03-15 17:09:14 +00:00
|
|
|
String url = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
2018-06-11 12:13:00 +00:00
|
|
|
Poco::URI uri(url);
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], args.getLocalContext());
|
2018-06-11 12:13:00 +00:00
|
|
|
|
2019-03-15 17:09:14 +00:00
|
|
|
String format_name = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
2018-06-11 12:13:00 +00:00
|
|
|
|
2019-11-19 12:46:07 +00:00
|
|
|
String compression_method;
|
|
|
|
if (engine_args.size() == 3)
|
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.getLocalContext());
|
2019-11-19 12:46:07 +00:00
|
|
|
compression_method = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
2020-11-05 11:28:20 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
compression_method = "auto";
|
|
|
|
}
|
|
|
|
|
|
|
|
// Use format settings from global server context + settings from
|
|
|
|
// the SETTINGS clause of the create query. Settings from current
|
|
|
|
// session and user are ignored.
|
|
|
|
FormatSettings format_settings;
|
|
|
|
if (args.storage_def->settings)
|
|
|
|
{
|
2020-11-07 08:53:39 +00:00
|
|
|
FormatFactorySettings user_format_settings;
|
|
|
|
|
|
|
|
// Apply changed settings from global context, but ignore the
|
|
|
|
// unknown ones, because we only have the format settings here.
|
2021-04-10 23:33:54 +00:00
|
|
|
const auto & changes = args.getContext()->getSettingsRef().changes();
|
2020-11-07 08:53:39 +00:00
|
|
|
for (const auto & change : changes)
|
|
|
|
{
|
|
|
|
if (user_format_settings.has(change.name))
|
|
|
|
{
|
|
|
|
user_format_settings.set(change.name, change.value);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Apply changes from SETTINGS clause, with validation.
|
|
|
|
user_format_settings.applyChanges(args.storage_def->settings->changes);
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
format_settings = getFormatSettings(args.getContext(),
|
2020-11-07 08:53:39 +00:00
|
|
|
user_format_settings);
|
2020-11-05 11:28:20 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
format_settings = getFormatSettings(args.getContext());
|
2020-11-05 11:28:20 +00:00
|
|
|
}
|
2019-11-19 12:46:07 +00:00
|
|
|
|
|
|
|
return StorageURL::create(
|
|
|
|
uri,
|
2019-12-04 16:06:55 +00:00
|
|
|
args.table_id,
|
2019-11-19 12:46:07 +00:00
|
|
|
format_name,
|
2020-11-05 11:28:20 +00:00
|
|
|
format_settings,
|
2021-04-10 23:33:54 +00:00
|
|
|
args.columns, args.constraints, args.getContext(),
|
2019-11-19 12:46:07 +00:00
|
|
|
compression_method);
|
2020-04-06 05:19:40 +00:00
|
|
|
},
|
|
|
|
{
|
2020-11-05 11:28:20 +00:00
|
|
|
.supports_settings = true,
|
2020-04-06 05:19:40 +00:00
|
|
|
.source_access_type = AccessType::URL,
|
2018-06-11 12:13:00 +00:00
|
|
|
});
|
|
|
|
}
|
|
|
|
}
|