Merge branch 'master' of https://github.com/yandex/ClickHouse into CLICKHOUSE-2720

This commit is contained in:
Ivan Blinkov 2018-04-26 12:11:50 +03:00
commit 65accb230f
96 changed files with 1616 additions and 521 deletions

View File

@ -1,4 +1,54 @@
# ClickHouse release 1.1.54378, 2018-04-13
# 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 at the front of the 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 `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

View File

@ -1,3 +1,23 @@
# ClickHouse release 1.1.54380, 2018-04-21
## Новые возможности:
* Добавлена табличная функция `file(path, format, structure)`. Пример, читающий байты из `/dev/urandom`: `ln -s /dev/urandom /var/lib/clickhouse/user_files/random` `clickhouse-client -q "SELECT * FROM file('random', 'RowBinary', 'd UInt8') LIMIT 10"`.
## Улучшения:
* Добавлена возможность оборачивать подзапросы скобками `()` для повышения читаемости запросов. Например: `(SELECT 1) UNION ALL (SELECT 1)`.
* Простые запросы `SELECT` из таблицы `system.processes` не учитываются в ограничении `max_concurrent_queries`.
* Возможность отключить логирование путем удаления `<log/>` или `<errorlog/>` из конфигурации сервера.
## Исправление ошибок:
* Убрана поддержка выражений типа `(a, b) IN (SELECT (a, b))` (можно использовать эквивалентные выражение `(a, b) IN (SELECT a, b)`), которые приводили к недетерминированному поведению фильтрации `WHERE`.
* Исправлена неправильная работа оператора `IN` в `MATERIALIZED VIEW`.
* Исправлена неправильная работа индекса по ключу партиционирования в выражениях типа `partition_key_column IN (...)`.
* Исправлена невозможность выполнить `OPTIMIZE` запрос на лидирующей реплике после выполнения `RENAME` таблицы.
* Исправлены ошибки авторизации возникающие при выполнении запросов `OPTIMIZE` и `ALTER` на нелидирующей реплике.
* Исправлены зависания запросов `KILL QUERY`.
* Исправлена ошибка в клиентской библиотеке ZooKeeper, которая при использовании непустого префикса `chroot` в конфигурации приводила к потере watch'ей, остановке очереди distributed DDL запросов и замедлению репликации.
# ClickHouse release 1.1.54378, 2018-04-16
## Новые возможности:
@ -44,7 +64,7 @@
## Изменения сборки:
* Поддержка `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`. Для совместимости, пакеты со старыми именами продолжают загружаться в репозиторий.
* Переименованы пакеты `clickhouse-server-base` в `clickhouse-common-static`; `clickhouse-server-common` в `clickhouse-server`; `clickhouse-common-dbg` в `clickhouse-common-static-dbg`. Для установки используйте `clickhouse-server clickhouse-client`. Для совместимости, пакеты со старыми именами продолжают загружаться в репозиторий.
## Обратно несовместимые изменения:

View File

@ -1,4 +1,4 @@
option (ENABLE_EMBEDDED_COMPILER "Set to TRUE to enable support for 'compile' option for query execution" FALSE)
option (ENABLE_EMBEDDED_COMPILER "Set to TRUE to enable support for 'compile' option for query execution" 1)
if (ENABLE_EMBEDDED_COMPILER)
# Based on source code of YT.
@ -32,7 +32,7 @@ if (ENABLE_EMBEDDED_COMPILER)
mark_as_advanced(LLVM_CONFIG_EXECUTABLE)
if(NOT LLVM_CONFIG_EXECUTABLE)
message(FATAL_ERROR "Cannot find LLVM (looking for `llvm-config${LLVM_VERSION_POSTFIX}`, `llvm-config`, `llvm-config-devel`). Please, provide LLVM_ROOT environment variable.")
message(WARNING "Cannot find LLVM (looking for `llvm-config${LLVM_VERSION_POSTFIX}`, `llvm-config`, `llvm-config-devel`). Please, provide LLVM_ROOT environment variable.")
else()
set(LLVM_FOUND TRUE)
@ -102,6 +102,6 @@ if (ENABLE_EMBEDDED_COMPILER)
endif()
if (LLVM_FOUND AND LLVM_INCLUDE_DIRS AND LLVM_LIBRARY_DIRS)
set(USE_EMBEDDED_COMPILER TRUE)
set (USE_EMBEDDED_COMPILER 1)
endif()
endif()

2
contrib/librdkafka vendored

@ -1 +1 @@
Subproject commit c3d50eb613704fb9c8ab3bce95a88275cb5875b7
Subproject commit 7478b5ef16aadd6543fe38bc6a2deb895c70da98

2
contrib/poco vendored

@ -1 +1 @@
Subproject commit a107b0c9cee109fe0abfbf509df3c78a1e0c05fa
Subproject commit 2d5a158303adf9d47b980cdcfdb26cee1460704e

View File

@ -1,7 +1,7 @@
# This strings autochanged from release_lib.sh:
set(VERSION_DESCRIBE v1.1.54378-testing)
set(VERSION_REVISION 54378)
set(VERSION_GITHASH 5b19d89133a5ff7c72e40cc8c0226cb00466ba10)
set(VERSION_DESCRIBE v1.1.54380-testing)
set(VERSION_REVISION 54380)
set(VERSION_GITHASH fb4a44a9132ee6c84aa3b0adff9c8d09a8473a15)
# end of autochange
set (VERSION_MAJOR 1)

View File

@ -69,4 +69,14 @@ String Macros::expand(const String & s, size_t level) const
return expand(res, level + 1);
}
Names Macros::expand(const Names & source_names, size_t level) const
{
Names result_names;
result_names.reserve(source_names.size());
for (const String & name : source_names)
result_names.push_back(expand(name, level));
return result_names;
}
}

View File

@ -1,9 +1,11 @@
#pragma once
#include <Core/Types.h>
#include <Core/Names.h>
#include <map>
namespace Poco
{
namespace Util
@ -29,6 +31,10 @@ public:
*/
String expand(const String & s, size_t level = 0) const;
/** Apply expand for the list.
*/
Names expand(const Names & source_names, size_t level = 0) const;
using MacroMap = std::map<String, String>;
const MacroMap getMacroMap() const { return macros; }

View File

@ -7,6 +7,7 @@
#include <emmintrin.h>
#endif
namespace DB
{
@ -55,27 +56,18 @@ inline size_t countCodePoints(const UInt8 * data, size_t size)
const auto end = data + size;
#if __SSE2__
const auto bytes_sse = sizeof(__m128i);
const auto src_end_sse = (data + size) - (size % bytes_sse);
constexpr auto bytes_sse = sizeof(__m128i);
const auto src_end_sse = data + size / bytes_sse * bytes_sse;
const auto align_sse = _mm_set1_epi8(0x40);
const auto upper_bound = _mm_set1_epi8(0xBF);
const auto threshold = _mm_set1_epi8(0xBF);
for (; data < src_end_sse; data += bytes_sse)
{
const auto chars = _mm_loadu_si128(reinterpret_cast<const __m128i *>(data));
///Align to zero for the solve two case
const auto align_res = _mm_adds_epu8(chars, align_sse);
const auto less_than_and_equals = _mm_cmpeq_epi8(_mm_min_epu8(align_res, upper_bound), align_res);
res += __builtin_popcount(_mm_movemask_epi8(less_than_and_equals));
}
res += __builtin_popcount(_mm_movemask_epi8(
_mm_cmpgt_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i *>(data)), threshold)));
#endif
for (; data < end; ++data) /// Skip UTF-8 continuation bytes.
res += (*data <= 0x7F || *data >= 0xC0);
res += static_cast<Int8>(*data) > static_cast<Int8>(0xBF);
return res;
}

View File

@ -1,6 +1,7 @@
#pragma once
#include "ZooKeeper.h"
#include "KeeperException.h"
#include <functional>
#include <memory>
#include <common/logger_useful.h>
@ -68,7 +69,7 @@ private:
std::thread thread;
std::atomic<bool> shutdown_called {false};
zkutil::EventPtr event = std::make_shared<Poco::Event>();
EventPtr event = std::make_shared<Poco::Event>();
CurrentMetrics::Increment metric_increment{CurrentMetrics::LeaderElection};
@ -115,6 +116,13 @@ private:
success = true;
}
catch (const KeeperException & e)
{
DB::tryLogCurrentException("LeaderElection");
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
break;
}
catch (...)
{
DB::tryLogCurrentException("LeaderElection");

View File

@ -432,6 +432,35 @@ void ZooKeeper::read(T & x)
}
void addRootPath(String & path, const String & root_path)
{
if (path.empty())
throw Exception("Path cannot be empty", ZooKeeper::ZBADARGUMENTS);
if (path[0] != '/')
throw Exception("Path must begin with /", ZooKeeper::ZBADARGUMENTS);
if (root_path.empty())
return;
if (path.size() == 1) /// "/"
path = root_path;
else
path = root_path + path;
}
void removeRootPath(String & path, const String & root_path)
{
if (root_path.empty())
return;
if (path.size() <= root_path.size())
throw Exception("Received path is not longer than root_path", ZooKeeper::ZDATAINCONSISTENCY);
path = path.substr(root_path.size());
}
static constexpr int32_t protocol_version = 0;
static constexpr ZooKeeper::XID watch_xid = -1;
@ -735,6 +764,7 @@ void ZooKeeper::sendThread()
if (expired)
break;
info.request->addRootPath(root_path);
info.request->write(*out);
if (info.request->xid == close_xid)
@ -844,35 +874,6 @@ ZooKeeper::ResponsePtr ZooKeeper::MultiRequest::makeResponse() const { return st
ZooKeeper::ResponsePtr ZooKeeper::CloseRequest::makeResponse() const { return std::make_shared<CloseResponse>(); }
void addRootPath(String & path, const String & root_path)
{
if (path.empty())
throw Exception("Path cannot be empty", ZooKeeper::ZBADARGUMENTS);
if (path[0] != '/')
throw Exception("Path must begin with /", ZooKeeper::ZBADARGUMENTS);
if (root_path.empty())
return;
if (path.size() == 1) /// "/"
path = root_path;
else
path = root_path + path;
}
void removeRootPath(String & path, const String & root_path)
{
if (root_path.empty())
return;
if (path.size() <= root_path.size())
throw Exception("Received path is not longer than root_path", ZooKeeper::ZDATAINCONSISTENCY);
path = path.substr(root_path.size());
}
void ZooKeeper::CreateRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); }
void ZooKeeper::RemoveRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); }
void ZooKeeper::ExistsRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); }
@ -1108,7 +1109,6 @@ void ZooKeeper::finalize(bool error_send, bool error_receive)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
if (info.watch)
{
@ -1335,8 +1335,6 @@ void ZooKeeper::pushRequest(RequestInfo && info)
{
try
{
info.request->addRootPath(root_path);
info.time = clock::now();
if (!info.request->xid)

View File

@ -596,7 +596,7 @@ private:
std::mutex operations_mutex;
using WatchCallbacks = std::vector<WatchCallback>;
using Watches = std::map<String /* path */, WatchCallbacks>;
using Watches = std::map<String /* path, relative of root_path */, WatchCallbacks>;
Watches watches;
std::mutex watches_mutex;

View File

@ -61,7 +61,7 @@ TEST(zkutil, multi_nice_exception_msg)
String msg = getCurrentExceptionMessage(false);
bool msg_has_reqired_patterns = msg.find("/clickhouse_test/zkutil_multi/a") != std::string::npos && msg.find("#2") != std::string::npos;
bool msg_has_reqired_patterns = msg.find("#2") != std::string::npos;
EXPECT_TRUE(msg_has_reqired_patterns) << msg;
}
}
@ -129,40 +129,54 @@ TEST(zkutil, multi_async)
}
}
/// Run this test under sudo
TEST(zkutil, multi_async_libzookeeper_segfault)
TEST(zkutil, watch_get_children_with_chroot)
{
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181", "", 1000);
zkutil::Requests ops;
try
{
const String zk_server = "localhost:2181";
const String prefix = "/clickhouse_test/zkutil/watch_get_children_with_chroot";
ops.emplace_back(zkutil::makeCheckRequest("/clickhouse_test/zkutil_multi", 0));
/// Create chroot node firstly
auto zookeeper = std::make_unique<zkutil::ZooKeeper>(zk_server);
zookeeper->createAncestors(prefix + "/");
zookeeper = std::make_unique<zkutil::ZooKeeper>(zk_server, "", zkutil::DEFAULT_SESSION_TIMEOUT, prefix);
/// Uncomment to test
//auto cmd = ShellCommand::execute("sudo service zookeeper restart");
//cmd->wait();
String queue_path = "/queue";
zookeeper->tryRemoveRecursive(queue_path);
zookeeper->createAncestors(queue_path + "/");
auto future = zookeeper->asyncMulti(ops);
auto res = future.get();
EXPECT_TRUE(zkutil::isHardwareError(res.error));
zkutil::EventPtr event = std::make_shared<Poco::Event>();
zookeeper->getChildren(queue_path, nullptr, event);
{
auto zookeeper2 = std::make_unique<zkutil::ZooKeeper>(zk_server, "", zkutil::DEFAULT_SESSION_TIMEOUT, prefix);
zookeeper2->create(queue_path + "/children-", "", zkutil::CreateMode::PersistentSequential);
}
event->wait();
}
catch (...)
{
std::cerr << getCurrentExceptionMessage(true);
throw;
}
}
TEST(zkutil, multi_create_sequential)
{
try
{
const String zk_server = "localhost:2181";
const String prefix = "/clickhouse_test/zkutil";
/// Create chroot node firstly
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181");
zookeeper->createAncestors("/clickhouse_test/");
auto zookeeper = std::make_unique<zkutil::ZooKeeper>(zk_server);
zookeeper->createAncestors(prefix + "/");
zookeeper = std::make_unique<zkutil::ZooKeeper>(zk_server, "", zkutil::DEFAULT_SESSION_TIMEOUT, "/clickhouse_test");
zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181", "", zkutil::DEFAULT_SESSION_TIMEOUT, "/clickhouse_test");
zkutil::Requests ops;
String base_path = "/zkutil/multi_create_sequential";
String base_path = "/multi_create_sequential";
zookeeper->tryRemoveRecursive(base_path);
zookeeper->createAncestors(base_path + "/");
zkutil::Requests ops;
String sequential_node_prefix = base_path + "/queue-";
ops.emplace_back(zkutil::makeCreateRequest(sequential_node_prefix, "", zkutil::CreateMode::EphemeralSequential));
auto results = zookeeper->multi(ops);
@ -180,3 +194,4 @@ TEST(zkutil, multi_create_sequential)
}

View File

@ -16,6 +16,28 @@ namespace ErrorCodes
}
CreatingSetsBlockInputStream::CreatingSetsBlockInputStream(
const BlockInputStreamPtr & input,
const SubqueriesForSets & subqueries_for_sets_,
const SizeLimits & network_transfer_limits)
: subqueries_for_sets(subqueries_for_sets_),
network_transfer_limits(network_transfer_limits)
{
for (auto & elem : subqueries_for_sets)
{
if (elem.second.source)
{
children.push_back(elem.second.source);
if (elem.second.set)
elem.second.set->setHeader(elem.second.source->getHeader());
}
}
children.push_back(input);
}
Block CreatingSetsBlockInputStream::readImpl()
{
Block res;

View File

@ -20,16 +20,7 @@ public:
CreatingSetsBlockInputStream(
const BlockInputStreamPtr & input,
const SubqueriesForSets & subqueries_for_sets_,
const SizeLimits & network_transfer_limits)
: subqueries_for_sets(subqueries_for_sets_),
network_transfer_limits(network_transfer_limits)
{
for (auto & elem : subqueries_for_sets)
if (elem.second.source)
children.push_back(elem.second.source);
children.push_back(input);
}
const SizeLimits & network_transfer_limits);
String getName() const override { return "CreatingSets"; }

View File

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

View File

@ -20,8 +20,6 @@ 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_);
FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, const String & filter_column_name_);
String getName() const override;

View File

@ -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

@ -128,7 +128,7 @@ int main(int, char **)
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, {}, context, stage, 8192, 1)[0];
in = std::make_shared<FilterBlockInputStream>(in, expression, 4);
in = std::make_shared<FilterBlockInputStream>(in, expression, "equals(URL, 'http://mail.yandex.ru/neo2/#inbox')");
//in = std::make_shared<LimitBlockInputStream>(in, 10, 0);
WriteBufferFromOStream ob(std::cout);

View File

@ -70,8 +70,13 @@ size_t IDataType::getSizeOfValueInMemory() const
String IDataType::getFileNameForStream(const String & column_name, const IDataType::SubstreamPath & path)
{
/// Sizes of arrays (elements of Nested type) are shared (all reside in single file).
String nested_table_name = Nested::extractTableName(column_name);
bool is_sizes_of_nested_type = !path.empty() && path.back().type == IDataType::Substream::ArraySizes
bool is_sizes_of_nested_type =
path.size() == 1 /// Nested structure may have arrays as nested elements (so effectively we have multidimentional arrays).
/// Sizes of arrays are shared only at first level.
&& path[0].type == IDataType::Substream::ArraySizes
&& nested_table_name != column_name;
size_t array_level = 0;

View File

@ -32,13 +32,14 @@ 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)
: log(&Logger::get("HTTPDictionarySource")),

View File

@ -484,18 +484,17 @@ using FunctionExp = FunctionMathUnaryFloat64<UnaryFunctionVectorized<ExpName, ex
using FunctionLog = FunctionMathUnaryFloat64<UnaryFunctionVectorized<LogName, log>>;
using FunctionExp2 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp2Name, exp2>>;
using FunctionLog2 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log2Name, log2>>;
using FunctionExp10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp10Name, preciseExp10>>;
using FunctionLog10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log10Name, log10>>;
using FunctionSqrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SqrtName, sqrt>>;
using FunctionCbrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CbrtName,
using FunctionExp10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Exp10Name,
#if USE_VECTORCLASS
Power_rational<1, 3>::pow
exp10
#else
cbrt
preciseExp10
#endif
>>;
using FunctionLog10 = FunctionMathUnaryFloat64<UnaryFunctionVectorized<Log10Name, log10>>;
using FunctionSqrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SqrtName, sqrt>>;
using FunctionCbrt = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CbrtName, cbrt>>;
using FunctionSin = FunctionMathUnaryFloat64<UnaryFunctionVectorized<SinName, sin>>;
using FunctionCos = FunctionMathUnaryFloat64<UnaryFunctionVectorized<CosName, cos>>;
using FunctionTan = FunctionMathUnaryFloat64<UnaryFunctionVectorized<TanName, tan>>;

View File

@ -813,7 +813,7 @@ public:
/** The `indexHint` function takes any number of any arguments and always returns one.
*
* This function has a special meaning (see ExpressionAnalyzer, PKCondition)
* This function has a special meaning (see ExpressionAnalyzer, KeyCondition)
* - the expressions inside it are not evaluated;
* - but when analyzing the index (selecting ranges for reading), this function is treated the same way,
* as if instead of using it the expression itself would be.

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

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

@ -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();
@ -1341,8 +1358,7 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const
bool Context::hasZooKeeper() const
{
std::lock_guard<std::mutex> lock(shared->zookeeper_mutex);
return shared->zookeeper != nullptr;
return getConfigRef().has("zookeeper");
}

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>;
@ -301,6 +304,7 @@ public:
const MergeList & getMergeList() const;
/// If the current session is expired at the time of the call, synchronously creates and returns a new session with the startNewSession() call.
/// If no ZooKeeper configured, throws an exception.
std::shared_ptr<zkutil::ZooKeeper> getZooKeeper() const;
/// Has ready or expired ZooKeeper
bool hasZooKeeper() const;

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>
@ -1475,6 +1474,7 @@ void ExpressionAnalyzer::tryMakeSetFromSubquery(const ASTPtr & subquery_or_table
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode));
set->setHeader(res.in->getHeader());
while (Block block = res.in->read())
{
/// If the limits have been exceeded, give up and let the default subquery processing actions take place.
@ -2021,7 +2021,7 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
}
/// A special function `indexHint`. Everything that is inside it is not calculated
/// (and is used only for index analysis, see PKCondition).
/// (and is used only for index analysis, see KeyCondition).
if (node->name == "indexHint")
{
actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName(
@ -2068,7 +2068,7 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
const SetPtr & set = prepared_sets[child.get()];
/// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name,
/// so that sets with the same record do not fuse together (they can have different types).
/// so that sets with the same literal representation do not fuse together (they can have different types).
if (!set->empty())
column.name = getUniqueName(actions_stack.getSampleBlock(), "__set");
else
@ -2888,7 +2888,7 @@ void ExpressionAnalyzer::getRequiredSourceColumnsImpl(const ASTPtr & ast,
}
/// A special function `indexHint`. Everything that is inside it is not calculated
/// (and is used only for index analysis, see PKCondition).
/// (and is used only for index analysis, see KeyCondition).
if (node->name == "indexHint")
return;
}

View File

@ -50,8 +50,15 @@ namespace ErrorCodes
static void throwIfReadOnly(Context & context)
{
if (context.getSettingsRef().readonly)
{
const auto & client_info = context.getClientInfo();
if (client_info.interface == ClientInfo::Interface::HTTP && client_info.http_method == ClientInfo::HTTPMethod::GET)
throw Exception("Cannot execute query in readonly mode. "
"For queries over HTTP, method GET implies readonly. You should use method POST for modifying queries.", ErrorCodes::READONLY);
else
throw Exception("Cannot execute query in readonly mode", ErrorCodes::READONLY);
}
}
std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context & context, QueryProcessingStage::Enum stage)

View File

@ -630,8 +630,6 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline
QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns;
query_analyzer->makeSetsForIndex();
/// Initialize the initial data streams to which the query transforms are superimposed. Table or subquery or prepared input?
if (!pipeline.streams.empty())
{
@ -657,6 +655,8 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline
if (max_streams > 1 && !is_remote)
max_streams *= settings.max_streams_to_max_threads_ratio;
query_analyzer->makeSetsForIndex();
SelectQueryInfo query_info;
query_info.query = query_ptr;
query_info.sets = query_analyzer->getPreparedSets();

View File

@ -10,6 +10,8 @@
#include <Common/typeid_cast.h>
#include <common/logger_useful.h>
#include <chrono>
namespace DB
{
@ -33,6 +35,10 @@ static bool isUnlimitedQuery(const IAST * 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())
@ -77,15 +83,15 @@ ProcessList::EntryPtr ProcessList::insert(
bool is_unlimited_query = isUnlimitedQuery(ast);
{
std::lock_guard<std::mutex> lock(mutex);
std::unique_lock lock(mutex);
if (!is_unlimited_query && max_size && cur_size >= max_size)
{
if (!settings.queue_max_wait_ms.totalMilliseconds() || !have_space.tryWait(mutex, settings.queue_max_wait_ms.totalMilliseconds()))
{
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,
@ -224,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)

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>
@ -255,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

@ -21,7 +21,7 @@
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/NullableUtils.h>
#include <Storages/MergeTree/PKCondition.h>
#include <Storages/MergeTree/KeyCondition.h>
namespace DB
@ -62,7 +62,6 @@ void NO_INLINE Set::insertFromBlockImplCase(
{
typename Method::State state;
state.init(key_columns);
size_t keys_size = key_columns.size();
/// For all rows
for (size_t i = 0; i < rows; ++i)
@ -83,19 +82,17 @@ void NO_INLINE Set::insertFromBlockImplCase(
}
bool Set::insertFromBlock(const Block & block, bool fill_set_elements)
void Set::setHeader(const Block & block)
{
std::unique_lock lock(rwlock);
size_t keys_size = block.columns();
if (!empty())
return;
keys_size = block.columns();
ColumnRawPtrs key_columns;
key_columns.reserve(keys_size);
if (empty())
{
data_types.clear();
data_types.reserve(keys_size);
}
/// The constant columns to the right of IN are not supported directly. For this, they first materialize.
Columns materialized_columns;
@ -104,8 +101,6 @@ bool Set::insertFromBlock(const Block & block, bool fill_set_elements)
for (size_t i = 0; i < keys_size; ++i)
{
key_columns.emplace_back(block.safeGetByPosition(i).column.get());
if (empty())
data_types.emplace_back(block.safeGetByPosition(i).type);
if (ColumnPtr converted = key_columns.back()->convertToFullColumnIfConst())
@ -115,31 +110,38 @@ 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]);
/// We will insert to the Set only keys, where all components are not NULL.
ColumnPtr null_map_holder;
ConstNullMapPtr null_map{};
extractNestedColumnsAndNullMap(key_columns, null_map_holder, null_map);
key_columns.pop_back();
const Columns & tuple_elements = column.getColumns();
for (const auto & elem : tuple_elements)
key_columns.push_back(elem.get());
/// Choose data structure to use for the set.
data.init(data.chooseMethod(key_columns, key_sizes));
}
bool Set::insertFromBlock(const Block & block, bool fill_set_elements)
{
std::unique_lock lock(rwlock);
if (empty())
throw Exception("Method Set::setHeader must be called before Set::insertFromBlock", ErrorCodes::LOGICAL_ERROR);
ColumnRawPtrs key_columns;
key_columns.reserve(keys_size);
/// The constant columns to the right of IN are not supported directly. For this, they first materialize.
Columns materialized_columns;
/// Remember the columns we will work with
for (size_t i = 0; i < keys_size; ++i)
{
data_types.pop_back();
data_types.insert(data_types.end(), tuple->getElements().begin(), tuple->getElements().end());
}
key_columns.emplace_back(block.safeGetByPosition(i).column.get());
if (ColumnPtr converted = key_columns.back()->convertToFullColumnIfConst())
{
materialized_columns.emplace_back(converted);
key_columns.back() = materialized_columns.back().get();
}
}
@ -150,10 +152,6 @@ bool Set::insertFromBlock(const Block & block, bool fill_set_elements)
ConstNullMapPtr null_map{};
extractNestedColumnsAndNullMap(key_columns, null_map_holder, null_map);
/// Choose data structure to use for the set.
if (empty())
data.init(data.chooseMethod(key_columns, key_sizes));
switch (data.type)
{
case SetVariants::Type::EMPTY:
@ -172,9 +170,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));
}
}
@ -182,6 +179,7 @@ bool Set::insertFromBlock(const Block & block, bool fill_set_elements)
return limits.check(getTotalRowCount(), getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
}
static Field extractValueFromNode(ASTPtr & node, const IDataType & type, const Context & context)
{
if (ASTLiteral * lit = typeid_cast<ASTLiteral *>(node.get()))
@ -202,16 +200,19 @@ void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & co
{
/// Will form a block with values from the set.
size_t size = types.size();
MutableColumns columns(types.size());
for (size_t i = 0; i < size; ++i)
columns[i] = types[i]->createColumn();
Block header;
size_t num_columns = types.size();
for (size_t i = 0; i < num_columns; ++i)
header.insert(ColumnWithTypeAndName(types[i]->createColumn(), types[i], "_" + toString(i)));
setHeader(header);
MutableColumns columns = header.cloneEmptyColumns();
Row tuple_values;
ASTExpressionList & list = typeid_cast<ASTExpressionList &>(*node);
for (auto & elem : list.children)
{
if (types.size() == 1)
if (num_columns == 1)
{
Field value = extractValueFromNode(elem, *types[0], context);
@ -224,8 +225,9 @@ void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & co
throw Exception("Incorrect element of set. Must be tuple.", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
size_t tuple_size = func->arguments->children.size();
if (tuple_size != types.size())
throw Exception("Incorrect size of tuple in set.", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
if (tuple_size != num_columns)
throw Exception("Incorrect size of tuple in set: " + toString(tuple_size) + " instead of " + toString(num_columns),
ErrorCodes::INCORRECT_ELEMENT_OF_SET);
if (tuple_values.empty())
tuple_values.resize(tuple_size);
@ -250,10 +252,7 @@ void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & co
throw Exception("Incorrect element of set", ErrorCodes::INCORRECT_ELEMENT_OF_SET);
}
Block block;
for (size_t i = 0, size = types.size(); i < size; ++i)
block.insert(ColumnWithTypeAndName(std::move(columns[i]), types[i], "_" + toString(i)));
Block block = header.cloneWithColumns(std::move(columns));
insertFromBlock(block, fill_set_elements);
}
@ -350,7 +349,6 @@ void NO_INLINE Set::executeImplCase(
{
typename Method::State state;
state.init(key_columns);
size_t keys_size = key_columns.size();
/// NOTE Optimization is not used for consecutive identical values.
@ -391,21 +389,21 @@ void Set::executeOrdinary(
}
MergeTreeSetIndex::MergeTreeSetIndex(const SetElements & set_elements, std::vector<PKTuplePositionMapping> && index_mapping_)
MergeTreeSetIndex::MergeTreeSetIndex(const SetElements & set_elements, std::vector<KeyTuplePositionMapping> && index_mapping_)
: ordered_set(),
indexes_mapping(std::move(index_mapping_))
{
std::sort(indexes_mapping.begin(), indexes_mapping.end(),
[](const PKTuplePositionMapping & l, const PKTuplePositionMapping & r)
[](const KeyTuplePositionMapping & l, const KeyTuplePositionMapping & r)
{
return std::forward_as_tuple(l.pk_index, l.tuple_index) < std::forward_as_tuple(r.pk_index, r.tuple_index);
return std::forward_as_tuple(l.key_index, l.tuple_index) < std::forward_as_tuple(r.key_index, r.tuple_index);
});
indexes_mapping.erase(std::unique(
indexes_mapping.begin(), indexes_mapping.end(),
[](const PKTuplePositionMapping & l, const PKTuplePositionMapping & r)
[](const KeyTuplePositionMapping & l, const KeyTuplePositionMapping & r)
{
return l.pk_index == r.pk_index;
return l.key_index == r.key_index;
}), indexes_mapping.end());
for (size_t i = 0; i < set_elements.size(); ++i)
@ -437,10 +435,10 @@ BoolMask MergeTreeSetIndex::mayBeTrueInRange(const std::vector<Range> & key_rang
for (size_t i = 0; i < indexes_mapping.size(); ++i)
{
std::optional<Range> new_range = PKCondition::applyMonotonicFunctionsChainToRange(
key_ranges[indexes_mapping[i].pk_index],
std::optional<Range> new_range = KeyCondition::applyMonotonicFunctionsChainToRange(
key_ranges[indexes_mapping[i].key_index],
indexes_mapping[i].functions,
data_types[indexes_mapping[i].pk_index]);
data_types[indexes_mapping[i].key_index]);
if (!new_range)
return {true, true};

View File

@ -38,6 +38,9 @@ public:
bool empty() const { return data.empty(); }
/** Set can be created either from AST or from a stream of data (subquery result).
*/
/** Create a Set from expression (specified literally in the query).
* 'types' - types of what are on the left hand side of IN.
* 'node' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6).
@ -45,8 +48,12 @@ public:
*/
void createFromAST(const DataTypes & types, ASTPtr node, const Context & context, bool fill_set_elements);
/** Returns false, if some limit was exceeded and no need to insert more data.
/** Create a Set from stream.
* Call setHeader, then call insertFromBlock for each block.
*/
void setHeader(const Block & header);
/// Returns false, if some limit was exceeded and no need to insert more data.
bool insertFromBlock(const Block & block, bool fill_set_elements);
/** For columns of 'block', check belonging of corresponding rows to the set.
@ -56,9 +63,13 @@ 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:
size_t keys_size;
Sizes key_sizes;
SetVariants data;
@ -156,21 +167,24 @@ public:
* position of pk index and data type of this pk column
* and functions chain applied to this column.
*/
struct PKTuplePositionMapping
struct KeyTuplePositionMapping
{
size_t tuple_index;
size_t pk_index;
size_t key_index;
std::vector<FunctionBasePtr> functions;
};
MergeTreeSetIndex(const SetElements & set_elements, std::vector<PKTuplePositionMapping> && indexes_mapping_);
MergeTreeSetIndex(const SetElements & set_elements, std::vector<KeyTuplePositionMapping> && indexes_mapping_);
size_t size() const { return ordered_set.size(); }
BoolMask mayBeTrueInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types);
private:
using OrderedTuples = std::vector<std::vector<FieldWithInfinity>>;
OrderedTuples ordered_set;
std::vector<PKTuplePositionMapping> indexes_mapping;
std::vector<KeyTuplePositionMapping> indexes_mapping;
};
}

View File

@ -6,12 +6,12 @@
namespace DB
{
struct ASTShowProcesslisIDAndQueryNames
struct ASTShowProcesslistIDAndQueryNames
{
static constexpr auto ID = "ShowProcesslistQuery";
static constexpr auto Query = "SHOW PROCESSLIST";
};
using ASTShowProcesslistQuery = ASTQueryWithOutputImpl<ASTShowProcesslisIDAndQueryNames>;
using ASTShowProcesslistQuery = ASTQueryWithOutputImpl<ASTShowProcesslistIDAndQueryNames>;
}

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

@ -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/");
}
/** 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

@ -1,4 +1,4 @@
#include <Storages/MergeTree/PKCondition.h>
#include <Storages/MergeTree/KeyCondition.h>
#include <Storages/MergeTree/BoolMask.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/ExpressionAnalyzer.h>
@ -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;
@ -97,7 +105,7 @@ static String firstStringThatIsGreaterThanAllStringsWithPrefix(const String & pr
/// A dictionary containing actions to the corresponding functions to turn them into `RPNElement`
const PKCondition::AtomMap PKCondition::atom_map
const KeyCondition::AtomMap KeyCondition::atom_map
{
{
"notEquals",
@ -241,7 +249,7 @@ bool FieldWithInfinity::operator==(const FieldWithInfinity & other) const
/** Calculate expressions, that depend only on constants.
* For index to work when something like "WHERE Date = toDate(now())" is written.
*/
Block PKCondition::getBlockWithConstants(
Block KeyCondition::getBlockWithConstants(
const ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns)
{
Block result
@ -257,19 +265,19 @@ Block PKCondition::getBlockWithConstants(
}
PKCondition::PKCondition(
KeyCondition::KeyCondition(
const SelectQueryInfo & query_info,
const Context & context,
const NamesAndTypesList & all_columns,
const SortDescription & sort_descr_,
const ExpressionActionsPtr & pk_expr_)
: sort_descr(sort_descr_), pk_expr(pk_expr_), prepared_sets(query_info.sets)
const ExpressionActionsPtr & key_expr_)
: sort_descr(sort_descr_), key_expr(key_expr_), prepared_sets(query_info.sets)
{
for (size_t i = 0; i < sort_descr.size(); ++i)
{
std::string name = sort_descr[i].column_name;
if (!pk_columns.count(name))
pk_columns[name] = i;
if (!key_columns.count(name))
key_columns[name] = i;
}
/** Evaluation of expressions that depend only on constants.
@ -299,11 +307,11 @@ PKCondition::PKCondition(
}
}
bool PKCondition::addCondition(const String & column, const Range & range)
bool KeyCondition::addCondition(const String & column, const Range & range)
{
if (!pk_columns.count(column))
if (!key_columns.count(column))
return false;
rpn.emplace_back(RPNElement::FUNCTION_IN_RANGE, pk_columns[column], range);
rpn.emplace_back(RPNElement::FUNCTION_IN_RANGE, key_columns[column], range);
rpn.emplace_back(RPNElement::FUNCTION_AND);
return true;
}
@ -360,7 +368,7 @@ static void applyFunction(
}
void PKCondition::traverseAST(const ASTPtr & node, const Context & context, Block & block_with_constants)
void KeyCondition::traverseAST(const ASTPtr & node, const Context & context, Block & block_with_constants)
{
RPNElement element;
@ -393,23 +401,23 @@ void PKCondition::traverseAST(const ASTPtr & node, const Context & context, Bloc
}
bool PKCondition::canConstantBeWrappedByMonotonicFunctions(
bool KeyCondition::canConstantBeWrappedByMonotonicFunctions(
const ASTPtr & node,
size_t & out_primary_key_column_num,
DataTypePtr & out_primary_key_column_type,
size_t & out_key_column_num,
DataTypePtr & out_key_column_type,
Field & out_value,
DataTypePtr & out_type)
{
String expr_name = node->getColumnName();
const auto & sample_block = pk_expr->getSampleBlock();
const auto & sample_block = key_expr->getSampleBlock();
if (!sample_block.has(expr_name))
return false;
bool found_transformation = false;
for (const ExpressionAction & a : pk_expr->getActions())
for (const ExpressionAction & a : key_expr->getActions())
{
/** The primary key functional expression constraint may be inferred from a plain column in the expression.
* For example, if the primary key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`,
/** The key functional expression constraint may be inferred from a plain column in the expression.
* For example, if the key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`,
* it can be assumed that if `toStartOfHour()` is monotonic on [now(), inf), the `toStartOfHour(Timestamp) >= toStartOfHour(now())`
* condition also holds, so the index may be used to select only parts satisfying this condition.
*
@ -438,12 +446,12 @@ bool PKCondition::canConstantBeWrappedByMonotonicFunctions(
out_type.swap(new_type);
expr_name = a.result_name;
// Transformation results in a primary key expression, accept
auto it = pk_columns.find(expr_name);
if (pk_columns.end() != it)
// Transformation results in a key expression, accept
auto it = key_columns.find(expr_name);
if (key_columns.end() != it)
{
out_primary_key_column_num = it->second;
out_primary_key_column_type = sample_block.getByName(it->first).type;
out_key_column_num = it->second;
out_key_column_type = sample_block.getByName(it->first).type;
found_transformation = true;
break;
}
@ -453,56 +461,67 @@ bool PKCondition::canConstantBeWrappedByMonotonicFunctions(
return found_transformation;
}
void PKCondition::getPKTuplePositionMapping(
void KeyCondition::getKeyTuplePositionMapping(
const ASTPtr & node,
const Context & context,
std::vector<MergeTreeSetIndex::PKTuplePositionMapping> & indexes_mapping,
std::vector<MergeTreeSetIndex::KeyTuplePositionMapping> & indexes_mapping,
const size_t tuple_index,
size_t & out_primary_key_column_num)
size_t & out_key_column_num)
{
MergeTreeSetIndex::PKTuplePositionMapping index_mapping;
MergeTreeSetIndex::KeyTuplePositionMapping index_mapping;
index_mapping.tuple_index = tuple_index;
DataTypePtr data_type;
if (isPrimaryKeyPossiblyWrappedByMonotonicFunctions(
node, context, index_mapping.pk_index,
if (isKeyPossiblyWrappedByMonotonicFunctions(
node, context, index_mapping.key_index,
data_type, index_mapping.functions))
{
indexes_mapping.push_back(index_mapping);
if (out_primary_key_column_num < index_mapping.pk_index)
if (out_key_column_num < index_mapping.key_index)
{
out_primary_key_column_num = index_mapping.pk_index;
out_key_column_num = index_mapping.key_index;
}
}
}
// Try to prepare PKTuplePositionMapping for tuples from IN expression.
bool PKCondition::isTupleIndexable(
/// Try to prepare KeyTuplePositionMapping for tuples from IN expression.
bool KeyCondition::isTupleIndexable(
const ASTPtr & node,
const Context & context,
RPNElement & out,
const SetPtr & prepared_set,
size_t & out_primary_key_column_num)
size_t & out_key_column_num)
{
out_primary_key_column_num = 0;
out_key_column_num = 0;
std::vector<MergeTreeSetIndex::KeyTuplePositionMapping> indexes_mapping;
size_t num_key_columns = prepared_set->getDataTypes().size();
const ASTFunction * node_tuple = typeid_cast<const ASTFunction *>(node.get());
std::vector<MergeTreeSetIndex::PKTuplePositionMapping> indexes_mapping;
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);
getKeyTuplePositionMapping(arg, context, indexes_mapping, current_tuple_index, out_key_column_num);
++current_tuple_index;
}
}
else
{
getPKTuplePositionMapping(node, context, indexes_mapping, 0, out_primary_key_column_num);
getKeyTuplePositionMapping(node, context, indexes_mapping, 0, out_key_column_num);
}
if (indexes_mapping.empty())
{
return false;
}
out.set_index = std::make_shared<MergeTreeSetIndex>(
prepared_set->getSetElements(), std::move(indexes_mapping));
@ -510,54 +529,55 @@ bool PKCondition::isTupleIndexable(
return true;
}
bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctions(
bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions(
const ASTPtr & node,
const Context & context,
size_t & out_primary_key_column_num,
DataTypePtr & out_primary_key_res_column_type,
size_t & out_key_column_num,
DataTypePtr & out_key_res_column_type,
RPNElement::MonotonicFunctionsChain & out_functions_chain)
{
std::vector<const ASTFunction *> chain_not_tested_for_monotonicity;
DataTypePtr primary_key_column_type;
DataTypePtr key_column_type;
if (!isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(node, out_primary_key_column_num, primary_key_column_type, chain_not_tested_for_monotonicity))
if (!isKeyPossiblyWrappedByMonotonicFunctionsImpl(node, out_key_column_num, key_column_type, chain_not_tested_for_monotonicity))
return false;
for (auto it = chain_not_tested_for_monotonicity.rbegin(); it != chain_not_tested_for_monotonicity.rend(); ++it)
{
auto func_builder = FunctionFactory::instance().tryGet((*it)->name, context);
ColumnsWithTypeAndName arguments{{ nullptr, primary_key_column_type, "" }};
ColumnsWithTypeAndName arguments{{ nullptr, key_column_type, "" }};
auto func = func_builder->build(arguments);
if (!func || !func->hasInformationAboutMonotonicity())
return false;
primary_key_column_type = func->getReturnType();
key_column_type = func->getReturnType();
out_functions_chain.push_back(func);
}
out_primary_key_res_column_type = primary_key_column_type;
out_key_res_column_type = key_column_type;
return true;
}
bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(
bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
const ASTPtr & node,
size_t & out_primary_key_column_num,
DataTypePtr & out_primary_key_column_type,
size_t & out_key_column_num,
DataTypePtr & out_key_column_type,
std::vector<const ASTFunction *> & out_functions_chain)
{
/** By itself, the primary key column can be a functional expression. for example, `intHash32(UserID)`.
/** By itself, the key column can be a functional expression. for example, `intHash32(UserID)`.
* Therefore, use the full name of the expression for search.
*/
const auto & sample_block = pk_expr->getSampleBlock();
const auto & sample_block = key_expr->getSampleBlock();
String name = node->getColumnName();
auto it = pk_columns.find(name);
if (pk_columns.end() != it)
auto it = key_columns.find(name);
if (key_columns.end() != it)
{
out_primary_key_column_num = it->second;
out_primary_key_column_type = sample_block.getByName(it->first).type;
out_key_column_num = it->second;
out_key_column_type = sample_block.getByName(it->first).type;
return true;
}
@ -569,8 +589,7 @@ bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(
out_functions_chain.push_back(func);
if (!isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(args[0], out_primary_key_column_num, out_primary_key_column_type,
out_functions_chain))
if (!isKeyPossiblyWrappedByMonotonicFunctionsImpl(args[0], out_key_column_num, out_key_column_type, out_functions_chain))
return false;
return true;
@ -592,7 +611,7 @@ static void castValueToType(const DataTypePtr & desired_type, Field & src_value,
}
catch (...)
{
throw Exception("Primary key expression contains comparison between inconvertible types: " +
throw Exception("Key expression contains comparison between inconvertible types: " +
desired_type->getName() + " and " + src_type->getName() +
" inside " + queryToString(node),
ErrorCodes::BAD_TYPE_OF_FIELD);
@ -600,9 +619,9 @@ static void castValueToType(const DataTypePtr & desired_type, Field & src_value,
}
bool PKCondition::atomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out)
bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out)
{
/** Functions < > = != <= >= in `notIn`, where one argument is a constant, and the other is one of columns of primary key,
/** Functions < > = != <= >= in `notIn`, where one argument is a constant, and the other is one of columns of key,
* or itself, wrapped in a chain of possibly-monotonic functions,
* or constant expression - number.
*/
@ -615,9 +634,9 @@ bool PKCondition::atomFromAST(const ASTPtr & node, const Context & context, Bloc
if (args.size() != 2)
return false;
DataTypePtr key_expr_type; /// Type of expression containing primary key column
size_t key_arg_pos; /// Position of argument with primary key column (non-const argument)
size_t key_column_num; /// Number of a primary key column (inside sort_descr array)
DataTypePtr key_expr_type; /// Type of expression containing key column
size_t key_arg_pos; /// Position of argument with key column (non-const argument)
size_t key_column_num; /// Number of a key column (inside sort_descr array)
RPNElement::MonotonicFunctionsChain chain;
bool is_set_const = false;
bool is_constant_transformed = false;
@ -629,7 +648,7 @@ bool PKCondition::atomFromAST(const ASTPtr & node, const Context & context, Bloc
is_set_const = true;
}
else if (getConstant(args[1], block_with_constants, const_value, const_type)
&& isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain))
&& isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain))
{
key_arg_pos = 0;
}
@ -640,7 +659,7 @@ bool PKCondition::atomFromAST(const ASTPtr & node, const Context & context, Bloc
is_constant_transformed = true;
}
else if (getConstant(args[0], block_with_constants, const_value, const_type)
&& isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[1], context, key_column_num, key_expr_type, chain))
&& isKeyPossiblyWrappedByMonotonicFunctions(args[1], context, key_column_num, key_expr_type, chain))
{
key_arg_pos = 1;
}
@ -716,7 +735,7 @@ bool PKCondition::atomFromAST(const ASTPtr & node, const Context & context, Bloc
return false;
}
bool PKCondition::operatorFromAST(const ASTFunction * func, RPNElement & out)
bool KeyCondition::operatorFromAST(const ASTFunction * func, RPNElement & out)
{
/// Functions AND, OR, NOT.
/** Also a special function `indexHint` - works as if instead of calling a function there are just parentheses
@ -744,7 +763,7 @@ bool PKCondition::operatorFromAST(const ASTFunction * func, RPNElement & out)
return true;
}
String PKCondition::toString() const
String KeyCondition::toString() const
{
String res;
for (size_t i = 0; i < rpn.size(); ++i)
@ -757,16 +776,16 @@ String PKCondition::toString() const
}
/** Index is the value of primary key every `index_granularity` rows.
/** Index is the value of key every `index_granularity` rows.
* This value is called a "mark". That is, the index consists of marks.
*
* The primary key is the tuple.
* The data is sorted by primary key in the sense of lexicographic order over tuples.
* The key is the tuple.
* The data is sorted by key in the sense of lexicographic order over tuples.
*
* A pair of marks specifies a segment with respect to the order over the tuples.
* Denote it like this: [ x1 y1 z1 .. x2 y2 z2 ],
* where x1 y1 z1 - tuple - value of primary key in left border of segment;
* x2 y2 z2 - tuple - value of primary key in right boundary of segment.
* where x1 y1 z1 - tuple - value of key in left border of segment;
* x2 y2 z2 - tuple - value of key in right boundary of segment.
* In this section there are data between these marks.
*
* Or, the last mark specifies the range open on the right: [ a b c .. + inf )
@ -876,10 +895,10 @@ static bool forAnyParallelogram(
}
bool PKCondition::mayBeTrueInRange(
bool KeyCondition::mayBeTrueInRange(
size_t used_key_size,
const Field * left_pk,
const Field * right_pk,
const Field * left_key,
const Field * right_key,
const DataTypes & data_types,
bool right_bounded) const
{
@ -887,19 +906,19 @@ bool PKCondition::mayBeTrueInRange(
/* std::cerr << "Checking for: [";
for (size_t i = 0; i != used_key_size; ++i)
std::cerr << (i != 0 ? ", " : "") << applyVisitor(FieldVisitorToString(), left_pk[i]);
std::cerr << (i != 0 ? ", " : "") << applyVisitor(FieldVisitorToString(), left_key[i]);
std::cerr << " ... ";
if (right_bounded)
{
for (size_t i = 0; i != used_key_size; ++i)
std::cerr << (i != 0 ? ", " : "") << applyVisitor(FieldVisitorToString(), right_pk[i]);
std::cerr << (i != 0 ? ", " : "") << applyVisitor(FieldVisitorToString(), right_key[i]);
std::cerr << "]\n";
}
else
std::cerr << "+inf)\n";*/
return forAnyParallelogram(used_key_size, left_pk, right_pk, true, right_bounded, key_ranges, 0,
return forAnyParallelogram(used_key_size, left_key, right_key, true, right_bounded, key_ranges, 0,
[&] (const std::vector<Range> & key_ranges)
{
auto res = mayBeTrueInRangeImpl(key_ranges, data_types);
@ -913,7 +932,7 @@ bool PKCondition::mayBeTrueInRange(
});
}
std::optional<Range> PKCondition::applyMonotonicFunctionsChainToRange(
std::optional<Range> KeyCondition::applyMonotonicFunctionsChainToRange(
Range key_range,
RPNElement::MonotonicFunctionsChain & functions,
DataTypePtr current_type
@ -950,7 +969,7 @@ std::optional<Range> PKCondition::applyMonotonicFunctionsChainToRange(
return key_range;
}
bool PKCondition::mayBeTrueInRangeImpl(const std::vector<Range> & key_ranges, const DataTypes & data_types) const
bool KeyCondition::mayBeTrueInRangeImpl(const std::vector<Range> & key_ranges, const DataTypes & data_types) const
{
std::vector<BoolMask> rpn_stack;
for (size_t i = 0; i < rpn.size(); ++i)
@ -1002,14 +1021,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)
{
@ -1038,30 +1053,30 @@ bool PKCondition::mayBeTrueInRangeImpl(const std::vector<Range> & key_ranges, co
rpn_stack.emplace_back(true, false);
}
else
throw Exception("Unexpected function type in PKCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
throw Exception("Unexpected function type in KeyCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
}
if (rpn_stack.size() != 1)
throw Exception("Unexpected stack size in PKCondition::mayBeTrueInRange", ErrorCodes::LOGICAL_ERROR);
throw Exception("Unexpected stack size in KeyCondition::mayBeTrueInRange", ErrorCodes::LOGICAL_ERROR);
return rpn_stack[0].can_be_true;
}
bool PKCondition::mayBeTrueInRange(
size_t used_key_size, const Field * left_pk, const Field * right_pk, const DataTypes & data_types) const
bool KeyCondition::mayBeTrueInRange(
size_t used_key_size, const Field * left_key, const Field * right_key, const DataTypes & data_types) const
{
return mayBeTrueInRange(used_key_size, left_pk, right_pk, data_types, true);
return mayBeTrueInRange(used_key_size, left_key, right_key, data_types, true);
}
bool PKCondition::mayBeTrueAfter(
size_t used_key_size, const Field * left_pk, const DataTypes & data_types) const
bool KeyCondition::mayBeTrueAfter(
size_t used_key_size, const Field * left_key, const DataTypes & data_types) const
{
return mayBeTrueInRange(used_key_size, left_pk, nullptr, data_types, false);
return mayBeTrueInRange(used_key_size, left_key, nullptr, data_types, false);
}
String PKCondition::RPNElement::toString() const
String KeyCondition::RPNElement::toString() const
{
auto print_wrapped_column = [this](std::ostringstream & ss)
{
@ -1090,7 +1105,11 @@ String PKCondition::RPNElement::toString() const
{
ss << "(";
print_wrapped_column(ss);
ss << (function == FUNCTION_IN_SET ? " in set" : " notIn set");
ss << (function == FUNCTION_IN_SET ? " in " : " notIn ");
if (!set_index)
ss << "unknown size set";
else
ss << set_index->size() << "-element set";
ss << ")";
return ss.str();
}
@ -1113,7 +1132,7 @@ String PKCondition::RPNElement::toString() const
}
bool PKCondition::alwaysUnknownOrTrue() const
bool KeyCondition::alwaysUnknownOrTrue() const
{
std::vector<UInt8> rpn_stack;
@ -1150,14 +1169,14 @@ bool PKCondition::alwaysUnknownOrTrue() const
rpn_stack.back() = arg1 | arg2;
}
else
throw Exception("Unexpected function type in PKCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
throw Exception("Unexpected function type in KeyCondition::RPNElement", ErrorCodes::LOGICAL_ERROR);
}
return rpn_stack[0];
}
size_t PKCondition::getMaxKeyColumn() const
size_t KeyCondition::getMaxKeyColumn() const
{
size_t res = 0;
for (const auto & element : rpn)

View File

@ -224,34 +224,34 @@ private:
* Constructs a reverse polish notation from these conditions
* and can calculate (interpret) its satisfiability over key ranges.
*/
class PKCondition
class KeyCondition
{
public:
/// Does not take into account the SAMPLE section. all_columns - the set of all columns of the table.
PKCondition(
KeyCondition(
const SelectQueryInfo & query_info,
const Context & context,
const NamesAndTypesList & all_columns,
const SortDescription & sort_descr,
const ExpressionActionsPtr & pk_expr);
const ExpressionActionsPtr & key_expr);
/// Whether the condition is feasible in the key range.
/// left_pk and right_pk must contain all fields in the sort_descr in the appropriate order.
/// data_types - the types of the primary key columns.
bool mayBeTrueInRange(size_t used_key_size, const Field * left_pk, const Field * right_pk, const DataTypes & data_types) const;
/// left_key and right_key must contain all fields in the sort_descr in the appropriate order.
/// data_types - the types of the key columns.
bool mayBeTrueInRange(size_t used_key_size, const Field * left_key, const Field * right_key, const DataTypes & data_types) const;
/// Is the condition valid in a semi-infinite (not limited to the right) key range.
/// left_pk must contain all the fields in the sort_descr in the appropriate order.
bool mayBeTrueAfter(size_t used_key_size, const Field * left_pk, const DataTypes & data_types) const;
/// left_key must contain all the fields in the sort_descr in the appropriate order.
bool mayBeTrueAfter(size_t used_key_size, const Field * left_key, const DataTypes & data_types) const;
/// Checks that the index can not be used.
bool alwaysUnknownOrTrue() const;
/// Get the maximum number of the primary key element used in the condition.
/// Get the maximum number of the key element used in the condition.
size_t getMaxKeyColumn() const;
/// Impose an additional condition: the value in the column column must be in the `range` range.
/// Returns whether there is such a column in the primary key.
/// Returns whether there is such a column in the key.
bool addCondition(const String & column, const Range & range);
String toString() const;
@ -296,7 +296,7 @@ public:
MergeTreeSetIndexPtr set_index;
/** A chain of possibly monotone functions.
* If the primary key column is wrapped in functions that can be monotonous in some value ranges
* If the key column is wrapped in functions that can be monotonous in some value ranges
* (for example: -toFloat64(toDayOfWeek(date))), then here the functions will be located: toDayOfWeek, toFloat64, negate.
*/
using MonotonicFunctionsChain = std::vector<FunctionBasePtr>;
@ -320,8 +320,8 @@ private:
bool mayBeTrueInRange(
size_t used_key_size,
const Field * left_pk,
const Field * right_pk,
const Field * left_key,
const Field * right_key,
const DataTypes & data_types,
bool right_bounded) const;
@ -331,51 +331,51 @@ private:
bool atomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out);
bool operatorFromAST(const ASTFunction * func, RPNElement & out);
/** Is node the primary key column
* or expression in which column of primary key is wrapped by chain of functions,
/** Is node the key column
* or expression in which column of key is wrapped by chain of functions,
* that can be monotomic on certain ranges?
* If these conditions are true, then returns number of column in primary key, type of resulting expression
* If these conditions are true, then returns number of column in key, type of resulting expression
* and fills chain of possibly-monotonic functions.
*/
bool isPrimaryKeyPossiblyWrappedByMonotonicFunctions(
bool isKeyPossiblyWrappedByMonotonicFunctions(
const ASTPtr & node,
const Context & context,
size_t & out_primary_key_column_num,
DataTypePtr & out_primary_key_res_column_type,
size_t & out_key_column_num,
DataTypePtr & out_key_res_column_type,
RPNElement::MonotonicFunctionsChain & out_functions_chain);
bool isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(
bool isKeyPossiblyWrappedByMonotonicFunctionsImpl(
const ASTPtr & node,
size_t & out_primary_key_column_num,
DataTypePtr & out_primary_key_column_type,
size_t & out_key_column_num,
DataTypePtr & out_key_column_type,
std::vector<const ASTFunction *> & out_functions_chain);
bool canConstantBeWrappedByMonotonicFunctions(
const ASTPtr & node,
size_t & out_primary_key_column_num,
DataTypePtr & out_primary_key_column_type,
size_t & out_key_column_num,
DataTypePtr & out_key_column_type,
Field & out_value,
DataTypePtr & out_type);
void getPKTuplePositionMapping(
void getKeyTuplePositionMapping(
const ASTPtr & node,
const Context & context,
std::vector<MergeTreeSetIndex::PKTuplePositionMapping> & indexes_mapping,
std::vector<MergeTreeSetIndex::KeyTuplePositionMapping> & indexes_mapping,
const size_t tuple_index,
size_t & out_primary_key_column_num);
size_t & out_key_column_num);
bool isTupleIndexable(
const ASTPtr & node,
const Context & context,
RPNElement & out,
const SetPtr & prepared_set,
size_t & out_primary_key_column_num);
size_t & out_key_column_num);
RPN rpn;
SortDescription sort_descr;
ColumnIndices pk_columns;
ExpressionActionsPtr pk_expr;
ColumnIndices key_columns;
ExpressionActionsPtr key_expr;
PreparedSets prepared_sets;
};

View File

@ -2175,7 +2175,7 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit()
return total_covered_parts;
}
bool MergeTreeData::isPrimaryKeyOrPartitionKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const
bool MergeTreeData::isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const
{
String column_name = node->getColumnName();
@ -2183,33 +2183,35 @@ bool MergeTreeData::isPrimaryKeyOrPartitionKeyColumnPossiblyWrappedInFunctions(c
if (column_name == column.column_name)
return true;
if (partition_expr_ast && partition_expr_ast->children.at(0)->getColumnName() == column_name)
for (const auto & column : minmax_idx_sort_descr)
if (column_name == column.column_name)
return true;
if (const ASTFunction * func = typeid_cast<const ASTFunction *>(node.get()))
if (func->arguments->children.size() == 1)
return isPrimaryKeyOrPartitionKeyColumnPossiblyWrappedInFunctions(func->arguments->children.front());
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(func->arguments->children.front());
return false;
}
bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand) const
{
/// Make sure that the left side of the IN operator contain part of the primary key.
/// If there is a tuple on the left side of the IN operator, at least one item of the tuple must be part of the primary key (probably wrapped by a chain of some acceptable functions).
/// Make sure that the left side of the IN operator contain part of the key.
/// If there is a tuple on the left side of the IN operator, at least one item of the tuple
/// must be part of the key (probably wrapped by a chain of some acceptable functions).
const ASTFunction * left_in_operand_tuple = typeid_cast<const ASTFunction *>(left_in_operand.get());
if (left_in_operand_tuple && left_in_operand_tuple->name == "tuple")
{
for (const auto & item : left_in_operand_tuple->arguments->children)
if (isPrimaryKeyOrPartitionKeyColumnPossiblyWrappedInFunctions(item))
if (isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(item))
return true;
/// The tuple itself may be part of the primary key, so check that as a last resort.
return isPrimaryKeyOrPartitionKeyColumnPossiblyWrappedInFunctions(left_in_operand);
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand);
}
else
{
return isPrimaryKeyOrPartitionKeyColumnPossiblyWrappedInFunctions(left_in_operand);
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand);
}
}

View File

@ -502,7 +502,7 @@ public:
Names minmax_idx_columns;
DataTypes minmax_idx_column_types;
Int64 minmax_idx_date_column_pos = -1; /// In a common case minmax index includes a date column.
SortDescription minmax_idx_sort_descr; /// For use with PKCondition.
SortDescription minmax_idx_sort_descr; /// For use with KeyCondition.
/// Limiting parallel sends per one table, used in DataPartsExchange
std::atomic_uint current_table_sends {0};
@ -652,7 +652,7 @@ private:
std::lock_guard<std::mutex> & data_parts_lock) const;
/// Checks whether the column is in the primary key, possibly wrapped in a chain of functions with single argument.
bool isPrimaryKeyOrPartitionKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const;
bool isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const;
};
}

View File

@ -6,7 +6,7 @@
#include <Storages/MergeTree/MergeTreeBlockInputStream.h>
#include <Storages/MergeTree/MergeTreeReadPool.h>
#include <Storages/MergeTree/MergeTreeThreadBlockInputStream.h>
#include <Storages/MergeTree/PKCondition.h>
#include <Storages/MergeTree/KeyCondition.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSampleRatio.h>
@ -85,7 +85,7 @@ static Block getBlockWithPartColumn(const MergeTreeData::DataPartsVector & parts
size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead(
const MergeTreeData::DataPartsVector & parts, const PKCondition & key_condition, const Settings & settings) const
const MergeTreeData::DataPartsVector & parts, const KeyCondition & key_condition, const Settings & settings) const
{
size_t full_marks_count = 0;
@ -198,7 +198,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
const Settings & settings = context.getSettingsRef();
SortDescription sort_descr = data.getPrimarySortDescription();
PKCondition key_condition(query_info, context, available_real_and_virtual_columns, sort_descr,
KeyCondition key_condition(query_info, context, available_real_and_virtual_columns, sort_descr,
data.getPrimaryExpression());
if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue())
@ -212,7 +212,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
throw Exception(exception_message.str(), ErrorCodes::INDEX_NOT_USED);
}
std::optional<PKCondition> minmax_idx_condition;
std::optional<KeyCondition> minmax_idx_condition;
if (data.minmax_idx_expr)
{
minmax_idx_condition.emplace(
@ -843,7 +843,7 @@ void MergeTreeDataSelectExecutor::createPositiveSignCondition(
/// Calculates a set of mark ranges, that could possibly contain keys, required by condition.
/// In other words, it removes subranges from whole range, that definitely could not contain required keys.
MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
const MergeTreeData::DataPart::Index & index, const PKCondition & key_condition, const Settings & settings) const
const MergeTreeData::DataPart::Index & index, const KeyCondition & key_condition, const Settings & settings) const
{
size_t min_marks_for_seek = (settings.merge_tree_min_rows_for_seek + data.index_granularity - 1) / data.index_granularity;
@ -866,7 +866,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
*/
std::vector<MarkRange> ranges_stack{ {0, marks_count} };
/// NOTE Creating temporary Field objects to pass to PKCondition.
/// NOTE Creating temporary Field objects to pass to KeyCondition.
Row index_left(used_key_size);
Row index_right(used_key_size);

View File

@ -9,7 +9,7 @@
namespace DB
{
class PKCondition;
class KeyCondition;
/** Executes SELECT queries on data from the merge tree.
@ -60,7 +60,7 @@ private:
/// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index.
size_t getApproximateTotalRowsToRead(
const MergeTreeData::DataPartsVector & parts,
const PKCondition & key_condition,
const KeyCondition & key_condition,
const Settings & settings) const;
/// Create the expression "Sign == 1".
@ -71,7 +71,7 @@ private:
MarkRanges markRangesFromPKRange(
const MergeTreeData::DataPart::Index & index,
const PKCondition & key_condition,
const KeyCondition & key_condition,
const Settings & settings) const;
};

View File

@ -365,7 +365,7 @@ void MergeTreeReader::readData(
IDataType::InputStreamGetter stream_getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer *
{
/// If offsets for arrays have already been read.
if (!with_offsets && !path.empty() && path.back().type == IDataType::Substream::ArraySizes)
if (!with_offsets && path.size() == 1 && path[0].type == IDataType::Substream::ArraySizes)
return nullptr;
String stream_name = IDataType::getFileNameForStream(name, path);

View File

@ -1,6 +1,6 @@
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/PKCondition.h>
#include <Storages/MergeTree/KeyCondition.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
@ -43,7 +43,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer(
[] (const SortColumnDescription & col) { return col.column_name; })},
table_columns{ext::map<std::unordered_set>(data.getColumns().getAllPhysical(),
[] (const NameAndTypePair & col) { return col.name; })},
block_with_constants{PKCondition::getBlockWithConstants(query_info.query, context, data.getColumns().getAllPhysical())},
block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, context, data.getColumns().getAllPhysical())},
prepared_sets(query_info.sets),
log{log}
{
@ -321,7 +321,7 @@ bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const IAST * const ast) const
{
if (const auto func = typeid_cast<const ASTFunction *>(ast))
{
if (!PKCondition::atom_map.count(func->name))
if (!KeyCondition::atom_map.count(func->name))
return false;
const auto & args = func->arguments->children;

View File

@ -150,6 +150,7 @@ private:
class MergedColumnOnlyOutputStream final : public IMergedBlockOutputStream
{
public:
/// skip_offsets: used when ALTERing columns if we know that array offsets are not altered.
MergedColumnOnlyOutputStream(
MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, CompressionSettings compression_settings, bool skip_offsets_);

View File

@ -188,12 +188,21 @@ void ReplicatedMergeTreeAlterThread::run()
wakeup_event->wait();
}
catch (const zkutil::KeeperException & e)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
break;
force_recheck_parts = true;
wakeup_event->tryWait(ALTER_ERROR_SLEEP_MS);
}
catch (...)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
force_recheck_parts = true;
wakeup_event->tryWait(ALTER_ERROR_SLEEP_MS);
}
}

View File

@ -36,6 +36,13 @@ void ReplicatedMergeTreeCleanupThread::run()
{
iterate();
}
catch (const zkutil::KeeperException & e)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
break;
}
catch (...)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);

View File

@ -381,6 +381,15 @@ void ReplicatedMergeTreePartCheckThread::run()
}
}
}
catch (const zkutil::KeeperException & e)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
break;
wakeup_event.tryWait(PART_CHECK_ERROR_SLEEP_MS);
}
catch (...)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);

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;
@ -279,7 +299,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;
@ -291,23 +311,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

@ -6,6 +6,7 @@
#include <boost/algorithm/string/replace.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/trim.hpp>
#include <Common/Macros.h>
#include <Common/Exception.h>
#include <Common/setThreadName.h>
#include <Common/typeid_cast.h>
@ -225,7 +226,11 @@ StorageKafka::StorageKafka(
const String & format_name_, const String & schema_name_, size_t num_consumers_)
: IStorage{columns_},
table_name(table_name_), database_name(database_name_), context(context_),
topics(topics_), brokers(brokers_), group(group_), format_name(format_name_), schema_name(schema_name_),
topics(context.getMacros()->expand(topics_)),
brokers(context.getMacros()->expand(brokers_)),
group(context.getMacros()->expand(group_)),
format_name(context.getMacros()->expand(format_name_)),
schema_name(context.getMacros()->expand(schema_name_)),
num_consumers(num_consumers_), log(&Logger::get("StorageKafka (" + table_name_ + ")")),
semaphore(0, num_consumers_), mutex(), consumers(), event_update()
{

View File

@ -191,7 +191,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
const MergeTreeSettings & settings_,
bool has_force_restore_data_flag)
: context(context_),
current_zookeeper(context.getZooKeeper()), database_name(database_name_),
database_name(database_name_),
table_name(name_), full_path(path_ + escapeForFileName(table_name) + '/'),
zookeeper_path(context.getMacros()->expand(zookeeper_path_)),
replica_name(context.getMacros()->expand(replica_name_)),
@ -216,6 +216,9 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
zookeeper_path = "/" + zookeeper_path;
replica_path = zookeeper_path + "/replicas/" + replica_name;
if (context.hasZooKeeper())
current_zookeeper = context.getZooKeeper();
bool skip_sanity_checks = false;
if (current_zookeeper && current_zookeeper->exists(replica_path + "/flags/force_restore_data"))
@ -1596,6 +1599,15 @@ void StorageReplicatedMergeTree::queueUpdatingThread()
update_in_progress = false;
queue_updating_event->wait();
}
catch (const zkutil::KeeperException & e)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
break;
else
queue_updating_event->tryWait(QUEUE_UPDATE_ERROR_SLEEP_MS);
}
catch (...)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);

View File

@ -107,6 +107,10 @@ StorageSet::StorageSet(
: StorageSetOrJoinBase{path_, name_, columns_},
set(std::make_shared<Set>(SizeLimits()))
{
Block header = getSampleBlock();
header = header.sortColumns();
set->setHeader(header);
restore();
}

View File

@ -239,6 +239,8 @@ BlockInputStreams StorageSystemTables::read(
res_columns[j++]->insert(table.first);
res_columns[j++]->insert(table.second->getName());
res_columns[j++]->insert(UInt64(1));
res_columns[j++]->insertDefault();
res_columns[j++]->insertDefault();
if (has_metadata_modification_time)
res_columns[j++]->insertDefault();

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

@ -39,9 +39,9 @@ DataTypePtr getDataType(SQLSMALLINT type)
switch (type)
{
case SQL_INTEGER:
return factory.get("UInt32");
return factory.get("Int32");
case SQL_SMALLINT:
return factory.get("UInt16");
return factory.get("Int16");
case SQL_FLOAT:
return factory.get("Float32");
case SQL_REAL:

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

@ -1,38 +0,0 @@
<test>
<name>functions_length</name>
<type>once</type>
<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>string</name>
<values>
<value>materialize('')</value>
<value>materialize('Hello, world')</value>
<value>toString(number)</value>
<value>reinterpretAsString(number)</value>
<value>materialize('中文测试字符串')</value>
<value>materialize('https://github.com/yandex/ClickHouse/pull/1882')</value>
<value>materialize('https://zh.wikipedia.org/wiki/%E4%B8%AD%E6%97%A5%E9%9F%93%E7%B5%B1%E4%B8%80%E8%A1%A8%E6%84%8F%E6%96%87%E5%AD%97%E6%93%B4%E5%B1%95%E5%8D%80F')</value>
<value>concat('中文测试字符串 ', toString(number), ' Привет, мир!')</value>
<value>concat(concat('中文测试字符串 ', toString(number), ' Привет, мир!') AS x, x, x, x, x, x, x, x, x, x)</value>
<value>convertCharset(concat(reinterpretAsString(rand64(1)), reinterpretAsString(rand64(2)), reinterpretAsString(rand64(3)), reinterpretAsString(rand64(4)), reinterpretAsString(rand64(5)), reinterpretAsString(rand64(6)), reinterpretAsString(rand64(7)), reinterpretAsString(rand64(8)), reinterpretAsString(rand64(9)), reinterpretAsString(rand64(10))), 'UTF-16', 'UTF-8')</value>
</values>
</substitution>
</substitutions>
<query>SELECT count() FROM system.numbers WHERE NOT ignore(lengthUTF8({string}))</query>
</test>

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

@ -2,4 +2,5 @@ A
B
A 1 TinyLog CREATE TABLE test_show_tables.A ( A UInt8) ENGINE = TinyLog
B 1 TinyLog CREATE TABLE test_show_tables.B ( A UInt8) ENGINE = TinyLog
test_temporary_table
0

View File

@ -9,6 +9,9 @@ SHOW TABLES from test_show_tables;
SELECT name, toUInt32(metadata_modification_time) > 0, engine_full, create_table_query FROM system.tables WHERE database = 'test_show_tables' ORDER BY name FORMAT TSVRaw;
CREATE TEMPORARY TABLE test_temporary_table (id UInt64);
SELECT name FROM system.tables WHERE is_temporary = 1 AND name = 'test_temporary_table';
DROP DATABASE test_show_tables;

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

@ -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,5 @@
1
1
2
2
2

View File

@ -0,0 +1,19 @@
DROP TABLE IF EXISTS test.regression_for_in_operator_view;
DROP TABLE IF EXISTS test.regression_for_in_operator;
CREATE TABLE test.regression_for_in_operator (d Date, v UInt32, g String) ENGINE=MergeTree(d, d, 8192);
CREATE MATERIALIZED VIEW test.regression_for_in_operator_view ENGINE=AggregatingMergeTree(d, (d,g), 8192) AS SELECT d, g, maxState(v) FROM test.regression_for_in_operator GROUP BY d, g;
INSERT INTO test.regression_for_in_operator SELECT today(), toString(number % 10), number FROM system.numbers limit 1000;
SELECT count() FROM test.regression_for_in_operator_view WHERE g = '5';
SELECT count() FROM test.regression_for_in_operator_view WHERE g IN ('5');
SELECT count() FROM test.regression_for_in_operator_view WHERE g IN ('5','6');
SET optimize_min_equality_disjunction_chain_length = 1;
SELECT count() FROM test.regression_for_in_operator_view WHERE g = '5' OR g = '6';
SET optimize_min_equality_disjunction_chain_length = 3;
SELECT count() FROM test.regression_for_in_operator_view WHERE g = '5' OR g = '6';
DROP TABLE test.regression_for_in_operator_view;
DROP TABLE test.regression_for_in_operator;

View File

@ -0,0 +1,4 @@
1
2
2
3

View File

@ -0,0 +1,3 @@
(SELECT 1);
(SELECT 2) UNION ALL (SELECT 2);
((SELECT (((3)))));

View File

@ -0,0 +1,59 @@
111
112
113
114
-
121
122
123
124
-
131
132
133
134
---
211
212
213
214
-
221
222
223
224
-
231
232
233
234
---
311
312
313
314
-
321
322
323
324
-
331
332
333
334
---
411
412
413
414
-
421
422
423
424
-
431
432
433
434

View File

@ -0,0 +1,75 @@
drop table if exists test.test54378;
create table test.test54378 (part_date Date, pk_date Date, date Date) Engine=MergeTree(part_date, pk_date, 8192);
insert into test.test54378 values ('2018-04-19', '2018-04-19', '2018-04-19');
select 111 from test.test54378 where part_date = '2018-04-19';
select 112 from test.test54378 where part_date in ('2018-04-19');
select 113 from test.test54378 where pk_date in ('2018-04-19');
select 114 from test.test54378 where date in ('2018-04-19');
SELECT '-';
select 121 from test.test54378 where part_date = toDate('2018-04-19');
select 122 from test.test54378 where part_date in (toDate('2018-04-19'));
select 123 from test.test54378 where pk_date in (toDate('2018-04-19'));
select 124 from test.test54378 where date in (toDate('2018-04-19'));
SELECT '-';
select 131 from test.test54378 where part_date = (SELECT toDate('2018-04-19'));
select 132 from test.test54378 where part_date in (SELECT toDate('2018-04-19'));
select 133 from test.test54378 where pk_date in (SELECT toDate('2018-04-19'));
select 134 from test.test54378 where date in (SELECT toDate('2018-04-19'));
SELECT '---';
select 211 from test.test54378 prewhere part_date = '2018-04-19';
select 212 from test.test54378 prewhere part_date in ('2018-04-19');
select 213 from test.test54378 prewhere pk_date in ('2018-04-19');
select 214 from test.test54378 prewhere date in ('2018-04-19');
SELECT '-';
select 221 from test.test54378 prewhere part_date = toDate('2018-04-19');
select 222 from test.test54378 prewhere part_date in (toDate('2018-04-19'));
select 223 from test.test54378 prewhere pk_date in (toDate('2018-04-19'));
select 224 from test.test54378 prewhere date in (toDate('2018-04-19'));
SELECT '-';
select 231 from test.test54378 prewhere part_date = (SELECT toDate('2018-04-19'));
select 232 from test.test54378 prewhere part_date in (SELECT toDate('2018-04-19'));
select 233 from test.test54378 prewhere pk_date in (SELECT toDate('2018-04-19'));
select 234 from test.test54378 prewhere date in (SELECT toDate('2018-04-19'));
SELECT '---';
SET optimize_move_to_prewhere = 0;
select 311 from test.test54378 where part_date = '2018-04-19';
select 312 from test.test54378 where part_date in ('2018-04-19');
select 313 from test.test54378 where pk_date in ('2018-04-19');
select 314 from test.test54378 where date in ('2018-04-19');
SELECT '-';
select 321 from test.test54378 where part_date = toDate('2018-04-19');
select 322 from test.test54378 where part_date in (toDate('2018-04-19'));
select 323 from test.test54378 where pk_date in (toDate('2018-04-19'));
select 324 from test.test54378 where date in (toDate('2018-04-19'));
SELECT '-';
select 331 from test.test54378 where part_date = (SELECT toDate('2018-04-19'));
select 332 from test.test54378 where part_date in (SELECT toDate('2018-04-19'));
select 333 from test.test54378 where pk_date in (SELECT toDate('2018-04-19'));
select 334 from test.test54378 where date in (SELECT toDate('2018-04-19'));
SELECT '---';
SET optimize_move_to_prewhere = 1;
select 411 from test.test54378 where part_date = '2018-04-19';
select 412 from test.test54378 where part_date in ('2018-04-19');
select 413 from test.test54378 where pk_date in ('2018-04-19');
select 414 from test.test54378 where date in ('2018-04-19');
SELECT '-';
select 421 from test.test54378 where part_date = toDate('2018-04-19');
select 422 from test.test54378 where part_date in (toDate('2018-04-19'));
select 423 from test.test54378 where pk_date in (toDate('2018-04-19'));
select 424 from test.test54378 where date in (toDate('2018-04-19'));
SELECT '-';
select 431 from test.test54378 where part_date = (SELECT toDate('2018-04-19'));
select 432 from test.test54378 where part_date in (SELECT toDate('2018-04-19'));
select 433 from test.test54378 where pk_date in (SELECT toDate('2018-04-19'));
select 434 from test.test54378 where date in (SELECT toDate('2018-04-19'));
drop table test.test54378;

View File

@ -0,0 +1,15 @@
привет пр 9
0
h 1
hello 5
hello hello hello 17
п 1
пр 2
привет 6
привет привет 13
привет привет привет 20
你好 2
你好 你好 5
你好你好你好 6
你好你好你好你好 8
你好 你好 你好 你好 你好 14

View File

@ -0,0 +1,2 @@
SELECT 'привет пр' AS x, lengthUTF8(x) AS y;
SELECT x, lengthUTF8(x) AS y FROM (SELECT arrayJoin(['', 'h', 'hello', 'hello hello hello', 'п', 'пр', 'привет', 'привет привет', 'привет привет привет', '你好', '你好 你好', '你好你好你好', '你好你好你好你好', '你好 你好 你好 你好 你好']) AS x);

View File

@ -0,0 +1,12 @@
['Hello','World'] [['a'],['b','c']] [['PU','US'],['OTHER']]
['Hello','World'] [['a'],['b','c']] [['PU','US'],['OTHER']]
['GoodBye'] [['1','2']] [['PU','US','OTHER']]
['Hello','World'] [['a'],['b','c']] [['PU','US'],['OTHER']]
['GoodBye'] [['1','2']] [['PU','US','OTHER']]
['Hello','World'] [['a'],['b','c']] [['PU','US'],['OTHER']]
['GoodBye'] [['1','2']] [['PU','US','OTHER']]
['Hello','World'] [['a'],['b','c']] [['PU','US'],['OTHER']]
['Hello','World'] [['a'],['b','c']] [['PU','US'],['OTHER']]
['Hello','World'] [['a'],['b','c']] [['PU','US'],['OTHER']]
['Hello','World'] [['a'],['b','c']] [['PU','US'],['OTHER']]
['Hello','World'] [['a'],['b','c']] [['PU','US'],['OTHER']]

View File

@ -0,0 +1,99 @@
USE test;
DROP TABLE IF EXISTS nested;
CREATE TABLE nested
(
column Nested
(
name String,
names Array(String),
types Array(Enum8('PU' = 1, 'US' = 2, 'OTHER' = 3))
)
) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO nested VALUES (['Hello', 'World'], [['a'], ['b', 'c']], [['PU', 'US'], ['OTHER']]);
SELECT * FROM nested;
DETACH TABLE nested;
ATTACH TABLE nested;
SELECT * FROM nested;
INSERT INTO nested VALUES (['GoodBye'], [['1', '2']], [['PU', 'US', 'OTHER']]);
SELECT * FROM nested ORDER BY column.name;
OPTIMIZE TABLE nested PARTITION tuple() FINAL;
SELECT * FROM nested ORDER BY column.name;
DETACH TABLE nested;
ATTACH TABLE nested;
SELECT * FROM nested ORDER BY column.name;
DROP TABLE IF EXISTS nested;
CREATE TABLE nested
(
column Nested
(
name String,
names Array(String),
types Array(Enum8('PU' = 1, 'US' = 2, 'OTHER' = 3))
)
) ENGINE = Log;
INSERT INTO nested VALUES (['Hello', 'World'], [['a'], ['b', 'c']], [['PU', 'US'], ['OTHER']]);
SELECT * FROM nested;
DROP TABLE IF EXISTS nested;
CREATE TABLE nested
(
column Nested
(
name String,
names Array(String),
types Array(Enum8('PU' = 1, 'US' = 2, 'OTHER' = 3))
)
) ENGINE = TinyLog;
INSERT INTO nested VALUES (['Hello', 'World'], [['a'], ['b', 'c']], [['PU', 'US'], ['OTHER']]);
SELECT * FROM nested;
DROP TABLE IF EXISTS nested;
CREATE TABLE nested
(
column Nested
(
name String,
names Array(String),
types Array(Enum8('PU' = 1, 'US' = 2, 'OTHER' = 3))
)
) ENGINE = StripeLog;
INSERT INTO nested VALUES (['Hello', 'World'], [['a'], ['b', 'c']], [['PU', 'US'], ['OTHER']]);
SELECT * FROM nested;
DROP TABLE IF EXISTS nested;
CREATE TABLE nested
(
column Nested
(
name String,
names Array(String),
types Array(Enum8('PU' = 1, 'US' = 2, 'OTHER' = 3))
)
) ENGINE = Memory;
INSERT INTO nested VALUES (['Hello', 'World'], [['a'], ['b', 'c']], [['PU', 'US'], ['OTHER']]);
SELECT * FROM nested;
DROP TABLE nested;

2
debian/.pbuilderrc vendored
View File

@ -167,8 +167,10 @@ case "$DIST" in
export CMAKE_FLAGS="-DENABLE_EMBEDDED_COMPILER=1 -DLLVM_VERSION_POSTFIX=-6.0 $CMAKE_FLAGS"
;;
"artful" | "experimental" | "unstable" | "testing" )
if [ "$ARCH" != arm64 ]; then
EXTRAPACKAGES+=" liblld-5.0-dev libclang-5.0-dev liblld-5.0 "
export CMAKE_FLAGS="-DENABLE_EMBEDDED_COMPILER=1 $CMAKE_FLAGS"
fi
;;
esac

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (1.1.54378) unstable; urgency=low
clickhouse (1.1.54380) unstable; urgency=low
* Modified source code
-- <robot-metrika-test@yandex-team.ru> Fri, 13 Apr 2018 15:44:34 +0300
-- <robot-metrika-test@yandex-team.ru> Fri, 20 Apr 2018 22:47:20 +0300

14
debian/control vendored
View File

@ -4,7 +4,7 @@ Priority: optional
Maintainer: Alexey Milovidov <milovidov@yandex-team.ru>
Build-Depends: debhelper (>= 9),
cmake3 | cmake,
ninja-build,
ninja-build [amd64 i386],
gcc-7, g++-7,
libc6-dev,
libmariadbclient-dev | default-libmysqlclient-dev | libmysqlclient-dev,
@ -16,7 +16,7 @@ Build-Depends: debhelper (>= 9),
Standards-Version: 3.9.8
Package: clickhouse-client
Architecture: any
Architecture: all
Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-common-static (= ${binary:Version}) | clickhouse-server-base (= ${binary:Version})
Replaces: clickhouse-compressor
Conflicts: clickhouse-compressor
@ -38,7 +38,7 @@ Description: Common files for clickhouse
This package provides common files for both clickhouse server and client
Package: clickhouse-server
Architecture: any
Architecture: all
Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-common-static (= ${binary:Version}), adduser
Replaces: clickhouse-server-common, clickhouse-server-base
Provides: clickhouse-server-common
@ -60,7 +60,7 @@ Description: debugging symbols for clickhouse-common-static
Package: clickhouse-test
Priority: optional
Architecture: any
Architecture: all
Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-client, bash, expect, python, python-lxml, python-termcolor, python-requests, curl, perl, sudo, openssl
Description: Clickhouse tests
@ -72,10 +72,10 @@ Package: clickhouse-server-base
Architecture: any
Priority: optional
Depends: ${shlibs:Depends}, ${misc:Depends}, adduser, tzdata
Description: DEPRECATED PACKAGE: Server binary for clickhouse
Description: DEPRECATED PACKAGE (use clickhouse-common-static): Server binary for clickhouse
Package: clickhouse-server-common
Architecture: any
Architecture: all
Priority: optional
Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-server-base (= ${binary:Version})
Description: DEPRECATED PACKAGE: Common configuration files for clickhouse-server-base package
Description: DEPRECATED PACKAGE (use clickhouse-server): Common configuration files for clickhouse-server-base package

7
debian/rules vendored
View File

@ -53,8 +53,13 @@ ifndef DH_VERBOSE
CMAKE_FLAGS += -DCMAKE_VERBOSE_MAKEFILE=0
endif
# Useful for bulding on low memory systems
ifndef DISABLE_PARALLEL
DH_FLAGS += --parallel
endif
%:
dh $@ --parallel --buildsystem=cmake --builddirectory=$(BUILDDIR)
dh $@ $(DH_FLAGS) --buildsystem=cmake --builddirectory=$(BUILDDIR)
override_dh_auto_configure:
dh_auto_configure -- $(CMAKE_FLAGS)

View File

@ -9,7 +9,7 @@ RUN apt-get update && \
apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 && \
echo $repository | tee /etc/apt/sources.list.d/clickhouse.list && \
apt-get update && \
apt-get install --allow-unauthenticated -y "clickhouse-server|clickhouse-server-common=$version" libgcc-7-dev && \
apt-get install --allow-unauthenticated -y "clickhouse-server=$version" libgcc-7-dev && \
rm -rf /var/lib/apt/lists/* /var/cache/debconf && \
apt-get clean

View File

@ -2,7 +2,7 @@
Users and access rights are set up in the user config. This is usually `users.xml`.
Users are recorded in the 'users' section. Here is a fragment of the `users.xml` file:
Users are recorded in the `users` section. Here is a fragment of the `users.xml` file:
```xml
<!-- Users and ACL. -->
@ -55,6 +55,7 @@ Users are recorded in the 'users' section. Here is a fragment of the `users.xml`
<database>test</database>
</allow_databases>
</web>
</users>
```
You can see a declaration from two users: `default` and `web`. We added the `web` user separately.

View File

@ -25,9 +25,9 @@
<a href="{{ config.repo_url }}" rel="external nofollow" target="_blank" class="md-nav__link">
<script type="text/javascript">
if (window.location.pathname.indexOf('/ru/') >= 0) {
document.write('Исходный код');
document.write('Исходники ClickHouse');
} else {
document.write('Source code');
document.write('ClickHouse sources');
}
</script>
</a>

View File

@ -1,3 +1,5 @@
<a name="queries"></a>
# Запросы
## CREATE DATABASE

View File

@ -2,4 +2,69 @@
# clickhouse-local
Программа `clickhouse-local` позволяет выполнять быструю обработку локальных файлов, хранящих таблицы, не прибегая к развертыванию и настройке сервера ClickHouse.
Принимает на вход данные, которые можно представить в табличном виде и выполняет над ними операции, заданные на [языке запросов](../query_language/queries.md#queries) ClickHouse.
`clickhouse-local` использует движок сервера ClickHouse, т.е. поддерживает все форматы данных и движки таблиц, с которыми работает ClickHouse, при этом для выполнения операций не требуется запущенный сервер.
`clickhouse-local` при настройке по умолчанию не имеет доступа к данным, которыми управляет сервер ClickHouse, установленный на этом же хосте, однако можно подключить конфигурацию сервера с помощью ключа `--config-file`.
<div class="admonition warning">
Мы не рекомендуем подключать серверную конфигурацию к `clickhouse-local`, поскольку данные можно легко повредить неосторожными действиями.
</div>
## Вызов программы
Основной формат вызова:
``` bash
clickhouse-local --structure "table_structure" --input-format "format_of_incoming_data" -q "query"
```
Ключи команды:
- `-S`, `--structure` — структура таблицы, в которую будут помещены входящие данные.
- `-if`, `--input-format` — формат входящих данных. По умолчанию — `TSV`.
- `-f`, `--file` — путь к файлу с данными. По умолчанию — `stdin`.
- `-q` `--query` — запросы на выполнение. Разделитель запросов — `;`.
- `-N`, `--table` — имя таблицы, в которую будут помещены входящие данные. По умолчанию - `table`.
- `-of`, `--format`, `--output-format` — формат выходных данных. По умолчанию — `TSV`.
- `--stacktrace` — вывод отладочной информации при исключениях.
- `--verbose` — подробный вывод при выполнении запроса.
- `-s` — отключает вывод системных логов в `stderr`.
- `--config-file` — путь к файлу конфигурации. По умолчанию `clickhouse-local` запускается с пустой конфигурацией. Конфигурационный файл имеет тот же формат, что и для сервера ClickHouse и в нём можно использовать все конфигурационные параметры сервера. Обычно подключение конфигурации не требуется, если требуется установить отдельный параметр, то это можно сделать ключом с именем параметра.
- `--help` — вывод справочной информации о `clickhouse-local`.
## Примеры вызова
``` bash
echo -e "1,2\n3,4" | clickhouse-local -S "a Int64, b Int64" -if "CSV" -q "SELECT * FROM table"
Read 2 rows, 32.00 B in 0.000 sec., 5182 rows/sec., 80.97 KiB/sec.
1 2
3 4
```
Вызов выше эквивалентен следующему:
``` bash
$ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64) ENGINE = File(CSV, stdin); SELECT a, b FROM table; DROP TABLE table"
Read 2 rows, 32.00 B in 0.000 sec., 4987 rows/sec., 77.93 KiB/sec.
1 2
3 4
```
А теперь давайте выведем на экран объем оперативной памяти, занимаемой пользователями (Unix):
``` bash
$ ps aux | tail -n +2 | awk '{ printf("%s\t%s\n", $1, $4) }' | clickhouse-local -S "user String, mem Float64" -q "SELECT user, round(sum(mem), 2) as memTotal FROM table GROUP BY user ORDER BY memTotal DESC FORMAT Pretty"
Read 186 rows, 4.15 KiB in 0.035 sec., 5302 rows/sec., 118.34 KiB/sec.
┏━━━━━━━━━━┳━━━━━━━━━━┓
┃ user ┃ memTotal ┃
┡━━━━━━━━━━╇━━━━━━━━━━┩
│ bayonet │ 113.5 │
├──────────┼──────────┤
│ root │ 8.8 │
├──────────┼──────────┤
...
```

View File

@ -706,17 +706,18 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config)
Poco::AutoPtr<SplitterChannel> split = new SplitterChannel;
auto log_level = config.getString("logger.level", "trace");
if (config.hasProperty("logger.log"))
const auto log_path = config.getString("logger.log", "");
if (!log_path.empty())
{
createDirectory(config.getString("logger.log"));
std::cerr << "Logging " << log_level << " to " << config.getString("logger.log") << std::endl;
createDirectory(log_path);
std::cerr << "Logging " << log_level << " to " << log_path << std::endl;
// Set up two channel chains.
Poco::AutoPtr<OwnPatternFormatter> pf = new OwnPatternFormatter(this);
pf->setProperty("times", "local");
Poco::AutoPtr<FormattingChannel> log = new FormattingChannel(pf);
log_file = new FileChannel;
log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(config.getString("logger.log")).absolute().toString());
log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(log_path).absolute().toString());
log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M"));
log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number");
log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true"));
@ -728,17 +729,18 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config)
log_file->open();
}
if (config.hasProperty("logger.errorlog"))
const auto errorlog_path = config.getString("logger.errorlog", "");
if (!errorlog_path.empty())
{
createDirectory(config.getString("logger.errorlog"));
std::cerr << "Logging errors to " << config.getString("logger.errorlog") << std::endl;
createDirectory(errorlog_path);
std::cerr << "Logging errors to " << errorlog_path << std::endl;
Poco::AutoPtr<Poco::LevelFilterChannel> level = new Poco::LevelFilterChannel;
level->setLevel(Message::PRIO_NOTICE);
Poco::AutoPtr<OwnPatternFormatter> pf = new OwnPatternFormatter(this);
pf->setProperty("times", "local");
Poco::AutoPtr<FormattingChannel> errorlog = new FormattingChannel(pf);
error_log_file = new FileChannel;
error_log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(config.getString("logger.errorlog")).absolute().toString());
error_log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(errorlog_path).absolute().toString());
error_log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M"));
error_log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number");
error_log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true"));
@ -965,9 +967,9 @@ void BaseDaemon::initialize(Application & self)
}
/// Change path for logging.
if (config().hasProperty("logger.log"))
if (!log_path.empty())
{
std::string path = createDirectory(config().getString("logger.log"));
std::string path = createDirectory(log_path);
if (is_daemon
&& chdir(path.c_str()) != 0)
throw Poco::Exception("Cannot change directory to " + path);