2018-06-11 12:13:00 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <Poco/URI.h>
|
2021-07-23 14:25:35 +00:00
|
|
|
#include <Processors/Sinks/SinkToStorage.h>
|
2023-04-21 17:24:37 +00:00
|
|
|
#include <Processors/ISource.h>
|
2020-12-10 22:05:02 +00:00
|
|
|
#include <Formats/FormatSettings.h>
|
2020-04-28 00:56:44 +00:00
|
|
|
#include <IO/CompressionMethod.h>
|
2021-09-07 11:17:25 +00:00
|
|
|
#include <IO/ReadWriteBufferFromHTTP.h>
|
2022-12-16 22:57:09 +00:00
|
|
|
#include <IO/HTTPHeaderEntries.h>
|
|
|
|
#include <Storages/IStorage.h>
|
2021-04-21 12:32:57 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
2022-06-21 13:02:48 +00:00
|
|
|
#include <Storages/Cache/SchemaCache.h>
|
2023-04-21 12:11:18 +00:00
|
|
|
#include <Storages/StorageConfiguration.h>
|
2018-06-11 12:13:00 +00:00
|
|
|
|
2019-08-24 21:20:20 +00:00
|
|
|
|
2018-06-11 12:13:00 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
2020-12-10 22:05:02 +00:00
|
|
|
|
2021-10-11 16:11:50 +00:00
|
|
|
class IOutputFormat;
|
|
|
|
using OutputFormatPtr = std::shared_ptr<IOutputFormat>;
|
|
|
|
|
2023-06-16 15:51:18 +00:00
|
|
|
class IInputFormat;
|
2020-12-10 22:05:02 +00:00
|
|
|
struct ConnectionTimeouts;
|
2022-12-16 22:57:09 +00:00
|
|
|
class NamedCollection;
|
2023-04-21 17:24:37 +00:00
|
|
|
class PullingPipelineExecutor;
|
2020-12-10 22:05:02 +00:00
|
|
|
|
2018-06-11 12:13:00 +00:00
|
|
|
/**
|
|
|
|
* This class represents table engine for external urls.
|
|
|
|
* It sends HTTP GET to server when select is called and
|
2023-04-21 17:54:09 +00:00
|
|
|
* HTTP POST when insert is called. In POST request the data is send
|
2018-06-11 12:13:00 +00:00
|
|
|
* using Chunked transfer encoding, so server have to support it.
|
|
|
|
*/
|
2018-08-09 18:49:05 +00:00
|
|
|
class IStorageURLBase : public IStorage
|
2018-06-11 12:13:00 +00:00
|
|
|
{
|
|
|
|
public:
|
2020-08-03 13:54:14 +00:00
|
|
|
Pipe read(
|
2019-08-24 21:20:20 +00:00
|
|
|
const Names & column_names,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2020-09-20 17:52:17 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr 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,
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t num_streams) override;
|
2018-06-11 12:13:00 +00:00
|
|
|
|
2023-06-07 18:33:08 +00:00
|
|
|
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override;
|
2018-06-11 12:13:00 +00:00
|
|
|
|
2021-10-26 09:31:01 +00:00
|
|
|
bool supportsPartitionBy() const override { return true; }
|
|
|
|
|
2023-05-01 13:40:14 +00:00
|
|
|
NamesAndTypesList getVirtuals() const override;
|
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
static ColumnsDescription getTableStructureFromData(
|
|
|
|
const String & format,
|
|
|
|
const String & uri,
|
2022-08-25 11:18:04 +00:00
|
|
|
CompressionMethod compression_method,
|
2022-12-16 22:57:09 +00:00
|
|
|
const HTTPHeaderEntries & headers,
|
2021-12-15 11:30:57 +00:00
|
|
|
const std::optional<FormatSettings> & format_settings,
|
|
|
|
ContextPtr context);
|
|
|
|
|
2022-08-05 16:20:15 +00:00
|
|
|
static SchemaCache & getSchemaCache(const ContextPtr & context);
|
|
|
|
|
2018-06-11 12:13:00 +00:00
|
|
|
protected:
|
2019-08-24 21:20:20 +00:00
|
|
|
IStorageURLBase(
|
2021-10-26 09:31:01 +00:00
|
|
|
const String & uri_,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context_,
|
2019-12-04 16:06:55 +00:00
|
|
|
const StorageID & 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_,
|
2021-04-23 12:18:23 +00:00
|
|
|
const String & comment,
|
2021-09-07 11:17:25 +00:00
|
|
|
const String & compression_method_,
|
2022-12-16 22:57:09 +00:00
|
|
|
const HTTPHeaderEntries & headers_ = {},
|
2021-10-26 12:22:13 +00:00
|
|
|
const String & method_ = "",
|
2023-04-21 17:24:37 +00:00
|
|
|
ASTPtr partition_by = nullptr,
|
|
|
|
bool distributed_processing_ = false);
|
2018-06-11 12:13:00 +00:00
|
|
|
|
2021-10-26 09:31:01 +00:00
|
|
|
String uri;
|
2022-08-25 11:18:04 +00:00
|
|
|
CompressionMethod compression_method;
|
2018-06-11 12:13:00 +00:00
|
|
|
String format_name;
|
2020-11-05 11:28:20 +00:00
|
|
|
// For URL engine, we use format settings from server context + `SETTINGS`
|
|
|
|
// clause of the `CREATE` query. In this case, format_settings is set.
|
|
|
|
// For `url` table function, we use settings from current query context.
|
|
|
|
// In this case, format_settings is not set.
|
|
|
|
std::optional<FormatSettings> format_settings;
|
2022-12-16 22:57:09 +00:00
|
|
|
HTTPHeaderEntries headers;
|
2021-10-28 12:44:12 +00:00
|
|
|
String http_method; /// For insert can choose Put instead of default Post.
|
2021-10-26 12:22:13 +00:00
|
|
|
ASTPtr partition_by;
|
2023-04-21 17:24:37 +00:00
|
|
|
bool distributed_processing;
|
2018-06-11 12:13:00 +00:00
|
|
|
|
2018-08-09 18:49:05 +00:00
|
|
|
virtual std::string getReadMethod() const;
|
|
|
|
|
2019-08-24 21:20:20 +00:00
|
|
|
virtual std::vector<std::pair<std::string, std::string>> getReadURIParams(
|
|
|
|
const Names & column_names,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2018-08-09 18:49:05 +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,
|
2019-02-19 00:48:52 +00:00
|
|
|
size_t max_block_size) const;
|
2018-08-09 18:49:05 +00:00
|
|
|
|
2019-08-24 21:20:20 +00:00
|
|
|
virtual std::function<void(std::ostream &)> getReadPOSTDataCallback(
|
|
|
|
const Names & column_names,
|
2022-02-28 13:29:05 +00:00
|
|
|
const ColumnsDescription & columns_description,
|
2018-08-09 18:49:05 +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,
|
2019-02-19 00:48:52 +00:00
|
|
|
size_t max_block_size) const;
|
2018-09-22 14:58:03 +00:00
|
|
|
|
2022-05-13 18:39:19 +00:00
|
|
|
bool supportsSubsetOfColumns() const override;
|
2022-02-28 13:29:05 +00:00
|
|
|
|
2023-04-29 02:29:51 +00:00
|
|
|
bool prefersLargeBlocks() const override;
|
|
|
|
|
2023-05-05 04:18:46 +00:00
|
|
|
bool parallelizeOutputAfterReading(ContextPtr context) const override;
|
|
|
|
|
2021-04-21 14:36:04 +00:00
|
|
|
private:
|
2021-07-09 03:15:41 +00:00
|
|
|
virtual Block getHeaderBlock(const Names & column_names, const StorageSnapshotPtr & storage_snapshot) const = 0;
|
2022-06-21 13:02:48 +00:00
|
|
|
|
2022-06-27 12:43:24 +00:00
|
|
|
static std::optional<ColumnsDescription> tryGetColumnsFromCache(
|
|
|
|
const Strings & urls,
|
2022-12-16 22:57:09 +00:00
|
|
|
const HTTPHeaderEntries & headers,
|
2022-06-28 16:13:42 +00:00
|
|
|
const Poco::Net::HTTPBasicCredentials & credentials,
|
2022-06-27 12:43:24 +00:00
|
|
|
const String & format_name,
|
|
|
|
const std::optional<FormatSettings> & format_settings,
|
|
|
|
const ContextPtr & context);
|
|
|
|
|
|
|
|
static void addColumnsToCache(
|
|
|
|
const Strings & urls,
|
|
|
|
const ColumnsDescription & columns,
|
|
|
|
const String & format_name,
|
|
|
|
const std::optional<FormatSettings> & format_settings,
|
|
|
|
const ContextPtr & context);
|
|
|
|
|
2022-06-28 16:13:42 +00:00
|
|
|
static std::optional<time_t> getLastModificationTime(
|
|
|
|
const String & url,
|
2022-12-16 22:57:09 +00:00
|
|
|
const HTTPHeaderEntries & headers,
|
2022-06-28 16:13:42 +00:00
|
|
|
const Poco::Net::HTTPBasicCredentials & credentials,
|
|
|
|
const ContextPtr & context);
|
2018-08-09 18:49:05 +00:00
|
|
|
};
|
|
|
|
|
2023-04-21 17:24:37 +00:00
|
|
|
|
|
|
|
class StorageURLSource : public ISource
|
|
|
|
{
|
|
|
|
using URIParams = std::vector<std::pair<String, String>>;
|
|
|
|
|
|
|
|
public:
|
|
|
|
class DisclosedGlobIterator
|
|
|
|
{
|
|
|
|
public:
|
2023-04-21 17:35:17 +00:00
|
|
|
DisclosedGlobIterator(const String & uri_, size_t max_addresses);
|
2023-04-21 17:24:37 +00:00
|
|
|
String next();
|
|
|
|
size_t size();
|
|
|
|
private:
|
|
|
|
class Impl;
|
|
|
|
/// shared_ptr to have copy constructor
|
|
|
|
std::shared_ptr<Impl> pimpl;
|
|
|
|
};
|
|
|
|
|
|
|
|
using FailoverOptions = std::vector<String>;
|
|
|
|
using IteratorWrapper = std::function<FailoverOptions()>;
|
|
|
|
|
|
|
|
StorageURLSource(
|
2023-05-22 19:19:57 +00:00
|
|
|
const std::vector<NameAndTypePair> & requested_virtual_columns_,
|
2023-04-21 17:24:37 +00:00
|
|
|
std::shared_ptr<IteratorWrapper> uri_iterator_,
|
|
|
|
const std::string & http_method,
|
|
|
|
std::function<void(std::ostream &)> callback,
|
|
|
|
const String & format,
|
|
|
|
const std::optional<FormatSettings> & format_settings,
|
|
|
|
String name_,
|
|
|
|
const Block & sample_block,
|
|
|
|
ContextPtr context,
|
|
|
|
const ColumnsDescription & columns,
|
|
|
|
UInt64 max_block_size,
|
|
|
|
const ConnectionTimeouts & timeouts,
|
|
|
|
CompressionMethod compression_method,
|
|
|
|
size_t download_threads,
|
|
|
|
const HTTPHeaderEntries & headers_ = {},
|
|
|
|
const URIParams & params = {},
|
|
|
|
bool glob_url = false);
|
|
|
|
|
|
|
|
String getName() const override { return name; }
|
|
|
|
|
|
|
|
Chunk generate() override;
|
|
|
|
|
|
|
|
static void setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri);
|
|
|
|
|
2023-05-22 19:19:57 +00:00
|
|
|
static Block getHeader(Block sample_block, const std::vector<NameAndTypePair> & requested_virtual_columns);
|
|
|
|
|
2023-06-13 14:43:50 +00:00
|
|
|
static std::pair<Poco::URI, std::unique_ptr<ReadWriteBufferFromHTTP>> getFirstAvailableURIAndReadBuffer(
|
2023-04-21 17:24:37 +00:00
|
|
|
std::vector<String>::const_iterator & option,
|
|
|
|
const std::vector<String>::const_iterator & end,
|
|
|
|
ContextPtr context,
|
|
|
|
const URIParams & params,
|
|
|
|
const String & http_method,
|
|
|
|
std::function<void(std::ostream &)> callback,
|
|
|
|
const ConnectionTimeouts & timeouts,
|
|
|
|
Poco::Net::HTTPBasicCredentials & credentials,
|
|
|
|
const HTTPHeaderEntries & headers,
|
|
|
|
bool glob_url,
|
|
|
|
bool delay_initialization);
|
|
|
|
|
|
|
|
private:
|
2023-05-30 19:32:24 +00:00
|
|
|
using InitializeFunc = std::function<bool()>;
|
2023-04-21 17:24:37 +00:00
|
|
|
InitializeFunc initialize;
|
|
|
|
|
|
|
|
String name;
|
2023-05-22 19:19:57 +00:00
|
|
|
std::vector<NameAndTypePair> requested_virtual_columns;
|
2023-04-21 17:24:37 +00:00
|
|
|
std::shared_ptr<IteratorWrapper> uri_iterator;
|
2023-05-22 19:19:57 +00:00
|
|
|
Poco::URI curr_uri;
|
2023-04-21 17:24:37 +00:00
|
|
|
|
2023-05-05 03:11:51 +00:00
|
|
|
std::unique_ptr<ReadBuffer> read_buf;
|
2023-06-16 15:51:18 +00:00
|
|
|
std::shared_ptr<IInputFormat> input_format;
|
2023-04-21 17:24:37 +00:00
|
|
|
std::unique_ptr<QueryPipeline> pipeline;
|
|
|
|
std::unique_ptr<PullingPipelineExecutor> reader;
|
|
|
|
|
|
|
|
Poco::Net::HTTPBasicCredentials credentials;
|
|
|
|
|
|
|
|
size_t total_size = 0;
|
|
|
|
UInt64 total_rows_approx_max = 0;
|
|
|
|
size_t total_rows_count_times = 0;
|
|
|
|
UInt64 total_rows_approx_accumulated = 0;
|
|
|
|
};
|
|
|
|
|
2021-07-23 14:25:35 +00:00
|
|
|
class StorageURLSink : public SinkToStorage
|
2020-04-28 00:56:44 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-07-23 14:25:35 +00:00
|
|
|
StorageURLSink(
|
2021-10-26 09:31:01 +00:00
|
|
|
const String & uri,
|
2020-06-17 16:39:58 +00:00
|
|
|
const String & format,
|
2020-11-07 08:53:39 +00:00
|
|
|
const std::optional<FormatSettings> & format_settings,
|
2021-07-23 14:25:35 +00:00
|
|
|
const Block & sample_block,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2020-06-17 16:39:58 +00:00
|
|
|
const ConnectionTimeouts & timeouts,
|
2021-10-26 09:31:01 +00:00
|
|
|
CompressionMethod compression_method,
|
2023-04-21 12:11:18 +00:00
|
|
|
const HTTPHeaderEntries & headers = {},
|
2021-10-26 09:31:01 +00:00
|
|
|
const String & method = Poco::Net::HTTPRequest::HTTP_POST);
|
2020-04-28 00:56:44 +00:00
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
std::string getName() const override { return "StorageURLSink"; }
|
2021-07-23 14:25:35 +00:00
|
|
|
void consume(Chunk chunk) override;
|
2022-07-21 12:18:37 +00:00
|
|
|
void onCancel() override;
|
2022-05-06 17:30:18 +00:00
|
|
|
void onException() override;
|
2021-07-23 14:25:35 +00:00
|
|
|
void onFinish() override;
|
2020-04-28 00:56:44 +00:00
|
|
|
|
|
|
|
private:
|
2022-07-21 12:18:37 +00:00
|
|
|
void finalize();
|
2020-04-28 00:56:44 +00:00
|
|
|
std::unique_ptr<WriteBuffer> write_buf;
|
2021-10-11 16:11:50 +00:00
|
|
|
OutputFormatPtr writer;
|
2022-07-21 12:18:37 +00:00
|
|
|
std::mutex cancel_mutex;
|
|
|
|
bool cancelled = false;
|
2020-04-28 00:56:44 +00:00
|
|
|
};
|
2019-08-24 21:20:20 +00:00
|
|
|
|
2022-05-03 06:43:28 +00:00
|
|
|
class StorageURL : public IStorageURLBase
|
2018-08-09 18:49:05 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-04-23 12:18:23 +00:00
|
|
|
StorageURL(
|
2021-10-26 09:31:01 +00:00
|
|
|
const String & uri_,
|
2021-04-23 12:18:23 +00:00
|
|
|
const StorageID & table_id_,
|
|
|
|
const String & format_name_,
|
|
|
|
const std::optional<FormatSettings> & format_settings_,
|
|
|
|
const ColumnsDescription & columns_,
|
|
|
|
const ConstraintsDescription & constraints_,
|
|
|
|
const String & comment,
|
|
|
|
ContextPtr context_,
|
2021-09-07 11:17:25 +00:00
|
|
|
const String & compression_method_,
|
2022-12-16 22:57:09 +00:00
|
|
|
const HTTPHeaderEntries & headers_ = {},
|
2021-10-26 12:22:13 +00:00
|
|
|
const String & method_ = "",
|
2023-04-21 17:24:37 +00:00
|
|
|
ASTPtr partition_by_ = nullptr,
|
|
|
|
bool distributed_processing_ = false);
|
2018-08-09 18:49:05 +00:00
|
|
|
|
|
|
|
String getName() const override
|
|
|
|
{
|
|
|
|
return "URL";
|
|
|
|
}
|
2018-09-22 14:58:03 +00:00
|
|
|
|
2021-07-09 03:15:41 +00:00
|
|
|
Block getHeaderBlock(const Names & /*column_names*/, const StorageSnapshotPtr & storage_snapshot) const override
|
2018-09-22 14:58:03 +00:00
|
|
|
{
|
2021-07-09 03:15:41 +00:00
|
|
|
return storage_snapshot->metadata->getSampleBlock();
|
2018-09-22 14:58:03 +00:00
|
|
|
}
|
2021-04-21 12:32:57 +00:00
|
|
|
|
|
|
|
static FormatSettings getFormatSettingsFromArgs(const StorageFactory::Arguments & args);
|
2021-09-07 11:17:25 +00:00
|
|
|
|
2023-04-21 12:11:18 +00:00
|
|
|
struct Configuration : public StatelessTableEngineConfiguration
|
2022-12-16 22:57:09 +00:00
|
|
|
{
|
|
|
|
std::string url;
|
|
|
|
std::string http_method;
|
|
|
|
HTTPHeaderEntries headers;
|
2023-04-21 12:11:18 +00:00
|
|
|
std::string addresses_expr;
|
2022-12-16 22:57:09 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
static Configuration getConfiguration(ASTs & args, ContextPtr context);
|
|
|
|
|
|
|
|
static ASTs::iterator collectHeaders(ASTs & url_function_args, HTTPHeaderEntries & header_entries, ContextPtr context);
|
2022-06-17 12:53:16 +00:00
|
|
|
|
2022-12-16 22:57:09 +00:00
|
|
|
static void processNamedCollectionResult(Configuration & configuration, const NamedCollection & collection);
|
2018-06-11 12:13:00 +00:00
|
|
|
};
|
2021-04-21 14:36:04 +00:00
|
|
|
|
2023-04-21 12:11:18 +00:00
|
|
|
|
2021-04-21 14:36:04 +00:00
|
|
|
/// StorageURLWithFailover is allowed only for URL table function, not as a separate storage.
|
|
|
|
class StorageURLWithFailover final : public StorageURL
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
StorageURLWithFailover(
|
2023-04-21 12:11:18 +00:00
|
|
|
const std::vector<String> & uri_options_,
|
|
|
|
const StorageID & table_id_,
|
|
|
|
const String & format_name_,
|
|
|
|
const std::optional<FormatSettings> & format_settings_,
|
|
|
|
const ColumnsDescription & columns_,
|
|
|
|
const ConstraintsDescription & constraints_,
|
|
|
|
ContextPtr context_,
|
|
|
|
const String & compression_method_);
|
2021-04-21 14:36:04 +00:00
|
|
|
|
|
|
|
Pipe read(
|
|
|
|
const Names & column_names,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2021-04-21 14:36:04 +00:00
|
|
|
SelectQueryInfo & query_info,
|
|
|
|
ContextPtr context,
|
|
|
|
QueryProcessingStage::Enum processed_stage,
|
|
|
|
size_t max_block_size,
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t num_streams) override;
|
2021-04-21 14:36:04 +00:00
|
|
|
|
|
|
|
private:
|
2021-10-26 09:31:01 +00:00
|
|
|
std::vector<String> uri_options;
|
2018-06-11 12:13:00 +00:00
|
|
|
};
|
2023-04-21 17:24:37 +00:00
|
|
|
}
|