fix conflict

This commit is contained in:
feng lv 2020-12-07 13:14:22 +00:00
commit 87bdf028fd
28 changed files with 665 additions and 428 deletions

6
.gitmodules vendored
View File

@ -142,9 +142,6 @@
[submodule "contrib/replxx"]
path = contrib/replxx
url = https://github.com/ClickHouse-Extras/replxx.git
[submodule "contrib/ryu"]
path = contrib/ryu
url = https://github.com/ClickHouse-Extras/ryu.git
[submodule "contrib/avro"]
path = contrib/avro
url = https://github.com/ClickHouse-Extras/avro.git
@ -209,3 +206,6 @@
path = contrib/abseil-cpp
url = https://github.com/ClickHouse-Extras/abseil-cpp.git
branch = lts_2020_02_25
[submodule "contrib/dragonbox"]
path = contrib/dragonbox
url = https://github.com/ClickHouse-Extras/dragonbox.git

View File

@ -35,7 +35,6 @@ add_subdirectory (libmetrohash)
add_subdirectory (lz4-cmake)
add_subdirectory (murmurhash)
add_subdirectory (replxx-cmake)
add_subdirectory (ryu-cmake)
add_subdirectory (unixodbc-cmake)
add_subdirectory (xz)
@ -322,3 +321,5 @@ endif()
if (USE_INTERNAL_ROCKSDB_LIBRARY)
add_subdirectory(rocksdb-cmake)
endif()
add_subdirectory(dragonbox)

1
contrib/dragonbox vendored Submodule

@ -0,0 +1 @@
Subproject commit b2751c65c0592c0239aec3becd53d0ea2fde9329

1
contrib/ryu vendored

@ -1 +0,0 @@
Subproject commit 5b4a853534b47438b4d97935370f6b2397137c2b

View File

@ -1,10 +0,0 @@
SET(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/ryu)
add_library(ryu
${LIBRARY_DIR}/ryu/d2fixed.c
${LIBRARY_DIR}/ryu/d2s.c
${LIBRARY_DIR}/ryu/f2s.c
${LIBRARY_DIR}/ryu/generic_128.c
)
target_include_directories(ryu SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}")

View File

@ -137,7 +137,6 @@ function clone_submodules
contrib/libxml2
contrib/poco
contrib/libunwind
contrib/ryu
contrib/fmtlib
contrib/base64
contrib/cctz
@ -155,6 +154,7 @@ function clone_submodules
contrib/croaring
contrib/miniselect
contrib/xz
contrib/dragonbox
)
git submodule sync

View File

@ -1,12 +1,36 @@
---
toc_priority: 14
toc_title: "\u266A\u64CD\u573A\u266A"
toc_title: 体验平台
---
# ClickHouse体验平台 {#clickhouse-playground}
[ClickHouse体验平台](https://play.clickhouse.tech?file=welcome) 允许人们通过即时运行查询来尝试ClickHouse而无需设置他们的服务器或集群。
体验平台中提供了几个示例数据集以及显示ClickHouse特性的示例查询。
体验平台中提供几个示例数据集以及显示ClickHouse特性的示例查询。还有一些ClickHouse LTS版本可供尝试。
ClickHouse体验平台提供了小型集群[Managed Service for ClickHouse](https://cloud.yandex.com/services/managed-clickhouse)实例配置(4 vCPU, 32 GB RAM)它们托管在[Yandex.Cloud](https://cloud.yandex.com/). 更多信息查询[cloud providers](../commercial/cloud.md).
您可以使用任何HTTP客户端对ClickHouse体验平台进行查询例如[curl](https://curl.haxx.se)或者[wget](https://www.gnu.org/software/wget/),或使用[JDBC](../interfaces/jdbc.md)或者[ODBC](../interfaces/odbc.md)驱动连接。关于支持ClickHouse的软件产品的更多信息详见[here](../interfaces/index.md).
## Credentials {#credentials}
| 参数 | 值 |
|:--------------------|:----------------------------------------|
| HTTPS端点 | `https://play-api.clickhouse.tech:8443` |
| TCP端点 | `play-api.clickhouse.tech:9440` |
| 用户 | `playground` |
| 密码 | `clickhouse` |
还有一些带有特定ClickHouse版本的附加信息来试验它们之间的差异(端口和用户/密码与上面相同):
- 20.3 LTS: `play-api-v20-3.clickhouse.tech`
- 19.14 LTS: `play-api-v19-14.clickhouse.tech`
!!! note "注意"
所有这些端点都需要安全的TLS连接。
## 查询限制 {#limitations}
查询以只读用户身份执行。 这意味着一些局限性:
@ -14,33 +38,34 @@ toc_title: "\u266A\u64CD\u573A\u266A"
- 不允许插入查询
还强制执行以下设置:
- [`max_result_bytes=10485760`](../operations/settings/query_complexity/#max-result-bytes)
- [`max_result_rows=2000`](../operations/settings/query_complexity/#setting-max_result_rows)
- [`result_overflow_mode=break`](../operations/settings/query_complexity/#result-overflow-mode)
- [`max_execution_time=60000`](../operations/settings/query_complexity/#max-execution-time)
- [max_result_bytes=10485760](../operations/settings/query-complexity/#max-result-bytes)
- [max_result_rows=2000](../operations/settings/query-complexity/#setting-max_result_rows)
- [result_overflow_mode=break](../operations/settings/query-complexity/#result-overflow-mode)
- [max_execution_time=60000](../operations/settings/query-complexity/#max-execution-time)
ClickHouse体验还有如下
[ClickHouse管理服务](https://cloud.yandex.com/services/managed-clickhouse)
实例托管 [Yandex云](https://cloud.yandex.com/)。
更多信息 [云提供商](../commercial/cloud.md)。
ClickHouse体验平台界面实际上是通过ClickHouse [HTTP API](../interfaces/http.md)接口实现的.
体验平台后端只是一个ClickHouse集群没有任何额外的服务器端应用程序。
体验平台也同样提供了ClickHouse HTTPS服务端口。
## 示例 {#examples}
您可以使用任何HTTP客户端向体验平台进行查询例如 [curl](https://curl.haxx.se) 或 [wget](https://www.gnu.org/software/wget/),或使用以下方式建立连接 [JDBC](../interfaces/jdbc.md) 或 [ODBC](../interfaces/odbc.md) 驱动。
有关支持ClickHouse的软件产品的更多信息请访问 [这里](../interfaces/index.md)。
| 参数 | 值 |
|:---------|:--------------------------------------|
| 服务端口 | https://play-api.clickhouse.tech:8443 |
| 用户 | `playground` |
| 密码 | `clickhouse` |
请注意,此服务端口需要安全连接。
示例:
使用`curl`连接Https服务
``` bash
curl "https://play-api.clickhouse.tech:8443/?query=SELECT+'Play+ClickHouse!';&user=playground&password=clickhouse&database=datasets"
curl "https://play-api.clickhouse.tech:8443/?query=SELECT+'Play+ClickHouse\!';&user=playground&password=clickhouse&database=datasets"
```
TCP连接示例[CLI](../interfaces/cli.md):
``` bash
clickhouse client --secure -h play-api.clickhouse.tech --port 9440 -u playground --password clickhouse -q "SELECT 'Play ClickHouse\!'"
```
## Implementation Details {#implementation-details}
ClickHouse体验平台界面实际上是通过ClickHouse [HTTP API](../interfaces/http.md)接口实现的。
ClickHouse体验平台是一个ClickHouse集群没有任何附加的服务器端应用程序。如上所述ClickHouse的HTTPS和TCP/TLS端点也可以作为体验平台的一部分公开使用, 代理通过[Cloudflare Spectrum](https://www.cloudflare.com/products/cloudflare-spectrum/)增加一层额外的保护和改善连接。
!!! warning "注意"
**强烈不推荐**在任何其他情况下将ClickHouse服务器暴露给公共互联网。确保它只在私有网络上侦听并由正确配置的防火墙监控。

View File

@ -1,19 +1,27 @@
---
toc_folder_title: Interfaces
toc_priority: 14
toc_title: 客户端
---
# 客户端 {#interfaces}
ClickHouse提供了两个网络接口两者都可以选择包装在TLS中以提高安全性
ClickHouse提供了两个网络接口(两个都可以选择包装在TLS中以增加安全性):
- [HTTP](http.md),记录在案,易于使用.
- [本地TCP](tcp.md),这有较少的开销.
- [HTTP](http.md), 包含文档,易于使用。
- [Native TCP](../interfaces/tcp.md),简单,方便使用。
在大多数情况下,建议使用适当的工具或库,而不是直接与这些工具或库进行交互。 Yandex的官方支持如下
\* [命令行客户端](cli.md)
\* [JDBC驱动程序](jdbc.md)
\* [ODBC驱动程序](odbc.md)
\* [C++客户端库](cpp.md)
在大多数情况下建议使用适当的工具或库而不是直接与它们交互。Yandex官方支持的项目有:
还有许多第三方库可供使用ClickHouse
\* [客户端库](third-party/client-libraries.md)
\* [集成](third-party/integrations.md)
\* [可视界面](third-party/gui.md)
- [命令行客户端](../interfaces/cli.md)
- [JDBC驱动](../interfaces/jdbc.md)
- [ODBC驱动](../interfaces/odbc.md)
- [C++客户端](../interfaces/cpp.md)
[来源文章](https://clickhouse.tech/docs/zh/interfaces/) <!--hide-->
还有一些广泛的第三方库可供ClickHouse使用:
- [客户端库](../interfaces/third-party/client-libraries.md)
- [第三方集成库](../interfaces/third-party/integrations.md)
- [可视化UI](../interfaces/third-party/gui.md)
[来源文章](https://clickhouse.tech/docs/en/interfaces/) <!--hide-->

View File

@ -242,7 +242,7 @@ target_link_libraries (clickhouse_common_io
PUBLIC
common
${DOUBLE_CONVERSION_LIBRARIES}
ryu
dragonbox_to_chars
)
if(RE2_LIBRARY)

View File

@ -37,12 +37,16 @@ void encodeSHA256(const void * text, size_t size, unsigned char * out)
String getOpenSSLErrors()
{
BIO * mem = BIO_new(BIO_s_mem());
SCOPE_EXIT(BIO_free(mem));
ERR_print_errors(mem);
char * buf = nullptr;
size_t size = BIO_get_mem_data(mem, &buf);
return String(buf, size);
String res;
ERR_print_errors_cb([](const char * str, size_t len, void * ctx)
{
String & out = *reinterpret_cast<String*>(ctx);
if (!out.empty())
out += ", ";
out.append(str, len);
return 1;
}, &res);
return res;
}
}

View File

@ -216,7 +216,6 @@ std::pair<ResponsePtr, Undo> TestKeeperCreateRequest::process(TestKeeper::Contai
if (is_sequential)
{
auto seq_num = it->second.seq_num;
++it->second.seq_num;
std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
seq_num_str.exceptions(std::ios::failbit);
@ -225,18 +224,19 @@ std::pair<ResponsePtr, Undo> TestKeeperCreateRequest::process(TestKeeper::Contai
path_created += seq_num_str.str();
}
/// Increment sequential number even if node is not sequential
++it->second.seq_num;
response.path_created = path_created;
container.emplace(path_created, std::move(created_node));
undo = [&container, path_created, is_sequential = is_sequential, parent_path = it->first]
undo = [&container, path_created, parent_path = it->first]
{
container.erase(path_created);
auto & undo_parent = container.at(parent_path);
--undo_parent.stat.cversion;
--undo_parent.stat.numChildren;
if (is_sequential)
--undo_parent.seq_num;
--undo_parent.seq_num;
};
++it->second.stat.cversion;

View File

@ -112,9 +112,12 @@ static void validateKeyTypes(const DataTypes & key_types)
if (key_types.empty() || key_types.size() > 2)
throw Exception{"Expected a single IP address or IP with mask", ErrorCodes::TYPE_MISMATCH};
const auto & actual_type = key_types[0]->getName();
if (actual_type != "UInt32" && actual_type != "FixedString(16)")
throw Exception{"Key does not match, expected either UInt32 or FixedString(16)", ErrorCodes::TYPE_MISMATCH};
const auto * key_ipv4type = typeid_cast<const DataTypeUInt32 *>(key_types[0].get());
const auto * key_ipv6type = typeid_cast<const DataTypeFixedString *>(key_types[0].get());
if (key_ipv4type == nullptr && (key_ipv6type == nullptr || key_ipv6type->getN() != 16))
throw Exception{"Key does not match, expected either `IPv4` (`UInt32`) or `IPv6` (`FixedString(16)`)",
ErrorCodes::TYPE_MISMATCH};
if (key_types.size() > 1)
{

View File

@ -82,10 +82,9 @@ struct KeyHolder<CipherMode::MySQLCompatibility>
return foldEncryptionKeyInMySQLCompatitableMode(cipher_key_size, key, folded_key);
}
~KeyHolder()
{
OPENSSL_cleanse(folded_key.data(), folded_key.size());
}
/// There is a function to clear key securely.
/// It makes absolutely zero sense to call it here because
/// key comes from column and already copied multiple times through various memory buffers.
private:
std::array<char, EVP_MAX_KEY_LENGTH> folded_key;
@ -119,7 +118,7 @@ inline void validateCipherMode(const EVP_CIPHER * evp_cipher)
}
}
throw DB::Exception("Unsupported cipher mode " + std::string(EVP_CIPHER_name(evp_cipher)), DB::ErrorCodes::BAD_ARGUMENTS);
throw DB::Exception("Unsupported cipher mode", DB::ErrorCodes::BAD_ARGUMENTS);
}
template <CipherMode mode>

View File

@ -10,7 +10,7 @@
namespace DB::S3
{
PocoHTTPResponseStream::PocoHTTPResponseStream(std::shared_ptr<Poco::Net::HTTPClientSession> session_, std::istream & response_stream_)
: Aws::IStream(response_stream_.rdbuf()), session(std::move(session_))
: Aws::IOStream(response_stream_.rdbuf()), session(std::move(session_))
{
}

View File

@ -8,7 +8,7 @@ namespace DB::S3
/**
* Wrapper of IStream to store response stream and corresponding HTTP session.
*/
class PocoHTTPResponseStream : public Aws::IStream
class PocoHTTPResponseStream : public Aws::IOStream
{
public:
PocoHTTPResponseStream(std::shared_ptr<Poco::Net::HTTPClientSession> session_, std::istream & response_stream_);

View File

@ -29,7 +29,7 @@
#include <IO/DoubleConverter.h>
#include <IO/WriteBufferFromString.h>
#include <ryu/ryu.h>
#include <dragonbox/dragonbox_to_chars.h>
#include <Formats/FormatSettings.h>
@ -228,14 +228,14 @@ inline size_t writeFloatTextFastPath(T x, char * buffer)
if (DecomposedFloat64(x).is_inside_int64())
result = itoa(Int64(x), buffer) - buffer;
else
result = d2s_buffered_n(x, buffer);
result = jkj::dragonbox::to_chars_n(x, buffer) - buffer;
}
else
{
if (DecomposedFloat32(x).is_inside_int32())
result = itoa(Int32(x), buffer) - buffer;
else
result = f2s_buffered_n(x, buffer);
result = jkj::dragonbox::to_chars_n(x, buffer) - buffer;
}
if (result <= 0)

View File

@ -80,8 +80,8 @@ target_link_libraries (parse_date_time_best_effort PRIVATE clickhouse_common_io)
add_executable (zlib_ng_bug zlib_ng_bug.cpp)
target_link_libraries (zlib_ng_bug PRIVATE ${ZLIB_LIBRARIES})
add_executable (ryu_test ryu_test.cpp)
target_link_libraries (ryu_test PRIVATE ryu)
add_executable (dragonbox_test dragonbox_test.cpp)
target_link_libraries (dragonbox_test PRIVATE dragonbox_to_chars)
add_executable (zstd_buffers zstd_buffers.cpp)
target_link_libraries (zstd_buffers PRIVATE clickhouse_common_io)

View File

@ -1,8 +1,7 @@
#include <string>
#include <iostream>
#include <cstring>
#include <ryu/ryu.h>
#include <dragonbox/dragonbox_to_chars.h>
struct DecomposedFloat64
{
@ -84,7 +83,8 @@ int main(int argc, char ** argv)
double x = argc > 1 ? std::stod(argv[1]) : 0;
char buf[32];
d2s_buffered(x, buf);
std::cout << "dragonbox output" << std::endl;
jkj::dragonbox::to_chars(x, buf);
std::cout << buf << "\n";
std::cout << DecomposedFloat64(x).isInsideInt64() << "\n";

View File

@ -349,6 +349,16 @@ void ActionsDAG::removeUnusedActions()
stack.push(node);
}
/// We cannot remove arrayJoin because it changes the number of rows.
for (auto & node : nodes)
{
if (node.type == ActionType::ARRAY_JOIN && visited_nodes.count(&node) == 0)
{
visited_nodes.insert(&node);
stack.push(&node);
}
}
while (!stack.empty())
{
auto * node = stack.top();
@ -552,6 +562,15 @@ bool ActionsDAG::hasArrayJoin() const
return false;
}
bool ActionsDAG::hasStatefulFunctions() const
{
for (const auto & node : nodes)
if (node.type == ActionType::FUNCTION && node.function_base->isStateful())
return true;
return false;
}
bool ActionsDAG::empty() const
{
for (const auto & node : nodes)
@ -670,6 +689,111 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
return actions_dag;
}
ActionsDAGPtr ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second)
{
/// first: x (1), x (2), y ==> x (2), z, x (3)
/// second: x (1), x (2), x (3) ==> x (3), x (2), x (1)
/// merge: x (1), x (2), x (3), y =(first)=> x (3), y, x (2), z, x (4) =(second)=> y, z, x (4), x (2), x (3)
/// Will store merged result in `first`.
/// This map contains nodes which should be removed from `first` index, cause they are used as inputs for `second`.
std::unordered_set<Node *> removed_first_result;
/// Map inputs of `second` to nodes of `first`.
std::unordered_map<Node *, Node *> inputs_map;
/// Update inputs list.
{
/// Index may have multiple columns with same name. They also may be used by `second`. Order is important.
std::unordered_map<std::string_view, std::list<Node *>> first_result;
for (auto & node : first.index)
first_result[node->result_name].push_back(node);
for (auto & node : second.inputs)
{
auto it = first_result.find(node->result_name);
if (it == first_result.end() || it->second.empty())
{
if (first.settings.project_input)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Cannot find column {} in ActionsDAG result", node->result_name);
first.inputs.push_back(node);
}
else
{
inputs_map[node] = it->second.front();
removed_first_result.emplace(it->second.front());
it->second.pop_front();
}
}
}
/// Replace inputs from `second` to nodes from `first` result.
for (auto & node : second.nodes)
{
for (auto & child : node.children)
{
if (child->type == ActionType::INPUT)
{
auto it = inputs_map.find(child);
if (it != inputs_map.end())
child = it->second;
}
}
}
for (auto & node : second.index)
{
if (node->type == ActionType::INPUT)
{
auto it = inputs_map.find(node);
if (it != inputs_map.end())
node = it->second;
}
}
/// Update index.
if (second.settings.project_input)
{
first.index.swap(second.index);
first.settings.project_input = true;
}
else
{
/// Remove `second` inputs from index.
for (auto it = first.index.begin(); it != first.index.end();)
{
auto cur = it;
++it;
if (removed_first_result.count(*cur))
first.index.remove(cur);
}
for (auto * node : second.index)
first.index.insert(node);
}
first.nodes.splice(first.nodes.end(), std::move(second.nodes));
#if USE_EMBEDDED_COMPILER
if (first.compilation_cache == nullptr)
first.compilation_cache = second.compilation_cache;
#endif
first.settings.max_temporary_columns = std::max(first.settings.max_temporary_columns, second.settings.max_temporary_columns);
first.settings.max_temporary_non_const_columns = std::max(first.settings.max_temporary_non_const_columns, second.settings.max_temporary_non_const_columns);
first.settings.min_count_to_compile_expression = std::max(first.settings.min_count_to_compile_expression, second.settings.min_count_to_compile_expression);
first.settings.projected_output = second.settings.projected_output;
/// Drop unused inputs and, probably, some actions.
first.removeUnusedActions();
return std::make_shared<ActionsDAG>(std::move(first));
}
ActionsDAGPtr ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_joined_columns)
{
/// Split DAG into two parts.

View File

@ -143,6 +143,15 @@ public:
map.erase(it);
}
void remove(std::list<Node *>::iterator it)
{
auto map_it = map.find((*it)->result_name);
if (map_it != map.end() && map_it->second == it)
map.erase(map_it);
list.erase(it);
}
void swap(Index & other)
{
list.swap(other.list);
@ -176,6 +185,7 @@ private:
public:
ActionsDAG() = default;
ActionsDAG(ActionsDAG &&) = default;
ActionsDAG(const ActionsDAG &) = delete;
ActionsDAG & operator=(const ActionsDAG &) = delete;
explicit ActionsDAG(const NamesAndTypesList & inputs_);
@ -222,6 +232,7 @@ public:
ActionsDAGPtr splitActionsBeforeArrayJoin(const NameSet & array_joined_columns);
bool hasArrayJoin() const;
bool hasStatefulFunctions() const;
bool empty() const; /// If actions only contain inputs.
const ActionsSettings & getSettings() const { return settings; }
@ -248,6 +259,12 @@ public:
MatchColumnsMode mode,
bool ignore_constant_values = false); /// Do not check that constants are same. Use value from result_header.
/// Create ActionsDAG which represents expression equivalent to applying lhs and rhs actions consequently.
/// Is used to replace `(first -> second)` expression chain to single `merge(first, second)` expression.
/// If first.settings.project_input is set, then outputs of `first` must include inputs of `second`.
/// Otherwise, any two actions may be combined.
static ActionsDAGPtr merge(ActionsDAG && first, ActionsDAG && second);
private:
Node & addNode(Node node, bool can_replace = false);
Node & getNode(const std::string & name);

View File

@ -484,6 +484,38 @@ static void tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Node *
: filter_step->updateInputStream(array_join_step->getOutputStream(), true);
}
static bool tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Node * child_node)
{
auto & parent = parent_node->step;
auto & child = child_node->step;
/// TODO: FilterStep
auto * parent_expr = typeid_cast<ExpressionStep *>(parent.get());
auto * child_expr = typeid_cast<ExpressionStep *>(child.get());
if (parent_expr && child_expr)
{
const auto & child_actions = child_expr->getExpression();
const auto & parent_actions = parent_expr->getExpression();
/// We cannot combine actions with arrayJoin and stateful function because we not always can reorder them.
/// Example: select rowNumberInBlock() from (select arrayJoin([1, 2]))
/// Such a query will return two zeroes if we combine actions together.
if (child_actions->hasArrayJoin() && parent_actions->hasStatefulFunctions())
return false;
auto merged = ActionsDAG::merge(std::move(*child_actions), std::move(*parent_actions));
auto expr = std::make_unique<ExpressionStep>(child_expr->getInputStreams().front(), merged);
expr->setStepDescription(parent_expr->getStepDescription() + " + " + child_expr->getStepDescription());
parent_node->step = std::move(expr);
parent_node->children.swap(child_node->children);
return true;
}
return false;
}
void QueryPlan::optimize()
{
struct Frame
@ -503,7 +535,11 @@ void QueryPlan::optimize()
{
/// First entrance, try push down.
if (frame.node->children.size() == 1)
{
tryPushDownLimit(frame.node->step, frame.node->children.front());
while (tryMergeExpressions(frame.node, frame.node->children.front()));
}
}
if (frame.next_child < frame.node->children.size())

View File

@ -50,6 +50,8 @@
1
1
1
1
1
***ipv4 trie dict mask***
1
1
@ -111,6 +113,16 @@
1
1
1
1
1
1
1
1
1
1
1
1
1
***ipv6 trie dict***
1
1
@ -418,8 +430,3 @@
1
1
1
1
1
1
1
1

View File

@ -95,6 +95,10 @@ SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(I
SELECT 11211 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('202.79.32.2')));
-- check that dictionary works with aliased types `IPv4` and `IPv6`
SELECT 11211 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(toIPv4('202.79.32.2')));
SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(toIPv6('::ffff:101.79.55.22')));
CREATE TABLE database_for_dict.table_from_ipv4_trie_dict
(
prefix String,
@ -217,6 +221,17 @@ SELECT 18 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4
SELECT 19 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.255')));
SELECT 20 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.255.128')));
SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:0')));
SELECT 4 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:1')));
SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:7f')));
SELECT 2 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:ff7f')));
SELECT 15 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:7f7f')));
SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:8009')));
SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:807f')));
SELECT 18 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:800a')));
SELECT 19 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:80ff')));
SELECT 20 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:ff80')));
SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.0')));
SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.1')));
SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.127')));
@ -446,7 +461,7 @@ FROM VALUES ('number UInt32', 5, 13, 24, 48, 49, 99, 127);
INSERT INTO database_for_dict.table_ip_trie VALUES ('101.79.55.22', 'JA');
INSERT INTO database_for_dict.table_ipv4_trie
INSERT INTO database_for_dict.table_ip_trie
SELECT
'255.255.255.255/' || toString(number) AS prefix,
toString(number) AS val
@ -607,27 +622,20 @@ SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6
SELECT '127' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe')));
SELECT '127' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff')));
SELECT '3' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.0')));
SELECT '4' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.1')));
SELECT '3' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.127')));
SELECT '2' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.255.127')));
SELECT '15' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.127.127')));
SELECT '16' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.9')));
SELECT '16' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.127')));
SELECT '18' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.10')));
SELECT '19' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.255')));
SELECT '20' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.255.128')));
SELECT '3' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:0')));
SELECT '4' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:1')));
SELECT '3' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:7f')));
SELECT '2' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:ff7f')));
SELECT '15' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:7f7f')));
SELECT '16' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:8009')));
SELECT '16' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:807f')));
SELECT '18' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:800a')));
SELECT '19' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:80ff')));
SELECT '20' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:ff80')));
SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('0.0.0.0')));
SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('128.0.0.0')));
SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('240.0.0.0')));
SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('248.0.0.0')));
SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('252.0.0.0')));
SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.240.0.0')));
SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.248.0.0')));
SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.252.0.0')));
SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.254.0')));
SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.0')));
SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.128')));
SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.248')));
SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.252')));
SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.254')));
SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.255')));
DROP DATABASE IF EXISTS database_for_dict;

View File

@ -1,9 +1,6 @@
Expression (Projection)
Expression (Projection + Before ORDER BY and SELECT)
Header: x UInt8
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
Header: dummy UInt8
1 UInt8
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Header: dummy UInt8
ReadFromStorage (SystemOne)
Header: dummy UInt8

View File

@ -1,30 +1,28 @@
(Expression)
ExpressionTransform
(Expression)
ExpressionTransform
(Aggregating)
FinalizingSimpleTransform
AggregatingSortedTransform 3 → 1
AggregatingInOrderTransform × 3
(Expression)
ExpressionTransform × 3
(SettingQuotaAndLimits)
(Expression)
ExpressionTransform × 3
(Union)
(MergingSorted)
(Expression)
ExpressionTransform
(ReadFromStorage)
MergeTree 0 → 1
(MergingSorted)
MergingSortedTransform 2 → 1
(Expression)
ExpressionTransform × 2
(ReadFromStorage)
MergeTree × 2 0 → 1
(MergingSorted)
(Expression)
ExpressionTransform
(ReadFromStorage)
MergeTree 0 → 1
(Aggregating)
FinalizingSimpleTransform
AggregatingSortedTransform 3 → 1
AggregatingInOrderTransform × 3
(Expression)
ExpressionTransform × 3
(SettingQuotaAndLimits)
(Expression)
ExpressionTransform × 3
(Union)
(MergingSorted)
(Expression)
ExpressionTransform
(ReadFromStorage)
MergeTree 0 → 1
(MergingSorted)
MergingSortedTransform 2 → 1
(Expression)
ExpressionTransform × 2
(ReadFromStorage)
MergeTree × 2 0 → 1
(MergingSorted)
(Expression)
ExpressionTransform
(ReadFromStorage)
MergeTree 0 → 1

View File

@ -1,324 +1,252 @@
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Union
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Union
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Union
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Distinct
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Union
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection)
Expression (Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)
Expression (Projection + Before ORDER BY and SELECT)
SettingQuotaAndLimits (Set limits and quota after reading from storage)
ReadFromStorage (SystemOne)

View File

@ -72,6 +72,94 @@ If you want to run only a single test such as the `/clickhouse/rbac/syntax/grant
For more information, please see [Filtering](https://testflows.com/handbook/#Filtering) section in the [TestFlows Handbook].
## How To Debug Why Test Failed
### Step 1: find which tests failed
If [TestFlows] check does not pass you should look at the end of the `test_run.txt.out.log` to find the list
of failing tests. For example,
```bash
clickhouse_testflows_tests_volume
Start tests
➤ Dec 02,2020 22:22:24 /clickhouse
...
Failing
✘ [ Fail ] /clickhouse/rbac/syntax/grant privilege/grant privileges/privilege='SELECT', on=('db0.table0', 'db0.*', '*.*', 'tb0', '*'), allow_column=True, allow_introspection=False
✘ [ Fail ] /clickhouse/rbac/syntax/grant privilege/grant privileges
✘ [ Fail ] /clickhouse/rbac/syntax/grant privilege
✘ [ Fail ] /clickhouse/rbac/syntax
✘ [ Fail ] /clickhouse/rbac
✘ [ Fail ] /clickhouse
```
In this case the failing test is
```
/clickhouse/rbac/syntax/grant privilege/grant privileges/privilege='SELECT', on=('db0.table0', 'db0.*', '*.*', 'tb0', '*'), allow_column=True, allow_introspection=False
```
while the others
```
✘ [ Fail ] /clickhouse/rbac/syntax/grant privilege/grant privileges
✘ [ Fail ] /clickhouse/rbac/syntax/grant privilege
✘ [ Fail ] /clickhouse/rbac/syntax
✘ [ Fail ] /clickhouse/rbac
✘ [ Fail ] /clickhouse
```
failed because the first fail gets "bubble-up" the test execution tree all the way to the top level test which is the
`/clickhouse`.
### Step 2: download `test.log` that contains all raw messages
You need to download the `test.log` that contains all raw messages.
### Step 3: get messages for the failing test
Once you know the name of the failing test and you have the `test.log` that contains all the raw messages
for all the tests, you can use `tfs show test messages` command.
> You get the `tfs` command by installing [TestFlows].
For example,
```bash
cat test.log | tfs show test messages "/clickhouse/rbac/syntax/grant privilege/grant privileges/privilege='SELECT', on=\('db0.table0', 'db0.\*', '\*.\*', 'tb0', '\*'\), allow_column=True, allow_introspection=False"
```
> Note: that characters that are treated as special in extended regular expressions need to be escaped. In this case
> we have to escape the `*`, `(`, and the `)` characters in the test name.
### Step 4: working with the `test.log`
You can use the `test.log` with many of the commands provided by the
`tfs` utility.
> See `tfs --help` for more information.
For example, you can get a list of failing tests from the `test.log` using the
`tfs show fails` command as follows
```bash
$ cat test.log | tfs show fails
```
or get the results using the `tfs show results` command as follows
```bash
$ cat test.log | tfs show results
```
or you can transform the log to see only the new fails using the
`tfs transform fail --new` command as follows
```bash
$ cat test.log | tfs transform fails --new
```
[Python 3]: https://www.python.org/
[Ubuntu]: https://ubuntu.com/
[TestFlows]: https://testflows.com

View File

@ -1,15 +1,19 @@
v20.11.5.18-stable 2020-12-06
v20.11.4.13-stable 2020-11-20
v20.11.3.3-stable 2020-11-13
v20.11.2.1-stable 2020-11-11
v20.10.6.27-stable 2020-12-06
v20.10.5.10-stable 2020-11-20
v20.10.4.1-stable 2020-11-13
v20.10.3.30-stable 2020-10-29
v20.10.2.20-stable 2020-10-23
v20.9.7.11-stable 2020-12-07
v20.9.6.14-stable 2020-11-20
v20.9.5.5-stable 2020-11-13
v20.9.4.76-stable 2020-10-29
v20.9.3.45-stable 2020-10-09
v20.9.2.20-stable 2020-09-22
v20.8.8.2-lts 2020-12-07
v20.8.7.15-lts 2020-11-20
v20.8.6.6-lts 2020-11-13
v20.8.5.45-lts 2020-10-29

1 v20.11.4.13-stable v20.11.5.18-stable 2020-11-20 2020-12-06
1 v20.11.5.18-stable 2020-12-06
2 v20.11.4.13-stable v20.11.4.13-stable 2020-11-20 2020-11-20
3 v20.11.3.3-stable v20.11.3.3-stable 2020-11-13 2020-11-13
4 v20.11.2.1-stable v20.11.2.1-stable 2020-11-11 2020-11-11
5 v20.10.6.27-stable 2020-12-06
6 v20.10.5.10-stable v20.10.5.10-stable 2020-11-20 2020-11-20
7 v20.10.4.1-stable v20.10.4.1-stable 2020-11-13 2020-11-13
8 v20.10.3.30-stable v20.10.3.30-stable 2020-10-29 2020-10-29
9 v20.10.2.20-stable v20.10.2.20-stable 2020-10-23 2020-10-23
10 v20.9.7.11-stable 2020-12-07
11 v20.9.6.14-stable v20.9.6.14-stable 2020-11-20 2020-11-20
12 v20.9.5.5-stable v20.9.5.5-stable 2020-11-13 2020-11-13
13 v20.9.4.76-stable v20.9.4.76-stable 2020-10-29 2020-10-29
14 v20.9.3.45-stable v20.9.3.45-stable 2020-10-09 2020-10-09
15 v20.9.2.20-stable v20.9.2.20-stable 2020-09-22 2020-09-22
16 v20.8.8.2-lts 2020-12-07
17 v20.8.7.15-lts v20.8.7.15-lts 2020-11-20 2020-11-20
18 v20.8.6.6-lts v20.8.6.6-lts 2020-11-13 2020-11-13
19 v20.8.5.45-lts v20.8.5.45-lts 2020-10-29 2020-10-29