Merge branch 'master' into break_some_tests

This commit is contained in:
mergify[bot] 2021-08-18 09:53:43 +00:00 committed by GitHub
commit f3199f6070
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
146 changed files with 4902 additions and 3873 deletions

View File

@ -628,6 +628,9 @@ cat analyze/errors.log >> report/errors.log ||:
cat profile-errors.log >> report/errors.log ||:
clickhouse-local --query "
-- We use decimals specifically to get fixed-point, fixed-width formatting.
set output_format_decimal_trailing_zeros = 1;
create view query_display_names as select * from
file('analyze/query-display-names.tsv', TSV,
'test text, query_index int, query_display_name text')
@ -975,6 +978,9 @@ for version in {right,left}
do
rm -rf data
clickhouse-local --query "
-- We use decimals specifically to get fixed-point, fixed-width formatting.
set output_format_decimal_trailing_zeros = 1;
create view query_profiles as
with 0 as left, 1 as right
select * from file('analyze/query-profiles.tsv', TSV,
@ -1170,6 +1176,9 @@ rm -rf metrics ||:
mkdir metrics
clickhouse-local --query "
-- We use decimals specifically to get fixed-point, fixed-width formatting.
set output_format_decimal_trailing_zeros = 1;
create view right_async_metric_log as
select * from file('right-async-metric-log.tsv', TSVWithNamesAndTypes,
'$(cat right-async-metric-log.tsv.columns)')

View File

@ -105,10 +105,6 @@ def process_result(result_path):
description += ", skipped: {}".format(skipped)
if unknown != 0:
description += ", unknown: {}".format(unknown)
# Temporary green for tests with DatabaseReplicated:
if 1 == int(os.environ.get('USE_DATABASE_REPLICATED', 0)):
state = "success"
else:
state = "failure"
description = "Output log doesn't exist"

View File

@ -38,9 +38,7 @@ A table for the Graphite data should have the following columns for the followin
- Value of the metric. Data type: any numeric.
- Version of the metric. Data type: any numeric.
ClickHouse saves the rows with the highest version or the last written if versions are the same. Other rows are deleted during the merge of data parts.
- Version of the metric. Data type: any numeric (ClickHouse saves the rows with the highest version or the last written if versions are the same. Other rows are deleted during the merge of data parts).
The names of these columns should be set in the rollup configuration.
@ -132,7 +130,7 @@ Fields for `pattern` and `default` sections:
- `regexp` A pattern for the metric name.
- `age` The minimum age of the data in seconds.
- `precision` How precisely to define the age of the data in seconds. Should be a divisor for 86400 (seconds in a day).
- `function` The name of the aggregating function to apply to data whose age falls within the range `[age, age + precision]`.
- `function` The name of the aggregating function to apply to data whose age falls within the range `[age, age + precision]`. Accepted functions: min / max / any / avg. The average is calculated imprecisely, like the average of the averages.
### Configuration Example {#configuration-example}
@ -169,4 +167,7 @@ Fields for `pattern` and `default` sections:
</graphite_rollup>
```
!!! warning "Warning"
Data rollup is performed during merges. Usually, for old partitions, merges are not started, so for rollup it is necessary to trigger an unscheduled merge using [optimize](../../../sql-reference/statements/optimize.md). Or use additional tools, for example [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer).
[Original article](https://clickhouse.tech/docs/en/operations/table_engines/graphitemergetree/) <!--hide-->

View File

@ -1247,6 +1247,7 @@ Default value: `/var/lib/clickhouse/access/`.
Section of the configuration file that contains settings:
- Path to configuration file with predefined users.
- Path to folder where users created by SQL commands are stored.
- ZooKeeper node path where users created by SQL commands are stored and replicated (experimental).
If this section is specified, the path from [users_config](../../operations/server-configuration-parameters/settings.md#users-config) and [access_control_path](../../operations/server-configuration-parameters/settings.md#access_control_path) won't be used.
@ -1262,6 +1263,9 @@ The `user_directories` section can contain any number of items, the order of the
<local_directory>
<path>/var/lib/clickhouse/access/</path>
</local_directory>
<replicated>
<zookeeper_path>/clickhouse/access/</zookeeper_path>
</replicated>
</user_directories>
```

View File

@ -38,9 +38,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
- Значение метрики. Тип данных: любой числовой.
- Версия метрики. Тип данных: любой числовой.
ClickHouse сохраняет строки с последней версией или последнюю записанную строку, если версии совпадают. Другие строки удаляются при слиянии кусков данных.
- Версия метрики. Тип данных: любой числовой (ClickHouse сохраняет строки с последней версией или последнюю записанную строку, если версии совпадают. Другие строки удаляются при слиянии кусков данных).
Имена этих столбцов должны быть заданы в конфигурации rollup.
@ -173,4 +171,4 @@ default
!!! warning "Внимание"
Прореживание данных производится во время слияний. Обычно для старых партций слияния не запускаются, поэтому для прореживания надо иницировать незапланированное слияние используя [optimize](../../../sql-reference/statements/optimize/). Или использовать дополнительные инструменты, например [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer).
Прореживание данных производится во время слияний. Обычно для старых партций слияния не запускаются, поэтому для прореживания надо иницировать незапланированное слияние используя [optimize](../../../sql-reference/statements/optimize.md). Или использовать дополнительные инструменты, например [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer).

View File

@ -1,13 +1,11 @@
---
machine_translated: true
machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd
toc_priority: 49
toc_title: "\u6570\u636E\u5907\u4EFD"
---
# 数据备份 {#data-backup}
尽管 [副本] (../engines/table-engines/mergetree-family/replication.md) 可以提供针对硬件的错误防护, 但是它不能预防人为操作失误: 数据的意外删除, 错误表的删除或者错误集群上表的删除, 以及导致错误数据处理或者数据损坏的软件bug. 在很多案例中,这类意外可能会影响所有的副本. ClickHouse 有内置的保护措施可以预防一些错误 — 例如, 默认情况下 [不能人工删除使用带有MergeTree引擎且包含超过50Gb数据的表] (server-configuration-parameters/settings.md#max-table-size-to-drop). 但是,这些保护措施不能覆盖所有可能情况,并且这些措施可以被绕过。
尽管 [副本](../engines/table-engines/mergetree-family/replication.md) 可以提供针对硬件的错误防护, 但是它不能预防人为操作失误: 数据的意外删除, 错误表的删除或者错误集群上表的删除, 以及导致错误数据处理或者数据损坏的软件bug. 在很多案例中,这类意外可能会影响所有的副本. ClickHouse 有内置的保护措施可以预防一些错误 — 例如, 默认情况下 [不能人工删除使用带有MergeTree引擎且包含超过50Gb数据的表](server-configuration-parameters/settings.md#max-table-size-to-drop). 但是,这些保护措施不能覆盖所有可能情况,并且这些措施可以被绕过。
为了有效地减少可能的人为错误,您应该 **提前** 仔细的准备备份和数据还原的策略.
@ -18,26 +16,26 @@ toc_title: "\u6570\u636E\u5907\u4EFD"
## 将源数据复制到其它地方 {#duplicating-source-data-somewhere-else}
通常摄入到ClickHouse的数据是通过某种持久队列传递的例如 [Apache Kafka] (https://kafka.apache.org). 在这种情况下可以配置一组额外的订阅服务器这些订阅服务器将在写入ClickHouse时读取相同的数据流并将其存储在冷存储中。 大多数公司已经有一些默认推荐的冷存储,可能是对象存储或分布式文件系统,如 [HDFS] (https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html).
通常摄入到ClickHouse的数据是通过某种持久队列传递的例如 [Apache Kafka](https://kafka.apache.org). 在这种情况下可以配置一组额外的订阅服务器这些订阅服务器将在写入ClickHouse时读取相同的数据流并将其存储在冷存储中。 大多数公司已经有一些默认推荐的冷存储,可能是对象存储或分布式文件系统,如 [HDFS](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html).
## 文件系统快照 {#filesystem-snapshots}
某些本地文件系统提供快照功能(例如, [ZFS] (https://en.wikipedia.org/wiki/ZFS)),但它们可能不是提供实时查询的最佳选择。 一个可能的解决方案是使用这种文件系统创建额外的副本,并将它们与用于`SELECT` 查询的 [分布式] (../engines/table-engines/special/distributed.md) 表分离。 任何修改数据的查询都无法访问此类副本上的快照。 作为回报,这些副本可能具有特殊的硬件配置,每个服务器附加更多的磁盘,这将是经济高效的。
某些本地文件系统提供快照功能(例如, [ZFS](https://en.wikipedia.org/wiki/ZFS)),但它们可能不是提供实时查询的最佳选择。 一个可能的解决方案是使用这种文件系统创建额外的副本,并将它们与用于`SELECT` 查询的 [分布式](../engines/table-engines/special/distributed.md) 表分离。 任何修改数据的查询都无法访问此类副本上的快照。 作为回报,这些副本可能具有特殊的硬件配置,每个服务器附加更多的磁盘,这将是经济高效的。
## clickhouse-copier {#clickhouse-copier}
[clickhouse-copier] (utilities/clickhouse-copier.md) 是一个多功能工具最初创建它是为了用于重新切分pb大小的表。 因为它能够在ClickHouse表和集群之间可靠地复制数据所以它也可用于备份和还原数据。
[clickhouse-copier](utilities/clickhouse-copier.md) 是一个多功能工具最初创建它是为了用于重新切分pb大小的表。 因为它能够在ClickHouse表和集群之间可靠地复制数据所以它也可用于备份和还原数据。
对于较小的数据量,一个简单的 `INSERT INTO ... SELECT ...` 到远程表也可以工作。
## part操作 {#manipulations-with-parts}
ClickHouse允许使用 `ALTER TABLE ... FREEZE PARTITION ...` 查询以创建表分区的本地副本。 这是利用硬链接(hardlink)到 `/var/lib/clickhouse/shadow/` 文件夹中实现的,所以它通常不会因为旧数据而占用额外的磁盘空间。 创建的文件副本不由ClickHouse服务器处理所以你可以把它们留在那里你将有一个简单的备份不需要任何额外的外部系统但它仍然容易出现硬件问题。 出于这个原因,最好将它们远程复制到另一个位置,然后删除本地副本。 分布式文件系统和对象存储仍然是一个不错的选择,但是具有足够大容量的正常附加文件服务器也可以工作(在这种情况下,传输将通过网络文件系统或者也许是 [rsync] (https://en.wikipedia.org/wiki/Rsync) 来进行).
ClickHouse允许使用 `ALTER TABLE ... FREEZE PARTITION ...` 查询以创建表分区的本地副本。 这是利用硬链接(hardlink)到 `/var/lib/clickhouse/shadow/` 文件夹中实现的,所以它通常不会因为旧数据而占用额外的磁盘空间。 创建的文件副本不由ClickHouse服务器处理所以你可以把它们留在那里你将有一个简单的备份不需要任何额外的外部系统但它仍然容易出现硬件问题。 出于这个原因,最好将它们远程复制到另一个位置,然后删除本地副本。 分布式文件系统和对象存储仍然是一个不错的选择,但是具有足够大容量的正常附加文件服务器也可以工作(在这种情况下,传输将通过网络文件系统或者也许是 [rsync](https://en.wikipedia.org/wiki/Rsync) 来进行).
数据可以使用 `ALTER TABLE ... ATTACH PARTITION ...` 从备份中恢复。
有关与分区操作相关的查询的详细信息,请参阅 [更改文档] (../sql-reference/statements/alter.md#alter_manipulations-with-partitions).
有关与分区操作相关的查询的详细信息,请参阅 [更改文档](../sql-reference/statements/alter.md#alter_manipulations-with-partitions).
第三方工具可用于自动化此方法: [clickhouse-backup] (https://github.com/AlexAkulov/clickhouse-backup).
第三方工具可用于自动化此方法: [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup).
[原始文章] (https://clickhouse.tech/docs/en/operations/backup/) <!--hide-->
[原始文章](https://clickhouse.tech/docs/en/operations/backup/) <!--hide-->

View File

@ -1,9 +1,5 @@
---
machine_translated: true
machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3
---
# 系统。data_type_families {#system_tables-data_type_families}
# system.data_type_families {#system_tables-data_type_families}
包含有关受支持的[数据类型](../../sql-reference/data-types/)的信息.

View File

@ -1,6 +1,7 @@
#include <Access/AccessControlManager.h>
#include <Access/MultipleAccessStorage.h>
#include <Access/MemoryAccessStorage.h>
#include <Access/ReplicatedAccessStorage.h>
#include <Access/UsersConfigAccessStorage.h>
#include <Access/DiskAccessStorage.h>
#include <Access/LDAPAccessStorage.h>
@ -225,6 +226,22 @@ void AccessControlManager::startPeriodicReloadingUsersConfigs()
}
}
void AccessControlManager::addReplicatedStorage(
const String & storage_name_,
const String & zookeeper_path_,
const zkutil::GetZooKeeper & get_zookeeper_function_)
{
auto storages = getStoragesPtr();
for (const auto & storage : *storages)
{
if (auto replicated_storage = typeid_cast<std::shared_ptr<ReplicatedAccessStorage>>(storage))
return;
}
auto new_storage = std::make_shared<ReplicatedAccessStorage>(storage_name_, zookeeper_path_, get_zookeeper_function_);
addStorage(new_storage);
LOG_DEBUG(getLogger(), "Added {} access storage '{}'", String(new_storage->getStorageType()), new_storage->getStorageName());
new_storage->startup();
}
void AccessControlManager::addDiskStorage(const String & directory_, bool readonly_)
{
@ -322,6 +339,11 @@ void AccessControlManager::addStoragesFromUserDirectoriesConfig(
{
addLDAPStorage(name, config, prefix);
}
else if (type == ReplicatedAccessStorage::STORAGE_TYPE)
{
String zookeeper_path = config.getString(prefix + ".zookeeper_path");
addReplicatedStorage(name, zookeeper_path, get_zookeeper_function);
}
else
throw Exception("Unknown storage type '" + type + "' at " + prefix + " in config", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
}

View File

@ -84,6 +84,10 @@ public:
/// Adds LDAPAccessStorage which allows querying remote LDAP server for user info.
void addLDAPStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & config_, const String & prefix_);
void addReplicatedStorage(const String & storage_name,
const String & zookeeper_path,
const zkutil::GetZooKeeper & get_zookeeper_function);
/// Adds storages from <users_directories> config.
void addStoragesFromUserDirectoriesConfig(const Poco::Util::AbstractConfiguration & config,
const String & key,

View File

@ -0,0 +1,175 @@
#include <Access/AccessEntityIO.h>
#include <Access/IAccessEntity.h>
#include <Access/IAccessStorage.h>
#include <Access/Quota.h>
#include <Access/Role.h>
#include <Access/RowPolicy.h>
#include <Access/SettingsProfile.h>
#include <Access/User.h>
#include <Core/Defines.h>
#include <Interpreters/InterpreterCreateQuotaQuery.h>
#include <Interpreters/InterpreterCreateRoleQuery.h>
#include <Interpreters/InterpreterCreateRowPolicyQuery.h>
#include <Interpreters/InterpreterCreateSettingsProfileQuery.h>
#include <Interpreters/InterpreterCreateUserQuery.h>
#include <Interpreters/InterpreterGrantQuery.h>
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/InterpreterShowGrantsQuery.h>
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTCreateRoleQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTCreateSettingsProfileQuery.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ParserCreateQuotaQuery.h>
#include <Parsers/ParserCreateRoleQuery.h>
#include <Parsers/ParserCreateRowPolicyQuery.h>
#include <Parsers/ParserCreateSettingsProfileQuery.h>
#include <Parsers/ParserCreateUserQuery.h>
#include <Parsers/ParserGrantQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <boost/range/algorithm/copy.hpp>
#include <boost/range/algorithm_ext/push_back.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_ACCESS_ENTITY_DEFINITION;
}
using EntityType = IAccessStorage::EntityType;
using EntityTypeInfo = IAccessStorage::EntityTypeInfo;
namespace
{
/// Special parser for the 'ATTACH access entity' queries.
class ParserAttachAccessEntity : public IParserBase
{
protected:
const char * getName() const override { return "ATTACH access entity query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
ParserCreateUserQuery create_user_p;
ParserCreateRoleQuery create_role_p;
ParserCreateRowPolicyQuery create_policy_p;
ParserCreateQuotaQuery create_quota_p;
ParserCreateSettingsProfileQuery create_profile_p;
ParserGrantQuery grant_p;
create_user_p.useAttachMode();
create_role_p.useAttachMode();
create_policy_p.useAttachMode();
create_quota_p.useAttachMode();
create_profile_p.useAttachMode();
grant_p.useAttachMode();
return create_user_p.parse(pos, node, expected) || create_role_p.parse(pos, node, expected)
|| create_policy_p.parse(pos, node, expected) || create_quota_p.parse(pos, node, expected)
|| create_profile_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected);
}
};
}
String serializeAccessEntity(const IAccessEntity & entity)
{
/// Build list of ATTACH queries.
ASTs queries;
queries.push_back(InterpreterShowCreateAccessEntityQuery::getAttachQuery(entity));
if ((entity.getType() == EntityType::USER) || (entity.getType() == EntityType::ROLE))
boost::range::push_back(queries, InterpreterShowGrantsQuery::getAttachGrantQueries(entity));
/// Serialize the list of ATTACH queries to a string.
WriteBufferFromOwnString buf;
for (const ASTPtr & query : queries)
{
formatAST(*query, buf, false, true);
buf.write(";\n", 2);
}
return buf.str();
}
AccessEntityPtr deserializeAccessEntity(const String & definition, const String & path)
{
ASTs queries;
ParserAttachAccessEntity parser;
const char * begin = definition.data(); /// begin of current query
const char * pos = begin; /// parser moves pos from begin to the end of current query
const char * end = begin + definition.size();
while (pos < end)
{
queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH));
while (isWhitespaceASCII(*pos) || *pos == ';')
++pos;
}
/// Interpret the AST to build an access entity.
std::shared_ptr<User> user;
std::shared_ptr<Role> role;
std::shared_ptr<RowPolicy> policy;
std::shared_ptr<Quota> quota;
std::shared_ptr<SettingsProfile> profile;
AccessEntityPtr res;
for (const auto & query : queries)
{
if (auto * create_user_query = query->as<ASTCreateUserQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = user = std::make_unique<User>();
InterpreterCreateUserQuery::updateUserFromQuery(*user, *create_user_query);
}
else if (auto * create_role_query = query->as<ASTCreateRoleQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = role = std::make_unique<Role>();
InterpreterCreateRoleQuery::updateRoleFromQuery(*role, *create_role_query);
}
else if (auto * create_policy_query = query->as<ASTCreateRowPolicyQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = policy = std::make_unique<RowPolicy>();
InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(*policy, *create_policy_query);
}
else if (auto * create_quota_query = query->as<ASTCreateQuotaQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = quota = std::make_unique<Quota>();
InterpreterCreateQuotaQuery::updateQuotaFromQuery(*quota, *create_quota_query);
}
else if (auto * create_profile_query = query->as<ASTCreateSettingsProfileQuery>())
{
if (res)
throw Exception("Two access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = profile = std::make_unique<SettingsProfile>();
InterpreterCreateSettingsProfileQuery::updateSettingsProfileFromQuery(*profile, *create_profile_query);
}
else if (auto * grant_query = query->as<ASTGrantQuery>())
{
if (!user && !role)
throw Exception(
"A user or role should be attached before grant in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
if (user)
InterpreterGrantQuery::updateUserFromQuery(*user, *grant_query);
else
InterpreterGrantQuery::updateRoleFromQuery(*role, *grant_query);
}
else
throw Exception("No interpreter found for query " + query->getID(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
}
if (!res)
throw Exception("No access entities attached in " + path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
return res;
}
}

View File

@ -0,0 +1,12 @@
#pragma once
#include <Access/IAccessEntity.h>
namespace DB
{
String serializeAccessEntity(const IAccessEntity & entity);
AccessEntityPtr deserializeAccessEntity(const String & definition, const String & path);
}

View File

@ -4,41 +4,20 @@
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromString.h>
#include <Access/AccessEntityIO.h>
#include <Access/User.h>
#include <Access/Role.h>
#include <Access/RowPolicy.h>
#include <Access/Quota.h>
#include <Access/SettingsProfile.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTCreateRoleQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTCreateSettingsProfileQuery.h>
#include <Parsers/ASTGrantQuery.h>
#include <Parsers/ParserCreateUserQuery.h>
#include <Parsers/ParserCreateRoleQuery.h>
#include <Parsers/ParserCreateRowPolicyQuery.h>
#include <Parsers/ParserCreateQuotaQuery.h>
#include <Parsers/ParserCreateSettingsProfileQuery.h>
#include <Parsers/ParserGrantQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Interpreters/InterpreterCreateUserQuery.h>
#include <Interpreters/InterpreterCreateRoleQuery.h>
#include <Interpreters/InterpreterCreateRowPolicyQuery.h>
#include <Interpreters/InterpreterCreateQuotaQuery.h>
#include <Interpreters/InterpreterCreateSettingsProfileQuery.h>
#include <Interpreters/InterpreterGrantQuery.h>
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/InterpreterShowGrantsQuery.h>
#include <Common/quoteString.h>
#include <Core/Defines.h>
#include <Poco/JSON/JSON.h>
#include <Poco/JSON/Object.h>
#include <Poco/JSON/Stringifier.h>
#include <boost/range/adaptor/map.hpp>
#include <boost/range/algorithm/copy.hpp>
#include <boost/range/algorithm_ext/push_back.hpp>
#include <filesystem>
#include <fstream>
@ -49,7 +28,6 @@ namespace ErrorCodes
{
extern const int DIRECTORY_DOESNT_EXIST;
extern const int FILE_DOESNT_EXIST;
extern const int INCORRECT_ACCESS_ENTITY_DEFINITION;
}
@ -58,34 +36,6 @@ namespace
using EntityType = IAccessStorage::EntityType;
using EntityTypeInfo = IAccessStorage::EntityTypeInfo;
/// Special parser for the 'ATTACH access entity' queries.
class ParserAttachAccessEntity : public IParserBase
{
protected:
const char * getName() const override { return "ATTACH access entity query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
ParserCreateUserQuery create_user_p;
ParserCreateRoleQuery create_role_p;
ParserCreateRowPolicyQuery create_policy_p;
ParserCreateQuotaQuery create_quota_p;
ParserCreateSettingsProfileQuery create_profile_p;
ParserGrantQuery grant_p;
create_user_p.useAttachMode();
create_role_p.useAttachMode();
create_policy_p.useAttachMode();
create_quota_p.useAttachMode();
create_profile_p.useAttachMode();
grant_p.useAttachMode();
return create_user_p.parse(pos, node, expected) || create_role_p.parse(pos, node, expected)
|| create_policy_p.parse(pos, node, expected) || create_quota_p.parse(pos, node, expected)
|| create_profile_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected);
}
};
/// Reads a file containing ATTACH queries and then parses it to build an access entity.
AccessEntityPtr readEntityFile(const String & file_path)
@ -96,80 +46,7 @@ namespace
readStringUntilEOF(file_contents, in);
/// Parse the file contents.
ASTs queries;
ParserAttachAccessEntity parser;
const char * begin = file_contents.data(); /// begin of current query
const char * pos = begin; /// parser moves pos from begin to the end of current query
const char * end = begin + file_contents.size();
while (pos < end)
{
queries.emplace_back(parseQueryAndMovePosition(parser, pos, end, "", true, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH));
while (isWhitespaceASCII(*pos) || *pos == ';')
++pos;
}
/// Interpret the AST to build an access entity.
std::shared_ptr<User> user;
std::shared_ptr<Role> role;
std::shared_ptr<RowPolicy> policy;
std::shared_ptr<Quota> quota;
std::shared_ptr<SettingsProfile> profile;
AccessEntityPtr res;
for (const auto & query : queries)
{
if (auto * create_user_query = query->as<ASTCreateUserQuery>())
{
if (res)
throw Exception("Two access entities in one file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = user = std::make_unique<User>();
InterpreterCreateUserQuery::updateUserFromQuery(*user, *create_user_query);
}
else if (auto * create_role_query = query->as<ASTCreateRoleQuery>())
{
if (res)
throw Exception("Two access entities in one file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = role = std::make_unique<Role>();
InterpreterCreateRoleQuery::updateRoleFromQuery(*role, *create_role_query);
}
else if (auto * create_policy_query = query->as<ASTCreateRowPolicyQuery>())
{
if (res)
throw Exception("Two access entities in one file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = policy = std::make_unique<RowPolicy>();
InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(*policy, *create_policy_query);
}
else if (auto * create_quota_query = query->as<ASTCreateQuotaQuery>())
{
if (res)
throw Exception("Two access entities are attached in the same file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = quota = std::make_unique<Quota>();
InterpreterCreateQuotaQuery::updateQuotaFromQuery(*quota, *create_quota_query);
}
else if (auto * create_profile_query = query->as<ASTCreateSettingsProfileQuery>())
{
if (res)
throw Exception("Two access entities are attached in the same file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
res = profile = std::make_unique<SettingsProfile>();
InterpreterCreateSettingsProfileQuery::updateSettingsProfileFromQuery(*profile, *create_profile_query);
}
else if (auto * grant_query = query->as<ASTGrantQuery>())
{
if (!user && !role)
throw Exception("A user or role should be attached before grant in file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
if (user)
InterpreterGrantQuery::updateUserFromQuery(*user, *grant_query);
else
InterpreterGrantQuery::updateRoleFromQuery(*role, *grant_query);
}
else
throw Exception("No interpreter found for query " + query->getID(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
}
if (!res)
throw Exception("No access entities attached in file " + file_path, ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION);
return res;
return deserializeAccessEntity(file_contents, file_path);
}
@ -186,24 +63,10 @@ namespace
}
}
/// Writes ATTACH queries for building a specified access entity to a file.
void writeEntityFile(const String & file_path, const IAccessEntity & entity)
{
/// Build list of ATTACH queries.
ASTs queries;
queries.push_back(InterpreterShowCreateAccessEntityQuery::getAttachQuery(entity));
if ((entity.getType() == EntityType::USER) || (entity.getType() == EntityType::ROLE))
boost::range::push_back(queries, InterpreterShowGrantsQuery::getAttachGrantQueries(entity));
/// Serialize the list of ATTACH queries to a string.
WriteBufferFromOwnString buf;
for (const ASTPtr & query : queries)
{
formatAST(*query, buf, false, true);
buf.write(";\n", 2);
}
String file_contents = buf.str();
String file_contents = serializeAccessEntity(entity);
/// First we save *.tmp file and then we rename if everything's ok.
auto tmp_file_path = std::filesystem::path{file_path}.replace_extension(".tmp");

View File

@ -0,0 +1,618 @@
#include <Access/AccessEntityIO.h>
#include <Access/MemoryAccessStorage.h>
#include <Access/ReplicatedAccessStorage.h>
#include <IO/ReadHelpers.h>
#include <boost/container/flat_set.hpp>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/ZooKeeper/Types.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/escapeForFileName.h>
#include <common/range.h>
#include <common/sleep.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NO_ZOOKEEPER;
}
static UUID parseUUID(const String & text)
{
UUID uuid = UUIDHelpers::Nil;
auto buffer = ReadBufferFromMemory(text.data(), text.length());
readUUIDText(uuid, buffer);
return uuid;
}
ReplicatedAccessStorage::ReplicatedAccessStorage(
const String & storage_name_,
const String & zookeeper_path_,
zkutil::GetZooKeeper get_zookeeper_)
: IAccessStorage(storage_name_)
, zookeeper_path(zookeeper_path_)
, get_zookeeper(get_zookeeper_)
{
if (zookeeper_path.empty())
throw Exception("ZooKeeper path must be non-empty", ErrorCodes::BAD_ARGUMENTS);
if (zookeeper_path.back() == '/')
zookeeper_path.resize(zookeeper_path.size() - 1);
/// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it.
if (zookeeper_path.front() != '/')
zookeeper_path = "/" + zookeeper_path;
}
ReplicatedAccessStorage::~ReplicatedAccessStorage()
{
ReplicatedAccessStorage::shutdown();
}
void ReplicatedAccessStorage::startup()
{
initializeZookeeper();
worker_thread = ThreadFromGlobalPool(&ReplicatedAccessStorage::runWorkerThread, this);
}
void ReplicatedAccessStorage::shutdown()
{
bool prev_stop_flag = stop_flag.exchange(true);
if (!prev_stop_flag)
{
/// Notify the worker thread to stop waiting for new queue items
refresh_queue.push(UUIDHelpers::Nil);
worker_thread.join();
}
}
template <typename Func>
static void retryOnZooKeeperUserError(size_t attempts, Func && function)
{
while (attempts > 0)
{
try
{
function();
return;
}
catch (zkutil::KeeperException & keeper_exception)
{
if (Coordination::isUserError(keeper_exception.code) && attempts > 1)
attempts -= 1;
else
throw;
}
}
}
UUID ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists)
{
const UUID id = generateRandomID();
const EntityTypeInfo type_info = EntityTypeInfo::get(new_entity->getType());
const String & name = new_entity->getName();
LOG_DEBUG(getLogger(), "Inserting entity of type {} named {} with id {}", type_info.name, name, toString(id));
auto zookeeper = get_zookeeper();
retryOnZooKeeperUserError(10, [&]{ insertZooKeeper(zookeeper, id, new_entity, replace_if_exists); });
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
refreshEntityNoLock(zookeeper, id, notifications);
return id;
}
void ReplicatedAccessStorage::insertZooKeeper(
const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists)
{
const String & name = new_entity->getName();
const EntityType type = new_entity->getType();
const EntityTypeInfo type_info = EntityTypeInfo::get(type);
const String entity_uuid = toString(id);
/// The entity data will be stored here, this ensures all entities have unique ids
const String entity_path = zookeeper_path + "/uuid/" + entity_uuid;
/// Then we create a znode with the entity name, inside the znode of each entity type
/// This ensure all entities of the same type have a unique name
const String name_path = zookeeper_path + "/" + type_info.unique_char + "/" + escapeForFileName(name);
Coordination::Requests ops;
const String new_entity_definition = serializeAccessEntity(*new_entity);
ops.emplace_back(zkutil::makeCreateRequest(entity_path, new_entity_definition, zkutil::CreateMode::Persistent));
/// The content of the "name" znode is the uuid of the entity owning that name
ops.emplace_back(zkutil::makeCreateRequest(name_path, entity_uuid, zkutil::CreateMode::Persistent));
Coordination::Responses responses;
const Coordination::Error res = zookeeper->tryMulti(ops, responses);
if (res == Coordination::Error::ZNODEEXISTS)
{
if (responses[0]->error == Coordination::Error::ZNODEEXISTS)
{
/// The UUID already exists, simply fail.
/// To fail with a nice error message, we need info about what already exists.
/// This itself could fail if the conflicting uuid disappears in the meantime.
/// If that happens, then we'll just retry from the start.
String existing_entity_definition = zookeeper->get(entity_path);
AccessEntityPtr existing_entity = deserializeAccessEntity(existing_entity_definition, entity_path);
EntityType existing_type = existing_entity->getType();
String existing_name = existing_entity->getName();
throwIDCollisionCannotInsert(id, type, name, existing_type, existing_name);
}
else if (replace_if_exists)
{
/// The name already exists for this type.
/// If asked to, we need to replace the existing entity.
/// First get the uuid of the existing entity
/// This itself could fail if the conflicting name disappears in the meantime.
/// If that happens, then we'll just retry from the start.
Coordination::Stat name_stat;
String existing_entity_uuid = zookeeper->get(name_path, &name_stat);
const String existing_entity_path = zookeeper_path + "/uuid/" + existing_entity_uuid;
Coordination::Requests replace_ops;
replace_ops.emplace_back(zkutil::makeRemoveRequest(existing_entity_path, -1));
replace_ops.emplace_back(zkutil::makeCreateRequest(entity_path, new_entity_definition, zkutil::CreateMode::Persistent));
replace_ops.emplace_back(zkutil::makeSetRequest(name_path, entity_uuid, name_stat.version));
/// If this fails, then we'll just retry from the start.
zookeeper->multi(replace_ops);
}
else
{
throwNameCollisionCannotInsert(type, name);
}
}
else
{
zkutil::KeeperMultiException::check(res, ops, responses);
}
}
void ReplicatedAccessStorage::removeImpl(const UUID & id)
{
LOG_DEBUG(getLogger(), "Removing entity {}", toString(id));
auto zookeeper = get_zookeeper();
retryOnZooKeeperUserError(10, [&] { removeZooKeeper(zookeeper, id); });
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
removeEntityNoLock(id, notifications);
}
void ReplicatedAccessStorage::removeZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id)
{
const String entity_uuid = toString(id);
const String entity_path = zookeeper_path + "/uuid/" + entity_uuid;
String entity_definition;
Coordination::Stat entity_stat;
const bool uuid_exists = zookeeper->tryGet(entity_path, entity_definition, &entity_stat);
if (!uuid_exists)
throwNotFound(id);
const AccessEntityPtr entity = deserializeAccessEntity(entity_definition, entity_path);
const EntityTypeInfo type_info = EntityTypeInfo::get(entity->getType());
const String & name = entity->getName();
const String entity_name_path = zookeeper_path + "/" + type_info.unique_char + "/" + escapeForFileName(name);
Coordination::Requests ops;
ops.emplace_back(zkutil::makeRemoveRequest(entity_path, entity_stat.version));
ops.emplace_back(zkutil::makeRemoveRequest(entity_name_path, -1));
/// If this fails, then we'll just retry from the start.
zookeeper->multi(ops);
}
void ReplicatedAccessStorage::updateImpl(const UUID & id, const UpdateFunc & update_func)
{
LOG_DEBUG(getLogger(), "Updating entity {}", toString(id));
auto zookeeper = get_zookeeper();
retryOnZooKeeperUserError(10, [&] { updateZooKeeper(zookeeper, id, update_func); });
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
refreshEntityNoLock(zookeeper, id, notifications);
}
void ReplicatedAccessStorage::updateZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const UpdateFunc & update_func)
{
const String entity_uuid = toString(id);
const String entity_path = zookeeper_path + "/uuid/" + entity_uuid;
String old_entity_definition;
Coordination::Stat stat;
const bool uuid_exists = zookeeper->tryGet(entity_path, old_entity_definition, &stat);
if (!uuid_exists)
throwNotFound(id);
const AccessEntityPtr old_entity = deserializeAccessEntity(old_entity_definition, entity_path);
const AccessEntityPtr new_entity = update_func(old_entity);
if (!new_entity->isTypeOf(old_entity->getType()))
throwBadCast(id, new_entity->getType(), new_entity->getName(), old_entity->getType());
const EntityTypeInfo type_info = EntityTypeInfo::get(new_entity->getType());
Coordination::Requests ops;
const String new_entity_definition = serializeAccessEntity(*new_entity);
ops.emplace_back(zkutil::makeSetRequest(entity_path, new_entity_definition, stat.version));
const String & old_name = old_entity->getName();
const String & new_name = new_entity->getName();
if (new_name != old_name)
{
auto old_name_path = zookeeper_path + "/" + type_info.unique_char + "/" + escapeForFileName(old_name);
auto new_name_path = zookeeper_path + "/" + type_info.unique_char + "/" + escapeForFileName(new_name);
ops.emplace_back(zkutil::makeRemoveRequest(old_name_path, -1));
ops.emplace_back(zkutil::makeCreateRequest(new_name_path, entity_uuid, zkutil::CreateMode::Persistent));
}
Coordination::Responses responses;
const Coordination::Error res = zookeeper->tryMulti(ops, responses);
if (res == Coordination::Error::ZNODEEXISTS)
{
throwNameCollisionCannotRename(new_entity->getType(), old_name, new_name);
}
else if (res == Coordination::Error::ZNONODE)
{
throwNotFound(id);
}
else
{
zkutil::KeeperMultiException::check(res, ops, responses);
}
}
void ReplicatedAccessStorage::runWorkerThread()
{
LOG_DEBUG(getLogger(), "Started worker thread");
while (!stop_flag)
{
try
{
if (!initialized)
initializeZookeeper();
refresh();
}
catch (...)
{
tryLogCurrentException(getLogger(), "Unexpected error, will try to restart worker thread:");
resetAfterError();
sleepForSeconds(5);
}
}
}
void ReplicatedAccessStorage::resetAfterError()
{
initialized = false;
UUID id;
while (refresh_queue.tryPop(id)) {}
std::lock_guard lock{mutex};
for (const auto type : collections::range(EntityType::MAX))
entries_by_name_and_type[static_cast<size_t>(type)].clear();
entries_by_id.clear();
}
void ReplicatedAccessStorage::initializeZookeeper()
{
assert(!initialized);
auto zookeeper = get_zookeeper();
if (!zookeeper)
throw Exception("Can't have Replicated access without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);
createRootNodes(zookeeper);
refreshEntities(zookeeper);
initialized = true;
}
void ReplicatedAccessStorage::createRootNodes(const zkutil::ZooKeeperPtr & zookeeper)
{
zookeeper->createAncestors(zookeeper_path);
zookeeper->createIfNotExists(zookeeper_path, "");
zookeeper->createIfNotExists(zookeeper_path + "/uuid", "");
for (const auto type : collections::range(EntityType::MAX))
{
/// Create a znode for each type of AccessEntity
const auto type_info = EntityTypeInfo::get(type);
zookeeper->createIfNotExists(zookeeper_path + "/" + type_info.unique_char, "");
}
}
void ReplicatedAccessStorage::refresh()
{
UUID id;
if (refresh_queue.tryPop(id, /* timeout_ms: */ 10000))
{
if (stop_flag)
return;
auto zookeeper = get_zookeeper();
if (id == UUIDHelpers::Nil)
refreshEntities(zookeeper);
else
refreshEntity(zookeeper, id);
}
}
void ReplicatedAccessStorage::refreshEntities(const zkutil::ZooKeeperPtr & zookeeper)
{
LOG_DEBUG(getLogger(), "Refreshing entities list");
const String zookeeper_uuids_path = zookeeper_path + "/uuid";
auto watch_entities_list = [this](const Coordination::WatchResponse &)
{
refresh_queue.push(UUIDHelpers::Nil);
};
Coordination::Stat stat;
const auto entity_uuid_strs = zookeeper->getChildrenWatch(zookeeper_uuids_path, &stat, watch_entities_list);
std::unordered_set<UUID> entity_uuids;
entity_uuids.reserve(entity_uuid_strs.size());
for (const String & entity_uuid_str : entity_uuid_strs)
entity_uuids.insert(parseUUID(entity_uuid_str));
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
std::vector<UUID> entities_to_remove;
/// Locally remove entities that were removed from ZooKeeper
for (const auto & pair : entries_by_id)
{
const UUID & entity_uuid = pair.first;
if (!entity_uuids.contains(entity_uuid))
entities_to_remove.push_back(entity_uuid);
}
for (const auto & entity_uuid : entities_to_remove)
removeEntityNoLock(entity_uuid, notifications);
/// Locally add entities that were added to ZooKeeper
for (const auto & entity_uuid : entity_uuids)
{
const auto it = entries_by_id.find(entity_uuid);
if (it == entries_by_id.end())
refreshEntityNoLock(zookeeper, entity_uuid, notifications);
}
LOG_DEBUG(getLogger(), "Refreshing entities list finished");
}
void ReplicatedAccessStorage::refreshEntity(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id)
{
Notifications notifications;
SCOPE_EXIT({ notify(notifications); });
std::lock_guard lock{mutex};
refreshEntityNoLock(zookeeper, id, notifications);
}
void ReplicatedAccessStorage::refreshEntityNoLock(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, Notifications & notifications)
{
LOG_DEBUG(getLogger(), "Refreshing entity {}", toString(id));
const auto watch_entity = [this, id](const Coordination::WatchResponse & response)
{
if (response.type == Coordination::Event::CHANGED)
refresh_queue.push(id);
};
Coordination::Stat entity_stat;
const String entity_path = zookeeper_path + "/uuid/" + toString(id);
String entity_definition;
const bool exists = zookeeper->tryGetWatch(entity_path, entity_definition, &entity_stat, watch_entity);
if (exists)
{
const AccessEntityPtr entity = deserializeAccessEntity(entity_definition, entity_path);
setEntityNoLock(id, entity, notifications);
}
else
{
removeEntityNoLock(id, notifications);
}
}
void ReplicatedAccessStorage::setEntityNoLock(const UUID & id, const AccessEntityPtr & entity, Notifications & notifications)
{
LOG_DEBUG(getLogger(), "Setting id {} to entity named {}", toString(id), entity->getName());
const EntityType type = entity->getType();
const String & name = entity->getName();
/// If the type+name already exists and is a different entity, remove old entity
auto & entries_by_name = entries_by_name_and_type[static_cast<size_t>(type)];
if (auto it = entries_by_name.find(name); it != entries_by_name.end() && it->second->id != id)
{
removeEntityNoLock(it->second->id, notifications);
}
/// If the entity already exists under a different type+name, remove old type+name
if (auto it = entries_by_id.find(id); it != entries_by_id.end())
{
const AccessEntityPtr & existing_entity = it->second.entity;
const EntityType existing_type = existing_entity->getType();
const String & existing_name = existing_entity->getName();
if (existing_type != type || existing_name != name)
{
auto & existing_entries_by_name = entries_by_name_and_type[static_cast<size_t>(existing_type)];
existing_entries_by_name.erase(existing_name);
}
}
auto & entry = entries_by_id[id];
entry.id = id;
entry.entity = entity;
entries_by_name[name] = &entry;
prepareNotifications(entry, false, notifications);
}
void ReplicatedAccessStorage::removeEntityNoLock(const UUID & id, Notifications & notifications)
{
LOG_DEBUG(getLogger(), "Removing entity with id {}", toString(id));
const auto it = entries_by_id.find(id);
if (it == entries_by_id.end())
{
LOG_DEBUG(getLogger(), "Id {} not found, ignoring removal", toString(id));
return;
}
const Entry & entry = it->second;
const EntityType type = entry.entity->getType();
const String & name = entry.entity->getName();
prepareNotifications(entry, true, notifications);
auto & entries_by_name = entries_by_name_and_type[static_cast<size_t>(type)];
const auto name_it = entries_by_name.find(name);
if (name_it == entries_by_name.end())
LOG_WARNING(getLogger(), "Entity {} not found in names, ignoring removal of name", toString(id));
else if (name_it->second != &(it->second))
LOG_WARNING(getLogger(), "Name {} not pointing to entity {}, ignoring removal of name", name, toString(id));
else
entries_by_name.erase(name);
entries_by_id.erase(id);
LOG_DEBUG(getLogger(), "Removed entity with id {}", toString(id));
}
std::optional<UUID> ReplicatedAccessStorage::findImpl(EntityType type, const String & name) const
{
std::lock_guard lock{mutex};
const auto & entries_by_name = entries_by_name_and_type[static_cast<size_t>(type)];
const auto it = entries_by_name.find(name);
if (it == entries_by_name.end())
return {};
const Entry * entry = it->second;
return entry->id;
}
std::vector<UUID> ReplicatedAccessStorage::findAllImpl(EntityType type) const
{
std::lock_guard lock{mutex};
std::vector<UUID> result;
result.reserve(entries_by_id.size());
for (const auto & [id, entry] : entries_by_id)
if (entry.entity->isTypeOf(type))
result.emplace_back(id);
return result;
}
bool ReplicatedAccessStorage::existsImpl(const UUID & id) const
{
std::lock_guard lock{mutex};
return entries_by_id.count(id);
}
AccessEntityPtr ReplicatedAccessStorage::readImpl(const UUID & id) const
{
std::lock_guard lock{mutex};
const auto it = entries_by_id.find(id);
if (it == entries_by_id.end())
throwNotFound(id);
const Entry & entry = it->second;
return entry.entity;
}
String ReplicatedAccessStorage::readNameImpl(const UUID & id) const
{
return readImpl(id)->getName();
}
void ReplicatedAccessStorage::prepareNotifications(const Entry & entry, bool remove, Notifications & notifications) const
{
const AccessEntityPtr entity = remove ? nullptr : entry.entity;
for (const auto & handler : entry.handlers_by_id)
notifications.push_back({handler, entry.id, entity});
for (const auto & handler : handlers_by_type[static_cast<size_t>(entry.entity->getType())])
notifications.push_back({handler, entry.id, entity});
}
scope_guard ReplicatedAccessStorage::subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const
{
std::lock_guard lock{mutex};
auto & handlers = handlers_by_type[static_cast<size_t>(type)];
handlers.push_back(handler);
auto handler_it = std::prev(handlers.end());
return [this, type, handler_it]
{
std::lock_guard lock2{mutex};
auto & handlers2 = handlers_by_type[static_cast<size_t>(type)];
handlers2.erase(handler_it);
};
}
scope_guard ReplicatedAccessStorage::subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const
{
std::lock_guard lock{mutex};
const auto it = entries_by_id.find(id);
if (it == entries_by_id.end())
return {};
const Entry & entry = it->second;
auto handler_it = entry.handlers_by_id.insert(entry.handlers_by_id.end(), handler);
return [this, id, handler_it]
{
std::lock_guard lock2{mutex};
auto it2 = entries_by_id.find(id);
if (it2 != entries_by_id.end())
{
const Entry & entry2 = it2->second;
entry2.handlers_by_id.erase(handler_it);
}
};
}
bool ReplicatedAccessStorage::hasSubscriptionImpl(const UUID & id) const
{
std::lock_guard lock{mutex};
const auto & it = entries_by_id.find(id);
if (it != entries_by_id.end())
{
const Entry & entry = it->second;
return !entry.handlers_by_id.empty();
}
return false;
}
bool ReplicatedAccessStorage::hasSubscriptionImpl(EntityType type) const
{
std::lock_guard lock{mutex};
const auto & handlers = handlers_by_type[static_cast<size_t>(type)];
return !handlers.empty();
}
}

View File

@ -0,0 +1,87 @@
#pragma once
#include <Access/IAccessStorage.h>
#include <Common/ThreadPool.h>
#include <Common/ZooKeeper/Common.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <common/scope_guard.h>
#include <Coordination/ThreadSafeQueue.h>
#include <atomic>
#include <list>
#include <memory>
#include <mutex>
#include <unordered_map>
namespace DB
{
/// Implementation of IAccessStorage which keeps all data in zookeeper.
class ReplicatedAccessStorage : public IAccessStorage
{
public:
static constexpr char STORAGE_TYPE[] = "replicated";
ReplicatedAccessStorage(const String & storage_name, const String & zookeeper_path, zkutil::GetZooKeeper get_zookeeper);
virtual ~ReplicatedAccessStorage() override;
const char * getStorageType() const override { return STORAGE_TYPE; }
virtual void startup();
virtual void shutdown();
private:
String zookeeper_path;
zkutil::GetZooKeeper get_zookeeper;
std::atomic<bool> initialized = false;
std::atomic<bool> stop_flag = false;
ThreadFromGlobalPool worker_thread;
ThreadSafeQueue<UUID> refresh_queue;
UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override;
void removeImpl(const UUID & id) override;
void updateImpl(const UUID & id, const UpdateFunc & update_func) override;
void insertZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists);
void removeZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id);
void updateZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const UpdateFunc & update_func);
void runWorkerThread();
void resetAfterError();
void initializeZookeeper();
void createRootNodes(const zkutil::ZooKeeperPtr & zookeeper);
void refresh();
void refreshEntities(const zkutil::ZooKeeperPtr & zookeeper);
void refreshEntity(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id);
void refreshEntityNoLock(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, Notifications & notifications);
void setEntityNoLock(const UUID & id, const AccessEntityPtr & entity, Notifications & notifications);
void removeEntityNoLock(const UUID & id, Notifications & notifications);
struct Entry
{
UUID id;
AccessEntityPtr entity;
mutable std::list<OnChangedHandler> handlers_by_id;
};
std::optional<UUID> findImpl(EntityType type, const String & name) const override;
std::vector<UUID> findAllImpl(EntityType type) const override;
bool existsImpl(const UUID & id) const override;
AccessEntityPtr readImpl(const UUID & id) const override;
String readNameImpl(const UUID & id) const override;
bool canInsertImpl(const AccessEntityPtr &) const override { return true; }
void prepareNotifications(const Entry & entry, bool remove, Notifications & notifications) const;
scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override;
scope_guard subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const override;
bool hasSubscriptionImpl(const UUID & id) const override;
bool hasSubscriptionImpl(EntityType type) const override;
mutable std::mutex mutex;
std::unordered_map<UUID, Entry> entries_by_id;
std::unordered_map<String, Entry *> entries_by_name_and_type[static_cast<size_t>(EntityType::MAX)];
mutable std::list<OnChangedHandler> handlers_by_type[static_cast<size_t>(EntityType::MAX)];
};
}

View File

@ -10,6 +10,7 @@ PEERDIR(
SRCS(
AccessControlManager.cpp
AccessEntityIO.cpp
AccessRights.cpp
AccessRightsElement.cpp
AllowedClientHosts.cpp
@ -34,6 +35,7 @@ SRCS(
Quota.cpp
QuotaCache.cpp
QuotaUsage.cpp
ReplicatedAccessStorage.cpp
Role.cpp
RoleCache.cpp
RolesOrUsersSet.cpp

View File

@ -20,7 +20,7 @@ template <typename T>
static inline void writeQuoted(const DecimalField<T> & x, WriteBuffer & buf)
{
writeChar('\'', buf);
writeText(x.getValue(), x.getScale(), buf);
writeText(x.getValue(), x.getScale(), buf, {});
writeChar('\'', buf);
}

View File

@ -26,7 +26,7 @@ template <typename T>
static inline void writeQuoted(const DecimalField<T> & x, WriteBuffer & buf)
{
writeChar('\'', buf);
writeText(x.getValue(), x.getScale(), buf);
writeText(x.getValue(), x.getScale(), buf, {});
writeChar('\'', buf);
}

View File

@ -277,5 +277,4 @@ GTEST_TEST(WideInteger, DecimalFormatting)
Int128 fractional = DecimalUtils::getFractionalPart(x, 2);
EXPECT_EQ(fractional, 40);
EXPECT_EQ(decimalFractional(fractional, 2), "40");
}

View File

@ -23,65 +23,99 @@ inline Field getBinaryValue(UInt8 type, ReadBuffer & buf)
{
switch (type)
{
case Field::Types::Null: {
return DB::Field();
case Field::Types::Null:
{
return Field();
}
case Field::Types::UInt64: {
case Field::Types::UInt64:
{
UInt64 value;
DB::readVarUInt(value, buf);
readVarUInt(value, buf);
return value;
}
case Field::Types::UInt128: {
case Field::Types::UInt128:
{
UInt128 value;
DB::readBinary(value, buf);
readBinary(value, buf);
return value;
}
case Field::Types::Int64: {
case Field::Types::UInt256:
{
UInt256 value;
readBinary(value, buf);
return value;
}
case Field::Types::UUID:
{
UUID value;
readBinary(value, buf);
return value;
}
case Field::Types::Int64:
{
Int64 value;
DB::readVarInt(value, buf);
readVarInt(value, buf);
return value;
}
case Field::Types::Float64: {
case Field::Types::Int128:
{
Int128 value;
readBinary(value, buf);
return value;
}
case Field::Types::Int256:
{
Int256 value;
readBinary(value, buf);
return value;
}
case Field::Types::Float64:
{
Float64 value;
DB::readFloatBinary(value, buf);
readFloatBinary(value, buf);
return value;
}
case Field::Types::String: {
case Field::Types::String:
{
std::string value;
DB::readStringBinary(value, buf);
readStringBinary(value, buf);
return value;
}
case Field::Types::Array: {
case Field::Types::Array:
{
Array value;
DB::readBinary(value, buf);
readBinary(value, buf);
return value;
}
case Field::Types::Tuple: {
case Field::Types::Tuple:
{
Tuple value;
DB::readBinary(value, buf);
readBinary(value, buf);
return value;
}
case Field::Types::Map: {
case Field::Types::Map:
{
Map value;
DB::readBinary(value, buf);
readBinary(value, buf);
return value;
}
case Field::Types::AggregateFunctionState: {
case Field::Types::AggregateFunctionState:
{
AggregateFunctionStateData value;
DB::readStringBinary(value.name, buf);
DB::readStringBinary(value.data, buf);
readStringBinary(value.name, buf);
readStringBinary(value.data, buf);
return value;
}
}
return DB::Field();
return Field();
}
void readBinary(Array & x, ReadBuffer & buf)
{
size_t size;
UInt8 type;
DB::readBinary(type, buf);
DB::readBinary(size, buf);
readBinary(type, buf);
readBinary(size, buf);
for (size_t index = 0; index < size; ++index)
x.push_back(getBinaryValue(type, buf));
@ -93,8 +127,8 @@ void writeBinary(const Array & x, WriteBuffer & buf)
size_t size = x.size();
if (size)
type = x.front().getType();
DB::writeBinary(type, buf);
DB::writeBinary(size, buf);
writeBinary(type, buf);
writeBinary(size, buf);
for (const auto & elem : x)
Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, elem);
@ -102,19 +136,19 @@ void writeBinary(const Array & x, WriteBuffer & buf)
void writeText(const Array & x, WriteBuffer & buf)
{
DB::String res = applyVisitor(FieldVisitorToString(), DB::Field(x));
String res = applyVisitor(FieldVisitorToString(), Field(x));
buf.write(res.data(), res.size());
}
void readBinary(Tuple & x, ReadBuffer & buf)
{
size_t size;
DB::readBinary(size, buf);
readBinary(size, buf);
for (size_t index = 0; index < size; ++index)
{
UInt8 type;
DB::readBinary(type, buf);
readBinary(type, buf);
x.push_back(getBinaryValue(type, buf));
}
}
@ -122,30 +156,30 @@ void readBinary(Tuple & x, ReadBuffer & buf)
void writeBinary(const Tuple & x, WriteBuffer & buf)
{
const size_t size = x.size();
DB::writeBinary(size, buf);
writeBinary(size, buf);
for (const auto & elem : x)
{
const UInt8 type = elem.getType();
DB::writeBinary(type, buf);
writeBinary(type, buf);
Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, elem);
}
}
void writeText(const Tuple & x, WriteBuffer & buf)
{
writeFieldText(DB::Field(x), buf);
writeFieldText(Field(x), buf);
}
void readBinary(Map & x, ReadBuffer & buf)
{
size_t size;
DB::readBinary(size, buf);
readBinary(size, buf);
for (size_t index = 0; index < size; ++index)
{
UInt8 type;
DB::readBinary(type, buf);
readBinary(type, buf);
x.push_back(getBinaryValue(type, buf));
}
}
@ -153,19 +187,19 @@ void readBinary(Map & x, ReadBuffer & buf)
void writeBinary(const Map & x, WriteBuffer & buf)
{
const size_t size = x.size();
DB::writeBinary(size, buf);
writeBinary(size, buf);
for (const auto & elem : x)
{
const UInt8 type = elem.getType();
DB::writeBinary(type, buf);
writeBinary(type, buf);
Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, elem);
}
}
void writeText(const Map & x, WriteBuffer & buf)
{
writeFieldText(DB::Field(x), buf);
writeFieldText(Field(x), buf);
}
template <typename T>

View File

@ -974,9 +974,9 @@ __attribute__ ((noreturn)) inline void writeText(const AggregateFunctionStateDat
}
template <typename T>
inline void writeText(const DecimalField<T> & value, WriteBuffer & buf)
inline void writeText(const DecimalField<T> & value, WriteBuffer & buf, bool trailing_zeros = false)
{
writeText(value.getValue(), value.getScale(), buf);
writeText(value.getValue(), value.getScale(), buf, trailing_zeros);
}
template <typename T>

View File

@ -561,6 +561,7 @@ class IColumn;
M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \
M(Bool, output_format_tsv_crlf_end_of_line, false, "If it is set true, end of line in TSV format will be \\r\\n instead of \\n.", 0) \
M(String, output_format_tsv_null_representation, "\\N", "Custom NULL representation in TSV format", 0) \
M(Bool, output_format_decimal_trailing_zeros, false, "Output trailing zeros when printing Decimal values. E.g. 1.230000 instead of 1.23.", 0) \
\
M(UInt64, input_format_allow_errors_num, 0, "Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \
M(Float, input_format_allow_errors_ratio, 0, "Maximum relative amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \

View File

@ -44,10 +44,10 @@ void SerializationDecimal<T>::readText(T & x, ReadBuffer & istr, UInt32 precisio
}
template <typename T>
void SerializationDecimal<T>::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
void SerializationDecimal<T>::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
T value = assert_cast<const ColumnType &>(column).getData()[row_num];
writeText(value, this->scale, ostr);
writeText(value, this->scale, ostr, settings.decimal_trailing_zeros);
}
template <typename T>

View File

@ -151,7 +151,7 @@ Columns CacheDictionary<dictionary_key_type>::getColumns(
* use default value.
*/
if (dictionary_key_type == DictionaryKeyType::complex)
if (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
@ -268,7 +268,7 @@ ColumnUInt8::Ptr CacheDictionary<dictionary_key_type>::hasKeys(const Columns & k
* Check that key was fetched during update for that key set true in result array.
*/
if (dictionary_key_type == DictionaryKeyType::complex)
if (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types);
@ -364,7 +364,7 @@ ColumnPtr CacheDictionary<dictionary_key_type>::getHierarchy(
ColumnPtr key_column [[maybe_unused]],
const DataTypePtr & key_type [[maybe_unused]]) const
{
if (dictionary_key_type == DictionaryKeyType::simple)
if (dictionary_key_type == DictionaryKeyType::Simple)
{
size_t keys_found;
auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type, keys_found);
@ -382,7 +382,7 @@ ColumnUInt8::Ptr CacheDictionary<dictionary_key_type>::isInHierarchy(
ColumnPtr in_key_column [[maybe_unused]],
const DataTypePtr & key_type [[maybe_unused]]) const
{
if (dictionary_key_type == DictionaryKeyType::simple)
if (dictionary_key_type == DictionaryKeyType::Simple)
{
size_t keys_found;
auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type, keys_found);
@ -492,7 +492,7 @@ Pipe CacheDictionary<dictionary_key_type>::read(const Names & column_names, size
/// Write lock on storage
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
data.emplace(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names);
else
{
@ -534,7 +534,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
std::vector<UInt64> requested_keys_vector;
std::vector<size_t> requested_complex_key_rows;
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
requested_keys_vector.reserve(requested_keys.size());
else
requested_complex_key_rows.reserve(requested_keys.size());
@ -546,7 +546,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
if (key_index_to_state_from_storage[i].isExpired()
|| key_index_to_state_from_storage[i].isNotFound())
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
requested_keys_vector.emplace_back(requested_keys[i]);
else
requested_complex_key_rows.emplace_back(i);
@ -572,7 +572,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
Stopwatch watch;
QueryPipeline pipeline;
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
pipeline.init(current_source_ptr->loadIds(requested_keys_vector));
else
pipeline.init(current_source_ptr->loadKeys(update_unit_ptr->key_columns, requested_complex_key_rows));
@ -684,7 +684,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
}
}
template class CacheDictionary<DictionaryKeyType::simple>;
template class CacheDictionary<DictionaryKeyType::complex>;
template class CacheDictionary<DictionaryKeyType::Simple>;
template class CacheDictionary<DictionaryKeyType::Complex>;
}

View File

@ -51,8 +51,7 @@ template <DictionaryKeyType dictionary_key_type>
class CacheDictionary final : public IDictionary
{
public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by cache dictionary");
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
CacheDictionary(
const StorageID & dict_id_,
@ -118,7 +117,7 @@ public:
DictionaryKeyType getKeyType() const override
{
return dictionary_key_type == DictionaryKeyType::simple ? DictionaryKeyType::simple : DictionaryKeyType::complex;
return dictionary_key_type == DictionaryKeyType::Simple ? DictionaryKeyType::Simple : DictionaryKeyType::Complex;
}
ColumnPtr getColumn(
@ -141,7 +140,7 @@ public:
std::exception_ptr getLastException() const override;
bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && dict_struct.hierarchical_attribute_index.has_value(); }
bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::Simple && dict_struct.hierarchical_attribute_index.has_value(); }
ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & key_type) const override;
@ -151,7 +150,7 @@ public:
const DataTypePtr & key_type) const override;
private:
using FetchResult = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, SimpleKeysStorageFetchResult, ComplexKeysStorageFetchResult>;
using FetchResult = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, SimpleKeysStorageFetchResult, ComplexKeysStorageFetchResult>;
static MutableColumns aggregateColumnsInOrderOfKeys(
const PaddedPODArray<KeyType> & keys,
@ -219,7 +218,7 @@ private:
};
extern template class CacheDictionary<DictionaryKeyType::simple>;
extern template class CacheDictionary<DictionaryKeyType::complex>;
extern template class CacheDictionary<DictionaryKeyType::Simple>;
extern template class CacheDictionary<DictionaryKeyType::Complex>;
}

View File

@ -41,8 +41,7 @@ class CacheDictionaryStorage final : public ICacheDictionaryStorage
static constexpr size_t max_collision_length = 10;
public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionaryStorage");
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
explicit CacheDictionaryStorage(
const DictionaryStructure & dictionary_structure,
@ -62,19 +61,19 @@ public:
String getName() const override
{
if (dictionary_key_type == DictionaryKeyType::simple)
if (dictionary_key_type == DictionaryKeyType::Simple)
return "Cache";
else
return "ComplexKeyCache";
}
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::simple; }
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::Simple; }
SimpleKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<UInt64> & keys,
const DictionaryStorageFetchRequest & fetch_request) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for complex key storage");
@ -82,7 +81,7 @@ public:
void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
insertColumnsForKeysImpl(keys, columns);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage");
@ -90,7 +89,7 @@ public:
void insertDefaultKeys(const PaddedPODArray<UInt64> & keys) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
insertDefaultKeysImpl(keys);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage");
@ -98,19 +97,19 @@ public:
PaddedPODArray<UInt64> getCachedSimpleKeys() const override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return getCachedKeysImpl();
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage");
}
bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::complex; }
bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::Complex; }
ComplexKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<StringRef> & keys,
const DictionaryStorageFetchRequest & column_fetch_requests) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, column_fetch_requests);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage");
@ -118,7 +117,7 @@ public:
void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
insertColumnsForKeysImpl(keys, columns);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage");
@ -126,7 +125,7 @@ public:
void insertDefaultKeys(const PaddedPODArray<StringRef> & keys) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
insertDefaultKeysImpl(keys);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage");
@ -134,7 +133,7 @@ public:
PaddedPODArray<StringRef> getCachedComplexKeys() const override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return getCachedKeysImpl();
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedComplexKeys is not supported for simple key storage");

View File

@ -14,8 +14,8 @@ namespace ErrorCodes
extern const int TIMEOUT_EXCEEDED;
}
template class CacheDictionaryUpdateUnit<DictionaryKeyType::simple>;
template class CacheDictionaryUpdateUnit<DictionaryKeyType::complex>;
template class CacheDictionaryUpdateUnit<DictionaryKeyType::Simple>;
template class CacheDictionaryUpdateUnit<DictionaryKeyType::Complex>;
template <DictionaryKeyType dictionary_key_type>
CacheDictionaryUpdateQueue<dictionary_key_type>::CacheDictionaryUpdateQueue(
@ -155,7 +155,7 @@ void CacheDictionaryUpdateQueue<dictionary_key_type>::updateThreadFunction()
}
}
template class CacheDictionaryUpdateQueue<DictionaryKeyType::simple>;
template class CacheDictionaryUpdateQueue<DictionaryKeyType::complex>;
template class CacheDictionaryUpdateQueue<DictionaryKeyType::Simple>;
template class CacheDictionaryUpdateQueue<DictionaryKeyType::Complex>;
}

View File

@ -39,7 +39,7 @@ template <DictionaryKeyType dictionary_key_type>
class CacheDictionaryUpdateUnit
{
public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
/// Constructor for complex keys update request
explicit CacheDictionaryUpdateUnit(
@ -85,8 +85,8 @@ private:
template <DictionaryKeyType dictionary_key_type>
using CacheDictionaryUpdateUnitPtr = std::shared_ptr<CacheDictionaryUpdateUnit<dictionary_key_type>>;
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::simple>;
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::complex>;
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::Simple>;
extern template class CacheDictionaryUpdateUnit<DictionaryKeyType::Complex>;
struct CacheDictionaryUpdateQueueConfiguration
{
@ -110,7 +110,6 @@ class CacheDictionaryUpdateQueue
public:
/// Client of update queue must provide this function in constructor and perform update using update unit.
using UpdateFunction = std::function<void (CacheDictionaryUpdateUnitPtr<dictionary_key_type>)>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionaryUpdateQueue");
CacheDictionaryUpdateQueue(
String dictionary_name_for_logs_,
@ -167,7 +166,7 @@ private:
std::atomic<bool> finished{false};
};
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::simple>;
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::complex>;
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::Simple>;
extern template class CacheDictionaryUpdateQueue<DictionaryKeyType::Complex>;
}

View File

@ -380,14 +380,14 @@ template <DictionaryKeyType key_type>
class DictionaryKeysArenaHolder;
template <>
class DictionaryKeysArenaHolder<DictionaryKeyType::simple>
class DictionaryKeysArenaHolder<DictionaryKeyType::Simple>
{
public:
static Arena * getComplexKeyArena() { return nullptr; }
};
template <>
class DictionaryKeysArenaHolder<DictionaryKeyType::complex>
class DictionaryKeysArenaHolder<DictionaryKeyType::Complex>
{
public:
@ -402,8 +402,7 @@ template <DictionaryKeyType key_type>
class DictionaryKeysExtractor
{
public:
using KeyType = std::conditional_t<key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(key_type != DictionaryKeyType::range, "Range key type is not supported by DictionaryKeysExtractor");
using KeyType = std::conditional_t<key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
explicit DictionaryKeysExtractor(const Columns & key_columns_, Arena * complex_key_arena_)
: key_columns(key_columns_)
@ -411,7 +410,7 @@ public:
{
assert(!key_columns.empty());
if constexpr (key_type == DictionaryKeyType::simple)
if constexpr (key_type == DictionaryKeyType::Simple)
{
key_columns[0] = key_columns[0]->convertToFullColumnIfConst();
@ -437,7 +436,7 @@ public:
{
assert(current_key_index < keys_size);
if constexpr (key_type == DictionaryKeyType::simple)
if constexpr (key_type == DictionaryKeyType::Simple)
{
const auto & column_vector = static_cast<const ColumnVector<UInt64> &>(*key_columns[0]);
const auto & data = column_vector.getData();
@ -465,7 +464,7 @@ public:
void rollbackCurrentKey() const
{
if constexpr (key_type == DictionaryKeyType::complex)
if constexpr (key_type == DictionaryKeyType::Complex)
complex_key_arena->rollback(current_complex_key.size);
}
@ -521,8 +520,7 @@ void mergeBlockWithPipe(
Block & block_to_update,
Pipe pipe)
{
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by updatePreviousyLoadedBlockWithStream");
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
Columns saved_block_key_columns;
saved_block_key_columns.reserve(key_columns_size);

View File

@ -132,7 +132,7 @@ Block DictionarySourceData::fillBlock(
{
ColumnPtr column;
if (dictionary_key_type == DictionaryKeyType::simple)
if (dictionary_key_type == DictionaryKeyType::Simple)
{
column = dictionary->getColumn(
attribute.name,

View File

@ -40,7 +40,7 @@ Columns DirectDictionary<dictionary_key_type>::getColumns(
const DataTypes & key_types [[maybe_unused]],
const Columns & default_values_columns) const
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
@ -161,7 +161,7 @@ ColumnUInt8::Ptr DirectDictionary<dictionary_key_type>::hasKeys(
const Columns & key_columns,
const DataTypes & key_types [[maybe_unused]]) const
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
@ -230,7 +230,7 @@ ColumnPtr DirectDictionary<dictionary_key_type>::getHierarchy(
ColumnPtr key_column,
const DataTypePtr & key_type) const
{
if (dictionary_key_type == DictionaryKeyType::simple)
if (dictionary_key_type == DictionaryKeyType::Simple)
{
size_t keys_found;
auto result = getKeysHierarchyDefaultImplementation(this, key_column, key_type, keys_found);
@ -248,7 +248,7 @@ ColumnUInt8::Ptr DirectDictionary<dictionary_key_type>::isInHierarchy(
ColumnPtr in_key_column,
const DataTypePtr & key_type) const
{
if (dictionary_key_type == DictionaryKeyType::simple)
if (dictionary_key_type == DictionaryKeyType::Simple)
{
size_t keys_found = 0;
auto result = getKeysIsInHierarchyDefaultImplementation(this, key_column, in_key_column, key_type, keys_found);
@ -269,7 +269,7 @@ Pipe DirectDictionary<dictionary_key_type>::getSourceBlockInputStream(
Pipe pipe;
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{
std::vector<UInt64> ids;
ids.reserve(requested_keys_size);
@ -310,9 +310,9 @@ namespace
ContextPtr /* context */,
bool /* created_from_ddl */)
{
const auto * layout_name = dictionary_key_type == DictionaryKeyType::simple ? "direct" : "complex_key_direct";
const auto * layout_name = dictionary_key_type == DictionaryKeyType::Simple ? "direct" : "complex_key_direct";
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{
if (dict_struct.key)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
@ -344,13 +344,13 @@ namespace
}
}
template class DirectDictionary<DictionaryKeyType::simple>;
template class DirectDictionary<DictionaryKeyType::complex>;
template class DirectDictionary<DictionaryKeyType::Simple>;
template class DirectDictionary<DictionaryKeyType::Complex>;
void registerDictionaryDirect(DictionaryFactory & factory)
{
factory.registerLayout("direct", createDirectDictionary<DictionaryKeyType::simple>, false);
factory.registerLayout("complex_key_direct", createDirectDictionary<DictionaryKeyType::complex>, true);
factory.registerLayout("direct", createDirectDictionary<DictionaryKeyType::Simple>, false);
factory.registerLayout("complex_key_direct", createDirectDictionary<DictionaryKeyType::Complex>, true);
}

View File

@ -20,8 +20,7 @@ template <DictionaryKeyType dictionary_key_type>
class DirectDictionary final : public IDictionary
{
public:
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by direct dictionary");
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
DirectDictionary(
const StorageID & dict_id_,
@ -30,7 +29,7 @@ public:
std::string getTypeName() const override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return "Direct";
else
return "ComplexKeyDirect";
@ -110,7 +109,7 @@ private:
mutable std::atomic<size_t> found_count{0};
};
extern template class DirectDictionary<DictionaryKeyType::simple>;
extern template class DirectDictionary<DictionaryKeyType::complex>;
extern template class DirectDictionary<DictionaryKeyType::Simple>;
extern template class DirectDictionary<DictionaryKeyType::Complex>;
}

View File

@ -289,8 +289,8 @@ void FlatDictionary::blockToAttributes(const Block & block)
{
const auto keys_column = block.safeGetByPosition(0).column;
DictionaryKeysArenaHolder<DictionaryKeyType::simple> arena_holder;
DictionaryKeysExtractor<DictionaryKeyType::simple> keys_extractor({ keys_column }, arena_holder.getComplexKeyArena());
DictionaryKeysArenaHolder<DictionaryKeyType::Simple> arena_holder;
DictionaryKeysExtractor<DictionaryKeyType::Simple> keys_extractor({ keys_column }, arena_holder.getComplexKeyArena());
auto keys = keys_extractor.extractAllKeys();
HashSet<UInt64> already_processed_keys;
@ -344,7 +344,7 @@ void FlatDictionary::updateData()
else
{
Pipe pipe(source_ptr->loadUpdatedAll());
mergeBlockWithPipe<DictionaryKeyType::simple>(
mergeBlockWithPipe<DictionaryKeyType::Simple>(
dict_struct.getKeysSize(),
*update_field_loaded_block,
std::move(pipe));

View File

@ -72,7 +72,7 @@ public:
return dict_struct.getAttribute(attribute_name).injective;
}
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::simple; }
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Simple; }
ColumnPtr getColumn(
const std::string& attribute_name,

View File

@ -61,7 +61,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getColumn(
const DataTypes & key_types [[maybe_unused]],
const ColumnPtr & default_values_column) const
{
if (dictionary_key_type == DictionaryKeyType::complex)
if (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types);
ColumnPtr result;
@ -163,7 +163,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getColumn(
template <DictionaryKeyType dictionary_key_type, bool sparse>
ColumnUInt8::Ptr HashedDictionary<dictionary_key_type, sparse>::hasKeys(const Columns & key_columns, const DataTypes & key_types) const
{
if (dictionary_key_type == DictionaryKeyType::complex)
if (dictionary_key_type == DictionaryKeyType::Complex)
dict_struct.validateKeyTypes(key_types);
DictionaryKeysArenaHolder<dictionary_key_type> arena_holder;
@ -210,7 +210,7 @@ ColumnUInt8::Ptr HashedDictionary<dictionary_key_type, sparse>::hasKeys(const Co
template <DictionaryKeyType dictionary_key_type, bool sparse>
ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getHierarchy(ColumnPtr key_column [[maybe_unused]], const DataTypePtr &) const
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{
PaddedPODArray<UInt64> keys_backup_storage;
const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage);
@ -258,7 +258,7 @@ ColumnUInt8::Ptr HashedDictionary<dictionary_key_type, sparse>::isInHierarchy(
ColumnPtr in_key_column [[maybe_unused]],
const DataTypePtr &) const
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{
PaddedPODArray<UInt64> keys_backup_storage;
const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage);
@ -309,7 +309,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getDescendants(
const DataTypePtr &,
size_t level [[maybe_unused]]) const
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{
PaddedPODArray<UInt64> keys_backup;
const auto & keys = getColumnVectorData(this, key_column, keys_backup);
@ -665,7 +665,7 @@ Pipe HashedDictionary<dictionary_key_type, sparse>::read(const Names & column_na
});
}
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return Pipe(std::make_shared<DictionarySource>(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size));
else
return Pipe(std::make_shared<DictionarySource>(DictionarySourceData(shared_from_this(), keys, column_names), max_block_size));
@ -702,10 +702,10 @@ void HashedDictionary<dictionary_key_type, sparse>::getAttributeContainer(size_t
});
}
template class HashedDictionary<DictionaryKeyType::simple, true>;
template class HashedDictionary<DictionaryKeyType::simple, false>;
template class HashedDictionary<DictionaryKeyType::complex, true>;
template class HashedDictionary<DictionaryKeyType::complex, false>;
template class HashedDictionary<DictionaryKeyType::Simple, true>;
template class HashedDictionary<DictionaryKeyType::Simple, false>;
template class HashedDictionary<DictionaryKeyType::Complex, true>;
template class HashedDictionary<DictionaryKeyType::Complex, false>;
void registerDictionaryHashed(DictionaryFactory & factory)
{
@ -717,9 +717,9 @@ void registerDictionaryHashed(DictionaryFactory & factory)
DictionaryKeyType dictionary_key_type,
bool sparse) -> DictionaryPtr
{
if (dictionary_key_type == DictionaryKeyType::simple && dict_struct.key)
if (dictionary_key_type == DictionaryKeyType::Simple && dict_struct.key)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for simple key hashed dictionary");
else if (dictionary_key_type == DictionaryKeyType::complex && dict_struct.id)
else if (dictionary_key_type == DictionaryKeyType::Complex && dict_struct.id)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is not supported for complex key hashed dictionary");
if (dict_struct.range_min || dict_struct.range_max)
@ -737,32 +737,32 @@ void registerDictionaryHashed(DictionaryFactory & factory)
HashedDictionaryStorageConfiguration configuration{preallocate, require_nonempty, dict_lifetime};
if (dictionary_key_type == DictionaryKeyType::simple)
if (dictionary_key_type == DictionaryKeyType::Simple)
{
if (sparse)
return std::make_unique<HashedDictionary<DictionaryKeyType::simple, true>>(dict_id, dict_struct, std::move(source_ptr), configuration);
return std::make_unique<HashedDictionary<DictionaryKeyType::Simple, true>>(dict_id, dict_struct, std::move(source_ptr), configuration);
else
return std::make_unique<HashedDictionary<DictionaryKeyType::simple, false>>(dict_id, dict_struct, std::move(source_ptr), configuration);
return std::make_unique<HashedDictionary<DictionaryKeyType::Simple, false>>(dict_id, dict_struct, std::move(source_ptr), configuration);
}
else
{
if (sparse)
return std::make_unique<HashedDictionary<DictionaryKeyType::complex, true>>(dict_id, dict_struct, std::move(source_ptr), configuration);
return std::make_unique<HashedDictionary<DictionaryKeyType::Complex, true>>(dict_id, dict_struct, std::move(source_ptr), configuration);
else
return std::make_unique<HashedDictionary<DictionaryKeyType::complex, false>>(dict_id, dict_struct, std::move(source_ptr), configuration);
return std::make_unique<HashedDictionary<DictionaryKeyType::Complex, false>>(dict_id, dict_struct, std::move(source_ptr), configuration);
}
};
using namespace std::placeholders;
factory.registerLayout("hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ false); }, false);
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ false); }, false);
factory.registerLayout("sparse_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ true); }, false);
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ true); }, false);
factory.registerLayout("complex_key_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ false); }, true);
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ false); }, true);
factory.registerLayout("complex_key_sparse_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true);
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ true); }, true);
}

View File

@ -35,8 +35,7 @@ template <DictionaryKeyType dictionary_key_type, bool sparse>
class HashedDictionary final : public IDictionary
{
public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by hashed dictionary");
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
HashedDictionary(
const StorageID & dict_id_,
@ -47,11 +46,11 @@ public:
std::string getTypeName() const override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple && sparse)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple && sparse)
return "SparseHashed";
else if constexpr (dictionary_key_type == DictionaryKeyType::simple && !sparse)
else if constexpr (dictionary_key_type == DictionaryKeyType::Simple && !sparse)
return "Hashed";
else if constexpr (dictionary_key_type == DictionaryKeyType::complex && sparse)
else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && sparse)
return "ComplexKeySparseHashed";
else
return "ComplexKeyHashed";
@ -102,7 +101,7 @@ public:
ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override;
bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::simple && dict_struct.hierarchical_attribute_index.has_value(); }
bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::Simple && dict_struct.hierarchical_attribute_index.has_value(); }
ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & hierarchy_attribute_type) const override;
@ -121,13 +120,13 @@ public:
private:
template <typename Value>
using CollectionTypeNonSparse = std::conditional_t<
dictionary_key_type == DictionaryKeyType::simple,
dictionary_key_type == DictionaryKeyType::Simple,
HashMap<UInt64, Value>,
HashMapWithSavedHash<StringRef, Value, DefaultHash<StringRef>>>;
template <typename Value>
using CollectionTypeSparse = std::conditional_t<
dictionary_key_type == DictionaryKeyType::simple,
dictionary_key_type == DictionaryKeyType::Simple,
SparseHashMap<UInt64, Value>,
SparseHashMap<StringRef, Value>>;
@ -211,10 +210,10 @@ private:
Arena complex_key_arena;
};
extern template class HashedDictionary<DictionaryKeyType::simple, false>;
extern template class HashedDictionary<DictionaryKeyType::simple, true>;
extern template class HashedDictionary<DictionaryKeyType::Simple, false>;
extern template class HashedDictionary<DictionaryKeyType::Simple, true>;
extern template class HashedDictionary<DictionaryKeyType::complex, false>;
extern template class HashedDictionary<DictionaryKeyType::complex, true>;
extern template class HashedDictionary<DictionaryKeyType::Complex, false>;
extern template class HashedDictionary<DictionaryKeyType::Complex, true>;
}

View File

@ -33,15 +33,20 @@ using DictionaryPtr = std::unique_ptr<IDictionary>;
* Simple is for dictionaries that support UInt64 key column.
*
* Complex is for dictionaries that support any combination of key columns.
*
* Range is for dictionary that support combination of UInt64 key column,
* and numeric representable range key column.
*/
enum class DictionaryKeyType
{
simple,
complex,
range
Simple,
Complex
};
/** DictionarySpecialKeyType provides IDictionary client information about
* which special key type is supported by dictionary.
*/
enum class DictionarySpecialKeyType
{
None,
Range
};
/**
@ -56,6 +61,7 @@ struct IDictionary : public IExternalLoadable
}
const std::string & getFullName() const{ return full_name; }
StorageID getDictionaryID() const
{
std::lock_guard lock{name_mutex};
@ -109,6 +115,8 @@ struct IDictionary : public IExternalLoadable
*/
virtual DictionaryKeyType getKeyType() const = 0;
virtual DictionarySpecialKeyType getSpecialKeyType() const { return DictionarySpecialKeyType::None;}
/** Subclass must validate key columns and keys types
* and return column representation of dictionary attribute.
*

View File

@ -67,7 +67,7 @@ public:
return dict_struct.getAttribute(attribute_name).injective;
}
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::complex; }
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Complex; }
ColumnPtr getColumn(
const std::string& attribute_name,

View File

@ -86,7 +86,7 @@ public:
bool isInjective(const std::string & attribute_name) const override { return dict_struct.getAttribute(attribute_name).injective; }
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::complex; }
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::Complex; }
ColumnPtr getColumn(
const std::string& attribute_name,

View File

@ -14,18 +14,12 @@
namespace DB
{
enum class RangeDictionaryType
{
simple,
complex
};
template <RangeDictionaryType range_dictionary_type, typename RangeType>
template <DictionaryKeyType dictionary_key_type, typename RangeType>
class RangeDictionarySourceData
{
public:
using KeyType = std::conditional_t<range_dictionary_type == RangeDictionaryType::simple, UInt64, StringRef>;
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
RangeDictionarySourceData(
std::shared_ptr<const IDictionary> dictionary,
@ -58,8 +52,8 @@ private:
};
template <RangeDictionaryType range_dictionary_type, typename RangeType>
RangeDictionarySourceData<range_dictionary_type, RangeType>::RangeDictionarySourceData(
template <DictionaryKeyType dictionary_key_type, typename RangeType>
RangeDictionarySourceData<dictionary_key_type, RangeType>::RangeDictionarySourceData(
std::shared_ptr<const IDictionary> dictionary_,
const Names & column_names_,
PaddedPODArray<KeyType> && keys,
@ -73,8 +67,8 @@ RangeDictionarySourceData<range_dictionary_type, RangeType>::RangeDictionarySour
{
}
template <RangeDictionaryType range_dictionary_type, typename RangeType>
Block RangeDictionarySourceData<range_dictionary_type, RangeType>::getBlock(size_t start, size_t length) const
template <DictionaryKeyType dictionary_key_type, typename RangeType>
Block RangeDictionarySourceData<dictionary_key_type, RangeType>::getBlock(size_t start, size_t length) const
{
PaddedPODArray<KeyType> block_keys;
PaddedPODArray<RangeType> block_start_dates;
@ -93,8 +87,8 @@ Block RangeDictionarySourceData<range_dictionary_type, RangeType>::getBlock(size
return fillBlock(block_keys, block_start_dates, block_end_dates, start, start + length);
}
template <RangeDictionaryType range_dictionary_type, typename RangeType>
PaddedPODArray<Int64> RangeDictionarySourceData<range_dictionary_type, RangeType>::makeDateKeys(
template <DictionaryKeyType dictionary_key_type, typename RangeType>
PaddedPODArray<Int64> RangeDictionarySourceData<dictionary_key_type, RangeType>::makeDateKeys(
const PaddedPODArray<RangeType> & block_start_dates,
const PaddedPODArray<RangeType> & block_end_dates) const
{
@ -112,24 +106,14 @@ PaddedPODArray<Int64> RangeDictionarySourceData<range_dictionary_type, RangeType
}
template <RangeDictionaryType range_dictionary_type, typename RangeType>
Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock(
template <DictionaryKeyType dictionary_key_type, typename RangeType>
Block RangeDictionarySourceData<dictionary_key_type, RangeType>::fillBlock(
const PaddedPODArray<KeyType> & keys_to_fill,
const PaddedPODArray<RangeType> & block_start_dates,
const PaddedPODArray<RangeType> & block_end_dates,
size_t start,
size_t end) const
{
std::cerr << "RangeDictionarySourceData::fillBlock keys_to_fill " << keys_to_fill.size() << std::endl;
if constexpr (range_dictionary_type == RangeDictionaryType::simple)
{
for (auto & key : keys_to_fill)
{
std::cerr << key << std::endl;
}
}
ColumnsWithTypeAndName columns;
const DictionaryStructure & dictionary_structure = dictionary->getStructure();
@ -137,7 +121,7 @@ Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock(
Columns keys_columns;
Strings keys_names = dictionary_structure.getKeysNames();
if constexpr (range_dictionary_type == RangeDictionaryType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{
keys_columns = {getColumnFromPODArray(keys_to_fill)};
keys_types = {std::make_shared<DataTypeUInt64>()};
@ -154,9 +138,6 @@ Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock(
size_t keys_size = keys_names.size();
std::cerr << "Keys size " << keys_size << " key columns size " << keys_columns.size();
std::cerr << " keys types size " << keys_types.size() << std::endl;
assert(keys_columns.size() == keys_size);
assert(keys_types.size() == keys_size);
@ -204,51 +185,33 @@ Block RangeDictionarySourceData<range_dictionary_type, RangeType>::fillBlock(
columns.emplace_back(std::move(column), attribute.type, attribute.name);
}
auto result = Block(columns);
Field value;
std::cerr << "RangeDictionarySourceData::fillBlock result" << std::endl;
for (auto & block_column : result)
{
std::cerr << "Column name " << block_column.name << " type " << block_column.type->getName() << std::endl;
auto & column = block_column.column;
size_t column_size = column->size();
for (size_t i = 0; i < column_size; ++i)
{
column->get(i, value);
std::cerr << "Index " << i << " value " << value.dump() << std::endl;
}
}
return Block(columns);
}
template <RangeDictionaryType range_dictionary_type, typename RangeType>
template <DictionaryKeyType dictionary_key_type, typename RangeType>
class RangeDictionarySource : public DictionarySourceBase
{
public:
RangeDictionarySource(RangeDictionarySourceData<range_dictionary_type, RangeType> data_, size_t max_block_size);
RangeDictionarySource(RangeDictionarySourceData<dictionary_key_type, RangeType> data_, size_t max_block_size);
String getName() const override { return "RangeDictionarySource"; }
protected:
Block getBlock(size_t start, size_t length) const override;
RangeDictionarySourceData<range_dictionary_type, RangeType> data;
RangeDictionarySourceData<dictionary_key_type, RangeType> data;
};
template <RangeDictionaryType range_dictionary_type, typename RangeType>
RangeDictionarySource<range_dictionary_type, RangeType>::RangeDictionarySource(RangeDictionarySourceData<range_dictionary_type, RangeType> data_, size_t max_block_size)
template <DictionaryKeyType dictionary_key_type, typename RangeType>
RangeDictionarySource<dictionary_key_type, RangeType>::RangeDictionarySource(RangeDictionarySourceData<dictionary_key_type, RangeType> data_, size_t max_block_size)
: DictionarySourceBase(data_.getBlock(0, 0), data_.getNumRows(), max_block_size)
, data(std::move(data_))
{
}
template <RangeDictionaryType range_dictionary_type, typename RangeType>
Block RangeDictionarySource<range_dictionary_type, RangeType>::getBlock(size_t start, size_t length) const
template <DictionaryKeyType dictionary_key_type, typename RangeType>
Block RangeDictionarySource<dictionary_key_type, RangeType>::getBlock(size_t start, size_t length) const
{
return data.getBlock(start, length);
}

View File

@ -95,6 +95,13 @@ ColumnPtr RangeHashedDictionary<dictionary_key_type>::getColumn(
const DataTypes & key_types,
const ColumnPtr & default_values_column) const
{
if (dictionary_key_type == DictionaryKeyType::Complex)
{
auto key_types_copy = key_types;
key_types_copy.pop_back();
dict_struct.validateKeyTypes(key_types_copy);
}
ColumnPtr result;
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
@ -206,9 +213,15 @@ ColumnPtr RangeHashedDictionary<dictionary_key_type>::getColumn(
template <DictionaryKeyType dictionary_key_type>
ColumnUInt8::Ptr RangeHashedDictionary<dictionary_key_type>::hasKeys(const Columns & key_columns, const DataTypes & key_types) const
{
if (dictionary_key_type == DictionaryKeyType::Complex)
{
auto key_types_copy = key_types;
key_types_copy.pop_back();
dict_struct.validateKeyTypes(key_types_copy);
}
auto range_column_storage_type = std::make_shared<DataTypeInt64>();
auto range_storage_column = key_columns.back();
ColumnWithTypeAndName column_to_cast = {range_storage_column->convertToFullColumnIfConst(), key_types[1], ""};
ColumnWithTypeAndName column_to_cast = {range_storage_column->convertToFullColumnIfConst(), key_types.back(), ""};
auto range_column_updated = castColumnAccurate(column_to_cast, range_column_storage_type);
PaddedPODArray<RangeStorageType> range_backup_storage;
const PaddedPODArray<RangeStorageType> & dates = getColumnVectorData(this, range_column_updated, range_backup_storage);
@ -383,7 +396,7 @@ void RangeHashedDictionary<dictionary_key_type>::calculateBytesAllocated()
callOnDictionaryAttributeType(attribute.type, type_call);
}
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
bytes_allocated += complex_key_arena.size();
}
@ -607,10 +620,9 @@ Pipe RangeHashedDictionary<dictionary_key_type>::readImpl(const Names & column_n
PaddedPODArray<RangeType> end_dates;
getKeysAndDates(keys, start_dates, end_dates);
static constexpr RangeDictionaryType range_dictionary_type = (dictionary_key_type == DictionaryKeyType::simple) ? RangeDictionaryType::simple : RangeDictionaryType::complex;
using RangeDictionarySourceType = RangeDictionarySource<range_dictionary_type, RangeType>;
using RangeDictionarySourceType = RangeDictionarySource<dictionary_key_type, RangeType>;
auto source_data = RangeDictionarySourceData<range_dictionary_type, RangeType>(
auto source_data = RangeDictionarySourceData<dictionary_key_type, RangeType>(
shared_from_this(),
column_names,
std::move(keys),
@ -690,7 +702,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<RangeHashedDictionary<DictionaryKeyType::simple>>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
return std::make_unique<RangeHashedDictionary<DictionaryKeyType::Simple>>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("range_hashed", create_layout_simple, false);
@ -713,7 +725,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<RangeHashedDictionary<DictionaryKeyType::complex>>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
return std::make_unique<RangeHashedDictionary<DictionaryKeyType::Complex>>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("complex_key_range_hashed", create_layout_complex, true);
}

View File

@ -32,8 +32,7 @@ template <DictionaryKeyType dictionary_key_type>
class RangeHashedDictionary final : public IDictionary
{
public:
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by hashed dictionary");
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
RangeHashedDictionary(
const StorageID & dict_id_,
@ -78,7 +77,9 @@ public:
return dict_struct.getAttribute(attribute_name).injective;
}
DictionaryKeyType getKeyType() const override { return DictionaryKeyType::range; }
DictionaryKeyType getKeyType() const override { return dictionary_key_type; }
DictionarySpecialKeyType getSpecialKeyType() const override { return DictionarySpecialKeyType::Range;}
ColumnPtr getColumn(
const std::string& attribute_name,
@ -104,7 +105,7 @@ private:
template <typename Value>
using CollectionType = std::conditional_t<
dictionary_key_type == DictionaryKeyType::simple,
dictionary_key_type == DictionaryKeyType::Simple,
HashMap<UInt64, Values<Value>>,
HashMapWithSavedHash<StringRef, Values<Value>, DefaultHash<StringRef>>>;

View File

@ -823,8 +823,8 @@ template <DictionaryKeyType dictionary_key_type>
class SSDCacheDictionaryStorage final : public ICacheDictionaryStorage
{
public:
using SSDCacheKeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, SSDCacheSimpleKey, SSDCacheComplexKey>;
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
using SSDCacheKeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, SSDCacheSimpleKey, SSDCacheComplexKey>;
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::Simple, UInt64, StringRef>;
explicit SSDCacheDictionaryStorage(const SSDCacheDictionaryStorageConfiguration & configuration_)
: configuration(configuration_)
@ -838,19 +838,19 @@ public:
String getName() const override
{
if (dictionary_key_type == DictionaryKeyType::simple)
if (dictionary_key_type == DictionaryKeyType::Simple)
return "SSDCache";
else
return "SSDComplexKeyCache";
}
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::simple; }
bool supportsSimpleKeys() const override { return dictionary_key_type == DictionaryKeyType::Simple; }
SimpleKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<UInt64> & keys,
const DictionaryStorageFetchRequest & fetch_request) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return fetchColumnsForKeysImpl<SimpleKeysStorageFetchResult>(keys, fetch_request);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage");
@ -858,7 +858,7 @@ public:
void insertColumnsForKeys(const PaddedPODArray<UInt64> & keys, Columns columns) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
insertColumnsForKeysImpl(keys, columns);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for complex key storage");
@ -866,7 +866,7 @@ public:
void insertDefaultKeys(const PaddedPODArray<UInt64> & keys) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
insertDefaultKeysImpl(keys);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for complex key storage");
@ -874,19 +874,19 @@ public:
PaddedPODArray<UInt64> getCachedSimpleKeys() const override
{
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
return getCachedKeysImpl();
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for complex key storage");
}
bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::complex; }
bool supportsComplexKeys() const override { return dictionary_key_type == DictionaryKeyType::Complex; }
ComplexKeysStorageFetchResult fetchColumnsForKeys(
const PaddedPODArray<StringRef> & keys,
const DictionaryStorageFetchRequest & fetch_request) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return fetchColumnsForKeysImpl<ComplexKeysStorageFetchResult>(keys, fetch_request);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fetchColumnsForKeys is not supported for simple key storage");
@ -894,7 +894,7 @@ public:
void insertColumnsForKeys(const PaddedPODArray<StringRef> & keys, Columns columns) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
insertColumnsForKeysImpl(keys, columns);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertColumnsForKeys is not supported for simple key storage");
@ -902,7 +902,7 @@ public:
void insertDefaultKeys(const PaddedPODArray<StringRef> & keys) override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
insertDefaultKeysImpl(keys);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method insertDefaultKeysImpl is not supported for simple key storage");
@ -910,7 +910,7 @@ public:
PaddedPODArray<StringRef> getCachedComplexKeys() const override
{
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
return getCachedKeysImpl();
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getCachedSimpleKeys is not supported for simple key storage");
@ -1134,7 +1134,7 @@ private:
Cell cell;
setCellDeadline(cell, now);
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
{
/// Copy complex key into arena and put in cache
size_t key_size = key.size;
@ -1166,7 +1166,7 @@ private:
cell.state = Cell::default_value;
if constexpr (dictionary_key_type == DictionaryKeyType::complex)
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
{
/// Copy complex key into arena and put in cache
size_t key_size = key.size;
@ -1382,7 +1382,7 @@ private:
using ComplexKeyHashMap = HashMapWithSavedHash<StringRef, Cell>;
using CacheMap = std::conditional_t<
dictionary_key_type == DictionaryKeyType::simple,
dictionary_key_type == DictionaryKeyType::Simple,
SimpleKeyHashMap,
ComplexKeyHashMap>;

View File

@ -157,24 +157,23 @@ DictionaryPtr createCacheDictionaryLayout(
ContextPtr context [[maybe_unused]],
bool created_from_ddl [[maybe_unused]])
{
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionary");
String layout_type;
if constexpr (dictionary_key_type == DictionaryKeyType::simple && !ssd)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple && !ssd)
layout_type = "cache";
else if constexpr (dictionary_key_type == DictionaryKeyType::simple && ssd)
else if constexpr (dictionary_key_type == DictionaryKeyType::Simple && ssd)
layout_type = "ssd_cache";
else if constexpr (dictionary_key_type == DictionaryKeyType::complex && !ssd)
else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && !ssd)
layout_type = "complex_key_cache";
else if constexpr (dictionary_key_type == DictionaryKeyType::complex && ssd)
else if constexpr (dictionary_key_type == DictionaryKeyType::Complex && ssd)
layout_type = "complex_key_ssd_cache";
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
if constexpr (dictionary_key_type == DictionaryKeyType::Simple)
{
if (dict_struct.key)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: dictionary of layout '{}' 'key' is not supported", full_name, layout_type);
}
else if constexpr (dictionary_key_type == DictionaryKeyType::complex)
else if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
{
if (dict_struct.id)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "{}: dictionary of layout '{}' 'id' is not supported", full_name, layout_type);
@ -243,7 +242,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
ContextPtr context,
bool created_from_ddl) -> DictionaryPtr
{
return createCacheDictionaryLayout<DictionaryKeyType::simple, false/* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
return createCacheDictionaryLayout<DictionaryKeyType::Simple, false/* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
};
factory.registerLayout("cache", create_simple_cache_layout, false);
@ -256,7 +255,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
ContextPtr context,
bool created_from_ddl) -> DictionaryPtr
{
return createCacheDictionaryLayout<DictionaryKeyType::complex, false /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
return createCacheDictionaryLayout<DictionaryKeyType::Complex, false /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
};
factory.registerLayout("complex_key_cache", create_complex_key_cache_layout, true);
@ -271,7 +270,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
ContextPtr context,
bool created_from_ddl) -> DictionaryPtr
{
return createCacheDictionaryLayout<DictionaryKeyType::simple, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
return createCacheDictionaryLayout<DictionaryKeyType::Simple, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
};
factory.registerLayout("ssd_cache", create_simple_ssd_cache_layout, false);
@ -283,7 +282,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
DictionarySourcePtr source_ptr,
ContextPtr context,
bool created_from_ddl) -> DictionaryPtr {
return createCacheDictionaryLayout<DictionaryKeyType::complex, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
return createCacheDictionaryLayout<DictionaryKeyType::Complex, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl);
};
factory.registerLayout("complex_key_ssd_cache", create_complex_key_ssd_cache_layout, true);

View File

@ -81,6 +81,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers;
format_settings.json.quote_denormals = settings.output_format_json_quote_denormals;
format_settings.null_as_default = settings.input_format_null_as_default;
format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros;
format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size;
format_settings.parquet.import_nested = settings.input_format_parquet_import_nested;
format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8;

View File

@ -28,6 +28,7 @@ struct FormatSettings
bool write_statistics = true;
bool import_nested_json = false;
bool null_as_default = true;
bool decimal_trailing_zeros = false;
enum class DateTimeInputFormat
{

View File

@ -1239,7 +1239,7 @@ namespace
else
{
WriteBufferFromOwnString buf;
writeText(decimal, scale, buf);
writeText(decimal, scale, buf, false);
cannotConvertValue(buf.str(), TypeName<DecimalType>, field_descriptor.type_name());
}
};
@ -1316,9 +1316,9 @@ namespace
{
WriteBufferFromString buf{str};
if constexpr (std::is_same_v<DecimalType, DateTime64>)
writeDateTimeText(decimal, scale, buf);
writeDateTimeText(decimal, scale, buf);
else
writeText(decimal, scale, buf);
writeText(decimal, scale, buf, false);
}
DecimalType stringToDecimal(const String & str) const

View File

@ -733,7 +733,7 @@ struct FormatImpl<DataTypeDecimal<FieldType>>
template <typename ReturnType = void>
static ReturnType execute(const FieldType x, WriteBuffer & wb, const DataTypeDecimal<FieldType> * type, const DateLUTImpl *)
{
writeText(x, type->getScale(), wb);
writeText(x, type->getScale(), wb, false);
return ReturnType(true);
}
};

View File

@ -42,7 +42,6 @@ namespace ErrorCodes
extern const int UNSUPPORTED_METHOD;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
extern const int BAD_ARGUMENTS;
extern const int TYPE_MISMATCH;
}
@ -182,15 +181,16 @@ public:
auto dictionary = helper.getDictionary(arguments[0].column);
auto dictionary_key_type = dictionary->getKeyType();
auto dictionary_special_key_type = dictionary->getSpecialKeyType();
const ColumnWithTypeAndName & key_column_with_type = arguments[1];
const auto & key_column_with_type = arguments[1];
auto key_column = key_column_with_type.column;
auto key_column_type = key_column_with_type.type;
ColumnPtr range_col = nullptr;
DataTypePtr range_col_type = nullptr;
ColumnPtr range_col;
DataTypePtr range_col_type;
if (dictionary_key_type == DictionaryKeyType::range)
if (dictionary_special_key_type == DictionarySpecialKeyType::Range)
{
if (arguments.size() != 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
@ -207,7 +207,10 @@ public:
getName());
}
if (dictionary_key_type == DictionaryKeyType::simple)
Columns key_columns;
DataTypes key_types;
if (dictionary_key_type == DictionaryKeyType::Simple)
{
if (!WhichDataType(key_column_type).isUInt64())
throw Exception(
@ -216,16 +219,23 @@ public:
getName(),
key_column_with_type.type->getName());
return dictionary->hasKeys({key_column}, {std::make_shared<DataTypeUInt64>()});
key_columns = {key_column};
key_types = {std::make_shared<DataTypeUInt64>()};
}
else if (dictionary_key_type == DictionaryKeyType::complex)
else if (dictionary_key_type == DictionaryKeyType::Complex)
{
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
key_column = key_column->convertToFullColumnIfConst();
size_t keys_size = dictionary->getStructure().getKeysSize();
if (!isTuple(key_column_type))
if (isTuple(key_column_type))
{
key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
}
else
{
size_t keys_size = dictionary->getStructure().getKeysSize();
if (keys_size > 1)
{
throw Exception(
@ -237,41 +247,19 @@ public:
}
else
{
Columns tuple_columns = {std::move(key_column)};
key_column = ColumnTuple::create(tuple_columns);
DataTypes tuple_types = {key_column_type};
key_column_type = std::make_shared<DataTypeTuple>(tuple_types);
key_columns = {key_column};
key_types = {key_column_type};
}
}
const auto & key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
const auto & key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
return dictionary->hasKeys(key_columns, key_types);
}
else
if (dictionary_special_key_type == DictionarySpecialKeyType::Range)
{
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column = key_column_with_type.column->convertToFullColumnIfConst();
DataTypePtr key_column_type = key_column_with_type.type;
Columns key_columns;
DataTypes key_types;
if (isTuple(key_column_type))
{
key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
}
else
{
key_columns = {key_column, range_col};
key_types = {std::make_shared<DataTypeUInt64>(), range_col_type};
}
return dictionary->hasKeys({key_column, range_col}, {std::make_shared<DataTypeUInt64>(), range_col_type});
key_columns.emplace_back(range_col);
key_types.emplace_back(range_col_type);
}
return dictionary->hasKeys(key_columns, key_types);
}
private:
@ -369,13 +357,14 @@ public:
auto dictionary = helper.getDictionary(dictionary_name);
auto dictionary_key_type = dictionary->getKeyType();
auto dictionary_special_key_type = dictionary->getSpecialKeyType();
size_t current_arguments_index = 3;
ColumnPtr range_col = nullptr;
DataTypePtr range_col_type = nullptr;
if (dictionary_key_type == DictionaryKeyType::range)
if (dictionary_special_key_type == DictionarySpecialKeyType::Range)
{
if (current_arguments_index >= arguments.size())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
@ -437,12 +426,13 @@ public:
default_cols.emplace_back(nullptr);
}
ColumnPtr result;
const auto & key_col_with_type = arguments[2];
auto key_column = key_col_with_type.column;
const ColumnWithTypeAndName & key_col_with_type = arguments[2];
const auto key_column = key_col_with_type.column;
Columns key_columns;
DataTypes key_types;
if (dictionary_key_type == DictionaryKeyType::simple)
if (dictionary_key_type == DictionaryKeyType::Simple)
{
if (!WhichDataType(key_col_with_type.type).isUInt64())
throw Exception(
@ -451,24 +441,24 @@ public:
getName(),
key_col_with_type.type->getName());
result = executeDictionaryRequest(
dictionary,
attribute_names,
{key_column},
{std::make_shared<DataTypeUInt64>()},
result_type,
default_cols);
key_columns = {key_column};
key_types = {std::make_shared<DataTypeUInt64>()};
}
else if (dictionary_key_type == DictionaryKeyType::complex)
else if (dictionary_key_type == DictionaryKeyType::Complex)
{
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column = key_col_with_type.column->convertToFullColumnIfConst();
key_column = key_column->convertToFullColumnIfConst();
DataTypePtr key_column_type = key_col_with_type.type;
size_t keys_size = dictionary->getStructure().getKeysSize();
if (!isTuple(key_column_type))
if (isTuple(key_column_type))
{
key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
}
else if (!isTuple(key_column_type))
{
size_t keys_size = dictionary->getStructure().getKeysSize();
if (keys_size > 1)
{
throw Exception(
@ -480,60 +470,19 @@ public:
}
else
{
Columns tuple_columns = {std::move(key_column)};
key_column = ColumnTuple::create(tuple_columns);
DataTypes tuple_types = {key_column_type};
key_column_type = std::make_shared<DataTypeTuple>(tuple_types);
key_columns = {std::move(key_column)};
key_types = {std::move(key_column_type)};
}
}
const auto & key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
const auto & key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
result = executeDictionaryRequest(
dictionary,
attribute_names,
key_columns,
key_types,
result_type,
default_cols);
}
else if (dictionary_key_type == DictionaryKeyType::range)
if (dictionary_special_key_type == DictionarySpecialKeyType::Range)
{
/// Functions in external dictionaries_loader only support full-value (not constant) columns with keys.
ColumnPtr key_column = key_col_with_type.column->convertToFullColumnIfConst();
DataTypePtr key_column_type = key_col_with_type.type;
Columns key_columns;
DataTypes key_types;
if (isTuple(key_column_type))
{
key_columns = assert_cast<const ColumnTuple &>(*key_column).getColumnsCopy();
key_types = assert_cast<const DataTypeTuple &>(*key_column_type).getElements();
}
else
{
key_columns = {key_column};
key_types = {std::make_shared<DataTypeUInt64>()};
}
key_columns.emplace_back(range_col);
key_types.emplace_back(range_col_type);
result = executeDictionaryRequest(
dictionary,
attribute_names,
key_columns,
key_types,
result_type,
default_cols);
}
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown dictionary identifier type");
return result;
return executeDictionaryRequest(dictionary, attribute_names, key_columns, key_types, result_type, default_cols);
}
private:

View File

@ -901,34 +901,67 @@ inline void writeText(const LocalDateTime & x, WriteBuffer & buf) { writeDateTim
inline void writeText(const UUID & x, WriteBuffer & buf) { writeUUIDText(x, buf); }
template <typename T>
String decimalFractional(const T & x, UInt32 scale)
void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros)
{
/// If it's big integer, but the number of digits is small,
/// use the implementation for smaller integers for more efficient arithmetic.
if constexpr (std::is_same_v<T, Int256>)
{
if (x <= std::numeric_limits<UInt32>::max())
return decimalFractional(static_cast<UInt32>(x), scale);
{
writeDecimalFractional(static_cast<UInt32>(x), scale, ostr, trailing_zeros);
return;
}
else if (x <= std::numeric_limits<UInt64>::max())
return decimalFractional(static_cast<UInt64>(x), scale);
{
writeDecimalFractional(static_cast<UInt64>(x), scale, ostr, trailing_zeros);
return;
}
else if (x <= std::numeric_limits<UInt128>::max())
return decimalFractional(static_cast<UInt128>(x), scale);
{
writeDecimalFractional(static_cast<UInt128>(x), scale, ostr, trailing_zeros);
return;
}
}
else if constexpr (std::is_same_v<T, Int128>)
{
if (x <= std::numeric_limits<UInt32>::max())
return decimalFractional(static_cast<UInt32>(x), scale);
{
writeDecimalFractional(static_cast<UInt32>(x), scale, ostr, trailing_zeros);
return;
}
else if (x <= std::numeric_limits<UInt64>::max())
return decimalFractional(static_cast<UInt64>(x), scale);
{
writeDecimalFractional(static_cast<UInt64>(x), scale, ostr, trailing_zeros);
return;
}
}
String str(scale, '0');
constexpr size_t max_digits = std::numeric_limits<UInt256>::digits10;
assert(scale <= max_digits);
char buf[max_digits];
memset(buf, '0', scale);
T value = x;
for (Int32 pos = scale - 1; pos >= 0; --pos, value /= 10)
str[pos] += static_cast<char>(value % 10);
return str;
Int32 last_nonzero_pos = 0;
for (Int32 pos = scale - 1; pos >= 0; --pos)
{
auto remainder = value % 10;
value /= 10;
if (remainder != 0 && last_nonzero_pos == 0)
last_nonzero_pos = pos;
buf[pos] += static_cast<char>(remainder);
}
writeChar('.', ostr);
ostr.write(buf, trailing_zeros ? scale : last_nonzero_pos + 1);
}
template <typename T>
void writeText(Decimal<T> x, UInt32 scale, WriteBuffer & ostr)
void writeText(Decimal<T> x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros)
{
T part = DecimalUtils::getWholePart(x, scale);
@ -941,10 +974,9 @@ void writeText(Decimal<T> x, UInt32 scale, WriteBuffer & ostr)
if (scale)
{
writeChar('.', ostr);
part = DecimalUtils::getFractionalPart(x, scale);
String fractional = decimalFractional(part, scale);
ostr.write(fractional.data(), scale);
if (part || trailing_zeros)
writeDecimalFractional(part, scale, ostr, trailing_zeros);
}
}

View File

@ -1175,7 +1175,6 @@ void DDLWorker::runMainThread()
new_stat.numChildren,
new_stat.pzxid);
context->getZooKeeperLog()->flush();
abort();
}
}
}

View File

@ -8,6 +8,7 @@
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSelectIntersectExceptQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h>
@ -878,16 +879,39 @@ static bool hasWithTotalsInAnySubqueryInFromClause(const ASTSelectQuery & query)
{
if (const auto * ast_union = query_table->as<ASTSelectWithUnionQuery>())
{
/// NOTE: Child of subquery can be ASTSelectWithUnionQuery or ASTSelectQuery,
/// and after normalization, the height of the AST tree is at most 2
for (const auto & elem : ast_union->list_of_selects->children)
{
/// After normalization for union child node the height of the AST tree is at most 2.
if (const auto * child_union = elem->as<ASTSelectWithUnionQuery>())
{
for (const auto & child_elem : child_union->list_of_selects->children)
if (hasWithTotalsInAnySubqueryInFromClause(child_elem->as<ASTSelectQuery &>()))
return true;
}
/// After normalization in case there are intersect or except nodes, the height of
/// the AST tree can have any depth (each intersect/except adds a level), but the
/// number of children in those nodes is always 2.
else if (elem->as<ASTSelectIntersectExceptQuery>())
{
std::function<bool(ASTPtr)> traverse_recursively = [&](ASTPtr child_ast) -> bool
{
if (const auto * child = child_ast->as <ASTSelectQuery>())
return hasWithTotalsInAnySubqueryInFromClause(child->as<ASTSelectQuery &>());
if (const auto * child = child_ast->as<ASTSelectWithUnionQuery>())
for (const auto & subchild : child->list_of_selects->children)
if (traverse_recursively(subchild))
return true;
if (const auto * child = child_ast->as<ASTSelectIntersectExceptQuery>())
for (const auto & subchild : child->children)
if (traverse_recursively(subchild))
return true;
return false;
};
if (traverse_recursively(elem))
return true;
}
else
{
if (hasWithTotalsInAnySubqueryInFromClause(elem->as<ASTSelectQuery &>()))

View File

@ -33,18 +33,21 @@ void ASTSelectWithUnionQuery::formatQueryImpl(const FormatSettings & settings, F
auto mode_to_str = [&](auto mode)
{
if (mode == Mode::Unspecified)
return "";
else if (mode == Mode::ALL)
return " ALL";
else
return " DISTINCT";
if (mode == Mode::ALL)
return "UNION ALL";
else if (mode == Mode::DISTINCT)
return "UNION DISTINCT";
else if (mode == Mode::INTERSECT)
return "INTERSECT";
else if (mode == Mode::EXCEPT)
return "EXCEPT";
return "";
};
for (ASTs::const_iterator it = list_of_selects->children.begin(); it != list_of_selects->children.end(); ++it)
{
if (it != list_of_selects->children.begin())
settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "") << "UNION"
settings.ostr << settings.nl_or_ws << indent_str << (settings.hilite ? hilite_keyword : "")
<< mode_to_str((is_normalized) ? union_mode : list_of_modes[it - list_of_selects->children.begin() - 1])
<< (settings.hilite ? hilite_none : "");

View File

@ -232,7 +232,7 @@ uint16_values = [0, 1, 65535]
int8_values = [0, 1, -1, 127, -128]
uint8_values = [0, 1, 255]
# string_values = ["'ClickHouse'", 'NULL']
string_values = ["'ClickHouse'"]
string_values = ["'ClickHouse'"]
decimal_values = [0, 0.123, 0.4, 5.67, 8.91011, 123456789.123, -0.123, -0.4, -5.67, -8.91011, -123456789.123]
@ -319,7 +319,8 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m
)
clickhouse_query_settings = dict(
mysql_datatypes_support_level=setting_mysql_datatypes_support_level
mysql_datatypes_support_level=setting_mysql_datatypes_support_level,
output_format_decimal_trailing_zeros=1
)
def execute_query(node, query, **kwargs):

View File

@ -258,7 +258,7 @@ def test_different_data_types(started_cluster):
check_tables_are_synchronized('test_data_types', 'id');
result = instance.query('SELECT * FROM test_database.test_data_types ORDER BY id LIMIT 1;')
assert(result == '0\t-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12.012345\t2000-05-12\t0.20000\t0.20000\n')
assert(result == '0\t-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12.012345\t2000-05-12\t0.2\t0.2\n')
for i in range(10):
col = random.choice(['a', 'b', 'c'])

View File

@ -0,0 +1,22 @@
<yandex>
<remote_servers>
<default>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</default>
</remote_servers>
<user_directories replace="replace">
<replicated>
<zookeeper_path>/clickhouse/access</zookeeper_path>
</replicated>
</user_directories>
</yandex>

View File

@ -0,0 +1,73 @@
import pytest
from dataclasses import dataclass
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/config.xml'], with_zookeeper=True, stay_alive=True)
node2 = cluster.add_instance('node2', main_configs=['configs/config.xml'], with_zookeeper=True, stay_alive=True)
all_nodes = [node1, node2]
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
@dataclass(frozen=True)
class Entity:
keyword: str
name: str
options: str = ""
entities = [
Entity(keyword="USER", name="theuser"),
Entity(keyword="ROLE", name="therole"),
Entity(keyword="ROW POLICY", name="thepolicy", options=" ON default.t1"),
Entity(keyword="QUOTA", name="thequota"),
Entity(keyword="SETTINGS PROFILE", name="theprofile")
]
def get_entity_id(entity):
return entity.keyword
@pytest.mark.parametrize("entity", entities, ids=get_entity_id)
def test_create_replicated(started_cluster, entity):
node1.query(f"CREATE {entity.keyword} {entity.name} {entity.options}")
assert f"cannot insert because {entity.keyword.lower()} `{entity.name}{entity.options}` already exists in replicated" in \
node2.query_and_get_error(f"CREATE {entity.keyword} {entity.name} {entity.options}")
node1.query(f"DROP {entity.keyword} {entity.name} {entity.options}")
@pytest.mark.parametrize("entity", entities, ids=get_entity_id)
def test_create_and_delete_replicated(started_cluster, entity):
node1.query(f"CREATE {entity.keyword} {entity.name} {entity.options}")
node2.query(f"DROP {entity.keyword} {entity.name} {entity.options}")
@pytest.mark.parametrize("entity", entities, ids=get_entity_id)
def test_create_replicated_on_cluster(started_cluster, entity):
assert f"cannot insert because {entity.keyword.lower()} `{entity.name}{entity.options}` already exists in replicated" in \
node1.query_and_get_error(f"CREATE {entity.keyword} {entity.name} ON CLUSTER default {entity.options}")
node1.query(f"DROP {entity.keyword} {entity.name} {entity.options}")
@pytest.mark.parametrize("entity", entities, ids=get_entity_id)
def test_create_replicated_if_not_exists_on_cluster(started_cluster, entity):
node1.query(f"CREATE {entity.keyword} IF NOT EXISTS {entity.name} ON CLUSTER default {entity.options}")
node1.query(f"DROP {entity.keyword} {entity.name} {entity.options}")
@pytest.mark.parametrize("entity", entities, ids=get_entity_id)
def test_rename_replicated(started_cluster, entity):
node1.query(f"CREATE {entity.keyword} {entity.name} {entity.options}")
node2.query(f"ALTER {entity.keyword} {entity.name} {entity.options} RENAME TO {entity.name}2")
node1.query(f"DROP {entity.keyword} {entity.name}2 {entity.options}")

View File

@ -0,0 +1,3 @@
<test>
<query>SELECT count() FROM zeros(10000000) WHERE NOT ignore(toString((rand() / 1000000)::Decimal64(6)))</query>
</test>

View File

@ -1,5 +1,5 @@
0 9
0 9
1970-01-01 1970-01-10
0.00 9.00
0 9
4 1

View File

@ -22,5 +22,5 @@
([1.01],[1])
(['a','b'],[1,2])
(['a','ab','abc'],[3,2,1])
([1,2,3,4,5,6,7,8],[1.00000,2.00000,6.00000,8.00000,10.00000,12.00000,7.00000,8.00000])
([1,2,3,4,5,6,7,8],[1.00000,2.00000,6.00000,8.00000,10.00000,12.00000,7.00000,8.00000])
([1,2,3,4,5,6,7,8],[1,2,6,8,10,12,7,8])
([1,2,3,4,5,6,7,8],[1,2,6,8,10,12,7,8])

View File

@ -1,72 +1,72 @@
101 101 101
[-50.0000,50.0000] [-16.66666666,16.66666666] [-10.00000000,10.00000000]
0.0000 0.00000000 0.00000000 0.0000 0.00000000 0.00000000
1275.0000 424.99999983 255.00000000 1275.0000 424.99999983 255.00000000
-1275.0000 -424.99999983 -255.00000000 -1275.0000 -424.99999983 -255.00000000
101.0000 101.00000000 101.00000000 101.0000 101.00000000 101.00000000
-101.0000 -101.00000000 -101.00000000 -101.0000 -101.00000000 -101.00000000
[-50,50] [-16.66666666,16.66666666] [-10,10]
0 0 0 0 0 0
1275 424.99999983 255 1275 424.99999983 255
-1275 -424.99999983 -255 -1275 -424.99999983 -255
101 101 101 101 101 101
-101 -101 -101 -101 -101 -101
(101,101,101) (101,101,101) (101,101,101) (101,101,101) (102,100,101)
5 5 5
10 10 10
-50.0000 -50.0000 -16.66666666 -16.66666666 -10.00000000 -10.00000000
1.0000 1.0000 0.33333333 0.33333333 0.20000000 0.20000000
50.0000 50.0000 16.66666666 16.66666666 10.00000000 10.00000000
-1.0000 -1.0000 -0.33333333 -0.33333333 -0.20000000 -0.20000000
0.0000 0.00000000 0.00000000 Decimal(38, 8)
-25.5000 -8.49999999 -5.10000000 Decimal(38, 8)
0.0000 0.00000000 0.00000000
10.0000 3.33333333 2.00000000
20.0000 6.66666666 4.00000000
30.0000 10.00000000 6.00000000
40.0000 13.33333333 8.00000000
50.0000 16.66666666 10.00000000
[-50.0000,-40.0000,-30.0000,-20.0000,-10.0000,0.0000,10.0000,20.0000,30.0000,40.0000,50.0000]
[-16.66666666,-13.33333333,-10.00000000,-6.66666666,-3.33333333,0.00000000,3.33333333,6.66666666,10.00000000,13.33333333,16.66666666]
[-10.00000000,-8.00000000,-6.00000000,-4.00000000,-2.00000000,0.00000000,2.00000000,4.00000000,6.00000000,8.00000000,10.00000000]
0.0000 0.00000000 0.00000000 Decimal(38, 8)
-25.0000 -8.33333333 -5.00000000 Decimal(38, 8)
0.0000 0.00000000 0.00000000
10.0000 3.33333333 2.00000000
20.0000 6.66666666 4.00000000
30.0000 10.00000000 6.00000000
40.0000 13.33333333 8.00000000
50.0000 16.66666666 10.00000000
[-50.0000,-40.0000,-30.0000,-20.0000,-10.0000,0.0000,10.0000,20.0000,30.0000,40.0000,50.0000]
[-16.66666666,-13.33333333,-10.00000000,-6.66666666,-3.33333333,0.00000000,3.33333333,6.66666666,10.00000000,13.33333333,16.66666666]
[-10.00000000,-8.00000000,-6.00000000,-4.00000000,-2.00000000,0.00000000,2.00000000,4.00000000,6.00000000,8.00000000,10.00000000]
0.0000 0.00000000 0.00000000 Decimal(38, 8)
-26.0000 -8.66666666 -5.20000000 Decimal(38, 8)
0.0000 0.00000000 0.00000000
10.0000 3.33333333 2.00000000
20.0000 6.66666666 4.00000000
30.0000 10.00000000 6.00000000
40.0000 13.33333333 8.00000000
50.0000 16.66666666 10.00000000
[-50.0000,-40.0000,-30.0000,-20.0000,-10.0000,0.0000,10.0000,20.0000,30.0000,40.0000,50.0000]
[-16.66666666,-13.33333333,-10.00000000,-6.66666666,-3.33333333,0.00000000,3.33333333,6.66666666,10.00000000,13.33333333,16.66666666]
[-10.00000000,-8.00000000,-6.00000000,-4.00000000,-2.00000000,0.00000000,2.00000000,4.00000000,6.00000000,8.00000000,10.00000000]
0.0000 0.00000000 0.00000000 Decimal(38, 8)
-25.0000 -8.33333333 -5.00000000 Decimal(38, 8)
0.0000 0.00000000 0.00000000
10.0000 3.33333333 2.00000000
20.0000 6.66666666 4.00000000
30.0000 10.00000000 6.00000000
40.0000 13.33333333 8.00000000
50.0000 16.66666666 10.00000000
[-50.0000,-40.0000,-30.0000,-20.0000,-10.0000,0.0000,10.0000,20.0000,30.0000,40.0000,50.0000]
[-16.66666666,-13.33333333,-10.00000000,-6.66666666,-3.33333333,0.00000000,3.33333333,6.66666666,10.00000000,13.33333333,16.66666666]
[-10.00000000,-8.00000000,-6.00000000,-4.00000000,-2.00000000,0.00000000,2.00000000,4.00000000,6.00000000,8.00000000,10.00000000]
0.0000 0.00000000 0.00000000 Decimal(38, 8)
-26.0000 -8.66666666 -5.20000000 Decimal(38, 8)
0.0000 0.00000000 0.00000000
10.0000 3.33333333 2.00000000
20.0000 6.66666666 4.00000000
30.0000 10.00000000 6.00000000
40.0000 13.33333333 8.00000000
50.0000 16.66666666 10.00000000
[-50.0000,-40.0000,-30.0000,-20.0000,-10.0000,0.0000,10.0000,20.0000,30.0000,40.0000,50.0000]
[-16.66666666,-13.33333333,-10.00000000,-6.66666666,-3.33333333,0.00000000,3.33333333,6.66666666,10.00000000,13.33333333,16.66666666]
[-10.00000000,-8.00000000,-6.00000000,-4.00000000,-2.00000000,0.00000000,2.00000000,4.00000000,6.00000000,8.00000000,10.00000000]
-50 -50 -16.66666666 -16.66666666 -10 -10
1 1 0.33333333 0.33333333 0.2 0.2
50 50 16.66666666 16.66666666 10 10
-1 -1 -0.33333333 -0.33333333 -0.2 -0.2
0 0 0 Decimal(38, 8)
-25.5 -8.49999999 -5.1 Decimal(38, 8)
0 0 0
10 3.33333333 2
20 6.66666666 4
30 10 6
40 13.33333333 8
50 16.66666666 10
[-50,-40,-30,-20,-10,0,10,20,30,40,50]
[-16.66666666,-13.33333333,-10,-6.66666666,-3.33333333,0,3.33333333,6.66666666,10,13.33333333,16.66666666]
[-10,-8,-6,-4,-2,0,2,4,6,8,10]
0 0 0 Decimal(38, 8)
-25 -8.33333333 -5 Decimal(38, 8)
0 0 0
10 3.33333333 2
20 6.66666666 4
30 10 6
40 13.33333333 8
50 16.66666666 10
[-50,-40,-30,-20,-10,0,10,20,30,40,50]
[-16.66666666,-13.33333333,-10,-6.66666666,-3.33333333,0,3.33333333,6.66666666,10,13.33333333,16.66666666]
[-10,-8,-6,-4,-2,0,2,4,6,8,10]
0 0 0 Decimal(38, 8)
-26 -8.66666666 -5.2 Decimal(38, 8)
0 0 0
10 3.33333333 2
20 6.66666666 4
30 10 6
40 13.33333333 8
50 16.66666666 10
[-50,-40,-30,-20,-10,0,10,20,30,40,50]
[-16.66666666,-13.33333333,-10,-6.66666666,-3.33333333,0,3.33333333,6.66666666,10,13.33333333,16.66666666]
[-10,-8,-6,-4,-2,0,2,4,6,8,10]
0 0 0 Decimal(38, 8)
-25 -8.33333333 -5 Decimal(38, 8)
0 0 0
10 3.33333333 2
20 6.66666666 4
30 10 6
40 13.33333333 8
50 16.66666666 10
[-50,-40,-30,-20,-10,0,10,20,30,40,50]
[-16.66666666,-13.33333333,-10,-6.66666666,-3.33333333,0,3.33333333,6.66666666,10,13.33333333,16.66666666]
[-10,-8,-6,-4,-2,0,2,4,6,8,10]
0 0 0 Decimal(38, 8)
-26 -8.66666666 -5.2 Decimal(38, 8)
0 0 0
10 3.33333333 2
20 6.66666666 4
30 10 6
40 13.33333333 8
50 16.66666666 10
[-50,-40,-30,-20,-10,0,10,20,30,40,50]
[-16.66666666,-13.33333333,-10,-6.66666666,-3.33333333,0,3.33333333,6.66666666,10,13.33333333,16.66666666]
[-10,-8,-6,-4,-2,0,2,4,6,8,10]
850 94.44444438684269 34 Float64 Float64 Float64
850 94.4444443868427 34.00000000000001
858.5 95.38888883071111 34.34 Float64 Float64 Float64

View File

@ -1,37 +1,37 @@
84 0 1764 1 1 1
84 0 1764 1 1 1
84 0 1764 1 1 1
84.840 0.000 1799.456400 1.000 1.000 1.000
84.840000000 0.000000000
84.840000000000000000 0.000000000000000000 98.044565395307682683126962841158942720 1.000000000000000000 1.000000000000000000 1.000000000000000000
84.840000000000000000 0.000000000000000000
84.84 0.00 1799.4564 1.00 1.00 1.00
84.84 0 1799.4564 1 1 1
84.84 0
84.84 0 98.04456539530768268312696284115894272 1 1 1
84.84 0
84.84 0 1799.4564 1 1 1
63 21 -42 882 -882 2 0 2 0
63 21 -42 882 -882 2 0 2 0
63 21 -42 882 -882 2 0 2 0
1.00305798474369219219752355409390731264 -0.16305798474369219219752355409390731264 1.49059173023461586584365185794205286400 -1.38847100762815390390123822295304634368 1.38847100762815390390123822295304634368 0.02000000000000000000000000000000000000 0.00500000000000000000000000000000000000
63.420 21.420 -41.580 890.820 -890.820 2.020 0.505 2.020 0.505
63.420000000 21.420000000 -41.580000000 890.820000000 -890.820000000 2.020000000 0.505000000 2.020000000 0.505000000
63.420000000000000000 21.420000000000000000 -41.580000000000000000 890.820000000000000000 -890.820000000000000000 2.020000000000000000 0.505000000000000000 2.020000000000000000 0.505000000000000000
63.42 21.42 -41.58 890.82 -890.82 2.02 0.50 2.02 0.50
1.00305798474369219219752355409390731264 -0.16305798474369219219752355409390731264 1.490591730234615865843651857942052864 -1.38847100762815390390123822295304634368 1.38847100762815390390123822295304634368 0.02 0.005
63.42 21.42 -41.58 890.82 -890.82 2.02 0.505 2.02 0.505
63.42 21.42 -41.58 890.82 -890.82 2.02 0.505 2.02 0.505
63.42 21.42 -41.58 890.82 -890.82 2.02 0.505 2.02 0.505
63.42 21.42 -41.58 890.82 -890.82 2.02 0.5 2.02 0.5
63 -21 42 882 -882 0 2 0 2
63 -21 42 882 -882 0 2 0 2
63 -21 42 882 -882 0 2 0 2
1.00305798474369219219752355409390731264 0.16305798474369219219752355409390731264 -1.49059173023461586584365185794205286400 -1.38847100762815390390123822295304634368 1.38847100762815390390123822295304634368 -0.00000000000000000000000000000000000001 0.00000000000000000000000000000000000001
63.420 -21.420 41.580 890.820 -890.820 0.495 1.980 0.495 1.980
63.420000000 -21.420000000 41.580000000 890.820000000 -890.820000000
63.420000000000000000 -21.420000000000000000 41.580000000000000000 890.820000000000000000 -890.820000000000000000 0.495049504950495049 1.980198019801980198 0.495049504950495049 1.980198019801980198
1.00305798474369219219752355409390731264 0.16305798474369219219752355409390731264 -1.490591730234615865843651857942052864 -1.38847100762815390390123822295304634368 1.38847100762815390390123822295304634368 -0.00000000000000000000000000000000000001 0.00000000000000000000000000000000000001
63.42 -21.42 41.58 890.82 -890.82 0.495 1.98 0.495 1.98
63.42 -21.42 41.58 890.82 -890.82
63.42 -21.42 41.58 890.82 -890.82 0.495049504950495049 1.980198019801980198 0.495049504950495049 1.980198019801980198
63.42 -21.42 41.58 890.82 -890.82 0.49 1.98 0.49 1.98
-42 42 42 42 0.420000000 0.420000000000000000 0.42000000000000000000000000000000000000 42.420 42.420000000 42.42
0 0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.000 0.000000000 0.00
42 -42 -42 -42 -0.420000000 -0.420000000000000000 -0.42000000000000000000000000000000000000 -42.420 -42.420000000 -42.42
42 42 42 0.420000000 0.420000000000000000 0.42000000000000000000000000000000000000 42.420 42.420000000 42.42
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.000 0.000000000 0.00
42 42 42 0.420000000 0.420000000000000000 0.42000000000000000000000000000000000000 42.420 42.420000000 42.42
-42 42 42 42 0.42 0.42 0.42 42.42 42.42 42.42
0 0 0 0 0 0 0 0 0 0
42 -42 -42 -42 -0.42 -0.42 -0.42 -42.42 -42.42 -42.42
42 42 42 0.42 0.42 0.42 42.42 42.42 42.42
0 0 0 0 0 0 0 0 0
42 42 42 0.42 0.42 0.42 42.42 42.42 42.42
1 1
1 1
1 0 1 0
1 0 1 0
0.0000 \N \N
0.00000000 \N \N
0.000000000000000000 \N \N
0 \N \N
0 \N \N
0 \N \N

View File

@ -1,20 +1,20 @@
[0.0000,1.0000] Array(Decimal(9, 4))
[0.00000000,1.00000000] Array(Decimal(18, 8))
[0.00000000,1.00000000] Array(Decimal(38, 8))
[0,1] Array(Decimal(9, 4))
[0,1] Array(Decimal(18, 8))
[0,1] Array(Decimal(38, 8))
-
1.0000 Decimal(38, 4)
1.00000000 Decimal(38, 8)
1.00000000 Decimal(38, 8)
1 Decimal(38, 4)
1 Decimal(38, 8)
1 Decimal(38, 8)
-
[1.0000,2.0000] Array(Decimal(38, 4))
[1.00000000,2.00000000] Array(Decimal(38, 8))
[1.00000000,2.00000000] Array(Decimal(38, 8))
[1,2] Array(Decimal(38, 4))
[1,2] Array(Decimal(38, 8))
[1,2] Array(Decimal(38, 8))
-
[1.0000,2.0000] Array(Decimal(38, 4))
[1.00000000,2.00000000] Array(Decimal(38, 8))
[1.00000000,2.00000000] Array(Decimal(38, 8))
[1,2] Array(Decimal(38, 4))
[1,2] Array(Decimal(38, 8))
[1,2] Array(Decimal(38, 8))
-
[1.0000] Array(Decimal(9, 4))
[1.00000000] Array(Decimal(18, 8))
[1.00000000] Array(Decimal(38, 8))
[1] Array(Decimal(9, 4))
[1] Array(Decimal(18, 8))
[1] Array(Decimal(38, 8))
-

View File

@ -1,43 +1,43 @@
-999999999 -999999999999999999 0 -0.999999999 0.000000000000000000 0.00000000000000000000000000000000000000 -9999.99999 0.000000000 0.000000000000000000 0
-900000000 -900000000000000000 -90000000000000000000000000000000000000 -0.000000009 -0.000000000000000009 -0.00000000000000000000000000000000000009 0.00000 0.000000000 0.000000000000000000 0
-1 -1 -1 -0.000000001 0.000000000000000000 0.00000000000000000000000000000000000000 -0.00001 -0.000000001 0.000000000000000000 -1
0 0 -99999999999999999999999999999999999999 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 -0.999999999999999999 0.00000000000000000000000000000000000000 0.00000 -999999999.999999999 0.000000000000000000 0
0 0 0 0.000000000 -0.000000000000000001 -0.00000000000000000000000000000000000001 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 -0.99999999999999999999999999999999999999 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 -99999999999999999999.999999999999999999 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 -0.000000000000000001 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000001 0
0 0 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 99999999999999999999.999999999999999999 0
0 0 0 0.000000000 0.000000000000000000 0.99999999999999999999999999999999999999 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.000000000000000001 0.00000000000000000000000000000000000001 0.00000 0.000000000 0.000000000000000000 0
0 0 0 0.000000000 0.999999999999999999 0.00000000000000000000000000000000000000 0.00000 999999999.999999999 0.000000000000000000 0
0 0 99999999999999999999999999999999999999 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.00000 0.000000000 0.000000000000000000 0
1 1 1 0.000000001 0.000000000000000000 0.00000000000000000000000000000000000000 0.00001 0.000000001 0.000000000000000000 1
42 42 0 0.000000000 0.000000000000000000 0.00000000000000000000000000000000000000 0.99999 0.000000000 0.000000000000000000 0
900000000 900000000000000000 90000000000000000000000000000000000000 0.000000009 0.000000000000000009 0.00000000000000000000000000000000000009 0.00000 0.000000000 0.000000000000000000 0
999999999 999999999999999999 0 0.999999999 0.000000000000000000 0.00000000000000000000000000000000000000 9999.99999 0.000000000 0.000000000000000000 0
-999999999 -999999999999999999 0 -0.999999999 0 0 -9999.99999 0 0 0
-900000000 -900000000000000000 -90000000000000000000000000000000000000 -0.000000009 -0.000000000000000009 -0.00000000000000000000000000000000000009 0 0 0 0
-1 -1 -1 -0.000000001 0 0 -0.00001 -0.000000001 0 -1
0 0 -99999999999999999999999999999999999999 0 0 0 0 0 0 0
0 0 0 0 -0.999999999999999999 0 0 -999999999.999999999 0 0
0 0 0 0 -0.000000000000000001 -0.00000000000000000000000000000000000001 0 0 0 0
0 0 0 0 0 -0.99999999999999999999999999999999999999 0 0 0 0
0 0 0 0 0 0 0 0 -99999999999999999999.999999999999999999 0
0 0 0 0 0 0 0 0 -0.000000000000000001 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0 0
0 0 0 0 0 0 0 0 0.000000000000000001 0
0 0 0 0 0 0 0 0 99999999999999999999.999999999999999999 0
0 0 0 0 0 0.99999999999999999999999999999999999999 0 0 0 0
0 0 0 0 0.000000000000000001 0.00000000000000000000000000000000000001 0 0 0 0
0 0 0 0 0.999999999999999999 0 0 999999999.999999999 0 0
0 0 99999999999999999999999999999999999999 0 0 0 0 0 0 0
1 1 1 0.000000001 0 0 0.00001 0.000000001 0 1
42 42 0 0 0 0 0.99999 0 0 0
900000000 900000000000000000 90000000000000000000000000000000000000 0.000000009 0.000000000000000009 0.00000000000000000000000000000000000009 0 0 0 0
999999999 999999999999999999 0 0.999999999 0 0 9999.99999 0 0 0

View File

@ -1,30 +1,30 @@
1.1 1.10 1.10000000
1.1 1.1 1.1
1
1 1.1 1.10 1.10000000
1 1.1 1.1 1.1
0.1 0
-0.1 0
0.1 0
-0.1 0
0.1 0
-0.1 0
0.0000000001 0.000000000
-0.0000000001 0.000000000
0.0000000000000000001 0.000000000000000000
-0.0000000000000000001 0.000000000000000000
0.000000000000000000000000000000000000001 0.00000000000000000000000000000000000000
-0.000000000000000000000000000000000000001 0.00000000000000000000000000000000000000
0.0000000001 0
-0.0000000001 0
0.0000000000000000001 0
-0.0000000000000000001 0
0.000000000000000000000000000000000000001 0
-0.000000000000000000000000000000000000001 0
1e-1 0
-1e-1 0
1e-1 0
-1e-1 0
1e-1 0
-1e-1 0
1e-10 0.000000000
-1e-10 0.000000000
1e-19 0.000000000000000000
-1e-19 0.000000000000000000
1e-39 0.00000000000000000000000000000000000000
-1e-39 0.00000000000000000000000000000000000000
1e-10 0
-1e-10 0
1e-19 0
-1e-19 0
1e-39 0
-1e-39 0
9999999 9999999 -9999999 9999999 -9999999
999999.9 999999.9 -999999.9 999999.9 -999999.9
99999.99 99999.99 -99999.99 99999.99 -99999.99
@ -33,8 +33,8 @@
99.99999 99.99999 -99.99999 99.99999 -99.99999
9.999999 9.999999 -9.999999 9.999999 -9.999999
0.9999999 0.9999999 -0.9999999 0.9999999 -0.9999999
10 10.00000000 -10.00000000 10.00000000 -10.00000000
1 1.000000000 -1.000000000 1.000000000 -1.000000000
10 10 -10 10 -10
1 1 -1 1 -1
999999999 999999999 -999999999 999999999 -999999999
99999999.9 99999999.9 -99999999.9 99999999.9 -99999999.9
9999999.99 9999999.99 -9999999.99 9999999.99 -9999999.99
@ -45,111 +45,111 @@
99.9999999 99.9999999 -99.9999999 99.9999999 -99.9999999
9.99999999 9.99999998 -9.99999998 9.99999998 -9.99999998
0.999999999 0.999999999 -0.999999999 0.999999999 -0.999999999
1000000000 1000000000.000000000 -1000000000.000000000
100000000 100000000.0000000000 -100000000.0000000000
10000000 10000000.00000000000 -10000000.00000000000
1000000 1000000.000000000000 -1000000.000000000000
100000 100000.0000000000000 -100000.0000000000000
10000 10000.00000000000000 -10000.00000000000000
1000 1000.000000000000000 -1000.000000000000000
100 100.0000000000000000 -100.0000000000000000
10 10.00000000000000000 -10.00000000000000000
1 1.000000000000000000 -1.000000000000000000
1000000000 1000000000 -1000000000
100000000 100000000 -100000000
10000000 10000000 -10000000
1000000 1000000 -1000000
100000 100000 -100000
10000 10000 -10000
1000 1000 -1000
100 100 -100
10 10 -10
1 1 -1
1000000000000000000 1000000000000000000 -1000000000000000000
100000000000000000 100000000000000000.0 -100000000000000000.0
10000000000000000 10000000000000000.00 -10000000000000000.00
1000000000000000 1000000000000000.000 -1000000000000000.000
100000000000000 100000000000000.0000 -100000000000000.0000
10000000000000 10000000000000.00000 -10000000000000.00000
1000000000000 1000000000000.000000 -1000000000000.000000
100000000000 100000000000.0000000 -100000000000.0000000
10000000000 10000000000.00000000 -10000000000.00000000
1000000000 1000000000.000000000 -1000000000.000000000
1000000000 1000000000.000000000 -1000000000.000000000
100000000 100000000.0000000000 -100000000.0000000000
10000000 10000000.00000000000 -10000000.00000000000
1000000 1000000.000000000000 -1000000.000000000000
100000 100000.0000000000000 -100000.0000000000000
10000 10000.00000000000000 -10000.00000000000000
1000 1000.000000000000000 -1000.000000000000000
100 100.0000000000000000 -100.0000000000000000
10 10.00000000000000000 -10.00000000000000000
1 1.000000000000000000 -1.000000000000000000
0.0000 0.00 0.00000000
1.0000 0.11 0.11000000
2.0000 0.22 0.22000000
3.0000 0.33 0.33000000
4.0000 0.44 0.44000000
5.0000 0.55 0.55000000
6.0000 0.66 0.66000000
7.0000 0.77 0.77000000
8.0000 0.88 0.88000000
9.0000 1.00 1.00000000
0.0000 0.00000000 0.00
1.0000 0.11110000 0.11
2.0000 0.22220000 0.22
3.0000 0.33330000 0.33
4.0000 0.44440000 0.44
5.0000 0.55550000 0.55
6.0000 0.66660000 0.66
7.0000 0.77770000 0.77
8.0000 0.88880000 0.88
9.0000 1.00000000 1.00
0.00000000 0.0000 0.00
1.00000000 0.1111 0.11
2.00000000 0.2222 0.22
3.00000000 0.3333 0.33
4.00000000 0.4444 0.44
5.00000000 0.5555 0.55
6.00000000 0.6666 0.66
7.00000000 0.7777 0.77
8.00000000 0.8888 0.88
9.00000000 1.0000 1.00
0.0000 0.00 0.00000000
1.0000 0.11 0.11000000
2.0000 0.22 0.22000000
3.0000 0.33 0.33000000
4.0000 0.44 0.44000000
5.0000 0.55 0.55000000
6.0000 0.66 0.66000000
7.0000 0.77 0.77000000
8.0000 0.88 0.88000000
9.0000 1.00 1.00000000
0.0000 0.00000000 0.00
1.0000 0.11110000 0.11
2.0000 0.22220000 0.22
3.0000 0.33330000 0.33
4.0000 0.44440000 0.44
5.0000 0.55550000 0.55
6.0000 0.66660000 0.66
7.0000 0.77770000 0.77
8.0000 0.88880000 0.88
9.0000 1.00000000 1.00
0.00000000 0.0000 0.00
1.00000000 0.1111 0.11
2.00000000 0.2222 0.22
3.00000000 0.3333 0.33
4.00000000 0.4444 0.44
5.00000000 0.5555 0.55
6.00000000 0.6666 0.66
7.00000000 0.7777 0.77
8.00000000 0.8888 0.88
9.00000000 1.0000 1.00
100000000000000000 100000000000000000 -100000000000000000
10000000000000000 10000000000000000 -10000000000000000
1000000000000000 1000000000000000 -1000000000000000
100000000000000 100000000000000 -100000000000000
10000000000000 10000000000000 -10000000000000
1000000000000 1000000000000 -1000000000000
100000000000 100000000000 -100000000000
10000000000 10000000000 -10000000000
1000000000 1000000000 -1000000000
1000000000 1000000000 -1000000000
100000000 100000000 -100000000
10000000 10000000 -10000000
1000000 1000000 -1000000
100000 100000 -100000
10000 10000 -10000
1000 1000 -1000
100 100 -100
10 10 -10
1 1 -1
0 0 0
1 0.11 0.11
2 0.22 0.22
3 0.33 0.33
4 0.44 0.44
5 0.55 0.55
6 0.66 0.66
7 0.77 0.77
8 0.88 0.88
9 1 1
0 0 0
1 0.1111 0.11
2 0.2222 0.22
3 0.3333 0.33
4 0.4444 0.44
5 0.5555 0.55
6 0.6666 0.66
7 0.7777 0.77
8 0.8888 0.88
9 1 1
0 0 0
1 0.1111 0.11
2 0.2222 0.22
3 0.3333 0.33
4 0.4444 0.44
5 0.5555 0.55
6 0.6666 0.66
7 0.7777 0.77
8 0.8888 0.88
9 1 1
0 0 0
1 0.11 0.11
2 0.22 0.22
3 0.33 0.33
4 0.44 0.44
5 0.55 0.55
6 0.66 0.66
7 0.77 0.77
8 0.88 0.88
9 1 1
0 0 0
1 0.1111 0.11
2 0.2222 0.22
3 0.3333 0.33
4 0.4444 0.44
5 0.5555 0.55
6 0.6666 0.66
7 0.7777 0.77
8 0.8888 0.88
9 1 1
0 0 0
1 0.1111 0.11
2 0.2222 0.22
3 0.3333 0.33
4 0.4444 0.44
5 0.5555 0.55
6 0.6666 0.66
7 0.7777 0.77
8 0.8888 0.88
9 1 1
99 99 -99 99 -99
9999 9999 -9999 9999 -9999
999999999 999999999 -999999999 999999999 -999999999
999999999 999999999 -999999999 999999999 -999999999
999999999 999999999.000000000 -999999999.000000000 999999999.00000000000000000000000000000 -999999999.00000000000000000000000000000
999999999 999999999.000000000 -999999999.000000000 999999999.00000000000000000000000000000 -999999999.00000000000000000000000000000
999999999 999999999 -999999999 999999999 -999999999
999999999 999999999 -999999999 999999999 -999999999
999999999999999999 999999999999999999 -999999999999999999
999999999999999999 999999999999999999 -999999999999999999
999999999999999999 999999999999999999 -999999999999999999
999999999999999999 999999999999999999.00000000000000000000 -999999999999999999.00000000000000000000
99 99 99
9999 9999 9999
999999999 999999999 999999999
999999999 999999999 999999999
42.42 42.42 42.42 42.42
42.42 42.4200000 42.4200000000000000 42.420000000000000000000000000000000000
42.42 42.42 42.42 42.42
123456789 123456789123456789
12345678901234567890123456789012345678
0.123456789 0.123456789123456789

View File

@ -1,36 +1,36 @@
1234567890.0000000000000000000000000000 1234567890.00000000000000000000000000000 1234567890.00000000000000000000000000000
1234567890 1234567890 1234567890
-126561577.683753853853498429727072845824
1234567890.00000000 1234567890.000000000 1234567890.000000000
12345678.0 12345678.00 12345678.00
9223372036854775807.000000 9223372036854775807 -9223372036854775807
1234567890 1234567890 1234567890
12345678 12345678 12345678
9223372036854775807 9223372036854775807 -9223372036854775807
9223372036854775800 9223372036854775800 -9223372036854775800
92233720368547758.00 92233720368547758 -92233720368547758
2147483647.0000000000 2147483647 -2147483647
2147483647.00 2147483647 -2147483647
92233720368547758 92233720368547758 -92233720368547758
2147483647 2147483647 -2147483647
2147483647 2147483647 -2147483647
92233720368547757.99 92233720368547757 -92233720368547757
2147483640.99 2147483640 -2147483640
-0.90000000 0
-0.90000000 0
-0.90000000 0
-0.8000 0
-0.8000 0
-0.8000 0
-0.70 0
-0.70 0
-0.70 0
-0.600000 0
-0.600000 0
-0.600000 0
-0.9 0
-0.9 0
-0.9 0
-0.8 0
-0.8 0
-0.8 0
-0.7 0
-0.7 0
-0.7 0
-0.6 0
-0.6 0
-0.6 0
18446744073709551615 18446744073709551615
18446744073709551615 18446744073709551615
18446744073709551615.00000000 18446744073709551615
4294967295 4294967295
4294967295.0000000000 4294967295
4294967295 4294967295
4294967295.0000 4294967295
4294967295 4294967295
4294967295 4294967295
65535 65535
65535 65535
65535 65535
65535.0000000000 65535
65535 65535
65535.0000 65535
2147483647 2147483647
-2147483647 -2147483647
2147483647 2147483647

View File

@ -2,27 +2,27 @@
1
-42 -42 1 0 0 0 1 1
42 42 1 0 0 0 1 1
-42 -42.42000 0 0 1 1 0 1
42 42.42000 0 1 0 1 1 0
-42 -42.42 0 0 1 1 0 1
42 42.42 0 1 0 1 1 0
1 1 1
0 0 0
-42 0 0 0 0
42 1 1 1 1
-42 0 0 0 0
42 1 1 1 1
0.420000000 0.420000000000000000 0.42000000000000000000000000000000000000
42.42 42.420000000 42.420000000000000000 42.42
-42.42 -42.420000000 -42.420000000000000000 -42.42
0.42 0.42 0.42
42.42 42.42 42.42 42.42
-42.42 -42.42 -42.42 -42.42
42 42 42
42 42 42
42 42 42
42 42 42
-42 -42.42000 -42 -42.00000
42 42.00000 42 42.42000
-42 -42 -42.42000
0 0 0.00000
0 0 0.00000
42 42 42.42000
-42 -42.42 -42 -42
42 42 42 42.42
-42 -42 -42.42
0 0 0
0 0 0
42 42 42.42
1 0
1 0
1 0
@ -35,5 +35,5 @@
0 1
0 1
0 1
-42 -42 -42 -0.420000000 -0.420000000000000000 -0.42000000000000000000000000000000000000 -42.42000 -42.420000000 -42.420000000000000000 -42.42
42 42 42 0.420000000 0.420000000000000000 0.42000000000000000000000000000000000000 42.42000 42.420000000 42.420000000000000000 42.42
-42 -42 -42 -0.42 -0.42 -0.42 -42.42 -42.42 -42.42 -42.42
42 42 42 0.42 0.42 0.42 42.42 42.42 42.42 42.42

View File

@ -3,31 +3,31 @@ Array(Decimal(9, 2)) Array(Decimal(18, 2)) Array(Decimal(38, 2))
Decimal(9, 3) Decimal(18, 3) Decimal(38, 3)
Decimal(9, 2) Decimal(18, 2) Decimal(38, 2)
Tuple(Decimal(9, 1), Decimal(18, 1), Decimal(38, 1)) Decimal(9, 1) Decimal(18, 1) Decimal(38, 1)
0.100
0.200
0.300
0.400
0.500
0.600
0.700
0.800
0.900
0.1
0.2
0.3
0.4
0.5
0.6
0.7
0.8
0.9
(9.1,9.2,9.3) 9.1 9.2 9.3
[0.100,0.200,0.300] [0.100,0.200] [0.200,0.300] [0.100] [0.200]
[0.400,0.500,0.600] [0.400,0.500] [0.500,0.600] [0.400] [0.500]
[0.700,0.800,0.900] [0.700,0.800] [0.800,0.900] [0.700] [0.800]
[1.10,1.20] [1.10] [1.20] [1.10] [1.20]
[2.10,2.20] [2.10] [2.20] [2.10] [2.20]
[3.10,3.20] [3.10] [3.20] [3.10] [3.20]
[0.100,0.200,0.300,0.000] [0.000,0.100,0.200,0.300]
[0.400,0.500,0.600,0.000] [0.000,0.400,0.500,0.600]
[0.700,0.800,0.900,0.000] [0.000,0.700,0.800,0.900]
[0.100,0.200,0.300,0.000] Array(Decimal(9, 3))
[0.400,0.500,0.600,0.000] Array(Decimal(18, 3))
[0.700,0.800,0.900,0.000] Array(Decimal(38, 3))
[0.0000,0.1000,0.2000,0.3000] Array(Decimal(9, 4))
[0.0000,0.4000,0.5000,0.6000] Array(Decimal(18, 4))
[0.0000,0.7000,0.8000,0.9000] Array(Decimal(38, 4))
[0.1,0.2,0.3] [0.1,0.2] [0.2,0.3] [0.1] [0.2]
[0.4,0.5,0.6] [0.4,0.5] [0.5,0.6] [0.4] [0.5]
[0.7,0.8,0.9] [0.7,0.8] [0.8,0.9] [0.7] [0.8]
[1.1,1.2] [1.1] [1.2] [1.1] [1.2]
[2.1,2.2] [2.1] [2.2] [2.1] [2.2]
[3.1,3.2] [3.1] [3.2] [3.1] [3.2]
[0.1,0.2,0.3,0] [0,0.1,0.2,0.3]
[0.4,0.5,0.6,0] [0,0.4,0.5,0.6]
[0.7,0.8,0.9,0] [0,0.7,0.8,0.9]
[0.1,0.2,0.3,0] Array(Decimal(9, 3))
[0.4,0.5,0.6,0] Array(Decimal(18, 3))
[0.7,0.8,0.9,0] Array(Decimal(38, 3))
[0,0.1,0.2,0.3] Array(Decimal(9, 4))
[0,0.4,0.5,0.6] Array(Decimal(18, 4))
[0,0.7,0.8,0.9] Array(Decimal(38, 4))
3 3 3
2 2 2
0 0 0
@ -66,24 +66,24 @@ Tuple(Decimal(9, 1), Decimal(18, 1), Decimal(38, 1)) Decimal(9, 1) Decimal(18, 1
1
1
1
[0.100,0.200,0.300,0.400,0.500,0.600] Array(Decimal(18, 3))
[0.100,0.200,0.300,0.700,0.800,0.900] Array(Decimal(38, 3))
[0.400,0.500,0.600,0.700,0.800,0.900] Array(Decimal(38, 3))
[0.100,0.200,0.300,1.100,1.200] Array(Decimal(9, 3))
[0.400,0.500,0.600,2.100,2.200] Array(Decimal(18, 3))
[0.700,0.800,0.900,3.100,3.200] Array(Decimal(38, 3))
[0.100,0.200,0.300,2.100,2.200] Array(Decimal(18, 3))
[0.100,0.200,0.300,3.100,3.200] Array(Decimal(38, 3))
[0.400,0.500,0.600,1.100,1.200] Array(Decimal(18, 3))
[0.400,0.500,0.600,3.100,3.200] Array(Decimal(38, 3))
[0.700,0.800,0.900,1.100,1.200] Array(Decimal(38, 3))
[0.700,0.800,0.900,2.100,2.200] Array(Decimal(38, 3))
[0.1,0.2,0.3,0.4,0.5,0.6] Array(Decimal(18, 3))
[0.1,0.2,0.3,0.7,0.8,0.9] Array(Decimal(38, 3))
[0.4,0.5,0.6,0.7,0.8,0.9] Array(Decimal(38, 3))
[0.1,0.2,0.3,1.1,1.2] Array(Decimal(9, 3))
[0.4,0.5,0.6,2.1,2.2] Array(Decimal(18, 3))
[0.7,0.8,0.9,3.1,3.2] Array(Decimal(38, 3))
[0.1,0.2,0.3,2.1,2.2] Array(Decimal(18, 3))
[0.1,0.2,0.3,3.1,3.2] Array(Decimal(38, 3))
[0.4,0.5,0.6,1.1,1.2] Array(Decimal(18, 3))
[0.4,0.5,0.6,3.1,3.2] Array(Decimal(38, 3))
[0.7,0.8,0.9,1.1,1.2] Array(Decimal(38, 3))
[0.7,0.8,0.9,2.1,2.2] Array(Decimal(38, 3))
12345.6789 2 2 2
-12345.6789 2 2 2
123456789.123456784 2 2 2
-123456789.123456784 2 2 2
0.123456789123456784 2 2 2
-0.123456789112345680 2 2 2
-0.12345678911234568 2 2 2
Decimal(9, 5)
Decimal(9, 5)
Decimal(9, 4)
@ -114,21 +114,21 @@ Decimal(38, 4)
Decimal(38, 4)
Decimal(9, 0)
Decimal(18, 0)
32.20000
32.10000
64.20000
32.10000
128.20000
32.10000
32.20000
64.10000
64.20000
64.10000
128.20000
64.10000
32.20000
128.10000
64.20000
128.10000
128.20000
128.10000
32.2
32.1
64.2
32.1
128.2
32.1
32.2
64.1
64.2
64.1
128.2
64.1
32.2
128.1
64.2
128.1
128.2
128.1

View File

@ -4,11 +4,11 @@ c Decimal(38, 4) DEFAULT b / 3
d Decimal(9, 4) MATERIALIZED a + toDecimal32(\'0.2\', 1)
e Decimal(18, 4) ALIAS b * 2
f Decimal(38, 4) ALIAS c * 6
0.0000 0.0000 0.0000
1.0000 0.5000 0.1666
2.0000 1.0000 0.3333
3.0000 1.5000 0.5000
0.0000 0.0000 0.0000 0.2000 0.0000 0.0000
1.0000 0.5000 0.1666 1.2000 1.0000 0.9996
2.0000 1.0000 0.3333 2.2000 2.0000 1.9998
3.0000 1.5000 0.5000 3.2000 3.0000 3.0000
0 0 0
1 0.5 0.1666
2 1 0.3333
3 1.5 0.5
0 0 0 0.2 0 0
1 0.5 0.1666 1.2 1 0.9996
2 1 0.3333 2.2 2 1.9998
3 1.5 0.5 3.2 3 3

View File

@ -1,50 +1,50 @@
0 0 0
[0.0000,0.0000] [0.0000000,0.0000000] [0.00000000,0.00000000]
0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000
[0,0] [0,0] [0,0]
0 0 0 0 0 0
0 0 0 0 0 0
0 0 0 0 0 0
0 0 0 0 0 0
0 0 0 0 0 0
(0,0,0) (0,0,0) (0,0,0) (0,0,0) (0,0,0)
0 0 0
0 0 0
0.0000 0.0000 0.0000000 0.0000000 0.00000000 0.00000000
0.0000 0.0000 0.0000000 0.0000000 0.00000000 0.00000000
0.0000 0.0000 0.0000000 0.0000000 0.00000000 0.00000000
0.0000 0.0000 0.0000000 0.0000000 0.00000000 0.00000000
0.0000 0.0000000 0.00000000 Decimal(6, 4) Decimal(16, 7) Decimal(20, 8)
0.0000 0.0000000 0.00000000 Decimal(6, 4) Decimal(16, 7) Decimal(20, 8)
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
[0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000]
[0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000]
[0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000]
0.0000 0.0000000 0.00000000 Decimal(20, 8)
0.0000 0.0000000 0.00000000 Decimal(20, 8)
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
[0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000]
[0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000]
[0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000]
0.0000 0.0000000 0.00000000 Decimal(20, 8)
0.0000 0.0000000 0.00000000 Decimal(20, 8)
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000
[0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000,0.0000]
[0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000,0.0000000]
[0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000,0.00000000]
0 0 0 0 0 0
0 0 0 0 0 0
0 0 0 0 0 0
0 0 0 0 0 0
0 0 0 Decimal(6, 4) Decimal(16, 7) Decimal(20, 8)
0 0 0 Decimal(6, 4) Decimal(16, 7) Decimal(20, 8)
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
[0,0,0,0,0,0,0,0,0,0,0]
[0,0,0,0,0,0,0,0,0,0,0]
[0,0,0,0,0,0,0,0,0,0,0]
0 0 0 Decimal(20, 8)
0 0 0 Decimal(20, 8)
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
[0,0,0,0,0,0,0,0,0,0,0]
[0,0,0,0,0,0,0,0,0,0,0]
[0,0,0,0,0,0,0,0,0,0,0]
0 0 0 Decimal(20, 8)
0 0 0 Decimal(20, 8)
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
0 0 0
[0,0,0,0,0,0,0,0,0,0,0]
[0,0,0,0,0,0,0,0,0,0,0]
[0,0,0,0,0,0,0,0,0,0,0]
inf inf inf Float64 Float64 Float64
nan nan nan
nan nan nan Float64 Float64 Float64

View File

@ -1,42 +1,42 @@
{"a":0.055,"b":-0.000000005,"c":0.000000000000000005}
{"a":0.100,"b":-0.100000000,"c":0.100000000000000000}
{"a":0.200,"b":-0.200000000,"c":0.200000000000000000}
{"a":0.300,"b":-0.300000000,"c":0.300000000000000000}
{"a":0.420,"b":-0.420000000,"c":0.420000000000000000}
{"a":1.000,"b":-1.000000000,"c":1.000000000000000000}
{"a":1.100,"b":-1.100000000,"c":1.100000000000000000}
{"a":2.000,"b":-2.000000000,"c":2.000000000000000000}
{"a":2.200,"b":-2.200000000,"c":2.200000000000000000}
{"a":3.000,"b":-3.000000000,"c":3.000000000000000000}
{"a":3.300,"b":-3.300000000,"c":3.300000000000000000}
{"a":42.000,"b":-42.000000000,"c":42.000000000000000000}
{"a":42.420,"b":-42.420000000,"c":42.420000000000000000}
{"a":440000.000,"b":-400000000.000000000,"c":40000000000000000000.000000000000000000}
{"a":0.1,"b":-0.1,"c":0.1}
{"a":0.2,"b":-0.2,"c":0.2}
{"a":0.3,"b":-0.3,"c":0.3}
{"a":0.42,"b":-0.42,"c":0.42}
{"a":1,"b":-1,"c":1}
{"a":1.1,"b":-1.1,"c":1.1}
{"a":2,"b":-2,"c":2}
{"a":2.2,"b":-2.2,"c":2.2}
{"a":3,"b":-3,"c":3}
{"a":3.3,"b":-3.3,"c":3.3}
{"a":42,"b":-42,"c":42}
{"a":42.42,"b":-42.42,"c":42.42}
{"a":440000,"b":-400000000,"c":40000000000000000000}
0.055,-0.000000005,0.000000000000000005
0.100,-0.100000000,0.100000000000000000
0.200,-0.200000000,0.200000000000000000
0.300,-0.300000000,0.300000000000000000
0.420,-0.420000000,0.420000000000000000
1.000,-1.000000000,1.000000000000000000
1.100,-1.100000000,1.100000000000000000
2.000,-2.000000000,2.000000000000000000
2.200,-2.200000000,2.200000000000000000
3.000,-3.000000000,3.000000000000000000
3.300,-3.300000000,3.300000000000000000
42.000,-42.000000000,42.000000000000000000
42.420,-42.420000000,42.420000000000000000
440000.000,-400000000.000000000,40000000000000000000.000000000000000000
0.1,-0.1,0.1
0.2,-0.2,0.2
0.3,-0.3,0.3
0.42,-0.42,0.42
1,-1,1
1.1,-1.1,1.1
2,-2,2
2.2,-2.2,2.2
3,-3,3
3.3,-3.3,3.3
42,-42,42
42.42,-42.42,42.42
440000,-400000000,40000000000000000000
0.055 -0.000000005 0.000000000000000005
0.100 -0.100000000 0.100000000000000000
0.200 -0.200000000 0.200000000000000000
0.300 -0.300000000 0.300000000000000000
0.420 -0.420000000 0.420000000000000000
1.000 -1.000000000 1.000000000000000000
1.100 -1.100000000 1.100000000000000000
2.000 -2.000000000 2.000000000000000000
2.200 -2.200000000 2.200000000000000000
3.000 -3.000000000 3.000000000000000000
3.300 -3.300000000 3.300000000000000000
42.000 -42.000000000 42.000000000000000000
42.420 -42.420000000 42.420000000000000000
440000.000 -400000000.000000000 40000000000000000000.000000000000000000
0.1 -0.1 0.1
0.2 -0.2 0.2
0.3 -0.3 0.3
0.42 -0.42 0.42
1 -1 1
1.1 -1.1 1.1
2 -2 2
2.2 -2.2 2.2
3 -3 3
3.3 -3.3 3.3
42 -42 42
42.42 -42.42 42.42
440000 -400000000 40000000000000000000

View File

@ -1,13 +1,13 @@
[2.000]
[2.0000000000]
[2.000000000000000000]
[1.000]
[1.0000000000]
[1.000000000000000000]
-
[2.000]
[1]
[2.000000000000000000]
[1.000]
[2]
[1.000000000000000000]
[2]
[2]
[1]
[1]
[1]
-
[2]
[1]
[2]
[1]
[2]
[1]

View File

@ -5,25 +5,25 @@
1
1
5
9.00000000 29.00000000 29.00000000
8.00000000 28.00000000 28.00000000
7.00000000 27.00000000 27.00000000
6.00000000 26.00000000 26.00000000
9.00000000 19.00000000 19.00000000
8.00000000 18.00000000 18.00000000
7.00000000 17.00000000 17.00000000
6.00000000 16.00000000 16.00000000
9.00000000 9.00000000 9.00000000
8.00000000 8.00000000 8.00000000
7.00000000 7.00000000 7.00000000
6.00000000 6.00000000 6.00000000
1.00000000 1.00000000 1.00000000
3.00000000 3.00000000 3.00000000
1.00000000 11.00000000 11.00000000
3.00000000 13.00000000 13.00000000
1.00000000 21.00000000 21.00000000
3.00000000 23.00000000 23.00000000
1.00000000 31.00000000 31.00000000
3.00000000 33.00000000 33.00000000
1.00000000 41.00000000 41.00000000
3.00000000 43.00000000 43.00000000
9 29 29
8 28 28
7 27 27
6 26 26
9 19 19
8 18 18
7 17 17
6 16 16
9 9 9
8 8 8
7 7 7
6 6 6
1 1 1
3 3 3
1 11 11
3 13 13
1 21 21
3 23 23
1 31 31
3 33 33
1 41 41
3 43 43

View File

@ -1,30 +1,30 @@
42.4200 3.7476 42.419154
42.4200 5.4066 42.417862
42.4200 1.6275 42.413098
42.4200 6.513 42.419169
42.4200 3.4875 42.417263671875
1.00000 0.8427007929497149 0.15729920705028513
42.4200 115.60113124678627 1.6029995567009473e50
0.00 0 1 0
42.42 3.7476 42.419154
42.42 5.4066 42.417862
42.42 1.6275 42.413098
42.42 6.513 42.419169
42.42 3.4875 42.417263671875
1 0.8427007929497149 0.15729920705028513
42.42 115.60113124678627 1.6029995567009473e50
0 0 1 0
3.14159265 0 -1 -0
1.00 1.5707963267948966 0 0.7853981633974483
42.4200 3.7476 42.419154
42.4200 5.4066 42.417862
42.4200 1.6275 42.413098
42.4200 6.513 42.419169
42.4200 3.4875 42.417263671875
1.00000 0.8427007929497149 0.15729920705028513
42.4200 115.60113124678627 1.6029995567009473e50
0.00 0 1 0
1 1.5707963267948966 0 0.7853981633974483
42.42 3.7476 42.419154
42.42 5.4066 42.417862
42.42 1.6275 42.413098
42.42 6.513 42.419169
42.42 3.4875 42.417263671875
1 0.8427007929497149 0.15729920705028513
42.42 115.60113124678627 1.6029995567009473e50
0 0 1 0
3.14159265358979328 0 -1 -0
1.00 1.5707963267948966 0 0.7853981633974483
42.4200 3.7476 42.419154
42.4200 5.4066 42.417862
42.4200 1.6275 42.413098
42.4200 6.513 42.419169
42.4200 3.4875 42.417263671875
1.00000 0.8427007929497149 0.15729920705028513
42.4200 115.60113124678627 1.6029995567009473e50
0.00 0 1 0
1 1.5707963267948966 0 0.7853981633974483
42.42 3.7476 42.419154
42.42 5.4066 42.417862
42.42 1.6275 42.413098
42.42 6.513 42.419169
42.42 3.4875 42.417263671875
1 0.8427007929497149 0.15729920705028513
42.42 115.60113124678627 1.6029995567009473e50
0 0 1 0
3.14159265358979 0 -1 -0
1.00 1.5707963267948966 0 0.7853981633974483
1 1.5707963267948966 0 0.7853981633974483

View File

@ -17,11 +17,11 @@
\N
1
1
1.10 1.10000 1.10000 1.1000 1.10000000 1.10000000
2.20 2.20000 2.20000 2.2000 \N \N
3.30 3.30000 3.30000 \N 3.30000000 \N
4.40 4.40000 4.40000 \N \N 4.40000000
5.50 5.50000 5.50000 \N \N \N
1.1 1.1 1.1 1.1 1.1 1.1
2.2 2.2 2.2 2.2 \N \N
3.3 3.3 3.3 \N 3.3 \N
4.4 4.4 4.4 \N \N 4.4
5.5 5.5 5.5 \N \N \N
0 1
0 1
0 1

View File

@ -1,75 +1,75 @@
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12345.0000 12345.6000 12345.6700 12345.6780 12345.6789 12345.6789
12345.6789 12345.0000 12345.6000 12345.6700 12345.6780 12345.6789 12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12345.0000 -12345.6000 -12345.6700 -12345.6780 -12345.6789 -12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12345.0000 -12345.6000 -12345.6700 -12345.6780 -12345.6789 -12345.6789
12345.6789 12350.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12350.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12350.0000 12400.0000 13000.0000 20000.0000 100000.0000
12345.6789 12340.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12340.0000 12300.0000 12000.0000 10000.0000 0.0000
-12345.6789 -12350.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12350.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12340.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12350.0000 -12400.0000 -13000.0000 -20000.0000 -100000.0000
-12345.6789 -12340.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12345.0000 12345.6000 12345.6700 12345.6780 12345.6789 12345.6789
12345.6789 12345.0000 12345.6000 12345.6700 12345.6780 12345.6789 12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12345.0000 -12345.6000 -12345.6700 -12345.6780 -12345.6789 -12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12345.0000 -12345.6000 -12345.6700 -12345.6780 -12345.6789 -12345.6789
12345.6789 12350.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12350.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12350.0000 12400.0000 13000.0000 20000.0000 100000.0000
12345.6789 12340.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12340.0000 12300.0000 12000.0000 10000.0000 0.0000
-12345.6789 -12350.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12350.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12340.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12350.0000 -12400.0000 -13000.0000 -20000.0000 -100000.0000
-12345.6789 -12340.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12346.0000 12345.7000 12345.6800 12345.6790 12345.6789 12345.6789
12345.6789 12345.0000 12345.6000 12345.6700 12345.6780 12345.6789 12345.6789
12345.6789 12345.0000 12345.6000 12345.6700 12345.6780 12345.6789 12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12345.0000 -12345.6000 -12345.6700 -12345.6780 -12345.6789 -12345.6789
-12345.6789 -12346.0000 -12345.7000 -12345.6800 -12345.6790 -12345.6789 -12345.6789
-12345.6789 -12345.0000 -12345.6000 -12345.6700 -12345.6780 -12345.6789 -12345.6789
12345.6789 12350.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12350.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12350.0000 12400.0000 13000.0000 20000.0000 100000.0000
12345.6789 12340.0000 12300.0000 12000.0000 10000.0000 0.0000
12345.6789 12340.0000 12300.0000 12000.0000 10000.0000 0.0000
-12345.6789 -12350.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12350.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12340.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
-12345.6789 -12350.0000 -12400.0000 -13000.0000 -20000.0000 -100000.0000
-12345.6789 -12340.0000 -12300.0000 -12000.0000 -10000.0000 0.0000
123456789.123456789 -123456789.123456789 123456789.000000000 -123456789.000000000 123456789.123460000 -123456789.123460000 123500000.000000000 -123500000.000000000
123456789.123456789 -123456789.123456789 123456789.000000000 -123456789.000000000 123456789.123460000 -123456789.123460000 123500000.000000000 -123500000.000000000
123456789.123456789 -123456789.123456789 123456790.000000000 -123456789.000000000 123456789.123460000 -123456789.123450000 123500000.000000000 -123400000.000000000
123456789.123456789 -123456789.123456789 123456789.000000000 -123456790.000000000 123456789.123450000 -123456789.123460000 123400000.000000000 -123500000.000000000
123456789.123456789 -123456789.123456789 123456789.000000000 -123456789.000000000 123456789.123450000 -123456789.123450000 123400000.000000000 -123400000.000000000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789.000000000 -12345678901234567890123456789.000000000 12345678901234567890123456789.123000000 -12345678901234567890123456789.123000000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789.000000000 -12345678901234567890123456789.000000000 12345678901234567890123456789.123000000 -12345678901234567890123456789.123000000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456790.000000000 -12345678901234567890123456789.000000000 12345678901234567890123456789.124000000 -12345678901234567890123456789.123000000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789.000000000 -12345678901234567890123456790.000000000 12345678901234567890123456789.123000000 -12345678901234567890123456789.124000000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789.000000000 -12345678901234567890123456789.000000000 12345678901234567890123456789.123000000 -12345678901234567890123456789.123000000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789.000000000 -12345678901234567890123456789.000000000 12345678901234567890123457000.000000000 -12345678901234567890123457000.000000000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789.000000000 -12345678901234567890123456789.000000000 12345678901234567890123457000.000000000 -12345678901234567890123457000.000000000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456790.000000000 -12345678901234567890123456789.000000000 12345678901234567890123457000.000000000 -12345678901234567890123456000.000000000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789.000000000 -12345678901234567890123456790.000000000 12345678901234567890123456000.000000000 -12345678901234567890123457000.000000000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789.000000000 -12345678901234567890123456789.000000000 12345678901234567890123456000.000000000 -12345678901234567890123456000.000000000
12345.6789 12346 12345.7 12345.68 12345.679 12345.6789 12345.6789
12345.6789 12346 12345.7 12345.68 12345.679 12345.6789 12345.6789
12345.6789 12346 12345.7 12345.68 12345.679 12345.6789 12345.6789
12345.6789 12345 12345.6 12345.67 12345.678 12345.6789 12345.6789
12345.6789 12345 12345.6 12345.67 12345.678 12345.6789 12345.6789
-12345.6789 -12346 -12345.7 -12345.68 -12345.679 -12345.6789 -12345.6789
-12345.6789 -12346 -12345.7 -12345.68 -12345.679 -12345.6789 -12345.6789
-12345.6789 -12345 -12345.6 -12345.67 -12345.678 -12345.6789 -12345.6789
-12345.6789 -12346 -12345.7 -12345.68 -12345.679 -12345.6789 -12345.6789
-12345.6789 -12345 -12345.6 -12345.67 -12345.678 -12345.6789 -12345.6789
12345.6789 12350 12300 12000 10000 0
12345.6789 12350 12300 12000 10000 0
12345.6789 12350 12400 13000 20000 100000
12345.6789 12340 12300 12000 10000 0
12345.6789 12340 12300 12000 10000 0
-12345.6789 -12350 -12300 -12000 -10000 0
-12345.6789 -12350 -12300 -12000 -10000 0
-12345.6789 -12340 -12300 -12000 -10000 0
-12345.6789 -12350 -12400 -13000 -20000 -100000
-12345.6789 -12340 -12300 -12000 -10000 0
12345.6789 12346 12345.7 12345.68 12345.679 12345.6789 12345.6789
12345.6789 12346 12345.7 12345.68 12345.679 12345.6789 12345.6789
12345.6789 12346 12345.7 12345.68 12345.679 12345.6789 12345.6789
12345.6789 12345 12345.6 12345.67 12345.678 12345.6789 12345.6789
12345.6789 12345 12345.6 12345.67 12345.678 12345.6789 12345.6789
-12345.6789 -12346 -12345.7 -12345.68 -12345.679 -12345.6789 -12345.6789
-12345.6789 -12346 -12345.7 -12345.68 -12345.679 -12345.6789 -12345.6789
-12345.6789 -12345 -12345.6 -12345.67 -12345.678 -12345.6789 -12345.6789
-12345.6789 -12346 -12345.7 -12345.68 -12345.679 -12345.6789 -12345.6789
-12345.6789 -12345 -12345.6 -12345.67 -12345.678 -12345.6789 -12345.6789
12345.6789 12350 12300 12000 10000 0
12345.6789 12350 12300 12000 10000 0
12345.6789 12350 12400 13000 20000 100000
12345.6789 12340 12300 12000 10000 0
12345.6789 12340 12300 12000 10000 0
-12345.6789 -12350 -12300 -12000 -10000 0
-12345.6789 -12350 -12300 -12000 -10000 0
-12345.6789 -12340 -12300 -12000 -10000 0
-12345.6789 -12350 -12400 -13000 -20000 -100000
-12345.6789 -12340 -12300 -12000 -10000 0
12345.6789 12346 12345.7 12345.68 12345.679 12345.6789 12345.6789
12345.6789 12346 12345.7 12345.68 12345.679 12345.6789 12345.6789
12345.6789 12346 12345.7 12345.68 12345.679 12345.6789 12345.6789
12345.6789 12345 12345.6 12345.67 12345.678 12345.6789 12345.6789
12345.6789 12345 12345.6 12345.67 12345.678 12345.6789 12345.6789
-12345.6789 -12346 -12345.7 -12345.68 -12345.679 -12345.6789 -12345.6789
-12345.6789 -12346 -12345.7 -12345.68 -12345.679 -12345.6789 -12345.6789
-12345.6789 -12345 -12345.6 -12345.67 -12345.678 -12345.6789 -12345.6789
-12345.6789 -12346 -12345.7 -12345.68 -12345.679 -12345.6789 -12345.6789
-12345.6789 -12345 -12345.6 -12345.67 -12345.678 -12345.6789 -12345.6789
12345.6789 12350 12300 12000 10000 0
12345.6789 12350 12300 12000 10000 0
12345.6789 12350 12400 13000 20000 100000
12345.6789 12340 12300 12000 10000 0
12345.6789 12340 12300 12000 10000 0
-12345.6789 -12350 -12300 -12000 -10000 0
-12345.6789 -12350 -12300 -12000 -10000 0
-12345.6789 -12340 -12300 -12000 -10000 0
-12345.6789 -12350 -12400 -13000 -20000 -100000
-12345.6789 -12340 -12300 -12000 -10000 0
123456789.123456789 -123456789.123456789 123456789 -123456789 123456789.12346 -123456789.12346 123500000 -123500000
123456789.123456789 -123456789.123456789 123456789 -123456789 123456789.12346 -123456789.12346 123500000 -123500000
123456789.123456789 -123456789.123456789 123456790 -123456789 123456789.12346 -123456789.12345 123500000 -123400000
123456789.123456789 -123456789.123456789 123456789 -123456790 123456789.12345 -123456789.12346 123400000 -123500000
123456789.123456789 -123456789.123456789 123456789 -123456789 123456789.12345 -123456789.12345 123400000 -123400000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789 -12345678901234567890123456789 12345678901234567890123456789.123 -12345678901234567890123456789.123
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789 -12345678901234567890123456789 12345678901234567890123456789.123 -12345678901234567890123456789.123
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456790 -12345678901234567890123456789 12345678901234567890123456789.124 -12345678901234567890123456789.123
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789 -12345678901234567890123456790 12345678901234567890123456789.123 -12345678901234567890123456789.124
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789 -12345678901234567890123456789 12345678901234567890123456789.123 -12345678901234567890123456789.123
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789 -12345678901234567890123456789 12345678901234567890123457000 -12345678901234567890123457000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789 -12345678901234567890123456789 12345678901234567890123457000 -12345678901234567890123457000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456790 -12345678901234567890123456789 12345678901234567890123457000 -12345678901234567890123456000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789 -12345678901234567890123456790 12345678901234567890123456000 -12345678901234567890123457000
12345678901234567890123456789.123456789 -12345678901234567890123456789.123456789 12345678901234567890123456789 -12345678901234567890123456789 12345678901234567890123456000 -12345678901234567890123456000

View File

@ -1,8 +1,8 @@
1.1 1.10 1.10000000
1.1 1.1 1.1
0
0 0.42
0 0.420
0 0.4200
0 0.42
0 0.42
999999999 0
-999999999 0
999999999999999999 0
@ -12,11 +12,11 @@
-99999999999999999999999999999999999999
0
----
1.1 1.10 1.10000000
1.1 1.1 1.1
\N
\N -0.42
\N -0.420
\N -0.4200
\N -0.42
\N -0.42
999999999 \N
-999999999 \N
999999999999999999 \N

View File

@ -1,2 +1,2 @@
2001-01-01 2.0000 0.00000000 -2.0000000000
2001-01-01 0.0000 1.00000000 0.0000000000
2001-01-01 2 0 -2
2001-01-01 0 1 0

View File

@ -1,11 +1,11 @@
1.10
2.1000
3.100000000000
1.20
2.2000
3.200000000000
1.30
2.3000
3.300000000000
1 1.000000000000000000 10.000000000000000000
1 1.000000000000000000 10.000000000000000000
1.1
2.1
3.1
1.2
2.2
3.2
1.3
2.3
3.3
1 1 10
1 1 10

View File

@ -11,7 +11,7 @@
9175437371954010821
CREATE TABLE default.compression_codec_multiple_more_types\n(\n `id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)),\n `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), NONE, NONE, NONE, LZ4HC(0)),\n `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)),\n `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192
1.5555555555555 hello world! [77] ['John']
7.1000000000000 xxxxxxxxxxxx [127] ['Henry']
7.1 xxxxxxxxxxxx [127] ['Henry']
!
222
!ZSTD

View File

@ -51,33 +51,33 @@
7 6.25
8 7.5
9 8.75
0.00000 4.00000
0.01000 4.00000
0.02000 4.00000
0.03000 4.00000
0.04000 4.00000
0.05000 4.00000
0.06000 4.00000
0.07000 4.00000
0.08000 4.00000
0.09000 4.00000
0.00000 0.04000
0.01000 0.04000
0.02000 0.04000
0.03000 0.04000
0.04000 0.04000
0.05000 0.05000
0.06000 0.06000
0.07000 0.06000
0.08000 0.06000
0.09000 0.06000
0.00000 0.04000
0.01000 0.04000
0.02000 0.04000
0.03000 0.04000
0.04000 0.04000
0.05000 0.05000
0.06000 0.06000
0.07000 0.06000
0.08000 0.06000
0.09000 0.06000
0 4
0.01 4
0.02 4
0.03 4
0.04 4
0.05 4
0.06 4
0.07 4
0.08 4
0.09 4
0 0.04
0.01 0.04
0.02 0.04
0.03 0.04
0.04 0.04
0.05 0.05
0.06 0.06
0.07 0.06
0.08 0.06
0.09 0.06
0 0.04
0.01 0.04
0.02 0.04
0.03 0.04
0.04 0.04
0.05 0.05
0.06 0.06
0.07 0.06
0.08 0.06
0.09 0.06

View File

@ -1,6 +1,6 @@
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753215,37.622504] 3.14 214.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753215,37.622504] 3.14 214.1 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 jpg \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970682,33.074981] 3.14159265358979 100000000000.00 800 -3.2 154400000 ['pound'] [16] 503 []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970682,33.074981] 3.14159265358979 100000000000 800 -3.2 154400000 ['pound'] [16] 503 []
Schema 00825_protobuf_format_persons:Person
@ -150,9 +150,9 @@ nestiness {
Binary representation is as expected
Roundtrip:
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753216,37.622504] 3.14 214.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753216,37.622504] 3.14 214.1 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 jpg \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970680,33.074982] 3.14159265358979 100000000000.00 800 -3.2 154400000 ['pound'] [16] 503 []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.97068,33.074982] 3.14159265358979 100000000000 800 -3.2 154400000 ['pound'] [16] 503 []
Schema 00825_protobuf_format_persons:AltPerson
@ -264,14 +264,14 @@ nestiness_a_b_c_d: 503
Binary representation is as expected
Roundtrip:
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 \N 74951234567\0\0 1 2019-01-05 18:45:00 38 capricorn [] [255,0,0] [55.000000,37.000000] 3.140000104904175 214.00 0.1 5.0 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 \N \N 0 \N 26 pisces [] [100,200,50] [50.000000,-4.000000] 3.141590118408203 \N 0.007 5.0 -20000000000000 [] [] \N []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 \N 442012345678\0 1 2018-12-30 00:00:00 23 leo [] [250,244,10] [68.000000,33.000000] 3.1415927410125732 100000000000.00 800 -3.0 154400000 ['pound'] [16] 503 []
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 \N 74951234567\0\0 1 2019-01-05 18:45:00 38 capricorn [] [255,0,0] [55,37] 3.140000104904175 214 0.1 5 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 \N \N 0 \N 26 pisces [] [100,200,50] [50,-4] 3.141590118408203 \N 0.007 5 -20000000000000 [] [] \N []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 \N 442012345678\0 1 2018-12-30 00:00:00 23 leo [] [250,244,10] [68,33] 3.1415927410125732 100000000000 800 -3 154400000 ['pound'] [16] 503 []
Schema 00825_protobuf_format_persons:StrPerson
Binary representation:
00000000 a7 02 0a 24 61 37 35 32 32 31 35 38 2d 33 64 34 |...$a7522158-3d4|
00000000 a6 02 0a 24 61 37 35 32 32 31 35 38 2d 33 64 34 |...$a7522158-3d4|
00000010 31 2d 34 62 37 37 2d 61 64 36 39 2d 36 63 35 39 |1-4b77-ad69-6c59|
00000020 38 65 65 35 35 63 34 39 12 04 49 76 61 6e 1a 06 |8ee55c49..Ivan..|
00000030 50 65 74 72 6f 76 22 04 6d 61 6c 65 2a 0a 31 39 |Petrov".male*.19|
@ -283,42 +283,42 @@ Binary representation:
00000090 72 73 6a 03 32 35 35 6a 01 30 6a 01 30 72 06 4d |rsj.255j.0j.0r.M|
000000a0 6f 73 63 6f 77 7a 09 35 35 2e 37 35 33 32 31 35 |oscowz.55.753215|
000000b0 7a 09 33 37 2e 36 32 32 35 30 34 82 01 04 33 2e |z.37.622504...3.|
000000c0 31 34 8a 01 06 32 31 34 2e 31 30 92 01 03 30 2e |14...214.10...0.|
000000d0 31 9a 01 03 35 2e 38 a2 01 0b 31 37 30 36 30 30 |1...5.8...170600|
000000e0 30 30 30 30 30 aa 01 2d 0a 05 6d 65 74 65 72 0a |00000..-..meter.|
000000f0 0a 63 65 6e 74 69 6d 65 74 65 72 0a 09 6b 69 6c |.centimeter..kil|
00000100 6f 6d 65 74 65 72 12 01 31 12 04 30 2e 30 31 12 |ometer..1..0.01.|
00000110 04 31 30 30 30 b2 01 11 0a 0f 0a 03 35 30 30 12 |.1000.......500.|
00000120 03 35 30 31 12 03 35 30 32 b4 01 0a 24 63 36 39 |.501..502...$c69|
00000130 34 61 64 38 61 2d 66 37 31 34 2d 34 65 61 33 2d |4ad8a-f714-4ea3-|
00000140 39 30 37 64 2d 66 64 35 34 66 62 32 35 64 39 62 |907d-fd54fb25d9b|
00000150 35 12 07 4e 61 74 61 6c 69 61 1a 08 53 6f 6b 6f |5..Natalia..Soko|
00000160 6c 6f 76 61 22 06 66 65 6d 61 6c 65 2a 0a 31 39 |lova".female*.19|
00000170 39 32 2d 30 33 2d 30 38 42 01 30 52 02 32 36 5a |92-03-08B.0R.26Z|
00000180 06 70 69 73 63 65 73 6a 03 31 30 30 6a 03 32 30 |.piscesj.100j.20|
00000190 30 6a 02 35 30 72 08 50 6c 79 6d 6f 75 74 68 7a |0j.50r.Plymouthz|
000001a0 09 35 30 2e 34 30 33 37 32 34 7a 09 2d 34 2e 31 |.50.403724z.-4.1|
000001b0 34 32 31 32 33 82 01 07 33 2e 31 34 31 35 39 92 |42123...3.14159.|
000001c0 01 05 30 2e 30 30 37 9a 01 03 35 2e 34 a2 01 0f |..0.007...5.4...|
000001d0 2d 32 30 30 30 30 30 30 30 30 30 30 30 30 30 84 |-20000000000000.|
000001e0 02 0a 24 61 37 64 61 31 61 61 36 2d 66 34 32 35 |..$a7da1aa6-f425|
000001f0 2d 34 37 38 39 2d 38 39 34 37 2d 62 30 33 34 37 |-4789-8947-b0347|
00000200 38 36 65 64 33 37 34 12 06 56 61 73 69 6c 79 1a |86ed374..Vasily.|
00000210 07 53 69 64 6f 72 6f 76 22 04 6d 61 6c 65 2a 0a |.Sidorov".male*.|
00000220 31 39 39 35 2d 30 37 2d 32 38 3a 0d 2b 34 34 32 |1995-07-28:.+442|
00000230 30 31 32 33 34 35 36 37 38 42 01 31 4a 13 32 30 |012345678B.1J.20|
00000240 31 38 2d 31 32 2d 33 30 20 30 30 3a 30 30 3a 30 |18-12-30 00:00:0|
00000250 30 52 02 32 33 5a 03 6c 65 6f 62 05 53 75 6e 6e |0R.23Z.leob.Sunn|
00000260 79 6a 03 32 35 30 6a 03 32 34 34 6a 02 31 30 72 |yj.250j.244j.10r|
00000270 08 4d 75 72 6d 61 6e 73 6b 7a 09 36 38 2e 39 37 |.Murmanskz.68.97|
00000280 30 36 38 32 7a 09 33 33 2e 30 37 34 39 38 31 82 |0682z.33.074981.|
00000290 01 10 33 2e 31 34 31 35 39 32 36 35 33 35 38 39 |..3.141592653589|
000002a0 37 39 8a 01 0f 31 30 30 30 30 30 30 30 30 30 30 |79...10000000000|
000002b0 30 2e 30 30 92 01 03 38 30 30 9a 01 04 2d 33 2e |0.00...800...-3.|
000002c0 32 a2 01 09 31 35 34 34 30 30 30 30 30 aa 01 0b |2...154400000...|
000002d0 0a 05 70 6f 75 6e 64 12 02 31 36 b2 01 07 0a 05 |..pound..16.....|
000002e0 0a 03 35 30 33 |..503|
000002e5
000000c0 31 34 8a 01 05 32 31 34 2e 31 92 01 03 30 2e 31 |14...214.1...0.1|
000000d0 9a 01 03 35 2e 38 a2 01 0b 31 37 30 36 30 30 30 |...5.8...1706000|
000000e0 30 30 30 30 aa 01 2d 0a 05 6d 65 74 65 72 0a 0a |0000..-..meter..|
000000f0 63 65 6e 74 69 6d 65 74 65 72 0a 09 6b 69 6c 6f |centimeter..kilo|
00000100 6d 65 74 65 72 12 01 31 12 04 30 2e 30 31 12 04 |meter..1..0.01..|
00000110 31 30 30 30 b2 01 11 0a 0f 0a 03 35 30 30 12 03 |1000.......500..|
00000120 35 30 31 12 03 35 30 32 b4 01 0a 24 63 36 39 34 |501..502...$c694|
00000130 61 64 38 61 2d 66 37 31 34 2d 34 65 61 33 2d 39 |ad8a-f714-4ea3-9|
00000140 30 37 64 2d 66 64 35 34 66 62 32 35 64 39 62 35 |07d-fd54fb25d9b5|
00000150 12 07 4e 61 74 61 6c 69 61 1a 08 53 6f 6b 6f 6c |..Natalia..Sokol|
00000160 6f 76 61 22 06 66 65 6d 61 6c 65 2a 0a 31 39 39 |ova".female*.199|
00000170 32 2d 30 33 2d 30 38 42 01 30 52 02 32 36 5a 06 |2-03-08B.0R.26Z.|
00000180 70 69 73 63 65 73 6a 03 31 30 30 6a 03 32 30 30 |piscesj.100j.200|
00000190 6a 02 35 30 72 08 50 6c 79 6d 6f 75 74 68 7a 09 |j.50r.Plymouthz.|
000001a0 35 30 2e 34 30 33 37 32 34 7a 09 2d 34 2e 31 34 |50.403724z.-4.14|
000001b0 32 31 32 33 82 01 07 33 2e 31 34 31 35 39 92 01 |2123...3.14159..|
000001c0 05 30 2e 30 30 37 9a 01 03 35 2e 34 a2 01 0f 2d |.0.007...5.4...-|
000001d0 32 30 30 30 30 30 30 30 30 30 30 30 30 30 81 02 |20000000000000..|
000001e0 0a 24 61 37 64 61 31 61 61 36 2d 66 34 32 35 2d |.$a7da1aa6-f425-|
000001f0 34 37 38 39 2d 38 39 34 37 2d 62 30 33 34 37 38 |4789-8947-b03478|
00000200 36 65 64 33 37 34 12 06 56 61 73 69 6c 79 1a 07 |6ed374..Vasily..|
00000210 53 69 64 6f 72 6f 76 22 04 6d 61 6c 65 2a 0a 31 |Sidorov".male*.1|
00000220 39 39 35 2d 30 37 2d 32 38 3a 0d 2b 34 34 32 30 |995-07-28:.+4420|
00000230 31 32 33 34 35 36 37 38 42 01 31 4a 13 32 30 31 |12345678B.1J.201|
00000240 38 2d 31 32 2d 33 30 20 30 30 3a 30 30 3a 30 30 |8-12-30 00:00:00|
00000250 52 02 32 33 5a 03 6c 65 6f 62 05 53 75 6e 6e 79 |R.23Z.leob.Sunny|
00000260 6a 03 32 35 30 6a 03 32 34 34 6a 02 31 30 72 08 |j.250j.244j.10r.|
00000270 4d 75 72 6d 61 6e 73 6b 7a 09 36 38 2e 39 37 30 |Murmanskz.68.970|
00000280 36 38 32 7a 09 33 33 2e 30 37 34 39 38 31 82 01 |682z.33.074981..|
00000290 10 33 2e 31 34 31 35 39 32 36 35 33 35 38 39 37 |.3.1415926535897|
000002a0 39 8a 01 0c 31 30 30 30 30 30 30 30 30 30 30 30 |9...100000000000|
000002b0 92 01 03 38 30 30 9a 01 04 2d 33 2e 32 a2 01 09 |...800...-3.2...|
000002c0 31 35 34 34 30 30 30 30 30 aa 01 0b 0a 05 70 6f |154400000.....po|
000002d0 75 6e 64 12 02 31 36 b2 01 07 0a 05 0a 03 35 30 |und..16.......50|
000002e0 33 |3|
000002e1
MESSAGE #1 AT 0x00000002
uuid: "a7522158-3d41-4b77-ad69-6c598ee55c49"
@ -340,7 +340,7 @@ hometown: "Moscow"
location: "55.753215"
location: "37.622504"
pi: "3.14"
lotteryWin: "214.10"
lotteryWin: "214.1"
someRatio: "0.1"
temperature: "5.8"
randomBigNumber: "17060000000"
@ -359,7 +359,7 @@ nestiness_a {
e: "502"
}
}
MESSAGE #2 AT 0x0000012B
MESSAGE #2 AT 0x0000012A
uuid: "c694ad8a-f714-4ea3-907d-fd54fb25d9b5"
name: "Natalia"
surname: "Sokolova"
@ -378,7 +378,7 @@ pi: "3.14159"
someRatio: "0.007"
temperature: "5.4"
randomBigNumber: "-20000000000000"
MESSAGE #3 AT 0x000001E1
MESSAGE #3 AT 0x000001E0
uuid: "a7da1aa6-f425-4789-8947-b034786ed374"
name: "Vasily"
surname: "Sidorov"
@ -397,7 +397,7 @@ hometown: "Murmansk"
location: "68.970682"
location: "33.074981"
pi: "3.14159265358979"
lotteryWin: "100000000000.00"
lotteryWin: "100000000000"
someRatio: "800"
temperature: "-3.2"
randomBigNumber: "154400000"
@ -414,9 +414,9 @@ nestiness_a {
Binary representation is as expected
Roundtrip:
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 \N +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753215,37.622504] 3.14 214.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 \N +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753215,37.622504] 3.14 214.1 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 \N \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 \N +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970682,33.074981] 3.14159265358979 100000000000.00 800 -3.2 154400000 ['pound'] [16] 503 []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 \N +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970682,33.074981] 3.14159265358979 100000000000 800 -3.2 154400000 ['pound'] [16] 503 []
Schema 00825_protobuf_format_syntax2:Syntax2Person
@ -564,6 +564,6 @@ Nestiness {
Binary representation is as expected
Roundtrip:
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753216,37.622504] 3.14 214.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753216,37.622504] 3.14 214.1 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 jpg \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970680,33.074982] 3.14159265358979 100000000000.00 800 -3.2 154400000 ['pound'] [16] 503 []
a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.97068,33.074982] 3.14159265358979 100000000000 800 -3.2 154400000 ['pound'] [16] 503 []

View File

@ -1,6 +1,6 @@
0 5 4.7 6.50 cba b 2014-01-04
1 5 4.7 6.50 cba b 2014-03-11
11 5 4.7 6.50 cba b 2014-06-11
12 5 4.7 6.50 cba b 2015-01-01
0 5 4.7 6.5 cba b 2014-01-04
1 5 4.7 6.5 cba b 2014-03-11
11 5 4.7 6.5 cba b 2014-06-11
12 5 4.7 6.5 cba b 2015-01-01
"rows_read": 4,
"rows_read": 2,

View File

@ -1,8 +1,8 @@
0 5 4.7 6.50 cba b 2014-01-04
1 5 4.7 6.50 cba b 2014-03-11
12 5 4.7 6.50 cba b 2014-06-11
13 5 4.7 6.50 cba b 2015-01-01
0 5 4.7 6.50 cba b 2014-01-04
1 5 4.7 6.50 cba b 2014-03-11
12 5 4.7 6.50 cba b 2014-06-11
13 5 4.7 6.50 cba b 2015-01-01
0 5 4.7 6.5 cba b 2014-01-04
1 5 4.7 6.5 cba b 2014-03-11
12 5 4.7 6.5 cba b 2014-06-11
13 5 4.7 6.5 cba b 2015-01-01
0 5 4.7 6.5 cba b 2014-01-04
1 5 4.7 6.5 cba b 2014-03-11
12 5 4.7 6.5 cba b 2014-06-11
13 5 4.7 6.5 cba b 2015-01-01

View File

@ -1,24 +1,24 @@
0 5 4.7 6.50 cba b 2014-01-04
1 5 4.7 6.50 cba b 2014-03-11
12 5 4.7 6.50 cba b 2014-06-11
13 5 4.7 6.50 cba b 2015-01-01
0 5 4.7 6.5 cba b 2014-01-04
1 5 4.7 6.5 cba b 2014-03-11
12 5 4.7 6.5 cba b 2014-06-11
13 5 4.7 6.5 cba b 2015-01-01
"rows_read": 4,
2 2 4.5 2.50 abc a 2014-01-01
6 2 4.5 2.50 abc a 2014-02-11
2 2 4.5 2.5 abc a 2014-01-01
6 2 4.5 2.5 abc a 2014-02-11
7 5 6.9 1.57 bac c 2014-04-11
8 2 4.5 2.50 abc a 2014-05-11
8 2 4.5 2.5 abc a 2014-05-11
9 5 6.9 1.57 bac c 2014-07-11
5 5 6.9 1.57 bac c 2014-11-11
4 2 4.5 2.50 abc a 2016-01-01
4 2 4.5 2.5 abc a 2016-01-01
3 5 6.9 1.57 bac c 2017-01-01
"rows_read": 8,
"rows_read": 2,
2 2 4.5 2.50 abc a 2014-01-01
6 2 4.5 2.50 abc a 2014-02-11
2 2 4.5 2.5 abc a 2014-01-01
6 2 4.5 2.5 abc a 2014-02-11
7 5 6.9 1.57 bac c 2014-04-11
8 2 4.5 2.50 abc a 2014-05-11
8 2 4.5 2.5 abc a 2014-05-11
9 5 6.9 1.57 bac c 2014-07-11
5 5 6.9 1.57 bac c 2014-11-11
4 2 4.5 2.50 abc a 2016-01-01
4 2 4.5 2.5 abc a 2016-01-01
3 5 6.9 1.57 bac c 2017-01-01
"rows_read": 8,

View File

@ -1,5 +1,5 @@
1 1.00 1.00 1.00
2 -1.00 -1.00 -1.00
3 1.00 1.00 1.00
4 -0.10 -0.10 -0.10
1 1 1 1
2 -1 -1 -1
3 1 1 1
4 -0.1 -0.1 -0.1
5 0.01 0.01 0.01

View File

@ -1,18 +1,18 @@
128.00 128.00
128.00 128.00
128.00 128.00
128.00 128.00
128.00 128.00
128.00 128.00
32.00 32.00
32.00 32.00
32.00 32.00
32.00 32.00
32.00 32.00
32.00 32.00
64.00 64.00
64.00 64.00
64.00 64.00
64.00 64.00
64.00 64.00
64.00 64.00
128 128
128 128
128 128
128 128
128 128
128 128
32 32
32 32
32 32
32 32
32 32
32 32
64 64
64 64
64 64
64 64
64 64
64 64

View File

@ -60,17 +60,17 @@ dest from null:
-108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06
127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06
\N \N \N \N \N \N \N \N \N \N \N \N \N \N
1 [1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01]
1 [1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01]
1 [1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.2,10,4] [4,10000.1,10000.1] [1000000000,90,101001.01]
1 [1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.2,10,4] [4,10000.1,10000.1] [1000000000,90,101001.01]
2 [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] []
2 [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] []
1 [1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42]
1 [1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42]
1 [1,NULL,2] [NULL,'Some string',NULL] [0,NULL,42.42]
1 [1,NULL,2] [NULL,'Some string',NULL] [0,NULL,42.42]
2 [NULL] [NULL] [NULL]
2 [NULL] [NULL] [NULL]
3 [] [] []
3 [] [] []
[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]]
[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]]
0.1230 0.12312312 0.1231231231230000 0.12312312312312312300000000000000
0.1230 0.12312312 0.1231231231230000 0.12312312312312312300000000000000
0.123 0.12312312 0.123123123123 0.123123123123123123
0.123 0.12312312 0.123123123123 0.123123123123123123

View File

@ -3,8 +3,8 @@
1 0 1 1 1 10 1.1 10.1 01/01/09 1 1230768060
=== Try load data from alltypes_list.parquet
[] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] []
[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01]
[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01]
[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.2,10,4] [4,10000.1,10000.1] [1000000000,90,101001.01]
[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.2,10,4] [4,10000.1,10000.1] [1000000000,90,101001.01]
=== Try load data from alltypes_plain.parquet
4 1 0 0 0 0 0 0 03/01/09 0 1235865600
5 0 1 1 1 10 1.1 10.1 03/01/09 1 1235865660
@ -64,30 +64,30 @@ idx10 ['This','is','a','test']
\n
=== Try load data from byte_array_decimal.parquet
1.00
2.00
3.00
4.00
5.00
6.00
7.00
8.00
9.00
10.00
11.00
12.00
13.00
14.00
15.00
16.00
17.00
18.00
19.00
20.00
21.00
22.00
23.00
24.00
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
=== Try load data from datapage_v2.snappy.parquet
Code: 33. DB::ParsingEx---tion: Error while reading Parquet data: IOError: Not yet implemented: Unsupported encoding.: While executing ParquetBlockInputFormat: data for INSERT was parsed from stdin. (CANNOT_READ_ALL_DATA)
@ -137,135 +137,135 @@ Code: 33. DB::ParsingEx---tion: Error while reading Parquet data: IOError: Not y
1552
1552
=== Try load data from fixed_length_decimal.parquet
1.00
2.00
3.00
4.00
5.00
6.00
7.00
8.00
9.00
10.00
11.00
12.00
13.00
14.00
15.00
16.00
17.00
18.00
19.00
20.00
21.00
22.00
23.00
24.00
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
=== Try load data from fixed_length_decimal_1.parquet
1.00
2.00
3.00
4.00
5.00
6.00
7.00
8.00
9.00
10.00
11.00
12.00
13.00
14.00
15.00
16.00
17.00
18.00
19.00
20.00
21.00
22.00
23.00
24.00
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
=== Try load data from fixed_length_decimal_legacy.parquet
1.00
2.00
3.00
4.00
5.00
6.00
7.00
8.00
9.00
10.00
11.00
12.00
13.00
14.00
15.00
16.00
17.00
18.00
19.00
20.00
21.00
22.00
23.00
24.00
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
=== Try load data from hadoop_lz4_compressed.parquet
1593604800 abc 42
1593604800 def 7.7
1593604801 abc 42.125
1593604801 def 7.7
=== Try load data from int32_decimal.parquet
1.00
2.00
3.00
4.00
5.00
6.00
7.00
8.00
9.00
10.00
11.00
12.00
13.00
14.00
15.00
16.00
17.00
18.00
19.00
20.00
21.00
22.00
23.00
24.00
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
=== Try load data from int64_decimal.parquet
1.00
2.00
3.00
4.00
5.00
6.00
7.00
8.00
9.00
10.00
11.00
12.00
13.00
14.00
15.00
16.00
17.00
18.00
19.00
20.00
21.00
22.00
23.00
24.00
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
=== Try load data from list_columns.parquet
[1,2,3] ['abc','efg','hij']
[NULL,1] []
@ -325,7 +325,7 @@ Code: 33. DB::ParsingEx---tion: Error while reading Parquet data: IOError: Not y
6 [] [] {} [] (NULL,[],([]),{})
7 [] [[],[5,6]] {'k1':NULL,'k3':NULL} [] (7,[2,3,NULL],([[],[(NULL,NULL)],[]]),{})
=== Try load data from nullable_list.parquet
[1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42]
[1,NULL,2] [NULL,'Some string',NULL] [0,NULL,42.42]
[NULL] [NULL] [NULL]
[] [] []
=== Try load data from nulls.snappy.parquet

View File

@ -1,4 +1,4 @@
[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01]
[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.2,10,4] [4,10000.1,10000.1] [1000000000,90,101001.01]
[] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] []
[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01]
[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.2,10,4] [4,10000.1,10000.1] [1000000000,90,101001.01]
[] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] []

View File

@ -1,6 +1,6 @@
[1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42]
[1,NULL,2] [NULL,'Some string',NULL] [0,NULL,42.42]
[NULL] [NULL] [NULL]
[] [] []
[1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42]
[1,NULL,2] [NULL,'Some string',NULL] [0,NULL,42.42]
[NULL] [NULL] [NULL]
[] [] []

View File

@ -1,9 +1,9 @@
[1.00]
[1.00000]
[1.0000000000]
[499500.00]
[499500.00000]
[499500.0000000000]
[1]
[1]
[1]
[499500]
[499500]
[499500]
1545081300 [('ed87e57c-9331-462a-80b4-9f0c005e88c8',0.44)]
4341757 5657967 2018-11-01 16:47:46 txt 321.380000000000 315.080000000000 0.000000000000 2018-11-02 00:00:00
4360430 5681495 2018-11-02 09:00:07 txt 274.350000000000 268.970000000000 0.000000000000 2018-11-02 00:00:00
4341757 5657967 2018-11-01 16:47:46 txt 321.38 315.08 0 2018-11-02 00:00:00
4360430 5681495 2018-11-02 09:00:07 txt 274.35 268.97 0 2018-11-02 00:00:00

View File

@ -22,6 +22,6 @@
9175437371954010821
CREATE TABLE default.compression_codec_multiple_more_types_replicated\n(\n `id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)),\n `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)),\n `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)),\n `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_00910/compression_codec_multiple_more_types_replicated\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192
1.5555555555555 hello world! [77] ['John']
7.1000000000000 xxxxxxxxxxxx [127] ['Henry']
7.1 xxxxxxxxxxxx [127] ['Henry']
!
222

View File

@ -13,15 +13,15 @@
2 1970-01-01 03:00:01 1 0
2 1970-01-01 03:00:03 3 3
2 1970-01-01 03:00:05 5 3
2 1.00000 1 0
2 3.00000 3 3
2 5.00000 5 3
2 1.00000 1 0
2 3.00000 3 3
2 5.00000 5 3
2 1.00000 1 0
2 3.00000 3 3
2 5.00000 5 3
2 1 1 0
2 3 3 3
2 5 5 3
2 1 1 0
2 3 3 3
2 5 5 3
2 1 1 0
2 3 3 3
2 5 5 3
2 1970-01-01 03:00:00.001 1 0
2 1970-01-01 03:00:00.003 3 3
2 1970-01-01 03:00:00.005 5 3

View File

@ -31,18 +31,18 @@ dictGetOrDefault complex_hashed_strings 0 * *
dictGet complex_cache_strings 1 1 1
dictGetOrDefault complex_cache_strings 1 1 1
dictGetOrDefault complex_cache_strings 0 * *
dictGet flat_decimals 1 1.0000 1.000000 1.0 (1.0000,1.000000,1.0)
dictGetOrDefault flat_decimals 1 1.0000 1.000000 1.0 (1.0000,1.000000,1.0)
dictGetOrDefault flat_decimals 0 42.0000 42.000000 42.0 (42.0000,42.000000,42.0)
dictGet hashed_decimals 1 1.0000 1.000000 1.0 (1.0000,1.000000,1.0)
dictGetOrDefault hashed_decimals 1 1.0000 1.000000 1.0 (1.0000,1.000000,1.0)
dictGetOrDefault hashed_decimals 0 42.0000 42.000000 42.0 (42.0000,42.000000,42.0)
dictGet cache_decimals 1 1.0000 1.000000 1.0 (1.0000,1.000000,1.0)
dictGetOrDefault cache_decimals 1 1.0000 1.000000 1.0 (1.0000,1.000000,1.0)
dictGetOrDefault cache_decimals 0 42.0000 42.000000 42.0 (42.0000,42.000000,42.0)
dictGet complex_hashed_decimals (1) 1.0000 1.000000 1.0 (1.0000,1.000000,1.0)
dictGetOrDefault complex_hashed_decimals (1) 1.0000 1.000000 1.0 (1.0000,1.000000,1.0)
dictGetOrDefault complex_hashed_decimals (0) 42.0000 42.000000 42.0 (42.0000,42.000000,42.0)
dictGet complex_cache_decimals (1) 1.0000 1.000000 1.0 (1.0000,1.000000,1.0)
dictGetOrDefault complex_cache_decimals (1) 1.0000 1.000000 1.0 (1.0000,1.000000,1.0)
dictGetOrDefault complex_cache_decimals (0) 42.0000 42.000000 42.0 (42.0000,42.000000,42.0)
dictGet flat_decimals 1 1 1 1 (1,1,1)
dictGetOrDefault flat_decimals 1 1 1 1 (1,1,1)
dictGetOrDefault flat_decimals 0 42 42 42 (42,42,42)
dictGet hashed_decimals 1 1 1 1 (1,1,1)
dictGetOrDefault hashed_decimals 1 1 1 1 (1,1,1)
dictGetOrDefault hashed_decimals 0 42 42 42 (42,42,42)
dictGet cache_decimals 1 1 1 1 (1,1,1)
dictGetOrDefault cache_decimals 1 1 1 1 (1,1,1)
dictGetOrDefault cache_decimals 0 42 42 42 (42,42,42)
dictGet complex_hashed_decimals (1) 1 1 1 (1,1,1)
dictGetOrDefault complex_hashed_decimals (1) 1 1 1 (1,1,1)
dictGetOrDefault complex_hashed_decimals (0) 42 42 42 (42,42,42)
dictGet complex_cache_decimals (1) 1 1 1 (1,1,1)
dictGetOrDefault complex_cache_decimals (1) 1 1 1 (1,1,1)
dictGetOrDefault complex_cache_decimals (0) 42 42 42 (42,42,42)

View File

@ -10,5 +10,5 @@ cadabra
abracadabra
23 23 23
24 24 24
1.6660 a b
1.666 a b
\N

View File

@ -2,5 +2,5 @@
7777
7777
7777
7777.000
7777.000
7777
7777

View File

@ -1,7 +1,7 @@
1.00
1.00
1.00
1.00
1.00
1.00
1.00
1
1
1
1
1
1
1

View File

@ -45,7 +45,7 @@ nan
\N
0
\N
0.00
0
\N
--- Other Types Non-empty ---
hello

View File

@ -1,6 +1,6 @@
0 5 4.7 6.50 cba b 2014-01-04
1 5 4.7 6.50 cba b 2014-03-11
11 5 4.7 6.50 cba b 2014-06-11
12 5 4.7 6.50 cba b 2015-01-01
0 5 4.7 6.5 cba b 2014-01-04
1 5 4.7 6.5 cba b 2014-03-11
11 5 4.7 6.5 cba b 2014-06-11
12 5 4.7 6.5 cba b 2015-01-01
"rows_read": 4,
"rows_read": 2,

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