mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 11:02:08 +00:00
Merge pull request #49356 from Ziy1-Tan/vcol
Support for `_path` and `_file` virtual columns for table function `url`.
This commit is contained in:
commit
d4b89cb643
@ -46,3 +46,12 @@ SELECT * FROM test_table;
|
|||||||
|
|
||||||
Patterns in curly brackets `{ }` are used to generate a set of shards or to specify failover addresses. Supported pattern types and examples see in the description of the [remote](remote.md#globs-in-addresses) function.
|
Patterns in curly brackets `{ }` are used to generate a set of shards or to specify failover addresses. Supported pattern types and examples see in the description of the [remote](remote.md#globs-in-addresses) function.
|
||||||
Character `|` inside patterns is used to specify failover addresses. They are iterated in the same order as listed in the pattern. The number of generated addresses is limited by [glob_expansion_max_elements](../../operations/settings/settings.md#glob_expansion_max_elements) setting.
|
Character `|` inside patterns is used to specify failover addresses. They are iterated in the same order as listed in the pattern. The number of generated addresses is limited by [glob_expansion_max_elements](../../operations/settings/settings.md#glob_expansion_max_elements) setting.
|
||||||
|
|
||||||
|
## Virtual Columns
|
||||||
|
|
||||||
|
- `_path` — Path to the `URL`.
|
||||||
|
- `_file` — Resource name of the `URL`.
|
||||||
|
|
||||||
|
**See Also**
|
||||||
|
|
||||||
|
- [Virtual columns](/docs/en/engines/table-engines/index.md#table_engines-virtual_columns)
|
||||||
|
@ -46,3 +46,12 @@ SELECT * FROM test_table;
|
|||||||
|
|
||||||
Шаблоны в фигурных скобках `{ }` используются, чтобы сгенерировать список шардов или указать альтернативные адреса на случай отказа. Поддерживаемые типы шаблонов и примеры смотрите в описании функции [remote](remote.md#globs-in-addresses).
|
Шаблоны в фигурных скобках `{ }` используются, чтобы сгенерировать список шардов или указать альтернативные адреса на случай отказа. Поддерживаемые типы шаблонов и примеры смотрите в описании функции [remote](remote.md#globs-in-addresses).
|
||||||
Символ `|` внутри шаблонов используется, чтобы задать адреса, если предыдущие оказались недоступны. Эти адреса перебираются в том же порядке, в котором они указаны в шаблоне. Количество адресов, которые могут быть сгенерированы, ограничено настройкой [glob_expansion_max_elements](../../operations/settings/settings.md#glob_expansion_max_elements).
|
Символ `|` внутри шаблонов используется, чтобы задать адреса, если предыдущие оказались недоступны. Эти адреса перебираются в том же порядке, в котором они указаны в шаблоне. Количество адресов, которые могут быть сгенерированы, ограничено настройкой [glob_expansion_max_elements](../../operations/settings/settings.md#glob_expansion_max_elements).
|
||||||
|
|
||||||
|
## Виртуальные столбцы
|
||||||
|
|
||||||
|
- `_path` — Путь до `URL`.
|
||||||
|
- `_file` — Имя ресурса `URL`.
|
||||||
|
|
||||||
|
**Смотрите также**
|
||||||
|
|
||||||
|
- [Виртуальные столбцы](index.md#table_engines-virtual_columns)
|
||||||
|
@ -41,3 +41,11 @@ CREATE TABLE test_table (column1 String, column2 UInt32) ENGINE=Memory;
|
|||||||
INSERT INTO FUNCTION url('http://127.0.0.1:8123/?query=INSERT+INTO+test_table+FORMAT+CSV', 'CSV', 'column1 String, column2 UInt32') VALUES ('http interface', 42);
|
INSERT INTO FUNCTION url('http://127.0.0.1:8123/?query=INSERT+INTO+test_table+FORMAT+CSV', 'CSV', 'column1 String, column2 UInt32') VALUES ('http interface', 42);
|
||||||
SELECT * FROM test_table;
|
SELECT * FROM test_table;
|
||||||
```
|
```
|
||||||
|
## 虚拟列 {#virtual-columns}
|
||||||
|
|
||||||
|
- `_path` — `URL`路径。
|
||||||
|
- `_file` — 资源名称。
|
||||||
|
|
||||||
|
**另请参阅**
|
||||||
|
|
||||||
|
- [虚拟列](https://clickhouse.com/docs/en/operations/table_engines/#table_engines-virtual_columns)
|
||||||
|
@ -64,7 +64,8 @@ static bool parseNumber(const String & description, size_t l, size_t r, size_t &
|
|||||||
* abc{1..9}de{f,g,h} - is a direct product, 27 shards.
|
* abc{1..9}de{f,g,h} - is a direct product, 27 shards.
|
||||||
* abc{1..9}de{0|1} - is a direct product, 9 shards, in each 2 replicas.
|
* abc{1..9}de{0|1} - is a direct product, 9 shards, in each 2 replicas.
|
||||||
*/
|
*/
|
||||||
std::vector<String> parseRemoteDescription(const String & description, size_t l, size_t r, char separator, size_t max_addresses)
|
std::vector<String>
|
||||||
|
parseRemoteDescription(const String & description, size_t l, size_t r, char separator, size_t max_addresses, const String & func_name)
|
||||||
{
|
{
|
||||||
std::vector<String> res;
|
std::vector<String> res;
|
||||||
std::vector<String> cur;
|
std::vector<String> cur;
|
||||||
@ -97,28 +98,41 @@ std::vector<String> parseRemoteDescription(const String & description, size_t l,
|
|||||||
if (cnt == 0) break;
|
if (cnt == 0) break;
|
||||||
}
|
}
|
||||||
if (cnt != 0)
|
if (cnt != 0)
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'remote': incorrect brace sequence in first argument");
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}': incorrect brace sequence in first argument", func_name);
|
||||||
/// The presence of a dot - numeric interval
|
/// The presence of a dot - numeric interval
|
||||||
if (last_dot != -1)
|
if (last_dot != -1)
|
||||||
{
|
{
|
||||||
size_t left, right;
|
size_t left, right;
|
||||||
if (description[last_dot - 1] != '.')
|
if (description[last_dot - 1] != '.')
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'remote': incorrect argument in braces (only one dot): {}",
|
throw Exception(
|
||||||
description.substr(i, m - i + 1));
|
ErrorCodes::BAD_ARGUMENTS,
|
||||||
|
"Table function '{}': incorrect argument in braces (only one dot): {}",
|
||||||
|
func_name,
|
||||||
|
description.substr(i, m - i + 1));
|
||||||
if (!parseNumber(description, i + 1, last_dot - 1, left))
|
if (!parseNumber(description, i + 1, last_dot - 1, left))
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'remote': "
|
throw Exception(
|
||||||
"incorrect argument in braces (Incorrect left number): {}",
|
ErrorCodes::BAD_ARGUMENTS,
|
||||||
description.substr(i, m - i + 1));
|
"Table function '{}': "
|
||||||
|
"incorrect argument in braces (Incorrect left number): {}",
|
||||||
|
func_name,
|
||||||
|
description.substr(i, m - i + 1));
|
||||||
if (!parseNumber(description, last_dot + 1, m, right))
|
if (!parseNumber(description, last_dot + 1, m, right))
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'remote': "
|
throw Exception(
|
||||||
"incorrect argument in braces (Incorrect right number): {}",
|
ErrorCodes::BAD_ARGUMENTS,
|
||||||
description.substr(i, m - i + 1));
|
"Table function '{}': "
|
||||||
|
"incorrect argument in braces (Incorrect right number): {}",
|
||||||
|
func_name,
|
||||||
|
description.substr(i, m - i + 1));
|
||||||
if (left > right)
|
if (left > right)
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'remote': "
|
throw Exception(
|
||||||
"incorrect argument in braces (left number is greater then right): {}",
|
ErrorCodes::BAD_ARGUMENTS,
|
||||||
description.substr(i, m - i + 1));
|
"Table function '{}': "
|
||||||
|
"incorrect argument in braces (left number is greater then right): {}",
|
||||||
|
func_name,
|
||||||
|
description.substr(i, m - i + 1));
|
||||||
if (right - left + 1 > max_addresses)
|
if (right - left + 1 > max_addresses)
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'remote': first argument generates too many result addresses");
|
throw Exception(
|
||||||
|
ErrorCodes::BAD_ARGUMENTS, "Table function '{}': first argument generates too many result addresses", func_name);
|
||||||
bool add_leading_zeroes = false;
|
bool add_leading_zeroes = false;
|
||||||
size_t len = last_dot - 1 - (i + 1);
|
size_t len = last_dot - 1 - (i + 1);
|
||||||
/// If the left and right borders have equal numbers, then you must add leading zeros.
|
/// If the left and right borders have equal numbers, then you must add leading zeros.
|
||||||
@ -161,7 +175,7 @@ std::vector<String> parseRemoteDescription(const String & description, size_t l,
|
|||||||
|
|
||||||
res.insert(res.end(), cur.begin(), cur.end());
|
res.insert(res.end(), cur.begin(), cur.end());
|
||||||
if (res.size() > max_addresses)
|
if (res.size() > max_addresses)
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'remote': first argument generates too many result addresses");
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}': first argument generates too many result addresses", func_name);
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
@ -15,7 +15,8 @@ namespace DB
|
|||||||
* abc{1..9}de{f,g,h} - is a direct product, 27 shards.
|
* abc{1..9}de{f,g,h} - is a direct product, 27 shards.
|
||||||
* abc{1..9}de{0|1} - is a direct product, 9 shards, in each 2 replicas.
|
* abc{1..9}de{0|1} - is a direct product, 9 shards, in each 2 replicas.
|
||||||
*/
|
*/
|
||||||
std::vector<String> parseRemoteDescription(const String & description, size_t l, size_t r, char separator, size_t max_addresses);
|
std::vector<String> parseRemoteDescription(
|
||||||
|
const String & description, size_t l, size_t r, char separator, size_t max_addresses, const String & func_name = "remote");
|
||||||
|
|
||||||
/// Parse remote description for external database (MySQL or PostgreSQL).
|
/// Parse remote description for external database (MySQL or PostgreSQL).
|
||||||
std::vector<std::pair<String, uint16_t>> parseRemoteDescriptionForExternalDatabase(const String & description, size_t max_addresses, UInt16 default_port);
|
std::vector<std::pair<String, uint16_t>> parseRemoteDescriptionForExternalDatabase(const String & description, size_t max_addresses, UInt16 default_port);
|
||||||
|
@ -38,6 +38,7 @@
|
|||||||
#include <Common/logger_useful.h>
|
#include <Common/logger_useful.h>
|
||||||
#include <Poco/Net/HTTPRequest.h>
|
#include <Poco/Net/HTTPRequest.h>
|
||||||
#include <regex>
|
#include <regex>
|
||||||
|
#include <DataTypes/DataTypeString.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -159,6 +160,9 @@ namespace
|
|||||||
using FailoverOptions = std::vector<String>;
|
using FailoverOptions = std::vector<String>;
|
||||||
std::vector<FailoverOptions> uri_list_to_read;
|
std::vector<FailoverOptions> uri_list_to_read;
|
||||||
std::atomic<size_t> next_uri_to_read = 0;
|
std::atomic<size_t> next_uri_to_read = 0;
|
||||||
|
|
||||||
|
bool need_path_column = false;
|
||||||
|
bool need_file_column = false;
|
||||||
};
|
};
|
||||||
using URIInfoPtr = std::shared_ptr<URIInfo>;
|
using URIInfoPtr = std::shared_ptr<URIInfo>;
|
||||||
|
|
||||||
@ -176,6 +180,27 @@ namespace
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static Block getBlockForSource(const Block & block_for_format, const URIInfoPtr & uri_info)
|
||||||
|
{
|
||||||
|
auto res = block_for_format;
|
||||||
|
if (uri_info->need_path_column)
|
||||||
|
{
|
||||||
|
res.insert(
|
||||||
|
{DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn(),
|
||||||
|
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
|
||||||
|
"_path"});
|
||||||
|
}
|
||||||
|
|
||||||
|
if (uri_info->need_file_column)
|
||||||
|
{
|
||||||
|
res.insert(
|
||||||
|
{DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn(),
|
||||||
|
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
|
||||||
|
"_file"});
|
||||||
|
}
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
StorageURLSource(
|
StorageURLSource(
|
||||||
URIInfoPtr uri_info_,
|
URIInfoPtr uri_info_,
|
||||||
const std::string & http_method,
|
const std::string & http_method,
|
||||||
@ -193,7 +218,7 @@ namespace
|
|||||||
const HTTPHeaderEntries & headers_ = {},
|
const HTTPHeaderEntries & headers_ = {},
|
||||||
const URIParams & params = {},
|
const URIParams & params = {},
|
||||||
bool glob_url = false)
|
bool glob_url = false)
|
||||||
: ISource(sample_block), name(std::move(name_)), uri_info(uri_info_)
|
: ISource(getBlockForSource(sample_block, uri_info_)), name(std::move(name_)), uri_info(uri_info_)
|
||||||
{
|
{
|
||||||
auto headers = getHeaders(headers_);
|
auto headers = getHeaders(headers_);
|
||||||
|
|
||||||
@ -204,7 +229,7 @@ namespace
|
|||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty url list");
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty url list");
|
||||||
|
|
||||||
auto first_option = uri_options.begin();
|
auto first_option = uri_options.begin();
|
||||||
auto buf_factory = getFirstAvailableURLReadBuffer(
|
auto [actual_uri, buf_factory] = getFirstAvailableURIAndReadBuffer(
|
||||||
first_option,
|
first_option,
|
||||||
uri_options.end(),
|
uri_options.end(),
|
||||||
context,
|
context,
|
||||||
@ -217,6 +242,8 @@ namespace
|
|||||||
glob_url,
|
glob_url,
|
||||||
uri_options.size() == 1);
|
uri_options.size() == 1);
|
||||||
|
|
||||||
|
curr_uri = actual_uri;
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
total_size += buf_factory->getFileSize();
|
total_size += buf_factory->getFileSize();
|
||||||
@ -269,15 +296,32 @@ namespace
|
|||||||
if (current_uri_pos >= uri_info->uri_list_to_read.size())
|
if (current_uri_pos >= uri_info->uri_list_to_read.size())
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
auto current_uri = uri_info->uri_list_to_read[current_uri_pos];
|
auto current_uri_options = uri_info->uri_list_to_read[current_uri_pos];
|
||||||
|
|
||||||
initialize(current_uri);
|
initialize(current_uri_options);
|
||||||
}
|
}
|
||||||
|
|
||||||
Chunk chunk;
|
Chunk chunk;
|
||||||
if (reader->pull(chunk))
|
if (reader->pull(chunk))
|
||||||
{
|
{
|
||||||
UInt64 num_rows = chunk.getNumRows();
|
UInt64 num_rows = chunk.getNumRows();
|
||||||
|
|
||||||
|
const String & path{curr_uri.getPath()};
|
||||||
|
if (uri_info->need_path_column)
|
||||||
|
{
|
||||||
|
auto column = DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumnConst(num_rows, path);
|
||||||
|
chunk.addColumn(column->convertToFullColumnIfConst());
|
||||||
|
}
|
||||||
|
|
||||||
|
if (uri_info->need_file_column)
|
||||||
|
{
|
||||||
|
const size_t last_slash_pos = path.find_last_of('/');
|
||||||
|
auto file_name = path.substr(last_slash_pos + 1);
|
||||||
|
auto column
|
||||||
|
= DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumnConst(num_rows, std::move(file_name));
|
||||||
|
chunk.addColumn(column->convertToFullColumnIfConst());
|
||||||
|
}
|
||||||
|
|
||||||
if (num_rows && total_size)
|
if (num_rows && total_size)
|
||||||
updateRowsProgressApprox(
|
updateRowsProgressApprox(
|
||||||
*this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max);
|
*this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max);
|
||||||
@ -291,7 +335,7 @@ namespace
|
|||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
|
|
||||||
static SeekableReadBufferFactoryPtr getFirstAvailableURLReadBuffer(
|
static std::tuple<Poco::URI, SeekableReadBufferFactoryPtr> getFirstAvailableURIAndReadBuffer(
|
||||||
std::vector<String>::const_iterator & option,
|
std::vector<String>::const_iterator & option,
|
||||||
const std::vector<String>::const_iterator & end,
|
const std::vector<String>::const_iterator & end,
|
||||||
ContextPtr context,
|
ContextPtr context,
|
||||||
@ -352,7 +396,7 @@ namespace
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return res;
|
return std::make_tuple(request_uri, std::move(res));
|
||||||
}
|
}
|
||||||
|
|
||||||
throw Exception(ErrorCodes::NETWORK_ERROR, "All uri ({}) options are unreachable: {}", options, first_exception_message);
|
throw Exception(ErrorCodes::NETWORK_ERROR, "All uri ({}) options are unreachable: {}", options, first_exception_message);
|
||||||
@ -364,6 +408,7 @@ namespace
|
|||||||
|
|
||||||
String name;
|
String name;
|
||||||
URIInfoPtr uri_info;
|
URIInfoPtr uri_info;
|
||||||
|
Poco::URI curr_uri;
|
||||||
|
|
||||||
std::unique_ptr<QueryPipeline> pipeline;
|
std::unique_ptr<QueryPipeline> pipeline;
|
||||||
std::unique_ptr<PullingPipelineExecutor> reader;
|
std::unique_ptr<PullingPipelineExecutor> reader;
|
||||||
@ -538,10 +583,10 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData(
|
|||||||
if (urlWithGlobs(uri))
|
if (urlWithGlobs(uri))
|
||||||
{
|
{
|
||||||
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
|
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
|
||||||
auto uri_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses);
|
auto uri_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses, "url");
|
||||||
for (const auto & description : uri_descriptions)
|
for (const auto & description : uri_descriptions)
|
||||||
{
|
{
|
||||||
auto options = parseRemoteDescription(description, 0, description.size(), '|', max_addresses);
|
auto options = parseRemoteDescription(description, 0, description.size(), '|', max_addresses, "url");
|
||||||
urls_to_check.insert(urls_to_check.end(), options.begin(), options.end());
|
urls_to_check.insert(urls_to_check.end(), options.begin(), options.end());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -559,7 +604,7 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData(
|
|||||||
if (it == urls_to_check.cend())
|
if (it == urls_to_check.cend())
|
||||||
return nullptr;
|
return nullptr;
|
||||||
|
|
||||||
auto buf_factory = StorageURLSource::getFirstAvailableURLReadBuffer(
|
auto [_, buf_factory] = StorageURLSource::getFirstAvailableURIAndReadBuffer(
|
||||||
it,
|
it,
|
||||||
urls_to_check.cend(),
|
urls_to_check.cend(),
|
||||||
context,
|
context,
|
||||||
@ -631,18 +676,27 @@ Pipe IStorageURLBase::read(
|
|||||||
|
|
||||||
size_t max_download_threads = local_context->getSettingsRef().max_download_threads;
|
size_t max_download_threads = local_context->getSettingsRef().max_download_threads;
|
||||||
|
|
||||||
|
auto uri_info = std::make_shared<StorageURLSource::URIInfo>();
|
||||||
|
for (const auto & column : column_names)
|
||||||
|
{
|
||||||
|
if (column == "_path")
|
||||||
|
uri_info->need_path_column = true;
|
||||||
|
if (column == "_file")
|
||||||
|
uri_info->need_file_column = true;
|
||||||
|
}
|
||||||
|
|
||||||
if (urlWithGlobs(uri))
|
if (urlWithGlobs(uri))
|
||||||
{
|
{
|
||||||
size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements;
|
size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements;
|
||||||
auto uri_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses);
|
auto uri_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses, "url");
|
||||||
|
|
||||||
if (num_streams > uri_descriptions.size())
|
if (num_streams > uri_descriptions.size())
|
||||||
num_streams = uri_descriptions.size();
|
num_streams = uri_descriptions.size();
|
||||||
|
|
||||||
/// For each uri (which acts like shard) check if it has failover options
|
/// For each uri (which acts like shard) check if it has failover options
|
||||||
auto uri_info = std::make_shared<StorageURLSource::URIInfo>();
|
uri_info->uri_list_to_read.reserve(uri_descriptions.size());
|
||||||
for (const auto & description : uri_descriptions)
|
for (const auto & description : uri_descriptions)
|
||||||
uri_info->uri_list_to_read.emplace_back(parseRemoteDescription(description, 0, description.size(), '|', max_addresses));
|
uri_info->uri_list_to_read.emplace_back(parseRemoteDescription(description, 0, description.size(), '|', max_addresses, "url"));
|
||||||
|
|
||||||
Pipes pipes;
|
Pipes pipes;
|
||||||
pipes.reserve(num_streams);
|
pipes.reserve(num_streams);
|
||||||
@ -672,7 +726,6 @@ Pipe IStorageURLBase::read(
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
auto uri_info = std::make_shared<StorageURLSource::URIInfo>();
|
|
||||||
uri_info->uri_list_to_read.emplace_back(std::vector<String>{uri});
|
uri_info->uri_list_to_read.emplace_back(std::vector<String>{uri});
|
||||||
return Pipe(std::make_shared<StorageURLSource>(
|
return Pipe(std::make_shared<StorageURLSource>(
|
||||||
uri_info,
|
uri_info,
|
||||||
@ -781,6 +834,13 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
NamesAndTypesList IStorageURLBase::getVirtuals() const
|
||||||
|
{
|
||||||
|
return NamesAndTypesList{
|
||||||
|
{"_path", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
|
||||||
|
{"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())}};
|
||||||
|
}
|
||||||
|
|
||||||
SchemaCache & IStorageURLBase::getSchemaCache(const ContextPtr & context)
|
SchemaCache & IStorageURLBase::getSchemaCache(const ContextPtr & context)
|
||||||
{
|
{
|
||||||
static SchemaCache schema_cache(context->getConfigRef().getUInt("schema_inference_cache_max_elements_for_url", DEFAULT_SCHEMA_CACHE_ELEMENTS));
|
static SchemaCache schema_cache(context->getConfigRef().getUInt("schema_inference_cache_max_elements_for_url", DEFAULT_SCHEMA_CACHE_ELEMENTS));
|
||||||
|
@ -43,6 +43,8 @@ public:
|
|||||||
|
|
||||||
bool supportsPartitionBy() const override { return true; }
|
bool supportsPartitionBy() const override { return true; }
|
||||||
|
|
||||||
|
NamesAndTypesList getVirtuals() const override;
|
||||||
|
|
||||||
static ColumnsDescription getTableStructureFromData(
|
static ColumnsDescription getTableStructureFromData(
|
||||||
const String & format,
|
const String & format,
|
||||||
const String & uri,
|
const String & uri,
|
||||||
|
@ -7,6 +7,7 @@ import tempfile
|
|||||||
import threading
|
import threading
|
||||||
import os
|
import os
|
||||||
import traceback
|
import traceback
|
||||||
|
from urllib.parse import urljoin
|
||||||
import urllib.request
|
import urllib.request
|
||||||
import subprocess
|
import subprocess
|
||||||
from io import StringIO
|
from io import StringIO
|
||||||
@ -163,6 +164,7 @@ def test_select(
|
|||||||
requests=[],
|
requests=[],
|
||||||
answers=[],
|
answers=[],
|
||||||
test_data="",
|
test_data="",
|
||||||
|
res_path="",
|
||||||
):
|
):
|
||||||
with open(CSV_DATA, "w") as f: # clear file
|
with open(CSV_DATA, "w") as f: # clear file
|
||||||
f.write("")
|
f.write("")
|
||||||
@ -183,7 +185,7 @@ def test_select(
|
|||||||
tbl = table_name
|
tbl = table_name
|
||||||
if not tbl:
|
if not tbl:
|
||||||
tbl = "url('{addr}', 'CSV', '{schema}')".format(
|
tbl = "url('{addr}', 'CSV', '{schema}')".format(
|
||||||
addr=HTTP_SERVER_URL_STR, schema=schema
|
addr=urljoin(HTTP_SERVER_URL_STR, res_path), schema=schema
|
||||||
)
|
)
|
||||||
check_answers(requests[i].format(tbl=tbl), answers[i])
|
check_answers(requests[i].format(tbl=tbl), answers[i])
|
||||||
|
|
||||||
@ -252,6 +254,20 @@ def main():
|
|||||||
"select double, count(*) from {tbl} group by double order by double": "7.7\t2\n9.9\t10",
|
"select double, count(*) from {tbl} group by double order by double": "7.7\t2\n9.9\t10",
|
||||||
}
|
}
|
||||||
|
|
||||||
|
pathname = CSV_DATA
|
||||||
|
filename = os.path.basename(CSV_DATA)
|
||||||
|
select_virtual_requests = {
|
||||||
|
"select _path from {tbl}": "\n".join(pathname for _ in range(2)),
|
||||||
|
"select _file from {tbl}": "\n".join(filename for _ in range(2)),
|
||||||
|
"select _file, from {tbl} order by _path": "\n".join(
|
||||||
|
filename for _ in range(2)
|
||||||
|
),
|
||||||
|
"select _path, _file from {tbl}": "\n".join(
|
||||||
|
f"{pathname}\t{filename}" for _ in range(2)
|
||||||
|
),
|
||||||
|
"select _path, count(*) from {tbl} group by _path": f"{pathname}\t2",
|
||||||
|
}
|
||||||
|
|
||||||
t, httpd = start_server()
|
t, httpd = start_server()
|
||||||
t.start()
|
t.start()
|
||||||
# test table with url engine
|
# test table with url engine
|
||||||
@ -267,6 +283,14 @@ def main():
|
|||||||
answers=list(select_only_requests.values()),
|
answers=list(select_only_requests.values()),
|
||||||
test_data=test_data,
|
test_data=test_data,
|
||||||
)
|
)
|
||||||
|
# test table function url for virtual column
|
||||||
|
test_select(
|
||||||
|
requests=list(select_virtual_requests.keys()),
|
||||||
|
answers=list(select_virtual_requests.values()),
|
||||||
|
test_data=test_data,
|
||||||
|
res_path=CSV_DATA,
|
||||||
|
)
|
||||||
|
|
||||||
# test insert into table with url engine
|
# test insert into table with url engine
|
||||||
test_insert(
|
test_insert(
|
||||||
table_name="test_table_insert",
|
table_name="test_table_insert",
|
||||||
|
@ -0,0 +1,4 @@
|
|||||||
|
/
|
||||||
|
|
||||||
|
1
|
||||||
|
/ 1
|
@ -0,0 +1,8 @@
|
|||||||
|
-- Tags: no-parallel
|
||||||
|
|
||||||
|
select _path from url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String');
|
||||||
|
select _file from url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String');
|
||||||
|
select _file, count() from url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String') group by _file;
|
||||||
|
select _path, _file, s from url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String');
|
||||||
|
select _path, _file, s from url('http://127.0.0.1:8123/?query=select+1&user=default&password=wrong', LineAsString, 's String'); -- { serverError RECEIVED_ERROR_FROM_REMOTE_IO_SERVER }
|
||||||
|
|
Loading…
Reference in New Issue
Block a user