Merge pull request #24350 from MyroTk/tfs_extended_precision_data_types_testing

Testflows tests for Extended Precision Data Types: Int128, UInt128, Int256, UInt256, Decimal256
This commit is contained in:
alexey-milovidov 2021-05-26 22:21:18 +03:00 committed by GitHub
commit 91f910a612
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
27 changed files with 4786 additions and 0 deletions

View File

@ -0,0 +1,118 @@
import uuid
from contextlib import contextmanager
from testflows.core.name import basename, parentname
from testflows._core.testtype import TestSubType
from testflows.asserts import values, error, snapshot
from helpers.common import *
@contextmanager
def allow_experimental_bigint(node):
"""Enable experimental big int setting in Clickhouse.
"""
setting = ("allow_experimental_bigint_types", 1)
default_query_settings = None
try:
with Given("I add allow_experimental_bigint to the default query settings"):
default_query_settings = getsattr(current().context, "default_query_settings", [])
default_query_settings.append(setting)
yield
finally:
with Finally("I remove allow_experimental_bigint from the default query settings"):
if default_query_settings:
try:
default_query_settings.pop(default_query_settings.index(setting))
except ValueError:
pass
@TestStep(Given)
def allow_experimental_map_type(self):
"""Set allow_experimental_map_type = 1
"""
setting = ("allow_experimental_map_type", 1)
default_query_settings = None
try:
with By("adding allow_experimental_map_type to the default query settings"):
default_query_settings = getsattr(current().context, "default_query_settings", [])
default_query_settings.append(setting)
yield
finally:
with Finally("I remove allow_experimental_map_type from the default query settings"):
if default_query_settings:
try:
default_query_settings.pop(default_query_settings.index(setting))
except ValueError:
pass
def execute_query(sql, expected=None, format="TabSeparatedWithNames", compare_func=None):
"""Execute SQL query and compare the output to the snapshot.
"""
name = basename(current().name)
with When("I execute query", description=sql):
r = current().context.node.query(sql + " FORMAT " + format)
if expected is not None:
with Then("I check output against expected"):
if compare_func is None:
assert r.output.strip() == expected, error()
else:
assert compare_func(r.output.strip(), expected), error()
else:
with Then("I check output against snapshot"):
with values() as that:
assert that(snapshot("\n" + r.output.strip() + "\n", "tests", name=name, encoder=str)), error()
@TestStep(Given)
def table(self, data_type, name="table0"):
"""Create a table.
"""
node = current().context.node
try:
with By("creating table"):
node.query(f"CREATE TABLE {name}(a {data_type}) ENGINE = Memory")
yield
finally:
with Finally("drop the table"):
node.query(f"DROP TABLE IF EXISTS {name}")
def getuid():
"""Create a unique variable name based on the test it is called from.
"""
if current().subtype == TestSubType.Example:
testname = f"{basename(parentname(current().name)).replace(' ', '_').replace(',','')}"
else:
testname = f"{basename(current().name).replace(' ', '_').replace(',','')}"
for char in ['(', ')', '[', ']','\'']:
testname = testname.replace(f'{char}', '')
return testname + "_" + str(uuid.uuid1()).replace('-', '_')
def to_data_type(data_type, value):
"""Return a conversion statement based on the data type provided
"""
if data_type in ['Decimal256(0)']:
return f'toDecimal256(\'{value}\',0)'
else:
return f'to{data_type}(\'{value}\')'
data_types = [
('Int128', '-170141183460469231731687303715884105728', '170141183460469231731687303715884105727'),
('Int256', '-57896044618658097711785492504343953926634992332820282019728792003956564819968', '57896044618658097711785492504343953926634992332820282019728792003956564819967'),
('UInt128','0','340282366920938463463374607431768211455'),
('UInt256', '0', '115792089237316195423570985008687907853269984665640564039457584007913129639935'),
]
Decimal256_min_max = -1000000000000000000000000000000000000000000000000000000000000000000000000000,1000000000000000000000000000000000000000000000000000000000000000000000000000

View File

@ -0,0 +1,6 @@
<yandex>
<timezone>Europe/Moscow</timezone>
<listen_host replace="replace">::</listen_host>
<path>/var/lib/clickhouse/</path>
<tmp_path>/var/lib/clickhouse/tmp/</tmp_path>
</yandex>

View File

@ -0,0 +1,17 @@
<yandex>
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/log.log</log>
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
<size>1000M</size>
<count>10</count>
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
</logger>
<part_log>
<database>system</database>
<table>part_log</table>
<flush_interval_milliseconds>500</flush_interval_milliseconds>
</part_log>
</yandex>

View File

@ -0,0 +1,20 @@
<yandex>
<storage_configuration>
<disks>
<default>
<keep_free_space_bytes>1024</keep_free_space_bytes>
</default>
</disks>
<policies>
<default>
<volumes>
<default>
<disk>default</disk>
</default>
</volumes>
</default>
</policies>
</storage_configuration>
</yandex>

View File

@ -0,0 +1,448 @@
<?xml version="1.0"?>
<!--
NOTE: User and query level settings are set up in "users.xml" file.
-->
<yandex>
<logger>
<!-- Possible levels: https://github.com/pocoproject/poco/blob/develop/Foundation/include/Poco/Logger.h#L105 -->
<level>trace</level>
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
<size>1000M</size>
<count>10</count>
<!-- <console>1</console> --> <!-- Default behavior is autodetection (log to console if not daemon mode and is tty) -->
</logger>
<!--display_name>production</display_name--> <!-- It is the name that will be shown in the client -->
<http_port>8123</http_port>
<tcp_port>9000</tcp_port>
<!-- For HTTPS and SSL over native protocol. -->
<!--
<https_port>8443</https_port>
<tcp_port_secure>9440</tcp_port_secure>
-->
<!-- Used with https_port and tcp_port_secure. Full ssl options list: https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h#L71 -->
<openSSL>
<server> <!-- Used for https server AND secure tcp port -->
<!-- openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt -->
<certificateFile>/etc/clickhouse-server/server.crt</certificateFile>
<privateKeyFile>/etc/clickhouse-server/server.key</privateKeyFile>
<!-- openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096 -->
<dhParamsFile>/etc/clickhouse-server/dhparam.pem</dhParamsFile>
<verificationMode>none</verificationMode>
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
</server>
<client> <!-- Used for connecting to https dictionary source -->
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
<!-- Use for self-signed: <verificationMode>none</verificationMode> -->
<invalidCertificateHandler>
<!-- Use for self-signed: <name>AcceptCertificateHandler</name> -->
<name>RejectCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
<!-- Default root page on http[s] server. For example load UI from https://tabix.io/ when opening http://localhost:8123 -->
<!--
<http_server_default_response><![CDATA[<html ng-app="SMI2"><head><base href="http://ui.tabix.io/"></head><body><div ui-view="" class="content-ui"></div><script src="http://loader.tabix.io/master.js"></script></body></html>]]></http_server_default_response>
-->
<!-- Port for communication between replicas. Used for data exchange. -->
<interserver_http_port>9009</interserver_http_port>
<!-- Hostname that is used by other replicas to request this server.
If not specified, than it is determined analoguous to 'hostname -f' command.
This setting could be used to switch replication to another network interface.
-->
<!--
<interserver_http_host>example.yandex.ru</interserver_http_host>
-->
<!-- Listen specified host. use :: (wildcard IPv6 address), if you want to accept connections both with IPv4 and IPv6 from everywhere. -->
<!-- <listen_host>::</listen_host> -->
<!-- Same for hosts with disabled ipv6: -->
<listen_host>0.0.0.0</listen_host>
<!-- Default values - try listen localhost on ipv4 and ipv6: -->
<!--
<listen_host>::1</listen_host>
<listen_host>127.0.0.1</listen_host>
-->
<!-- Don't exit if ipv6 or ipv4 unavailable, but listen_host with this protocol specified -->
<!-- <listen_try>0</listen_try> -->
<!-- Allow listen on same address:port -->
<!-- <listen_reuse_port>0</listen_reuse_port> -->
<!-- <listen_backlog>64</listen_backlog> -->
<max_connections>4096</max_connections>
<keep_alive_timeout>3</keep_alive_timeout>
<!-- Maximum number of concurrent queries. -->
<max_concurrent_queries>100</max_concurrent_queries>
<!-- Set limit on number of open files (default: maximum). This setting makes sense on Mac OS X because getrlimit() fails to retrieve
correct maximum value. -->
<!-- <max_open_files>262144</max_open_files> -->
<!-- Size of cache of uncompressed blocks of data, used in tables of MergeTree family.
In bytes. Cache is single for server. Memory is allocated only on demand.
Cache is used when 'use_uncompressed_cache' user setting turned on (off by default).
Uncompressed cache is advantageous only for very short queries and in rare cases.
-->
<uncompressed_cache_size>8589934592</uncompressed_cache_size>
<!-- Approximate size of mark cache, used in tables of MergeTree family.
In bytes. Cache is single for server. Memory is allocated only on demand.
You should not lower this value.
-->
<mark_cache_size>5368709120</mark_cache_size>
<!-- Path to data directory, with trailing slash. -->
<path>/var/lib/clickhouse/</path>
<!-- Path to temporary data for processing hard queries. -->
<tmp_path>/var/lib/clickhouse/tmp/</tmp_path>
<!-- Directory with user provided files that are accessible by 'file' table function. -->
<user_files_path>/var/lib/clickhouse/user_files/</user_files_path>
<!-- Path to folder where users and roles created by SQL commands are stored. -->
<access_control_path>/var/lib/clickhouse/access/</access_control_path>
<!-- Sources to read users, roles, access rights, profiles of settings, quotas. -->
<user_directories>
<users_xml>
<!-- Path to configuration file with predefined users. -->
<path>users.xml</path>
</users_xml>
<local_directory>
<!-- Path to folder where users created by SQL commands are stored. -->
<path>/var/lib/clickhouse/access/</path>
</local_directory>
</user_directories>
<!-- Path to configuration file with users, access rights, profiles of settings, quotas. -->
<users_config>users.xml</users_config>
<!-- Default profile of settings. -->
<default_profile>default</default_profile>
<!-- System profile of settings. This settings are used by internal processes (Buffer storage, Distibuted DDL worker and so on). -->
<!-- <system_profile>default</system_profile> -->
<!-- Default database. -->
<default_database>default</default_database>
<!-- Server time zone could be set here.
Time zone is used when converting between String and DateTime types,
when printing DateTime in text formats and parsing DateTime from text,
it is used in date and time related functions, if specific time zone was not passed as an argument.
Time zone is specified as identifier from IANA time zone database, like UTC or Africa/Abidjan.
If not specified, system time zone at server startup is used.
Please note, that server could display time zone alias instead of specified name.
Example: W-SU is an alias for Europe/Moscow and Zulu is an alias for UTC.
-->
<!-- <timezone>Europe/Moscow</timezone> -->
<!-- You can specify umask here (see "man umask"). Server will apply it on startup.
Number is always parsed as octal. Default umask is 027 (other users cannot read logs, data files, etc; group can only read).
-->
<!-- <umask>022</umask> -->
<!-- Perform mlockall after startup to lower first queries latency
and to prevent clickhouse executable from being paged out under high IO load.
Enabling this option is recommended but will lead to increased startup time for up to a few seconds.
-->
<mlock_executable>false</mlock_executable>
<!-- Configuration of clusters that could be used in Distributed tables.
https://clickhouse.yandex/docs/en/table_engines/distributed/
-->
<remote_servers incl="clickhouse_remote_servers" >
<!-- Test only shard config for testing distributed storage -->
<test_shard_localhost>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
</test_shard_localhost>
<test_cluster_two_shards_localhost>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
</test_cluster_two_shards_localhost>
<test_shard_localhost_secure>
<shard>
<replica>
<host>localhost</host>
<port>9440</port>
<secure>1</secure>
</replica>
</shard>
</test_shard_localhost_secure>
<test_unavailable_shard>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>localhost</host>
<port>1</port>
</replica>
</shard>
</test_unavailable_shard>
</remote_servers>
<!-- If element has 'incl' attribute, then for it's value will be used corresponding substitution from another file.
By default, path to file with substitutions is /etc/metrika.xml. It could be changed in config in 'include_from' element.
Values for substitutions are specified in /yandex/name_of_substitution elements in that file.
-->
<!-- ZooKeeper is used to store metadata about replicas, when using Replicated tables.
Optional. If you don't use replicated tables, you could omit that.
See https://clickhouse.yandex/docs/en/table_engines/replication/
-->
<zookeeper incl="zookeeper-servers" optional="true" />
<!-- Substitutions for parameters of replicated tables.
Optional. If you don't use replicated tables, you could omit that.
See https://clickhouse.yandex/docs/en/table_engines/replication/#creating-replicated-tables
-->
<macros incl="macros" optional="true" />
<!-- Reloading interval for embedded dictionaries, in seconds. Default: 3600. -->
<builtin_dictionaries_reload_interval>3600</builtin_dictionaries_reload_interval>
<!-- Maximum session timeout, in seconds. Default: 3600. -->
<max_session_timeout>3600</max_session_timeout>
<!-- Default session timeout, in seconds. Default: 60. -->
<default_session_timeout>60</default_session_timeout>
<!-- Sending data to Graphite for monitoring. Several sections can be defined. -->
<!--
interval - send every X second
root_path - prefix for keys
hostname_in_path - append hostname to root_path (default = true)
metrics - send data from table system.metrics
events - send data from table system.events
asynchronous_metrics - send data from table system.asynchronous_metrics
-->
<!--
<graphite>
<host>localhost</host>
<port>42000</port>
<timeout>0.1</timeout>
<interval>60</interval>
<root_path>one_min</root_path>
<hostname_in_path>true</hostname_in_path>
<metrics>true</metrics>
<events>true</events>
<asynchronous_metrics>true</asynchronous_metrics>
</graphite>
<graphite>
<host>localhost</host>
<port>42000</port>
<timeout>0.1</timeout>
<interval>1</interval>
<root_path>one_sec</root_path>
<metrics>true</metrics>
<events>true</events>
<asynchronous_metrics>false</asynchronous_metrics>
</graphite>
-->
<!-- Query log. Used only for queries with setting log_queries = 1. -->
<query_log>
<!-- What table to insert data. If table is not exist, it will be created.
When query log structure is changed after system update,
then old table will be renamed and new table will be created automatically.
-->
<database>system</database>
<table>query_log</table>
<!--
PARTITION BY expr https://clickhouse.yandex/docs/en/table_engines/custom_partitioning_key/
Example:
event_date
toMonday(event_date)
toYYYYMM(event_date)
toStartOfHour(event_time)
-->
<partition_by>toYYYYMM(event_date)</partition_by>
<!-- Interval of flushing data. -->
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</query_log>
<!-- Trace log. Stores stack traces collected by query profilers.
See query_profiler_real_time_period_ns and query_profiler_cpu_time_period_ns settings. -->
<trace_log>
<database>system</database>
<table>trace_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</trace_log>
<!-- Query thread log. Has information about all threads participated in query execution.
Used only for queries with setting log_query_threads = 1. -->
<query_thread_log>
<database>system</database>
<table>query_thread_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</query_thread_log>
<!-- Uncomment if use part log.
Part log contains information about all actions with parts in MergeTree tables (creation, deletion, merges, downloads).
<part_log>
<database>system</database>
<table>part_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</part_log>
-->
<!-- Uncomment to write text log into table.
Text log contains all information from usual server log but stores it in structured and efficient way.
<text_log>
<database>system</database>
<table>text_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</text_log>
-->
<!-- Parameters for embedded dictionaries, used in Yandex.Metrica.
See https://clickhouse.yandex/docs/en/dicts/internal_dicts/
-->
<!-- Path to file with region hierarchy. -->
<!-- <path_to_regions_hierarchy_file>/opt/geo/regions_hierarchy.txt</path_to_regions_hierarchy_file> -->
<!-- Path to directory with files containing names of regions -->
<!-- <path_to_regions_names_files>/opt/geo/</path_to_regions_names_files> -->
<!-- Configuration of external dictionaries. See:
https://clickhouse.yandex/docs/en/dicts/external_dicts/
-->
<dictionaries_config>*_dictionary.xml</dictionaries_config>
<!-- Uncomment if you want data to be compressed 30-100% better.
Don't do that if you just started using ClickHouse.
-->
<compression incl="clickhouse_compression">
<!--
<!- - Set of variants. Checked in order. Last matching case wins. If nothing matches, lz4 will be used. - ->
<case>
<!- - Conditions. All must be satisfied. Some conditions may be omitted. - ->
<min_part_size>10000000000</min_part_size> <!- - Min part size in bytes. - ->
<min_part_size_ratio>0.01</min_part_size_ratio> <!- - Min size of part relative to whole table size. - ->
<!- - What compression method to use. - ->
<method>zstd</method>
</case>
-->
</compression>
<!-- Allow to execute distributed DDL queries (CREATE, DROP, ALTER, RENAME) on cluster.
Works only if ZooKeeper is enabled. Comment it if such functionality isn't required. -->
<distributed_ddl>
<!-- Path in ZooKeeper to queue with DDL queries -->
<path>/clickhouse/task_queue/ddl</path>
<!-- Settings from this profile will be used to execute DDL queries -->
<!-- <profile>default</profile> -->
</distributed_ddl>
<!-- Settings to fine tune MergeTree tables. See documentation in source code, in MergeTreeSettings.h -->
<!--
<merge_tree>
<max_suspicious_broken_parts>5</max_suspicious_broken_parts>
</merge_tree>
-->
<!-- Protection from accidental DROP.
If size of a MergeTree table is greater than max_table_size_to_drop (in bytes) than table could not be dropped with any DROP query.
If you want do delete one table and don't want to restart clickhouse-server, you could create special file <clickhouse-path>/flags/force_drop_table and make DROP once.
By default max_table_size_to_drop is 50GB; max_table_size_to_drop=0 allows to DROP any tables.
The same for max_partition_size_to_drop.
Uncomment to disable protection.
-->
<!-- <max_table_size_to_drop>0</max_table_size_to_drop> -->
<!-- <max_partition_size_to_drop>0</max_partition_size_to_drop> -->
<!-- Example of parameters for GraphiteMergeTree table engine -->
<graphite_rollup_example>
<pattern>
<regexp>click_cost</regexp>
<function>any</function>
<retention>
<age>0</age>
<precision>3600</precision>
</retention>
<retention>
<age>86400</age>
<precision>60</precision>
</retention>
</pattern>
<default>
<function>max</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>3600</age>
<precision>300</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</default>
</graphite_rollup_example>
<!-- Directory in <clickhouse-path> containing schema files for various input formats.
The directory will be created if it doesn't exist.
-->
<format_schema_path>/var/lib/clickhouse/format_schemas/</format_schema_path>
<!-- Uncomment to disable ClickHouse internal DNS caching. -->
<!-- <disable_internal_dns_cache>1</disable_internal_dns_cache> -->
</yandex>

View File

@ -0,0 +1,133 @@
<?xml version="1.0"?>
<yandex>
<!-- Profiles of settings. -->
<profiles>
<!-- Default settings. -->
<default>
<!-- Maximum memory usage for processing single query, in bytes. -->
<max_memory_usage>10000000000</max_memory_usage>
<!-- Use cache of uncompressed blocks of data. Meaningful only for processing many of very short queries. -->
<use_uncompressed_cache>0</use_uncompressed_cache>
<!-- How to choose between replicas during distributed query processing.
random - choose random replica from set of replicas with minimum number of errors
nearest_hostname - from set of replicas with minimum number of errors, choose replica
with minimum number of different symbols between replica's hostname and local hostname
(Hamming distance).
in_order - first live replica is chosen in specified order.
first_or_random - if first replica one has higher number of errors, pick a random one from replicas with minimum number of errors.
-->
<load_balancing>random</load_balancing>
</default>
<!-- Profile that allows only read queries. -->
<readonly>
<readonly>1</readonly>
</readonly>
</profiles>
<!-- Users and ACL. -->
<users>
<!-- If user name was not specified, 'default' user is used. -->
<default>
<!-- Password could be specified in plaintext or in SHA256 (in hex format).
If you want to specify password in plaintext (not recommended), place it in 'password' element.
Example: <password>qwerty</password>.
Password could be empty.
If you want to specify SHA256, place it in 'password_sha256_hex' element.
Example: <password_sha256_hex>65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5</password_sha256_hex>
Restrictions of SHA256: impossibility to connect to ClickHouse using MySQL JS client (as of July 2019).
If you want to specify double SHA1, place it in 'password_double_sha1_hex' element.
Example: <password_double_sha1_hex>e395796d6546b1b65db9d665cd43f0e858dd4303</password_double_sha1_hex>
How to generate decent password:
Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | sha256sum | tr -d '-'
In first line will be password and in second - corresponding SHA256.
How to generate double SHA1:
Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | openssl dgst -sha1 -binary | openssl dgst -sha1
In first line will be password and in second - corresponding double SHA1.
-->
<password></password>
<!-- List of networks with open access.
To open access from everywhere, specify:
<ip>::/0</ip>
To open access only from localhost, specify:
<ip>::1</ip>
<ip>127.0.0.1</ip>
Each element of list has one of the following forms:
<ip> IP-address or network mask. Examples: 213.180.204.3 or 10.0.0.1/8 or 10.0.0.1/255.255.255.0
2a02:6b8::3 or 2a02:6b8::3/64 or 2a02:6b8::3/ffff:ffff:ffff:ffff::.
<host> Hostname. Example: server01.yandex.ru.
To check access, DNS query is performed, and all received addresses compared to peer address.
<host_regexp> Regular expression for host names. Example, ^server\d\d-\d\d-\d\.yandex\.ru$
To check access, DNS PTR query is performed for peer address and then regexp is applied.
Then, for result of PTR query, another DNS query is performed and all received addresses compared to peer address.
Strongly recommended that regexp is ends with $
All results of DNS requests are cached till server restart.
-->
<networks incl="networks" replace="replace">
<ip>::/0</ip>
</networks>
<!-- Settings profile for user. -->
<profile>default</profile>
<!-- Quota for user. -->
<quota>default</quota>
<!-- Allow access management -->
<access_management>1</access_management>
<!-- Example of row level security policy. -->
<!-- <databases>
<test>
<filtered_table1>
<filter>a = 1</filter>
</filtered_table1>
<filtered_table2>
<filter>a + b &lt; 1 or c - d &gt; 5</filter>
</filtered_table2>
</test>
</databases> -->
</default>
<!-- Example of user with readonly access. -->
<!-- <readonly>
<password></password>
<networks incl="networks" replace="replace">
<ip>::1</ip>
<ip>127.0.0.1</ip>
</networks>
<profile>readonly</profile>
<quota>default</quota>
</readonly> -->
</users>
<!-- Quotas. -->
<quotas>
<!-- Name of quota. -->
<default>
<!-- Limits for time interval. You could specify many intervals with different limits. -->
<interval>
<!-- Length of interval. -->
<duration>3600</duration>
<!-- No limits. Just calculate resource usage for time interval. -->
<queries>0</queries>
<errors>0</errors>
<result_rows>0</result_rows>
<read_rows>0</read_rows>
<execution_time>0</execution_time>
</interval>
</default>
</quotas>
</yandex>

View File

@ -0,0 +1,11 @@
def not_implemented_bigints(name):
return(48, f"Exception: {name} is not implemented for big integers")
def bigints_not_implements(name):
return(48, f'Exception: {name} for big integers is not implemented')
def illegal_type():
return(43, 'Exception: Illegal type')
def illegal_column():
return(44, 'Exception: Illegal column')

View File

@ -0,0 +1,27 @@
version: '2.3'
services:
clickhouse:
image: yandex/clickhouse-integration-test
expose:
- "9000"
- "9009"
- "8123"
volumes:
- "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/config.d:/etc/clickhouse-server/config.d"
- "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/users.d:/etc/clickhouse-server/users.d"
- "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/config.xml:/etc/clickhouse-server/config.xml"
- "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/users.xml:/etc/clickhouse-server/users.xml"
- "${CLICKHOUSE_TESTS_SERVER_BIN_PATH:-/usr/bin/clickhouse}:/usr/bin/clickhouse"
- "${CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH:-/usr/bin/clickhouse-odbc-bridge}:/usr/bin/clickhouse-odbc-bridge"
entrypoint: bash -c "clickhouse server --config-file=/etc/clickhouse-server/config.xml --log-file=/var/log/clickhouse-server/clickhouse-server.log --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log"
healthcheck:
test: clickhouse client --query='select 1'
interval: 10s
timeout: 10s
retries: 3
start_period: 300s
cap_add:
- SYS_PTRACE
security_opt:
- label:disable

View File

@ -0,0 +1,30 @@
version: '2.3'
services:
mysql1:
extends:
file: mysql-service.yml
service: mysql
hostname: mysql1
volumes:
- "${CLICKHOUSE_TESTS_DIR}/_instances/mysql1/database:/var/lib/mysql"
clickhouse1:
extends:
file: clickhouse-service.yml
service: clickhouse
hostname: clickhouse1
volumes:
- "${CLICKHOUSE_TESTS_DIR}/_instances/clickhouse1/database/:/var/lib/clickhouse/"
- "${CLICKHOUSE_TESTS_DIR}/_instances/clickhouse1/logs/:/var/log/clickhouse-server/"
# dummy service which does nothing, but allows to postpone
# 'docker-compose up -d' till all dependecies will go healthy
all_services_ready:
image: hello-world
depends_on:
clickhouse1:
condition: service_healthy
mysql1:
condition: service_healthy

View File

@ -0,0 +1,19 @@
version: '2.3'
services:
mysql:
image: mysql:5.7.30
restart: always
environment:
MYSQL_DATABASE: 'db'
MYSQL_USER: 'user'
MYSQL_PASSWORD: 'password'
MYSQL_ROOT_PASSWORD: 'password'
expose:
- '3306'
healthcheck:
test: mysql -D db -u user --password=password -e "select 1;"
interval: 3s
timeout: 2s
retries: 40
start_period: 2s

View File

@ -0,0 +1,52 @@
#!/usr/bin/env python3
import os
import sys
from testflows.core import *
append_path(sys.path, "..")
from helpers.cluster import Cluster
from helpers.argparser import argparser
from extended_precision_data_types.requirements import *
xfails = {
}
xflags = {
}
@TestModule
@ArgumentParser(argparser)
@XFails(xfails)
@XFlags(xflags)
@Name("extended precision data types")
@Specifications(
QA_SRS020_ClickHouse_Extended_Precision_Data_Types
)
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision("1.0"),
)
def regression(self, local, clickhouse_binary_path, stress=None, parallel=None):
"""Extended precision data type regression.
"""
top().terminating = False
nodes = {
"clickhouse":
("clickhouse1",)
}
with Cluster(local, clickhouse_binary_path, nodes=nodes,
docker_compose_project_dir=os.path.join(current_dir(), "extended-precision-data-type_env")) as cluster:
self.context.cluster = cluster
self.context.stress = stress
if parallel is not None:
self.context.parallel = parallel
Feature(run=load("extended_precision_data_types.tests.feature", "feature"))
if main():
regression()

View File

@ -0,0 +1 @@
from .requirements import *

View File

@ -0,0 +1,565 @@
# QA-SRS020 ClickHouse Extended Precision Data Types
# Software Requirements Specification
## Table of Contents
* 1 [Revision History](#revision-history)
* 2 [Introduction](#introduction)
* 3 [Terminology](#terminology)
* 3.1 [Extended Precision Data Types](#extended-precision-data-types)
* 4 [Requirements](#requirements)
* 4.1 [RQ.SRS-020.ClickHouse.Extended.Precision](#rqsrs-020clickhouseextendedprecision)
* 4.2 [Conversion](#conversion)
* 4.2.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toInt128](#rqsrs-020clickhouseextendedprecisionconversiontoint128)
* 4.2.2 [RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toUInt128](#rqsrs-020clickhouseextendedprecisionconversiontouint128)
* 4.2.3 [RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toInt256](#rqsrs-020clickhouseextendedprecisionconversiontoint256)
* 4.2.4 [RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toUInt256](#rqsrs-020clickhouseextendedprecisionconversiontouint256)
* 4.2.5 [RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toDecimal256](#rqsrs-020clickhouseextendedprecisionconversiontodecimal256)
* 4.2.6 [RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.FromMySQL](#rqsrs-020clickhouseextendedprecisionconversionfrommysql)
* 4.2.7 [RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.ToMySQL](#rqsrs-020clickhouseextendedprecisionconversiontomysql)
* 4.3 [Arithmetic](#arithmetic)
* 4.3.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Arithmetic.Int.Supported](#rqsrs-020clickhouseextendedprecisionarithmeticintsupported)
* 4.3.2 [RQ.SRS-020.ClickHouse.Extended.Precision.Arithmetic.Dec.Supported](#rqsrs-020clickhouseextendedprecisionarithmeticdecsupported)
* 4.3.3 [RQ.SRS-020.ClickHouse.Extended.Precision.Arithmetic.Dec.NotSupported](#rqsrs-020clickhouseextendedprecisionarithmeticdecnotsupported)
* 4.4 [Arrays](#arrays)
* 4.4.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Int.Supported](#rqsrs-020clickhouseextendedprecisionarraysintsupported)
* 4.4.2 [RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Int.NotSupported](#rqsrs-020clickhouseextendedprecisionarraysintnotsupported)
* 4.4.3 [RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Dec.Supported](#rqsrs-020clickhouseextendedprecisionarraysdecsupported)
* 4.4.4 [RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Dec.NotSupported](#rqsrs-020clickhouseextendedprecisionarraysdecnotsupported)
* 4.5 [Comparison](#comparison)
* 4.5.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Comparison](#rqsrs-020clickhouseextendedprecisioncomparison)
* 4.6 [Logical Functions](#logical-functions)
* 4.6.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Logical](#rqsrs-020clickhouseextendedprecisionlogical)
* 4.7 [Mathematical Functions](#mathematical-functions)
* 4.7.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Mathematical.Supported](#rqsrs-020clickhouseextendedprecisionmathematicalsupported)
* 4.7.2 [RQ.SRS-020.ClickHouse.Extended.Precision.Mathematical.NotSupported](#rqsrs-020clickhouseextendedprecisionmathematicalnotsupported)
* 4.8 [Rounding Functions](#rounding-functions)
* 4.8.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Int.Supported](#rqsrs-020clickhouseextendedprecisionroundingintsupported)
* 4.8.2 [RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Int.NotSupported](#rqsrs-020clickhouseextendedprecisionroundingintnotsupported)
* 4.8.3 [RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Dec.Supported](#rqsrs-020clickhouseextendedprecisionroundingdecsupported)
* 4.8.4 [RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Dec.NotSupported](#rqsrs-020clickhouseextendedprecisionroundingdecnotsupported)
* 4.9 [Bit Functions](#bit-functions)
* 4.9.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Bit.Int.Supported](#rqsrs-020clickhouseextendedprecisionbitintsupported)
* 4.9.2 [RQ.SRS-020.ClickHouse.Extended.Precision.Bit.Int.NotSupported](#rqsrs-020clickhouseextendedprecisionbitintnotsupported)
* 4.9.3 [RQ.SRS-020.ClickHouse.Extended.Precision.Bit.Dec.NotSupported](#rqsrs-020clickhouseextendedprecisionbitdecnotsupported)
* 4.10 [Null Functions](#null-functions)
* 4.10.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Null](#rqsrs-020clickhouseextendedprecisionnull)
* 4.11 [Tuple Functions](#tuple-functions)
* 4.11.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Tuple](#rqsrs-020clickhouseextendedprecisiontuple)
* 4.12 [Map Functions](#map-functions)
* 4.12.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Map.Supported](#rqsrs-020clickhouseextendedprecisionmapsupported)
* 4.12.2 [RQ.SRS-020.ClickHouse.Extended.Precision.Map.NotSupported](#rqsrs-020clickhouseextendedprecisionmapnotsupported)
* 4.13 [Create](#create)
* 4.13.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Create.Table](#rqsrs-020clickhouseextendedprecisioncreatetable)
* 5 [References](#references)
## Revision History
This document is stored in an electronic form using [Git] source control management software
hosted in a [GitHub Repository].
All the updates are tracked using the [Revision History].
## Introduction
This software requirements specification covers requirements related to [ClickHouse]
using extended precision data types.
## Terminology
### Extended Precision Data Types
Inclusive bounds:
* Int128 - [-170141183460469231731687303715884105728 : 170141183460469231731687303715884105727]
* UInt128 - [0 : 340282366920938463463374607431768211455]
* Int256 - [-57896044618658097711785492504343953926634992332820282019728792003956564819968 : 57896044618658097711785492504343953926634992332820282019728792003956564819967]
* UInt256 - [0 : 115792089237316195423570985008687907853269984665640564039457584007913129639935]
Exclusive bounds:
* Decimal256 - (10^(76 - S): 10^(76 - S)), where S is the scale.
## Requirements
### RQ.SRS-020.ClickHouse.Extended.Precision
version: 1.0
[ClickHouse] SHALL support using [Extended Precision Data Types].
### Conversion
#### RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toInt128
version: 1.0
[ClickHouse] SHALL support converting values to `Int128` using the `toInt128` function.
For example,
```sql
SELECT toInt128(1)
```
#### RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toUInt128
version: 1.0
[ClickHouse] SHALL support converting values to `UInt128` format using `toUInt128` function.
For example,
```sql
SELECT toUInt128(1)
```
#### RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toInt256
version: 1.0
[ClickHouse] SHALL support converting values to `Int256` using `toInt256` function.
For example,
```sql
SELECT toInt256(1)
```
#### RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toUInt256
version: 1.0
[ClickHouse] SHALL support converting values to `UInt256` format using `toUInt256` function.
For example,
```sql
SELECT toUInt256(1)
```
#### RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toDecimal256
version: 1.0
[ClickHouse] SHALL support converting values to `Decimal256` format using `toDecimal256` function.
For example,
```sql
SELECT toDecimal256(1,2)
```
#### RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.FromMySQL
version: 1.0
[ClickHouse] SHALL support converting to [Extended Precision Data Types] from MySQL.
#### RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.ToMySQL
version: 1.0
[ClickHouse] MAY not support converting from [Extended Precision Data Types] to MySQL.
### Arithmetic
#### RQ.SRS-020.ClickHouse.Extended.Precision.Arithmetic.Int.Supported
version: 1.0
[ClickHouse] SHALL support using [Arithmetic functions] with Int128, UInt128, Int256, and UInt256.
Arithmetic functions:
* plus
* minus
* multiply
* divide
* intDiv
* intDivOrZero
* modulo
* moduloOrZero
* negate
* abs
* gcd
* lcm
#### RQ.SRS-020.ClickHouse.Extended.Precision.Arithmetic.Dec.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Arithmetic functions] with Decimal256:
* plus
* minus
* multiply
* divide
* intDiv
* intDivOrZero
* negate
* abs
#### RQ.SRS-020.ClickHouse.Extended.Precision.Arithmetic.Dec.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Arithmetic functions] with Decimal256:
* modulo
* moduloOrZero
* gcd
* lcm
### Arrays
#### RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Int.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Array functions] with Int128, UInt128, Int256, and UInt256.
* empty
* notEmpty
* length
* arrayCount
* arrayPopBack
* arrayPopFront
* arraySort
* arrayReverseSort
* arrayUniq
* arrayJoin
* arrayDistinct
* arrayEnumerate
* arrayEnumerateDense
* arrayEnumerateUniq
* arrayReverse
* reverse
* arrayFlatten
* arrayCompact
* arrayExists
* arrayAll
* arrayMin
* arrayMax
* arraySum
* arrayAvg
* arrayReduce
* arrayReduceInRanges
* arrayZip
* arrayMap
* arrayFilter
* arrayFill
* arrayReverseFill
* arraySplit
* arrayFirst
* arrayFirstIndex
* arrayConcat
* hasAll
* hasAny
* hasSubstr
* arrayElement
* has
* indexOf
* countEqual
* arrayPushBack
* arrayPushFront
* arrayResize
* arraySlice
#### RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Int.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Array functions] with Int128, UInt128, Int256, and UInt256:
* arrayDifference
* arrayCumSum
* arrayCumSumNonNegative
#### RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Dec.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Array functions] with Decimal256:
* empty
* notEmpty
* length
* arrayCount
* arrayPopBack
* arrayPopFront
* arraySort
* arrayReverseSort
* arrayUniq
* arrayJoin
* arrayDistinct
* arrayEnumerate
* arrayEnumerateDense
* arrayEnumerateUniq
* arrayReverse
* reverse
* arrayFlatten
* arrayCompact
* arrayExists
* arrayAll
* arrayReduce
* arrayReduceInRanges
* arrayZip
* arrayMap
* arrayFilter
* arrayFill
* arrayReverseFill
* arraySplit
* arrayFirst
* arrayFirstIndex
* arrayConcat
* hasAll
* hasAny
* hasSubstr
* arrayElement
* has
* indexOf
* countEqual
* arrayPushBack
* arrayPushFront
* arrayResize
* arraySlice
#### RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Dec.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Array functions] with Decimal256:
* arrayMin
* arrayMax
* arraaySum
* arrayAvg
* arrayDifference
* arrayCumSum
* arrayCumSumNonNegative
### Comparison
#### RQ.SRS-020.ClickHouse.Extended.Precision.Comparison
version: 1.0
[ClickHouse] SHALL support using [Comparison functions] with [Extended Precision Data Types].
Comparison functions:
* equals
* notEquals
* less
* greater
* lessOrEquals
* greaterOrEquals
### Logical Functions
#### RQ.SRS-020.ClickHouse.Extended.Precision.Logical
version: 1.0
[ClickHouse] MAY not support using [Logical functions] with [Extended Precision Data Types].
Logical functions:
* and
* or
* not
* xor
### Mathematical Functions
#### RQ.SRS-020.ClickHouse.Extended.Precision.Mathematical.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Mathematical functions] with [Extended Precision Data Types]:
* exp
* log, ln
* exp2
* log2
* exp10
* log10
* sqrt
* cbrt
* erf
* erfc
* lgamma
* tgamma
* sin
* cos
* tan
* asin
* acos
* atan
* cosh
* acosh
* sinh
* asinh
* tanh
* atanh
* log1p
* sign
#### RQ.SRS-020.ClickHouse.Extended.Precision.Mathematical.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Mathematical functions] with [Extended Precision Data Types]:
* pow, power
* intExp2
* intExp10
* atan2
* hypot
### Rounding Functions
#### RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Int.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Rounding functions] with Int128, UInt128, Int256, and UInt256:
* floor
* ceil
* trunc
* round
* roundBankers
* roundDuration
* roundAge
#### RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Int.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Rounding functions] with Int128, UInt128, Int256, and UInt256:
* roundDown
* roundToExp2
#### RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Dec.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Rounding functions] with Decimal256:
* floor
* ceil
* trunc
* round
* roundBankers
#### RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Dec.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Rounding functions] with Decimal256:
* roundDuration
* roundAge
* roundDown
* roundToExp2
### Bit Functions
#### RQ.SRS-020.ClickHouse.Extended.Precision.Bit.Int.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Bit functions] with Int128, UInt128, Int256, and UInt256:
* bitAnd
* bitOr
* bitXor
* bitNot
* bitShiftLeft
* bitShiftRight
* bitCount
#### RQ.SRS-020.ClickHouse.Extended.Precision.Bit.Int.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Bit functions] with Int128, UInt128, Int256, and UInt256:
* bitRotateLeft
* bitRotateRight
* bitTest
* bitTestAll
* bitTestAny
#### RQ.SRS-020.ClickHouse.Extended.Precision.Bit.Dec.NotSupported
version: 1.0
[ClickHouse] MAY not support using [Bit functions] with Decimal256.
Bit functions:
* bitAnd
* bitOr
* bitXor
* bitNot
* bitShiftLeft
* bitShiftRight
* bitCount
* bitRotateLeft
* bitRotateRight
* bitTest
* bitTestAll
* bitTestAny
### Null Functions
#### RQ.SRS-020.ClickHouse.Extended.Precision.Null
version: 1.0
[ClickHouse] SHALL support using [Null functions] with [Extended Precision Data Types].
Null functions:
* isNull
* isNotNull
* coalesce
* ifNull
* nullIf
* assumeNotNull
* toNullable
### Tuple Functions
#### RQ.SRS-020.ClickHouse.Extended.Precision.Tuple
version: 1.0
[ClickHouse] SHALL support using [Tuple functions] with [Extended Precision Data Types].
Tuple functions:
* tuple
* tupleElement
* untuple
### Map Functions
#### RQ.SRS-020.ClickHouse.Extended.Precision.Map.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Map functions] with [Extended Precision Data Types]:
* map
* mapContains
* mapKeys
* mapValues
#### RQ.SRS-020.ClickHouse.Extended.Precision.Map.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Map functions] with [Extended Precision Data Types]:
* mapAdd
* mapSubtract
* mapPopulateSeries
### Create
#### RQ.SRS-020.ClickHouse.Extended.Precision.Create.Table
version: 1.0
[ClickHouse] SHALL support creating table with columns that use [Extended Precision Data Types].
## References
* **ClickHouse:** https://clickhouse.tech
* **GitHub Repository**: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/extended_precision_data_types/requirements/requirements.md
* **Revision History**: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/extended_precision_data_types/requirements/requirements.md
* **Git:** https://git-scm.com/
[Extended Precision Data Types]: #extended-precision-data-types
[Arithmetic functions]: https://clickhouse.tech/docs/en/sql-reference/functions/arithmetic-functions/
[Array functions]: https://clickhouse.tech/docs/en/sql-reference/functions/array-functions/
[Comparison functions]: https://clickhouse.tech/docs/en/sql-reference/functions/comparison-functions/
[Logical Functions]: https://clickhouse.tech/docs/en/sql-reference/functions/logical-functions/
[Mathematical Functions]: https://clickhouse.tech/docs/en/sql-reference/functions/math-functions/
[Rounding Functions]: https://clickhouse.tech/docs/en/sql-reference/functions/rounding-functions/
[Bit Functions]: https://clickhouse.tech/docs/en/sql-reference/functions/bit-functions/
[Null Functions]: https://clickhouse.tech/docs/en/sql-reference/functions/functions-for-nulls/
[Tuple Functions]: https://clickhouse.tech/docs/en/sql-reference/functions/tuple-functions/
[Map Functions]: https://clickhouse.tech/docs/en/sql-reference/functions/tuple-map-functions/
[SRS]: #srs
[ClickHouse]: https://clickhouse.tech
[GitHub Repository]: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/extended_precision_data_types/requirements/requirements.md
[Revision History]: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/extended_precision_data_types/requirements/requirements.md
[Git]: https://git-scm.com/
[GitHub]: https://github.com

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,217 @@
import os
import textwrap
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
funcs = [
('plus', '2'),
('minus', '0'),
('multiply', '1'),
('divide', '1'),
('intDiv', '1'),
('intDivOrZero', '1'),
('modulo', '0'),
('moduloOrZero', '0'),
('negate', '-1'),
('abs', '1'),
('gcd', '1'),
('lcm', '1'),
]
Examples_list = [tuple(list(func)+list(data_type)+[Name(f'{func[0]} - {data_type[0]}')]) for func in funcs for data_type in data_types]
Examples_dec_list = [tuple(list(func)+[Name(f'{func[0]} - Decimal256')]) for func in funcs]
@TestOutline
@Examples('arithmetic_func expected_result int_type min max', Examples_list)
def inline_check(self, arithmetic_func, expected_result, int_type, min, max, node=None):
"""Check that arithmetic functions work using inline tests with Int128, UInt128, Int256, and UInt256.
"""
if node is None:
node = self.context.node
if arithmetic_func in ['negate','abs']:
with When(f"I check {arithmetic_func} with {int_type}"):
output = node.query(f"SELECT {arithmetic_func}(to{int_type}(1))").output
assert output == expected_result, error()
with When(f"I check {arithmetic_func} with {int_type} max and min value"):
execute_query(f"""
SELECT {arithmetic_func}(to{int_type}(\'{max}\')), {arithmetic_func}(to{int_type}(\'{min}\'))
""")
else:
with When(f"I check {arithmetic_func} with {int_type}"):
output = node.query(f"SELECT {arithmetic_func}(to{int_type}(1), to{int_type}(1))").output
assert output == expected_result, error()
if arithmetic_func in ['gcd','lcm']:
if int_type in ['UInt128','UInt256']:
exitcode=153
else:
exitcode=151
with When(f"I check {arithmetic_func} with {int_type} max and min value"):
node.query(f"SELECT {arithmetic_func}(to{int_type}(\'{max}\'), to{int_type}(1)), {arithmetic_func}(to{int_type}(\'{min}\'), to{int_type}(1))",
exitcode = exitcode, message = 'Exception:')
else:
with When(f"I check {arithmetic_func} with {int_type} max and min value"):
execute_query(f"""
SELECT {arithmetic_func}(to{int_type}(\'{max}\'), to{int_type}(1)), {arithmetic_func}(to{int_type}(\'{min}\'), to{int_type}(1))
""")
@TestOutline
@Examples('arithmetic_func expected_result int_type min max', Examples_list)
def table_check(self, arithmetic_func, expected_result, int_type, min, max, node=None):
"""Check that arithmetic functions work using tables with Int128, UInt128, Int256, and UInt256.
"""
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = int_type)
if arithmetic_func in ['negate','abs']:
for value in [1, min, max]:
with When(f"I insert {arithmetic_func} with {int_type} {value} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(to{int_type}(\'{value}\'))")
with Then(f"I check the table output of {arithmetic_func} with {int_type}"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
else:
with When(f"I insert {arithmetic_func} with {int_type} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(to{int_type}(1), to{int_type}(1))")
with Then("I check that the output matches the expected value"):
output = node.query(f"SELECT * FROM {table_name}").output
assert output == expected_result, error()
if arithmetic_func in ['gcd', 'lcm']:
if int_type in ['UInt128', 'UInt256']:
with When(f"I insert {arithmetic_func} with {int_type} {min} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(to{int_type}(\'{min}\'), to{int_type}(1))",
exitcode = 153, message = 'Exception:')
with And(f"I insert {arithmetic_func} with {int_type} {max} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(to{int_type}(\'{max}\'), to{int_type}(1))")
else:
for value in [min, max]:
with When(f"I insert {arithmetic_func} with {int_type} {value} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(to{int_type}(\'{value}\'), to{int_type}(1))",
exitcode = 151, message = 'Exception:')
else:
for value in [min, max]:
with When(f"I insert {arithmetic_func} with {int_type} {value} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(to{int_type}(\'{value}\'), to{int_type}(1))")
with Then(f"I check the table output of {arithmetic_func} with {int_type}"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestOutline
@Examples('arithmetic_func expected_result', Examples_dec_list)
def inline_check_dec(self, arithmetic_func, expected_result, node=None):
"""Check that arithmetic functions work using inline with Decimal256.
"""
if node is None:
node = self.context.node
if arithmetic_func is 'negate' or arithmetic_func is 'abs':
with When(f"I check {arithmetic_func} with toDecimal256"):
output = node.query(f"SELECT {arithmetic_func}(toDecimal256(1,0))").output
assert output == expected_result, error()
elif arithmetic_func in ['modulo', 'moduloOrZero', 'gcd', 'lcm']:
with When(f"I check {arithmetic_func} with toDecimal256"):
node.query(f"SELECT {arithmetic_func}(toDecimal256(1,0), toDecimal256(1,0))",
exitcode=43, message = 'Exception:')
else:
with When(f"I check {arithmetic_func} with toDecimal256"):
output = node.query(f"SELECT {arithmetic_func}(toDecimal256(1,0), toDecimal256(1,0))").output
assert output == expected_result, error()
@TestOutline
@Examples('arithmetic_func expected_result', Examples_dec_list)
def table_check_dec(self, arithmetic_func, expected_result, node=None):
"""Check that arithmetic functions work using tables with Decimal256.
"""
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = 'Decimal256(0)')
if arithmetic_func in ['negate','abs']:
with When(f"I insert {arithmetic_func} with toDecimal256 into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(toDecimal256(1,0))")
with Then(f"I check the table for output of {arithmetic_func} with Decimal256"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
elif arithmetic_func in ['modulo', 'moduloOrZero', 'gcd', 'lcm']:
with When(f"I check {arithmetic_func} with toDecimal256"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(toDecimal256(1,0), toDecimal256(1,0))",
exitcode=43, message = 'Exception:')
else:
with When(f"I insert {arithmetic_func} with toDecimal256 into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(toDecimal256(1,0), toDecimal256(1,0))")
with Then("I check that the output matches the expected value"):
output = node.query(f"SELECT * FROM {table_name}").output
assert output == expected_result, error()
@TestFeature
@Name("arithmetic")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Arithmetic_Int_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Arithmetic_Dec_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Arithmetic_Dec_NotSupported("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that arithmetic functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
with allow_experimental_bigint(self.context.node):
Scenario(run = inline_check)
Scenario(run = table_check)
Scenario(run = inline_check_dec)
Scenario(run = table_check_dec)

View File

@ -0,0 +1,484 @@
import uuid
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
def get_table_name():
return "table" + "_" + str(uuid.uuid1()).replace('-', '_')
@TestOutline(Suite)
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Arrays_Int_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Arrays_Int_NotSupported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Arrays_Dec_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Arrays_Dec_NotSupported("1.0"),
)
def array_func(self, data_type, node=None):
"""Check array functions with extended precision data types.
"""
if node is None:
node = self.context.node
for func in ['arrayPopBack(',
'arrayPopFront(',
'arraySort(',
'arrayReverseSort(',
'arrayDistinct(',
'arrayEnumerate(',
'arrayEnumerateDense(',
'arrayEnumerateUniq(',
'arrayReverse(',
'reverse(',
'arrayFlatten(',
'arrayCompact(',
'arrayReduceInRanges(\'sum\', [(1, 5)],',
'arrayMap(x -> (x + 2),',
'arrayFill(x -> x=3,',
'arrayReverseFill(x -> x=3,',
f'arrayConcat([{to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}],',
'arrayFilter(x -> x == 1, ']:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = f'Array({data_type})')
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in ['arraySplit((x, y) -> x=y, [0, 0, 0],']:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = f'Array(Array({data_type}))')
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in [f'arrayZip([{to_data_type(data_type,1)}],']:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}array({to_data_type(data_type,3)}))
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = f'Array(Tuple({data_type}, {data_type}))')
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}array({to_data_type(data_type,1)}))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in ['empty(',
'notEmpty(',
'length(',
'arrayCount(x -> x == 1, ',
'arrayUniq(',
'arrayJoin(',
'arrayExists(x -> x==1,',
'arrayAll(x -> x==1,',
'arrayMin(',
'arrayMax(',
'arraySum(',
'arrayAvg(',
'arrayReduce(\'max\', ',
'arrayFirst(x -> x==3,',
'arrayFirstIndex(x -> x==3,',
f'hasAll([{to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}], ',
f'hasAny([{to_data_type(data_type,2)}, {to_data_type(data_type,1)}], ',
f'hasSubstr([{to_data_type(data_type,2)}, {to_data_type(data_type,1)}], ']:
if func in ['arrayMin(','arrayMax(','arraySum(', 'arrayAvg('] and data_type in ['Decimal256(0)']:
with Scenario(f"Inline - {data_type} - {func}"):
node.query(f"SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))",
exitcode = 44, message = 'Exception:')
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))",
exitcode = 44, message = 'Exception:')
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
else:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in ['arrayDifference(',
'arrayCumSum(',
'arrayCumSumNonNegative(']:
if data_type in ['Decimal256(0)']:
exitcode = 44
else:
exitcode = 43
with Scenario(f"Inline - {data_type} - {func}"):
node.query(f"SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))",
exitcode = exitcode, message = 'Exception:')
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))",
exitcode = exitcode, message = 'Exception:')
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in ['arrayElement']:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), 1)
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), 1)")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in ['arrayPushBack',
'arrayPushFront']:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), {to_data_type(data_type,1)})
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = f'Array({data_type})')
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), {to_data_type(data_type,1)})")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in ['arrayResize',
'arraySlice']:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), 1)
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = f'Array({data_type})')
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), 1)")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in ['has',
'indexOf',
'countEqual']:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), NULL)
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), NULL)")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestOutline(Suite)
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Tuple("1.0"),
)
def tuple_func(self, data_type, node=None):
"""Check tuple functions with extended precision data types.
"""
if node is None:
node = self.context.node
with Scenario(f"Creating a tuple with {data_type}"):
node.query(f"SELECT tuple({to_data_type(data_type,1)}, {to_data_type(data_type,1)}, {to_data_type(data_type,1)})")
with Scenario(f"Creating a tuple with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = f'Tuple({data_type}, {data_type}, {data_type})')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT tuple({to_data_type(data_type,1)}, {to_data_type(data_type,1)}, {to_data_type(data_type,1)})")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"tupleElement with {data_type}"):
node.query(f"SELECT tupleElement(({to_data_type(data_type,1)}, {to_data_type(data_type,1)}), 1)")
with Scenario(f"tupleElement with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT tupleElement(({to_data_type(data_type,1)}, {to_data_type(data_type,1)}), 1)")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"untuple with {data_type}"):
node.query(f"SELECT untuple(({to_data_type(data_type,1)},))")
with Scenario(f"untuple with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT untuple(({to_data_type(data_type,1)},))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"tupleHammingDistance with {data_type}"):
node.query(f"SELECT tupleHammingDistance(({to_data_type(data_type,1)}, {to_data_type(data_type,1)}), ({to_data_type(data_type,2)}, {to_data_type(data_type,2)}))")
with Scenario(f"tupleHammingDistance with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT tupleHammingDistance(({to_data_type(data_type,1)}, {to_data_type(data_type,1)}), ({to_data_type(data_type,2)}, {to_data_type(data_type,2)}))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestOutline(Suite)
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Map_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Map_NotSupported("1.0"),
)
def map_func(self, data_type, node=None):
"""Check Map functions with extended precision data types.
"""
if node is None:
node = self.context.node
with Scenario(f"Creating a map with {data_type}"):
node.query(f"SELECT map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)})")
with Scenario(f"Creating a map with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = f'Map(String, {data_type})')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)})")
execute_query(f"""
SELECT * FROM {table_name}
""")
with Scenario(f"mapAdd with {data_type}"):
node.query(f"SELECT mapAdd(([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]), ([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]))",
exitcode = 44, message='Exception:')
with Scenario(f"mapAdd with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = f'Tuple(Array({data_type}), Array({data_type}))')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT mapAdd(([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]), ([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]))",
exitcode = 44, message='Exception:')
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"mapSubtract with {data_type}"):
node.query(f"SELECT mapSubtract(([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]), ([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]))",
exitcode = 44, message='Exception:')
with Scenario(f"mapSubtract with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = f'Tuple(Array({data_type}), Array({data_type}))')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT mapSubtract(([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]), ([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]))",
exitcode = 44, message='Exception:')
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"mapPopulateSeries with {data_type}"):
node.query(f"SELECT mapPopulateSeries([1,2,3], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)",
exitcode = 44, message='Exception:')
with Scenario(f"mapPopulateSeries with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = f'Tuple(Array({data_type}), Array({data_type}))')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT mapPopulateSeries([1,2,3], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)",
exitcode = 44, message='Exception:')
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"mapContains with {data_type}"):
node.query(f"SELECT mapContains( map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)}), 'key1')")
with Scenario(f"mapContains with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT mapContains( map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)}), 'key1')")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"mapKeys with {data_type}"):
node.query(f"SELECT mapKeys( map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)}))")
with Scenario(f"mapKeys with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = 'Array(String)')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT mapKeys( map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)}))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"mapValues with {data_type}"):
node.query(f"SELECT mapValues( map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)}))")
with Scenario(f"mapValues with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = f'Array({data_type})')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT mapValues( map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)}))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestFeature
@Name("array, tuple, map")
@Examples("data_type",[
('Int128',),
('Int256',),
('UInt128',),
('UInt256',),
('Decimal256(0)',),
])
def feature(self, node="clickhouse1", stress=None, parallel=None):
"""Check that array, tuple, and map functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
with allow_experimental_bigint(self.context.node):
for example in self.examples:
data_type, = example
with Feature(data_type):
Suite(test=array_func)(data_type=data_type)
Suite(test=tuple_func)(data_type=data_type)
with Given("I allow experimental map type"):
allow_experimental_map_type()
Suite(test=map_func)(data_type=data_type)

View File

@ -0,0 +1,179 @@
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
from extended_precision_data_types.errors import *
funcs = [
('bitAnd', True, None),
('bitOr', True, None),
('bitXor', True, None),
('bitShiftLeft', True, None),
('bitShiftRight', True, None),
('bitRotateLeft', False, not_implemented_bigints('Bit rotate')),
('bitRotateRight', False, not_implemented_bigints('Bit rotate')),
('bitTest', False, not_implemented_bigints('bitTest')),
('bitTestAll', False, illegal_column()),
('bitTestAny', False, illegal_column()),
('bitNot', True, None),
('bitCount', True, None)
]
Examples_list = [tuple(list(func)+list(data_type)+[Name(f'{func[0]} - {data_type[0]}')]) for func in funcs for data_type in data_types]
Examples_dec_list = [tuple(list(func)+[Name(f'{func[0]} - Decimal256')]) for func in funcs]
@TestOutline(Scenario)
@Examples('func supported error int_type min max', Examples_list)
def bit_int_inline(self, func, supported, error, int_type, min, max, node=None):
""" Check bit functions with Int128, UInt128, Int256, and UInt256 using inline tests.
"""
if error is not None:
exitcode,message = error
if node is None:
node = self.context.node
if func in ["bitNot", "bitCount"]:
with When(f"Check {func} with {int_type}"):
execute_query(f"""
SELECT {func}(to{int_type}(1)), {func}(to{int_type}(\'{max}\')), {func}(to{int_type}(\'{min}\'))
""")
elif supported:
with When(f"I check {func} with {int_type}"):
execute_query(f"""
SELECT {func}(to{int_type}(1), 1), {func}(to{int_type}(\'{max}\'), 1), {func}(to{int_type}(\'{min}\'), 1)
""")
else:
with When(f"I check {func} with {int_type}"):
node.query(f"SELECT {func}(to{int_type}(1), 1), {func}(to{int_type}(\'{max}\'), 1), {func}(to{int_type}(\'{min}\'), 1)",
exitcode=exitcode, message = message)
@TestOutline(Scenario)
@Examples('func supported error int_type min max', Examples_list)
def bit_int_table(self, func, supported, error, int_type, min, max, node=None):
""" Check bit functions with Int128, UInt128, Int256, and UInt256 using table tests.
"""
table_name = f"table_{getuid()}"
if node is None:
node = self.context.node
if error is not None:
exitcode,message = error
with Given(f"I have a table"):
table(name = table_name, data_type = int_type)
if func in ["bitNot", "bitCount"]:
for value in [1, min, max]:
with When(f"I insert the output of {func} with {int_type} and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(to{int_type}(\'{value}\'))")
with Then(f"I check the table with values of {func} and {int_type}"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
elif supported:
for value in [1, min, max]:
with When(f"I insert the output of {func} with {int_type} and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(to{int_type}(\'{value}\'), 1)")
with Then(f"I check the table with values of {func} and {int_type}"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
else:
for value in [1, min, max]:
with When(f"I insert the output of {func} with {int_type} and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(to{int_type}(\'{value}\'), 1)",
exitcode=exitcode, message=message)
@TestOutline(Scenario)
@Examples('func supported error', Examples_dec_list)
def bit_dec_inline(self, func, supported, error, node=None):
""" Check bit functions with Decimal256 using inline tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
exitcode, message = illegal_type()
if node is None:
node = self.context.node
if func in ["bitNot", "bitCount"]:
with When(f"Check {func} with Decimal256"):
node.query(f"SELECT {func}(toDecimal256(1,0)), {func}(toDecimal256(\'{max}\',0)), {func}(toDecimal256(\'{min}\',0))",
exitcode=exitcode, message = message)
else:
with When(f"I check {func} with Decimal256"):
node.query(f"SELECT {func}(toDecimal256(1,0), 1), {func}(toDecimal256(\'{max}\',0), 1), {func}(toDecimal256(\'{min}\',0), 1)",
exitcode=exitcode, message = message)
@TestOutline(Scenario)
@Examples('func supported error', Examples_dec_list)
def bit_dec_table(self, func, supported, error, node=None):
""" Check bit functions with Decimal256 using table tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
table_name = f"table_{getuid()}"
exitcode, message = illegal_type()
if node is None:
node = self.context.node
with Given(f"I have a table"):
table(name = table_name, data_type = 'Decimal256(0)')
if func in ["bitNot", "bitCount"]:
for value in [1, min, max]:
with When(f"I insert the output of {func} with Decimal256 and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(toDecimal256(\'{value}\',0))",
exitcode=exitcode, message = message)
else:
for value in [1, min, max]:
with When(f"I insert the output of {func} with Decimal256 and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(toDecimal256(\'{value}\',0), 1)",
exitcode=exitcode, message=message)
@TestFeature
@Name("bit")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Bit_Int_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Bit_Int_NotSupported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Bit_Dec_NotSupported("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that bit functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
with allow_experimental_bigint(self.context.node):
Scenario(run=bit_int_inline)
Scenario(run=bit_int_table)
Scenario(run=bit_dec_inline)
Scenario(run=bit_dec_table)

View File

@ -0,0 +1,110 @@
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
funcs = [
('equals',),
('notEquals',),
('less',),
('greater',),
('lessOrEquals',),
('greaterOrEquals',)
]
Examples_list = [tuple(list(func)+list(data_type)+[Name(f'{func[0]} - {data_type[0]}')]) for func in funcs for data_type in data_types]
@TestOutline(Scenario)
@Examples('func int_type min max', Examples_list)
def comp_int_inline(self, func, int_type, min, max, node=None):
"""Check comparison functions with Int128, UInt128, Int256, and UInt256 using inline tests.
"""
if node is None:
node = self.context.node
with When(f"I check {func} with {int_type}"):
execute_query(f"""
SELECT {func}(to{int_type}(1), to{int_type}(1)), {func}(to{int_type}(\'{max}\'), to{int_type}(\'{min}\'))
""")
@TestOutline(Scenario)
@Examples('func int_type min max', Examples_list)
def comp_int_table(self, func, int_type, min, max, node=None):
"""Check comparison functions with Int128, UInt128, Int256, and UInt256 using table tests.
"""
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = int_type)
for value in [1, max, min]:
with When(f"I insert into a table the output {func} with {int_type} and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(to{int_type}(\'{value}\'), to{int_type}(1))")
with Then(f"I check the table for the output of {func} with {int_type}"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestOutline(Scenario)
@Examples('func', funcs)
def comp_dec_inline(self, func, node=None):
"""Check comparison functions with Decimal256 using inline tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
with When(f"I check {func} with Decimal256"):
execute_query(f"""
SELECT {func}(toDecimal256(1,0), toDecimal256(1,0)), {func}(toDecimal256(\'{max}\',0), toDecimal256(\'{min}\',0))
""")
@TestOutline(Scenario)
@Examples('func', funcs)
def comp_dec_table(self, func, node=None):
"""Check comparison functions with Decimal256 using table tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = 'Decimal256(0)')
for value in [1, max, min]:
with When(f"I insert into a table the output {func} with Decimal256 and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(toDecimal256(\'{value}\',0), toDecimal256(1,0))")
with Then(f"I check the table for the output of {func} with Decimal256"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestFeature
@Name("comparison")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Comparison("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that comparison functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
with allow_experimental_bigint(self.context.node):
Scenario(run=comp_int_inline)
Scenario(run=comp_int_table)
Scenario(run=comp_dec_inline)
Scenario(run=comp_dec_table)

View File

@ -0,0 +1,275 @@
import os
import textwrap
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
@contextmanager
def dictionary(name, node, mysql_node):
"""Create a table in MySQL and use it a source for a dictionary.
"""
try:
with Given("table in MySQL"):
sql = f"""
CREATE TABLE {name}(
id INT NOT NULL AUTO_INCREMENT,
int128 BIGINT,
uint128 BIGINT,
int256 BIGINT,
uint256 BIGINT,
dec256 DECIMAL,
PRIMARY KEY ( id )
);
"""
with When("I drop the table if exists"):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user -e \"DROP TABLE IF EXISTS {name};\"", exitcode=0)
with And("I create a table"):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user <<'EOF'{textwrap.dedent(sql)}\nEOF", exitcode=0)
with And("dictionary that uses MySQL table as the external source"):
with When("I drop the dictionary if exists"):
node.query(f"DROP DICTIONARY IF EXISTS dict_{name}")
with And("I create the dictionary"):
sql = f"""
CREATE DICTIONARY dict_{name}
(
id UInt8,
int128 Int128,
uint128 UInt128,
int256 Int256,
uint256 UInt256,
dec256 Decimal256(0)
)
PRIMARY KEY id
SOURCE(MYSQL(
USER 'user'
PASSWORD 'password'
DB 'db'
TABLE '{name}'
REPLICA(PRIORITY 1 HOST '{mysql_node.name}' PORT 3306)
))
LAYOUT(HASHED())
LIFETIME(0)
"""
node.query(textwrap.dedent(sql))
yield f"dict_{name}"
finally:
with Finally("I drop the dictionary", flags=TE):
node.query(f"DROP DICTIONARY IF EXISTS dict_{name}")
with And("I drop a table in MySQL", flags=TE):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user -e \"DROP TABLE IF EXISTS {name};\"", exitcode=0)
@contextmanager
def table(name, node, mysql_node):
"""Create a table in MySQL and use it a source for a table in ClickHouse.
"""
try:
with Given("table in MySQL"):
sql = f"""
CREATE TABLE {name}(
id INT NOT NULL AUTO_INCREMENT,
int128 BIGINT,
uint128 BIGINT,
int256 BIGINT,
uint256 BIGINT,
dec256 DECIMAL,
PRIMARY KEY ( id )
);
"""
with When("I drop the table if exists"):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user -e \"DROP TABLE IF EXISTS {name};\"", exitcode=0)
with And("I create a table"):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user <<'EOF'{textwrap.dedent(sql)}\nEOF", exitcode=0)
with And("table that uses MySQL table as the external source"):
with When("I drop the table if exists"):
node.query(f"DROP TABLE IF EXISTS {name}")
with And("I create the table"):
sql = f"""
CREATE TABLE {name}
(
id UInt8,
int128 Int128,
uint128 UInt128,
int256 Int256,
uint256 UInt256,
dec256 Decimal256(0)
)
ENGINE = MySQL('{mysql_node.name}:3306', 'default', '{name}', 'default', 'password')
"""
node.query(textwrap.dedent(sql))
yield f"table_{name}"
finally:
with Finally("I drop the table", flags=TE):
node.query(f"DROP TABLE IF EXISTS {name}")
with And("I drop a table in MySQL", flags=TE):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user -e \"DROP TABLE IF EXISTS {name};\"", exitcode=0)
@contextmanager
def table_func(name, node, mysql_node):
"""Create a table in MySQL and use it a source for a table using mysql table function.
"""
try:
with Given("table in MySQL"):
sql = f"""
CREATE TABLE {name}(
id INT NOT NULL AUTO_INCREMENT,
int128 BIGINT,
uint128 BIGINT,
int256 BIGINT,
uint256 BIGINT,
dec256 DECIMAL,
PRIMARY KEY ( id )
);
"""
with When("I drop the table if exists"):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user -e \"DROP TABLE IF EXISTS {name};\"", exitcode=0)
with And("I create a table"):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user <<'EOF'{textwrap.dedent(sql)}\nEOF", exitcode=0)
yield f"mysql('{mysql_node.name}:3306', 'db', '{name}', 'user', 'password')"
finally:
with Finally("I drop the table", flags=TE):
node.query(f"DROP TABLE IF EXISTS {name}")
with And("I drop a table in MySQL", flags=TE):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user -e \"DROP TABLE IF EXISTS {name};\"", exitcode=0)
@TestOutline(Scenario)
@Examples('int_type min max',[
('Int128', '-170141183460469231731687303715884105728', '170141183460469231731687303715884105727', Requirements(RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_toInt128("1.0")), Name('Int128')),
('Int256', '-57896044618658097711785492504343953926634992332820282019728792003956564819968', '57896044618658097711785492504343953926634992332820282019728792003956564819967', Requirements(RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_toInt256("1.0")), Name('Int256')),
('UInt128','0','340282366920938463463374607431768211455', Requirements(RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_toUInt128("1.0")), Name('UInt128')),
('UInt256', '0', '115792089237316195423570985008687907853269984665640564039457584007913129639935', Requirements(RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_toUInt256("1.0")), Name('UInt256')),
])
def int_conversion(self, int_type, min, max, node=None):
"""Check that ClickHouse converts values to Int128.
"""
if node is None:
node = self.context.node
with When(f"I convert {min}, {max}, 1 to {int_type}"):
output = node.query(f"SELECT to{int_type}(\'{min}\'), to{int_type}(\'{max}\'), to{int_type}(1) format TabSeparatedRaw").output
assert output == f'{min}\t{max}\t1', error()
@TestScenario
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_toDecimal256("1.0"),
)
def to_decimal256(self, node=None):
"""Check that ClickHouse converts values to Int128.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
for value in [1,min,max]:
output = node.query(f"SELECT toDecimal256(\'{value}\',0)").output
assert output == str(value), error()
@TestScenario
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_ToMySQL("1.0"),
)
def MySQL_table(self, node=None):
"""Check that ClickHouse converts MySQL values from MySQL table into ClickHouse table.
"""
table_name = f'table_{getuid()}'
node = self.context.node
mysql_node = self.context.mysql_node
with table(table_name, node, mysql_node):
with When("I insert parameters values in MySQL"):
sql = f"""
INSERT INTO {table_name}(int128, uint128, int256, uint256, dec256) VALUES (1,1,1,1,1);
"""
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user <<'EOF'{textwrap.dedent(sql)}\nEOF", exitcode=0)
with Then("I select from the table on top of the mysql table"):
node.query(f"SELECT * FROM {table_name}",
exitcode=50, message='Exception:')
@TestScenario
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_FromMySQL("1.0"),
)
def MySQL_func(self, node=None):
"""Check that ClickHouse converts MySQL values into a ClickHouse table using the MySQL table function.
"""
table_name = f'table_{getuid()}'
node = self.context.node
mysql_node = self.context.mysql_node
with table_func(table_name, node, mysql_node) as table_function:
with When("I insert parameters values in MySQL"):
sql = f"""
INSERT INTO {table_name}(int128, uint128, int256, uint256, dec256) VALUES (1,1,1,1,1);
"""
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user <<'EOF'{textwrap.dedent(sql)}\nEOF", exitcode=0)
with And("I make sure the table doesn't exist"):
node.query(f"DROP TABLE IF EXISTS {table_name}")
with And("I create the table"):
node.query(f"CREATE TABLE {table_name} (id UInt8, int128 Int128, uint128 UInt128, int256 Int256, uint256 UInt256, dec256 Decimal256(0)) Engine = Memory")
with And("I insert into the clickhouse table from the mysql table"):
node.query(f"INSERT INTO {table_name} SELECT * FROM {table_function}")
with Then("I select from the clickhouse table"):
output = node.query(f"SELECT * FROM {table_name}").output
assert output == '1\t1\t1\t1\t1\t1', error()
@TestScenario
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_ToMySQL("1.0"),
)
def MySQL_dict(self, node=None):
"""Check that ClickHouse converts MySQL values from MySQL table into ClickHouse dictionary.
"""
node = self.context.node
mysql_node = self.context.mysql_node
table_name = f'table_{getuid()}'
with dictionary(table_name, node, mysql_node):
with When("I insert parameters values in MySQL"):
sql = f"""
INSERT INTO {table_name}(int128, uint128, int256, uint256, dec256) VALUES (1,1,1,1,1);
"""
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user <<'EOF'{textwrap.dedent(sql)}\nEOF", exitcode=0)
with Then("I select from the table on top of the mysql table"):
node.query(f"SELECT * FROM dict_{table_name}",
exitcode=50, message='Exception:')
@TestFeature
@Name("conversion")
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check the conversion of extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
for scenario in loads(current_module(), Scenario):
with allow_experimental_bigint(self.context.node):
Scenario(run=scenario)

View File

@ -0,0 +1,19 @@
from testflows.core import *
from testflows.core.name import basename, parentname
from testflows._core.testtype import TestSubType
@TestFeature
@Name("tests")
def feature(self):
"""Check functions with Int128, Int256, UInt256, and Decimal256.
"""
Feature(run=load("extended_precision_data_types.tests.conversion", "feature"))
Feature(run=load("extended_precision_data_types.tests.arithmetic", "feature"))
Feature(run=load("extended_precision_data_types.tests.array_tuple_map", "feature"))
Feature(run=load("extended_precision_data_types.tests.comparison", "feature"))
Feature(run=load("extended_precision_data_types.tests.logical", "feature"))
Feature(run=load("extended_precision_data_types.tests.mathematical", "feature"))
Feature(run=load("extended_precision_data_types.tests.rounding", "feature"))
Feature(run=load("extended_precision_data_types.tests.bit", "feature"))
Feature(run=load("extended_precision_data_types.tests.null", "feature"))
Feature(run=load("extended_precision_data_types.tests.table", "feature"))

View File

@ -0,0 +1,99 @@
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
funcs = [
('and',),
('or',),
('not',),
('xor',),
]
Examples_list = [tuple(list(func)+list(data_type)+[Name(f'{func[0]} - {data_type[0]}')]) for func in funcs for data_type in data_types]
@TestOutline(Scenario)
@Examples('func int_type min max', Examples_list)
def log_int_inline(self, func, int_type, min, max, node=None):
"""Check logical functions with Int128, Int256, and UInt256 using inline tests.
"""
table_name = f'table_{getuid()}'
if node is None:
node = self.context.node
with When(f"Check {func} with {int_type}"):
node.query(f"SELECT {func}(to{int_type}(1), to{int_type}(1)), {func}(to{int_type}(\'{max}\'), to{int_type}(1)), {func}(to{int_type}(\'{min}\'), to{int_type}(1))",
exitcode=43, message = 'Exception: Illegal type ')
@TestOutline(Scenario)
@Examples('func int_type min max', Examples_list)
def log_int_table(self, func, int_type, min, max, node=None):
"""Check logical functions with Int128, Int256, and UInt256 using table tests.
"""
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = int_type)
for value in [1, min, max]:
with When(f"Check {func} with {int_type} and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(to{int_type}(\'{value}\'), to{int_type}(\'{value}\'))",
exitcode=43, message = 'Exception: Illegal type')
@TestOutline(Scenario)
@Examples('func', funcs)
def log_dec_inline(self, func, node=None):
"""Check logical functions with Decimal256 using inline tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
with When(f"Check {func} with Decimal256"):
node.query(f"SELECT {func}(toDecimal256(1,0), toDecimal256(1,0)), {func}(toDecimal256(\'{max}\',0), toDecimal256(1)), {func}(toDecimal256(\'{min}\',0), toDecimal256(1))",
exitcode=43, message = 'Exception: Illegal type ')
@TestOutline(Scenario)
@Examples('func', funcs)
def log_dec_table(self, func, node=None):
"""Check logical functions with Decimal256 using table tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = 'Decimal256(0)')
for value in [1, min, max]:
with When(f"Check {func} with Decimal256 and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(toDecimal256(\'{value}\',0), toDecimal256(\'{value}\',0))",
exitcode=43, message = 'Exception: Illegal type ')
@TestFeature
@Name("logical")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Logical("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that comparison functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
with allow_experimental_bigint(self.context.node):
Scenario(run=log_int_inline)
Scenario(run=log_int_table)
Scenario(run=log_dec_inline)
Scenario(run=log_dec_table)

View File

@ -0,0 +1,187 @@
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
funcs = [
('exp(', 3, 0),
('log(', 0, 0),
('ln(', 0, 0),
('exp2(', 2, 0),
('log2(', 0, 0),
('exp10(', 10, 0),
('log10(', 0, 0),
('sqrt(', 1, 0),
('cbrt(', 1, 0),
('erf(', 1, 0),
('erfc(', 0, 0),
('lgamma(', 0, 0),
('tgamma(', 1, 0),
('sin(', 1, 0),
('cos(', 1, 0),
('tan(', 2, 0),
('asin(', 2, 0),
('acos(', 0, 0),
('atan(', 1, 0),
('intExp2(', 2, 48),
('intExp10(', 10, 48),
('cosh(', 2, 0),
('acosh(', 0, 0),
('sinh(', 1, 0),
('asinh(', 1, 0),
('tanh(', 1, 0),
('atanh(', 'inf', 0),
('log1p(', 1, 0),
('sign(', 1, 0),
('pow(1,', 1, 43),
('power(1,', 1, 43),
('atan2(1,', 1, 43),
('hypot(1,', 1, 43),
]
Examples_list = [tuple(list(func)+list(data_type)+[Name(f'{func[0]} - {data_type[0]}')]) for func in funcs for data_type in data_types]
Examples_dec_list = [tuple(list(func)+[Name(f'{func[0]} - Decimal256')]) for func in funcs]
@TestOutline(Scenario)
@Examples('func expected_result exitcode int_type min max', Examples_list)
def math_int_inline(self, func, expected_result, exitcode, int_type, min, max, node=None):
"""Check mathematical functions with Int128, UInt128, Int256, and UInt256 using inline tests.
"""
if node is None:
node = self.context.node
if func in ['intExp2(', 'intExp10(', 'pow(1,', 'power(1,', 'atan2(1,', 'hypot(1,']:
with When(f"I check {func} with {int_type} using 1, max, and min"):
node.query(f"SELECT {func} to{int_type}(1)), {func} to{int_type}(\'{max}\')), {func} to{int_type}(\'{min}\'))",
exitcode=exitcode, message = 'Exception:')
else:
with When(f"I check {func} with {int_type} using 1"):
output = node.query(f"SELECT {func} to{int_type}(1))").output
if output == 'inf':
pass
else:
assert round(float(output)) == expected_result, error()
with And(f"I check {func} with {int_type} using max and min"):
execute_query(f"""
SELECT {func} to{int_type}(\'{max}\')), {func} to{int_type}(\'{min}\'))
""")
@TestOutline(Scenario)
@Examples('func expected_result exitcode int_type min max', Examples_list)
def math_int_table(self, func, expected_result, exitcode, int_type, min, max, node=None):
"""Check mathematical functions with Int128, UInt128, Int256, and UInt256 using table tests.
"""
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = f'Nullable({int_type})')
if func in ['intExp2(', 'intExp10(', 'pow(1,', 'power(1,', 'atan2(1,', 'hypot(1,']:
for value in [1, max, min]:
with When(f"I insert the output of {func} with {int_type} using {value} into a table"):
node.query(f"INSERT INTO {table_name} SELECT {func} to{int_type}(\'{value}\'))",
exitcode=exitcode, message = 'Exception:')
else:
for value in [1, max, min]:
with And(f"I insert the output of {func} with {int_type} using {value} into a table"):
node.query(f"INSERT INTO {table_name} SELECT to{int_type}OrZero( toString({func} to{int_type}(\'{value}\'))))")
with Then(f"I check the outputs of {func} with {int_type}"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestOutline(Scenario)
@Examples('func expected_result exitcode', Examples_dec_list)
def math_dec_inline(self, func, expected_result, exitcode, node=None):
"""Check mathematical functions with Decimal256 using inline tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
if func in ['intExp2(', 'intExp10(', 'pow(1,', 'power(1,', 'atan2(1,', 'hypot(1,']:
with When(f"I check {func} with Decimal256 using 1, max, and min"):
node.query(f"SELECT {func} toDecimal256(1,0)), {func} toDecimal256(\'{max}\',0)), {func} toDecimal256(\'{min}\',0))",
exitcode=43, message = 'Exception: Illegal type ')
else:
with When(f"I check {func} with Decimal256 using 1"):
output = node.query(f"SELECT {func} toDecimal256(1,0))").output
if output == 'inf':
pass
else:
assert round(float(output)) == expected_result, error()
with And(f"I check {func} with Decimal256 using max and min"):
execute_query(f"""
SELECT {func} toDecimal256(\'{max}\',0)), {func} toDecimal256(\'{min}\',0))
""")
@TestOutline(Scenario)
@Examples('func expected_result exitcode', Examples_dec_list)
def math_dec_table(self, func, expected_result, exitcode, node=None):
"""Check mathematical functions with Decimal256 using table tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = 'Decimal256(0)')
if func in ['intExp2(', 'intExp10(', 'pow(1,', 'power(1,', 'atan2(1,', 'hypot(1,']:
for value in [1, max, min]:
with When(f"I insert the output of {func} with Decimal256 using {value} into a table"):
node.query(f"INSERT INTO {table_name} SELECT {func} toDecimal256(\'{value}\',0))",
exitcode=43, message = 'Exception: Illegal type ')
else:
for value in [1, max, min]:
with When(f"I insert the output of {func} with Decimal256 using {value} into a table"):
node.query(f"INSERT INTO {table_name} SELECT toDecimal256OrZero( toString({func} toDecimal256(\'{value}\',0))),0)")
with Then(f"I check the outputs of {func} with Decimal256"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestFeature
@Name("mathematical")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Mathematical_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Mathematical_NotSupported("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that mathematical functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
with allow_experimental_bigint(self.context.node):
Scenario(run=math_int_inline)
Scenario(run=math_int_table)
Scenario(run=math_dec_inline)
Scenario(run=math_dec_table)

View File

@ -0,0 +1,119 @@
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
funcs = [
('isNull(', 0),
('isNotNull(', 1),
('coalesce(', 1),
('assumeNotNull(', 1),
('toNullable(', 1),
('ifNull(1,', 1),
('nullIf(1,', '\\N'),
]
Examples_list = [tuple(list(func)+list(data_type)+[Name(f'{func[0]} - {data_type[0]}')]) for func in funcs for data_type in data_types]
@TestOutline(Scenario)
@Examples('func expected_result int_type min max', Examples_list)
def null_int_inline(self, func, expected_result, int_type, min, max, node=None):
"""Check null function with Int128, UInt128, Int256, and UInt256 using inline tests.
"""
if node is None:
node = self.context.node
with When(f"I check {func} with {int_type}"):
output = node.query(f"SELECT {func} to{int_type}(1))").output
assert output == str(expected_result), error()
with And(f"I check {func} with {int_type} using min and max"):
execute_query(f"""
SELECT {func} to{int_type}(\'{min}\')), {func} to{int_type}(\'{max}\'))
""")
@TestOutline(Scenario)
@Examples('func expected_result int_type min max', Examples_list)
def null_int_table(self, func, expected_result, int_type, min, max, node=None):
"""Check null function with Int128, UInt128, Int256, and UInt256 using table tests.
"""
table_name = f"table_{getuid()}"
if node is None:
node = self.context.node
with Given("I have a table"):
table(name = table_name, data_type = f'Nullable({int_type})')
for value in [1, min, max]:
with When(f"I insert the output of {func} with {int_type} and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func} to{int_type}(\'{value}\'))")
with Then(f"I check {func} with {int_type} on the table"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestOutline(Scenario)
@Examples('func expected_result', funcs)
def null_dec_inline(self, func, expected_result, node=None):
"""Check null function with Decimal256 using inline tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
with When(f"I check {func} with Decimal256"):
output = node.query(f"SELECT {func} toDecimal256(1,0))").output
assert output == str(expected_result), error()
with And(f"I check {func} with Decimal256 using min and max"):
execute_query(f"""
SELECT {func} toDecimal256(\'{min}\',0)), {func} toDecimal256(\'{max}\',0))
""")
@TestOutline(Scenario)
@Examples('func expected_result', funcs)
def null_dec_table(self, func, expected_result, node=None):
"""Check null function with Decimal256 using table tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
table_name = f"table_{getuid()}"
if node is None:
node = self.context.node
with Given("I have a table"):
table(name = table_name, data_type = 'Nullable(Decimal256(0))')
for value in [1, min, max]:
with When(f"I insert the output of {func} with Decimal256 and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func} toDecimal256(\'{value}\',0))")
with Then(f"I check {func} with Decimal256 on the table"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestFeature
@Name("null")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Null("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that null functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
with allow_experimental_bigint(self.context.node):
Scenario(run=null_int_inline)
Scenario(run=null_int_table)
Scenario(run=null_dec_inline)
Scenario(run=null_dec_table)

View File

@ -0,0 +1,191 @@
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
funcs = [
('ceil', 1, True),
('floor', 1, True),
('trunc', 1, True),
('round', 1, True),
('roundBankers', 1, True),
('roundToExp2', 1, False),
('roundDuration', 1, True),
('roundAge', 17, True),
('roundDown', 1, False)
]
Examples_list = [tuple(list(func)+list(data_type)+[Name(f'{func[0]} - {data_type[0]}')]) for func in funcs for data_type in data_types]
Examples_dec_list = [tuple(list(func)+[Name(f'{func[0]} - Decimal256')]) for func in funcs]
@TestOutline(Scenario)
@Examples('func expected_result supported int_type min max', Examples_list)
def round_int_inline(self, func, expected_result, supported, int_type, min, max, node=None):
"""Check rounding functions with Int128, UInt128, Int256, and UInt256 using inline tests.
"""
if node is None:
node = self.context.node
if func is 'roundDown':
with When(f"I check roundDown with {int_type}"):
node.query(f"SELECT roundDown(to{int_type}(1), [0,2]), roundDown(to{int_type}(\'{max}\'), [0,2]), roundDown(to{int_type}(\'{min}\'), [0,2])",
exitcode=44, message=f'Exception: Illegal column {int_type} of first argument of function roundDown')
elif supported:
with When(f"I check {func} with {int_type}"):
output = node.query(f"SELECT {func}(to{int_type}(1))").output
assert output == str(expected_result), error()
with And(f'I check {func} with {int_type} using min and max values'):
execute_query(f"""
SELECT {func}(to{int_type}(\'{min}\')), {func}(to{int_type}(\'{max}\'))
""")
else:
with When(f"I check {func} with {int_type}"):
node.query(f"SELECT {func}(to{int_type}(1)), {func}(to{int_type}(\'{max}\')), {func}(to{int_type}(\'{min}\'))",
exitcode=48, message=f'Exception: {func}() for big integers is not implemented:')
@TestOutline(Scenario)
@Examples('func expected_result supported int_type min max', Examples_list)
def round_int_table(self, func, expected_result, supported, int_type, min, max, node=None):
"""Check rounding functions with Int128, UInt128, Int256, and UInt256 using table tests.
"""
table_name = f"table_{getuid()}"
if node is None:
node = self.context.node
with Given("I have a table"):
table(name = table_name, data_type = int_type)
if func is 'roundDown':
for value in [1,max,min]:
with When(f"I check roundDown with {int_type} and {value}"):
node.query(f"INSERT INTO {table_name} SELECT roundDown(to{int_type}(\'{value}\'), [0,2])",
exitcode=44, message=f'Exception: Illegal column {int_type} of first argument of function roundDown')
elif supported:
for value in [1,max,min]:
with When(f"I insert the output of {func} with {int_type} and {value} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(to{int_type}(\'{value}\'))")
with Then(f"I select the output of {func} with {int_type} from the table"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
else:
for value in [1,max,min]:
with When(f"I insert the output of {func} with {int_type} and {value} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(to{int_type}(1))",
exitcode=48, message=f'Exception: {func}() for big integers is not implemented:')
@TestOutline(Scenario)
@Examples('func expected_result supported', Examples_dec_list)
def round_dec_inline(self, func, expected_result, supported, node=None):
"""Check rounding functions with Decimal256 using inline tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
if func is 'roundDown':
with When(f"I check roundDown with Decimal256"):
node.query(f"""SELECT roundDown(toDecimal256(1,0), [toDecimal256(0,0),toDecimal256(2,0)]),
roundDown(toDecimal256(\'{max}\',0), [toDecimal256(0,0),toDecimal256(2,0)]),
roundDown(toDecimal256(\'{min}\',0), [toDecimal256(0,0),toDecimal256(2,0)])""",
exitcode=44, message=f'Exception: Illegal column Decimal256 of first argument of function roundDown')
elif func not in ['roundDuration', 'roundAge', 'roundToExp2']:
with When(f"I check {func} with Decimal256"):
output = node.query(f"SELECT {func}(toDecimal256(1,0))").output
assert output == str(expected_result), error()
with And(f'I check {func} with Decimal256 using min and max values'):
execute_query(f"""
SELECT {func}(toDecimal256(\'{min}\',0)), {func}(toDecimal256(\'{max}\',0))
""")
else:
with When(f"I check {func} with Decimal256"):
node.query(f"SELECT {func}(toDecimal256(1,0)), {func}(toDecimal256(\'{max}\',0)), {func}(toDecimal256(\'{min}\',0))",
exitcode=43, message=f'Exception: Illegal type Decimal(76, 0)')
@TestOutline(Scenario)
@Examples('func expected_result supported', Examples_dec_list)
def round_dec_table(self, func, expected_result, supported, node=None):
"""Check rounding functions with Decimal256 using table tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
table_name = f"table_{getuid()}"
if node is None:
node = self.context.node
with Given("I have a table"):
table(name = table_name, data_type = 'Decimal256(0)')
if func is 'roundDown':
for value in [1, max, min]:
with When(f"I check roundDown with Decimal256 and {value}"):
node.query(f"INSERT INTO {table_name} SELECT roundDown(toDecimal256(\'{value}\',0), [toDecimal256(0,0),toDecimal256(2,0)])",
exitcode=44, message=f'Exception: Illegal column Decimal256 of first argument of function roundDown')
elif func not in ['roundDuration', 'roundAge', 'roundToExp2']:
for value in [1, max, min]:
with When(f"I insert the output of {func} with Decimal256 and {value} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(toDecimal256(\'{value}\',0))")
with Then(f"I select the output of {func} with Decimal256 from the table"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
else:
for value in [1, max, min]:
with When(f"I insert the output of {func} with Decimal256 and {value} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(toDecimal256(\'{value}\',0))",
exitcode=43, message=f'Exception: Illegal type Decimal(76, 0)')
@TestFeature
@Name("rounding")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Rounding_Int_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Rounding_Int_NotSupported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Rounding_Dec_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Rounding_Dec_NotSupported("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that rounding functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
with allow_experimental_bigint(self.context.node):
Scenario(run=round_int_inline)
Scenario(run=round_int_table)
Scenario(run=round_dec_inline)
Scenario(run=round_dec_table)

View File

@ -0,0 +1,35 @@
from testflows.core import *
from testflows.asserts import error
from contextlib import contextmanager
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
@TestFeature
@Name("table")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Create_Table("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that clickhouse is able to create a table with extended precision data types.
"""
node = self.context.cluster.node(node)
table_name = f"table_{getuid()}"
with allow_experimental_bigint(node):
try:
with When("I create a table with Int128, UInt128, Int256, UInt256, Decimal256"):
node.query(f"CREATE TABLE {table_name}(a Int128, b UInt128, c Int256, d UInt256, e Decimal256(0)) ENGINE = Memory")
with And("I insert values into the table"):
node.query(f"INSERT INTO {table_name} VALUES (toInt128(1), toUInt128(1), toInt256(1), toUInt256(1), toDecimal256(1,0))")
with Then("I select from the table"):
output = node.query(f"SELECT * FROM {table_name}").output
assert output == '1\t1\t1\t1\t1', error()
finally:
with Finally("I drop the table"):
node.query(f"DROP TABLE IF EXISTS {table_name}")

View File

@ -30,6 +30,7 @@ def regression(self, local, clickhouse_binary_path, stress=None, parallel=None):
run_scenario(pool, tasks, Feature(test=load("window_functions.regression", "regression")), args) run_scenario(pool, tasks, Feature(test=load("window_functions.regression", "regression")), args)
run_scenario(pool, tasks, Feature(test=load("datetime64_extended_range.regression", "regression")), args) run_scenario(pool, tasks, Feature(test=load("datetime64_extended_range.regression", "regression")), args)
#run_scenario(pool, tasks, Feature(test=load("kerberos.regression", "regression")), args) #run_scenario(pool, tasks, Feature(test=load("kerberos.regression", "regression")), args)
run_scenario(pool, tasks, Feature(test=load("extended_precision_data_types.regression", "regression")), args)
finally: finally:
join(tasks) join(tasks)