merged with master

This commit is contained in:
Nikolai Kochetov 2018-04-19 18:10:44 +03:00
commit dd10c2073c
121 changed files with 1381 additions and 344 deletions

View File

@ -1,3 +1,56 @@
# ClickHouse release 1.1.54378, 2018-04-16
## New features:
* Logging level can be changed without restarting the server.
* Added the `SHOW CREATE DATABASE` query.
* The `query_id` can be passed to `clickhouse-client` (elBroom).
* New setting: `max_network_bandwidth_for_all_users`.
* Added support for `ALTER TABLE ... PARTITION ... ` for `MATERIALIZED VIEW`.
* Added information about the size of data parts in uncompressed form in the system table.
* Server-to-server encryption support for distributed tables (`<secure>1</secure>` in the replica config in `<remote_servers>`).
* Configuration of the table level for the `ReplicatedMergeTree` family in order to minimize the amount of data stored in zookeeper: `use_minimalistic_checksums_in_zookeeper = 1`
* Configuration of the `clickhouse-client` prompt. By default, server names are now output to the prompt. The server's display name can be changed; it's also sent in the `X-ClickHouse-Display-Name` HTTP header (Kirill Shvakov).
* Multiple comma-separated `topics` can be specified for the `Kafka` engine (Tobias Adamson).
* When a query is stopped by `KILL QUERY` or `replace_running_query`, the client receives the `Query was cancelled` exception instead of an incomplete response.
## Improvements:
* `ALTER TABLE ... DROP/DETACH PARTITION` queries are run in the front of replication queue.
* `SELECT ... FINAL` and `OPTIMIZE ... FINAL` can be used even when the table has a single data part.
* A `query_log` table is recreated on the fly if it was deleted manually (Kirill Shvakov).
* The `lengthUTF8` function runs faster (zhang2014).
* Improved performance of synchronous inserts in `Distributed` tables (`insert_distributed_sync = 1`) when there is a very large number of shards.
* The server accepts the `send_timeout` and `receive_timeout` settings from the client and applies them when connecting to the client (they are applied in reverse order: the server socket's `send_timeout` is set to the `receive_timeout` value received from the client, and vice versa).
* More robust crash recovery for asynchronous insertion into `Distributed` tables.
* The return type of the `countEqual` function changed from `UInt32` to `UInt64` (谢磊).
## Bug fixes:
* Fixed an error with `IN` when the left side of the expression is `Nullable`.
* Correct results are now returned when using tuples with `IN` when some of the tuple components are in the table index.
* The `max_execution_time` limit now works correctly with distributed queries.
* Fixed errors when calculating the size of composite columns in the `system.columns` table.
* Fixed an error when creating a temporary table `CREATE TEMPORARY TABLE IF NOT EXISTS`.
* Fixed errors in `StorageKafka` (#2075)
* Fixed server crashes from invalid arguments of certain aggregate functions.
* Fixed the error that prevented the `DETACH DATABASE` query from stopping background tasks for `ReplicatedMergeTree` tables.
* `Too many parts` state is less likely to happen when inserting into aggregated materialized views (#2084).
* Corrected recursive handling of substitutions in the config if a substitution must be followed by another substitution on the same level.
* Corrected the syntax in the metadata file when creating a `VIEW` that uses a query with `UNION ALL`.
* `SummingMergeTree` now works correctly for summation of nested data structures with a composite key.
* Fixed the possibility of a race condition when choosing the leader for `ReplicatedMergeTree` tables.
## Build changes:
* The build supports `ninja` instead of `make` and uses it by default for building releases.
* Renamed packages: `clickhouse-server-base` is now `clickhouse-common-static`; `clickhouse-server-common` is now `clickhouse-server`; `clickhouse-common-dbg` is now `clickhouse-common-static-dbg`. To install, use only `clickhouse-server clickhouse-client`. Packages with the old names will still load in the repositories for backward compatibility.
## Backward-incompatible changes:
* Removed the special interpretation of an IN expression if an array is specified on the left side. Previously, the expression `arr IN (set)` was interpreted as "at least one `arr` element belongs to the `set`". To get the same behavior in the new version, write `arrayExists(x -> x IN (set), arr)`.
* Disabled the incorrect use of the socket option `SO_REUSEPORT`, which was incorrectly enabled by default in the Poco library. Note that on Linux there is no longer any reason to simultaneously specify the addresses `::` and `0.0.0.0` for listen use just `::`, which allows listening to the connection both over IPv4 and IPv6 (with the default kernel config settings). You can also revert to the behavior from previous versions by specifying `<listen_reuse_port>1</listen_reuse_port>` in the config.
# ClickHouse release 1.1.54370, 2018-03-16
## New features:

View File

@ -1,3 +1,57 @@
# ClickHouse release 1.1.54378, 2018-04-16
## Новые возможности:
* Возможность изменения уровня логгирования без перезагрузки сервера.
* Добавлен запрос `SHOW CREATE DATABASE`.
* Возможность передать `query_id` в `clickhouse-client` (elBroom).
* Добавлена настройка `max_network_bandwidth_for_all_users`.
* Добавлена поддержка `ALTER TABLE ... PARTITION ... ` для `MATERIALIZED VIEW`.
* Добавлена информация о размере кусков данных в несжатом виде в системные таблицы.
* Поддержка межсерверного шифрования для distributed таблиц (`<secure>1</secure>` в конфигурации реплики в `<remote_servers>`).
* Добавлена настройка уровня таблицы семейства `ReplicatedMergeTree` для уменьшения объема данных, хранимых в zookeeper: `use_minimalistic_checksums_in_zookeeper = 1`
* Возможность настройки приглашения `clickhouse-client`. По-умолчанию добавлен вывод имени сервера в приглашение. Возможность изменить отображаемое имя сервера. Отправка его в HTTP заголовке `X-ClickHouse-Display-Name` (Kirill Shvakov).
* Возможность указания нескольких `topics` через запятую для движка `Kafka` (Tobias Adamson)
* При остановке запроса по причине `KILL QUERY` или `replace_running_query`, клиент получает исключение `Query was cancelled` вместо неполного результата.
## Улучшения:
* Запросы вида `ALTER TABLE ... DROP/DETACH PARTITION` выполняются впереди очереди репликации.
* Возможность использовать `SELECT ... FINAL` и `OPTIMIZE ... FINAL` даже в случае, если данные в таблице представлены одним куском.
* Пересоздание таблицы `query_log` налету в случае если было произведено её удаление вручную (Kirill Shvakov).
* Ускорение функции `lengthUTF8` (zhang2014).
* Улучшена производительность синхронной вставки в `Distributed` таблицы (`insert_distributed_sync = 1`) в случае очень большого количества шардов.
* Сервер принимает настройки `send_timeout` и `receive_timeout` от клиента и применяет их на своей стороне для соединения с клиентом (в переставленном порядке: `send_timeout` у сокета на стороне сервера выставляется в значение `receive_timeout` принятое от клиента, и наоборот).
* Более надёжное восстановление после сбоев при асинхронной вставке в `Distributed` таблицы.
* Возвращаемый тип функции `countEqual` изменён с `UInt32` на `UInt64` (谢磊)
## Исправление ошибок:
* Исправлена ошибка c `IN` где левая часть выражения `Nullable`.
* Исправлен неправильный результат при использовании кортежей с `IN` в случае, если часть компоненнтов кортежа есть в индексе таблицы.
* Исправлена работа ограничения `max_execution_time` с распределенными запросами.
* Исправлены ошибки при вычислении размеров составных столбцов в таблице `system.columns`.
* Исправлена ошибка при создании временной таблицы `CREATE TEMPORARY TABLE IF NOT EXISTS`
* Исправлены ошибки в `StorageKafka` (#2075)
* Исправлены падения сервера от некорректных аргументов некоторых аггрегатных функций.
* Исправлена ошибка, из-за которой запрос `DETACH DATABASE` мог не приводить к остановке фоновых задач таблицы типа `ReplicatedMergeTree`.
* Исправлена проблема с появлением `Too many parts` в агрегирующих материализованных представлениях (#2084).
* Исправлена рекурсивная обработка подстановок в конфиге, если после одной подстановки, требуется другая подстановка на том же уровне.
* Исправлена ошибка с неправильным синтаксисом в файле с метаданными при создании `VIEW`, использующих запрос с `UNION ALL`.
* Исправлена работа `SummingMergeTree` в случае суммирования вложенных структур данных с составным ключом.
* Исправлена возможность возникновения race condition при выборе лидера таблиц `ReplicatedMergeTree`.
## Изменения сборки:
* Поддержка `ninja` вместо `make` при сборке. `ninja` используется по-умолчанию при сборке релизов.
* Переименованы пакеты `clickhouse-server-base` в `clickhouse-common-static`; `clickhouse-server-common` в `clickhouse-server`; `clickhouse-common-dbg` в `clickhouse-common-static-dbg`. Для установки используйте только `clickhouse-server clickhouse-client`. Для совместимости, пакеты со старыми именами продолжают загружаться в репозиторий.
## Обратно несовместимые изменения:
* Удалена специальная интерпретация выражения IN, если слева указан массив. Ранее выражение вида `arr IN (set)` воспринималось как "хотя бы один элемент `arr` принадлежит множеству `set`". Для получения такого же поведения в новой версии, напишите `arrayExists(x -> x IN (set), arr)`.
* Отключено ошибочное использование опции сокета `SO_REUSEPORT` (которая по ошибке включена по-умолчанию в библиотеке Poco). Стоит обратить внимание, что на Linux системах теперь не имеет смысла указывать одновременно адреса `::` и `0.0.0.0` для listen - следует использовать лишь адрес `::`, который (с настройками ядра по-умолчанию) позволяет слушать соединения как по IPv4 так и по IPv6. Также вы можете вернуть поведение старых версий, указав в конфиге `<listen_reuse_port>1</listen_reuse_port>`.
# ClickHouse release 1.1.54370, 2018-03-16
## Новые возможности:

View File

@ -1,6 +1,8 @@
# ClickHouse
ClickHouse is an open-source column-oriented database management system that allows generating analytical data reports in real time.
🎤🥂 **ClickHouse Meetup in [Sunnyvale](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/248898966/) & [San Francisco](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/249162518/), April 23-27** 🍰🔥🐻
Learn more about ClickHouse at [https://clickhouse.yandex/](https://clickhouse.yandex/)
[![Build Status](https://travis-ci.org/yandex/ClickHouse.svg?branch=master)](https://travis-ci.org/yandex/ClickHouse)

View File

@ -65,7 +65,7 @@ void getAggregateFunctionNameAndParametersArray(
ParserExpressionList params_parser(false);
ASTPtr args_ast = parseQuery(params_parser,
parameters_str.data(), parameters_str.data() + parameters_str.size(),
"parameters of aggregate function in " + error_context);
"parameters of aggregate function in " + error_context, 0);
ASTExpressionList & args_list = typeid_cast<ASTExpressionList &>(*args_ast);
if (args_list.children.empty())

View File

@ -30,7 +30,7 @@ try
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query");
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
Context context = Context::createGlobal();

View File

@ -23,7 +23,7 @@ try
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query");
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
AnalyzeLambdas analyzer;
analyzer.process(ast);

View File

@ -26,7 +26,7 @@ try
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query");
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
Context context = Context::createGlobal();

View File

@ -21,7 +21,7 @@ try
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query");
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
CollectAliases collector;
collector.process(ast);

View File

@ -27,7 +27,7 @@ try
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query");
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
Context context = Context::createGlobal();

View File

@ -33,7 +33,7 @@ try
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query");
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
Context context = Context::createGlobal();

View File

@ -21,7 +21,7 @@ try
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query");
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
TranslatePositionalArguments translator;
translator.process(ast);

View File

@ -38,7 +38,7 @@ try
readStringUntilEOF(query, in);
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query");
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "query", 0);
Context context = Context::createGlobal();

View File

@ -17,13 +17,6 @@ namespace ErrorCodes
}
FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_,
ssize_t filter_column_, bool remove_filter)
: remove_filter(remove_filter), expression(expression_), filter_column(filter_column_)
{
children.push_back(input);
}
FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_,
const String & filter_column_name, bool remove_filter)
: remove_filter(remove_filter), expression(expression_)

View File

@ -20,9 +20,8 @@ private:
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
public:
/// filter_column_ - the number of the column with filter conditions.
FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, ssize_t filter_column_, bool remove_filter = false);
FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, const String & filter_column_name_, bool remove_filter = false);
FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_,
const String & filter_column_name_, bool remove_filter = false);
String getName() const override;
Block getTotals() override;

View File

@ -3,6 +3,7 @@
#include <vector>
#include <memory>
#include <mutex>
#include <shared_mutex>
#include <functional>
#include <boost/noncopyable.hpp>
#include <Core/Block.h>
@ -108,7 +109,9 @@ public:
template <typename F>
void forEachChild(F && f)
{
std::lock_guard lock(children_mutex);
/// NOTE: Acquire a read lock, therefore f() should be thread safe
std::shared_lock lock(children_mutex);
for (auto & child : children)
if (f(*child))
return;
@ -116,7 +119,7 @@ public:
protected:
BlockInputStreams children;
std::mutex children_mutex;
std::shared_mutex children_mutex;
private:
TableStructureReadLocks table_locks;

View File

@ -190,7 +190,7 @@ protected:
void addChild(BlockInputStreamPtr & child)
{
std::lock_guard lock(children_mutex);
std::unique_lock lock(children_mutex);
children.push_back(child);
}
@ -231,7 +231,9 @@ private:
template <typename F>
void forEachProfilingChild(F && f)
{
std::lock_guard lock(children_mutex);
/// NOTE: Acquire a read lock, therefore f() should be thread safe
std::shared_lock lock(children_mutex);
for (auto & child : children)
if (IProfilingBlockInputStream * p_child = dynamic_cast<IProfilingBlockInputStream *>(child.get()))
if (f(*p_child))

View File

@ -31,7 +31,7 @@ try
std::string input = "SELECT number, number / 3, number * number";
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
Context context = Context::createGlobal();

View File

@ -33,7 +33,7 @@ try
std::string input = "SELECT number, number % 3 == 1";
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
formatAST(*ast, std::cerr);
std::cerr << std::endl;
@ -55,7 +55,7 @@ try
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, {}, context, stage, 8192, 1)[0];
in = std::make_shared<FilterBlockInputStream>(in, expression, 1);
in = std::make_shared<FilterBlockInputStream>(in, expression, "equals(modulo(number, 3), 1)");
in = std::make_shared<LimitBlockInputStream>(in, 10, std::max(static_cast<Int64>(0), static_cast<Int64>(n) - 10));
WriteBufferFromOStream ob(std::cout);

View File

@ -97,7 +97,7 @@ int main(int, char **)
std::string input = "SELECT UniqID, URL, CounterID, IsLink WHERE URL = 'http://mail.yandex.ru/neo2/#inbox'";
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
formatAST(*ast, std::cerr);
std::cerr << std::endl;
@ -126,10 +126,10 @@ int main(int, char **)
"IsLink",
};
QueryProcessingStage::Enum stage;
QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context);
BlockInputStreamPtr in = table->read(column_names, {}, context, stage, 8192, 1)[0];
in = std::make_shared<FilterBlockInputStream>(in, expression, 4, *remove_filter);
in = std::make_shared<FilterBlockInputStream>(in, expression, "equals(URL, 'http://mail.yandex.ru/neo2/#inbox')", *remove_filter);
//in = std::make_shared<LimitBlockInputStream>(in, 10, 0);
WriteBufferFromOStream ob(std::cout);

View File

@ -100,7 +100,7 @@ try
std::string input = "SELECT UniqID, URL, CounterID, IsLink";
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
formatAST(*ast, std::cerr);
std::cerr << std::endl;

View File

@ -24,7 +24,7 @@ namespace ErrorCodes
DataTypePtr DataTypeFactory::get(const String & full_name) const
{
ParserIdentifierWithOptionalParameters parser;
ASTPtr ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type");
ASTPtr ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type", 0);
return get(ast);
}

View File

@ -180,7 +180,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const Context & context,
const char * pos = query.data();
std::string error_message;
auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message,
/* hilite = */ false, "", /* allow_multi_statements = */ false);
/* hilite = */ false, "", /* allow_multi_statements = */ false, 0);
if (!ast && throw_on_error)
throw Exception(error_message, ErrorCodes::SYNTAX_ERROR);
@ -206,7 +206,7 @@ ASTPtr DatabaseDictionary::getCreateDatabaseQuery(const Context & /*context*/) c
buffer << "CREATE DATABASE " << backQuoteIfNeed(name) << " ENGINE = Dictionary";
}
ParserCreateQuery parser;
return parseQuery(parser, query.data(), query.data() + query.size(), "");
return parseQuery(parser, query.data(), query.data() + query.size(), "", 0);
}
void DatabaseDictionary::shutdown()

View File

@ -352,7 +352,7 @@ static ASTPtr getQueryFromMetadata(const String & metadata_path, bool throw_on_e
const char * pos = query.data();
std::string error_message;
auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message, /* hilite = */ false,
"in file " + metadata_path, /* allow_multi_statements = */ false);
"in file " + metadata_path, /* allow_multi_statements = */ false, 0);
if (!ast && throw_on_error)
throw Exception(error_message, ErrorCodes::SYNTAX_ERROR);
@ -480,7 +480,7 @@ ASTPtr DatabaseOrdinary::getCreateDatabaseQuery(const Context & /*context*/) con
/// Handle databases (such as default) for which there are no database.sql files.
String query = "CREATE DATABASE " + backQuoteIfNeed(name) + " ENGINE = Ordinary";
ParserCreateQuery parser;
ast = parseQuery(parser, query.data(), query.data() + query.size(), "");
ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0);
}
return ast;
@ -533,7 +533,7 @@ void DatabaseOrdinary::alterTable(
}
ParserCreateQuery parser;
ASTPtr ast = parseQuery(parser, statement.data(), statement.data() + statement.size(), "in file " + table_metadata_path);
ASTPtr ast = parseQuery(parser, statement.data(), statement.data() + statement.size(), "in file " + table_metadata_path, 0);
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);

View File

@ -58,7 +58,7 @@ std::pair<String, StoragePtr> createTableFromDefinition(
const String & description_for_error_message)
{
ParserCreateQuery parser;
ASTPtr ast = parseQuery(parser, definition.data(), definition.data() + definition.size(), description_for_error_message);
ASTPtr ast = parseQuery(parser, definition.data(), definition.data() + definition.size(), description_for_error_message, 0);
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
ast_create_query.attach = true;

View File

@ -32,12 +32,13 @@ HTTPDictionarySource::HTTPDictionarySource(const DictionaryStructure & dict_stru
if (update_field.empty())
return;
/// TODO This code is totally wrong and ignorant.
/// What if URL contains fragment (#). What if update_field contains characters that must be %-encoded.
std::string::size_type option = url.find("?");
if (option == std::string::npos) {
update_field = "?&" + update_field;
} else {
if (option == std::string::npos)
update_field = '?' + update_field;
else
update_field = '&' + update_field;
}
}
HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other)

View File

@ -158,7 +158,6 @@ using FunctionCutQueryStringAndFragment = FunctionStringToString<CutSubstringImp
using FunctionExtractURLParameter = FunctionsStringSearchToString<ExtractURLParameterImpl, NameExtractURLParameter>;
using FunctionCutURLParameter = FunctionsStringSearchToString<CutURLParameterImpl, NameCutURLParameter>;
using FunctionExtractURLParameters = FunctionTokens<ExtractURLParametersImpl>;
using FunctionExtractURLParameters = FunctionTokens<ExtractURLParametersImpl>;
using FunctionURLHierarchy = FunctionTokens<URLHierarchyImpl>;
using FunctionURLPathHierarchy = FunctionTokens<URLPathHierarchyImpl>;
using FunctionExtractURLParameterNames = FunctionTokens<ExtractURLParameterNamesImpl>;

View File

@ -88,34 +88,35 @@ inline StringView getURLScheme(const StringView & url)
/// Extracts host from given url.
inline StringView getURLHost(const StringView & url)
{
StringView scheme = getURLScheme(url);
const char * p = url.data() + scheme.size();
const char * end = url.data() + url.size();
Pos pos = url.data();
Pos end = url.data() + url.size();
// Colon must follows after scheme.
if (p == end || *p != ':')
if (nullptr == (pos = strchr(pos, '/')))
return StringView();
// Authority component must starts with "//".
if (end - p < 2 || (p[1] != '/' || p[2] != '/'))
return StringView();
else
p += 3;
const char * st = p;
for (; p < end; ++p)
if (pos != url.data())
{
if (*p == '@')
{
st = p + 1;
}
else if (*p == ':' || *p == '/' || *p == '?' || *p == '#')
{
break;
}
StringView scheme = getURLScheme(url);
Pos scheme_end = url.data() + scheme.size();
// Colon must follows after scheme.
if (pos - scheme_end != 1 || *scheme_end != ':')
return StringView();
}
return (p == st) ? StringView() : StringView(st, p - st);
if (end - pos < 2 || *(pos) != '/' || *(pos + 1) != '/')
return StringView();
const char *start_of_host = (pos += 2);
for (; pos < end; ++pos)
{
if (*pos == '@')
start_of_host = pos + 1;
else if (*pos == ':' || *pos == '/' || *pos == '?' || *pos == '#')
break;
}
return (pos == start_of_host) ? StringView() : StringView(start_of_host, pos - start_of_host);
}
@ -392,18 +393,35 @@ struct ExtractWWW
Pos pos = data;
Pos end = pos + size;
Pos tmp;
size_t protocol_length;
ExtractProtocol::execute(data, size, tmp, protocol_length);
pos += protocol_length + 3;
if (pos >= end || pos[-1] != '/' || pos[-2] != '/')
return;
if (pos + 4 < end && !strncmp(pos, "www.", 4))
if (nullptr != (pos = strchr(pos, '/')))
{
res_data = pos;
res_size = 4;
if (pos != data)
{
Pos tmp;
size_t protocol_length;
ExtractProtocol::execute(data, size, tmp, protocol_length);
if (pos != data + protocol_length + 1)
return;
}
if (end - pos < 2 || *(pos) != '/' || *(pos + 1) != '/')
return;
const char *start_of_host = (pos += 2);
for (; pos < end; ++pos)
{
if (*pos == '@')
start_of_host = pos + 1;
else if (*pos == ':' || *pos == '/' || *pos == '?' || *pos == '#')
break;
}
if (start_of_host + 4 < end && !strncmp(start_of_host, "www.", 4))
{
res_data = start_of_host;
res_size = 4;
}
}
}
};

View File

@ -47,6 +47,8 @@ public:
String current_user;
String current_query_id;
Poco::Net::SocketAddress current_address;
/// Use current user and password when sending query to replica leader
String current_password;
/// When query_kind == INITIAL_QUERY, these values are equal to current.
String initial_user;

View File

@ -111,6 +111,7 @@ struct ContextShared
String path; /// Path to the data directory, with a slash at the end.
String tmp_path; /// The path to the temporary files that occur when processing the request.
String flags_path; /// Path to the directory with some control flags for server maintenance.
String user_files_path; /// Path to the directory with user provided files, usable by 'file' table function.
ConfigurationPtr config; /// Global configuration settings.
Databases databases; /// List of databases and tables in them.
@ -482,19 +483,29 @@ String Context::getTemporaryPath() const
String Context::getFlagsPath() const
{
auto lock = getLock();
if (!shared->flags_path.empty())
return shared->flags_path;
shared->flags_path = shared->path + "flags/";
Poco::File(shared->flags_path).createDirectories();
return shared->flags_path;
}
String Context::getUserFilesPath() const
{
auto lock = getLock();
return shared->user_files_path;
}
void Context::setPath(const String & path)
{
auto lock = getLock();
shared->path = path;
if (shared->tmp_path.empty())
shared->tmp_path = shared->path + "tmp/";
if (shared->flags_path.empty())
shared->flags_path = shared->path + "flags/";
if (shared->user_files_path.empty())
shared->user_files_path = shared->path + "user_files/";
}
void Context::setTemporaryPath(const String & path)
@ -509,6 +520,12 @@ void Context::setFlagsPath(const String & path)
shared->flags_path = path;
}
void Context::setUserFilesPath(const String & path)
{
auto lock = getLock();
shared->user_files_path = path;
}
void Context::setConfig(const ConfigurationPtr & config)
{
auto lock = getLock();
@ -564,6 +581,7 @@ void Context::setUser(const String & name, const String & password, const Poco::
client_info.current_user = name;
client_info.current_address = address;
client_info.current_password = password;
if (!quota_key.empty())
client_info.quota_key = quota_key;
@ -588,6 +606,12 @@ QuotaForIntervals & Context::getQuota()
}
void Context::checkDatabaseAccessRights(const std::string & database_name) const
{
auto lock = getLock();
checkDatabaseAccessRightsImpl(database_name);
}
void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) const
{
if (client_info.current_user.empty() || (database_name == "system"))
{
@ -602,8 +626,8 @@ void Context::checkDatabaseAccessRights(const std::string & database_name) const
void Context::addDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where)
{
auto lock = getLock();
checkDatabaseAccessRights(from.first);
checkDatabaseAccessRights(where.first);
checkDatabaseAccessRightsImpl(from.first);
checkDatabaseAccessRightsImpl(where.first);
shared->view_dependencies[from].insert(where);
// Notify table of dependencies change
@ -615,8 +639,8 @@ void Context::addDependency(const DatabaseAndTableName & from, const DatabaseAnd
void Context::removeDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where)
{
auto lock = getLock();
checkDatabaseAccessRights(from.first);
checkDatabaseAccessRights(where.first);
checkDatabaseAccessRightsImpl(from.first);
checkDatabaseAccessRightsImpl(where.first);
shared->view_dependencies[from].erase(where);
// Notify table of dependencies change
@ -637,7 +661,7 @@ Dependencies Context::getDependencies(const String & database_name, const String
}
else
{
checkDatabaseAccessRights(db);
checkDatabaseAccessRightsImpl(db);
}
ViewDependencies::const_iterator iter = shared->view_dependencies.find(DatabaseAndTableName(db, table_name));
@ -652,7 +676,7 @@ bool Context::isTableExist(const String & database_name, const String & table_na
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
checkDatabaseAccessRights(db);
checkDatabaseAccessRightsImpl(db);
Databases::const_iterator it = shared->databases.find(db);
return shared->databases.end() != it
@ -664,7 +688,7 @@ bool Context::isDatabaseExist(const String & database_name) const
{
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
checkDatabaseAccessRights(db);
checkDatabaseAccessRightsImpl(db);
return shared->databases.end() != shared->databases.find(db);
}
@ -679,7 +703,7 @@ void Context::assertTableExists(const String & database_name, const String & tab
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
checkDatabaseAccessRights(db);
checkDatabaseAccessRightsImpl(db);
Databases::const_iterator it = shared->databases.find(db);
if (shared->databases.end() == it)
@ -696,7 +720,7 @@ void Context::assertTableDoesntExist(const String & database_name, const String
String db = resolveDatabase(database_name, current_database);
if (check_database_access_rights)
checkDatabaseAccessRights(db);
checkDatabaseAccessRightsImpl(db);
Databases::const_iterator it = shared->databases.find(db);
if (shared->databases.end() != it && it->second->isTableExist(*this, table_name))
@ -710,7 +734,7 @@ void Context::assertDatabaseExists(const String & database_name, bool check_data
String db = resolveDatabase(database_name, current_database);
if (check_database_access_rights)
checkDatabaseAccessRights(db);
checkDatabaseAccessRightsImpl(db);
if (shared->databases.end() == shared->databases.find(db))
throw Exception("Database " + backQuoteIfNeed(db) + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
@ -722,7 +746,7 @@ void Context::assertDatabaseDoesntExist(const String & database_name) const
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
checkDatabaseAccessRights(db);
checkDatabaseAccessRightsImpl(db);
if (shared->databases.end() != shared->databases.find(db))
throw Exception("Database " + backQuoteIfNeed(db) + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS);
@ -789,7 +813,7 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta
}
String db = resolveDatabase(database_name, current_database);
checkDatabaseAccessRights(db);
checkDatabaseAccessRightsImpl(db);
Databases::const_iterator it = shared->databases.find(db);
if (shared->databases.end() == it)

View File

@ -132,9 +132,12 @@ public:
String getPath() const;
String getTemporaryPath() const;
String getFlagsPath() const;
String getUserFilesPath() const;
void setPath(const String & path);
void setTemporaryPath(const String & path);
void setFlagsPath(const String & path);
void setUserFilesPath(const String & path);
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
@ -178,6 +181,7 @@ public:
void assertDatabaseExists(const String & database_name, bool check_database_acccess_rights = true) const;
void assertDatabaseDoesntExist(const String & database_name) const;
void checkDatabaseAccessRights(const std::string & database_name) const;
Tables getExternalTables() const;
StoragePtr tryGetExternalTable(const String & table_name) const;
@ -392,7 +396,7 @@ private:
* If access is denied, throw an exception.
* NOTE: This method should always be called when the `shared->mutex` mutex is acquired.
*/
void checkDatabaseAccessRights(const std::string & database_name) const;
void checkDatabaseAccessRightsImpl(const std::string & database_name) const;
EmbeddedDictionaries & getEmbeddedDictionariesImpl(bool throw_on_error) const;
ExternalDictionaries & getExternalDictionariesImpl(bool throw_on_error) const;

View File

@ -425,7 +425,7 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task)
ParserQuery parser_query(end);
String description;
task.query = parseQuery(parser_query, begin, end, description);
task.query = parseQuery(parser_query, begin, end, description, 0);
}
if (!task.query || !(task.query_on_cluster = dynamic_cast<ASTQueryWithOnCluster *>(task.query.get())))
@ -960,15 +960,25 @@ public:
{
Block res;
if (num_hosts_finished >= waiting_hosts.size())
{
if (first_exception)
throw Exception(*first_exception);
return res;
}
auto zookeeper = context.getZooKeeper();
size_t try_number = 0;
while(res.rows() == 0)
while (res.rows() == 0)
{
if (isCancelled())
{
if (first_exception)
throw Exception(*first_exception);
return res;
}
if (timeout_seconds >= 0 && watch.elapsedSeconds() > timeout_seconds)
{
@ -1020,6 +1030,9 @@ public:
UInt16 port;
Cluster::Address::fromString(host_id, host, port);
if (status.code != 0 && first_exception == nullptr)
first_exception = std::make_unique<Exception>("There was an error on " + host + ": " + status.message, status.code);
++num_hosts_finished;
columns[0]->insert(host);
@ -1092,11 +1105,14 @@ private:
Strings current_active_hosts; /// Hosts that were in active state at the last check
size_t num_hosts_finished = 0;
/// Save the first detected error and throw it at the end of excecution
std::unique_ptr<Exception> first_exception;
Int64 timeout_seconds = 120;
};
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & context)
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & context, const NameSet & query_databases)
{
/// Remove FORMAT <fmt> and INTO OUTFILE <file> if exists
ASTPtr query_ptr = query_ptr_->clone();
@ -1128,13 +1144,26 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & cont
entry.query = queryToString(query_ptr);
entry.initiator = ddl_worker.getCommonHostID();
/// Check database access rights, assume that all servers have the same users config
NameSet databases_to_check_access_rights;
Cluster::AddressesWithFailover shards = cluster->getShardsAddresses();
for (const auto & shard : shards)
{
for (const auto & addr : shard)
{
entry.hosts.emplace_back(addr);
/// Expand empty database name to shards' default database name
for (const String & database : query_databases)
databases_to_check_access_rights.emplace(database.empty() ? addr.default_database : database);
}
}
for (const String & database : databases_to_check_access_rights)
context.checkDatabaseAccessRights(database.empty() ? context.getCurrentDatabase() : database);
String node_path = ddl_worker.enqueueQuery(entry);
BlockIO io;

View File

@ -18,7 +18,8 @@ struct DDLLogEntry;
struct DDLTask;
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context);
/// Pushes distributed DDL query to the queue
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context, const NameSet & query_databases);
class DDLWorker

View File

@ -17,7 +17,6 @@
#include <DataTypes/DataTypeSet.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypesNumber.h>

View File

@ -42,7 +42,7 @@ BlockIO InterpreterAlterQuery::execute()
auto & alter = typeid_cast<ASTAlterQuery &>(*query_ptr);
if (!alter.cluster.empty())
return executeDDLQueryOnCluster(query_ptr, context);
return executeDDLQueryOnCluster(query_ptr, context, {alter.table});
const String & table_name = alter.table;
String database_name = alter.database.empty() ? context.getCurrentDatabase() : alter.database;

View File

@ -66,7 +66,7 @@ InterpreterCreateQuery::InterpreterCreateQuery(const ASTPtr & query_ptr_, Contex
BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
{
if (!create.cluster.empty())
return executeDDLQueryOnCluster(query_ptr, context);
return executeDDLQueryOnCluster(query_ptr, context, {create.database});
String database_name = create.database;
@ -291,7 +291,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
const auto end = pos + type_name->size();
ParserIdentifierWithOptionalParameters storage_p;
column_declaration->type = parseQuery(storage_p, pos, end, "data type");
column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0);
column_declaration->type->owned_string = type_name;
columns_list->children.emplace_back(column_declaration);
}
@ -315,7 +315,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
const auto end = pos + type_name->size();
ParserIdentifierWithOptionalParameters storage_p;
column_declaration->type = parseQuery(storage_p, pos, end, "data type");
column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0);
column_declaration->type->owned_string = type_name;
const auto it = columns.defaults.find(column.name);
@ -439,7 +439,13 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
{
if (!create.cluster.empty())
return executeDDLQueryOnCluster(query_ptr, context);
{
NameSet databases{create.database};
if (!create.to_table.empty())
databases.emplace(create.to_database);
return executeDDLQueryOnCluster(query_ptr, context, databases);
}
String path = context.getPath();
String current_database = context.getCurrentDatabase();

View File

@ -32,7 +32,7 @@ BlockIO InterpreterDropQuery::execute()
checkAccess(drop);
if (!drop.cluster.empty())
return executeDDLQueryOnCluster(query_ptr, context);
return executeDDLQueryOnCluster(query_ptr, context, {drop.database});
String path = context.getPath();
String current_database = context.getCurrentDatabase();

View File

@ -147,7 +147,6 @@ public:
}
/// KILL QUERY could be killed also
/// Probably interpreting KILL QUERIES as complete (not internal) queries is extra functionality
if (isCancelled())
break;

View File

@ -39,7 +39,16 @@ BlockIO InterpreterRenameQuery::execute()
ASTRenameQuery & rename = typeid_cast<ASTRenameQuery &>(*query_ptr);
if (!rename.cluster.empty())
return executeDDLQueryOnCluster(query_ptr, context);
{
NameSet databases;
for (const auto & elem : rename.elements)
{
databases.emplace(elem.from.database);
databases.emplace(elem.to.database);
}
return executeDDLQueryOnCluster(query_ptr, context, databases);
}
String path = context.getPath();
String current_database = context.getCurrentDatabase();

View File

@ -1,12 +1,17 @@
#include <Interpreters/ProcessList.h>
#include <Interpreters/Settings.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTKillQueryQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Common/typeid_cast.h>
#include <common/logger_useful.h>
#include <chrono>
namespace DB
{
@ -19,21 +24,74 @@ namespace ErrorCodes
}
/// Should we execute the query even if max_concurrent_queries limit is exhausted
static bool isUnlimitedQuery(const IAST * ast)
{
if (!ast)
return false;
/// It is KILL QUERY
if (typeid_cast<const ASTKillQueryQuery *>(ast))
return true;
/// It is SELECT FROM system.processes
/// NOTE: This is very rough check.
/// False negative: USE system; SELECT * FROM processes;
/// False positive: SELECT * FROM system.processes CROSS JOIN (SELECT ...)
if (auto ast_selects = typeid_cast<const ASTSelectWithUnionQuery *>(ast))
{
if (!ast_selects->list_of_selects || ast_selects->list_of_selects->children.empty())
return false;
auto ast_select = typeid_cast<ASTSelectQuery *>(ast_selects->list_of_selects->children[0].get());
if (!ast_select)
return false;
auto ast_database = ast_select->database();
if (!ast_database)
return false;
auto ast_table = ast_select->table();
if (!ast_table)
return false;
auto ast_database_id = typeid_cast<const ASTIdentifier *>(ast_database.get());
if (!ast_database_id)
return false;
auto ast_table_id = typeid_cast<const ASTIdentifier *>(ast_table.get());
if (!ast_table_id)
return false;
return ast_database_id->name == "system" && ast_table_id->name == "processes";
}
return false;
}
ProcessList::EntryPtr ProcessList::insert(
const String & query_, const IAST * ast, const ClientInfo & client_info, const Settings & settings)
{
EntryPtr res;
bool is_kill_query = ast && typeid_cast<const ASTKillQueryQuery *>(ast);
if (client_info.current_query_id.empty())
throw Exception("Query id cannot be empty", ErrorCodes::LOGICAL_ERROR);
{
std::lock_guard<std::mutex> lock(mutex);
bool is_unlimited_query = isUnlimitedQuery(ast);
if (!is_kill_query && max_size && cur_size >= max_size
&& (!settings.queue_max_wait_ms.totalMilliseconds() || !have_space.tryWait(mutex, settings.queue_max_wait_ms.totalMilliseconds())))
throw Exception("Too many simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES);
{
std::unique_lock lock(mutex);
if (!is_unlimited_query && max_size && cur_size >= max_size)
{
auto max_wait_ms = settings.queue_max_wait_ms.totalMilliseconds();
if (!max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(max_wait_ms), [&]{ return cur_size < max_size; }))
throw Exception("Too many simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES);
}
/** Why we use current user?
* Because initial one is passed by client and credentials for it is not verified,
@ -50,7 +108,7 @@ ProcessList::EntryPtr ProcessList::insert(
if (user_process_list != user_to_queries.end())
{
if (!is_kill_query && settings.max_concurrent_queries_for_user
if (!is_unlimited_query && settings.max_concurrent_queries_for_user
&& user_process_list->second.queries.size() >= settings.max_concurrent_queries_for_user)
throw Exception("Too many simultaneous queries for user " + client_info.current_user
+ ". Current: " + toString(user_process_list->second.queries.size())
@ -172,7 +230,7 @@ ProcessListEntry::~ProcessListEntry()
user_process_list.reset();
--parent.cur_size;
parent.have_space.signal();
parent.have_space.notify_one();
/// This removes memory_tracker for all requests. At this time, no other memory_trackers live.
if (parent.cur_size == 0)
@ -191,31 +249,37 @@ void ProcessListElement::setQueryStreams(const BlockIO & io)
query_stream_in = io.in;
query_stream_out = io.out;
query_streams_initialized = true;
query_streams_status = QueryStreamsStatus::Initialized;
}
void ProcessListElement::releaseQueryStreams()
{
std::lock_guard<std::mutex> lock(query_streams_mutex);
BlockInputStreamPtr in;
BlockOutputStreamPtr out;
query_streams_initialized = false;
query_streams_released = true;
query_stream_in.reset();
query_stream_out.reset();
{
std::lock_guard<std::mutex> lock(query_streams_mutex);
query_streams_status = QueryStreamsStatus::Released;
in = std::move(query_stream_in);
out = std::move(query_stream_out);
}
/// Destroy streams outside the mutex lock
}
bool ProcessListElement::streamsAreReleased()
{
std::lock_guard<std::mutex> lock(query_streams_mutex);
return query_streams_released;
return query_streams_status == QueryStreamsStatus::Released;
}
bool ProcessListElement::tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStreamPtr & out) const
{
std::lock_guard<std::mutex> lock(query_streams_mutex);
if (!query_streams_initialized)
if (query_streams_status != QueryStreamsStatus::Initialized)
return false;
in = query_stream_in;

View File

@ -4,8 +4,8 @@
#include <list>
#include <memory>
#include <mutex>
#include <condition_variable>
#include <unordered_map>
#include <Poco/Condition.h>
#include <Common/Stopwatch.h>
#include <Core/Defines.h>
#include <IO/Progress.h>
@ -91,8 +91,14 @@ private:
BlockInputStreamPtr query_stream_in;
BlockOutputStreamPtr query_stream_out;
bool query_streams_initialized{false};
bool query_streams_released{false};
enum QueryStreamsStatus
{
NotInitialized,
Initialized,
Released
};
QueryStreamsStatus query_streams_status{NotInitialized};
public:
ProcessListElement(
@ -249,7 +255,7 @@ public:
private:
mutable std::mutex mutex;
mutable Poco::Condition have_space; /// Number of currently running queries has become less than maximum.
mutable std::condition_variable have_space; /// Number of currently running queries has become less than maximum.
/// List of queries
Container cont;

View File

@ -115,34 +115,6 @@ bool Set::insertFromBlock(const Block & block, bool fill_set_elements)
}
}
/** Flatten tuples. For case when written
* (a, b) IN (SELECT (a, b) FROM table)
* instead of more typical
* (a, b) IN (SELECT a, b FROM table)
*
* Avoid flatten in case then we have more than one column:
* Ex.: 1, (2, 3) become just 1, 2, 3
*/
if (keys_size == 1)
{
const auto & col = block.getByPosition(0);
if (const DataTypeTuple * tuple = typeid_cast<const DataTypeTuple *>(col.type.get()))
{
const ColumnTuple & column = typeid_cast<const ColumnTuple &>(*key_columns[0]);
key_columns.pop_back();
const Columns & tuple_elements = column.getColumns();
for (const auto & elem : tuple_elements)
key_columns.push_back(elem.get());
if (empty())
{
data_types.pop_back();
data_types.insert(data_types.end(), tuple->getElements().begin(), tuple->getElements().end());
}
}
}
size_t rows = block.rows();
/// We will insert to the Set only keys, where all components are not NULL.
@ -172,9 +144,8 @@ bool Set::insertFromBlock(const Block & block, bool fill_set_elements)
{
std::vector<Field> new_set_elements;
for (size_t j = 0; j < keys_size; ++j)
{
new_set_elements.push_back((*key_columns[j])[i]);
}
set_elements->emplace_back(std::move(new_set_elements));
}
}
@ -401,20 +372,19 @@ MergeTreeSetIndex::MergeTreeSetIndex(const SetElements & set_elements, std::vect
return std::forward_as_tuple(l.pk_index, l.tuple_index) < std::forward_as_tuple(r.pk_index, r.tuple_index);
});
std::unique(
indexes_mapping.erase(std::unique(
indexes_mapping.begin(), indexes_mapping.end(),
[](const PKTuplePositionMapping & l, const PKTuplePositionMapping & r)
{
return l.pk_index == r.pk_index;
});
}), indexes_mapping.end());
for (size_t i = 0; i < set_elements.size(); ++i)
{
std::vector<FieldWithInfinity> new_set_values;
for (size_t j = 0; j < indexes_mapping.size(); ++j)
{
new_set_values.push_back(FieldWithInfinity(set_elements[i][indexes_mapping[j].tuple_index]));
}
new_set_values.emplace_back(set_elements[i][indexes_mapping[j].tuple_index]);
ordered_set.emplace_back(std::move(new_set_values));
}

View File

@ -56,6 +56,9 @@ public:
size_t getTotalRowCount() const { return data.getTotalRowCount(); }
size_t getTotalByteCount() const { return data.getTotalByteCount(); }
const DataTypes & getDataTypes() const { return data_types; }
SetElements & getSetElements() { return *set_elements.get(); }
private:

View File

@ -333,7 +333,7 @@ void SystemLog<LogElement>::prepareTable()
ParserStorage storage_parser;
ASTPtr storage_ast = parseQuery(
storage_parser, storage_def.data(), storage_def.data() + storage_def.size(),
"Storage to create table for " + LogElement::name());
"Storage to create table for " + LogElement::name(), 0);
create->set(create->storage, storage_ast);
InterpreterCreateQuery interpreter(create, context);

View File

@ -157,16 +157,12 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
try
{
/// TODO Parser should fail early when max_query_size limit is reached.
ast = parseQuery(parser, begin, end, "");
ast = parseQuery(parser, begin, end, "", max_query_size);
/// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion.
if (!(begin <= ast->range.first && ast->range.second <= end))
throw Exception("Unexpected behavior: AST chars range is not inside source range", ErrorCodes::LOGICAL_ERROR);
query_size = ast->range.second - begin;
if (max_query_size && query_size > max_query_size)
throw Exception("Query is too large (" + toString(query_size) + ")."
" max_query_size = " + toString(max_query_size), ErrorCodes::QUERY_IS_TOO_LARGE);
}
catch (...)
{
@ -417,7 +413,7 @@ void executeQuery(
size_t max_query_size = context.getSettingsRef().max_query_size;
if (istr.buffer().end() - istr.position() >= static_cast<ssize_t>(max_query_size))
if (istr.buffer().end() - istr.position() > static_cast<ssize_t>(max_query_size))
{
/// If remaining buffer space in 'istr' is enough to parse query up to 'max_query_size' bytes, then parse inplace.
begin = istr.position();
@ -427,8 +423,8 @@ void executeQuery(
else
{
/// If not - copy enough data into 'parse_buf'.
parse_buf.resize(max_query_size);
parse_buf.resize(istr.read(&parse_buf[0], max_query_size));
parse_buf.resize(max_query_size + 1);
parse_buf.resize(istr.read(&parse_buf[0], max_query_size + 1));
begin = &parse_buf[0];
end = begin + parse_buf.size();
}

View File

@ -35,7 +35,7 @@ static void executeCreateQuery(
bool has_force_restore_data_flag)
{
ParserCreateQuery parser;
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "in file " + file_name);
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "in file " + file_name, 0);
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
ast_create_query.attach = true;

View File

@ -76,7 +76,7 @@ try
") ENGINE = Log";
ParserCreateQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
Context context = Context::createGlobal();

View File

@ -41,7 +41,7 @@ int main(int argc, char ** argv)
"s1 < s2 AND x % 3 < x % 5";
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
formatAST(*ast, std::cerr);
std::cerr << std::endl;

View File

@ -1234,7 +1234,7 @@ bool parse(DB::ASTPtr & ast, const std::string & query)
std::string message;
auto begin = query.data();
auto end = begin + query.size();
ast = DB::tryParseQuery(parser, begin, end, message, false, "", false);
ast = DB::tryParseQuery(parser, begin, end, message, false, "", false, 0);
return ast != nullptr;
}

View File

@ -240,7 +240,7 @@ bool parse(DB::ASTPtr & ast, const std::string & query)
std::string message;
auto begin = query.data();
auto end = begin + query.size();
ast = DB::tryParseQuery(parser, begin, end, message, false, "", false);
ast = DB::tryParseQuery(parser, begin, end, message, false, "", false, 0);
return ast != nullptr;
}

View File

@ -50,6 +50,8 @@ Token quotedString(const char *& pos, const char * const token_begin, const char
Token Lexer::nextToken()
{
Token res = nextTokenImpl();
if (res.type != TokenType::EndOfStream && max_query_size && res.end > begin + max_query_size)
res.type = TokenType::ErrorMaxQuerySizeExceeded;
if (res.isSignificant())
prev_significant_token_type = res.type;
return res;
@ -341,6 +343,8 @@ const char * getErrorTokenDescription(TokenType type)
return "Pipe symbol could only occur in || operator";
case TokenType::ErrorWrongNumber:
return "Wrong number";
case TokenType::ErrorMaxQuerySizeExceeded:
return "Max query size exceeded";
default:
return "Not an error";
}

View File

@ -59,6 +59,8 @@ namespace DB
M(ErrorSingleExclamationMark) \
M(ErrorSinglePipeMark) \
M(ErrorWrongNumber) \
M(ErrorMaxQuerySizeExceeded) \
enum class TokenType
{
@ -91,7 +93,8 @@ struct Token
class Lexer
{
public:
Lexer(const char * begin, const char * end) : begin(begin), pos(begin), end(end) {}
Lexer(const char * begin, const char * end, size_t max_query_size = 0)
: begin(begin), pos(begin), end(end), max_query_size(max_query_size) {}
Token nextToken();
private:
@ -99,6 +102,8 @@ private:
const char * pos;
const char * const end;
const size_t max_query_size;
Token nextTokenImpl();
/// This is needed to disambiguate tuple access operator from floating point number (.1).

View File

@ -1,26 +1,46 @@
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ParserUnionQueryElement.h>
#include <Parsers/ASTExpressionList.h>
#include <Common/typeid_cast.h>
namespace DB
{
static void getSelectsFromUnionListNode(ASTPtr & ast_select, ASTs & selects)
{
if (ASTSelectWithUnionQuery * inner_union = typeid_cast<ASTSelectWithUnionQuery *>(ast_select.get()))
{
for (auto & child : inner_union->list_of_selects->children)
getSelectsFromUnionListNode(child, selects);
return;
}
selects.push_back(std::move(ast_select));
}
bool ParserSelectWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ASTPtr list_node;
ParserList parser(std::make_unique<ParserSelectQuery>(), std::make_unique<ParserKeyword>("UNION ALL"), false);
ParserList parser(std::make_unique<ParserUnionQueryElement>(), std::make_unique<ParserKeyword>("UNION ALL"), false);
if (!parser.parse(pos, list_node, expected))
return false;
auto res = std::make_shared<ASTSelectWithUnionQuery>();
auto select_with_union_query = std::make_shared<ASTSelectWithUnionQuery>();
res->list_of_selects = std::move(list_node);
res->children.push_back(res->list_of_selects);
node = select_with_union_query;
select_with_union_query->list_of_selects = std::make_shared<ASTExpressionList>();
select_with_union_query->children.push_back(select_with_union_query->list_of_selects);
// flatten inner union query
for (auto & child : list_node->children)
getSelectsFromUnionListNode(child, select_with_union_query->list_of_selects->children);
node = res;
return true;
}

View File

@ -1,7 +1,6 @@
#pragma once
#include <Parsers/ParserQueryWithOutput.h>
#include <Parsers/IParserBase.h>
namespace DB
{

View File

@ -0,0 +1,22 @@
#include <Parsers/ASTSubquery.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/ParserUnionQueryElement.h>
#include <Common/typeid_cast.h>
namespace DB
{
bool ParserUnionQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserSubquery().parse(pos, node, expected) && !ParserSelectQuery().parse(pos, node, expected))
return false;
if (auto * ast_sub_query = typeid_cast<ASTSubquery *>(node.get()))
node = ast_sub_query->children.at(0);
return true;
}
}

View File

@ -0,0 +1,17 @@
#pragma once
#include <Parsers/IParserBase.h>
namespace DB
{
class ParserUnionQueryElement : public IParserBase
{
protected:
const char * getName() const override { return "SELECT query, subquery, possibly with UNION"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -21,7 +21,7 @@ private:
Lexer lexer;
public:
Tokens(const char * begin, const char * end) : lexer(begin, end) {}
Tokens(const char * begin, const char * end, size_t max_query_size = 0) : lexer(begin, end, max_query_size) {}
const Token & operator[] (size_t index)
{

View File

@ -3,7 +3,7 @@
namespace DB
{
class Token;
struct Token;
std::ostream & operator<<(std::ostream & stream, const Token & what);
struct Expected;

View File

@ -213,9 +213,10 @@ ASTPtr tryParseQuery(
std::string & out_error_message,
bool hilite,
const std::string & query_description,
bool allow_multi_statements)
bool allow_multi_statements,
size_t max_query_size)
{
Tokens tokens(pos, end);
Tokens tokens(pos, end, max_query_size);
TokenIterator token_iterator(tokens);
if (token_iterator->isEnd()
@ -294,10 +295,11 @@ ASTPtr parseQueryAndMovePosition(
const char * & pos,
const char * end,
const std::string & query_description,
bool allow_multi_statements)
bool allow_multi_statements,
size_t max_query_size)
{
std::string error_message;
ASTPtr res = tryParseQuery(parser, pos, end, error_message, false, query_description, allow_multi_statements);
ASTPtr res = tryParseQuery(parser, pos, end, error_message, false, query_description, allow_multi_statements, max_query_size);
if (res)
return res;
@ -310,25 +312,27 @@ ASTPtr parseQuery(
IParser & parser,
const char * begin,
const char * end,
const std::string & query_description)
const std::string & query_description,
size_t max_query_size)
{
auto pos = begin;
return parseQueryAndMovePosition(parser, pos, end, query_description, false);
return parseQueryAndMovePosition(parser, pos, end, query_description, false, max_query_size);
}
ASTPtr parseQuery(
IParser & parser,
const std::string & query,
const std::string & query_description)
const std::string & query_description,
size_t max_query_size)
{
return parseQuery(parser, query.data(), query.data() + query.size(), query_description);
return parseQuery(parser, query.data(), query.data() + query.size(), query_description, max_query_size);
}
ASTPtr parseQuery(IParser & parser, const std::string & query)
ASTPtr parseQuery(IParser & parser, const std::string & query, size_t max_query_size)
{
return parseQuery(parser, query.data(), query.data() + query.size(), parser.getName());
return parseQuery(parser, query.data(), query.data() + query.size(), parser.getName(), max_query_size);
}
@ -348,7 +352,7 @@ std::pair<const char *, bool> splitMultipartQuery(const std::string & queries, s
{
begin = pos;
ast = parseQueryAndMovePosition(parser, pos, end, "", true);
ast = parseQueryAndMovePosition(parser, pos, end, "", true, 0);
if (!ast)
break;

View File

@ -14,7 +14,9 @@ ASTPtr tryParseQuery(
std::string & out_error_message,
bool hilite,
const std::string & description,
bool allow_multi_statements); /// If false, check for non-space characters after semicolon and set error message if any.
bool allow_multi_statements, /// If false, check for non-space characters after semicolon and set error message if any.
size_t max_query_size); /// If (end - pos) > max_query_size and query is longer than max_query_size then throws "Max query size exceeded".
/// Disabled if zero. Is used in order to check query size if buffer can contains data for INSERT query.
/// Parse query or throw an exception with error message.
@ -23,23 +25,27 @@ ASTPtr parseQueryAndMovePosition(
const char * & pos, /// Moved to end of parsed fragment.
const char * end,
const std::string & description,
bool allow_multi_statements);
bool allow_multi_statements,
size_t max_query_size);
ASTPtr parseQuery(
IParser & parser,
const char * begin,
const char * end,
const std::string & description);
const std::string & description,
size_t max_query_size);
ASTPtr parseQuery(
IParser & parser,
const std::string & query,
const std::string & query_description);
const std::string & query_description,
size_t max_query_size);
ASTPtr parseQuery(
IParser & parser,
const std::string & query);
const std::string & query,
size_t max_query_size);
/** Split queries separated by ; on to list of single queries

View File

@ -12,7 +12,7 @@ int main(int, char **)
std::string input = "CREATE TABLE hits (URL String, UserAgentMinor2 FixedString(2), EventTime DateTime) ENGINE = Log";
ParserCreateQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
formatAST(*ast, std::cerr);
std::cerr << std::endl;

View File

@ -57,6 +57,7 @@ std::map<TokenType, const char *> hilite =
{TokenType::ErrorBackQuoteIsNotClosed, "\033[0;41m"},
{TokenType::ErrorSingleExclamationMark, "\033[0;41m"},
{TokenType::ErrorWrongNumber, "\033[0;41m"},
{TokenType::ErrorMaxQuerySizeExceeded, "\033[0;41m"},
};

View File

@ -23,7 +23,7 @@ try
" FORMAT TabSeparated";
ParserQueryWithOutput parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
std::cout << "Success." << std::endl;
formatAST(*ast, std::cerr);

View File

@ -863,7 +863,7 @@ private:
if (is_interactive || ignore_error)
{
String message;
res = tryParseQuery(parser, pos, end, message, true, "", allow_multi_statements);
res = tryParseQuery(parser, pos, end, message, true, "", allow_multi_statements, 0);
if (!res)
{
@ -872,7 +872,7 @@ private:
}
}
else
res = parseQueryAndMovePosition(parser, pos, end, "", allow_multi_statements);
res = parseQueryAndMovePosition(parser, pos, end, "", allow_multi_statements, 0);
if (is_interactive)
{

View File

@ -547,14 +547,14 @@ TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfigurati
engine_push_str = config.getString(table_prefix + "engine");
{
ParserStorage parser_storage;
engine_push_ast = parseQuery(parser_storage, engine_push_str);
engine_push_ast = parseQuery(parser_storage, engine_push_str, 0);
engine_push_partition_key_ast = extractPartitionKey(engine_push_ast);
}
sharding_key_str = config.getString(table_prefix + "sharding_key");
{
ParserExpressionWithOptionalAlias parser_expression(false);
sharding_key_ast = parseQuery(parser_expression, sharding_key_str);
sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0);
engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, sharding_key_ast);
}
@ -562,7 +562,7 @@ TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfigurati
if (!where_condition_str.empty())
{
ParserExpressionWithOptionalAlias parser_expression(false);
where_condition_ast = parseQuery(parser_expression, where_condition_str);
where_condition_ast = parseQuery(parser_expression, where_condition_str, 0);
// Will use canonical expression form
where_condition_str = queryToString(where_condition_ast);
@ -1487,7 +1487,7 @@ protected:
query += " LIMIT " + limit;
ParserQuery p_query(query.data() + query.size());
return parseQuery(p_query, query);
return parseQuery(p_query, query, 0);
};
/// Load balancing
@ -1642,7 +1642,7 @@ protected:
query += "INSERT INTO " + getDatabaseDotTable(task_shard.table_split_shard) + " VALUES ";
ParserQuery p_query(query.data() + query.size());
query_insert_ast = parseQuery(p_query, query);
query_insert_ast = parseQuery(p_query, query, 0);
LOG_DEBUG(log, "Executing INSERT query: " << query);
}
@ -1791,7 +1791,7 @@ protected:
&task_cluster->settings_pull);
ParserCreateQuery parser_create_query;
return parseQuery(parser_create_query, create_query_pull_str);
return parseQuery(parser_create_query, create_query_pull_str, 0);
}
void createShardInternalTables(TaskShard & task_shard, bool create_split = true)
@ -1842,7 +1842,7 @@ protected:
}
ParserQuery parser_query(query.data() + query.size());
ASTPtr query_ast = parseQuery(parser_query, query);
ASTPtr query_ast = parseQuery(parser_query, query, 0);
LOG_DEBUG(log, "Computing destination partition set, executing query: " << query);
@ -1889,7 +1889,7 @@ protected:
<< partition_quoted_name << " existence, executing query: " << query);
ParserQuery parser_query(query.data() + query.size());
ASTPtr query_ast = parseQuery(parser_query, query);
ASTPtr query_ast = parseQuery(parser_query, query, 0);
Context local_context = context;
local_context.setSettings(task_cluster->settings_pull);
@ -1914,7 +1914,7 @@ protected:
if (query_ast_ == nullptr)
{
ParserQuery p_query(query.data() + query.size());
query_ast = parseQuery(p_query, query);
query_ast = parseQuery(p_query, query, 0);
}
else
query_ast = query_ast_;

View File

@ -50,7 +50,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv)
const char * end = pos + query.size();
ParserQuery parser(end);
ASTPtr res = parseQuery(parser, pos, end, "query");
ASTPtr res = parseQuery(parser, pos, end, "query", 0);
if (!quiet)
{

View File

@ -169,7 +169,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
DateLUT::instance();
LOG_TRACE(log, "Initialized DateLUT with time zone `" << DateLUT::instance().getTimeZone() << "'.");
/// Directory with temporary data for processing of hard queries.
/// Directory with temporary data for processing of heavy queries.
{
std::string tmp_path = config().getString("tmp_path", path + "tmp/");
global_context->setTemporaryPath(tmp_path);
@ -191,8 +191,19 @@ int Server::main(const std::vector<std::string> & /*args*/)
* Flags may be cleared automatically after being applied by the server.
* Examples: do repair of local data; clone all replicated tables from replica.
*/
Poco::File(path + "flags/").createDirectories();
global_context->setFlagsPath(path + "flags/");
{
Poco::File(path + "flags/").createDirectories();
global_context->setFlagsPath(path + "flags/");
}
/** Directory with user provided files that are usable by 'file' table function.
*/
{
std::string user_files_path = config().getString("user_files_path", path + "user_files/");
global_context->setUserFilesPath(user_files_path);
Poco::File(user_files_path).createDirectories();
}
if (config().has("interserver_http_port"))
{

View File

@ -1,5 +1,6 @@
<yandex>
<path replace="replace">./</path>
<tmp_path replace="replace">./tmp/</tmp_path>
<user_files_path replace="replace">./user_files/</user_files_path>
<format_schema_path replace="replace">./format_schemas/</format_schema_path>
</yandex>

View File

@ -111,6 +111,9 @@
<!-- Path to temporary data for processing hard queries. -->
<tmp_path>/var/lib/clickhouse/tmp/</tmp_path>
<!-- Directory with user provided files that are accessible by 'file' table function. -->
<user_files_path>/var/lib/clickhouse/user_files/</user_files_path>
<!-- Path to configuration file with users, access rights, profiles of settings, quotas. -->
<users_config>users.xml</users_config>

View File

@ -145,7 +145,7 @@ ColumnsDescription ColumnsDescription::parse(const String & str)
const char * begin = default_expr_str.data();
const auto end = begin + default_expr_str.size();
ASTPtr default_expr = parseQuery(expr_parser, begin, end, "default expression");
ASTPtr default_expr = parseQuery(expr_parser, begin, end, "default expression", 0);
if (ColumnDefaultKind::Default == default_kind)
result.ordinary.emplace_back(column_name, std::move(type));

View File

@ -129,7 +129,7 @@ MergeTreeData::MergeTreeData(
String partition_expr_str = "toYYYYMM(" + backQuoteIfNeed(date_column_name) + ")";
ParserNotEmptyExpressionList parser(/* allow_alias_without_as_keyword = */ false);
partition_expr_ast = parseQuery(
parser, partition_expr_str.data(), partition_expr_str.data() + partition_expr_str.length(), "partition expression");
parser, partition_expr_str.data(), partition_expr_str.data() + partition_expr_str.length(), "partition expression", 0);
initPartitionKey();
@ -149,12 +149,15 @@ MergeTreeData::MergeTreeData(
min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING;
}
auto path_exists = Poco::File(full_path).exists();
/// Creating directories, if not exist.
Poco::File(full_path).createDirectories();
Poco::File(full_path + "detached").createDirectory();
String version_file_path = full_path + "format_version.txt";
if (!attach)
// When data path not exists, ignore the format_version check
if (!attach || !path_exists)
{
format_version = min_format_version;
WriteBufferFromFile buf(version_file_path);

View File

@ -19,6 +19,14 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int BAD_TYPE_OF_FIELD;
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
}
String Range::toString() const
{
std::stringstream str;
@ -464,8 +472,8 @@ void PKCondition::getPKTuplePositionMapping(
index_mapping.tuple_index = tuple_index;
DataTypePtr data_type;
if (isPrimaryKeyPossiblyWrappedByMonotonicFunctions(
node, context, index_mapping.pk_index,
data_type, index_mapping.functions))
node, context, index_mapping.pk_index,
data_type, index_mapping.functions))
{
indexes_mapping.push_back(index_mapping);
if (out_primary_key_column_num < index_mapping.pk_index)
@ -475,7 +483,8 @@ void PKCondition::getPKTuplePositionMapping(
}
}
// Try to prepare PKTuplePositionMapping for tuples from IN expression.
/// Try to prepare PKTuplePositionMapping for tuples from IN expression.
bool PKCondition::isTupleIndexable(
const ASTPtr & node,
const Context & context,
@ -484,25 +493,41 @@ bool PKCondition::isTupleIndexable(
size_t & out_primary_key_column_num)
{
out_primary_key_column_num = 0;
const ASTFunction * node_tuple = typeid_cast<const ASTFunction *>(node.get());
std::vector<MergeTreeSetIndex::PKTuplePositionMapping> indexes_mapping;
size_t num_key_columns = prepared_set->getDataTypes().size();
if (num_key_columns == 0)
{
/// Empty set. It is "indexable" in a sense, that it implies that condition is always false (or true for NOT IN).
out.set_index = std::make_shared<MergeTreeSetIndex>(prepared_set->getSetElements(), std::move(indexes_mapping));
return true;
}
const ASTFunction * node_tuple = typeid_cast<const ASTFunction *>(node.get());
if (node_tuple && node_tuple->name == "tuple")
{
if (num_key_columns != node_tuple->arguments->children.size())
{
std::stringstream message;
message << "Number of columns in section IN doesn't match. "
<< node_tuple->arguments->children.size() << " at left, " << num_key_columns << " at right.";
throw Exception(message.str(), ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH);
}
size_t current_tuple_index = 0;
for (const auto & arg : node_tuple->arguments->children)
{
getPKTuplePositionMapping(arg, context, indexes_mapping, current_tuple_index++, out_primary_key_column_num);
getPKTuplePositionMapping(arg, context, indexes_mapping, current_tuple_index, out_primary_key_column_num);
++current_tuple_index;
}
}
else
{
getPKTuplePositionMapping(node, context, indexes_mapping, 0, out_primary_key_column_num);
getPKTuplePositionMapping(node, context, indexes_mapping, 0, out_primary_key_column_num);
}
if (indexes_mapping.empty())
{
return false;
}
out.set_index = std::make_shared<MergeTreeSetIndex>(
prepared_set->getSetElements(), std::move(indexes_mapping));
@ -510,6 +535,7 @@ bool PKCondition::isTupleIndexable(
return true;
}
bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctions(
const ASTPtr & node,
const Context & context,
@ -1002,14 +1028,10 @@ bool PKCondition::mayBeTrueInRangeImpl(const std::vector<Range> & key_ranges, co
{
rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(key_ranges, data_types));
if (element.function == RPNElement::FUNCTION_NOT_IN_SET)
{
rpn_stack.back() = !rpn_stack.back();
}
}
else
{
throw Exception("Set for IN is not created yet!", ErrorCodes::LOGICAL_ERROR);
}
throw Exception("Set for IN is not created yet", ErrorCodes::LOGICAL_ERROR);
}
else if (element.function == RPNElement::FUNCTION_NOT)
{

View File

@ -0,0 +1,42 @@
#include "ReplicatedMergeTreeAddress.h"
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
namespace DB
{
void ReplicatedMergeTreeAddress::writeText(WriteBuffer & out) const
{
out
<< "host: " << escape << host << '\n'
<< "port: " << replication_port << '\n'
<< "tcp_port: " << queries_port << '\n'
<< "database: " << escape << database << '\n'
<< "table: " << escape << table << '\n';
}
void ReplicatedMergeTreeAddress::readText(ReadBuffer & in)
{
in
>> "host: " >> escape >> host >> "\n"
>> "port: " >> replication_port >> "\n"
>> "tcp_port: " >> queries_port >> "\n"
>> "database: " >> escape >> database >> "\n"
>> "table: " >> escape >> table >> "\n";
}
String ReplicatedMergeTreeAddress::toString() const
{
WriteBufferFromOwnString out;
writeText(out);
return out.str();
}
void ReplicatedMergeTreeAddress::fromString(const String & str)
{
ReadBufferFromString in(str);
readText(in);
}
}

View File

@ -1,8 +1,7 @@
#pragma once
#include <Core/Types.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
namespace DB
@ -18,44 +17,19 @@ struct ReplicatedMergeTreeAddress
String database;
String table;
ReplicatedMergeTreeAddress() {}
ReplicatedMergeTreeAddress(const String & str)
ReplicatedMergeTreeAddress() = default;
explicit ReplicatedMergeTreeAddress(const String & str)
{
fromString(str);
}
void writeText(WriteBuffer & out) const
{
out
<< "host: " << escape << host << '\n'
<< "port: " << replication_port << '\n'
<< "tcp_port: " << queries_port << '\n'
<< "database: " << escape << database << '\n'
<< "table: " << escape << table << '\n';
}
void writeText(WriteBuffer & out) const;
void readText(ReadBuffer & in)
{
in
>> "host: " >> escape >> host >> "\n"
>> "port: " >> replication_port >> "\n"
>> "tcp_port: " >> queries_port >> "\n"
>> "database: " >> escape >> database >> "\n"
>> "table: " >> escape >> table >> "\n";
}
void readText(ReadBuffer & in);
String toString() const
{
WriteBufferFromOwnString out;
writeText(out);
return out.str();
}
String toString() const;
void fromString(const String & str)
{
ReadBufferFromString in(str);
readText(in);
}
void fromString(const String & str);
};
}

View File

@ -292,16 +292,10 @@ void ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart()
void ReplicatedMergeTreeRestartingThread::activateReplica()
{
auto host_port = storage.context.getInterserverIOAddress();
auto zookeeper = storage.getZooKeeper();
/// How other replicas can access this.
ReplicatedMergeTreeAddress address;
address.host = host_port.first;
address.replication_port = host_port.second;
address.queries_port = storage.context.getTCPPort();
address.database = storage.database_name;
address.table = storage.table_name;
/// How other replicas can access this one.
ReplicatedMergeTreeAddress address = storage.getReplicatedMergeTreeAddress();
String is_active_path = storage.replica_path + "/is_active";

View File

@ -20,6 +20,8 @@
#include <fcntl.h>
#include <Poco/Path.h>
#include <Poco/File.h>
namespace DB
{
@ -32,6 +34,7 @@ namespace ErrorCodes
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;
};
@ -41,10 +44,22 @@ static std::string getTablePath(const std::string & db_dir_path, const std::stri
return db_dir_path + escapeForFileName(table_name) + "/data." + escapeForFileName(format_name);
}
static void checkCreationIsAllowed(Context & context_global)
/// Both db_dir_path and table_path must be converted to absolute paths (in particular, path cannot contain '..').
static void checkCreationIsAllowed(Context & context_global, const std::string & db_dir_path, const std::string & table_path, int table_fd)
{
if (context_global.getApplicationType() == Context::ApplicationType::SERVER)
throw Exception("Using file descriptor or user specified path as source of storage isn't allowed for server daemons", ErrorCodes::DATABASE_ACCESS_DENIED);
if (context_global.getApplicationType() != Context::ApplicationType::SERVER)
return;
if (table_fd >= 0)
throw Exception("Using file descriptor as source of storage isn't allowed for server daemons", ErrorCodes::DATABASE_ACCESS_DENIED);
else if (!startsWith(table_path, db_dir_path))
throw Exception("Part path " + table_path + " is not inside " + db_dir_path, ErrorCodes::DATABASE_ACCESS_DENIED);
Poco::File table_path_poco_file = Poco::File(table_path);
if (!table_path_poco_file.exists())
throw Exception("File " + table_path + " is not exist", ErrorCodes::FILE_DOESNT_EXIST);
else if (table_path_poco_file.isDirectory())
throw Exception("File " + table_path + " must not be a directory", ErrorCodes::INCORRECT_FILE_NAME);
}
@ -65,8 +80,12 @@ StorageFile::StorageFile(
if (!table_path_.empty()) /// Is user's file
{
checkCreationIsAllowed(context_global);
path = Poco::Path(table_path_).absolute().toString();
Poco::Path poco_path = Poco::Path(table_path_);
if (poco_path.isRelative())
poco_path = Poco::Path(db_dir_path, poco_path);
path = poco_path.absolute().toString();
checkCreationIsAllowed(context_global, db_dir_path, path, table_fd);
is_db_table = false;
}
else /// Is DB's file
@ -81,7 +100,8 @@ StorageFile::StorageFile(
}
else /// Will use FD
{
checkCreationIsAllowed(context_global);
checkCreationIsAllowed(context_global, db_dir_path, path, table_fd);
is_db_table = false;
use_table_fd = true;
@ -280,7 +300,7 @@ void registerStorageFile(StorageFactory & factory)
{
/// Will use FD if engine_args[1] is int literal or identifier with std* name
if (ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(engine_args[1].get()))
if (const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(engine_args[1].get()))
{
if (identifier->name == "stdin")
source_fd = STDIN_FILENO;
@ -292,23 +312,22 @@ void registerStorageFile(StorageFactory & factory)
throw Exception("Unknown identifier '" + identifier->name + "' in second arg of File storage constructor",
ErrorCodes::UNKNOWN_IDENTIFIER);
}
if (const ASTLiteral * literal = typeid_cast<const ASTLiteral *>(engine_args[1].get()))
else if (const ASTLiteral * literal = typeid_cast<const ASTLiteral *>(engine_args[1].get()))
{
auto type = literal->value.getType();
if (type == Field::Types::Int64)
source_fd = static_cast<int>(literal->value.get<Int64>());
else if (type == Field::Types::UInt64)
source_fd = static_cast<int>(literal->value.get<UInt64>());
else if (type == Field::Types::String)
source_path = literal->value.get<String>();
}
engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], args.local_context);
source_path = static_cast<const ASTLiteral &>(*engine_args[1]).value.safeGet<String>();
}
return StorageFile::create(
source_path, source_fd,
args.data_path, args.table_name, format_name, args.columns,
args.data_path,
args.table_name, format_name, args.columns,
args.context);
});
}

View File

@ -3004,6 +3004,10 @@ void StorageReplicatedMergeTree::rename(const String & new_path_to_db, const Str
table_name = new_table_name;
full_path = new_full_path;
/// Update table name in zookeeper
auto zookeeper = getZooKeeper();
zookeeper->set(replica_path + "/host", getReplicatedMergeTreeAddress().toString());
/// TODO: You can update names of loggers.
}
@ -3270,14 +3274,14 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query
else
throw Exception("Can't proxy this query. Unsupported query type", ErrorCodes::NOT_IMPLEMENTED);
/// NOTE Works only if there is access from the default user without a password. You can fix it by adding a parameter to the server config.
/// Query send with current user credentials
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context.getSettingsRef());
Connection connection(
leader_address.host,
leader_address.queries_port,
leader_address.database,
"", "", timeouts, "ClickHouse replica");
context.getClientInfo().current_user, context.getClientInfo().current_password, timeouts, "ClickHouse replica");
RemoteBlockInputStream stream(connection, formattedAST(new_query), {}, context, &settings);
NullBlockOutputStream output({});
@ -3766,4 +3770,17 @@ void StorageReplicatedMergeTree::clearBlocksInPartition(
LOG_TRACE(log, "Deleted " << to_delete_futures.size() << " deduplication block IDs in partition ID " << partition_id);
}
ReplicatedMergeTreeAddress StorageReplicatedMergeTree::getReplicatedMergeTreeAddress() const
{
auto host_port = context.getInterserverIOAddress();
ReplicatedMergeTreeAddress res;
res.host = host_port.first;
res.replication_port = host_port.second;
res.queries_port = context.getTCPPort();
res.database = database_name;
res.table = table_name;
return res;
}
}

View File

@ -17,6 +17,7 @@
#include <Storages/MergeTree/AbandonableLockInZooKeeper.h>
#include <Storages/MergeTree/BackgroundProcessingPool.h>
#include <Storages/MergeTree/DataPartsExchange.h>
#include <Storages/MergeTree/ReplicatedMergeTreeAddress.h>
#include <DataTypes/DataTypesNumber.h>
#include <Common/randomSeed.h>
#include <Common/ZooKeeper/ZooKeeper.h>
@ -451,6 +452,9 @@ private:
void clearBlocksInPartition(
zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num);
/// Info about how other replicas can access this one.
ReplicatedMergeTreeAddress getReplicatedMergeTreeAddress() const;
protected:
/** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table.
*/

View File

@ -66,7 +66,7 @@ ColumnsDescription getStructureOfRemoteTable(
auto kind = columnDefaultKindFromString(kind_name);
String expr_str = (*default_expr)[i].get<const String &>();
ASTPtr expr = parseQuery(expr_parser, expr_str.data(), expr_str.data() + expr_str.size(), "default expression");
ASTPtr expr = parseQuery(expr_parser, expr_str.data(), expr_str.data() + expr_str.size(), "default expression", 0);
res.defaults.emplace(column_name, ColumnDefault{kind, expr});
if (ColumnDefaultKind::Default == kind)

View File

@ -0,0 +1,78 @@
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/TableFunctionFile.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Common/Exception.h>
#include <Common/typeid_cast.h>
#include <Storages/StorageFile.h>
#include <DataTypes/DataTypeFactory.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <boost/algorithm/string.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int DATABASE_ACCESS_DENIED;
}
StoragePtr TableFunctionFile::executeImpl(const ASTPtr & ast_function, const Context & context) const
{
// Parse args
ASTs & args_func = typeid_cast<ASTFunction &>(*ast_function).children;
if (args_func.size() != 1)
throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::LOGICAL_ERROR);
ASTs & args = typeid_cast<ASTExpressionList &>(*args_func.at(0)).children;
if (args.size() != 3)
throw Exception("Table function '" + getName() + "' requires exactly 3 arguments: path, format and structure.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (size_t i = 0; i < 3; ++i)
args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(args[i], context);
std::string path = static_cast<const ASTLiteral &>(*args[0]).value.safeGet<String>();
std::string format = static_cast<const ASTLiteral &>(*args[1]).value.safeGet<String>();
std::string structure = static_cast<const ASTLiteral &>(*args[2]).value.safeGet<String>();
// Create sample block
std::vector<std::string> structure_vals;
boost::split(structure_vals, structure, boost::algorithm::is_any_of(" ,"), boost::algorithm::token_compress_on);
if (structure_vals.size() % 2 != 0)
throw Exception("Odd number of elements in section structure: must be a list of name type pairs", ErrorCodes::LOGICAL_ERROR);
Block sample_block;
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
for (size_t i = 0, size = structure_vals.size(); i < size; i += 2)
{
ColumnWithTypeAndName column;
column.name = structure_vals[i];
column.type = data_type_factory.get(structure_vals[i + 1]);
column.column = column.type->createColumn();
sample_block.insert(std::move(column));
}
// Create table
StoragePtr storage = StorageFile::create(
path, -1, context.getUserFilesPath(), getName(), format,
ColumnsDescription{sample_block.getNamesAndTypesList()}, const_cast<Context &>(context));
storage->startup();
return storage;
}
void registerTableFunctionFile(TableFunctionFactory & factory)
{
factory.registerFunction<TableFunctionFile>();
}
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <TableFunctions/ITableFunction.h>
namespace DB
{
/* file(path, format, structure) - creates a temporary storage from file
*
*
* The file must be in the clickhouse data directory.
* The relative path begins with the clickhouse data directory.
*/
class TableFunctionFile : public ITableFunction
{
public:
static constexpr auto name = "file";
std::string getName() const override { return name; }
private:
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context) const override;
};
}

View File

@ -11,6 +11,8 @@ void registerTableFunctionRemote(TableFunctionFactory & factory);
void registerTableFunctionShardByHash(TableFunctionFactory & factory);
void registerTableFunctionNumbers(TableFunctionFactory & factory);
void registerTableFunctionCatBoostPool(TableFunctionFactory & factory);
void registerTableFunctionFile(TableFunctionFactory & factory);
#if Poco_SQLODBC_FOUND || Poco_DataODBC_FOUND
void registerTableFunctionODBC(TableFunctionFactory & factory);
#endif
@ -29,6 +31,7 @@ void registerTableFunctions()
registerTableFunctionShardByHash(factory);
registerTableFunctionNumbers(factory);
registerTableFunctionCatBoostPool(factory);
registerTableFunctionFile(factory);
#if Poco_SQLODBC_FOUND || Poco_DataODBC_FOUND
registerTableFunctionODBC(factory);

View File

@ -0,0 +1,16 @@
<yandex>
<users>
<restricted_user>
<password></password>
<profile>default</profile>
<quota>default</quota>
<networks>
<ip>::/0</ip>
</networks>
<allow_databases>
<database>db1</database>
</allow_databases>
</restricted_user>
</users>
</yandex>

View File

@ -315,6 +315,24 @@ def test_macro(started_cluster):
ddl_check_query(instance, "DROP TABLE IF EXISTS distr ON CLUSTER '{cluster}'")
ddl_check_query(instance, "DROP TABLE IF EXISTS tab ON CLUSTER '{cluster}'")
def test_allowed_databases(started_cluster):
instance = cluster.instances['ch2']
instance.query("CREATE DATABASE IF NOT EXISTS db1 ON CLUSTER cluster")
instance.query("CREATE DATABASE IF NOT EXISTS db2 ON CLUSTER cluster")
instance.query("CREATE TABLE db1.t1 ON CLUSTER cluster (i Int8) ENGINE = Memory", settings={"user" : "restricted_user"})
with pytest.raises(Exception):
instance.query("CREATE TABLE db2.t2 ON CLUSTER cluster (i Int8) ENGINE = Memory", settings={"user" : "restricted_user"})
with pytest.raises(Exception):
instance.query("CREATE TABLE t3 ON CLUSTER cluster (i Int8) ENGINE = Memory", settings={"user" : "restricted_user"})
with pytest.raises(Exception):
instance.query("DROP DATABASE db2 ON CLUSTER cluster", settings={"user" : "restricted_user"})
instance.query("DROP DATABASE db1 ON CLUSTER cluster", settings={"user" : "restricted_user"})
if __name__ == '__main__':
with contextmanager(started_cluster)() as cluster:
for name, instance in cluster.instances.items():

View File

@ -0,0 +1,51 @@
<test>
<name>url_hits</name>
<type>once</type>
<preconditions>
<table_exists>hits_100m_single</table_exists>
</preconditions>
<stop_conditions>
<all_of>
<total_time_ms>10000</total_time_ms>
</all_of>
<any_of>
<average_speed_not_changing_for_ms>5000</average_speed_not_changing_for_ms>
<total_time_ms>20000</total_time_ms>
</any_of>
</stop_conditions>
<main_metric>
<avg_rows_per_second/>
</main_metric>
<substitutions>
<substitution>
<name>func</name>
<values>
<value>protocol</value>
<value>domain</value>
<value>domainWithoutWWW</value>
<value>topLevelDomain</value>
<value>firstSignificantSubdomain</value>
<value>cutToFirstSignificantSubdomain</value>
<value>path</value>
<value>pathFull</value>
<value>queryString</value>
<value>fragment</value>
<value>queryStringAndFragment</value>
<value>extractURLParameters</value>
<value>extractURLParameterNames</value>
<value>URLHierarchy</value>
<value>URLPathHierarchy</value>
<value>decodeURLComponent</value>
<value>cutWWW</value>
<value>cutQueryString</value>
<value>cutQueryStringAndFragment</value>
</values>
</substitution>
</substitutions>
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore({func}(URL))</query>
</test>

View File

@ -10,5 +10,5 @@ SELECT (number AS n, n + 1, toString(n + 2), n + 3, n - 1) IN (1, 2, '3', 4, 0)
SELECT number, tuple FROM (SELECT 1 AS number, (2, 3) AS tuple) WHERE (number, tuple) IN (((1, (2, 3)), (4, (5, 6))));
SELECT number, tuple FROM (SELECT 2 AS number, (2, 3) AS tuple) WHERE (number, tuple) IN ((2, (2, 3)));
SELECT number, tuple FROM (SELECT 3 AS number, (2, 3) AS tuple) WHERE (number, tuple) IN (3, (2, 3));
SELECT number, tuple FROM (SELECT 4 AS number, (2, 3) AS tuple) WHERE (number, tuple) IN (SELECT (4, (2, 3)));
SELECT number, tuple FROM (SELECT 4 AS number, (2, 3) AS tuple) WHERE (number, tuple) IN (SELECT 4, (2, 3));
SELECT number, tuple FROM (SELECT 5 AS number, (2, 3) AS tuple) WHERE (number, tuple) IN (SELECT 5, (2, 3));

View File

@ -1,8 +1,8 @@
['a=b','c=d'] ['a=b','c=d','e=f'] ['a','c=d','e=f'] ['a=b','c=d','e=f','g=h'] ['a=b','c=d'] ['a=b','c=d','e','g=h'] ['a=b','c=d','e=f','g=h']
['a','c'] ['a','c','e'] ['a','c','e'] ['a','c','e','g'] ['a','c'] ['a','c','e','g'] ['a','c','e','g']
b d f d f h b d d h f h
http://yandex.ru/?c=d http://yandex.ru/?a=b http://yandex.ru/?a=b&c=d# http://yandex.ru/?a&c=d#e=f http://yandex.ru/?a#e=f http://yandex.ru/?a&c=d# http://yandex.ru/?a=b&c=d#e=f http://yandex.ru/?c=d#e http://yandex.ru/?a=b#e http://yandex.ru/?a=b&c=d#e http://yandex.ru/?a=b#e&g=h http://yandex.ru/?a=b&c=d#e&g=h http://yandex.ru/?a=b&c=d#e http://yandex.ru/?a=b&c=d#test?e=f&g=h http://yandex.ru/?a=b&c=d#test?g=h http://yandex.ru/?a=b&c=d#test?e=f
['a=b','c=d'] ['a=b','c=d','e=f'] ['a','c=d','e=f'] ['a=b','c=d','e=f','g=h'] ['a=b','c=d'] ['a=b','c=d','e','g=h'] ['a=b','c=d','e=f','g=h']
['a','c'] ['a','c','e'] ['a','c','e'] ['a','c','e','g'] ['a','c'] ['a','c','e','g'] ['a','c','e','g']
b d f d f h b d d h f h
http://yandex.ru/?c=d http://yandex.ru/?a=b http://yandex.ru/?a=b&c=d# http://yandex.ru/?a&c=d#e=f http://yandex.ru/?a#e=f http://yandex.ru/?a&c=d# http://yandex.ru/?a=b&c=d#e=f http://yandex.ru/?c=d#e http://yandex.ru/?a=b#e http://yandex.ru/?a=b&c=d#e http://yandex.ru/?a=b#e&g=h http://yandex.ru/?a=b&c=d#e&g=h http://yandex.ru/?a=b&c=d#e http://yandex.ru/?a=b&c=d#test?e=f&g=h http://yandex.ru/?a=b&c=d#test?g=h http://yandex.ru/?a=b&c=d#test?e=f
['a=b','c=d'] ['a=b','c=d','e=f'] ['a','c=d','e=f'] ['a=b','c=d','e=f','g=h'] ['a=b','c=d'] ['a=b','c=d','e','g=h'] ['a=b','c=d','e=f','g=h'] ['a=b','c=d'] ['a=b','c=d','e=f'] ['a','c=d','e=f'] ['a=b','c=d','e=f','g=h'] ['a=b','c=d'] ['a=b','c=d','e','g=h'] ['a=b','c=d','e=f','g=h']
['a','c'] ['a','c','e'] ['a','c','e'] ['a','c','e','g'] ['a','c'] ['a','c','e','g'] ['a','c','e','g'] ['a','c'] ['a','c','e'] ['a','c','e'] ['a','c','e','g'] ['a','c'] ['a','c','e','g'] ['a','c','e','g']
b d f d f h b d d h f h b d f d f h b d d h f h
http://yandex.ru/?c=d http://yandex.ru/?a=b http://yandex.ru/?a=b&c=d# http://yandex.ru/?a&c=d#e=f http://yandex.ru/?a#e=f http://yandex.ru/?a&c=d# http://yandex.ru/?a=b&c=d#e=f http://yandex.ru/?c=d#e http://yandex.ru/?a=b#e http://yandex.ru/?a=b&c=d#e http://yandex.ru/?a=b#e&g=h http://yandex.ru/?a=b&c=d#e&g=h http://yandex.ru/?a=b&c=d#e http://yandex.ru/?a=b&c=d#test?e=f&g=h http://yandex.ru/?a=b&c=d#test?g=h http://yandex.ru/?a=b&c=d#test?e=f //yandex.ru/?c=d //yandex.ru/?a=b //yandex.ru/?a=b&c=d# //yandex.ru/?a&c=d#e=f //yandex.ru/?a#e=f //yandex.ru/?a&c=d# //yandex.ru/?a=b&c=d#e=f //yandex.ru/?c=d#e //yandex.ru/?a=b#e //yandex.ru/?a=b&c=d#e //yandex.ru/?a=b#e&g=h //yandex.ru/?a=b&c=d#e&g=h //yandex.ru/?a=b&c=d#e //yandex.ru/?a=b&c=d#test?e=f&g=h //yandex.ru/?a=b&c=d#test?g=h //yandex.ru/?a=b&c=d#test?e=f
['a=b','c=d'] ['a=b','c=d','e=f'] ['a','c=d','e=f'] ['a=b','c=d','e=f','g=h'] ['a=b','c=d'] ['a=b','c=d','e','g=h'] ['a=b','c=d','e=f','g=h'] ['a=b','c=d'] ['a=b','c=d','e=f'] ['a','c=d','e=f'] ['a=b','c=d','e=f','g=h'] ['a=b','c=d'] ['a=b','c=d','e','g=h'] ['a=b','c=d','e=f','g=h']
['a','c'] ['a','c','e'] ['a','c','e'] ['a','c','e','g'] ['a','c'] ['a','c','e','g'] ['a','c','e','g'] ['a','c'] ['a','c','e'] ['a','c','e'] ['a','c','e','g'] ['a','c'] ['a','c','e','g'] ['a','c','e','g']
b d f d f h b d d h f h b d f d f h b d d h f h
http://yandex.ru/?c=d http://yandex.ru/?a=b http://yandex.ru/?a=b&c=d# http://yandex.ru/?a&c=d#e=f http://yandex.ru/?a#e=f http://yandex.ru/?a&c=d# http://yandex.ru/?a=b&c=d#e=f http://yandex.ru/?c=d#e http://yandex.ru/?a=b#e http://yandex.ru/?a=b&c=d#e http://yandex.ru/?a=b#e&g=h http://yandex.ru/?a=b&c=d#e&g=h http://yandex.ru/?a=b&c=d#e http://yandex.ru/?a=b&c=d#test?e=f&g=h http://yandex.ru/?a=b&c=d#test?g=h http://yandex.ru/?a=b&c=d#test?e=f //yandex.ru/?c=d //yandex.ru/?a=b //yandex.ru/?a=b&c=d# //yandex.ru/?a&c=d#e=f //yandex.ru/?a#e=f //yandex.ru/?a&c=d# //yandex.ru/?a=b&c=d#e=f //yandex.ru/?c=d#e //yandex.ru/?a=b#e //yandex.ru/?a=b&c=d#e //yandex.ru/?a=b#e&g=h //yandex.ru/?a=b&c=d#e&g=h //yandex.ru/?a=b&c=d#e //yandex.ru/?a=b&c=d#test?e=f&g=h //yandex.ru/?a=b&c=d#test?g=h //yandex.ru/?a=b&c=d#test?e=f

View File

@ -5,7 +5,14 @@ SELECT
extractURLParameters('http://yandex.ru/?a=b&c=d#e=f&g=h'),
extractURLParameters('http://yandex.ru/?a=b&c=d#e'),
extractURLParameters('http://yandex.ru/?a=b&c=d#e&g=h'),
extractURLParameters('http://yandex.ru/?a=b&c=d#test?e=f&g=h');
extractURLParameters('http://yandex.ru/?a=b&c=d#test?e=f&g=h'),
extractURLParameters('//yandex.ru/?a=b&c=d'),
extractURLParameters('//yandex.ru/?a=b&c=d#e=f'),
extractURLParameters('//yandex.ru/?a&c=d#e=f'),
extractURLParameters('//yandex.ru/?a=b&c=d#e=f&g=h'),
extractURLParameters('//yandex.ru/?a=b&c=d#e'),
extractURLParameters('//yandex.ru/?a=b&c=d#e&g=h'),
extractURLParameters('//yandex.ru/?a=b&c=d#test?e=f&g=h');
SELECT
extractURLParameterNames('http://yandex.ru/?a=b&c=d'),
@ -14,7 +21,14 @@ SELECT
extractURLParameterNames('http://yandex.ru/?a=b&c=d#e=f&g=h'),
extractURLParameterNames('http://yandex.ru/?a=b&c=d#e'),
extractURLParameterNames('http://yandex.ru/?a=b&c=d#e&g=h'),
extractURLParameterNames('http://yandex.ru/?a=b&c=d#test?e=f&g=h');
extractURLParameterNames('http://yandex.ru/?a=b&c=d#test?e=f&g=h'),
extractURLParameterNames('//yandex.ru/?a=b&c=d'),
extractURLParameterNames('//yandex.ru/?a=b&c=d#e=f'),
extractURLParameterNames('//yandex.ru/?a&c=d#e=f'),
extractURLParameterNames('//yandex.ru/?a=b&c=d#e=f&g=h'),
extractURLParameterNames('//yandex.ru/?a=b&c=d#e'),
extractURLParameterNames('//yandex.ru/?a=b&c=d#e&g=h'),
extractURLParameterNames('//yandex.ru/?a=b&c=d#test?e=f&g=h');
SELECT
extractURLParameter('http://yandex.ru/?a=b&c=d', 'a'),
@ -32,7 +46,23 @@ SELECT
extractURLParameter('http://yandex.ru/?a=b&c=d#e&g=h', 'g'),
extractURLParameter('http://yandex.ru/?a=b&c=d#test?e=f&g=h', 'test'),
extractURLParameter('http://yandex.ru/?a=b&c=d#test?e=f&g=h', 'e'),
extractURLParameter('http://yandex.ru/?a=b&c=d#test?e=f&g=h', 'g');
extractURLParameter('http://yandex.ru/?a=b&c=d#test?e=f&g=h', 'g'),
extractURLParameter('//yandex.ru/?a=b&c=d', 'a'),
extractURLParameter('//yandex.ru/?a=b&c=d', 'c'),
extractURLParameter('//yandex.ru/?a=b&c=d#e=f', 'e'),
extractURLParameter('//yandex.ru/?a&c=d#e=f', 'a'),
extractURLParameter('//yandex.ru/?a&c=d#e=f', 'c'),
extractURLParameter('//yandex.ru/?a&c=d#e=f', 'e'),
extractURLParameter('//yandex.ru/?a=b&c=d#e=f&g=h', 'g'),
extractURLParameter('//yandex.ru/?a=b&c=d#e', 'a'),
extractURLParameter('//yandex.ru/?a=b&c=d#e', 'c'),
extractURLParameter('//yandex.ru/?a=b&c=d#e', 'e'),
extractURLParameter('//yandex.ru/?a=b&c=d#e&g=h', 'c'),
extractURLParameter('//yandex.ru/?a=b&c=d#e&g=h', 'e'),
extractURLParameter('//yandex.ru/?a=b&c=d#e&g=h', 'g'),
extractURLParameter('//yandex.ru/?a=b&c=d#test?e=f&g=h', 'test'),
extractURLParameter('//yandex.ru/?a=b&c=d#test?e=f&g=h', 'e'),
extractURLParameter('//yandex.ru/?a=b&c=d#test?e=f&g=h', 'g');
SELECT
cutURLParameter('http://yandex.ru/?a=b&c=d', 'a'),
@ -50,7 +80,23 @@ SELECT
cutURLParameter('http://yandex.ru/?a=b&c=d#e&g=h', 'g'),
cutURLParameter('http://yandex.ru/?a=b&c=d#test?e=f&g=h', 'test'),
cutURLParameter('http://yandex.ru/?a=b&c=d#test?e=f&g=h', 'e'),
cutURLParameter('http://yandex.ru/?a=b&c=d#test?e=f&g=h', 'g');
cutURLParameter('http://yandex.ru/?a=b&c=d#test?e=f&g=h', 'g'),
cutURLParameter('//yandex.ru/?a=b&c=d', 'a'),
cutURLParameter('//yandex.ru/?a=b&c=d', 'c'),
cutURLParameter('//yandex.ru/?a=b&c=d#e=f', 'e'),
cutURLParameter('//yandex.ru/?a&c=d#e=f', 'a'),
cutURLParameter('//yandex.ru/?a&c=d#e=f', 'c'),
cutURLParameter('//yandex.ru/?a&c=d#e=f', 'e'),
cutURLParameter('//yandex.ru/?a=b&c=d#e=f&g=h', 'g'),
cutURLParameter('//yandex.ru/?a=b&c=d#e', 'a'),
cutURLParameter('//yandex.ru/?a=b&c=d#e', 'c'),
cutURLParameter('//yandex.ru/?a=b&c=d#e', 'e'),
cutURLParameter('//yandex.ru/?a=b&c=d#e&g=h', 'c'),
cutURLParameter('//yandex.ru/?a=b&c=d#e&g=h', 'e'),
cutURLParameter('//yandex.ru/?a=b&c=d#e&g=h', 'g'),
cutURLParameter('//yandex.ru/?a=b&c=d#test?e=f&g=h', 'test'),
cutURLParameter('//yandex.ru/?a=b&c=d#test?e=f&g=h', 'e'),
cutURLParameter('//yandex.ru/?a=b&c=d#test?e=f&g=h', 'g');
SELECT
@ -60,7 +106,14 @@ SELECT
extractURLParameters(materialize('http://yandex.ru/?a=b&c=d#e=f&g=h')),
extractURLParameters(materialize('http://yandex.ru/?a=b&c=d#e')),
extractURLParameters(materialize('http://yandex.ru/?a=b&c=d#e&g=h')),
extractURLParameters(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'));
extractURLParameters(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h')),
extractURLParameters(materialize('//yandex.ru/?a=b&c=d')),
extractURLParameters(materialize('//yandex.ru/?a=b&c=d#e=f')),
extractURLParameters(materialize('//yandex.ru/?a&c=d#e=f')),
extractURLParameters(materialize('//yandex.ru/?a=b&c=d#e=f&g=h')),
extractURLParameters(materialize('//yandex.ru/?a=b&c=d#e')),
extractURLParameters(materialize('//yandex.ru/?a=b&c=d#e&g=h')),
extractURLParameters(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'));
SELECT
extractURLParameterNames(materialize('http://yandex.ru/?a=b&c=d')),
@ -69,7 +122,14 @@ SELECT
extractURLParameterNames(materialize('http://yandex.ru/?a=b&c=d#e=f&g=h')),
extractURLParameterNames(materialize('http://yandex.ru/?a=b&c=d#e')),
extractURLParameterNames(materialize('http://yandex.ru/?a=b&c=d#e&g=h')),
extractURLParameterNames(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'));
extractURLParameterNames(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h')),
extractURLParameterNames(materialize('//yandex.ru/?a=b&c=d')),
extractURLParameterNames(materialize('//yandex.ru/?a=b&c=d#e=f')),
extractURLParameterNames(materialize('//yandex.ru/?a&c=d#e=f')),
extractURLParameterNames(materialize('//yandex.ru/?a=b&c=d#e=f&g=h')),
extractURLParameterNames(materialize('//yandex.ru/?a=b&c=d#e')),
extractURLParameterNames(materialize('//yandex.ru/?a=b&c=d#e&g=h')),
extractURLParameterNames(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'));
SELECT
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d'), 'a'),
@ -87,7 +147,23 @@ SELECT
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#e&g=h'), 'g'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'), 'test'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'), 'e'),
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'), 'g');
extractURLParameter(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'), 'g'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d'), 'a'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d'), 'c'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e=f'), 'e'),
extractURLParameter(materialize('//yandex.ru/?a&c=d#e=f'), 'a'),
extractURLParameter(materialize('//yandex.ru/?a&c=d#e=f'), 'c'),
extractURLParameter(materialize('//yandex.ru/?a&c=d#e=f'), 'e'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e=f&g=h'), 'g'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e'), 'a'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e'), 'c'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e'), 'e'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e&g=h'), 'c'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e&g=h'), 'e'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#e&g=h'), 'g'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'), 'test'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'), 'e'),
extractURLParameter(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'), 'g');
SELECT
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d'), 'a'),
@ -105,4 +181,20 @@ SELECT
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#e&g=h'), 'g'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'), 'test'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'), 'e'),
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'), 'g');
cutURLParameter(materialize('http://yandex.ru/?a=b&c=d#test?e=f&g=h'), 'g'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d'), 'a'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d'), 'c'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e=f'), 'e'),
cutURLParameter(materialize('//yandex.ru/?a&c=d#e=f'), 'a'),
cutURLParameter(materialize('//yandex.ru/?a&c=d#e=f'), 'c'),
cutURLParameter(materialize('//yandex.ru/?a&c=d#e=f'), 'e'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e=f&g=h'), 'g'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e'), 'a'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e'), 'c'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e'), 'e'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e&g=h'), 'c'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e&g=h'), 'e'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#e&g=h'), 'g'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'), 'test'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'), 'e'),
cutURLParameter(materialize('//yandex.ru/?a=b&c=d#test?e=f&g=h'), 'g');

View File

@ -1,3 +1,3 @@
canada congo net-domena
yandex yandex yandex yandex яндекс яндекс
yandex yandex yandex yandex яндекс яндекс yandex
canada hello hello hello hello hello canada canada

View File

@ -10,7 +10,8 @@ SELECT
firstSignificantSubdomain('ftp://www.yandex.co.uk/news.html'),
firstSignificantSubdomain('ftp://yandex.co.yandex'),
firstSignificantSubdomain('http://ввв.яндекс.org.рф'),
firstSignificantSubdomain('https://api.www3.static.dev.ввв.яндекс.рф');
firstSignificantSubdomain('https://api.www3.static.dev.ввв.яндекс.рф'),
firstSignificantSubdomain('//www.yandex.com.tr/news.html');
SELECT
firstSignificantSubdomain('http://hello.canada.c'),

View File

@ -1,18 +1,94 @@
====SCHEMA====
http
https
svn+ssh
http
====HOST====
www.example.com
www.example.com
127.0.0.1
www.example.com
www.example.com
example.com
example.com
====DOMAIN====
com
ru
ru
com
====PATH====
П
%D%9
/?query=hello world+foo+bar
/?query=hello world+foo+bar
/?query=hello world+foo+bar
/?query=hello world+foo+bar
/a/b/c
/a/b/c
/a/b/c
/a/b/c
====QUERY STRING====
query=hello world+foo+bar
query=hello world+foo+bar
query=hello world+foo+bar
query=hello world+foo+bar
====FRAGMENT====
a=b
a=b
a=b
====QUERY STRING AND FRAGMENT====
query=hello world+foo+bar
query=hello world+foo+bar#a=b
query=hello world+foo+bar#a=b
query=hello world+foo+bar#a=b
====CUT TO FIRST SIGNIFICANT SUBDOMAIN====
example.com
example.com
example.com
example.com
example.com
example.com
example.com
====CUT WWW====
http://example.com
http://example.com:1234
http://example.com/a/b/c
http://example.com/a/b/c?a=b
http://example.com/a/b/c?a=b#d=f
http://paul@example.com/a/b/c?a=b#d=f
//paul@example.com/a/b/c?a=b#d=f
====CUT QUERY STRING====
http://www.example.com
http://www.example.com:1234
http://www.example.com/a/b/c
http://www.example.com/a/b/c
http://www.example.com/a/b/c#d=f
http://paul@www.example.com/a/b/c#d=f
//paul@www.example.com/a/b/c#d=f
====CUT FRAGMENT====
http://www.example.com
http://www.example.com:1234
http://www.example.com/a/b/c
http://www.example.com/a/b/c?a=b
http://www.example.com/a/b/c?a=b
http://paul@www.example.com/a/b/c?a=b
//paul@www.example.com/a/b/c?a=b
====CUT QUERY STRING AND FRAGMENT====
http://www.example.com
http://www.example.com:1234
http://www.example.com/a/b/c
http://www.example.com/a/b/c
http://www.example.com/a/b/c
http://paul@www.example.com/a/b/c
//paul@www.example.com/a/b/c

View File

@ -1,21 +1,107 @@
SELECT '====SCHEMA====';
SELECT protocol('http://example.com') AS Scheme;
SELECT protocol('https://example.com/') AS Scheme;
SELECT protocol('svn+ssh://example.com?q=hello%20world') AS Scheme;
SELECT protocol('ftp!://example.com/') AS Scheme;
SELECT protocol('http://127.0.0.1:443/') AS Scheme;
SELECT protocol('//127.0.0.1:443/') AS Scheme;
SELECT '====HOST====';
SELECT domain('http://paul@www.example.com:80/') AS Host;
SELECT domain('http:/paul/example/com') AS Host;
SELECT domain('http://www.example.com?q=4') AS Host;
SELECT domain('http://127.0.0.1:443/') AS Host;
SELECT domain('//www.example.com') AS Host;
SELECT domain('//paul@www.example.com') AS Host;
SELECT domainWithoutWWW('//paul@www.example.com') AS Host;
SELECT domainWithoutWWW('http://paul@www.example.com:80/') AS Host;
SELECT '====DOMAIN====';
SELECT topLevelDomain('http://paul@www.example.com:80/') AS Domain;
SELECT topLevelDomain('http://127.0.0.1:443/') AS Domain;
SELECT topLevelDomain('svn+ssh://example.ru?q=hello%20world') AS Domain;
SELECT topLevelDomain('svn+ssh://example.ru.?q=hello%20world') AS Domain;
SELECT topLevelDomain('//www.example.com') AS Domain;
SELECT '====PATH====';
SELECT decodeURLComponent('%D0%9F');
SELECT decodeURLComponent('%D%9');
SELECT decodeURLComponent(pathFull('//127.0.0.1/?query=hello%20world+foo%2Bbar')) AS Path;
SELECT decodeURLComponent(pathFull('http://127.0.0.1/?query=hello%20world+foo%2Bbar')) AS Path;
SELECT decodeURLComponent(materialize(pathFull('http://127.0.0.1/?query=hello%20world+foo%2Bbar'))) AS Path;
SELECT decodeURLComponent(materialize(pathFull('//127.0.0.1/?query=hello%20world+foo%2Bbar'))) AS Path;
SELECT path('http://127.0.0.1') AS Path;
SELECT path('http://127.0.0.1/a/b/c') AS Path;
SELECT path('http://127.0.0.1:443/a/b/c') AS Path;
SELECT path('http://paul@127.0.0.1:443/a/b/c') AS Path;
SELECT path('//paul@127.0.0.1:443/a/b/c') AS Path;
SELECT '====QUERY STRING====';
SELECT decodeURLComponent(queryString('http://127.0.0.1/'));
SELECT decodeURLComponent(queryString('http://127.0.0.1/?'));
SELECT decodeURLComponent(queryString('http://127.0.0.1/?query=hello%20world+foo%2Bbar'));
SELECT decodeURLComponent(queryString('http://127.0.0.1:443/?query=hello%20world+foo%2Bbar'));
SELECT decodeURLComponent(queryString('http://paul@127.0.0.1:443/?query=hello%20world+foo%2Bbar'));
SELECT decodeURLComponent(queryString('//paul@127.0.0.1:443/?query=hello%20world+foo%2Bbar'));
SELECT '====FRAGMENT====';
SELECT decodeURLComponent(fragment('http://127.0.0.1/?query=hello%20world+foo%2Bbar'));
SELECT decodeURLComponent(fragment('http://127.0.0.1/?query=hello%20world+foo%2Bbar#'));
SELECT decodeURLComponent(fragment('http://127.0.0.1/?query=hello%20world+foo%2Bbar#a=b'));
SELECT decodeURLComponent(fragment('http://paul@127.0.0.1/?query=hello%20world+foo%2Bbar#a=b'));
SELECT decodeURLComponent(fragment('//paul@127.0.0.1/?query=hello%20world+foo%2Bbar#a=b'));
SELECT '====QUERY STRING AND FRAGMENT====';
SELECT decodeURLComponent(queryStringAndFragment('http://127.0.0.1/'));
SELECT decodeURLComponent(queryStringAndFragment('http://127.0.0.1/?'));
SELECT decodeURLComponent(queryStringAndFragment('http://127.0.0.1/?query=hello%20world+foo%2Bbar'));
SELECT decodeURLComponent(queryStringAndFragment('http://127.0.0.1/?query=hello%20world+foo%2Bbar#a=b'));
SELECT decodeURLComponent(queryStringAndFragment('http://paul@127.0.0.1/?query=hello%20world+foo%2Bbar#a=b'));
SELECT decodeURLComponent(queryStringAndFragment('//paul@127.0.0.1/?query=hello%20world+foo%2Bbar#a=b'));
SELECT '====CUT TO FIRST SIGNIFICANT SUBDOMAIN====';
SELECT cutToFirstSignificantSubdomain('http://www.example.com');
SELECT cutToFirstSignificantSubdomain('http://www.example.com:1234');
SELECT cutToFirstSignificantSubdomain('http://www.example.com/a/b/c');
SELECT cutToFirstSignificantSubdomain('http://www.example.com/a/b/c?a=b');
SELECT cutToFirstSignificantSubdomain('http://www.example.com/a/b/c?a=b#d=f');
SELECT cutToFirstSignificantSubdomain('http://paul@www.example.com/a/b/c?a=b#d=f');
SELECT cutToFirstSignificantSubdomain('//paul@www.example.com/a/b/c?a=b#d=f');
SELECT '====CUT WWW====';
SELECT cutWWW('http://www.example.com');
SELECT cutWWW('http://www.example.com:1234');
SELECT cutWWW('http://www.example.com/a/b/c');
SELECT cutWWW('http://www.example.com/a/b/c?a=b');
SELECT cutWWW('http://www.example.com/a/b/c?a=b#d=f');
SELECT cutWWW('http://paul@www.example.com/a/b/c?a=b#d=f');
SELECT cutWWW('//paul@www.example.com/a/b/c?a=b#d=f');
SELECT '====CUT QUERY STRING====';
SELECT cutQueryString('http://www.example.com');
SELECT cutQueryString('http://www.example.com:1234');
SELECT cutQueryString('http://www.example.com/a/b/c');
SELECT cutQueryString('http://www.example.com/a/b/c?a=b');
SELECT cutQueryString('http://www.example.com/a/b/c?a=b#d=f');
SELECT cutQueryString('http://paul@www.example.com/a/b/c?a=b#d=f');
SELECT cutQueryString('//paul@www.example.com/a/b/c?a=b#d=f');
SELECT '====CUT FRAGMENT====';
SELECT cutFragment('http://www.example.com');
SELECT cutFragment('http://www.example.com:1234');
SELECT cutFragment('http://www.example.com/a/b/c');
SELECT cutFragment('http://www.example.com/a/b/c?a=b');
SELECT cutFragment('http://www.example.com/a/b/c?a=b#d=f');
SELECT cutFragment('http://paul@www.example.com/a/b/c?a=b#d=f');
SELECT cutFragment('//paul@www.example.com/a/b/c?a=b#d=f');
SELECT '====CUT QUERY STRING AND FRAGMENT====';
SELECT cutQueryStringAndFragment('http://www.example.com');
SELECT cutQueryStringAndFragment('http://www.example.com:1234');
SELECT cutQueryStringAndFragment('http://www.example.com/a/b/c');
SELECT cutQueryStringAndFragment('http://www.example.com/a/b/c?a=b');
SELECT cutQueryStringAndFragment('http://www.example.com/a/b/c?a=b#d=f');
SELECT cutQueryStringAndFragment('http://paul@www.example.com/a/b/c?a=b#d=f');
SELECT cutQueryStringAndFragment('//paul@www.example.com/a/b/c?a=b#d=f');

View File

@ -61,11 +61,3 @@ ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200002';
ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200012', CLEAR COLUMN i IN PARTITION '200012';
-- Drop empty partition also Ok
ALTER TABLE test.clear_column1 DROP PARTITION '200012', DROP PARTITION '200011';
-- check optimize for non-leader replica (it is not related with CLEAR COLUMN)
OPTIMIZE TABLE test.clear_column1;
OPTIMIZE TABLE test.clear_column2;
DROP TABLE IF EXISTS test.clear_column1;
DROP TABLE IF EXISTS test.clear_column2;

View File

@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
set -e -o pipefail
$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL?query_id=hello" -d 'SELECT sum(ignore(*)) FROM (SELECT number % 1000 AS k, groupArray(number) FROM numbers(100000000) GROUP BY k)' | wc -l &
${CLICKHOUSE_CURL_COMMAND} --max-time 30 -sS "$CLICKHOUSE_URL?query_id=hello" -d 'SELECT sum(ignore(*)) FROM (SELECT number % 1000 AS k, groupArray(number) FROM numbers(20000000) GROUP BY k)' | wc -l &
sleep 0.1 # First query (usually) should be received by the server after this sleep.
$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = 'hello' FORMAT Null"
wait

View File

@ -0,0 +1,11 @@
-
Max query size exceeded
-
Max query size exceeded
1
2
3
4
5
Max query size exceeded
-

View File

@ -0,0 +1,46 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
echo 'select 1' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL}/?max_query_size=8 -d @- 2>&1 | grep -o "Max query size exceeded"
echo -
echo 'select 1' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL}/?max_query_size=7 -d @- 2>&1 | grep -o "Max query size exceeded"
echo "select '1'" | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL}/?max_query_size=10 -d @- 2>&1 | grep -o "Max query size exceeded"
echo -
echo "select '11'" | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL}/?max_query_size=10 -d @- 2>&1 | grep -o "Max query size exceeded"
echo 'drop table if exists test.tab' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
echo 'create table test.tab (key UInt64, val UInt64) engine = MergeTree order by key' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
echo 'into test.tab values (1, 1), (2, 2), (3, 3), (4, 4), (5, 5)' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}/?max_query_size=30&query=insert" -d @-
echo 'select val from test.tab order by val' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
echo 'drop table test.tab' | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @-
echo "
import requests
import os
url = os.environ['CLICKHOUSE_URL']
if not url.startswith('http'):
url = 'http://' + url
q = 'select sum(number) from (select * from system.numbers limit 10000000) where number = 0'
def gen_data(q):
yield q
yield ''.join([' '] * (1024 - len(q)))
pattern = ''' or toString(number) = '{}'\n'''
for i in range(1, 4 * 1024):
yield pattern.format(str(i).zfill(1024 - len(pattern) + 2))
s = requests.Session()
resp = s.post(url + '/?max_query_size={}'.format(1 << 21), timeout=1, data=gen_data(q), stream=True,
headers = {'Connection': 'close'})
for line in resp.iter_lines():
print line
" | python | grep -o "Max query size exceeded"
echo -

View File

@ -0,0 +1,12 @@
0
0
0
1
1
2
0
0
0
1
1
2

View File

@ -0,0 +1,2 @@
SELECT * FROM ((SELECT * FROM system.numbers LIMIT 1) UNION ALL SELECT * FROM system.numbers LIMIT 2 UNION ALL (SELECT * FROM system.numbers LIMIT 3)) ORDER BY number;
SELECT * FROM (SELECT * FROM system.numbers LIMIT 1 UNION ALL (SELECT * FROM system.numbers LIMIT 2 UNION ALL (SELECT * FROM system.numbers LIMIT 3))) ORDER BY number;

View File

@ -0,0 +1,20 @@
DROP TABLE IF EXISTS test.clear_column1;
DROP TABLE IF EXISTS test.clear_column2;
CREATE TABLE test.clear_column1 (p Int64, i Int64, v UInt64) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/test/clear_column', '1', v) PARTITION BY p ORDER BY i;
CREATE TABLE test.clear_column2 (p Int64, i Int64, v UInt64) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/test/clear_column', '2', v) PARTITION BY p ORDER BY i;
INSERT INTO test.clear_column1 VALUES (0, 1, 0);
INSERT INTO test.clear_column1 VALUES (0, 1, 1);
OPTIMIZE TABLE test.clear_column1;
OPTIMIZE TABLE test.clear_column2;
SELECT * FROM test.clear_column1;
RENAME TABLE test.clear_column2 TO test.clear_column3;
INSERT INTO test.clear_column1 VALUES (0, 1, 2);
OPTIMIZE TABLE test.clear_column3;
SELECT * FROM test.clear_column1;
DROP TABLE IF EXISTS test.clear_column1;
DROP TABLE IF EXISTS test.clear_column2;

View File

@ -0,0 +1,5 @@
1
1
2
2
2

Some files were not shown because too many files have changed in this diff Show More