From f96111b4236c1deb1e4dd4d9070815877a9e21ce Mon Sep 17 00:00:00 2001 From: nicelulu <821008736@qq.com> Date: Sat, 17 Nov 2018 22:03:32 +0800 Subject: [PATCH 001/230] Fixed the way to get username and password. --- .../Storages/StorageReplicatedMergeTree.cpp | 29 +++++++++++++++++-- .../src/Storages/StorageReplicatedMergeTree.h | 3 ++ 2 files changed, 30 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 1d29a6dba8b..4411d3d3554 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -110,6 +110,7 @@ namespace ErrorCodes extern const int KEEPER_EXCEPTION; extern const int ALL_REPLICAS_LOST; extern const int REPLICA_STATUS_CHANGED; + extern const int INCONSISTENT_CLUSTER_DEFINITION; } namespace ActionLocks @@ -3870,13 +3871,13 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query throw Exception("Can't proxy this query. Unsupported query type", ErrorCodes::NOT_IMPLEMENTED); /// Query send with current user credentials - + const Cluster::Address & address = findClusterAddress(leader_address); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context.getSettingsRef()); Connection connection( leader_address.host, leader_address.queries_port, leader_address.database, - context.getClientInfo().current_user, context.getClientInfo().current_password, timeouts, "ClickHouse replica"); + address.user, address.password, timeouts, "ClickHouse replica"); RemoteBlockInputStream stream(connection, formattedAST(new_query), {}, context, &settings); NullBlockOutputStream output({}); @@ -3885,6 +3886,30 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query return; } +const Cluster::Address & StorageReplicatedMergeTree::findClusterAddress(ReplicatedMergeTreeAddress & leader_address) const +{ + const auto & clusters = context.getClusters(); + const auto & clusterPtrs = clusters.getContainer(); + + for(auto iter = clusterPtrs.begin(); iter != clusterPtrs.end(); ++iter) + { + const auto & shards = iter->second->getShardsAddresses(); + + for (size_t shard_num = 0; shard_num < shards.size(); ++shard_num) + { + for (size_t replica_num = 0; replica_num < shards[shard_num].size(); ++replica_num) + { + const Cluster::Address & address = shards[shard_num][replica_num]; + + if (address.host_name == leader_address.host && address.port == leader_address.queries_port) + { + return address; + } + } + } + } + throw Exception("Not found host " + leader_address.host, ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); +} void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_) { diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 491f30d93e5..8afbdbdeb9f 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -24,6 +24,7 @@ #include #include #include +#include namespace DB @@ -512,6 +513,8 @@ private: bool dropPartsInPartition(zkutil::ZooKeeper & zookeeper, String & partition_id, StorageReplicatedMergeTree::LogEntry & entry, bool detach); + const Cluster::Address & findClusterAddress(ReplicatedMergeTreeAddress & leader_address) const; + protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. */ From 1327e05f82b633a87eff5f18fbd9c189845aa815 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=B0=8F=E8=B7=AF?= <821008736@qq.com> Date: Tue, 20 Nov 2018 16:58:01 +0800 Subject: [PATCH 002/230] Update StorageReplicatedMergeTree.h --- dbms/src/Storages/StorageReplicatedMergeTree.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index f6fe6d59548..ed081054f8d 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -514,8 +514,6 @@ private: const Cluster::Address & findClusterAddress(ReplicatedMergeTreeAddress & leader_address); - const Cluster::Address & findClusterAddress(ReplicatedMergeTreeAddress & leader_address) const; - protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. */ From 09cd9f5f024666bc2357509debd175ef2b10e9f8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=B0=8F=E8=B7=AF?= <821008736@qq.com> Date: Tue, 20 Nov 2018 17:29:57 +0800 Subject: [PATCH 003/230] Update StorageReplicatedMergeTree.h --- dbms/src/Storages/StorageReplicatedMergeTree.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index ed081054f8d..a92d072f7ce 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -511,7 +511,6 @@ private: bool dropPartsInPartition(zkutil::ZooKeeper & zookeeper, String & partition_id, StorageReplicatedMergeTree::LogEntry & entry, bool detach); - const Cluster::Address & findClusterAddress(ReplicatedMergeTreeAddress & leader_address); protected: From f69524cf49bab977fb14e12cbee1484a332e011e Mon Sep 17 00:00:00 2001 From: root Date: Tue, 20 Nov 2018 18:50:42 +0800 Subject: [PATCH 004/230] better --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index e0d0590e0cf..51ef376c6fa 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3906,7 +3906,7 @@ const Cluster::Address & StorageReplicatedMergeTree::findClusterAddress(Replicat } } } - throw Exception("Not found replicate leader host " + leader_address.host + ":" + leader_address.queries_port, ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); + throw Exception("Not found replicate leader host " + leader_address.host + ":" + DB::toString(leader_address.queries_port) + ", Please check your metrika.xml", ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); } void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_) From 811c7e9aef2891cab084377dab5eec51c131a36e Mon Sep 17 00:00:00 2001 From: root Date: Tue, 20 Nov 2018 18:55:40 +0800 Subject: [PATCH 005/230] better --- dbms/src/Storages/StorageReplicatedMergeTree.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index ed081054f8d..a551569f004 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -511,7 +511,7 @@ private: bool dropPartsInPartition(zkutil::ZooKeeper & zookeeper, String & partition_id, StorageReplicatedMergeTree::LogEntry & entry, bool detach); - + const Cluster::Address & findClusterAddress(ReplicatedMergeTreeAddress & leader_address); protected: From 3df876d54132f2dd98e8030a9abc54fb19990f90 Mon Sep 17 00:00:00 2001 From: root Date: Tue, 20 Nov 2018 19:02:45 +0800 Subject: [PATCH 006/230] better --- dbms/src/Storages/StorageReplicatedMergeTree.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 97d9eb69781..a551569f004 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -514,8 +514,6 @@ private: const Cluster::Address & findClusterAddress(ReplicatedMergeTreeAddress & leader_address); - const Cluster::Address & findClusterAddress(ReplicatedMergeTreeAddress & leader_address) const; - protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. */ From 4cfb51b14e5d512fb1b52c91e9c42e761b5d245a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Nov 2018 01:03:51 +0300 Subject: [PATCH 007/230] Update StorageReplicatedMergeTree.cpp --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 09092d74714..56845df7fda 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3926,7 +3926,7 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query else throw Exception("Can't proxy this query. Unsupported query type", ErrorCodes::NOT_IMPLEMENTED); - /// Query send with current user credentials + /// Query send with user credentials from the corresponding address in clusters configuration. const Cluster::Address & address = findClusterAddress(leader_address); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context.getSettingsRef()); Connection connection( From 65e6d447b86bca02e8e2362405046d20cd9fdfe2 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Nov 2018 01:04:25 +0300 Subject: [PATCH 008/230] Update StorageReplicatedMergeTree.cpp --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 56845df7fda..48704b38429 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3945,7 +3945,7 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query const Cluster::Address & StorageReplicatedMergeTree::findClusterAddress(ReplicatedMergeTreeAddress & leader_address) { - for(auto & iter : context.getClusters().getContainer()) + for (auto & iter : context.getClusters().getContainer()) { const auto & shards = iter.second->getShardsAddresses(); From 6f1846f1cce1746e5a5a4659cbdc1751434e23b5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Nov 2018 01:05:36 +0300 Subject: [PATCH 009/230] Update StorageReplicatedMergeTree.cpp --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 48704b38429..051956965ea 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3962,7 +3962,7 @@ const Cluster::Address & StorageReplicatedMergeTree::findClusterAddress(Replicat } } } - throw Exception("Not found replicate leader host " + leader_address.host + ":" + DB::toString(leader_address.queries_port) + ", Please check your metrika.xml", ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); + throw Exception("Not found replicate leader host " + leader_address.host + ":" + DB::toString(leader_address.queries_port) + ", Please check the 'remote_servers' section in your configuration file (config.xml, config.d, metrika.xml)", ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); } void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_) From 9580a874094cd5ed9b8a446596c21e76051d671f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Nov 2018 01:06:02 +0300 Subject: [PATCH 010/230] Update StorageReplicatedMergeTree.cpp --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 051956965ea..bf6df005f80 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3962,7 +3962,7 @@ const Cluster::Address & StorageReplicatedMergeTree::findClusterAddress(Replicat } } } - throw Exception("Not found replicate leader host " + leader_address.host + ":" + DB::toString(leader_address.queries_port) + ", Please check the 'remote_servers' section in your configuration file (config.xml, config.d, metrika.xml)", ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); + throw Exception("Not found replicate leader host " + leader_address.host + ":" + DB::toString(leader_address.queries_port) + ". Please check the 'remote_servers' section in your configuration file (config.xml, config.d, metrika.xml).", ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); } void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_) From 6d7aa73e449e1d66273effaa388a6b18890355c5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Nov 2018 01:07:35 +0300 Subject: [PATCH 011/230] Update StorageReplicatedMergeTree.cpp --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index bf6df005f80..e423f441e61 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3943,7 +3943,7 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query } -const Cluster::Address & StorageReplicatedMergeTree::findClusterAddress(ReplicatedMergeTreeAddress & leader_address) +const Cluster::Address & StorageReplicatedMergeTree::findClusterAddress(const ReplicatedMergeTreeAddress & leader_address) const { for (auto & iter : context.getClusters().getContainer()) { From 429d65e4f7954ec19e48f22db01db68607a74897 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Nov 2018 01:08:00 +0300 Subject: [PATCH 012/230] Update StorageReplicatedMergeTree.h --- dbms/src/Storages/StorageReplicatedMergeTree.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index c27c3633e80..484ff52bb35 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -515,7 +515,8 @@ private: bool dropPartsInPartition(zkutil::ZooKeeper & zookeeper, String & partition_id, StorageReplicatedMergeTree::LogEntry & entry, bool detach); - const Cluster::Address & findClusterAddress(ReplicatedMergeTreeAddress & leader_address); + const Cluster::Address & findClusterAddress(const ReplicatedMergeTreeAddress & leader_address) const; + // Partition helpers void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context); void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context); From 35419cb6c146466c18686e7230a1a9c8f95eef06 Mon Sep 17 00:00:00 2001 From: root Date: Fri, 23 Nov 2018 14:14:07 +0800 Subject: [PATCH 013/230] Add test --- .../__init__.py | 0 .../configs/node1.xml | 109 ++++++++++++++++ .../configs/node2.xml | 117 ++++++++++++++++++ .../configs/node3.xml | 109 ++++++++++++++++ .../configs/remote_servers.xml | 49 ++++++++ .../test_replicated_drop_partition/test.py | 79 ++++++++++++ 6 files changed, 463 insertions(+) create mode 100644 dbms/tests/integration/test_replicated_drop_partition/__init__.py create mode 100644 dbms/tests/integration/test_replicated_drop_partition/configs/node1.xml create mode 100644 dbms/tests/integration/test_replicated_drop_partition/configs/node2.xml create mode 100644 dbms/tests/integration/test_replicated_drop_partition/configs/node3.xml create mode 100644 dbms/tests/integration/test_replicated_drop_partition/configs/remote_servers.xml create mode 100644 dbms/tests/integration/test_replicated_drop_partition/test.py diff --git a/dbms/tests/integration/test_replicated_drop_partition/__init__.py b/dbms/tests/integration/test_replicated_drop_partition/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_replicated_drop_partition/configs/node1.xml b/dbms/tests/integration/test_replicated_drop_partition/configs/node1.xml new file mode 100644 index 00000000000..a8edd566a09 --- /dev/null +++ b/dbms/tests/integration/test_replicated_drop_partition/configs/node1.xml @@ -0,0 +1,109 @@ + + + + + + + + 10000000000 + + + 0 + + + random + + + + + 1 + + + + + + + + + pTe5Tb0s + + + + ::/0 + + + + default + + + default + + + + + + + ::1 + 127.0.0.1 + + readonly + default + + + + + + + + + + + 3600 + + + 0 + 0 + 0 + 0 + 0 + + + + diff --git a/dbms/tests/integration/test_replicated_drop_partition/configs/node2.xml b/dbms/tests/integration/test_replicated_drop_partition/configs/node2.xml new file mode 100644 index 00000000000..2186d210a29 --- /dev/null +++ b/dbms/tests/integration/test_replicated_drop_partition/configs/node2.xml @@ -0,0 +1,117 @@ + + + + + + + + 10000000000 + + + 0 + + + random + + + + + 1 + + + + + + + + + dtnDvTr9 + + + + ::/0 + + + + default + + + default + + + + azAUGBFl + + ::/0 + + default + default + + + + + + ::1 + 127.0.0.1 + + readonly + default + + + + + + + + + + + 3600 + + + 0 + 0 + 0 + 0 + 0 + + + + diff --git a/dbms/tests/integration/test_replicated_drop_partition/configs/node3.xml b/dbms/tests/integration/test_replicated_drop_partition/configs/node3.xml new file mode 100644 index 00000000000..d972d03b1c4 --- /dev/null +++ b/dbms/tests/integration/test_replicated_drop_partition/configs/node3.xml @@ -0,0 +1,109 @@ + + + + + + + + 10000000000 + + + 0 + + + random + + + + + 1 + + + + + + + + + ROgXGTDq + + + + ::/0 + + + + default + + + default + + + + + + + ::1 + 127.0.0.1 + + readonly + default + + + + + + + + + + + 3600 + + + 0 + 0 + 0 + 0 + 0 + + + + diff --git a/dbms/tests/integration/test_replicated_drop_partition/configs/remote_servers.xml b/dbms/tests/integration/test_replicated_drop_partition/configs/remote_servers.xml new file mode 100644 index 00000000000..866058d831a --- /dev/null +++ b/dbms/tests/integration/test_replicated_drop_partition/configs/remote_servers.xml @@ -0,0 +1,49 @@ + + + + + true + + node1 + 9000 + default + pTe5Tb0s + + + node2 + 9000 + default + dtnDvTr9 + + + + + + true + + node2 + 9000 + zhang + azAUGBFl + + + node3 + 9000 + default + ROgXGTDq + + + + + + ::/0 + + + + 10000000000 + 0.01 + lz4 + + + + diff --git a/dbms/tests/integration/test_replicated_drop_partition/test.py b/dbms/tests/integration/test_replicated_drop_partition/test.py new file mode 100644 index 00000000000..a889217567e --- /dev/null +++ b/dbms/tests/integration/test_replicated_drop_partition/test.py @@ -0,0 +1,79 @@ +import pytest +import time +import sys + +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager + +from helpers.test_tools import assert_eq_with_retry + + +cluster = ClickHouseCluster(__file__) + +def _fill_nodes(nodes, users, passwords, cluster): + for i in range(0,len(nodes)): + node[i].query( + ''' + CREATE DATABASE IF NOT EXISTS test; + + CREATE TABLE test_table{cluster} (date Date, id UInt32, dummy UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{cluster}/replicated', '{replica}', toYYYYMMDD(date), id, 8192); + '''.format(cluster=cluster, replica=node.name), settings={"password": passwords[i]}, user=users[i]) + + +node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], user_configs=['configs/node1.xml'], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], user_configs=['configs/node2.xml'], with_zookeeper=True) +node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml'], user_configs=['configs/node3.xml'], with_zookeeper=True) + +@pytest.fixture(scope="module") +def normal_work(): + try: + cluster.start() + + _fill_nodes([node1, node2],[default,default],[pTe5Tb0s,dtnDvTr9],1) + _fill_nodes([node2, node3],[zhang,default],[azAUGBFl,ROgXGTDq],2) + + yield cluster + + finally: + print('suss') + cluster.shutdown() + +def test_normal_work(normal_work): + print "test_normal_work" + node1.query("insert into test_table1 values ('2017-06-16', 111, 0)",settings={"password": pTe5Tb0s}, user=default) + node1.query("insert into test_table1 values ('2017-06-16', 222, 0)",settings={"password": pTe5Tb0s}, user=default) + node1.query("insert into test_table1 values ('2017-06-17', 333, 0)",settings={"password": pTe5Tb0s}, user=default) + node1.query("insert into test_table1 values ('2017-06-17', 444, 0)",settings={"password": pTe5Tb0s}, user=default) + node1.query("insert into test_table1 values ('2017-06-18', 555, 0)",settings={"password": pTe5Tb0s}, user=default) + node1.query("insert into test_table1 values ('2017-06-18', 666, 0)",settings={"password": pTe5Tb0s}, user=default) + + + assert_eq_with_retry(node1, "SELECT id FROM test_table1 order by id limit 1", '111',settings={"password": pTe5Tb0s}, user=default) + assert_eq_with_retry(node2, "SELECT id FROM test_table1 order by id limit 1", '111',settings={"password": dtnDvTr9}, user=default) + + + node2.query("insert into test_table2 values ('2017-06-17', 333, 0)",settings={"password": azAUGBFl}, user=zhang) + node2.query("insert into test_table2 values ('2017-06-17', 444, 0)",settings={"password": azAUGBFl}, user=zhang) + node2.query("insert into test_table2 values ('2017-06-18', 555, 0)",settings={"password": azAUGBFl}, user=zhang) + node2.query("insert into test_table2 values ('2017-06-18', 666, 0)",settings={"password": azAUGBFl}, user=zhang) + node2.query("insert into test_table2 values ('2017-06-19', 777, 0)",settings={"password": azAUGBFl}, user=zhang) + node2.query("insert into test_table2 values ('2017-06-19', 888, 0)",settings={"password": azAUGBFl}, user=zhang) + + + assert_eq_with_retry(node2, "SELECT id FROM test_table2 order by id limit 1", '333',settings={"password": azAUGBFl}, user=zhang) + assert_eq_with_retry(node3, "SELECT id FROM test_table2 order by id limit 1", '333',settings={"password": ROgXGTDq}, user=default) + + node1.query("ALTER TABLE test_table1 DROP PARTITION 20170616") + assert_eq_with_retry(node1, "SELECT id FROM test_table1 order by id limit 1", '333',settings={"password": pTe5Tb0s}, user=default) + assert_eq_with_retry(node2, "SELECT id FROM test_table1 order by id limit 1", '333',settings={"password": dtnDvTr9}, user=default) + + node1.query("ALTER TABLE test_table1 DROP PARTITION 20170617") + assert_eq_with_retry(node1, "SELECT id FROM test_table1 order by id limit 1", '555',settings={"password": pTe5Tb0s}, user=default) + assert_eq_with_retry(node2, "SELECT id FROM test_table1 order by id limit 1", '555',settings={"password": dtnDvTr9}, user=default) + + + node3.query("ALTER TABLE test_table2 DROP PARTITION 20170617") + assert_eq_with_retry(node2, "SELECT id FROM test_table2 order by id limit 1", '555',settings={"password": azAUGBFl}, user=zhang) + assert_eq_with_retry(node3, "SELECT id FROM test_table2 order by id limit 1", '555',settings={"password": ROgXGTDq}, user=default) + From f18e278e578b7b5a900c9c049d83daacf02eabe8 Mon Sep 17 00:00:00 2001 From: root Date: Fri, 23 Nov 2018 14:23:06 +0800 Subject: [PATCH 014/230] Add test --- .../integration/test_replicated_drop_partition/test.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/dbms/tests/integration/test_replicated_drop_partition/test.py b/dbms/tests/integration/test_replicated_drop_partition/test.py index a889217567e..ac56098cd4a 100644 --- a/dbms/tests/integration/test_replicated_drop_partition/test.py +++ b/dbms/tests/integration/test_replicated_drop_partition/test.py @@ -29,18 +29,16 @@ node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml' def normal_work(): try: cluster.start() - - _fill_nodes([node1, node2],[default,default],[pTe5Tb0s,dtnDvTr9],1) + + _fill_nodes([node1, node2],[default,default],[pTe5Tb0s,dtnDvTr9],1) _fill_nodes([node2, node3],[zhang,default],[azAUGBFl,ROgXGTDq],2) yield cluster finally: - print('suss') cluster.shutdown() def test_normal_work(normal_work): - print "test_normal_work" node1.query("insert into test_table1 values ('2017-06-16', 111, 0)",settings={"password": pTe5Tb0s}, user=default) node1.query("insert into test_table1 values ('2017-06-16', 222, 0)",settings={"password": pTe5Tb0s}, user=default) node1.query("insert into test_table1 values ('2017-06-17', 333, 0)",settings={"password": pTe5Tb0s}, user=default) From 2ccbdad9c826807781cfa5dfa308ac255597305f Mon Sep 17 00:00:00 2001 From: root Date: Fri, 23 Nov 2018 14:53:29 +0800 Subject: [PATCH 015/230] better --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 6 +++--- dbms/src/Storages/StorageReplicatedMergeTree.h | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 66701d03edd..0aaf93897ba 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3870,7 +3870,7 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query else throw Exception("Can't proxy this query. Unsupported query type", ErrorCodes::NOT_IMPLEMENTED); - /// Query send with current user credentials + /// Query send with current user credentials from the corresponding address in clusters configuration. const Cluster::Address & address = findClusterAddress(leader_address); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context.getSettingsRef()); Connection connection( @@ -3887,7 +3887,7 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query } -const Cluster::Address & StorageReplicatedMergeTree::findClusterAddress(ReplicatedMergeTreeAddress & leader_address) +const Cluster::Address & StorageReplicatedMergeTree::findClusterAddress(const ReplicatedMergeTreeAddress & leader_address) const { for(auto & iter : context.getClusters().getContainer()) { @@ -3906,7 +3906,7 @@ const Cluster::Address & StorageReplicatedMergeTree::findClusterAddress(Replicat } } } - throw Exception("Not found replicate leader host " + leader_address.host + ":" + DB::toString(leader_address.queries_port) + ", Please check your metrika.xml", ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); + throw Exception("Not found replicate leader host " + leader_address.host + ":" + DB::toString(leader_address.queries_port) + ". Please check the 'remote_servers' section in your configuration file (config.xml, config.d, metrika.xml).", ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); } void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index a551569f004..2dd200cd1cb 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -512,7 +512,7 @@ private: bool dropPartsInPartition(zkutil::ZooKeeper & zookeeper, String & partition_id, StorageReplicatedMergeTree::LogEntry & entry, bool detach); - const Cluster::Address & findClusterAddress(ReplicatedMergeTreeAddress & leader_address); + const Cluster::Address & findClusterAddress(const ReplicatedMergeTreeAddress & leader_address) const; protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. From ad90d8fadb73c1367360510f6dfc9b7b763c52c7 Mon Sep 17 00:00:00 2001 From: root Date: Fri, 23 Nov 2018 15:19:19 +0800 Subject: [PATCH 016/230] better --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 0aaf93897ba..732843c69fa 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3870,7 +3870,7 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query else throw Exception("Can't proxy this query. Unsupported query type", ErrorCodes::NOT_IMPLEMENTED); - /// Query send with current user credentials from the corresponding address in clusters configuration. + /// Query send with user credentials from the corresponding address in clusters configuration. const Cluster::Address & address = findClusterAddress(leader_address); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context.getSettingsRef()); Connection connection( From e53984216a45c82fd1b79e3a070bbb188e9f65a0 Mon Sep 17 00:00:00 2001 From: root Date: Fri, 23 Nov 2018 15:25:59 +0800 Subject: [PATCH 017/230] better --- dbms/src/Storages/StorageReplicatedMergeTree.h | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 2dd200cd1cb..4e88c7e0bf4 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -514,6 +514,13 @@ private: const Cluster::Address & findClusterAddress(const ReplicatedMergeTreeAddress & leader_address) const; + // Partition helpers + void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context); + void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context); + void attachPartition(const ASTPtr & partition, bool part, const Context & context); + void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context); + void fetchPartition(const ASTPtr & partition, const String & from, const Context & context); + protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. */ From 0ed4cdc24e451b73e12314865338e044a02f139b Mon Sep 17 00:00:00 2001 From: root Date: Fri, 23 Nov 2018 15:33:19 +0800 Subject: [PATCH 018/230] Add test --- dbms/tests/integration/test_replicated_drop_partition/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/integration/test_replicated_drop_partition/test.py b/dbms/tests/integration/test_replicated_drop_partition/test.py index ac56098cd4a..9aacb55d369 100644 --- a/dbms/tests/integration/test_replicated_drop_partition/test.py +++ b/dbms/tests/integration/test_replicated_drop_partition/test.py @@ -66,7 +66,7 @@ def test_normal_work(normal_work): assert_eq_with_retry(node1, "SELECT id FROM test_table1 order by id limit 1", '333',settings={"password": pTe5Tb0s}, user=default) assert_eq_with_retry(node2, "SELECT id FROM test_table1 order by id limit 1", '333',settings={"password": dtnDvTr9}, user=default) - node1.query("ALTER TABLE test_table1 DROP PARTITION 20170617") + node2.query("ALTER TABLE test_table1 DROP PARTITION 20170617") assert_eq_with_retry(node1, "SELECT id FROM test_table1 order by id limit 1", '555',settings={"password": pTe5Tb0s}, user=default) assert_eq_with_retry(node2, "SELECT id FROM test_table1 order by id limit 1", '555',settings={"password": dtnDvTr9}, user=default) From 6d0c4eaf893db5e24a65fea7d56a5c96311021a7 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 11 Oct 2018 10:57:48 +0800 Subject: [PATCH 019/230] =?UTF-8?q?ISSUES-838=20add=20lz4=E3=80=81none?= =?UTF-8?q?=E3=80=81zstd=20codec?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- dbms/CMakeLists.txt | 1 + dbms/src/CMakeLists.txt | 1 + dbms/src/Common/ErrorCodes.cpp | 2 + dbms/src/Compression/CMakeLists.txt | 0 .../src/Compression/CompressionCodecDelta.cpp | 57 +++++ dbms/src/Compression/CompressionCodecDelta.h | 26 +++ dbms/src/Compression/CompressionCodecLZ4.cpp | 46 ++++ dbms/src/Compression/CompressionCodecLZ4.h | 29 +++ .../Compression/CompressionCodecMultiple.cpp | 93 +++++++++ .../Compression/CompressionCodecMultiple.h | 29 +++ dbms/src/Compression/CompressionCodecNone.cpp | 38 ++++ dbms/src/Compression/CompressionCodecNone.h | 23 ++ dbms/src/Compression/CompressionCodecZSTD.cpp | 78 +++++++ dbms/src/Compression/CompressionCodecZSTD.h | 30 +++ dbms/src/Compression/CompressionFactory.cpp | 121 +++++++++++ dbms/src/Compression/CompressionFactory.h | 56 +++++ dbms/src/Compression/ICompressionCodec.cpp | 196 ++++++++++++++++++ dbms/src/Compression/ICompressionCodec.h | 92 ++++++++ dbms/src/IO/CachedCompressedReadBuffer.cpp | 18 +- dbms/src/IO/CachedCompressedReadBuffer.h | 9 +- dbms/src/IO/CompressedStream.h | 1 + .../Interpreters/InterpreterCreateQuery.cpp | 45 ++-- dbms/src/Parsers/ASTColumnDeclaration.h | 13 ++ dbms/src/Parsers/ParserCreateQuery.h | 143 ++++++++----- dbms/src/Storages/ColumnCodec.h | 11 + dbms/src/Storages/ColumnsDescription.cpp | 13 ++ dbms/src/Storages/ColumnsDescription.h | 7 +- .../Storages/MergeTree/MergeTreeReader.cpp | 27 ++- dbms/src/Storages/MergeTree/MergeTreeReader.h | 10 +- .../MergeTree/MergedBlockOutputStream.cpp | 20 +- .../MergeTree/MergedBlockOutputStream.h | 7 +- .../0_stateless/00732_compression_codec.sql | 9 + 32 files changed, 1156 insertions(+), 95 deletions(-) create mode 100644 dbms/src/Compression/CMakeLists.txt create mode 100644 dbms/src/Compression/CompressionCodecDelta.cpp create mode 100644 dbms/src/Compression/CompressionCodecDelta.h create mode 100644 dbms/src/Compression/CompressionCodecLZ4.cpp create mode 100644 dbms/src/Compression/CompressionCodecLZ4.h create mode 100644 dbms/src/Compression/CompressionCodecMultiple.cpp create mode 100644 dbms/src/Compression/CompressionCodecMultiple.h create mode 100644 dbms/src/Compression/CompressionCodecNone.cpp create mode 100644 dbms/src/Compression/CompressionCodecNone.h create mode 100644 dbms/src/Compression/CompressionCodecZSTD.cpp create mode 100644 dbms/src/Compression/CompressionCodecZSTD.h create mode 100644 dbms/src/Compression/CompressionFactory.cpp create mode 100644 dbms/src/Compression/CompressionFactory.h create mode 100644 dbms/src/Compression/ICompressionCodec.cpp create mode 100644 dbms/src/Compression/ICompressionCodec.h create mode 100644 dbms/src/Storages/ColumnCodec.h create mode 100644 dbms/tests/queries/0_stateless/00732_compression_codec.sql diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index b7f11731662..bac6d8776ca 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -72,6 +72,7 @@ add_headers_and_sources(dbms src/Storages/Kafka) add_headers_and_sources(dbms src/Storages/MergeTree) add_headers_and_sources(dbms src/Client) add_headers_and_sources(dbms src/Formats) +add_headers_and_sources(dbms src/Compression) list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) diff --git a/dbms/src/CMakeLists.txt b/dbms/src/CMakeLists.txt index f6fa96e1d47..c5b661b5912 100644 --- a/dbms/src/CMakeLists.txt +++ b/dbms/src/CMakeLists.txt @@ -13,3 +13,4 @@ add_subdirectory (AggregateFunctions) add_subdirectory (Client) add_subdirectory (TableFunctions) add_subdirectory (Formats) +add_subdirectory (Compression) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index e5b6028594b..a5ba25219db 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -402,6 +402,8 @@ namespace ErrorCodes extern const int SYSTEM_ERROR = 425; extern const int NULL_POINTER_DEREFERENCE = 426; extern const int CANNOT_COMPILE_REGEXP = 427; + extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE = 428; + extern const int UNKNOWN_CODEC = 429; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Compression/CMakeLists.txt b/dbms/src/Compression/CMakeLists.txt new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/src/Compression/CompressionCodecDelta.cpp b/dbms/src/Compression/CompressionCodecDelta.cpp new file mode 100644 index 00000000000..25f048a2dbe --- /dev/null +++ b/dbms/src/Compression/CompressionCodecDelta.cpp @@ -0,0 +1,57 @@ +//#include +//#include +//#include +// +// +//namespace DB +//{ +// +//char CompressionCodecDelta::getMethodByte() +//{ +// return static_cast(CompressionMethodByte::LZ4); +//} +// +//void CompressionCodecDelta::getCodecDesc(String & codec_desc) +//{ +// codec_desc = "DELTA"; +//} +// +//size_t CompressionCodecDelta::compress(char * source, size_t source_size, char * dest) +//{ +// /// TODO: use SIMD +// return LZ4_compress_default(source, dest, source_size, LZ4_COMPRESSBOUND(source_size)); +//} +// +//size_t CompressionCodecDelta::decompress(char * source, size_t source_size, char * dest, size_t size_decompressed) +//{ +// LZ4::decompress(source, dest, source_size, size_decompressed, lz4_stat); +// return size_decompressed; +//} +// +//void registerCodecLZ4(CompressionCodecFactory & factory) +//{ +// factory.registerCompressionCodec("DELTA", static_cast(CompressionMethodByte::DELTA), [&](ASTPtr & parameters) +// { +// int width = 1; +// +// if (arguments && !arguments->children.empty()) +// { +// const auto children = arguments->children; +// const ASTIdentifier * identifier = static_cast(children[0].get()); +// +// String delta_type = identifier->name; +// if (delta_type == "Int8" || delta_type == "UInt8") +// width = 1; +// else if (delta_type == "Int16" || delta_type == "UInt16") +// width = 2; +// else if (delta_type == "Int32" || delta_type == "UInt32") +// width = 4; +// else if (delta_type == "Int64" || delta_type == "UInt64") +// width = 8; +// } +// +// return std::make_shared(width); +// }); +//} +// +//} diff --git a/dbms/src/Compression/CompressionCodecDelta.h b/dbms/src/Compression/CompressionCodecDelta.h new file mode 100644 index 00000000000..d35edb46f68 --- /dev/null +++ b/dbms/src/Compression/CompressionCodecDelta.h @@ -0,0 +1,26 @@ +//#pragma once +// +//#include +//#include +//#include +//#include +// +//namespace DB +//{ +// +//class CompressionCodecDelta : public ICompressionCodec +//{ +//public: +// char getMethodByte() override; +// +// void getCodecDesc(String & codec_desc) override; +// +// size_t compress(char * source, size_t source_size, char * dest) override; +// +// size_t getCompressedReserveSize(size_t uncompressed_size) override; +// +// size_t decompress(char * source, size_t source_size, char * dest, size_t decompressed_size) override; +// +//}; +// +//} \ No newline at end of file diff --git a/dbms/src/Compression/CompressionCodecLZ4.cpp b/dbms/src/Compression/CompressionCodecLZ4.cpp new file mode 100644 index 00000000000..ea9c7e65cf9 --- /dev/null +++ b/dbms/src/Compression/CompressionCodecLZ4.cpp @@ -0,0 +1,46 @@ +#include +#include +#include +#include +#include +#include +#include "CompressionCodecLZ4.h" + + +namespace DB +{ + +char CompressionCodecLZ4::getMethodByte() +{ + return static_cast(CompressionMethodByte::LZ4); +} + +void CompressionCodecLZ4::getCodecDesc(String & codec_desc) +{ + codec_desc = "LZ4"; +} + +size_t CompressionCodecLZ4::getCompressedReserveSize(size_t uncompressed_size) +{ + return LZ4_COMPRESSBOUND(uncompressed_size); +} + +size_t CompressionCodecLZ4::compress(char * source, size_t source_size, char * dest) +{ + return LZ4_compress_default(source, dest, source_size, LZ4_COMPRESSBOUND(source_size)); +} + +size_t CompressionCodecLZ4::decompress(char * source, size_t source_size, char * dest, size_t size_decompressed) +{ + LZ4::decompress(source, dest, source_size, size_decompressed, lz4_stat); + return size_decompressed; +} + +void registerCodecLZ4(CompressionCodecFactory & factory) +{ + factory.registerSimpleCompressionCodec("LZ4", static_cast(CompressionMethodByte::LZ4), [&](){ + return std::make_shared(); + }); +} + +} diff --git a/dbms/src/Compression/CompressionCodecLZ4.h b/dbms/src/Compression/CompressionCodecLZ4.h new file mode 100644 index 00000000000..0ee0286c4ee --- /dev/null +++ b/dbms/src/Compression/CompressionCodecLZ4.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class CompressionCodecLZ4 : public ICompressionCodec +{ +public: + char getMethodByte() override; + + void getCodecDesc(String & codec_desc) override; + + size_t compress(char * source, size_t source_size, char * dest) override; + + size_t getCompressedReserveSize(size_t uncompressed_size) override; + + size_t decompress(char * source, size_t source_size, char * dest, size_t decompressed_size) override; + +private: + LZ4::PerformanceStatistics lz4_stat; +}; + +} \ No newline at end of file diff --git a/dbms/src/Compression/CompressionCodecMultiple.cpp b/dbms/src/Compression/CompressionCodecMultiple.cpp new file mode 100644 index 00000000000..0795990e52c --- /dev/null +++ b/dbms/src/Compression/CompressionCodecMultiple.cpp @@ -0,0 +1,93 @@ +#include +#include +#include +#include + + +namespace DB +{ + +CompressionCodecMultiple::CompressionCodecMultiple(Codecs codecs) + : codecs(codecs) +{ + for (size_t idx = 0; idx < codecs.size(); idx++) + { + if (idx != 0) + codec_desc = codec_desc + ','; + + const auto codec = codecs[idx]; + String inner_codec_desc; + codec->getCodecDesc(inner_codec_desc); + codec_desc = codec_desc + inner_codec_desc; + } +} + +char CompressionCodecMultiple::getMethodByte() +{ + return static_cast(CompressionMethodByte::Multiple); +} + +void CompressionCodecMultiple::getCodecDesc(String & codec_desc_) +{ + codec_desc_ = codec_desc; +} + +size_t CompressionCodecMultiple::getCompressedReserveSize(size_t uncompressed_size) +{ + for (auto & codec : codecs) + uncompressed_size = codec->getCompressedReserveSize(uncompressed_size); + + return sizeof(UInt8) + codecs.size() + uncompressed_size; +} + +size_t CompressionCodecMultiple::compress(char * source, size_t source_size, char * dest) +{ + static constexpr size_t without_method_header_size = sizeof(UInt32) + sizeof(UInt32); + + PODArray compressed_buf; + PODArray uncompressed_buf(source_size); + uncompressed_buf.insert(source, source + source_size); + + dest[0] = static_cast(codecs.size()); + for (size_t idx = 0; idx < codecs.size(); ++idx) + { + const auto codec = codecs[idx]; + dest[idx + 1] = codec->getMethodByte(); + compressed_buf.resize(without_method_header_size + codec->getCompressedReserveSize(source_size)); + + size_t size_compressed = without_method_header_size; + size_compressed += codec->compress(&uncompressed_buf[0], source_size, &compressed_buf[without_method_header_size]); + + UInt32 compressed_size_32 = size_compressed; + UInt32 uncompressed_size_32 = source_size; + unalignedStore(&compressed_buf[0], compressed_size_32); + unalignedStore(&compressed_buf[4], uncompressed_size_32); + uncompressed_buf.swap(compressed_buf); + source_size = size_compressed; + } + + memcpy(&dest[codecs.size() + 1], &compressed_buf[0], source_size); + return source_size; +} + +size_t CompressionCodecMultiple::decompress(char * source, size_t source_size, char * dest, size_t decompressed_size) +{ + UInt8 compression_methods_size = source[0]; + + PODArray compressed_buf; + PODArray uncompressed_buf; + compressed_buf.insert(&source[compression_methods_size + 1], source_size - (compression_methods_size + 1)); + + for (size_t idx = 0; idx < compression_methods_size; ++idx) + { + UInt8 compression_method = source[idx + 1]; + const auto codec = CompressionCodecFactory::instance().get(compression_method); + codec->decompress(&compressed_buf[8], 0, uncompressed_buf.data(), 0); + uncompressed_buf.swap(compressed_buf); + } + + memcpy(dest, uncompressed_buf.data(), decompressed_size); + return decompressed_size; +} + +} \ No newline at end of file diff --git a/dbms/src/Compression/CompressionCodecMultiple.h b/dbms/src/Compression/CompressionCodecMultiple.h new file mode 100644 index 00000000000..7457eec35e2 --- /dev/null +++ b/dbms/src/Compression/CompressionCodecMultiple.h @@ -0,0 +1,29 @@ +#pragma once + +#include + +namespace DB +{ + +class CompressionCodecMultiple final : public ICompressionCodec +{ +public: + CompressionCodecMultiple(Codecs codecs); + + char getMethodByte() override; + + void getCodecDesc(String & codec_desc) override; + + size_t compress(char * source, size_t source_size, char * dest) override; + + size_t getCompressedReserveSize(size_t uncompressed_size) override; + + size_t decompress(char *source, size_t source_size, char *dest, size_t decompressed_size) override; + +private: + Codecs codecs; + String codec_desc; + +}; + +} diff --git a/dbms/src/Compression/CompressionCodecNone.cpp b/dbms/src/Compression/CompressionCodecNone.cpp new file mode 100644 index 00000000000..c35cfefd5dd --- /dev/null +++ b/dbms/src/Compression/CompressionCodecNone.cpp @@ -0,0 +1,38 @@ +#include +#include +#include + + +namespace DB +{ + +char CompressionCodecNone::getMethodByte() +{ + return static_cast(CompressionMethodByte::NONE); +} + +void CompressionCodecNone::getCodecDesc(String & codec_desc) +{ + codec_desc = "NONE"; +} + +size_t CompressionCodecNone::compress(char * source, size_t source_size, char * dest) +{ + memcpy(dest, source, source_size); + return source_size; +} + +size_t CompressionCodecNone::decompress(char *source, size_t /*source_size*/, char *dest, size_t size_decompressed) +{ + memcpy(dest, source, size_decompressed); + return size_decompressed; +} + +void registerCodecNone(CompressionCodecFactory & factory) +{ + factory.registerSimpleCompressionCodec("NONE", static_cast(CompressionMethodByte::NONE), [&](){ + return std::make_shared(); + }); +} + +} \ No newline at end of file diff --git a/dbms/src/Compression/CompressionCodecNone.h b/dbms/src/Compression/CompressionCodecNone.h new file mode 100644 index 00000000000..89aeb218e10 --- /dev/null +++ b/dbms/src/Compression/CompressionCodecNone.h @@ -0,0 +1,23 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class CompressionCodecNone : public ICompressionCodec +{ +public: + char getMethodByte() override; + + void getCodecDesc(String & codec_desc) override; + + size_t compress(char * source, size_t source_size, char * compressed_buf) override; + + size_t decompress(char *source, size_t source_size, char *dest, size_t decompressed_size) override; +}; + +} \ No newline at end of file diff --git a/dbms/src/Compression/CompressionCodecZSTD.cpp b/dbms/src/Compression/CompressionCodecZSTD.cpp new file mode 100644 index 00000000000..f907ae93498 --- /dev/null +++ b/dbms/src/Compression/CompressionCodecZSTD.cpp @@ -0,0 +1,78 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_COMPRESS; + extern const int CANNOT_DECOMPRESS; +} + +char CompressionCodecZSTD::getMethodByte() +{ + return static_cast(CompressionMethodByte::ZSTD); +} + +void CompressionCodecZSTD::getCodecDesc(String & codec_desc) +{ + codec_desc = "ZSTD"; +} + +size_t CompressionCodecZSTD::getCompressedReserveSize(size_t uncompressed_size) +{ + return ZSTD_compressBound(uncompressed_size); +} + +size_t CompressionCodecZSTD::compress(char * source, size_t source_size, char * dest) +{ + size_t compressed_size = ZSTD_compress(dest, ZSTD_compressBound(source_size), source, source_size, level); + + if (ZSTD_isError(compressed_size)) + throw Exception("Cannot compress block with ZSTD: " + std::string(ZSTD_getErrorName(compressed_size)), ErrorCodes::CANNOT_COMPRESS); + + return compressed_size; +} + +size_t CompressionCodecZSTD::decompress(char * source, size_t source_size, char * dest, size_t size_decompressed) +{ + size_t res = ZSTD_decompress(dest, size_decompressed, source, source_size); +// compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, size_compressed_without_checksum - COMPRESSED_BLOCK_HEADER_SIZE); + + if (ZSTD_isError(res)) + throw Exception("Cannot ZSTD_decompress: " + std::string(ZSTD_getErrorName(res)), ErrorCodes::CANNOT_DECOMPRESS); + + return size_decompressed; +} + +CompressionCodecZSTD::CompressionCodecZSTD(int level) + :level(level) +{ +} + +void registerCodecZSTD(CompressionCodecFactory & factory) +{ + UInt8 method_code = static_cast(CompressionMethodByte::ZSTD); + factory.registerCompressionCodec("ZSTD", method_code, [&](const ASTPtr & arguments) -> CompressionCodecPtr + { + int level = 0; + if (arguments && !arguments->children.empty()) + { + const auto children = arguments->children; + const ASTLiteral * literal = static_cast(children[0].get()); + level = literal->value.safeGet(); + } + + return std::make_shared(level); + }); +} + +} diff --git a/dbms/src/Compression/CompressionCodecZSTD.h b/dbms/src/Compression/CompressionCodecZSTD.h new file mode 100644 index 00000000000..9f6e3cd7497 --- /dev/null +++ b/dbms/src/Compression/CompressionCodecZSTD.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class CompressionCodecZSTD : public ICompressionCodec +{ +public: + CompressionCodecZSTD(int level); + + char getMethodByte() override; + + void getCodecDesc(String & codec_desc) override; + + size_t compress(char * source, size_t source_size, char * dest) override; + + size_t getCompressedReserveSize(size_t uncompressed_size) override; + + size_t decompress(char *source, size_t source_size, char *dest, size_t decompressed_size) override; + +private: + int level; +}; + +} \ No newline at end of file diff --git a/dbms/src/Compression/CompressionFactory.cpp b/dbms/src/Compression/CompressionFactory.cpp new file mode 100644 index 00000000000..c8eb8981185 --- /dev/null +++ b/dbms/src/Compression/CompressionFactory.cpp @@ -0,0 +1,121 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int UNKNOWN_CODEC; + extern const int UNEXPECTED_AST_STRUCTURE; + extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; + extern const int DATA_TYPE_CANNOT_HAVE_ARGUMENTS; +} + +CompressionCodecPtr CompressionCodecFactory::getDefaultCodec() const +{ + return default_codec; +} + +CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast) const +{ + if (const auto * func = typeid_cast(ast.get())) + { + if (func->parameters) + throw Exception("Compression codec cannot have multiple parenthesed parameters.", ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); + + if (Poco::toLower(func->name) != "codec") + throw Exception("", ErrorCodes::UNKNOWN_CODEC); + + Codecs codecs; + codecs.reserve(func->arguments->children.size()); + for (const auto & inner_codec_ast : func->arguments->children) + { + if (const auto * family_name = typeid_cast(inner_codec_ast.get())) + codecs.emplace_back(getImpl(family_name->name, {})); + else if (const auto * ast_func = typeid_cast(inner_codec_ast.get())) + codecs.emplace_back(getImpl(ast_func->name, ast_func->arguments)); + else + throw Exception("Unexpected AST element for compression codec.", ErrorCodes::UNEXPECTED_AST_STRUCTURE); + } + + if (codecs.size() == 1) + return codecs.back(); + else if (codecs.size() > 1) + return std::make_shared(codecs); + } + + throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC); +} + +CompressionCodecPtr CompressionCodecFactory::get(const UInt8 byte_code) const +{ + const auto family_code_and_creator = family_code_with_codec.find(byte_code); + + if (family_code_and_creator == family_code_with_codec.end()) + throw Exception("Unknown codec family code : " + toString(byte_code), ErrorCodes::UNKNOWN_CODEC); + + return family_code_and_creator->second({}); +} + +CompressionCodecPtr CompressionCodecFactory::getImpl(const String & family_name, const ASTPtr & arguments) const +{ + const auto family_and_creator = family_name_with_codec.find(family_name); + + if (family_and_creator == family_name_with_codec.end()) + throw Exception("Unknown codec family: " + family_name, ErrorCodes::UNKNOWN_CODEC); + + return family_and_creator->second(arguments); +} + +void CompressionCodecFactory::registerCompressionCodec(const String & family_name, UInt8 byte_code, Creator creator) +{ + if (creator == nullptr) + throw Exception("CompressionCodecFactory: the codec family " + family_name + " has been provided a null constructor", + ErrorCodes::LOGICAL_ERROR); + + if (!family_name_with_codec.emplace(family_name, creator).second) + throw Exception("CompressionCodecFactory: the codec family name '" + family_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); + + if (!family_code_with_codec.emplace(byte_code, creator).second) + throw Exception("CompressionCodecFactory: the codec family name '" + family_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); +} + +void CompressionCodecFactory::registerSimpleCompressionCodec(const String & family_name, UInt8 byte_code, + std::function creator) +{ + registerCompressionCodec(family_name, byte_code, [family_name, creator](const ASTPtr & ast) + { + if (ast) + throw Exception("Compression codec " + family_name + " cannot have arguments", ErrorCodes::DATA_TYPE_CANNOT_HAVE_ARGUMENTS); + return creator(); + }); +} + +void registerCodecLZ4(CompressionCodecFactory & factory); +void registerCodecNone(CompressionCodecFactory & factory); +void registerCodecZSTD(CompressionCodecFactory & factory); +//void registerCodecDelta(CompressionCodecFactory & factory); + +CompressionCodecFactory::CompressionCodecFactory() +{ + default_codec = std::make_shared(); + registerCodecLZ4(*this); + registerCodecNone(*this); + registerCodecZSTD(*this); +// registerCodecDelta(*this); +} + +} \ No newline at end of file diff --git a/dbms/src/Compression/CompressionFactory.h b/dbms/src/Compression/CompressionFactory.h new file mode 100644 index 00000000000..0298abb07e4 --- /dev/null +++ b/dbms/src/Compression/CompressionFactory.h @@ -0,0 +1,56 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class ICompressionCodec; + +using CompressionCodecPtr = std::shared_ptr; + +class IAST; + +using ASTPtr = std::shared_ptr; + +/** Creates a codec object by name of compression algorithm family and parameters. + */ +class CompressionCodecFactory final : public ext::singleton +{ +protected: + using Creator = std::function; + using SimpleCreator = std::function; + using CompressionCodecsDictionary = std::unordered_map; + using CompressionCodecsCodeDictionary = std::unordered_map; +public: + + CompressionCodecPtr getDefaultCodec() const; + + CompressionCodecPtr get(const ASTPtr & ast) const; + + CompressionCodecPtr get(const UInt8 byte_code) const; + + void registerCompressionCodec(const String & family_name, UInt8 byte_code, Creator creator); + + void registerSimpleCompressionCodec(const String & family_name, UInt8 byte_code, SimpleCreator creator); + +protected: + CompressionCodecPtr getImpl(const String & family_name, const ASTPtr & arguments) const; + +private: + CompressionCodecsDictionary family_name_with_codec; + CompressionCodecsCodeDictionary family_code_with_codec; + CompressionCodecPtr default_codec; + + CompressionCodecFactory(); + + friend class ext::singleton; +}; + +} \ No newline at end of file diff --git a/dbms/src/Compression/ICompressionCodec.cpp b/dbms/src/Compression/ICompressionCodec.cpp new file mode 100644 index 00000000000..e82fb891318 --- /dev/null +++ b/dbms/src/Compression/ICompressionCodec.cpp @@ -0,0 +1,196 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event ReadCompressedBytes; + extern const Event CompressedReadBufferBlocks; + extern const Event CompressedReadBufferBytes; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CHECKSUM_DOESNT_MATCH; + extern const int TOO_LARGE_SIZE_COMPRESSED; + extern const int UNKNOWN_COMPRESSION_METHOD; + extern const int CANNOT_DECOMPRESS; + extern const int SEEK_POSITION_OUT_OF_BOUND; +} + +CompressionCodecReadBufferPtr ICompressionCodec::liftCompressed(ReadBuffer & origin) +{ + return std::make_shared(origin); +} + +CompressionCodecWriteBufferPtr ICompressionCodec::liftCompressed(WriteBuffer & origin) +{ + return std::make_shared(*this, origin); +} + +CompressionCodecReadBuffer::CompressionCodecReadBuffer(ReadBuffer & origin) + : origin(origin) +{ +} + +/// Read compressed data into compressed_buffer. Get size of decompressed data from block header. Checksum if need. +/// Returns number of compressed bytes read. +size_t CompressionCodecReadBuffer::readCompressedData(size_t & size_decompressed, size_t & size_compressed) +{ + if (origin.eof()) + return 0; + + CityHash_v1_0_2::uint128 checksum; + origin.readStrict(reinterpret_cast(&checksum), sizeof(checksum)); + + own_compressed_buffer.resize(COMPRESSED_BLOCK_HEADER_SIZE); + origin.readStrict(own_compressed_buffer.data(), COMPRESSED_BLOCK_HEADER_SIZE); + + size_compressed = unalignedLoad(&own_compressed_buffer[1]); + size_decompressed = unalignedLoad(&own_compressed_buffer[5]); + + if (size_compressed > DBMS_MAX_COMPRESSED_SIZE) + throw Exception("Too large size_compressed: " + toString(size_compressed) + ". Most likely corrupted data.", ErrorCodes::TOO_LARGE_SIZE_COMPRESSED); + + ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed + sizeof(checksum)); + + /// Is whole compressed block located in 'origin' buffer? + if (origin.offset() >= COMPRESSED_BLOCK_HEADER_SIZE && + origin.position() + size_compressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER - COMPRESSED_BLOCK_HEADER_SIZE <= origin.buffer().end()) + { + origin.position() -= COMPRESSED_BLOCK_HEADER_SIZE; + compressed_buffer = origin.position(); + origin.position() += size_compressed; + } + else + { + own_compressed_buffer.resize(size_compressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); + compressed_buffer = own_compressed_buffer.data(); + origin.readStrict(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, size_compressed - COMPRESSED_BLOCK_HEADER_SIZE); + } + + auto checksum_calculated = CityHash_v1_0_2::CityHash128(compressed_buffer, size_compressed); + if (checksum != checksum_calculated) + throw Exception("Checksum doesn't match: corrupted data." + " Reference: " + getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second) + + ". Actual: " + getHexUIntLowercase(checksum_calculated.first) + getHexUIntLowercase(checksum_calculated.second) + + ". Size of compressed block: " + toString(size_compressed) + ".", + ErrorCodes::CHECKSUM_DOESNT_MATCH); + + return size_compressed + sizeof(checksum); +} + +void CompressionCodecReadBuffer::decompress(char * to, size_t size_decompressed, size_t size_compressed_without_checksum) +{ + ProfileEvents::increment(ProfileEvents::CompressedReadBufferBlocks); + ProfileEvents::increment(ProfileEvents::CompressedReadBufferBytes, size_decompressed); + + UInt8 current_method = compressed_buffer[0]; /// See CompressedWriteBuffer.h + if (current_method != method) + codec = CompressionCodecFactory::instance().get(method); + + codec->decompress(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, + size_compressed_without_checksum - COMPRESSED_BLOCK_HEADER_SIZE, to, size_decompressed); +} + +bool CompressionCodecReadBuffer::nextImpl() +{ + size_t size_decompressed; + size_t size_compressed_without_checksum; + + size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum); + if (!size_compressed) + return false; + + memory.resize(size_decompressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); + working_buffer = Buffer(memory.data(), &memory[size_decompressed]); + + decompress(working_buffer.begin(), size_decompressed, size_compressed_without_checksum); + + return true; +} + +void CompressionCodecReadBuffer::seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block) +{ + if (const auto file_in = dynamic_cast(&origin)) + { + if (size_compressed && + offset_in_compressed_file == file_in->getPositionInFile() - size_compressed && + offset_in_decompressed_block <= working_buffer.size()) + { + bytes += offset(); + pos = working_buffer.begin() + offset_in_decompressed_block; + /// `bytes` can overflow and get negative, but in `count()` everything will overflow back and get right. + bytes -= offset(); + } + else + { + file_in->seek(offset_in_compressed_file); + + bytes += offset(); + nextImpl(); + + if (offset_in_decompressed_block > working_buffer.size()) + throw Exception("Seek position is beyond the decompressed block" + " (pos: " + toString(offset_in_decompressed_block) + ", block size: " + toString(working_buffer.size()) + ")", + ErrorCodes::SEEK_POSITION_OUT_OF_BOUND); + + pos = working_buffer.begin() + offset_in_decompressed_block; + bytes -= offset(); + } + } + else + throw Exception("CompressionCodec: cannot seek in non-file buffer", ErrorCodes::LOGICAL_ERROR); +} + +CompressionCodecWriteBuffer::~CompressionCodecWriteBuffer() +{ + try + { + next(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +void CompressionCodecWriteBuffer::nextImpl() +{ + if (!offset()) + return; + + static constexpr size_t header_size = 1 + sizeof(UInt32) + sizeof(UInt32); + + size_t uncompressed_size = offset(); + size_t compressed_reserve_size = compression_codec.getCompressedReserveSize(uncompressed_size); + + compressed_buffer.resize(header_size + compressed_reserve_size); + compressed_buffer[0] = compression_codec.getMethodByte(); + size_t compressed_size = header_size + compression_codec.compress(working_buffer.begin(), uncompressed_size, &compressed_buffer[header_size]); + + UInt32 compressed_size_32 = compressed_size; + UInt32 uncompressed_size_32 = uncompressed_size; + unalignedStore(&compressed_buffer[1], compressed_size_32); + unalignedStore(&compressed_buffer[5], uncompressed_size_32); + CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(compressed_buffer.data(), compressed_size); + out.write(reinterpret_cast(&checksum), sizeof(checksum)); + out.write(compressed_buffer.data(), compressed_size); +} + +CompressionCodecWriteBuffer::CompressionCodecWriteBuffer(ICompressionCodec & compression_codec, WriteBuffer & out, size_t buf_size) + : BufferWithOwnMemory(buf_size), out(out), compression_codec(compression_codec) +{ +} + +} diff --git a/dbms/src/Compression/ICompressionCodec.h b/dbms/src/Compression/ICompressionCodec.h new file mode 100644 index 00000000000..7238dfbdbee --- /dev/null +++ b/dbms/src/Compression/ICompressionCodec.h @@ -0,0 +1,92 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class ICompressionCodec; + +using CompressionCodecPtr = std::shared_ptr; +using Codecs = std::vector; + +class CompressionCodecReadBuffer; +class CompressionCodecWriteBuffer; + +using CompressionCodecReadBufferPtr = std::shared_ptr; +using CompressionCodecWriteBufferPtr = std::shared_ptr; + +class CompressionCodecWriteBuffer : public BufferWithOwnMemory +{ +public: + CompressionCodecWriteBuffer(ICompressionCodec & compression_codec, WriteBuffer & out, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + + ~CompressionCodecWriteBuffer() override; + +private: + void nextImpl() override; + + +private: + WriteBuffer & out; + ICompressionCodec & compression_codec; + PODArray compressed_buffer; +}; + +class CompressionCodecReadBuffer : public BufferWithOwnMemory +{ +public: + size_t size_compressed = 0; + size_t size_decompressed = 0; + + CompressionCodecReadBuffer(ReadBuffer & origin); + + size_t readCompressedData(size_t & size_decompressed, size_t & size_compressed); + + void decompress(char * to, size_t size_decompressed, size_t size_compressed_without_checksum); + + void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); + +private: + ReadBuffer & origin; + char * compressed_buffer; + UInt8 method; + CompressionCodecPtr codec; + PODArray own_compressed_buffer; + + bool nextImpl() override; +}; + +/** +* +*/ +class ICompressionCodec : private boost::noncopyable +{ +public: + virtual ~ICompressionCodec() = default; + + CompressionCodecReadBufferPtr liftCompressed(ReadBuffer & origin); + + CompressionCodecWriteBufferPtr liftCompressed(WriteBuffer & origin); + + virtual char getMethodByte() = 0; + + virtual void getCodecDesc(String & codec_desc) = 0; + + virtual size_t compress(char * source, size_t source_size, char * dest) = 0; + + virtual size_t decompress(char * source, size_t source_size, char * dest, size_t decompressed_size) = 0; + + virtual size_t getCompressedReserveSize(size_t uncompressed_size) { return uncompressed_size; } +}; + +} diff --git a/dbms/src/IO/CachedCompressedReadBuffer.cpp b/dbms/src/IO/CachedCompressedReadBuffer.cpp index 6f6836718f4..bc36ed38afd 100644 --- a/dbms/src/IO/CachedCompressedReadBuffer.cpp +++ b/dbms/src/IO/CachedCompressedReadBuffer.cpp @@ -3,6 +3,7 @@ #include #include #include +#include "CachedCompressedReadBuffer.h" namespace DB @@ -19,7 +20,7 @@ void CachedCompressedReadBuffer::initInput() if (!file_in) { file_in = createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size); - compressed_in = &*file_in; + in = codec->liftCompressed(*file_in); if (profile_callback) file_in->setProfileCallback(profile_callback, clock_type); @@ -30,7 +31,6 @@ void CachedCompressedReadBuffer::initInput() bool CachedCompressedReadBuffer::nextImpl() { /// Let's check for the presence of a decompressed block in the cache, grab the ownership of this block, if it exists. - UInt128 key = cache->hash(path, file_pos); owned_cell = cache->get(key); @@ -44,12 +44,15 @@ bool CachedCompressedReadBuffer::nextImpl() size_t size_decompressed; size_t size_compressed_without_checksum; - owned_cell->compressed_size = readCompressedData(size_decompressed, size_compressed_without_checksum); + owned_cell->compressed_size = in->readCompressedData(size_decompressed, size_compressed_without_checksum); if (owned_cell->compressed_size) { owned_cell->data.resize(size_decompressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); - decompress(owned_cell->data.data(), size_decompressed, size_compressed_without_checksum); + in->decompress(owned_cell->data.data(), size_decompressed, size_compressed_without_checksum); + + in->buffer() = Buffer(owned_cell->data.data(), owned_cell->data.data() + owned_cell->data.size() - LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); + in->decompress(owned_cell->data.data(), size_decompressed, size_compressed_without_checksum); /// Put data into cache. cache->set(key, owned_cell); @@ -71,14 +74,13 @@ bool CachedCompressedReadBuffer::nextImpl() CachedCompressedReadBuffer::CachedCompressedReadBuffer( - const std::string & path_, UncompressedCache * cache_, size_t estimated_size_, size_t aio_threshold_, - size_t buf_size_) - : ReadBuffer(nullptr, 0), path(path_), cache(cache_), buf_size(buf_size_), estimated_size(estimated_size_), + const std::string & path_, UncompressedCache * cache_, const CompressionCodecPtr & codec, + size_t estimated_size_, size_t aio_threshold_, size_t buf_size_) + : ReadBuffer(nullptr, 0), path(path_), cache(cache_), codec(codec), buf_size(buf_size_), estimated_size(estimated_size_), aio_threshold(aio_threshold_), file_pos(0) { } - void CachedCompressedReadBuffer::seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block) { if (owned_cell && diff --git a/dbms/src/IO/CachedCompressedReadBuffer.h b/dbms/src/IO/CachedCompressedReadBuffer.h index 1b5e41972f3..2aaaaa21a1c 100644 --- a/dbms/src/IO/CachedCompressedReadBuffer.h +++ b/dbms/src/IO/CachedCompressedReadBuffer.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -18,15 +19,17 @@ namespace DB * Disadvantages: * - in case you need to read a lot of data in a row, but of them only a part is cached, you have to do seek-and. */ -class CachedCompressedReadBuffer : public CompressedReadBufferBase, public ReadBuffer +class CachedCompressedReadBuffer : public ReadBuffer { private: const std::string path; UncompressedCache * cache; + const CompressionCodecPtr & codec; size_t buf_size; size_t estimated_size; size_t aio_threshold; + CompressionCodecReadBufferPtr in; std::unique_ptr file_in; size_t file_pos; @@ -42,8 +45,8 @@ private: public: CachedCompressedReadBuffer( - const std::string & path_, UncompressedCache * cache_, size_t estimated_size_, size_t aio_threshold_, - size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); + const std::string & path_, UncompressedCache * cache_, const CompressionCodecPtr & codec, + size_t estimated_size_, size_t aio_threshold_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); diff --git a/dbms/src/IO/CompressedStream.h b/dbms/src/IO/CompressedStream.h index 5a00db0201d..be8fb254fab 100644 --- a/dbms/src/IO/CompressedStream.h +++ b/dbms/src/IO/CompressedStream.h @@ -47,6 +47,7 @@ enum class CompressionMethodByte : uint8_t NONE = 0x02, LZ4 = 0x82, ZSTD = 0x90, + Multiple = 0x91, }; } diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 66ce6bb3fb1..5ddfa2de100 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -44,6 +44,8 @@ #include +#include + namespace DB { @@ -170,14 +172,15 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) using ColumnsAndDefaults = std::pair; -using ParsedColumns = std::tuple; +using ColumnsDeclarationAndModifiers = std::tuple; /// AST to the list of columns with types. Columns of Nested type are expanded into a list of real columns. -static ParsedColumns parseColumns(const ASTExpressionList & column_list_ast, const Context & context) +static ColumnsDeclarationAndModifiers parseColumns(const ASTExpressionList & column_list_ast, const Context & context) { /// list of table columns in correct order NamesAndTypesList columns{}; ColumnDefaults defaults{}; + ColumnCodecs codecs{}; ColumnComments comments{}; /// Columns requiring type-deduction or default_expression type-check @@ -223,6 +226,12 @@ static ParsedColumns parseColumns(const ASTExpressionList & column_list_ast, con default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name)); } + if (col_decl.codec) + { + auto codec = CompressionCodecFactory::instance().get(col_decl.codec); + codecs.emplace(col_decl.name, codec); + } + if (col_decl.comment) { if (auto comment_str = typeid_cast(*col_decl.comment).value.get(); !comment_str.empty()) @@ -278,14 +287,14 @@ static ParsedColumns parseColumns(const ASTExpressionList & column_list_ast, con } } - return {Nested::flatten(columns), defaults, comments}; + return {Nested::flatten(columns), defaults, codecs, comments}; } -static NamesAndTypesList removeAndReturnColumns(ColumnsAndDefaults & columns_and_defaults, const ColumnDefaultKind kind) +static NamesAndTypesList removeAndReturnColumns(ColumnsDeclarationAndModifiers & columns_declare, const ColumnDefaultKind kind) { - auto & columns = columns_and_defaults.first; - auto & defaults = columns_and_defaults.second; + auto & columns = std::get<0>(columns_declare); + auto & defaults = std::get<1>(columns_declare); NamesAndTypesList removed{}; @@ -359,6 +368,18 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) column_declaration->comment = std::make_shared(Field(comments_it->second)); } + const auto ct = columns.codecs.find(column.name); + if (ct != std::end(columns.codecs)) + { + String codec_desc; + ct->second->getCodecDesc(codec_desc); + codec_desc = "CODEC(" + codec_desc + ")"; + auto pos = codec_desc.data(); + const auto end = pos + codec_desc.size(); + ParserIdentifierWithParameters codec_p; + column_declaration->codec = parseQuery(codec_p, pos, end, "column codec", 0); + } + columns_list->children.push_back(column_declaration_ptr); } @@ -370,12 +391,12 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpres ColumnsDescription res; auto && parsed_columns = parseColumns(columns, context); - auto columns_and_defaults = std::make_pair(std::move(std::get<0>(parsed_columns)), std::move(std::get<1>(parsed_columns))); - res.materialized = removeAndReturnColumns(columns_and_defaults, ColumnDefaultKind::Materialized); - res.aliases = removeAndReturnColumns(columns_and_defaults, ColumnDefaultKind::Alias); - res.ordinary = std::move(columns_and_defaults.first); - res.defaults = std::move(columns_and_defaults.second); - res.comments = std::move(std::get<2>(parsed_columns)); + res.ordinary = std::move(std::get<0>(parsed_columns)); + res.defaults = std::move(std::get<1>(parsed_columns)); + res.codecs = std::move(std::get<2>(parsed_columns)); + res.comments = std::move(std::get<3>(parsed_columns)); + res.aliases = removeAndReturnColumns(parsed_columns, ColumnDefaultKind::Alias); + res.materialized = removeAndReturnColumns(parsed_columns, ColumnDefaultKind::Materialized); if (res.ordinary.size() + res.materialized.size() == 0) throw Exception{"Cannot CREATE table without physical columns", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED}; diff --git a/dbms/src/Parsers/ASTColumnDeclaration.h b/dbms/src/Parsers/ASTColumnDeclaration.h index e288dbbcd58..54e52382328 100644 --- a/dbms/src/Parsers/ASTColumnDeclaration.h +++ b/dbms/src/Parsers/ASTColumnDeclaration.h @@ -15,6 +15,7 @@ public: ASTPtr type; String default_specifier; ASTPtr default_expression; + ASTPtr codec; ASTPtr comment; String getID(char delim) const override { return "ColumnDeclaration" + (delim + name); } @@ -36,6 +37,12 @@ public: res->children.push_back(res->default_expression); } + if (codec) + { + res->codec=codec->clone(); + res->children.push_back(res->codec); + } + if (comment) { res->comment = comment->clone(); @@ -63,6 +70,12 @@ public: default_expression->formatImpl(settings, state, frame); } + if (codec) + { + settings.ostr << ' '; + codec->formatImpl(settings, state, frame); + } + if (comment) { settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMMENT" << (settings.hilite ? hilite_none : "") << ' '; diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 1084f86d1eb..429ecbb09d1 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -102,9 +102,22 @@ public: } protected: + using ASTDeclarePtr = std::shared_ptr; + const char * getName() const { return "column declaration"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); + bool isDeclareColumnType(Pos & pos, Expected & expected); + + bool parseDeclarationCodec(Pos &pos, const ASTDeclarePtr &declaration, Expected &expected); + + bool parseDefaultExpression(Pos &pos, const ASTDeclarePtr &declaration, Expected &expected); + + bool parseDeclarationComment(Pos &pos, const ASTDeclarePtr &declaration, Expected &expected); + + bool isDeclareColumnCodec(Pos & pos, Expected & expected); + bool require_type = true; }; @@ -114,78 +127,112 @@ using ParserCompoundColumnDeclaration = IParserColumnDeclaration bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { + ASTPtr column_name; + ASTPtr column_type; + ASTPtr column_codec; NameParser name_parser; ParserIdentifierWithOptionalParameters type_parser; - ParserKeyword s_default{"DEFAULT"}; - ParserKeyword s_materialized{"MATERIALIZED"}; - ParserKeyword s_alias{"ALIAS"}; - ParserKeyword s_comment{"COMMENT"}; - ParserTernaryOperatorExpression expr_parser; - ParserStringLiteral string_literal_parser; - /// mandatory column name - ASTPtr name; - if (!name_parser.parse(pos, name, expected)) + if (!name_parser.parse(pos, column_name, expected)) return false; - /** column name should be followed by type name if it - * is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS, COMMENT} - */ - ASTPtr type; - String default_specifier; - ASTPtr default_expression; - ASTPtr comment_expression; + if (isDeclareColumnType(pos, expected) + && !type_parser.parse(pos, column_type, expected)) + return false; - if (!s_default.check_without_moving(pos, expected) && - !s_materialized.check_without_moving(pos, expected) && - !s_alias.check_without_moving(pos, expected) && - !s_comment.check_without_moving(pos, expected)) + const auto column_declaration = std::make_shared(); + + if (!parseDefaultExpression(pos, column_declaration, expected) + || !parseDeclarationCodec(pos, column_declaration, expected) + || !parseDeclarationComment(pos, column_declaration, expected)) + return false; + + if (require_type && !column_type && column_declaration->default_expression) + return false; + + if (column_type) { - if (!type_parser.parse(pos, type, expected)) - return false; + column_declaration->type = column_type; + column_declaration->children.push_back(std::move(column_type)); } + node = column_declaration; + return true; +} + +template +bool IParserColumnDeclaration::isDeclareColumnType(Pos & pos, Expected & expected) +{ + auto check_pos = pos; + return !ParserKeyword{"CODEC"}.check(check_pos, expected) && + !ParserKeyword{"ALIAS"}.check(check_pos, expected) && + !ParserKeyword{"COMMENT"}.check(check_pos, expected) && + !ParserKeyword{"DEFAULT"}.check(check_pos, expected) && + !ParserKeyword{"MATERIALIZED"}.check(check_pos, expected); +} +template +bool IParserColumnDeclaration::isDeclareColumnCodec(Pos & pos, Expected & expected) +{ + auto check_pos = pos; + return ParserKeyword{"CODEC"}.check(check_pos, expected); +} + +template +bool IParserColumnDeclaration::parseDeclarationCodec(Pos & pos, const ASTDeclarePtr & declaration, Expected & expected) +{ + ParserKeyword s_codec{"CODEC"}; + + ParserIdentifierWithParameters codec_parser; + + if (s_codec.ignore(pos, expected)) + { + if (!codec_parser.parse(pos, declaration->codec, expected)) + return false; + + declaration->children.push_back(declaration->codec); + } + + return true; +} + +template +bool IParserColumnDeclaration::parseDefaultExpression(Pos & pos, const ASTDeclarePtr & declaration, Expected & expected) +{ + ParserKeyword s_alias{"ALIAS"}; + ParserKeyword s_default{"DEFAULT"}; + ParserKeyword s_materialized{"MATERIALIZED"}; + + ParserTernaryOperatorExpression expr_parser; + Pos pos_before_specifier = pos; if (s_default.ignore(pos, expected) || s_materialized.ignore(pos, expected) || s_alias.ignore(pos, expected)) { - default_specifier = Poco::toUpper(std::string{pos_before_specifier->begin, pos_before_specifier->end}); + declaration->default_specifier = Poco::toUpper(std::string{pos_before_specifier->begin, pos_before_specifier->end}); /// should be followed by an expression - if (!expr_parser.parse(pos, default_expression, expected)) + if (!expr_parser.parse(pos, declaration->default_expression, expected)) return false; + + declaration->children.push_back(declaration->default_expression); } - if (require_type && !type && !default_expression) - return false; /// reject column name without type + return true; +} +template +bool IParserColumnDeclaration::parseDeclarationComment(Pos & pos, const ASTDeclarePtr & declaration, Expected & expected) +{ + ParserKeyword s_comment{"COMMENT"}; + + ParserStringLiteral string_literal_parser; if (s_comment.ignore(pos, expected)) { /// should be followed by a string literal - if (!string_literal_parser.parse(pos, comment_expression, expected)) + if (!string_literal_parser.parse(pos, declaration->comment, expected)) return false; - } - const auto column_declaration = std::make_shared(); - node = column_declaration; - column_declaration->name = typeid_cast(*name).name; - if (type) - { - column_declaration->type = type; - column_declaration->children.push_back(std::move(type)); - } - - if (default_expression) - { - column_declaration->default_specifier = default_specifier; - column_declaration->default_expression = default_expression; - column_declaration->children.push_back(std::move(default_expression)); - } - - if (comment_expression) - { - column_declaration->comment = comment_expression; - column_declaration->children.push_back(std::move(comment_expression)); + declaration->children.push_back(declaration->comment); } return true; diff --git a/dbms/src/Storages/ColumnCodec.h b/dbms/src/Storages/ColumnCodec.h new file mode 100644 index 00000000000..63a604c8198 --- /dev/null +++ b/dbms/src/Storages/ColumnCodec.h @@ -0,0 +1,11 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + using ColumnCodecs = std::unordered_map; +} diff --git a/dbms/src/Storages/ColumnsDescription.cpp b/dbms/src/Storages/ColumnsDescription.cpp index 0926fef14e8..726d488f957 100644 --- a/dbms/src/Storages/ColumnsDescription.cpp +++ b/dbms/src/Storages/ColumnsDescription.cpp @@ -20,6 +20,7 @@ #include #include +#include #include @@ -195,6 +196,18 @@ void parseColumn(ReadBufferFromString & buf, ColumnsDescription & result, const assertChar('\n', buf); } +CompressionCodecPtr ColumnsDescription::getCodec(const String & column_name, const CompressionSettings & /*compression_settings*/) const +{ + const auto codec = codecs.find(column_name); + + /// TODO get + if (codec == codecs.end()) + return CompressionCodecFactory::instance().getDefaultCodec(); +// return CompressionCodecFactory::instance().get(compression_settings.method, compression_settings.level); + + return codec->second; +} + ColumnsDescription ColumnsDescription::parse(const String & str) { ReadBufferFromString buf{str}; diff --git a/dbms/src/Storages/ColumnsDescription.h b/dbms/src/Storages/ColumnsDescription.h index 8beb7634000..c06418cf89a 100644 --- a/dbms/src/Storages/ColumnsDescription.h +++ b/dbms/src/Storages/ColumnsDescription.h @@ -4,6 +4,8 @@ #include #include #include +#include +#include namespace DB @@ -18,6 +20,7 @@ struct ColumnsDescription NamesAndTypesList materialized; NamesAndTypesList aliases; ColumnDefaults defaults; + ColumnCodecs codecs; ColumnComments comments; ColumnsDescription() = default; @@ -60,10 +63,12 @@ struct ColumnsDescription bool hasPhysical(const String & column_name) const; - String toString() const; + CompressionCodecPtr getCodec(const String & column_name, const CompressionSettings & compression_settings) const; + static ColumnsDescription parse(const String & str); + static const ColumnsDescription * loadFromContext(const Context & context, const String & db, const String & table); }; diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index 05c272f09f4..1b7b3c75982 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB @@ -49,8 +50,13 @@ MergeTreeReader::MergeTreeReader(const String & path, if (!Poco::File(path).exists()) throw Exception("Part " + path + " is missing", ErrorCodes::NOT_FOUND_EXPECTED_DATA_PART); + const auto columns_desc = storage.getColumns(); + for (const NameAndTypePair & column : columns) - addStreams(column.name, *column.type, all_mark_ranges, profile_callback, clock_type); + { + CompressionCodecPtr codec = columns_desc.getCodec(column.name, {}); + addStreams(column.name, *column.type, codec, all_mark_ranges, profile_callback, clock_type); + } } catch (...) { @@ -158,6 +164,7 @@ size_t MergeTreeReader::readRows(size_t from_mark, bool continue_reading, size_t MergeTreeReader::Stream::Stream( const String & path_prefix_, const String & extension_, size_t marks_count_, + const CompressionCodecPtr & codec, const MarkRanges & all_mark_ranges, MarkCache * mark_cache_, bool save_marks_in_cache_, UncompressedCache * uncompressed_cache, @@ -229,8 +236,8 @@ MergeTreeReader::Stream::Stream( /// Initialize the objects that shall be used to perform read operations. if (uncompressed_cache) { - auto buffer = std::make_unique( - path_prefix + extension, uncompressed_cache, estimated_size, aio_threshold, buffer_size); + auto buffer = std::make_shared( + path_prefix + extension, uncompressed_cache, codec, estimated_size, aio_threshold, buffer_size); if (profile_callback) buffer->setProfileCallback(profile_callback, clock_type); @@ -240,13 +247,13 @@ MergeTreeReader::Stream::Stream( } else { - auto buffer = std::make_unique( - path_prefix + extension, estimated_size, aio_threshold, buffer_size); + file_in = createReadBufferFromFileBase(path_prefix + extension, estimated_size, aio_threshold, buffer_size); if (profile_callback) - buffer->setProfileCallback(profile_callback, clock_type); + file_in->setProfileCallback(profile_callback, clock_type); - non_cached_buffer = std::move(buffer); + const auto compressed_buffer = codec->liftCompressed(*file_in); + non_cached_buffer = compressed_buffer; data_buffer = non_cached_buffer.get(); } } @@ -354,8 +361,8 @@ void MergeTreeReader::Stream::seekToStart() } -void MergeTreeReader::addStreams(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges, - const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type) +void MergeTreeReader::addStreams(const String & name, const IDataType & type, const CompressionCodecPtr & codec, + const MarkRanges & all_mark_ranges, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type) { IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path) { @@ -374,7 +381,7 @@ void MergeTreeReader::addStreams(const String & name, const IDataType & type, co streams.emplace(stream_name, std::make_unique( path + stream_name, DATA_FILE_EXTENSION, data_part->marks_count, - all_mark_ranges, mark_cache, save_marks_in_cache, + codec, all_mark_ranges, mark_cache, save_marks_in_cache, uncompressed_cache, aio_threshold, max_read_buffer_size, profile_callback, clock_type)); }; diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index dff0dbf2eae..73fcb1977e9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -63,6 +63,7 @@ private: public: Stream( const String & path_prefix_, const String & extension_, size_t marks_count_, + const CompressionCodecPtr & codec, const MarkRanges & all_mark_ranges, MarkCache * mark_cache, bool save_marks_in_cache, UncompressedCache * uncompressed_cache, @@ -91,8 +92,9 @@ private: bool save_marks_in_cache; MarkCache::MappedPtr marks; - std::unique_ptr cached_buffer; - std::unique_ptr non_cached_buffer; + std::unique_ptr file_in; + std::shared_ptr cached_buffer; + std::shared_ptr non_cached_buffer; }; using FileStreams = std::map>; @@ -121,8 +123,8 @@ private: size_t max_read_buffer_size; size_t index_granularity; - void addStreams(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges, - const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); + void addStreams(const String & name, const IDataType & type, const CompressionCodecPtr & codec, + const MarkRanges & all_mark_ranges, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); void readData( const String & name, const IDataType & type, IColumn & column, diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index a79b0681e9e..85dab8a2d53 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -41,6 +41,7 @@ void IMergedBlockOutputStream::addStreams( const String & path, const String & name, const IDataType & type, + const CompressionCodecPtr & codec, size_t estimated_size, bool skip_offsets) { @@ -59,8 +60,8 @@ void IMergedBlockOutputStream::addStreams( stream_name, path + stream_name, DATA_FILE_EXTENSION, path + stream_name, MARKS_FILE_EXTENSION, + codec, max_compress_block_size, - compression_settings, estimated_size, aio_threshold); }; @@ -183,15 +184,15 @@ IMergedBlockOutputStream::ColumnStream::ColumnStream( const std::string & data_file_extension_, const std::string & marks_path, const std::string & marks_file_extension_, + const CompressionCodecPtr & compression_codec, size_t max_compress_block_size, - CompressionSettings compression_settings, size_t estimated_size, size_t aio_threshold) : escaped_column_name(escaped_column_name_), data_file_extension{data_file_extension_}, marks_file_extension{marks_file_extension_}, plain_file(createWriteBufferFromFileBase(data_path + data_file_extension, estimated_size, aio_threshold, max_compress_block_size)), - plain_hashing(*plain_file), compressed_buf(plain_hashing, compression_settings), compressed(compressed_buf), + plain_hashing(*plain_file), compressed_buf(compression_codec->liftCompressed(plain_hashing)), compressed(*compressed_buf.get()), marks_file(marks_path + marks_file_extension, 4096, O_TRUNC | O_CREAT | O_WRONLY), marks(marks_file) { } @@ -239,7 +240,10 @@ MergedBlockOutputStream::MergedBlockOutputStream( { init(); for (const auto & it : columns_list) - addStreams(part_path, it.name, *it.type, 0, false); + { + const auto columns = storage.getColumns(); + addStreams(part_path, it.name, *it.type, columns.getCodec(it.name, compression_settings), 0, false); + } } MergedBlockOutputStream::MergedBlockOutputStream( @@ -270,7 +274,10 @@ MergedBlockOutputStream::MergedBlockOutputStream( } for (const auto & it : columns_list) - addStreams(part_path, it.name, *it.type, total_size, false); + { + const auto columns = storage.getColumns(); + addStreams(part_path, it.name, *it.type, columns.getCodec(it.name, compression_settings), total_size, false); + } } std::string MergedBlockOutputStream::getPartPath() const @@ -532,7 +539,8 @@ void MergedColumnOnlyOutputStream::write(const Block & block) { const auto & col = block.safeGetByPosition(i); - addStreams(part_path, col.name, *col.type, 0, skip_offsets); + const auto columns = storage.getColumns(); + addStreams(part_path, col.name, *col.type, columns.getCodec(col.name, compression_settings), 0, skip_offsets); serialization_states.emplace_back(nullptr); settings.getter = createStreamGetter(col.name, tmp_offset_columns, false); col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back()); diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h index a3f6a025c31..956aa15cc51 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -37,8 +37,8 @@ protected: const std::string & data_file_extension_, const std::string & marks_path, const std::string & marks_file_extension_, + const CompressionCodecPtr & compression_codec, size_t max_compress_block_size, - CompressionSettings compression_settings, size_t estimated_size, size_t aio_threshold); @@ -49,7 +49,7 @@ protected: /// compressed -> compressed_buf -> plain_hashing -> plain_file std::unique_ptr plain_file; HashingWriteBuffer plain_hashing; - CompressedWriteBuffer compressed_buf; + WriteBufferPtr compressed_buf; HashingWriteBuffer compressed; /// marks -> marks_file @@ -65,7 +65,8 @@ protected: using ColumnStreams = std::map>; - void addStreams(const String & path, const String & name, const IDataType & type, size_t estimated_size, bool skip_offsets); + void addStreams(const String & path, const String & name, const IDataType & type, + const CompressionCodecPtr & codec, size_t estimated_size, bool skip_offsets); IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenOffsetColumns & offset_columns, bool skip_offsets); diff --git a/dbms/tests/queries/0_stateless/00732_compression_codec.sql b/dbms/tests/queries/0_stateless/00732_compression_codec.sql new file mode 100644 index 00000000000..044a5c0db98 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00732_compression_codec.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS test.compression_codec; + +CREATE TABLE test.compression_codec(day Date CODEC(ZSTD), its UInt32 CODEC(Delta(UInt32), LZ4HC(2))) + +INSERT INTO test.compression_codec('2018-01-01', '') + +SELECT * FROM test.compression_codec; + +DROP TABLE IF EXISTS test.compression_codec; From 5c30bd2384e2cf8d771049211740a71088617fb5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 13 Dec 2018 13:25:11 +0300 Subject: [PATCH 020/230] Compilable code --- dbms/CMakeLists.txt | 3 +++ dbms/src/Compression/CompressionCodecMultiple.cpp | 4 ++-- dbms/src/IO/tests/CMakeLists.txt | 2 +- dbms/src/IO/tests/cached_compressed_read_buffer.cpp | 5 +++-- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index bac6d8776ca..109b289839d 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -178,6 +178,9 @@ target_link_libraries (clickhouse_common_io target_link_libraries (dbms PRIVATE + ${LZ4_LIBRARY} + ${ZSTD_LIBRARY} + PRIVATE clickhouse_parsers clickhouse_common_config PUBLIC diff --git a/dbms/src/Compression/CompressionCodecMultiple.cpp b/dbms/src/Compression/CompressionCodecMultiple.cpp index 0795990e52c..58387605110 100644 --- a/dbms/src/Compression/CompressionCodecMultiple.cpp +++ b/dbms/src/Compression/CompressionCodecMultiple.cpp @@ -76,7 +76,7 @@ size_t CompressionCodecMultiple::decompress(char * source, size_t source_size, c PODArray compressed_buf; PODArray uncompressed_buf; - compressed_buf.insert(&source[compression_methods_size + 1], source_size - (compression_methods_size + 1)); + compressed_buf.insert(&source[compression_methods_size + 1], &source[source_size - (compression_methods_size + 1)]); for (size_t idx = 0; idx < compression_methods_size; ++idx) { @@ -90,4 +90,4 @@ size_t CompressionCodecMultiple::decompress(char * source, size_t source_size, c return decompressed_size; } -} \ No newline at end of file +} diff --git a/dbms/src/IO/tests/CMakeLists.txt b/dbms/src/IO/tests/CMakeLists.txt index 75398ff9801..59a1e0088c2 100644 --- a/dbms/src/IO/tests/CMakeLists.txt +++ b/dbms/src/IO/tests/CMakeLists.txt @@ -44,7 +44,7 @@ add_executable (mempbrk mempbrk.cpp) target_link_libraries (mempbrk PRIVATE clickhouse_common_io) add_executable (cached_compressed_read_buffer cached_compressed_read_buffer.cpp) -target_link_libraries (cached_compressed_read_buffer PRIVATE clickhouse_common_io) +target_link_libraries (cached_compressed_read_buffer PRIVATE clickhouse_common_io dbms ) add_executable (o_direct_and_dirty_pages o_direct_and_dirty_pages.cpp) target_link_libraries (o_direct_and_dirty_pages PRIVATE clickhouse_common_io) diff --git a/dbms/src/IO/tests/cached_compressed_read_buffer.cpp b/dbms/src/IO/tests/cached_compressed_read_buffer.cpp index 92076db2a99..32c6fb16fb7 100644 --- a/dbms/src/IO/tests/cached_compressed_read_buffer.cpp +++ b/dbms/src/IO/tests/cached_compressed_read_buffer.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -31,7 +32,7 @@ int main(int argc, char ** argv) { Stopwatch watch; - CachedCompressedReadBuffer in(path, &cache, 0, 0); + CachedCompressedReadBuffer in(path, &cache, CompressionCodecFactory::instance().getDefaultCodec(), 0, 0); WriteBufferFromFile out("/dev/null"); copyData(in, out); @@ -43,7 +44,7 @@ int main(int argc, char ** argv) { Stopwatch watch; - CachedCompressedReadBuffer in(path, &cache, 0, 0); + CachedCompressedReadBuffer in(path, &cache, CompressionCodecFactory::instance().getDefaultCodec(), 0, 0); WriteBufferFromFile out("/dev/null"); copyData(in, out); From a492d0f6bff4658551a9c570e2ce7f1c19e98163 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 13 Dec 2018 18:26:28 +0300 Subject: [PATCH 021/230] Seems like working parser --- dbms/src/Compression/CompressionFactory.cpp | 10 +- dbms/src/Compression/ICompressionCodec.h | 1 + dbms/src/Parsers/ExpressionElementParsers.cpp | 31 ++++ dbms/src/Parsers/ExpressionElementParsers.h | 17 ++ dbms/src/Parsers/ParserCreateQuery.h | 163 ++++++++---------- ...4_test_custom_compression_codecs.reference | 4 + .../00804_test_custom_compression_codecs.sql | 11 ++ 7 files changed, 134 insertions(+), 103 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference create mode 100644 dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql diff --git a/dbms/src/Compression/CompressionFactory.cpp b/dbms/src/Compression/CompressionFactory.cpp index c8eb8981185..32d2cb6cd6d 100644 --- a/dbms/src/Compression/CompressionFactory.cpp +++ b/dbms/src/Compression/CompressionFactory.cpp @@ -33,12 +33,6 @@ CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast) const { if (const auto * func = typeid_cast(ast.get())) { - if (func->parameters) - throw Exception("Compression codec cannot have multiple parenthesed parameters.", ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); - - if (Poco::toLower(func->name) != "codec") - throw Exception("", ErrorCodes::UNKNOWN_CODEC); - Codecs codecs; codecs.reserve(func->arguments->children.size()); for (const auto & inner_codec_ast : func->arguments->children) @@ -48,7 +42,7 @@ CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast) const else if (const auto * ast_func = typeid_cast(inner_codec_ast.get())) codecs.emplace_back(getImpl(ast_func->name, ast_func->arguments)); else - throw Exception("Unexpected AST element for compression codec.", ErrorCodes::UNEXPECTED_AST_STRUCTURE); + throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE); } if (codecs.size() == 1) @@ -118,4 +112,4 @@ CompressionCodecFactory::CompressionCodecFactory() // registerCodecDelta(*this); } -} \ No newline at end of file +} diff --git a/dbms/src/Compression/ICompressionCodec.h b/dbms/src/Compression/ICompressionCodec.h index 7238dfbdbee..9e818ef969a 100644 --- a/dbms/src/Compression/ICompressionCodec.h +++ b/dbms/src/Compression/ICompressionCodec.h @@ -80,6 +80,7 @@ public: virtual char getMethodByte() = 0; + /// TODO(alesap) FIXME virtual void getCodecDesc(String & codec_desc) = 0; virtual size_t compress(char * source, size_t source_size, char * dest) = 0; diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 0912d2a5b7b..3c3207c6433 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -5,6 +5,7 @@ #include #include +#include #include #include @@ -25,6 +26,7 @@ #include #include +#include namespace DB @@ -281,6 +283,35 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return true; } +bool ParserCodecDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + return ParserList(std::make_unique(), + std::make_unique(TokenType::Comma), false).parse(pos, node, expected); +} + +bool ParserCodec::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserCodecDeclarationList codecs; + ASTPtr expr_list_args; + + if (pos->type != TokenType::OpeningRoundBracket) + return false; + + ++pos; + if (!codecs.parse(pos, expr_list_args, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + auto function_node = std::make_shared(); + function_node->arguments = expr_list_args; + function_node->children.push_back(function_node->arguments); + + node = function_node; + return true; +} bool ParserCastExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { diff --git a/dbms/src/Parsers/ExpressionElementParsers.h b/dbms/src/Parsers/ExpressionElementParsers.h index a52864d97d1..333a13a20ff 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.h +++ b/dbms/src/Parsers/ExpressionElementParsers.h @@ -89,6 +89,23 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); }; +class ParserCodecDeclarationList : public IParserBase +{ +protected: + const char * getName() const { return "codec declaration list"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); +}; + +/** Parse compression codec + * CODEC(ZSTD(2)) + */ +class ParserCodec : public IParserBase +{ +protected: + const char * getName() const { return "codec"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); +}; + class ParserCastExpression : public IParserBase { protected: diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 429ecbb09d1..27b4cd21fbd 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -108,16 +108,6 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); - bool isDeclareColumnType(Pos & pos, Expected & expected); - - bool parseDeclarationCodec(Pos &pos, const ASTDeclarePtr &declaration, Expected &expected); - - bool parseDefaultExpression(Pos &pos, const ASTDeclarePtr &declaration, Expected &expected); - - bool parseDeclarationComment(Pos &pos, const ASTDeclarePtr &declaration, Expected &expected); - - bool isDeclareColumnCodec(Pos & pos, Expected & expected); - bool require_type = true; }; @@ -127,112 +117,95 @@ using ParserCompoundColumnDeclaration = IParserColumnDeclaration bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - ASTPtr column_name; - ASTPtr column_type; - ASTPtr column_codec; NameParser name_parser; ParserIdentifierWithOptionalParameters type_parser; - - if (!name_parser.parse(pos, column_name, expected)) - return false; - - if (isDeclareColumnType(pos, expected) - && !type_parser.parse(pos, column_type, expected)) - return false; - - const auto column_declaration = std::make_shared(); - - if (!parseDefaultExpression(pos, column_declaration, expected) - || !parseDeclarationCodec(pos, column_declaration, expected) - || !parseDeclarationComment(pos, column_declaration, expected)) - return false; - - if (require_type && !column_type && column_declaration->default_expression) - return false; - - if (column_type) - { - column_declaration->type = column_type; - column_declaration->children.push_back(std::move(column_type)); - } - - node = column_declaration; - return true; -} - -template -bool IParserColumnDeclaration::isDeclareColumnType(Pos & pos, Expected & expected) -{ - auto check_pos = pos; - return !ParserKeyword{"CODEC"}.check(check_pos, expected) && - !ParserKeyword{"ALIAS"}.check(check_pos, expected) && - !ParserKeyword{"COMMENT"}.check(check_pos, expected) && - !ParserKeyword{"DEFAULT"}.check(check_pos, expected) && - !ParserKeyword{"MATERIALIZED"}.check(check_pos, expected); -} -template -bool IParserColumnDeclaration::isDeclareColumnCodec(Pos & pos, Expected & expected) -{ - auto check_pos = pos; - return ParserKeyword{"CODEC"}.check(check_pos, expected); -} - -template -bool IParserColumnDeclaration::parseDeclarationCodec(Pos & pos, const ASTDeclarePtr & declaration, Expected & expected) -{ - ParserKeyword s_codec{"CODEC"}; - - ParserIdentifierWithParameters codec_parser; - - if (s_codec.ignore(pos, expected)) - { - if (!codec_parser.parse(pos, declaration->codec, expected)) - return false; - - declaration->children.push_back(declaration->codec); - } - - return true; -} - -template -bool IParserColumnDeclaration::parseDefaultExpression(Pos & pos, const ASTDeclarePtr & declaration, Expected & expected) -{ - ParserKeyword s_alias{"ALIAS"}; ParserKeyword s_default{"DEFAULT"}; ParserKeyword s_materialized{"MATERIALIZED"}; - + ParserKeyword s_alias{"ALIAS"}; + ParserKeyword s_comment{"COMMENT"}; + ParserKeyword s_codec{"CODEC"}; ParserTernaryOperatorExpression expr_parser; + ParserStringLiteral string_literal_parser; + ParserCodec codec_parser; + + /// mandatory column name + ASTPtr name; + if (!name_parser.parse(pos, name, expected)) + return false; + + /** column name should be followed by type name if it + * is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS, COMMENT} + */ + ASTPtr type; + String default_specifier; + ASTPtr default_expression; + ASTPtr comment_expression; + ASTPtr codec_expression; + + if (!s_default.check_without_moving(pos, expected) && + !s_materialized.check_without_moving(pos, expected) && + !s_alias.check_without_moving(pos, expected) && + !s_comment.check_without_moving(pos, expected) && + !s_codec.check_without_moving(pos, expected)) + { + if (!type_parser.parse(pos, type, expected)) + return false; + } Pos pos_before_specifier = pos; if (s_default.ignore(pos, expected) || s_materialized.ignore(pos, expected) || s_alias.ignore(pos, expected)) { - declaration->default_specifier = Poco::toUpper(std::string{pos_before_specifier->begin, pos_before_specifier->end}); + default_specifier = Poco::toUpper(std::string{pos_before_specifier->begin, pos_before_specifier->end}); /// should be followed by an expression - if (!expr_parser.parse(pos, declaration->default_expression, expected)) + if (!expr_parser.parse(pos, default_expression, expected)) return false; - - declaration->children.push_back(declaration->default_expression); } - return true; -} + if (require_type && !type && !default_expression) + return false; /// reject column name without type -template -bool IParserColumnDeclaration::parseDeclarationComment(Pos & pos, const ASTDeclarePtr & declaration, Expected & expected) -{ - ParserKeyword s_comment{"COMMENT"}; - - ParserStringLiteral string_literal_parser; if (s_comment.ignore(pos, expected)) { /// should be followed by a string literal - if (!string_literal_parser.parse(pos, declaration->comment, expected)) + if (!string_literal_parser.parse(pos, comment_expression, expected)) return false; + } - declaration->children.push_back(declaration->comment); + if (s_codec.ignore(pos, expected)) + { + if (!codec_parser.parse(pos, codec_expression, expected)) + return false; + } + + const auto column_declaration = std::make_shared(); + node = column_declaration; + column_declaration->name = typeid_cast(*name).name; + + if (type) + { + column_declaration->type = type; + column_declaration->children.push_back(std::move(type)); + } + + if (default_expression) + { + column_declaration->default_specifier = default_specifier; + column_declaration->default_expression = default_expression; + column_declaration->children.push_back(std::move(default_expression)); + } + + if (comment_expression) + { + column_declaration->comment = comment_expression; + column_declaration->children.push_back(std::move(comment_expression)); + } + + if (codec_expression) + { + column_declaration->codec = codec_expression; + column_declaration->children.push_back(std::move(codec_expression)); } return true; diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference new file mode 100644 index 00000000000..38df1ff4364 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference @@ -0,0 +1,4 @@ +1 hello +2 world +3 ! + diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql new file mode 100644 index 00000000000..2b16b56744f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS test.compression_codec; + +CREATE TABLE test.compression_codec(id UInt64 CODEC(LZ4), data String CODEC(ZSTD)) ENGINE = MergeTree() order by tuple(); + +INSERT INTO test.compression_codec VALUES(1, 'hello'); +INSERT INTO test.compression_codec VALUES(2, 'world'); +INSERT INTO test.compression_codec VALUES(3, '!'); + +SELECT * FROM test.compression_codec; + +DROP TABLE IF EXISTS test.compression_codec; From d7bc4ae11574537289f9a7273b44fa6acffe02e8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 13 Dec 2018 19:21:08 +0300 Subject: [PATCH 022/230] Seems like first simpliest working code --- dbms/src/Compression/ICompressionCodec.cpp | 26 ++++++++++++++++++- ...4_test_custom_compression_codecs.reference | 1 - .../00804_test_custom_compression_codecs.sql | 2 +- 3 files changed, 26 insertions(+), 3 deletions(-) diff --git a/dbms/src/Compression/ICompressionCodec.cpp b/dbms/src/Compression/ICompressionCodec.cpp index e82fb891318..6989d52c5e4 100644 --- a/dbms/src/Compression/ICompressionCodec.cpp +++ b/dbms/src/Compression/ICompressionCodec.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace ProfileEvents { @@ -95,9 +96,22 @@ void CompressionCodecReadBuffer::decompress(char * to, size_t size_decompressed, ProfileEvents::increment(ProfileEvents::CompressedReadBufferBlocks); ProfileEvents::increment(ProfileEvents::CompressedReadBufferBytes, size_decompressed); + std::cerr << "Size:" << size_compressed_without_checksum << std::endl; + std::cerr << "Compressed buffer:\n"; + std::cerr << std::hex << std::endl; + for (size_t i = 0; i < size_compressed_without_checksum; ++i) + { + std::cerr << +(compressed_buffer[i]) << " "; + } + std::cerr << "\n"; UInt8 current_method = compressed_buffer[0]; /// See CompressedWriteBuffer.h + std::cerr << "CURRENT METHOD:" << +current_method << std::endl; + std::cerr << std::dec << std::endl; if (current_method != method) - codec = CompressionCodecFactory::instance().get(method); + { + method = current_method; + codec = CompressionCodecFactory::instance().get(current_method); + } codec->decompress(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, size_compressed_without_checksum - COMPRESSED_BLOCK_HEADER_SIZE, to, size_decompressed); @@ -177,6 +191,11 @@ void CompressionCodecWriteBuffer::nextImpl() compressed_buffer.resize(header_size + compressed_reserve_size); compressed_buffer[0] = compression_codec.getMethodByte(); + std::cerr << "Total compressed buffer size:" << compressed_buffer.size() << std::endl; + std::cerr << "Header size:" << header_size << std::endl; + std::cerr << std::hex << std::endl; + std::cerr << "Original byte:" << +compression_codec.getMethodByte() << std::endl; + std::cerr << std::dec << std::endl; size_t compressed_size = header_size + compression_codec.compress(working_buffer.begin(), uncompressed_size, &compressed_buffer[header_size]); UInt32 compressed_size_32 = compressed_size; @@ -184,6 +203,11 @@ void CompressionCodecWriteBuffer::nextImpl() unalignedStore(&compressed_buffer[1], compressed_size_32); unalignedStore(&compressed_buffer[5], uncompressed_size_32); CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(compressed_buffer.data(), compressed_size); + std::cerr << std::hex << std::endl; + std::cerr << "Codec sign:" << +(compressed_buffer[0]) << std::endl; + std::cerr << std::dec << std::endl; + std::cerr << "CHeckSum first byte:" << checksum.first << std::endl; + std::cerr << "CHeckSum second byte:" << checksum.second << std::endl; out.write(reinterpret_cast(&checksum), sizeof(checksum)); out.write(compressed_buffer.data(), compressed_size); } diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference index 38df1ff4364..c67c9c6f042 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference @@ -1,4 +1,3 @@ 1 hello 2 world 3 ! - diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 2b16b56744f..60b09a8c7c6 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -6,6 +6,6 @@ INSERT INTO test.compression_codec VALUES(1, 'hello'); INSERT INTO test.compression_codec VALUES(2, 'world'); INSERT INTO test.compression_codec VALUES(3, '!'); -SELECT * FROM test.compression_codec; +SELECT * FROM test.compression_codec order by id; DROP TABLE IF EXISTS test.compression_codec; From 5c823d519fcdd3342589f8befdd2293fa4a0a3ae Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 13 Dec 2018 19:23:06 +0300 Subject: [PATCH 023/230] Remove debug --- dbms/src/Compression/ICompressionCodec.cpp | 21 --------------------- 1 file changed, 21 deletions(-) diff --git a/dbms/src/Compression/ICompressionCodec.cpp b/dbms/src/Compression/ICompressionCodec.cpp index 6989d52c5e4..b3216294820 100644 --- a/dbms/src/Compression/ICompressionCodec.cpp +++ b/dbms/src/Compression/ICompressionCodec.cpp @@ -8,7 +8,6 @@ #include #include #include -#include namespace ProfileEvents { @@ -96,17 +95,7 @@ void CompressionCodecReadBuffer::decompress(char * to, size_t size_decompressed, ProfileEvents::increment(ProfileEvents::CompressedReadBufferBlocks); ProfileEvents::increment(ProfileEvents::CompressedReadBufferBytes, size_decompressed); - std::cerr << "Size:" << size_compressed_without_checksum << std::endl; - std::cerr << "Compressed buffer:\n"; - std::cerr << std::hex << std::endl; - for (size_t i = 0; i < size_compressed_without_checksum; ++i) - { - std::cerr << +(compressed_buffer[i]) << " "; - } - std::cerr << "\n"; UInt8 current_method = compressed_buffer[0]; /// See CompressedWriteBuffer.h - std::cerr << "CURRENT METHOD:" << +current_method << std::endl; - std::cerr << std::dec << std::endl; if (current_method != method) { method = current_method; @@ -191,11 +180,6 @@ void CompressionCodecWriteBuffer::nextImpl() compressed_buffer.resize(header_size + compressed_reserve_size); compressed_buffer[0] = compression_codec.getMethodByte(); - std::cerr << "Total compressed buffer size:" << compressed_buffer.size() << std::endl; - std::cerr << "Header size:" << header_size << std::endl; - std::cerr << std::hex << std::endl; - std::cerr << "Original byte:" << +compression_codec.getMethodByte() << std::endl; - std::cerr << std::dec << std::endl; size_t compressed_size = header_size + compression_codec.compress(working_buffer.begin(), uncompressed_size, &compressed_buffer[header_size]); UInt32 compressed_size_32 = compressed_size; @@ -203,11 +187,6 @@ void CompressionCodecWriteBuffer::nextImpl() unalignedStore(&compressed_buffer[1], compressed_size_32); unalignedStore(&compressed_buffer[5], uncompressed_size_32); CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(compressed_buffer.data(), compressed_size); - std::cerr << std::hex << std::endl; - std::cerr << "Codec sign:" << +(compressed_buffer[0]) << std::endl; - std::cerr << std::dec << std::endl; - std::cerr << "CHeckSum first byte:" << checksum.first << std::endl; - std::cerr << "CHeckSum second byte:" << checksum.second << std::endl; out.write(reinterpret_cast(&checksum), sizeof(checksum)); out.write(compressed_buffer.data(), compressed_size); } From 96da3195f5e8823030a67cbc0bad90ef4e90a493 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 13 Dec 2018 20:39:16 +0300 Subject: [PATCH 024/230] Rewrite strange logic to another strange logic --- dbms/src/Compression/ICompressionCodec.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Compression/ICompressionCodec.cpp b/dbms/src/Compression/ICompressionCodec.cpp index b3216294820..09b16b43b96 100644 --- a/dbms/src/Compression/ICompressionCodec.cpp +++ b/dbms/src/Compression/ICompressionCodec.cpp @@ -96,10 +96,10 @@ void CompressionCodecReadBuffer::decompress(char * to, size_t size_decompressed, ProfileEvents::increment(ProfileEvents::CompressedReadBufferBytes, size_decompressed); UInt8 current_method = compressed_buffer[0]; /// See CompressedWriteBuffer.h - if (current_method != method) + if (!codec || current_method != method) { method = current_method; - codec = CompressionCodecFactory::instance().get(current_method); + codec = CompressionCodecFactory::instance().get(method); } codec->decompress(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, From 3370a0a4fe1bd9d670b77e45510df0c2077472c3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 14 Dec 2018 14:29:12 +0300 Subject: [PATCH 025/230] Fix order of calls --- dbms/src/Interpreters/InterpreterCreateQuery.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 5ddfa2de100..2b27aa11b7b 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -291,7 +291,7 @@ static ColumnsDeclarationAndModifiers parseColumns(const ASTExpressionList & col } -static NamesAndTypesList removeAndReturnColumns(ColumnsDeclarationAndModifiers & columns_declare, const ColumnDefaultKind kind) +static NamesAndTypesList removeAndReturnColumns(ColumnsAndDefaults & columns_declare, const ColumnDefaultKind kind) { auto & columns = std::get<0>(columns_declare); auto & defaults = std::get<1>(columns_declare); @@ -391,12 +391,13 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpres ColumnsDescription res; auto && parsed_columns = parseColumns(columns, context); - res.ordinary = std::move(std::get<0>(parsed_columns)); - res.defaults = std::move(std::get<1>(parsed_columns)); + auto columns_and_defaults = std::make_pair(std::move(std::get<0>(parsed_columns)), std::move(std::get<1>(parsed_columns))); + res.aliases = removeAndReturnColumns(columns_and_defaults, ColumnDefaultKind::Alias); + res.materialized = removeAndReturnColumns(columns_and_defaults, ColumnDefaultKind::Materialized); + res.ordinary = std::move(columns_and_defaults.first); + res.defaults = std::move(columns_and_defaults.second); res.codecs = std::move(std::get<2>(parsed_columns)); res.comments = std::move(std::get<3>(parsed_columns)); - res.aliases = removeAndReturnColumns(parsed_columns, ColumnDefaultKind::Alias); - res.materialized = removeAndReturnColumns(parsed_columns, ColumnDefaultKind::Materialized); if (res.ordinary.size() + res.materialized.size() == 0) throw Exception{"Cannot CREATE table without physical columns", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED}; From cc229ac9ccb5372c3b64026613c709fe6a5734b6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 14 Dec 2018 14:32:23 +0300 Subject: [PATCH 026/230] Remove strange test --- .../queries/0_stateless/00732_compression_codec.sql | 9 --------- 1 file changed, 9 deletions(-) delete mode 100644 dbms/tests/queries/0_stateless/00732_compression_codec.sql diff --git a/dbms/tests/queries/0_stateless/00732_compression_codec.sql b/dbms/tests/queries/0_stateless/00732_compression_codec.sql deleted file mode 100644 index 044a5c0db98..00000000000 --- a/dbms/tests/queries/0_stateless/00732_compression_codec.sql +++ /dev/null @@ -1,9 +0,0 @@ -DROP TABLE IF EXISTS test.compression_codec; - -CREATE TABLE test.compression_codec(day Date CODEC(ZSTD), its UInt32 CODEC(Delta(UInt32), LZ4HC(2))) - -INSERT INTO test.compression_codec('2018-01-01', '') - -SELECT * FROM test.compression_codec; - -DROP TABLE IF EXISTS test.compression_codec; From 8a95eb9dc12c29235a1e5fe44c436efe77a9430b Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 14 Dec 2018 16:27:35 +0300 Subject: [PATCH 027/230] More tests for parser --- dbms/src/Compression/CompressionCodecZSTD.cpp | 5 ++++ ...4_test_custom_compression_codecs.reference | 7 +++-- .../00804_test_custom_compression_codecs.sql | 29 +++++++++++++++---- 3 files changed, 33 insertions(+), 8 deletions(-) diff --git a/dbms/src/Compression/CompressionCodecZSTD.cpp b/dbms/src/Compression/CompressionCodecZSTD.cpp index f907ae93498..9da1454a539 100644 --- a/dbms/src/Compression/CompressionCodecZSTD.cpp +++ b/dbms/src/Compression/CompressionCodecZSTD.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -15,6 +16,7 @@ namespace ErrorCodes { extern const int CANNOT_COMPRESS; extern const int CANNOT_DECOMPRESS; + extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; } char CompressionCodecZSTD::getMethodByte() @@ -66,6 +68,9 @@ void registerCodecZSTD(CompressionCodecFactory & factory) int level = 0; if (arguments && !arguments->children.empty()) { + if (arguments->children.size() != 1) + throw Exception("ZSTD codec must have 1 parameter, given " + std::to_string(arguments->children.size()), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); + const auto children = arguments->children; const ASTLiteral * literal = static_cast(children[0].get()); level = literal->value.safeGet(); diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference index c67c9c6f042..fd1689b374b 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference @@ -1,3 +1,4 @@ -1 hello -2 world -3 ! +1 hello 2018-12-14 1.1 +2 world 2018-12-15 2.2 +3 ! 2018-12-16 3.3 +2 diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 60b09a8c7c6..3991232a010 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -1,11 +1,30 @@ +SET send_logs_level = 'none'; DROP TABLE IF EXISTS test.compression_codec; -CREATE TABLE test.compression_codec(id UInt64 CODEC(LZ4), data String CODEC(ZSTD)) ENGINE = MergeTree() order by tuple(); +CREATE TABLE test.compression_codec(id UInt64 CODEC(LZ4), data String CODEC(ZSTD), ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2))) ENGINE = MergeTree() ORDER BY tuple(); -INSERT INTO test.compression_codec VALUES(1, 'hello'); -INSERT INTO test.compression_codec VALUES(2, 'world'); -INSERT INTO test.compression_codec VALUES(3, '!'); +INSERT INTO test.compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1); +INSERT INTO test.compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2); +INSERT INTO test.compression_codec VALUES(3, '!', toDate('2018-12-16'), 3.3); -SELECT * FROM test.compression_codec order by id; +SELECT * FROM test.compression_codec ORDER BY id; + +OPTIMIZE TABLE test.compression_codec FINAL; + +INSERT INTO test.compression_codec VALUES(2, '', toDate('2018-12-13'), 4.4); + +SELECT count(*) FROM test.compression_codec WHERE id = 2 GROUP BY id; DROP TABLE IF EXISTS test.compression_codec; + +DROP TABLE IF EXISTS test.bad_codec; +DROP TABLE IF EXISTS test.params_when_no_params; +DROP TABLE IF EXISTS test.too_many_params; + +CREATE TABLE test.bad_codec(id UInt64 CODEC(adssadads)) ENGINE = MergeTree() order by tuple(); -- { serverError 429 } +CREATE TABLE test.too_many_params(id UInt64 CODEC(ZSTD(2,3,4,5))) ENGINE = MergeTree() order by tuple(); -- { serverError 428 } +CREATE TABLE test.params_when_no_params(id UInt64 CODEC(LZ4(1))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 378 } + +DROP TABLE IF EXISTS test.bad_codec; +DROP TABLE IF EXISTS test.params_when_no_params; +DROP TABLE IF EXISTS test.too_many_params; From 0f29bff8daaf4594c6ed26586447558a3eb9e550 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 14 Dec 2018 16:28:34 +0300 Subject: [PATCH 028/230] Style --- dbms/src/Parsers/ASTColumnDeclaration.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Parsers/ASTColumnDeclaration.h b/dbms/src/Parsers/ASTColumnDeclaration.h index 54e52382328..b24b9935dbe 100644 --- a/dbms/src/Parsers/ASTColumnDeclaration.h +++ b/dbms/src/Parsers/ASTColumnDeclaration.h @@ -39,7 +39,7 @@ public: if (codec) { - res->codec=codec->clone(); + res->codec = codec->clone(); res->children.push_back(res->codec); } From 436845ff7ea9c4c11e0eabcc9db21703c002c75c Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 17 Dec 2018 11:31:59 +0300 Subject: [PATCH 029/230] Debuging codec multiple --- .../Compression/CompressionCodecMultiple.cpp | 33 ++++++++++++++----- .../Compression/CompressionCodecMultiple.h | 3 +- dbms/src/Compression/CompressionFactory.cpp | 5 +++ .../00804_test_custom_compression_codecs.sql | 1 + 4 files changed, 32 insertions(+), 10 deletions(-) diff --git a/dbms/src/Compression/CompressionCodecMultiple.cpp b/dbms/src/Compression/CompressionCodecMultiple.cpp index 58387605110..9e8f6adb13f 100644 --- a/dbms/src/Compression/CompressionCodecMultiple.cpp +++ b/dbms/src/Compression/CompressionCodecMultiple.cpp @@ -35,9 +35,10 @@ void CompressionCodecMultiple::getCodecDesc(String & codec_desc_) size_t CompressionCodecMultiple::getCompressedReserveSize(size_t uncompressed_size) { for (auto & codec : codecs) - uncompressed_size = codec->getCompressedReserveSize(uncompressed_size); + uncompressed_size += codec->getCompressedReserveSize(uncompressed_size); - return sizeof(UInt8) + codecs.size() + uncompressed_size; + /// MultipleCodecByte TotalCodecs ByteForEachCodec data + return sizeof(UInt8) + sizeof(UInt8) + codecs.size() + uncompressed_size; } size_t CompressionCodecMultiple::compress(char * source, size_t source_size, char * dest) @@ -48,11 +49,15 @@ size_t CompressionCodecMultiple::compress(char * source, size_t source_size, cha PODArray uncompressed_buf(source_size); uncompressed_buf.insert(source, source + source_size); - dest[0] = static_cast(codecs.size()); - for (size_t idx = 0; idx < codecs.size(); ++idx) + dest[0] = static_cast(getMethodByte()); + dest[1] = static_cast(codecs.size()); + std::cerr << "(compress) codecs size:" << codecs.size() << std::endl; + std::cerr << "(compress) desc:" << codec_desc << std::endl; + size_t codecs_byte_pos = 2; + for (size_t idx = 0; idx < codecs.size(); ++idx, ++codecs_byte_pos) { const auto codec = codecs[idx]; - dest[idx + 1] = codec->getMethodByte(); + dest[codecs_byte_pos] = codec->getMethodByte(); compressed_buf.resize(without_method_header_size + codec->getCompressedReserveSize(source_size)); size_t size_compressed = without_method_header_size; @@ -66,21 +71,24 @@ size_t CompressionCodecMultiple::compress(char * source, size_t source_size, cha source_size = size_compressed; } - memcpy(&dest[codecs.size() + 1], &compressed_buf[0], source_size); + memcpy(&dest[2 + codecs.size()], &compressed_buf[0], source_size); return source_size; } size_t CompressionCodecMultiple::decompress(char * source, size_t source_size, char * dest, size_t decompressed_size) { - UInt8 compression_methods_size = source[0]; + UInt8 compression_methods_size = source[1]; + std::cerr << "(decompress) Methods size:" << +compression_methods_size << std::endl; PODArray compressed_buf; PODArray uncompressed_buf; - compressed_buf.insert(&source[compression_methods_size + 1], &source[source_size - (compression_methods_size + 1)]); + compressed_buf.insert(&source[compression_methods_size + 2], &source[source_size - (compression_methods_size + 2)]); for (size_t idx = 0; idx < compression_methods_size; ++idx) { - UInt8 compression_method = source[idx + 1]; + UInt8 compression_method = source[idx + 2]; + std::cerr << std::hex; + std::cerr << "(decompress) Compression method byte:" << +compression_method << std::endl; const auto codec = CompressionCodecFactory::instance().get(compression_method); codec->decompress(&compressed_buf[8], 0, uncompressed_buf.data(), 0); uncompressed_buf.swap(compressed_buf); @@ -90,4 +98,11 @@ size_t CompressionCodecMultiple::decompress(char * source, size_t source_size, c return decompressed_size; } +void registerCodecMultiple(CompressionCodecFactory & factory) +{ + factory.registerSimpleCompressionCodec("Multiple", static_cast(CompressionMethodByte::Multiple), [&](){ + return std::make_shared(); + }); +} + } diff --git a/dbms/src/Compression/CompressionCodecMultiple.h b/dbms/src/Compression/CompressionCodecMultiple.h index 7457eec35e2..cea37956ffb 100644 --- a/dbms/src/Compression/CompressionCodecMultiple.h +++ b/dbms/src/Compression/CompressionCodecMultiple.h @@ -8,7 +8,8 @@ namespace DB class CompressionCodecMultiple final : public ICompressionCodec { public: - CompressionCodecMultiple(Codecs codecs); + CompressionCodecMultiple() = default; + explicit CompressionCodecMultiple(Codecs codecs); char getMethodByte() override; diff --git a/dbms/src/Compression/CompressionFactory.cpp b/dbms/src/Compression/CompressionFactory.cpp index 32d2cb6cd6d..dc8ab992a3e 100644 --- a/dbms/src/Compression/CompressionFactory.cpp +++ b/dbms/src/Compression/CompressionFactory.cpp @@ -66,6 +66,9 @@ CompressionCodecPtr CompressionCodecFactory::get(const UInt8 byte_code) const CompressionCodecPtr CompressionCodecFactory::getImpl(const String & family_name, const ASTPtr & arguments) const { + if (family_name == "MULTIPLE") + throw Exception("Codec MULTIPLE cannot be specified directly", ErrorCodes::UNKNOWN_CODEC); + const auto family_and_creator = family_name_with_codec.find(family_name); if (family_and_creator == family_name_with_codec.end()) @@ -101,6 +104,7 @@ void CompressionCodecFactory::registerSimpleCompressionCodec(const String & fami void registerCodecLZ4(CompressionCodecFactory & factory); void registerCodecNone(CompressionCodecFactory & factory); void registerCodecZSTD(CompressionCodecFactory & factory); +void registerCodecMultiple(CompressionCodecFactory & factory); //void registerCodecDelta(CompressionCodecFactory & factory); CompressionCodecFactory::CompressionCodecFactory() @@ -109,6 +113,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecLZ4(*this); registerCodecNone(*this); registerCodecZSTD(*this); + registerCodecMultiple(*this); // registerCodecDelta(*this); } diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 3991232a010..20d791e719f 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -2,6 +2,7 @@ SET send_logs_level = 'none'; DROP TABLE IF EXISTS test.compression_codec; CREATE TABLE test.compression_codec(id UInt64 CODEC(LZ4), data String CODEC(ZSTD), ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2))) ENGINE = MergeTree() ORDER BY tuple(); +CREATE TABLE test.compression_codec(id UInt64 CODEC(LZ4, ZSTD, NONE), data String CODEC(ZSTD), ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2))) ENGINE = MergeTree() ORDER BY tuple(); INSERT INTO test.compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1); INSERT INTO test.compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2); From b38f8050977450c231a238fcd2f1d68f071dabf6 Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 17 Dec 2018 19:20:15 +0300 Subject: [PATCH 030/230] JoinToSubqueryTransformVisitor CLICKHOUSE-3996 --- dbms/src/Interpreters/InDepthNodeVisitor.h | 46 ++++- .../JoinToSubqueryTransformVisitor.cpp | 183 ++++++++++++++++++ .../JoinToSubqueryTransformVisitor.h | 52 +++++ dbms/src/Interpreters/executeQuery.cpp | 9 + 4 files changed, 287 insertions(+), 3 deletions(-) create mode 100644 dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp create mode 100644 dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h diff --git a/dbms/src/Interpreters/InDepthNodeVisitor.h b/dbms/src/Interpreters/InDepthNodeVisitor.h index bdeb8ddb234..5cb73a23776 100644 --- a/dbms/src/Interpreters/InDepthNodeVisitor.h +++ b/dbms/src/Interpreters/InDepthNodeVisitor.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include namespace DB @@ -8,7 +9,7 @@ namespace DB /// Visits AST tree in depth, call functions for nodes according to Matcher type data. /// You need to define Data, label, visit() and needChildVisit() in Matcher class. -template +template class InDepthNodeVisitor { public: @@ -24,7 +25,7 @@ public: { DumpASTNode dump(*ast, ostr, visit_depth, Matcher::label); - if constexpr (!_topToBottom) + if constexpr (!_top_to_bottom) visitChildren(ast); /// It operates with ASTPtr * cause we may want to rewrite ASTPtr in visit(). @@ -34,7 +35,7 @@ public: for (ASTPtr * node : additional_nodes) visit(*node); - if constexpr (_topToBottom) + if constexpr (_top_to_bottom) visitChildren(ast); } @@ -51,4 +52,43 @@ private: } }; +/// Simple matcher for one node type without complex traversal logic. +template +class OneTypeMatcher +{ +public: + using Data = _Data; + using TypeToVisit = typename Data::TypeToVisit; + + static constexpr const char * label = ""; + + static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; } + + static std::vector visit(ASTPtr & ast, Data & data) + { + if (auto * t = typeid_cast(ast.get())) + data.visit(*t, ast); + return {}; + } +}; + +/// Links two simple matches into resulting one. There's no complex traversal logic: all the children would be visited. +template +class LinkedMatcher +{ +public: + using Data = std::pair; + + static constexpr const char * label = ""; + + static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; } + + static std::vector visit(ASTPtr & ast, Data & data) + { + First::visit(ast, data.first); + Second::visit(ast, data.second); + return {}; + } +}; + } diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp new file mode 100644 index 00000000000..25bd13fd6bb --- /dev/null +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -0,0 +1,183 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int TOO_DEEP_AST; +} + +/// Attaches 'with' section to the first visited ASTSelectQuery +struct AppendWithSectionVisitorData +{ + using TypeToVisit = ASTSelectQuery; + + const ASTPtr & with; + bool done = false; + + void visit(ASTSelectQuery & select, ASTPtr &) + { + if (done || !with) + return; + + if (select.with_expression_list) + { + for (auto & expr : with->children) + select.with_expression_list->children.push_back(expr->clone()); + } + else + select.with_expression_list = with->clone(); + done = true; + } +}; + +/// Replaces one table element with pair +struct RewriteTablesVisitorData +{ + using TypeToVisit = ASTTablesInSelectQuery; + + const ASTPtr & left; + const ASTPtr & right; + bool done = false; + + void visit(ASTTablesInSelectQuery &, ASTPtr & ast) + { + if (done) + return; + ast->children.clear(); + ast->children.push_back(left); + ast->children.push_back(right); + done = true; + } +}; + + +static String getTableNameOrAlias(const ASTPtr & table_element) +{ + auto element = static_cast(table_element.get()); + if (!element || element->children.empty()) + throw Exception("Expected TablesInSelectQueryElement with at least one child", ErrorCodes::LOGICAL_ERROR); + + auto table_expression = static_cast(element->children[0].get()); + if (!table_expression || table_expression->children.empty()) + throw Exception("Expected TableExpression with at least one child", ErrorCodes::LOGICAL_ERROR); + + String result = table_expression->children[0]->tryGetAlias(); + if (!result.empty()) + return result; + + auto identifier = static_cast(table_expression->children[0].get()); + if (!identifier) + throw Exception("Expected Identifier or subquery with alias", ErrorCodes::LOGICAL_ERROR); + return identifier->name; +} + +static void addHiddenNames(ASTPtr & with_expression_list, const std::vector & hidden_names, const String & new_name) +{ + if (!with_expression_list) + with_expression_list = std::make_shared(); + + ParserExpression parser; + for (auto & name : hidden_names) + { + if (name.empty()) + continue; + + String str_expression = "nameCoalesce(" + name + "," + new_name + ")"; + ASTPtr expr = parseQuery(parser, str_expression, 0); + if (!expr) + throw Exception("Cannot parse expression", ErrorCodes::LOGICAL_ERROR); + + with_expression_list->children.push_back(expr); + } +} + + +std::vector JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data) +{ + if (auto * t = typeid_cast(ast.get())) + visit(*t, ast, data); + return {}; +} + +void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast, Data & data) +{ + static String alias_prefix = "__join"; /// FIXME + static const size_t max_joins = 64; /// TODO: settings.max_subquery_depth + + auto tables = static_cast(select.tables.get()); + if (!tables) + throw Exception("TablesInSelectQuery expected", ErrorCodes::LOGICAL_ERROR); + + size_t num_tables = tables->children.size(); + if (num_tables <= 2) + return; + + if (num_tables > max_joins) + throw Exception("Too much joins", ErrorCodes::TOO_DEEP_AST); + + ASTPtr left = tables->children[0]; + + for (size_t i = 1; i < num_tables - 1; ++i) + { + ASTPtr right = tables->children[i]; + std::vector hidden_names = {getTableNameOrAlias(left), getTableNameOrAlias(right)}; + String subquery_name = alias_prefix + toString(i); + + left = replaceJoin(left, right, select.with_expression_list, subquery_name); + if (!left) + return; + + addHiddenNames(select.with_expression_list, hidden_names, subquery_name); + } + + select.tables = std::make_shared(); + select.tables->children.push_back(left); + select.tables->children.push_back(tables->children.back()); + + ast = ast->clone(); /// rewrite AST in right manner + data.done = true; +} + +ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTPtr ast_left, ASTPtr ast_right, ASTPtr with, const String & subquery_alias) +{ + using RewriteMatcher = LinkedMatcher< + OneTypeMatcher, + OneTypeMatcher>; + using RewriteVisitor = InDepthNodeVisitor; + + auto left = static_cast(ast_left.get()); + auto right = static_cast(ast_right.get()); + if (!left || !right) + throw Exception("Two TablesInSelectQueryElements expected", ErrorCodes::LOGICAL_ERROR); + + if (!right->table_join || right->array_join) + return {}; + + auto table_join = static_cast(right->table_join.get()); + if (table_join->kind != ASTTableJoin::Kind::Inner) + return {}; + + ParserTablesInSelectQueryElement parser(true); + String subquery = "(select * from _t) as " + subquery_alias; + ASTPtr res = parseQuery(parser, subquery, 0); + if (!res) + throw Exception("Cannot parse rewrite query", ErrorCodes::LOGICAL_ERROR); + + RewriteVisitor::Data visitor_data = + std::make_pair({ast_left, ast_right}, {with}); + RewriteVisitor(visitor_data).visit(res); + return res; +} + +} diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h new file mode 100644 index 00000000000..4fd563beaa6 --- /dev/null +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h @@ -0,0 +1,52 @@ +#pragma once + +#include + +namespace DB +{ + +class ASTSelectQuery; + +/// AST transformer. It replaces multiple joins to (subselect + join) track. +/// 'select * from t1 join t2 on ... join t3 on ... join t4 on ...' would be rewriten with +/// 'select * from (select * from t1 join t2 on ...) join t3 on ...) join t4 on ...' +class JoinToSubqueryTransformMatcher +{ +public: + struct Data + { + bool done = false; + }; + + static constexpr const char * label = "JoinToSubqueryTransform"; + + static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; } + static std::vector visit(ASTPtr & ast, Data & data); + +private: + /// - combines two source TablesInSelectQueryElement into resulting one (Subquery) + /// - adds table hidings to ASTSelectQuery.with_expression_list + /// + /// TablesInSelectQueryElement [result] + /// TableExpression + /// Subquery (alias __join1) + /// SelectWithUnionQuery + /// ExpressionList + /// SelectQuery + /// ExpressionList + /// Asterisk + /// TablesInSelectQuery + /// TablesInSelectQueryElement [source1] + /// TablesInSelectQueryElement [source2] + /// + /// + WITH nameCoalesce(source1, __join1), nameCoalesce(source2, __join2) + /// + static void visit(ASTSelectQuery & select, ASTPtr & ast, Data & data); + + /// @return combined TablesInSelectQueryElement or nullptr if cannot rewrite + static ASTPtr replaceJoin(ASTPtr left, ASTPtr right, ASTPtr with, const String & subquery_alias); +}; + +using JoinToSubqueryTransformVisitor = InDepthNodeVisitor; + +} diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 5d470852eee..2fd865ef0cb 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -19,7 +19,9 @@ #include #include #include +#include +#include #include #include #include @@ -186,6 +188,13 @@ static std::tuple executeQueryImpl( if (!internal) logQuery(query.substr(0, settings.log_queries_cut_to_length), context); +#if 1 + JoinToSubqueryTransformVisitor::Data join_to_subs_data; + JoinToSubqueryTransformVisitor(join_to_subs_data).visit(ast); + if (join_to_subs_data.done) + logQuery(queryToString(*ast), context); +#endif + /// Check the limits. checkASTSizeLimits(*ast, settings); From 9ef314aaaff97387abc02ea915a9d990e643d820 Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 17 Dec 2018 22:30:08 +0300 Subject: [PATCH 031/230] SemanticSelectQuery CLICKHOUSE-3996 --- .../JoinToSubqueryTransformVisitor.cpp | 68 ++++++------------- .../JoinToSubqueryTransformVisitor.h | 4 +- dbms/src/Interpreters/SemanticSelectQuery.h | 43 ++++++++++++ dbms/src/Parsers/ASTSelectQuery.cpp | 2 + dbms/src/Parsers/IAST.h | 13 ++++ 5 files changed, 81 insertions(+), 49 deletions(-) create mode 100644 dbms/src/Interpreters/SemanticSelectQuery.h diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 25bd13fd6bb..3da2140e706 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -17,31 +18,24 @@ namespace ErrorCodes extern const int TOO_DEEP_AST; } -/// Attaches 'with' section to the first visited ASTSelectQuery -struct AppendWithSectionVisitorData +/// Attach additional semantic info to generated select. +struct AppendSemanticVisitorData { using TypeToVisit = ASTSelectQuery; - const ASTPtr & with; + const SemanticPtr & semantic; bool done = false; void visit(ASTSelectQuery & select, ASTPtr &) { - if (done || !with) + if (done || !semantic) return; - - if (select.with_expression_list) - { - for (auto & expr : with->children) - select.with_expression_list->children.push_back(expr->clone()); - } - else - select.with_expression_list = with->clone(); + select.semantic = semantic->clone(); done = true; } }; -/// Replaces one table element with pair +/// Replaces one table element with pair. struct RewriteTablesVisitorData { using TypeToVisit = ASTTablesInSelectQuery; @@ -61,8 +55,7 @@ struct RewriteTablesVisitorData } }; - -static String getTableNameOrAlias(const ASTPtr & table_element) +static void appendTableNameAndAlias(std::vector & hidden, const ASTPtr & table_element) { auto element = static_cast(table_element.get()); if (!element || element->children.empty()) @@ -72,34 +65,14 @@ static String getTableNameOrAlias(const ASTPtr & table_element) if (!table_expression || table_expression->children.empty()) throw Exception("Expected TableExpression with at least one child", ErrorCodes::LOGICAL_ERROR); - String result = table_expression->children[0]->tryGetAlias(); - if (!result.empty()) - return result; + String alias = table_expression->children[0]->tryGetAlias(); + if (!alias.empty()) + hidden.push_back(alias); auto identifier = static_cast(table_expression->children[0].get()); - if (!identifier) + if (!identifier && alias.empty()) throw Exception("Expected Identifier or subquery with alias", ErrorCodes::LOGICAL_ERROR); - return identifier->name; -} - -static void addHiddenNames(ASTPtr & with_expression_list, const std::vector & hidden_names, const String & new_name) -{ - if (!with_expression_list) - with_expression_list = std::make_shared(); - - ParserExpression parser; - for (auto & name : hidden_names) - { - if (name.empty()) - continue; - - String str_expression = "nameCoalesce(" + name + "," + new_name + ")"; - ASTPtr expr = parseQuery(parser, str_expression, 0); - if (!expr) - throw Exception("Cannot parse expression", ErrorCodes::LOGICAL_ERROR); - - with_expression_list->children.push_back(expr); - } + hidden.push_back(identifier->name); } @@ -131,14 +104,17 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast for (size_t i = 1; i < num_tables - 1; ++i) { ASTPtr right = tables->children[i]; - std::vector hidden_names = {getTableNameOrAlias(left), getTableNameOrAlias(right)}; + std::vector hidden_names; + appendTableNameAndAlias(hidden_names, left); + appendTableNameAndAlias(hidden_names, right); + String subquery_name = alias_prefix + toString(i); - left = replaceJoin(left, right, select.with_expression_list, subquery_name); + left = replaceJoin(select, left, right, subquery_name); if (!left) return; - addHiddenNames(select.with_expression_list, hidden_names, subquery_name); + SemanticSelectQuery::hideNames(select, hidden_names, subquery_name); } select.tables = std::make_shared(); @@ -149,11 +125,11 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast data.done = true; } -ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTPtr ast_left, ASTPtr ast_right, ASTPtr with, const String & subquery_alias) +ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTSelectQuery & select, ASTPtr ast_left, ASTPtr ast_right, const String & subquery_alias) { using RewriteMatcher = LinkedMatcher< OneTypeMatcher, - OneTypeMatcher>; + OneTypeMatcher>; using RewriteVisitor = InDepthNodeVisitor; auto left = static_cast(ast_left.get()); @@ -175,7 +151,7 @@ ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTPtr ast_left, ASTPtr ast_r throw Exception("Cannot parse rewrite query", ErrorCodes::LOGICAL_ERROR); RewriteVisitor::Data visitor_data = - std::make_pair({ast_left, ast_right}, {with}); + std::make_pair({ast_left, ast_right}, {select.semantic}); RewriteVisitor(visitor_data).visit(res); return res; } diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h index 4fd563beaa6..2c8e2f1a63d 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h @@ -39,12 +39,10 @@ private: /// TablesInSelectQueryElement [source1] /// TablesInSelectQueryElement [source2] /// - /// + WITH nameCoalesce(source1, __join1), nameCoalesce(source2, __join2) - /// static void visit(ASTSelectQuery & select, ASTPtr & ast, Data & data); /// @return combined TablesInSelectQueryElement or nullptr if cannot rewrite - static ASTPtr replaceJoin(ASTPtr left, ASTPtr right, ASTPtr with, const String & subquery_alias); + static ASTPtr replaceJoin(ASTSelectQuery & select, ASTPtr left, ASTPtr right, const String & subquery_alias); }; using JoinToSubqueryTransformVisitor = InDepthNodeVisitor; diff --git a/dbms/src/Interpreters/SemanticSelectQuery.h b/dbms/src/Interpreters/SemanticSelectQuery.h new file mode 100644 index 00000000000..5e0b5fa9d1f --- /dev/null +++ b/dbms/src/Interpreters/SemanticSelectQuery.h @@ -0,0 +1,43 @@ +#pragma once +#include +#include + +namespace DB +{ + +/// Additional information for ASTSelectQuery +class SemanticSelectQuery : public ISemantic +{ +public: + SemanticPtr clone() const override { return std::make_shared(*this); } + + std::vector getPossibleNames(const String & name) const + { + std::vector res; + res.push_back(name); + + for (auto it = hidings.find(name); it != hidings.end(); it = hidings.find(it->second)) + res.push_back(it->second); + return res; + } + + static void hideNames(ASTSelectQuery & select, const std::vector & hidden, const String & new_name) + { + if (!select.semantic) + select.semantic = std::make_shared(); + + auto & sema = static_cast(*select.semantic); + sema.hideNames(hidden, new_name); + } + +private: + std::unordered_map hidings; + + void hideNames(const std::vector & hidden, const String & new_name) + { + for (auto & name : hidden) + hidings.emplace(name, new_name); + } +}; + +} diff --git a/dbms/src/Parsers/ASTSelectQuery.cpp b/dbms/src/Parsers/ASTSelectQuery.cpp index 35e9ed75a4e..2a20cec0a36 100644 --- a/dbms/src/Parsers/ASTSelectQuery.cpp +++ b/dbms/src/Parsers/ASTSelectQuery.cpp @@ -64,6 +64,8 @@ ASTPtr ASTSelectQuery::clone() const #undef CLONE + if (semantic) + res->semantic = semantic->clone(); return res; } diff --git a/dbms/src/Parsers/IAST.h b/dbms/src/Parsers/IAST.h index e30ac56738f..48bb99c7d6e 100644 --- a/dbms/src/Parsers/IAST.h +++ b/dbms/src/Parsers/IAST.h @@ -31,6 +31,18 @@ class IAST; using ASTPtr = std::shared_ptr; using ASTs = std::vector; +class ISemantic; +using SemanticPtr = std::shared_ptr; + +/// Interfase to set additional information to IAST. Derived classes should be named according to their AST nodes' types: +/// ASTIdentifier => SemanticIdentifer, ASTSome => SemanticSome, ... +class ISemantic +{ +public: + virtual ~ISemantic() = default; + virtual SemanticPtr clone() const = 0; +}; + class WriteBuffer; @@ -44,6 +56,7 @@ public: /// This pointer does not allow it to be deleted while the range refers to it. StringPtr owned_string; + SemanticPtr semantic; virtual ~IAST() = default; From fd49cc3c34d65f1bb2c9e4734b32c17b94816d34 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 18 Dec 2018 16:08:22 +0300 Subject: [PATCH 032/230] Simpliest codec multiple works --- .../Compression/CompressionCodecMultiple.cpp | 102 +++++++++++++++--- dbms/src/Compression/CompressionCodecNone.cpp | 4 +- .../postgres_odbc_hashed_dictionary.xml | 10 +- 3 files changed, 90 insertions(+), 26 deletions(-) diff --git a/dbms/src/Compression/CompressionCodecMultiple.cpp b/dbms/src/Compression/CompressionCodecMultiple.cpp index 9e8f6adb13f..ba969dbfcb2 100644 --- a/dbms/src/Compression/CompressionCodecMultiple.cpp +++ b/dbms/src/Compression/CompressionCodecMultiple.cpp @@ -2,10 +2,17 @@ #include #include #include +#include +#include namespace DB { +namespace ErrorCodes +{ +extern const int UNKNOWN_CODEC; +extern const int CORRUPTED_DATA; +} CompressionCodecMultiple::CompressionCodecMultiple(Codecs codecs) : codecs(codecs) @@ -43,58 +50,119 @@ size_t CompressionCodecMultiple::getCompressedReserveSize(size_t uncompressed_si size_t CompressionCodecMultiple::compress(char * source, size_t source_size, char * dest) { - static constexpr size_t without_method_header_size = sizeof(UInt32) + sizeof(UInt32); + static constexpr size_t header_for_size_store = sizeof(UInt32) + sizeof(UInt32); PODArray compressed_buf; - PODArray uncompressed_buf(source_size); - uncompressed_buf.insert(source, source + source_size); + PODArray uncompressed_buf(source, source + source_size); dest[0] = static_cast(getMethodByte()); dest[1] = static_cast(codecs.size()); - std::cerr << "(compress) codecs size:" << codecs.size() << std::endl; - std::cerr << "(compress) desc:" << codec_desc << std::endl; + //std::cerr << "(compress) codecs size:" << codecs.size() << std::endl; + //std::cerr << "(compress) desc:" << codec_desc << std::endl; + //std::cerr << "(compress) SOURCE WRITE:\n"; + //for (size_t i = 0 ; i < source_size; ++i) { + // std::cerr << std::hex << +source[i] << std::endl; + //} + //std::cerr << "(compress) UNCOMPRESSED BUF:\n"; + //for(size_t i = 0; i < source_size; ++i) + // std::cerr << std::hex << +uncompressed_buf[i] << " "; + //std::cerr << std::dec << std::endl; + size_t codecs_byte_pos = 2; for (size_t idx = 0; idx < codecs.size(); ++idx, ++codecs_byte_pos) { const auto codec = codecs[idx]; dest[codecs_byte_pos] = codec->getMethodByte(); - compressed_buf.resize(without_method_header_size + codec->getCompressedReserveSize(source_size)); + //String name; + //codec->getCodecDesc(name); + //std::cerr << std::hex << "(compress) Compression method byte:" << +codec->getMethodByte() << " name:" << name << std::endl; + compressed_buf.resize(header_for_size_store + codec->getCompressedReserveSize(source_size)); - size_t size_compressed = without_method_header_size; - size_compressed += codec->compress(&uncompressed_buf[0], source_size, &compressed_buf[without_method_header_size]); + + size_t size_compressed = header_for_size_store; + size_compressed += codec->compress(&uncompressed_buf[0], source_size, &compressed_buf[header_for_size_store]); UInt32 compressed_size_32 = size_compressed; UInt32 uncompressed_size_32 = source_size; + //std::cerr << std::dec << "(compress) Compressed size:" << compressed_size_32 << std::endl; + //std::cerr << std::dec << "(compress) Uncompressed size:" << uncompressed_size_32 << std::endl; unalignedStore(&compressed_buf[0], compressed_size_32); unalignedStore(&compressed_buf[4], uncompressed_size_32); + //std::cerr << "(compress) COMPRESSED BUF:\n"; + //for(size_t i = header_for_size_store; i < size_compressed; ++i) + // std::cerr << std::hex << +compressed_buf[i] << " "; + //std::cerr << std::dec << std::endl; + uncompressed_buf.swap(compressed_buf); source_size = size_compressed; } - memcpy(&dest[2 + codecs.size()], &compressed_buf[0], source_size); - return source_size; + memcpy(&dest[2 + codecs.size()], &uncompressed_buf[0], source_size); + + //std::cerr << std::dec; + //std::cerr << "(compress) WRITING BUF:" << 2 + codecs.size() + source_size << std::endl; + //for(size_t i = 0; i < 2 + codecs.size() + source_size; ++i) + // std::cerr << std::hex << +dest[i] << " "; + //std::cerr << std::endl; + + return 2 + codecs.size() + source_size; } size_t CompressionCodecMultiple::decompress(char * source, size_t source_size, char * dest, size_t decompressed_size) { + + static constexpr size_t header_for_size_store = sizeof(UInt32) + sizeof(UInt32); + + if (source[0] != getMethodByte()) + throw Exception("Incorrect compression method for codec multiple, given " + toString(source[0]) + ", expected " + toString(getMethodByte()), + ErrorCodes::UNKNOWN_CODEC); UInt8 compression_methods_size = source[1]; - std::cerr << "(decompress) Methods size:" << +compression_methods_size << std::endl; + //std::cerr << std::dec; + //std::cerr << "(decompress) Source size:" << source_size << std::endl; + //std::cerr << "(decompress) decompressed size:" << decompressed_size << std::endl; + //std::cerr << "(decompress) Methods size:" << +compression_methods_size << std::endl; + + //std::cerr << "READING BUF:" << std::dec << source_size << std::endl; + //for(size_t i = 0; i < source_size; ++i) + // std::cerr << std::hex << +source[i] << " "; + //std::cerr << std::endl; PODArray compressed_buf; PODArray uncompressed_buf; - compressed_buf.insert(&source[compression_methods_size + 2], &source[source_size - (compression_methods_size + 2)]); + /// Insert all data into compressed buf + compressed_buf.insert(&source[compression_methods_size + 2], &source[source_size]); - for (size_t idx = 0; idx < compression_methods_size; ++idx) + for (long idx = compression_methods_size - 1; idx >= 0; --idx) { UInt8 compression_method = source[idx + 2]; - std::cerr << std::hex; - std::cerr << "(decompress) Compression method byte:" << +compression_method << std::endl; const auto codec = CompressionCodecFactory::instance().get(compression_method); - codec->decompress(&compressed_buf[8], 0, uncompressed_buf.data(), 0); + //String name; + //codec->getCodecDesc(name); + //std::cerr << std::hex << "(decompress) Compression method byte:" << +compression_method << " name:" << name << std::endl; + UInt32 compressed_size = static_cast(compressed_buf[0]); + UInt32 uncompressed_size = static_cast(compressed_buf[4]); + if (idx == 0 && uncompressed_size != decompressed_size) + throw Exception("Wrong final decompressed size in codec Multiple, got " + toString(uncompressed_size) + ", expected " + toString(decompressed_size), ErrorCodes::CORRUPTED_DATA); + uncompressed_buf.resize(uncompressed_size); + //std::cerr << std::dec << "(decompress) expected compressed size:" << compressed_size << std::endl; + //std::cerr << std::dec << "(decompress) expected uncompressed size:" << uncompressed_size << std::endl; + //std::cerr << "COMPRESSED BUF:\n"; + //for(size_t i = header_for_size_store; i < compressed_size; ++i) + // std::cerr<< std::hex << +compressed_buf[i] << " "; + //std::cerr << std::dec << std::endl; + + codec->decompress(&compressed_buf[header_for_size_store], compressed_size - header_for_size_store, &uncompressed_buf[0], uncompressed_size); + //std::cerr << "FINAL UNCOMPRESSED:\n"; + //for (size_t i = 0; i < uncompressed_size; ++i) { + // std::cerr << std::hex << +uncompressed_buf[i] << std::endl; + //} + uncompressed_buf.swap(compressed_buf); } + //std::cerr << "FINISHED\n"; - memcpy(dest, uncompressed_buf.data(), decompressed_size); + + memcpy(dest, compressed_buf.data(), decompressed_size); return decompressed_size; } diff --git a/dbms/src/Compression/CompressionCodecNone.cpp b/dbms/src/Compression/CompressionCodecNone.cpp index c35cfefd5dd..133d4d69c85 100644 --- a/dbms/src/Compression/CompressionCodecNone.cpp +++ b/dbms/src/Compression/CompressionCodecNone.cpp @@ -22,7 +22,7 @@ size_t CompressionCodecNone::compress(char * source, size_t source_size, char * return source_size; } -size_t CompressionCodecNone::decompress(char *source, size_t /*source_size*/, char *dest, size_t size_decompressed) +size_t CompressionCodecNone::decompress(char * source, size_t /*source_size*/, char * dest, size_t size_decompressed) { memcpy(dest, source, size_decompressed); return size_decompressed; @@ -35,4 +35,4 @@ void registerCodecNone(CompressionCodecFactory & factory) }); } -} \ No newline at end of file +} diff --git a/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml b/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml index 1c293f66761..4871f268e9c 100644 --- a/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml +++ b/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml @@ -13,19 +13,15 @@ 5 - + - - column1 - - - + column1 Int64 1 - + column2 From bb4eb06bb6538b6386f5b4d00373241727648248 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 18 Dec 2018 16:30:22 +0300 Subject: [PATCH 033/230] Fix strange logic --- dbms/src/Compression/CompressionCodecMultiple.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/dbms/src/Compression/CompressionCodecMultiple.cpp b/dbms/src/Compression/CompressionCodecMultiple.cpp index ba969dbfcb2..14b636621d0 100644 --- a/dbms/src/Compression/CompressionCodecMultiple.cpp +++ b/dbms/src/Compression/CompressionCodecMultiple.cpp @@ -89,7 +89,7 @@ size_t CompressionCodecMultiple::compress(char * source, size_t source_size, cha unalignedStore(&compressed_buf[0], compressed_size_32); unalignedStore(&compressed_buf[4], uncompressed_size_32); //std::cerr << "(compress) COMPRESSED BUF:\n"; - //for(size_t i = header_for_size_store; i < size_compressed; ++i) + //for(size_t i = 0; i < size_compressed; ++i) // std::cerr << std::hex << +compressed_buf[i] << " "; //std::cerr << std::dec << std::endl; @@ -116,6 +116,7 @@ size_t CompressionCodecMultiple::decompress(char * source, size_t source_size, c if (source[0] != getMethodByte()) throw Exception("Incorrect compression method for codec multiple, given " + toString(source[0]) + ", expected " + toString(getMethodByte()), ErrorCodes::UNKNOWN_CODEC); + UInt8 compression_methods_size = source[1]; //std::cerr << std::dec; //std::cerr << "(decompress) Source size:" << source_size << std::endl; @@ -139,15 +140,15 @@ size_t CompressionCodecMultiple::decompress(char * source, size_t source_size, c //String name; //codec->getCodecDesc(name); //std::cerr << std::hex << "(decompress) Compression method byte:" << +compression_method << " name:" << name << std::endl; - UInt32 compressed_size = static_cast(compressed_buf[0]); - UInt32 uncompressed_size = static_cast(compressed_buf[4]); + UInt32 compressed_size = unalignedLoad(&compressed_buf[0]); + UInt32 uncompressed_size = unalignedLoad(&compressed_buf[4]); if (idx == 0 && uncompressed_size != decompressed_size) throw Exception("Wrong final decompressed size in codec Multiple, got " + toString(uncompressed_size) + ", expected " + toString(decompressed_size), ErrorCodes::CORRUPTED_DATA); uncompressed_buf.resize(uncompressed_size); //std::cerr << std::dec << "(decompress) expected compressed size:" << compressed_size << std::endl; //std::cerr << std::dec << "(decompress) expected uncompressed size:" << uncompressed_size << std::endl; //std::cerr << "COMPRESSED BUF:\n"; - //for(size_t i = header_for_size_store; i < compressed_size; ++i) + //for(size_t i = 0; i < compressed_size; ++i) // std::cerr<< std::hex << +compressed_buf[i] << " "; //std::cerr << std::dec << std::endl; From 60456eb7429b4d90e05090635551c91e64494e81 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 18 Dec 2018 17:34:18 +0300 Subject: [PATCH 034/230] More tests on compression codecs --- .../Compression/CompressionCodecMultiple.cpp | 2 + ...4_test_custom_compression_codecs.reference | 9 ++++ .../00804_test_custom_compression_codecs.sql | 43 ++++++++++++++++++- 3 files changed, 53 insertions(+), 1 deletion(-) diff --git a/dbms/src/Compression/CompressionCodecMultiple.cpp b/dbms/src/Compression/CompressionCodecMultiple.cpp index 14b636621d0..45ab6776002 100644 --- a/dbms/src/Compression/CompressionCodecMultiple.cpp +++ b/dbms/src/Compression/CompressionCodecMultiple.cpp @@ -8,6 +8,8 @@ namespace DB { + + namespace ErrorCodes { extern const int UNKNOWN_CODEC; diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference index fd1689b374b..4832f7f06b6 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference @@ -2,3 +2,12 @@ 2 world 2018-12-15 2.2 3 ! 2018-12-16 3.3 2 +1 world 2018-10-05 1.1 +2 hello 2018-10-01 2.2 +3 buy 2018-10-11 3.3 +10003 +10003 +274972506.6 +9175437371954010821 +1.5555555555555 hello world! [77] ['John'] +7.1000000000000 xxxxxxxxxxxx [127] ['Henry'] diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 20d791e719f..f7c060976e4 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -1,8 +1,8 @@ SET send_logs_level = 'none'; + DROP TABLE IF EXISTS test.compression_codec; CREATE TABLE test.compression_codec(id UInt64 CODEC(LZ4), data String CODEC(ZSTD), ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2))) ENGINE = MergeTree() ORDER BY tuple(); -CREATE TABLE test.compression_codec(id UInt64 CODEC(LZ4, ZSTD, NONE), data String CODEC(ZSTD), ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2))) ENGINE = MergeTree() ORDER BY tuple(); INSERT INTO test.compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1); INSERT INTO test.compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2); @@ -29,3 +29,44 @@ CREATE TABLE test.params_when_no_params(id UInt64 CODEC(LZ4(1))) ENGINE = MergeT DROP TABLE IF EXISTS test.bad_codec; DROP TABLE IF EXISTS test.params_when_no_params; DROP TABLE IF EXISTS test.too_many_params; + + +DROP TABLE IF EXISTS test.compression_codec_multiple; + +CREATE TABLE test.compression_codec_multiple ( + id UInt64 CODEC(LZ4, ZSTD, NONE), + data String CODEC(ZSTD, NONE, LZ4, LZ4), + ddd Date CODEC(NONE, NONE, NONE, LZ4, ZSTD), + somenum Float64 CODEC(LZ4, LZ4, ZSTD, ZSTD, ZSTD) +) ENGINE = MergeTree() ORDER BY tuple(); + +INSERT INTO test.compression_codec_multiple VALUES (1, 'world', toDate('2018-10-05'), 1.1), (2, 'hello', toDate('2018-10-01'), 2.2), (3, 'buy', toDate('2018-10-11'), 3.3); + +SELECT * FROM test.compression_codec_multiple ORDER BY id; + +INSERT INTO test.compression_codec_multiple select modulo(number, 100), toString(number), toDate('2018-12-01'), 5.5 * number from system.numbers limit 10000; + +SELECT count(*) from test.compression_codec_multiple; + +SELECT count(distinct data) from test.compression_codec_multiple; + +SELECT floor(sum(somenum), 1) from test.compression_codec_multiple; + +TRUNCATE TABLE test.compression_codec_multiple; + +INSERT INTO test.compression_codec_multiple select modulo(number, 100), toString(number), toDate('2018-12-01'), 5.5 * number from system.numbers limit 10000; + +SELECT sum(cityHash64(*)) from test.compression_codec_multiple; + +DROP TABLE IF EXISTS test.compression_codec_multiple_more_types; + +CREATE TABLE test.compression_codec_multiple_more_types ( + id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD), + data FixedString(12) CODEC(ZSTD, ZSTD, NONE, NONE, NONE), + ddd Nested (age UInt8, Name String) CODEC(LZ4, NONE, NONE, NONE, ZSTD) +) ENGINE = MergeTree() ORDER BY tuple(); + +INSERT INTO test.compression_codec_multiple_more_types VALUES(1.5555555555555, 'hello world!', [77], ['John']); +INSERT INTO test.compression_codec_multiple_more_types VALUES(7.1, 'xxxxxxxxxxxx', [127], ['Henry']); + +SELECT * FROM test.compression_codec_multiple_more_types order by id; From 3d0da644f16cdfe3e22232213eefd8f2f06f45d1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 18 Dec 2018 17:37:55 +0300 Subject: [PATCH 035/230] one more test --- .../0_stateless/00804_test_custom_compression_codecs.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index f7c060976e4..b3cb984cf86 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -21,14 +21,17 @@ DROP TABLE IF EXISTS test.compression_codec; DROP TABLE IF EXISTS test.bad_codec; DROP TABLE IF EXISTS test.params_when_no_params; DROP TABLE IF EXISTS test.too_many_params; +DROP TABLE IF EXISTS test.codec_multiple_direct_specification; CREATE TABLE test.bad_codec(id UInt64 CODEC(adssadads)) ENGINE = MergeTree() order by tuple(); -- { serverError 429 } CREATE TABLE test.too_many_params(id UInt64 CODEC(ZSTD(2,3,4,5))) ENGINE = MergeTree() order by tuple(); -- { serverError 428 } CREATE TABLE test.params_when_no_params(id UInt64 CODEC(LZ4(1))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 378 } +CREATE TABLE test.codec_multiple_direct_specification(id UInt64 CODEC(MULTIPLE(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 429 } DROP TABLE IF EXISTS test.bad_codec; DROP TABLE IF EXISTS test.params_when_no_params; DROP TABLE IF EXISTS test.too_many_params; +DROP TABLE IF EXISTS test.codec_multiple_direct_specification; DROP TABLE IF EXISTS test.compression_codec_multiple; From 314fbbd71fc1a3a62b06c368e9ceb1cac8a00f33 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 18 Dec 2018 17:41:58 +0300 Subject: [PATCH 036/230] Remove some debug output --- .../Compression/CompressionCodecMultiple.cpp | 53 ------------------- .../00804_test_custom_compression_codecs.sql | 1 - 2 files changed, 54 deletions(-) diff --git a/dbms/src/Compression/CompressionCodecMultiple.cpp b/dbms/src/Compression/CompressionCodecMultiple.cpp index 45ab6776002..47ac6931435 100644 --- a/dbms/src/Compression/CompressionCodecMultiple.cpp +++ b/dbms/src/Compression/CompressionCodecMultiple.cpp @@ -59,41 +59,21 @@ size_t CompressionCodecMultiple::compress(char * source, size_t source_size, cha dest[0] = static_cast(getMethodByte()); dest[1] = static_cast(codecs.size()); - //std::cerr << "(compress) codecs size:" << codecs.size() << std::endl; - //std::cerr << "(compress) desc:" << codec_desc << std::endl; - //std::cerr << "(compress) SOURCE WRITE:\n"; - //for (size_t i = 0 ; i < source_size; ++i) { - // std::cerr << std::hex << +source[i] << std::endl; - //} - //std::cerr << "(compress) UNCOMPRESSED BUF:\n"; - //for(size_t i = 0; i < source_size; ++i) - // std::cerr << std::hex << +uncompressed_buf[i] << " "; - //std::cerr << std::dec << std::endl; size_t codecs_byte_pos = 2; for (size_t idx = 0; idx < codecs.size(); ++idx, ++codecs_byte_pos) { const auto codec = codecs[idx]; dest[codecs_byte_pos] = codec->getMethodByte(); - //String name; - //codec->getCodecDesc(name); - //std::cerr << std::hex << "(compress) Compression method byte:" << +codec->getMethodByte() << " name:" << name << std::endl; compressed_buf.resize(header_for_size_store + codec->getCompressedReserveSize(source_size)); - size_t size_compressed = header_for_size_store; size_compressed += codec->compress(&uncompressed_buf[0], source_size, &compressed_buf[header_for_size_store]); UInt32 compressed_size_32 = size_compressed; UInt32 uncompressed_size_32 = source_size; - //std::cerr << std::dec << "(compress) Compressed size:" << compressed_size_32 << std::endl; - //std::cerr << std::dec << "(compress) Uncompressed size:" << uncompressed_size_32 << std::endl; unalignedStore(&compressed_buf[0], compressed_size_32); unalignedStore(&compressed_buf[4], uncompressed_size_32); - //std::cerr << "(compress) COMPRESSED BUF:\n"; - //for(size_t i = 0; i < size_compressed; ++i) - // std::cerr << std::hex << +compressed_buf[i] << " "; - //std::cerr << std::dec << std::endl; uncompressed_buf.swap(compressed_buf); source_size = size_compressed; @@ -101,12 +81,6 @@ size_t CompressionCodecMultiple::compress(char * source, size_t source_size, cha memcpy(&dest[2 + codecs.size()], &uncompressed_buf[0], source_size); - //std::cerr << std::dec; - //std::cerr << "(compress) WRITING BUF:" << 2 + codecs.size() + source_size << std::endl; - //for(size_t i = 0; i < 2 + codecs.size() + source_size; ++i) - // std::cerr << std::hex << +dest[i] << " "; - //std::cerr << std::endl; - return 2 + codecs.size() + source_size; } @@ -120,16 +94,6 @@ size_t CompressionCodecMultiple::decompress(char * source, size_t source_size, c ErrorCodes::UNKNOWN_CODEC); UInt8 compression_methods_size = source[1]; - //std::cerr << std::dec; - //std::cerr << "(decompress) Source size:" << source_size << std::endl; - //std::cerr << "(decompress) decompressed size:" << decompressed_size << std::endl; - //std::cerr << "(decompress) Methods size:" << +compression_methods_size << std::endl; - - //std::cerr << "READING BUF:" << std::dec << source_size << std::endl; - //for(size_t i = 0; i < source_size; ++i) - // std::cerr << std::hex << +source[i] << " "; - //std::cerr << std::endl; - PODArray compressed_buf; PODArray uncompressed_buf; /// Insert all data into compressed buf @@ -139,31 +103,14 @@ size_t CompressionCodecMultiple::decompress(char * source, size_t source_size, c { UInt8 compression_method = source[idx + 2]; const auto codec = CompressionCodecFactory::instance().get(compression_method); - //String name; - //codec->getCodecDesc(name); - //std::cerr << std::hex << "(decompress) Compression method byte:" << +compression_method << " name:" << name << std::endl; UInt32 compressed_size = unalignedLoad(&compressed_buf[0]); UInt32 uncompressed_size = unalignedLoad(&compressed_buf[4]); if (idx == 0 && uncompressed_size != decompressed_size) throw Exception("Wrong final decompressed size in codec Multiple, got " + toString(uncompressed_size) + ", expected " + toString(decompressed_size), ErrorCodes::CORRUPTED_DATA); uncompressed_buf.resize(uncompressed_size); - //std::cerr << std::dec << "(decompress) expected compressed size:" << compressed_size << std::endl; - //std::cerr << std::dec << "(decompress) expected uncompressed size:" << uncompressed_size << std::endl; - //std::cerr << "COMPRESSED BUF:\n"; - //for(size_t i = 0; i < compressed_size; ++i) - // std::cerr<< std::hex << +compressed_buf[i] << " "; - //std::cerr << std::dec << std::endl; - codec->decompress(&compressed_buf[header_for_size_store], compressed_size - header_for_size_store, &uncompressed_buf[0], uncompressed_size); - //std::cerr << "FINAL UNCOMPRESSED:\n"; - //for (size_t i = 0; i < uncompressed_size; ++i) { - // std::cerr << std::hex << +uncompressed_buf[i] << std::endl; - //} - uncompressed_buf.swap(compressed_buf); } - //std::cerr << "FINISHED\n"; - memcpy(dest, compressed_buf.data(), decompressed_size); return decompressed_size; diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index b3cb984cf86..9dc71542a37 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -33,7 +33,6 @@ DROP TABLE IF EXISTS test.params_when_no_params; DROP TABLE IF EXISTS test.too_many_params; DROP TABLE IF EXISTS test.codec_multiple_direct_specification; - DROP TABLE IF EXISTS test.compression_codec_multiple; CREATE TABLE test.compression_codec_multiple ( From bc0e0c220e79b284919119d6536209305d7ac713 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 18 Dec 2018 17:43:15 +0300 Subject: [PATCH 037/230] Add level test --- .../0_stateless/00804_test_custom_compression_codecs.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 9dc71542a37..b21b23ca436 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -37,9 +37,9 @@ DROP TABLE IF EXISTS test.compression_codec_multiple; CREATE TABLE test.compression_codec_multiple ( id UInt64 CODEC(LZ4, ZSTD, NONE), - data String CODEC(ZSTD, NONE, LZ4, LZ4), + data String CODEC(ZSTD(2), NONE, LZ4, LZ4), ddd Date CODEC(NONE, NONE, NONE, LZ4, ZSTD), - somenum Float64 CODEC(LZ4, LZ4, ZSTD, ZSTD, ZSTD) + somenum Float64 CODEC(LZ4, LZ4, ZSTD(2), ZSTD(3), ZSTD(1)) ) ENGINE = MergeTree() ORDER BY tuple(); INSERT INTO test.compression_codec_multiple VALUES (1, 'world', toDate('2018-10-05'), 1.1), (2, 'hello', toDate('2018-10-01'), 2.2), (3, 'buy', toDate('2018-10-11'), 3.3); From 9b0d47c0c343be71336a47e3fdc36f95a86bf7bc Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 18 Dec 2018 18:00:51 +0300 Subject: [PATCH 038/230] Add zstd params validation --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Compression/CompressionCodecZSTD.cpp | 5 ++++- .../0_stateless/00804_test_custom_compression_codecs.sql | 5 ++++- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index a5ba25219db..55e5af45a8f 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -404,6 +404,7 @@ namespace ErrorCodes extern const int CANNOT_COMPILE_REGEXP = 427; extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE = 428; extern const int UNKNOWN_CODEC = 429; + extern const int ILLEGAL_CODEC_PARAMETER = 430; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Compression/CompressionCodecZSTD.cpp b/dbms/src/Compression/CompressionCodecZSTD.cpp index 9da1454a539..cff70973d99 100644 --- a/dbms/src/Compression/CompressionCodecZSTD.cpp +++ b/dbms/src/Compression/CompressionCodecZSTD.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -17,6 +18,7 @@ namespace ErrorCodes extern const int CANNOT_COMPRESS; extern const int CANNOT_DECOMPRESS; extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; + extern const int ILLEGAL_CODEC_PARAMETER; } char CompressionCodecZSTD::getMethodByte() @@ -47,7 +49,6 @@ size_t CompressionCodecZSTD::compress(char * source, size_t source_size, char * size_t CompressionCodecZSTD::decompress(char * source, size_t source_size, char * dest, size_t size_decompressed) { size_t res = ZSTD_decompress(dest, size_decompressed, source, source_size); -// compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, size_compressed_without_checksum - COMPRESSED_BLOCK_HEADER_SIZE); if (ZSTD_isError(res)) throw Exception("Cannot ZSTD_decompress: " + std::string(ZSTD_getErrorName(res)), ErrorCodes::CANNOT_DECOMPRESS); @@ -74,6 +75,8 @@ void registerCodecZSTD(CompressionCodecFactory & factory) const auto children = arguments->children; const ASTLiteral * literal = static_cast(children[0].get()); level = literal->value.safeGet(); + if (level > ZSTD_maxCLevel()) + throw Exception("ZSTD codec can't have level more that " + toString(ZSTD_maxCLevel()) + ", given " + toString(level), ErrorCodes::ILLEGAL_CODEC_PARAMETER); } return std::make_shared(level); diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index b21b23ca436..e1860a8702c 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -20,16 +20,19 @@ DROP TABLE IF EXISTS test.compression_codec; DROP TABLE IF EXISTS test.bad_codec; DROP TABLE IF EXISTS test.params_when_no_params; +DROP TABLE IF EXISTS test.params_zstd; DROP TABLE IF EXISTS test.too_many_params; DROP TABLE IF EXISTS test.codec_multiple_direct_specification; CREATE TABLE test.bad_codec(id UInt64 CODEC(adssadads)) ENGINE = MergeTree() order by tuple(); -- { serverError 429 } CREATE TABLE test.too_many_params(id UInt64 CODEC(ZSTD(2,3,4,5))) ENGINE = MergeTree() order by tuple(); -- { serverError 428 } CREATE TABLE test.params_when_no_params(id UInt64 CODEC(LZ4(1))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 378 } +CREATE TABLE test.params_zstd(id UInt64 CODEC(ZSTD(33))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 430 } CREATE TABLE test.codec_multiple_direct_specification(id UInt64 CODEC(MULTIPLE(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 429 } DROP TABLE IF EXISTS test.bad_codec; DROP TABLE IF EXISTS test.params_when_no_params; +DROP TABLE IF EXISTS test.params_zstd; DROP TABLE IF EXISTS test.too_many_params; DROP TABLE IF EXISTS test.codec_multiple_direct_specification; @@ -39,7 +42,7 @@ CREATE TABLE test.compression_codec_multiple ( id UInt64 CODEC(LZ4, ZSTD, NONE), data String CODEC(ZSTD(2), NONE, LZ4, LZ4), ddd Date CODEC(NONE, NONE, NONE, LZ4, ZSTD), - somenum Float64 CODEC(LZ4, LZ4, ZSTD(2), ZSTD(3), ZSTD(1)) + somenum Float64 CODEC(LZ4, LZ4, ZSTD(2), ZSTD(3), ZSTD) ) ENGINE = MergeTree() ORDER BY tuple(); INSERT INTO test.compression_codec_multiple VALUES (1, 'world', toDate('2018-10-05'), 1.1), (2, 'hello', toDate('2018-10-01'), 2.2), (3, 'buy', toDate('2018-10-11'), 3.3); From f992f0a9a02267b06c25938fd7e72bb41db748d7 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 18 Dec 2018 21:28:02 +0300 Subject: [PATCH 039/230] better check when rewrites needed --- .../JoinToSubqueryTransformVisitor.cpp | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 3da2140e706..1520711aa68 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -55,6 +55,19 @@ struct RewriteTablesVisitorData } }; +static bool needRewrite(ASTSelectQuery & select) +{ + auto tables = static_cast(select.tables.get()); + if (!tables) + return false; + + size_t num_tables = tables->children.size(); + if (num_tables <= 2) + return false; + + return true; +} + static void appendTableNameAndAlias(std::vector & hidden, const ASTPtr & table_element) { auto element = static_cast(table_element.get()); @@ -86,19 +99,15 @@ std::vector JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast, Data & data) { static String alias_prefix = "__join"; /// FIXME - static const size_t max_joins = 64; /// TODO: settings.max_subquery_depth + + if (!needRewrite(select)) + return; auto tables = static_cast(select.tables.get()); if (!tables) throw Exception("TablesInSelectQuery expected", ErrorCodes::LOGICAL_ERROR); size_t num_tables = tables->children.size(); - if (num_tables <= 2) - return; - - if (num_tables > max_joins) - throw Exception("Too much joins", ErrorCodes::TOO_DEEP_AST); - ASTPtr left = tables->children[0]; for (size_t i = 1; i < num_tables - 1; ++i) From 512fe3c854841028ea44a9f6415fad1263fc36b1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 19 Dec 2018 20:20:18 +0300 Subject: [PATCH 040/230] Refactoring (sanitizer failure) --- dbms/src/Compression/CompressionCodecLZ4.cpp | 19 +- dbms/src/Compression/CompressionCodecLZ4.h | 21 +-- .../Compression/CompressionCodecMultiple.cpp | 84 ++++----- .../Compression/CompressionCodecMultiple.h | 11 +- dbms/src/Compression/CompressionCodecNone.cpp | 15 +- dbms/src/Compression/CompressionCodecNone.h | 12 +- dbms/src/Compression/CompressionCodecZSTD.cpp | 24 +-- dbms/src/Compression/CompressionCodecZSTD.h | 15 +- dbms/src/Compression/ICompressionCodec.cpp | 163 +++++++++++------- dbms/src/Compression/ICompressionCodec.h | 52 +++--- dbms/src/IO/CachedCompressedReadBuffer.cpp | 15 +- dbms/src/IO/CachedCompressedReadBuffer.h | 2 +- .../Interpreters/InterpreterCreateQuery.cpp | 3 +- .../Storages/MergeTree/MergeTreeReader.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 2 +- ...4_test_custom_compression_codecs.reference | 2 + .../00804_test_custom_compression_codecs.sql | 31 +++- 17 files changed, 277 insertions(+), 196 deletions(-) diff --git a/dbms/src/Compression/CompressionCodecLZ4.cpp b/dbms/src/Compression/CompressionCodecLZ4.cpp index ea9c7e65cf9..8c44ebfac43 100644 --- a/dbms/src/Compression/CompressionCodecLZ4.cpp +++ b/dbms/src/Compression/CompressionCodecLZ4.cpp @@ -10,35 +10,34 @@ namespace DB { -char CompressionCodecLZ4::getMethodByte() +UInt8 CompressionCodecLZ4::getMethodByte() const { - return static_cast(CompressionMethodByte::LZ4); + return static_cast(CompressionMethodByte::LZ4); } -void CompressionCodecLZ4::getCodecDesc(String & codec_desc) +String CompressionCodecLZ4::getCodecDesc() const { - codec_desc = "LZ4"; + return "LZ4"; } -size_t CompressionCodecLZ4::getCompressedReserveSize(size_t uncompressed_size) +UInt32 CompressionCodecLZ4::getCompressedDataSize(UInt32 uncompressed_size) const { return LZ4_COMPRESSBOUND(uncompressed_size); } -size_t CompressionCodecLZ4::compress(char * source, size_t source_size, char * dest) +UInt32 CompressionCodecLZ4::doCompressData(const char * source, UInt32 source_size, char * dest) const { return LZ4_compress_default(source, dest, source_size, LZ4_COMPRESSBOUND(source_size)); } -size_t CompressionCodecLZ4::decompress(char * source, size_t source_size, char * dest, size_t size_decompressed) +void CompressionCodecLZ4::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { - LZ4::decompress(source, dest, source_size, size_decompressed, lz4_stat); - return size_decompressed; + LZ4::decompress(source, dest, source_size, uncompressed_size, lz4_stat); } void registerCodecLZ4(CompressionCodecFactory & factory) { - factory.registerSimpleCompressionCodec("LZ4", static_cast(CompressionMethodByte::LZ4), [&](){ + factory.registerSimpleCompressionCodec("LZ4", static_cast(CompressionMethodByte::LZ4), [&](){ return std::make_shared(); }); } diff --git a/dbms/src/Compression/CompressionCodecLZ4.h b/dbms/src/Compression/CompressionCodecLZ4.h index 0ee0286c4ee..1e0a00fa753 100644 --- a/dbms/src/Compression/CompressionCodecLZ4.h +++ b/dbms/src/Compression/CompressionCodecLZ4.h @@ -12,18 +12,19 @@ namespace DB class CompressionCodecLZ4 : public ICompressionCodec { public: - char getMethodByte() override; + UInt8 getMethodByte() const override; - void getCodecDesc(String & codec_desc) override; - - size_t compress(char * source, size_t source_size, char * dest) override; - - size_t getCompressedReserveSize(size_t uncompressed_size) override; - - size_t decompress(char * source, size_t source_size, char * dest, size_t decompressed_size) override; + String getCodecDesc() const override; private: - LZ4::PerformanceStatistics lz4_stat; + + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; + + UInt32 getCompressedDataSize(UInt32 uncompressed_size) const override; + + mutable LZ4::PerformanceStatistics lz4_stat; }; -} \ No newline at end of file +} diff --git a/dbms/src/Compression/CompressionCodecMultiple.cpp b/dbms/src/Compression/CompressionCodecMultiple.cpp index 47ac6931435..d74a7129402 100644 --- a/dbms/src/Compression/CompressionCodecMultiple.cpp +++ b/dbms/src/Compression/CompressionCodecMultiple.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -25,100 +26,101 @@ CompressionCodecMultiple::CompressionCodecMultiple(Codecs codecs) codec_desc = codec_desc + ','; const auto codec = codecs[idx]; - String inner_codec_desc; - codec->getCodecDesc(inner_codec_desc); - codec_desc = codec_desc + inner_codec_desc; + codec_desc = codec_desc + codec->getCodecDesc(); } } -char CompressionCodecMultiple::getMethodByte() +UInt8 CompressionCodecMultiple::getMethodByte() const { - return static_cast(CompressionMethodByte::Multiple); + return static_cast(CompressionMethodByte::Multiple); } -void CompressionCodecMultiple::getCodecDesc(String & codec_desc_) +String CompressionCodecMultiple::getCodecDesc() const { - codec_desc_ = codec_desc; + return codec_desc; } -size_t CompressionCodecMultiple::getCompressedReserveSize(size_t uncompressed_size) +UInt32 CompressionCodecMultiple::getCompressedDataSize(UInt32 uncompressed_size) const { + UInt32 compressed_size = uncompressed_size; for (auto & codec : codecs) - uncompressed_size += codec->getCompressedReserveSize(uncompressed_size); + compressed_size = codec->getCompressedReserveSize(compressed_size); - /// MultipleCodecByte TotalCodecs ByteForEachCodec data - return sizeof(UInt8) + sizeof(UInt8) + codecs.size() + uncompressed_size; + /// TotalCodecs ByteForEachCodec data + return sizeof(UInt8) + codecs.size() + compressed_size; } -size_t CompressionCodecMultiple::compress(char * source, size_t source_size, char * dest) +UInt32 CompressionCodecMultiple::doCompressData(const char * source, UInt32 source_size, char * dest) const { - static constexpr size_t header_for_size_store = sizeof(UInt32) + sizeof(UInt32); PODArray compressed_buf; PODArray uncompressed_buf(source, source + source_size); - dest[0] = static_cast(getMethodByte()); - dest[1] = static_cast(codecs.size()); + dest[0] = static_cast(codecs.size()); - size_t codecs_byte_pos = 2; + size_t codecs_byte_pos = 1; for (size_t idx = 0; idx < codecs.size(); ++idx, ++codecs_byte_pos) { const auto codec = codecs[idx]; dest[codecs_byte_pos] = codec->getMethodByte(); - compressed_buf.resize(header_for_size_store + codec->getCompressedReserveSize(source_size)); + compressed_buf.resize(codec->getCompressedReserveSize(source_size)); - size_t size_compressed = header_for_size_store; - size_compressed += codec->compress(&uncompressed_buf[0], source_size, &compressed_buf[header_for_size_store]); - - UInt32 compressed_size_32 = size_compressed; - UInt32 uncompressed_size_32 = source_size; - unalignedStore(&compressed_buf[0], compressed_size_32); - unalignedStore(&compressed_buf[4], uncompressed_size_32); + UInt32 size_compressed = codec->compress(uncompressed_buf.data(), source_size, compressed_buf.data()); uncompressed_buf.swap(compressed_buf); source_size = size_compressed; } - memcpy(&dest[2 + codecs.size()], &uncompressed_buf[0], source_size); + //std::cerr << "(compress) BUF_SIZE_COMPRESSED:" << source_size << std::endl; - return 2 + codecs.size() + source_size; + memcpy(&dest[1 + codecs.size()], uncompressed_buf.data(), source_size); + + //std::cerr << "(compress) COMPRESSING BUF:\n"; + //for (size_t i = 0; i < source_size + 1 + codecs.size(); ++i) + // std::cerr << getHexUIntLowercase(+dest[i]) << " "; + //std::cerr << std::endl; + + return 1 + codecs.size() + source_size; } -size_t CompressionCodecMultiple::decompress(char * source, size_t source_size, char * dest, size_t decompressed_size) +void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 decompressed_size) const { + UInt8 compression_methods_size = source[0]; - static constexpr size_t header_for_size_store = sizeof(UInt32) + sizeof(UInt32); + //std::cerr << "(decompress) DECOMPRESSING BUF:\n"; + //for (size_t i = 0; i < source_size; ++i) + // std::cerr << getHexUIntLowercase(+source[i]) << " "; + //std::cerr << std::endl; - if (source[0] != getMethodByte()) - throw Exception("Incorrect compression method for codec multiple, given " + toString(source[0]) + ", expected " + toString(getMethodByte()), - ErrorCodes::UNKNOWN_CODEC); - - UInt8 compression_methods_size = source[1]; - PODArray compressed_buf; + //std::cerr << "(decompress) BUF_SIZE_COMPRESSED:" << source_size << std::endl; + //std::cerr << "(decompress) CODECS SIZE:" << +compression_methods_size << std::endl; + PODArray compressed_buf(&source[compression_methods_size + 1], &source[source_size]); PODArray uncompressed_buf; /// Insert all data into compressed buf - compressed_buf.insert(&source[compression_methods_size + 2], &source[source_size]); + source_size -= (compression_methods_size + 1); for (long idx = compression_methods_size - 1; idx >= 0; --idx) { - UInt8 compression_method = source[idx + 2]; + UInt8 compression_method = source[idx + 1]; const auto codec = CompressionCodecFactory::instance().get(compression_method); - UInt32 compressed_size = unalignedLoad(&compressed_buf[0]); - UInt32 uncompressed_size = unalignedLoad(&compressed_buf[4]); + UInt32 uncompressed_size = ICompressionCodec::readDecompressedBlockSize(compressed_buf.data()); + //std::cerr << "(decompress) UNCOMPRESSED SIZE READ:" << uncompressed_size << std::endl; + if (idx == 0 && uncompressed_size != decompressed_size) throw Exception("Wrong final decompressed size in codec Multiple, got " + toString(uncompressed_size) + ", expected " + toString(decompressed_size), ErrorCodes::CORRUPTED_DATA); + uncompressed_buf.resize(uncompressed_size); - codec->decompress(&compressed_buf[header_for_size_store], compressed_size - header_for_size_store, &uncompressed_buf[0], uncompressed_size); + codec->decompress(compressed_buf.data(), source_size, uncompressed_buf.data()); uncompressed_buf.swap(compressed_buf); + source_size = uncompressed_size; } memcpy(dest, compressed_buf.data(), decompressed_size); - return decompressed_size; } void registerCodecMultiple(CompressionCodecFactory & factory) { - factory.registerSimpleCompressionCodec("Multiple", static_cast(CompressionMethodByte::Multiple), [&](){ + factory.registerSimpleCompressionCodec("Multiple", static_cast(CompressionMethodByte::Multiple), [&](){ return std::make_shared(); }); } diff --git a/dbms/src/Compression/CompressionCodecMultiple.h b/dbms/src/Compression/CompressionCodecMultiple.h index cea37956ffb..f80f5bd7a6f 100644 --- a/dbms/src/Compression/CompressionCodecMultiple.h +++ b/dbms/src/Compression/CompressionCodecMultiple.h @@ -11,15 +11,16 @@ public: CompressionCodecMultiple() = default; explicit CompressionCodecMultiple(Codecs codecs); - char getMethodByte() override; + UInt8 getMethodByte() const override; - void getCodecDesc(String & codec_desc) override; + String getCodecDesc() const override; - size_t compress(char * source, size_t source_size, char * dest) override; + UInt32 getCompressedDataSize(UInt32 uncompressed_size) const override; - size_t getCompressedReserveSize(size_t uncompressed_size) override; +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - size_t decompress(char *source, size_t source_size, char *dest, size_t decompressed_size) override; + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; private: Codecs codecs; diff --git a/dbms/src/Compression/CompressionCodecNone.cpp b/dbms/src/Compression/CompressionCodecNone.cpp index 133d4d69c85..e090c09f9c1 100644 --- a/dbms/src/Compression/CompressionCodecNone.cpp +++ b/dbms/src/Compression/CompressionCodecNone.cpp @@ -6,26 +6,25 @@ namespace DB { -char CompressionCodecNone::getMethodByte() +UInt8 CompressionCodecNone::getMethodByte() const { - return static_cast(CompressionMethodByte::NONE); + return static_cast(CompressionMethodByte::NONE); } -void CompressionCodecNone::getCodecDesc(String & codec_desc) +String CompressionCodecNone::getCodecDesc() const { - codec_desc = "NONE"; + return "NONE"; } -size_t CompressionCodecNone::compress(char * source, size_t source_size, char * dest) +UInt32 CompressionCodecNone::doCompressData(const char * source, UInt32 source_size, char * dest) const { memcpy(dest, source, source_size); return source_size; } -size_t CompressionCodecNone::decompress(char * source, size_t /*source_size*/, char * dest, size_t size_decompressed) +void CompressionCodecNone::doDecompressData(const char * source, UInt32 /*source_size*/, char * dest, UInt32 uncompressed_size) const { - memcpy(dest, source, size_decompressed); - return size_decompressed; + memcpy(dest, source, uncompressed_size); } void registerCodecNone(CompressionCodecFactory & factory) diff --git a/dbms/src/Compression/CompressionCodecNone.h b/dbms/src/Compression/CompressionCodecNone.h index 89aeb218e10..ab3f1176734 100644 --- a/dbms/src/Compression/CompressionCodecNone.h +++ b/dbms/src/Compression/CompressionCodecNone.h @@ -11,13 +11,15 @@ namespace DB class CompressionCodecNone : public ICompressionCodec { public: - char getMethodByte() override; + UInt8 getMethodByte() const override; - void getCodecDesc(String & codec_desc) override; + String getCodecDesc() const override; - size_t compress(char * source, size_t source_size, char * compressed_buf) override; +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; - size_t decompress(char *source, size_t source_size, char *dest, size_t decompressed_size) override; }; -} \ No newline at end of file +} diff --git a/dbms/src/Compression/CompressionCodecZSTD.cpp b/dbms/src/Compression/CompressionCodecZSTD.cpp index cff70973d99..b42d2f54620 100644 --- a/dbms/src/Compression/CompressionCodecZSTD.cpp +++ b/dbms/src/Compression/CompressionCodecZSTD.cpp @@ -21,22 +21,23 @@ namespace ErrorCodes extern const int ILLEGAL_CODEC_PARAMETER; } -char CompressionCodecZSTD::getMethodByte() +UInt8 CompressionCodecZSTD::getMethodByte() const { - return static_cast(CompressionMethodByte::ZSTD); + return static_cast(CompressionMethodByte::ZSTD); } -void CompressionCodecZSTD::getCodecDesc(String & codec_desc) +String CompressionCodecZSTD::getCodecDesc() const { - codec_desc = "ZSTD"; + return "ZSTD"; } -size_t CompressionCodecZSTD::getCompressedReserveSize(size_t uncompressed_size) +UInt32 CompressionCodecZSTD::getCompressedDataSize(UInt32 uncompressed_size) const { return ZSTD_compressBound(uncompressed_size); } -size_t CompressionCodecZSTD::compress(char * source, size_t source_size, char * dest) + +UInt32 CompressionCodecZSTD::doCompressData(const char * source, UInt32 source_size, char * dest) const { size_t compressed_size = ZSTD_compress(dest, ZSTD_compressBound(source_size), source, source_size, level); @@ -46,18 +47,17 @@ size_t CompressionCodecZSTD::compress(char * source, size_t source_size, char * return compressed_size; } -size_t CompressionCodecZSTD::decompress(char * source, size_t source_size, char * dest, size_t size_decompressed) + +void CompressionCodecZSTD::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { - size_t res = ZSTD_decompress(dest, size_decompressed, source, source_size); + size_t res = ZSTD_decompress(dest, uncompressed_size, source, source_size); if (ZSTD_isError(res)) throw Exception("Cannot ZSTD_decompress: " + std::string(ZSTD_getErrorName(res)), ErrorCodes::CANNOT_DECOMPRESS); - - return size_decompressed; } -CompressionCodecZSTD::CompressionCodecZSTD(int level) - :level(level) +CompressionCodecZSTD::CompressionCodecZSTD(int level_) + :level(level_) { } diff --git a/dbms/src/Compression/CompressionCodecZSTD.h b/dbms/src/Compression/CompressionCodecZSTD.h index 9f6e3cd7497..ea2c1a9b17a 100644 --- a/dbms/src/Compression/CompressionCodecZSTD.h +++ b/dbms/src/Compression/CompressionCodecZSTD.h @@ -11,20 +11,21 @@ namespace DB class CompressionCodecZSTD : public ICompressionCodec { public: - CompressionCodecZSTD(int level); + CompressionCodecZSTD(int level_); - char getMethodByte() override; + UInt8 getMethodByte() const override; - void getCodecDesc(String & codec_desc) override; + String getCodecDesc() const override; - size_t compress(char * source, size_t source_size, char * dest) override; + UInt32 getCompressedDataSize(UInt32 uncompressed_size) const override; - size_t getCompressedReserveSize(size_t uncompressed_size) override; +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; - size_t decompress(char *source, size_t source_size, char *dest, size_t decompressed_size) override; + void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; private: int level; }; -} \ No newline at end of file +} diff --git a/dbms/src/Compression/ICompressionCodec.cpp b/dbms/src/Compression/ICompressionCodec.cpp index 09b16b43b96..df1efacf0cc 100644 --- a/dbms/src/Compression/ICompressionCodec.cpp +++ b/dbms/src/Compression/ICompressionCodec.cpp @@ -28,97 +28,147 @@ namespace ErrorCodes extern const int SEEK_POSITION_OUT_OF_BOUND; } -CompressionCodecReadBufferPtr ICompressionCodec::liftCompressed(ReadBuffer & origin) +static constexpr auto CHECKSUM_SIZE{sizeof(CityHash_v1_0_2::uint128)}; + +UInt32 ICompressionCodec::compress(char * source, UInt32 source_size, char * dest) const { - return std::make_shared(origin); + dest[0] = getMethodByte(); + UInt8 header_size = getHeaderSize(); + /// Write data from header_size + UInt32 compressed_bytes_written = doCompressData(source, source_size, &dest[header_size]); + unalignedStore(&dest[1], compressed_bytes_written + header_size); + unalignedStore(&dest[5], source_size); + return header_size + compressed_bytes_written; } -CompressionCodecWriteBufferPtr ICompressionCodec::liftCompressed(WriteBuffer & origin) + +UInt32 ICompressionCodec::decompress(char * source, UInt32 source_size, char * dest) const { - return std::make_shared(*this, origin); + UInt8 method = source[0]; + if (method != getMethodByte()) + throw Exception("Can't decompress data with codec byte " + toString(method) + " from codec with byte " + toString(method), ErrorCodes::CANNOT_DECOMPRESS); + + UInt8 header_size = getHeaderSize(); + UInt32 decompressed_size = unalignedLoad(&source[5]); + doDecompressData(&source[header_size], source_size - header_size, dest, decompressed_size); + return decompressed_size; + } -CompressionCodecReadBuffer::CompressionCodecReadBuffer(ReadBuffer & origin) - : origin(origin) +UInt32 ICompressionCodec::readCompressedBlockSize(const char * source) +{ + return unalignedLoad(&source[1]); +} + + +UInt32 ICompressionCodec::readDecompressedBlockSize(const char * source) +{ + return unalignedLoad(&source[5]); +} + + +UInt8 ICompressionCodec::readMethod(const char * source) +{ + return static_cast(source[0]); +} + +CompressionCodecReadBufferPtr liftCompressed(CompressionCodecPtr codec, ReadBuffer & origin) +{ + return std::make_shared(codec, origin); +} + +CompressionCodecWriteBufferPtr liftCompressed(CompressionCodecPtr codec, WriteBuffer & origin) +{ + return std::make_shared(codec, origin); +} + +CompressionCodecReadBuffer::CompressionCodecReadBuffer(CompressionCodecPtr codec_, ReadBuffer & origin_) + : codec(codec_) + , origin(origin_) { } + /// Read compressed data into compressed_buffer. Get size of decompressed data from block header. Checksum if need. /// Returns number of compressed bytes read. -size_t CompressionCodecReadBuffer::readCompressedData(size_t & size_decompressed, size_t & size_compressed) +std::pair CompressionCodecReadBuffer::readCompressedData() { if (origin.eof()) - return 0; + return std::make_pair(0, 0); CityHash_v1_0_2::uint128 checksum; - origin.readStrict(reinterpret_cast(&checksum), sizeof(checksum)); + origin.readStrict(reinterpret_cast(&checksum), CHECKSUM_SIZE); - own_compressed_buffer.resize(COMPRESSED_BLOCK_HEADER_SIZE); - origin.readStrict(own_compressed_buffer.data(), COMPRESSED_BLOCK_HEADER_SIZE); + UInt8 header_size = ICompressionCodec::getHeaderSize(); + own_compressed_buffer.resize(header_size); + origin.readStrict(own_compressed_buffer.data(), header_size); - size_compressed = unalignedLoad(&own_compressed_buffer[1]); - size_decompressed = unalignedLoad(&own_compressed_buffer[5]); + UInt8 method = ICompressionCodec::readMethod(own_compressed_buffer.data()); - if (size_compressed > DBMS_MAX_COMPRESSED_SIZE) - throw Exception("Too large size_compressed: " + toString(size_compressed) + ". Most likely corrupted data.", ErrorCodes::TOO_LARGE_SIZE_COMPRESSED); + if (method != codec->getMethodByte()) + throw Exception("Can't decompress with method " + getHexUIntLowercase(method) + ", with codec " + getHexUIntLowercase(codec->getMethodByte()), ErrorCodes::CANNOT_DECOMPRESS); - ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed + sizeof(checksum)); + UInt32 size_to_read_compressed = ICompressionCodec::readCompressedBlockSize(own_compressed_buffer.data()); + UInt32 size_decompressed = ICompressionCodec::readDecompressedBlockSize(own_compressed_buffer.data()); + + if (size_to_read_compressed > DBMS_MAX_COMPRESSED_SIZE) + throw Exception("Too large size_to_read_compressed: " + toString(size_to_read_compressed) + ". Most likely corrupted data.", ErrorCodes::TOO_LARGE_SIZE_COMPRESSED); + + ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_to_read_compressed + CHECKSUM_SIZE); /// Is whole compressed block located in 'origin' buffer? - if (origin.offset() >= COMPRESSED_BLOCK_HEADER_SIZE && - origin.position() + size_compressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER - COMPRESSED_BLOCK_HEADER_SIZE <= origin.buffer().end()) + if (origin.offset() >= header_size && + origin.position() + size_to_read_compressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER - header_size <= origin.buffer().end()) { - origin.position() -= COMPRESSED_BLOCK_HEADER_SIZE; + origin.position() -= header_size; compressed_buffer = origin.position(); - origin.position() += size_compressed; + origin.position() += size_to_read_compressed; } else { - own_compressed_buffer.resize(size_compressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); + own_compressed_buffer.resize(size_to_read_compressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); compressed_buffer = own_compressed_buffer.data(); - origin.readStrict(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, size_compressed - COMPRESSED_BLOCK_HEADER_SIZE); + origin.readStrict(compressed_buffer + header_size, size_to_read_compressed - header_size); } - auto checksum_calculated = CityHash_v1_0_2::CityHash128(compressed_buffer, size_compressed); + auto checksum_calculated = CityHash_v1_0_2::CityHash128(compressed_buffer, size_to_read_compressed); if (checksum != checksum_calculated) throw Exception("Checksum doesn't match: corrupted data." " Reference: " + getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second) + ". Actual: " + getHexUIntLowercase(checksum_calculated.first) + getHexUIntLowercase(checksum_calculated.second) - + ". Size of compressed block: " + toString(size_compressed) + ".", + + ". Size of compressed block: " + toString(size_to_read_compressed), ErrorCodes::CHECKSUM_DOESNT_MATCH); - return size_compressed + sizeof(checksum); + return std::make_pair(size_to_read_compressed, size_decompressed); } -void CompressionCodecReadBuffer::decompress(char * to, size_t size_decompressed, size_t size_compressed_without_checksum) +void CompressionCodecReadBuffer::decompress(char * to, UInt32 size_compressed) { - ProfileEvents::increment(ProfileEvents::CompressedReadBufferBlocks); - ProfileEvents::increment(ProfileEvents::CompressedReadBufferBytes, size_decompressed); + UInt8 method = ICompressionCodec::readMethod(compressed_buffer); - UInt8 current_method = compressed_buffer[0]; /// See CompressedWriteBuffer.h - if (!codec || current_method != method) - { - method = current_method; + if (!codec) codec = CompressionCodecFactory::instance().get(method); - } + else if (codec->getMethodByte() != method) + throw Exception("Can't decompress data with byte " + getHexUIntLowercase(method) + " expected byte " + getHexUIntLowercase(codec->getMethodByte()), ErrorCodes::CANNOT_DECOMPRESS); - codec->decompress(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, - size_compressed_without_checksum - COMPRESSED_BLOCK_HEADER_SIZE, to, size_decompressed); + codec->decompress(compressed_buffer, size_compressed, to); } bool CompressionCodecReadBuffer::nextImpl() { - size_t size_decompressed; - size_t size_compressed_without_checksum; + UInt32 size_decompressed; - size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum); - if (!size_compressed) + std::tie(read_compressed_bytes_for_last_time, size_decompressed) = readCompressedData(); + if (!read_compressed_bytes_for_last_time) return false; memory.resize(size_decompressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); working_buffer = Buffer(memory.data(), &memory[size_decompressed]); - decompress(working_buffer.begin(), size_decompressed, size_compressed_without_checksum); + decompress(working_buffer.begin(), read_compressed_bytes_for_last_time); + + ProfileEvents::increment(ProfileEvents::CompressedReadBufferBytes, size_decompressed); + ProfileEvents::increment(ProfileEvents::CompressedReadBufferBlocks); return true; } @@ -127,16 +177,19 @@ void CompressionCodecReadBuffer::seek(size_t offset_in_compressed_file, size_t o { if (const auto file_in = dynamic_cast(&origin)) { - if (size_compressed && - offset_in_compressed_file == file_in->getPositionInFile() - size_compressed && - offset_in_decompressed_block <= working_buffer.size()) + UInt32 readed_size_with_checksum = read_compressed_bytes_for_last_time + CHECKSUM_SIZE; + UInt32 last_readed_block_start_pos = file_in->getPositionInFile() - readed_size_with_checksum; + /// We seek in already uncompressed block + if (readed_size_with_checksum && /// we already have read something + offset_in_compressed_file == last_readed_block_start_pos && /// our position is exactly at required byte + offset_in_decompressed_block <= working_buffer.size()) /// our buffer size is more, than required position in uncompressed block { bytes += offset(); pos = working_buffer.begin() + offset_in_decompressed_block; /// `bytes` can overflow and get negative, but in `count()` everything will overflow back and get right. bytes -= offset(); } - else + else /// or we have to read and uncompress further { file_in->seek(offset_in_compressed_file); @@ -173,26 +226,18 @@ void CompressionCodecWriteBuffer::nextImpl() if (!offset()) return; - static constexpr size_t header_size = 1 + sizeof(UInt32) + sizeof(UInt32); + size_t decompressed_size = offset(); + UInt32 compressed_reserve_size = codec->getCompressedReserveSize(decompressed_size); + compressed_buffer.resize(compressed_reserve_size); + UInt32 compressed_size = codec->compress(working_buffer.begin(), decompressed_size, compressed_buffer.data()); - size_t uncompressed_size = offset(); - size_t compressed_reserve_size = compression_codec.getCompressedReserveSize(uncompressed_size); - - compressed_buffer.resize(header_size + compressed_reserve_size); - compressed_buffer[0] = compression_codec.getMethodByte(); - size_t compressed_size = header_size + compression_codec.compress(working_buffer.begin(), uncompressed_size, &compressed_buffer[header_size]); - - UInt32 compressed_size_32 = compressed_size; - UInt32 uncompressed_size_32 = uncompressed_size; - unalignedStore(&compressed_buffer[1], compressed_size_32); - unalignedStore(&compressed_buffer[5], uncompressed_size_32); CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(compressed_buffer.data(), compressed_size); - out.write(reinterpret_cast(&checksum), sizeof(checksum)); + out.write(reinterpret_cast(&checksum), CHECKSUM_SIZE); out.write(compressed_buffer.data(), compressed_size); } -CompressionCodecWriteBuffer::CompressionCodecWriteBuffer(ICompressionCodec & compression_codec, WriteBuffer & out, size_t buf_size) - : BufferWithOwnMemory(buf_size), out(out), compression_codec(compression_codec) +CompressionCodecWriteBuffer::CompressionCodecWriteBuffer(CompressionCodecPtr codec_, WriteBuffer & out_, size_t buf_size) + : BufferWithOwnMemory(buf_size), out(out_), codec(codec_) { } diff --git a/dbms/src/Compression/ICompressionCodec.h b/dbms/src/Compression/ICompressionCodec.h index 9e818ef969a..9a2babf527f 100644 --- a/dbms/src/Compression/ICompressionCodec.h +++ b/dbms/src/Compression/ICompressionCodec.h @@ -28,44 +28,45 @@ using CompressionCodecWriteBufferPtr = std::shared_ptr { public: - CompressionCodecWriteBuffer(ICompressionCodec & compression_codec, WriteBuffer & out, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + CompressionCodecWriteBuffer(CompressionCodecPtr codec_, WriteBuffer & out_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); ~CompressionCodecWriteBuffer() override; private: void nextImpl() override; - private: WriteBuffer & out; - ICompressionCodec & compression_codec; + CompressionCodecPtr codec; PODArray compressed_buffer; }; class CompressionCodecReadBuffer : public BufferWithOwnMemory { + + UInt32 read_compressed_bytes_for_last_time = 0; + public: - size_t size_compressed = 0; - size_t size_decompressed = 0; + std::pair readCompressedData(); - CompressionCodecReadBuffer(ReadBuffer & origin); + void decompress(char * to, UInt32 size_compressed); - size_t readCompressedData(size_t & size_decompressed, size_t & size_compressed); - - void decompress(char * to, size_t size_decompressed, size_t size_compressed_without_checksum); + CompressionCodecReadBuffer(CompressionCodecPtr codec_, ReadBuffer & origin_); void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); - private: + CompressionCodecPtr codec; ReadBuffer & origin; char * compressed_buffer; - UInt8 method; - CompressionCodecPtr codec; PODArray own_compressed_buffer; bool nextImpl() override; }; +CompressionCodecReadBufferPtr liftCompressed(CompressionCodecPtr codec, ReadBuffer & origin); + +CompressionCodecWriteBufferPtr liftCompressed(CompressionCodecPtr codec, WriteBuffer & origin); + /** * */ @@ -74,20 +75,31 @@ class ICompressionCodec : private boost::noncopyable public: virtual ~ICompressionCodec() = default; - CompressionCodecReadBufferPtr liftCompressed(ReadBuffer & origin); + virtual UInt8 getMethodByte() const = 0; - CompressionCodecWriteBufferPtr liftCompressed(WriteBuffer & origin); + virtual String getCodecDesc() const = 0; - virtual char getMethodByte() = 0; + virtual UInt32 compress(char * source, UInt32 source_size, char * dest) const; - /// TODO(alesap) FIXME - virtual void getCodecDesc(String & codec_desc) = 0; + virtual UInt32 decompress(char * source, UInt32 source_size, char * dest) const; - virtual size_t compress(char * source, size_t source_size, char * dest) = 0; + virtual UInt32 getCompressedReserveSize(UInt32 uncompressed_size) const { return getHeaderSize() + getCompressedDataSize(uncompressed_size); } - virtual size_t decompress(char * source, size_t source_size, char * dest, size_t decompressed_size) = 0; + static UInt8 getHeaderSize() { return 1 + 8; } - virtual size_t getCompressedReserveSize(size_t uncompressed_size) { return uncompressed_size; } + static UInt32 readCompressedBlockSize(const char * source); + + static UInt32 readDecompressedBlockSize(const char * source); + + static UInt8 readMethod(const char * source); + +protected: + + virtual UInt32 getCompressedDataSize(UInt32 uncompressed_size) const { return uncompressed_size; } + + virtual UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const = 0; + + virtual void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const = 0; }; } diff --git a/dbms/src/IO/CachedCompressedReadBuffer.cpp b/dbms/src/IO/CachedCompressedReadBuffer.cpp index bc36ed38afd..3eb74bb070d 100644 --- a/dbms/src/IO/CachedCompressedReadBuffer.cpp +++ b/dbms/src/IO/CachedCompressedReadBuffer.cpp @@ -20,7 +20,7 @@ void CachedCompressedReadBuffer::initInput() if (!file_in) { file_in = createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size); - in = codec->liftCompressed(*file_in); + in = liftCompressed(codec, *file_in); if (profile_callback) file_in->setProfileCallback(profile_callback, clock_type); @@ -42,17 +42,16 @@ bool CachedCompressedReadBuffer::nextImpl() owned_cell = std::make_shared(); - size_t size_decompressed; - size_t size_compressed_without_checksum; - owned_cell->compressed_size = in->readCompressedData(size_decompressed, size_compressed_without_checksum); + UInt32 size_decompressed; + std::tie(owned_cell->compressed_size, size_decompressed) = in->readCompressedData(); if (owned_cell->compressed_size) { owned_cell->data.resize(size_decompressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); - in->decompress(owned_cell->data.data(), size_decompressed, size_compressed_without_checksum); + in->decompress(owned_cell->data.data(), owned_cell->compressed_size); in->buffer() = Buffer(owned_cell->data.data(), owned_cell->data.data() + owned_cell->data.size() - LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); - in->decompress(owned_cell->data.data(), size_decompressed, size_compressed_without_checksum); + in->decompress(owned_cell->data.data(), owned_cell->compressed_size); /// Put data into cache. cache->set(key, owned_cell); @@ -74,9 +73,9 @@ bool CachedCompressedReadBuffer::nextImpl() CachedCompressedReadBuffer::CachedCompressedReadBuffer( - const std::string & path_, UncompressedCache * cache_, const CompressionCodecPtr & codec, + const std::string & path_, UncompressedCache * cache_, const CompressionCodecPtr & codec_, size_t estimated_size_, size_t aio_threshold_, size_t buf_size_) - : ReadBuffer(nullptr, 0), path(path_), cache(cache_), codec(codec), buf_size(buf_size_), estimated_size(estimated_size_), + : ReadBuffer(nullptr, 0), path(path_), cache(cache_), codec(codec_), buf_size(buf_size_), estimated_size(estimated_size_), aio_threshold(aio_threshold_), file_pos(0) { } diff --git a/dbms/src/IO/CachedCompressedReadBuffer.h b/dbms/src/IO/CachedCompressedReadBuffer.h index 2aaaaa21a1c..f0b2b725b10 100644 --- a/dbms/src/IO/CachedCompressedReadBuffer.h +++ b/dbms/src/IO/CachedCompressedReadBuffer.h @@ -45,7 +45,7 @@ private: public: CachedCompressedReadBuffer( - const std::string & path_, UncompressedCache * cache_, const CompressionCodecPtr & codec, + const std::string & path_, UncompressedCache * cache_, const CompressionCodecPtr & codec_, size_t estimated_size_, size_t aio_threshold_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 2b27aa11b7b..1f90d8048c0 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -371,8 +371,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) const auto ct = columns.codecs.find(column.name); if (ct != std::end(columns.codecs)) { - String codec_desc; - ct->second->getCodecDesc(codec_desc); + String codec_desc = ct->second->getCodecDesc(); codec_desc = "CODEC(" + codec_desc + ")"; auto pos = codec_desc.data(); const auto end = pos + codec_desc.size(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index 1b7b3c75982..39e3bb3c725 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -252,7 +252,7 @@ MergeTreeReader::Stream::Stream( if (profile_callback) file_in->setProfileCallback(profile_callback, clock_type); - const auto compressed_buffer = codec->liftCompressed(*file_in); + const auto compressed_buffer = liftCompressed(codec, *file_in); non_cached_buffer = compressed_buffer; data_buffer = non_cached_buffer.get(); } diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 5bf356201ee..6d2d5c132b5 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -192,7 +192,7 @@ IMergedBlockOutputStream::ColumnStream::ColumnStream( data_file_extension{data_file_extension_}, marks_file_extension{marks_file_extension_}, plain_file(createWriteBufferFromFileBase(data_path + data_file_extension, estimated_size, aio_threshold, max_compress_block_size)), - plain_hashing(*plain_file), compressed_buf(compression_codec->liftCompressed(plain_hashing)), compressed(*compressed_buf.get()), + plain_hashing(*plain_file), compressed_buf(liftCompressed(compression_codec, plain_hashing)), compressed(*compressed_buf.get()), marks_file(marks_path + marks_file_extension, 4096, O_TRUNC | O_CREAT | O_WRONLY), marks(marks_file) { } diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference index 4832f7f06b6..27037ff5641 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference @@ -11,3 +11,5 @@ 9175437371954010821 1.5555555555555 hello world! [77] ['John'] 7.1000000000000 xxxxxxxxxxxx [127] ['Henry'] +! +222 diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index e1860a8702c..c7d927c30b3 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -49,19 +49,19 @@ INSERT INTO test.compression_codec_multiple VALUES (1, 'world', toDate('2018-10- SELECT * FROM test.compression_codec_multiple ORDER BY id; -INSERT INTO test.compression_codec_multiple select modulo(number, 100), toString(number), toDate('2018-12-01'), 5.5 * number from system.numbers limit 10000; +INSERT INTO test.compression_codec_multiple select modulo(number, 100), toString(number), toDate('2018-12-01'), 5.5 * number FROM system.numbers limit 10000; -SELECT count(*) from test.compression_codec_multiple; +SELECT count(*) FROM test.compression_codec_multiple; -SELECT count(distinct data) from test.compression_codec_multiple; +SELECT count(distinct data) FROM test.compression_codec_multiple; -SELECT floor(sum(somenum), 1) from test.compression_codec_multiple; +SELECT floor(sum(somenum), 1) FROM test.compression_codec_multiple; TRUNCATE TABLE test.compression_codec_multiple; -INSERT INTO test.compression_codec_multiple select modulo(number, 100), toString(number), toDate('2018-12-01'), 5.5 * number from system.numbers limit 10000; +INSERT INTO test.compression_codec_multiple select modulo(number, 100), toString(number), toDate('2018-12-01'), 5.5 * number FROM system.numbers limit 10000; -SELECT sum(cityHash64(*)) from test.compression_codec_multiple; +SELECT sum(cityHash64(*)) FROM test.compression_codec_multiple; DROP TABLE IF EXISTS test.compression_codec_multiple_more_types; @@ -75,3 +75,22 @@ INSERT INTO test.compression_codec_multiple_more_types VALUES(1.5555555555555, ' INSERT INTO test.compression_codec_multiple_more_types VALUES(7.1, 'xxxxxxxxxxxx', [127], ['Henry']); SELECT * FROM test.compression_codec_multiple_more_types order by id; + +DROP TABLE IF EXISTS test.compression_codec_multiple_with_key; + +CREATE TABLE test.compression_codec_multiple_with_key ( + somedate Date CODEC(ZSTD, ZSTD, ZSTD(12)), + id UInt64 CODEC(LZ4, ZSTD, NONE), + data String CODEC(ZSTD(2), NONE, LZ4, LZ4) +) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; + + +INSERT INTO test.compression_codec_multiple_with_key VALUES(toDate('2018-10-12'), 100000, 'hello'), (toDate('2018-10-12'), 100002, 'world'), (toDate('2018-10-12'), 1111, '!'); + +SELECT data FROM test.compression_codec_multiple_with_key WHERE id BETWEEN 3 AND 1112; + +INSERT INTO test.compression_codec_multiple_with_key SELECT toDate('2018-10-12'), number, toString(number) FROM system.numbers LIMIT 1000; + +SELECT COUNT(DISTINCT data) FROM test.compression_codec_multiple_with_key WHERE id < 222; + +DROP TABLE IF EXISTS test.compression_codec_multiple_with_key; From 04902c569ad0bb30662476da0c95d189d8fa544b Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 20 Dec 2018 13:27:38 +0300 Subject: [PATCH 041/230] Non effective, but working code --- dbms/src/Compression/CompressionCodecLZ4.h | 1 + .../Compression/CompressionCodecMultiple.cpp | 18 ++---------------- dbms/src/Compression/ICompressionCodec.cpp | 6 +++--- dbms/src/Compression/ICompressionCodec.h | 2 ++ 4 files changed, 8 insertions(+), 19 deletions(-) diff --git a/dbms/src/Compression/CompressionCodecLZ4.h b/dbms/src/Compression/CompressionCodecLZ4.h index 1e0a00fa753..6af959ef4d8 100644 --- a/dbms/src/Compression/CompressionCodecLZ4.h +++ b/dbms/src/Compression/CompressionCodecLZ4.h @@ -16,6 +16,7 @@ public: String getCodecDesc() const override; + UInt32 getAdditionalSizeAtTheEndOfBuffer() const override { return LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER; } private: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; diff --git a/dbms/src/Compression/CompressionCodecMultiple.cpp b/dbms/src/Compression/CompressionCodecMultiple.cpp index d74a7129402..77514d6421e 100644 --- a/dbms/src/Compression/CompressionCodecMultiple.cpp +++ b/dbms/src/Compression/CompressionCodecMultiple.cpp @@ -71,15 +71,8 @@ UInt32 CompressionCodecMultiple::doCompressData(const char * source, UInt32 sour source_size = size_compressed; } - //std::cerr << "(compress) BUF_SIZE_COMPRESSED:" << source_size << std::endl; - memcpy(&dest[1 + codecs.size()], uncompressed_buf.data(), source_size); - //std::cerr << "(compress) COMPRESSING BUF:\n"; - //for (size_t i = 0; i < source_size + 1 + codecs.size(); ++i) - // std::cerr << getHexUIntLowercase(+dest[i]) << " "; - //std::cerr << std::endl; - return 1 + codecs.size() + source_size; } @@ -87,13 +80,6 @@ void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 sour { UInt8 compression_methods_size = source[0]; - //std::cerr << "(decompress) DECOMPRESSING BUF:\n"; - //for (size_t i = 0; i < source_size; ++i) - // std::cerr << getHexUIntLowercase(+source[i]) << " "; - //std::cerr << std::endl; - - //std::cerr << "(decompress) BUF_SIZE_COMPRESSED:" << source_size << std::endl; - //std::cerr << "(decompress) CODECS SIZE:" << +compression_methods_size << std::endl; PODArray compressed_buf(&source[compression_methods_size + 1], &source[source_size]); PODArray uncompressed_buf; /// Insert all data into compressed buf @@ -103,13 +89,13 @@ void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 sour { UInt8 compression_method = source[idx + 1]; const auto codec = CompressionCodecFactory::instance().get(compression_method); + compressed_buf.resize(compressed_buf.size() + codec->getAdditionalSizeAtTheEndOfBuffer()); UInt32 uncompressed_size = ICompressionCodec::readDecompressedBlockSize(compressed_buf.data()); - //std::cerr << "(decompress) UNCOMPRESSED SIZE READ:" << uncompressed_size << std::endl; if (idx == 0 && uncompressed_size != decompressed_size) throw Exception("Wrong final decompressed size in codec Multiple, got " + toString(uncompressed_size) + ", expected " + toString(decompressed_size), ErrorCodes::CORRUPTED_DATA); - uncompressed_buf.resize(uncompressed_size); + uncompressed_buf.resize(uncompressed_size + codec->getAdditionalSizeAtTheEndOfBuffer()); codec->decompress(compressed_buf.data(), source_size, uncompressed_buf.data()); uncompressed_buf.swap(compressed_buf); source_size = uncompressed_size; diff --git a/dbms/src/Compression/ICompressionCodec.cpp b/dbms/src/Compression/ICompressionCodec.cpp index df1efacf0cc..879bafa80ec 100644 --- a/dbms/src/Compression/ICompressionCodec.cpp +++ b/dbms/src/Compression/ICompressionCodec.cpp @@ -118,7 +118,7 @@ std::pair CompressionCodecReadBuffer::readCompressedData() /// Is whole compressed block located in 'origin' buffer? if (origin.offset() >= header_size && - origin.position() + size_to_read_compressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER - header_size <= origin.buffer().end()) + origin.position() + size_to_read_compressed + codec->getAdditionalSizeAtTheEndOfBuffer() - header_size <= origin.buffer().end()) { origin.position() -= header_size; compressed_buffer = origin.position(); @@ -126,7 +126,7 @@ std::pair CompressionCodecReadBuffer::readCompressedData() } else { - own_compressed_buffer.resize(size_to_read_compressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); + own_compressed_buffer.resize(size_to_read_compressed + codec->getAdditionalSizeAtTheEndOfBuffer()); compressed_buffer = own_compressed_buffer.data(); origin.readStrict(compressed_buffer + header_size, size_to_read_compressed - header_size); } @@ -162,7 +162,7 @@ bool CompressionCodecReadBuffer::nextImpl() if (!read_compressed_bytes_for_last_time) return false; - memory.resize(size_decompressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); + memory.resize(size_decompressed + codec->getAdditionalSizeAtTheEndOfBuffer()); working_buffer = Buffer(memory.data(), &memory[size_decompressed]); decompress(working_buffer.begin(), read_compressed_bytes_for_last_time); diff --git a/dbms/src/Compression/ICompressionCodec.h b/dbms/src/Compression/ICompressionCodec.h index 9a2babf527f..49bf4e2efc1 100644 --- a/dbms/src/Compression/ICompressionCodec.h +++ b/dbms/src/Compression/ICompressionCodec.h @@ -85,6 +85,8 @@ public: virtual UInt32 getCompressedReserveSize(UInt32 uncompressed_size) const { return getHeaderSize() + getCompressedDataSize(uncompressed_size); } + virtual UInt32 getAdditionalSizeAtTheEndOfBuffer() const { return 0; } + static UInt8 getHeaderSize() { return 1 + 8; } static UInt32 readCompressedBlockSize(const char * source); From aee6c0a606bbbb59b4c12b36104c91b05d0d3930 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 20 Dec 2018 20:37:02 +0300 Subject: [PATCH 042/230] Remove dump code --- dbms/CMakeLists.txt | 3 +- dbms/programs/server/TCPHandler.cpp | 2 +- dbms/src/Compression/CompressionFactory.cpp | 15 ++ dbms/src/Compression/CompressionFactory.h | 7 +- dbms/src/Compression/ICompressionCodec.cpp | 170 ------------------ dbms/src/Compression/ICompressionCodec.h | 48 ----- dbms/src/IO/CachedCompressedReadBuffer.cpp | 21 ++- dbms/src/IO/CachedCompressedReadBuffer.h | 7 +- dbms/src/IO/CompressedReadBuffer.cpp | 6 +- dbms/src/IO/CompressedReadBufferBase.cpp | 94 +++++----- dbms/src/IO/CompressedReadBufferBase.h | 7 +- dbms/src/IO/CompressedWriteBuffer.cpp | 120 ++----------- dbms/src/IO/CompressedWriteBuffer.h | 10 +- .../tests/cached_compressed_read_buffer.cpp | 4 +- dbms/src/Storages/ColumnsDescription.cpp | 7 +- dbms/src/Storages/ColumnsDescription.h | 9 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 - .../Storages/MergeTree/MergeTreeReader.cpp | 21 +-- dbms/src/Storages/MergeTree/MergeTreeReader.h | 14 +- .../MergeTree/MergedBlockOutputStream.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.h | 2 +- dbms/src/Storages/StorageLog.cpp | 2 +- .../System/StorageSystemPartsBase.cpp | 2 +- 23 files changed, 143 insertions(+), 432 deletions(-) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 109b289839d..f78bebade2f 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -56,6 +56,7 @@ set(dbms_sources) include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) add_headers_and_sources(clickhouse_common_io src/Common) +add_headers_and_sources(clickhouse_common_io src/Compression) add_headers_and_sources(clickhouse_common_io src/Common/HashTable) add_headers_and_sources(clickhouse_common_io src/IO) @@ -72,7 +73,6 @@ add_headers_and_sources(dbms src/Storages/Kafka) add_headers_and_sources(dbms src/Storages/MergeTree) add_headers_and_sources(dbms src/Client) add_headers_and_sources(dbms src/Formats) -add_headers_and_sources(dbms src/Compression) list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) @@ -151,6 +151,7 @@ target_link_libraries (clickhouse_common_io PUBLIC common PRIVATE + clickhouse_parsers string_utils widechar_width ${LINK_LIBRARIES_ONLY_ON_X86_64} diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index cfb0cd3cd58..23d20892c51 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -750,7 +750,7 @@ bool TCPHandler::receiveData() { NamesAndTypesList columns = block.getNamesAndTypesList(); storage = StorageMemory::create(external_table_name, - ColumnsDescription{columns, NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}, ColumnComments{}}); + ColumnsDescription{columns, NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}, ColumnComments{}, ColumnCodecs{}}); storage->startup(); query_context.addExternalTable(external_table_name, storage); } diff --git a/dbms/src/Compression/CompressionFactory.cpp b/dbms/src/Compression/CompressionFactory.cpp index dc8ab992a3e..58b72012755 100644 --- a/dbms/src/Compression/CompressionFactory.cpp +++ b/dbms/src/Compression/CompressionFactory.cpp @@ -29,6 +29,13 @@ CompressionCodecPtr CompressionCodecFactory::getDefaultCodec() const return default_codec; } + +CompressionCodecPtr CompressionCodecFactory::get(const CompressionSettings & settings) const +{ + ///TODO implement + return default_codec; +} + CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast) const { if (const auto * func = typeid_cast(ast.get())) @@ -101,6 +108,14 @@ void CompressionCodecFactory::registerSimpleCompressionCodec(const String & fami }); } + +ASTPtr CompressionCodecFactory::convertSettingsToAst(const CompressionSettings & settings) const +{ + ///TODO Implement + return nullptr; + //makeASTFunction(); +} + void registerCodecLZ4(CompressionCodecFactory & factory); void registerCodecNone(CompressionCodecFactory & factory); void registerCodecZSTD(CompressionCodecFactory & factory); diff --git a/dbms/src/Compression/CompressionFactory.h b/dbms/src/Compression/CompressionFactory.h index 0298abb07e4..b474aa887ee 100644 --- a/dbms/src/Compression/CompressionFactory.h +++ b/dbms/src/Compression/CompressionFactory.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -36,6 +37,8 @@ public: CompressionCodecPtr get(const UInt8 byte_code) const; + CompressionCodecPtr get(const CompressionSettings & settings) const; + void registerCompressionCodec(const String & family_name, UInt8 byte_code, Creator creator); void registerSimpleCompressionCodec(const String & family_name, UInt8 byte_code, SimpleCreator creator); @@ -48,9 +51,11 @@ private: CompressionCodecsCodeDictionary family_code_with_codec; CompressionCodecPtr default_codec; + ASTPtr convertSettingsToAst(const CompressionSettings & settings) const; + CompressionCodecFactory(); friend class ext::singleton; }; -} \ No newline at end of file +} diff --git a/dbms/src/Compression/ICompressionCodec.cpp b/dbms/src/Compression/ICompressionCodec.cpp index 879bafa80ec..99aaf4d9ef2 100644 --- a/dbms/src/Compression/ICompressionCodec.cpp +++ b/dbms/src/Compression/ICompressionCodec.cpp @@ -28,7 +28,6 @@ namespace ErrorCodes extern const int SEEK_POSITION_OUT_OF_BOUND; } -static constexpr auto CHECKSUM_SIZE{sizeof(CityHash_v1_0_2::uint128)}; UInt32 ICompressionCodec::compress(char * source, UInt32 source_size, char * dest) const { @@ -72,173 +71,4 @@ UInt8 ICompressionCodec::readMethod(const char * source) return static_cast(source[0]); } -CompressionCodecReadBufferPtr liftCompressed(CompressionCodecPtr codec, ReadBuffer & origin) -{ - return std::make_shared(codec, origin); -} - -CompressionCodecWriteBufferPtr liftCompressed(CompressionCodecPtr codec, WriteBuffer & origin) -{ - return std::make_shared(codec, origin); -} - -CompressionCodecReadBuffer::CompressionCodecReadBuffer(CompressionCodecPtr codec_, ReadBuffer & origin_) - : codec(codec_) - , origin(origin_) -{ -} - - -/// Read compressed data into compressed_buffer. Get size of decompressed data from block header. Checksum if need. -/// Returns number of compressed bytes read. -std::pair CompressionCodecReadBuffer::readCompressedData() -{ - if (origin.eof()) - return std::make_pair(0, 0); - - CityHash_v1_0_2::uint128 checksum; - origin.readStrict(reinterpret_cast(&checksum), CHECKSUM_SIZE); - - UInt8 header_size = ICompressionCodec::getHeaderSize(); - own_compressed_buffer.resize(header_size); - origin.readStrict(own_compressed_buffer.data(), header_size); - - UInt8 method = ICompressionCodec::readMethod(own_compressed_buffer.data()); - - if (method != codec->getMethodByte()) - throw Exception("Can't decompress with method " + getHexUIntLowercase(method) + ", with codec " + getHexUIntLowercase(codec->getMethodByte()), ErrorCodes::CANNOT_DECOMPRESS); - - UInt32 size_to_read_compressed = ICompressionCodec::readCompressedBlockSize(own_compressed_buffer.data()); - UInt32 size_decompressed = ICompressionCodec::readDecompressedBlockSize(own_compressed_buffer.data()); - - if (size_to_read_compressed > DBMS_MAX_COMPRESSED_SIZE) - throw Exception("Too large size_to_read_compressed: " + toString(size_to_read_compressed) + ". Most likely corrupted data.", ErrorCodes::TOO_LARGE_SIZE_COMPRESSED); - - ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_to_read_compressed + CHECKSUM_SIZE); - - /// Is whole compressed block located in 'origin' buffer? - if (origin.offset() >= header_size && - origin.position() + size_to_read_compressed + codec->getAdditionalSizeAtTheEndOfBuffer() - header_size <= origin.buffer().end()) - { - origin.position() -= header_size; - compressed_buffer = origin.position(); - origin.position() += size_to_read_compressed; - } - else - { - own_compressed_buffer.resize(size_to_read_compressed + codec->getAdditionalSizeAtTheEndOfBuffer()); - compressed_buffer = own_compressed_buffer.data(); - origin.readStrict(compressed_buffer + header_size, size_to_read_compressed - header_size); - } - - auto checksum_calculated = CityHash_v1_0_2::CityHash128(compressed_buffer, size_to_read_compressed); - if (checksum != checksum_calculated) - throw Exception("Checksum doesn't match: corrupted data." - " Reference: " + getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second) - + ". Actual: " + getHexUIntLowercase(checksum_calculated.first) + getHexUIntLowercase(checksum_calculated.second) - + ". Size of compressed block: " + toString(size_to_read_compressed), - ErrorCodes::CHECKSUM_DOESNT_MATCH); - - return std::make_pair(size_to_read_compressed, size_decompressed); -} - -void CompressionCodecReadBuffer::decompress(char * to, UInt32 size_compressed) -{ - UInt8 method = ICompressionCodec::readMethod(compressed_buffer); - - if (!codec) - codec = CompressionCodecFactory::instance().get(method); - else if (codec->getMethodByte() != method) - throw Exception("Can't decompress data with byte " + getHexUIntLowercase(method) + " expected byte " + getHexUIntLowercase(codec->getMethodByte()), ErrorCodes::CANNOT_DECOMPRESS); - - codec->decompress(compressed_buffer, size_compressed, to); -} - -bool CompressionCodecReadBuffer::nextImpl() -{ - UInt32 size_decompressed; - - std::tie(read_compressed_bytes_for_last_time, size_decompressed) = readCompressedData(); - if (!read_compressed_bytes_for_last_time) - return false; - - memory.resize(size_decompressed + codec->getAdditionalSizeAtTheEndOfBuffer()); - working_buffer = Buffer(memory.data(), &memory[size_decompressed]); - - decompress(working_buffer.begin(), read_compressed_bytes_for_last_time); - - ProfileEvents::increment(ProfileEvents::CompressedReadBufferBytes, size_decompressed); - ProfileEvents::increment(ProfileEvents::CompressedReadBufferBlocks); - - return true; -} - -void CompressionCodecReadBuffer::seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block) -{ - if (const auto file_in = dynamic_cast(&origin)) - { - UInt32 readed_size_with_checksum = read_compressed_bytes_for_last_time + CHECKSUM_SIZE; - UInt32 last_readed_block_start_pos = file_in->getPositionInFile() - readed_size_with_checksum; - /// We seek in already uncompressed block - if (readed_size_with_checksum && /// we already have read something - offset_in_compressed_file == last_readed_block_start_pos && /// our position is exactly at required byte - offset_in_decompressed_block <= working_buffer.size()) /// our buffer size is more, than required position in uncompressed block - { - bytes += offset(); - pos = working_buffer.begin() + offset_in_decompressed_block; - /// `bytes` can overflow and get negative, but in `count()` everything will overflow back and get right. - bytes -= offset(); - } - else /// or we have to read and uncompress further - { - file_in->seek(offset_in_compressed_file); - - bytes += offset(); - nextImpl(); - - if (offset_in_decompressed_block > working_buffer.size()) - throw Exception("Seek position is beyond the decompressed block" - " (pos: " + toString(offset_in_decompressed_block) + ", block size: " + toString(working_buffer.size()) + ")", - ErrorCodes::SEEK_POSITION_OUT_OF_BOUND); - - pos = working_buffer.begin() + offset_in_decompressed_block; - bytes -= offset(); - } - } - else - throw Exception("CompressionCodec: cannot seek in non-file buffer", ErrorCodes::LOGICAL_ERROR); -} - -CompressionCodecWriteBuffer::~CompressionCodecWriteBuffer() -{ - try - { - next(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} - -void CompressionCodecWriteBuffer::nextImpl() -{ - if (!offset()) - return; - - size_t decompressed_size = offset(); - UInt32 compressed_reserve_size = codec->getCompressedReserveSize(decompressed_size); - compressed_buffer.resize(compressed_reserve_size); - UInt32 compressed_size = codec->compress(working_buffer.begin(), decompressed_size, compressed_buffer.data()); - - CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(compressed_buffer.data(), compressed_size); - out.write(reinterpret_cast(&checksum), CHECKSUM_SIZE); - out.write(compressed_buffer.data(), compressed_size); -} - -CompressionCodecWriteBuffer::CompressionCodecWriteBuffer(CompressionCodecPtr codec_, WriteBuffer & out_, size_t buf_size) - : BufferWithOwnMemory(buf_size), out(out_), codec(codec_) -{ -} - } diff --git a/dbms/src/Compression/ICompressionCodec.h b/dbms/src/Compression/ICompressionCodec.h index 49bf4e2efc1..8a4c05b7210 100644 --- a/dbms/src/Compression/ICompressionCodec.h +++ b/dbms/src/Compression/ICompressionCodec.h @@ -19,54 +19,6 @@ class ICompressionCodec; using CompressionCodecPtr = std::shared_ptr; using Codecs = std::vector; -class CompressionCodecReadBuffer; -class CompressionCodecWriteBuffer; - -using CompressionCodecReadBufferPtr = std::shared_ptr; -using CompressionCodecWriteBufferPtr = std::shared_ptr; - -class CompressionCodecWriteBuffer : public BufferWithOwnMemory -{ -public: - CompressionCodecWriteBuffer(CompressionCodecPtr codec_, WriteBuffer & out_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); - - ~CompressionCodecWriteBuffer() override; - -private: - void nextImpl() override; - -private: - WriteBuffer & out; - CompressionCodecPtr codec; - PODArray compressed_buffer; -}; - -class CompressionCodecReadBuffer : public BufferWithOwnMemory -{ - - UInt32 read_compressed_bytes_for_last_time = 0; - -public: - std::pair readCompressedData(); - - void decompress(char * to, UInt32 size_compressed); - - CompressionCodecReadBuffer(CompressionCodecPtr codec_, ReadBuffer & origin_); - - void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); -private: - CompressionCodecPtr codec; - ReadBuffer & origin; - char * compressed_buffer; - PODArray own_compressed_buffer; - - bool nextImpl() override; -}; - -CompressionCodecReadBufferPtr liftCompressed(CompressionCodecPtr codec, ReadBuffer & origin); - -CompressionCodecWriteBufferPtr liftCompressed(CompressionCodecPtr codec, WriteBuffer & origin); - /** * */ diff --git a/dbms/src/IO/CachedCompressedReadBuffer.cpp b/dbms/src/IO/CachedCompressedReadBuffer.cpp index 3eb74bb070d..7d8d809a254 100644 --- a/dbms/src/IO/CachedCompressedReadBuffer.cpp +++ b/dbms/src/IO/CachedCompressedReadBuffer.cpp @@ -20,7 +20,7 @@ void CachedCompressedReadBuffer::initInput() if (!file_in) { file_in = createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size); - in = liftCompressed(codec, *file_in); + compressed_in = &*file_in; if (profile_callback) file_in->setProfileCallback(profile_callback, clock_type); @@ -42,16 +42,15 @@ bool CachedCompressedReadBuffer::nextImpl() owned_cell = std::make_shared(); - UInt32 size_decompressed; - std::tie(owned_cell->compressed_size, size_decompressed) = in->readCompressedData(); + + size_t size_decompressed; + size_t size_compressed_without_checksum; + owned_cell->compressed_size = readCompressedData(size_decompressed, size_compressed_without_checksum); if (owned_cell->compressed_size) { - owned_cell->data.resize(size_decompressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); - in->decompress(owned_cell->data.data(), owned_cell->compressed_size); - - in->buffer() = Buffer(owned_cell->data.data(), owned_cell->data.data() + owned_cell->data.size() - LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); - in->decompress(owned_cell->data.data(), owned_cell->compressed_size); + owned_cell->data.resize(size_decompressed + codec->getAdditionalSizeAtTheEndOfBuffer()); + decompress(owned_cell->data.data(), size_decompressed, owned_cell->compressed_size); /// Put data into cache. cache->set(key, owned_cell); @@ -64,7 +63,7 @@ bool CachedCompressedReadBuffer::nextImpl() return false; } - working_buffer = Buffer(owned_cell->data.data(), owned_cell->data.data() + owned_cell->data.size() - LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); + working_buffer = Buffer(owned_cell->data.data(), owned_cell->data.data() + owned_cell->data.size() - codec->getAdditionalSizeAtTheEndOfBuffer()); file_pos += owned_cell->compressed_size; @@ -73,9 +72,9 @@ bool CachedCompressedReadBuffer::nextImpl() CachedCompressedReadBuffer::CachedCompressedReadBuffer( - const std::string & path_, UncompressedCache * cache_, const CompressionCodecPtr & codec_, + const std::string & path_, UncompressedCache * cache_, size_t estimated_size_, size_t aio_threshold_, size_t buf_size_) - : ReadBuffer(nullptr, 0), path(path_), cache(cache_), codec(codec_), buf_size(buf_size_), estimated_size(estimated_size_), + : ReadBuffer(nullptr, 0), path(path_), cache(cache_), buf_size(buf_size_), estimated_size(estimated_size_), aio_threshold(aio_threshold_), file_pos(0) { } diff --git a/dbms/src/IO/CachedCompressedReadBuffer.h b/dbms/src/IO/CachedCompressedReadBuffer.h index f0b2b725b10..9135f0482ca 100644 --- a/dbms/src/IO/CachedCompressedReadBuffer.h +++ b/dbms/src/IO/CachedCompressedReadBuffer.h @@ -19,17 +19,15 @@ namespace DB * Disadvantages: * - in case you need to read a lot of data in a row, but of them only a part is cached, you have to do seek-and. */ -class CachedCompressedReadBuffer : public ReadBuffer +class CachedCompressedReadBuffer : public CompressedReadBufferBase, public ReadBuffer { private: const std::string path; UncompressedCache * cache; - const CompressionCodecPtr & codec; size_t buf_size; size_t estimated_size; size_t aio_threshold; - CompressionCodecReadBufferPtr in; std::unique_ptr file_in; size_t file_pos; @@ -45,8 +43,7 @@ private: public: CachedCompressedReadBuffer( - const std::string & path_, UncompressedCache * cache_, const CompressionCodecPtr & codec_, - size_t estimated_size_, size_t aio_threshold_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); + const std::string & path_, UncompressedCache * cache_, size_t estimated_size_, size_t aio_threshold_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); diff --git a/dbms/src/IO/CompressedReadBuffer.cpp b/dbms/src/IO/CompressedReadBuffer.cpp index 6e684ee444f..0df4664ba5e 100644 --- a/dbms/src/IO/CompressedReadBuffer.cpp +++ b/dbms/src/IO/CompressedReadBuffer.cpp @@ -14,7 +14,7 @@ bool CompressedReadBuffer::nextImpl() if (!size_compressed) return false; - memory.resize(size_decompressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); + memory.resize(size_decompressed + codec->getAdditionalSizeAtTheEndOfBuffer()); working_buffer = Buffer(memory.data(), &memory[size_decompressed]); decompress(working_buffer.begin(), size_decompressed, size_compressed_without_checksum); @@ -40,7 +40,7 @@ size_t CompressedReadBuffer::readBig(char * to, size_t n) return bytes_read; /// If the decompressed block is placed entirely where it needs to be copied. - if (size_decompressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER <= n - bytes_read) + if (size_decompressed + codec->getAdditionalSizeAtTheEndOfBuffer() <= n - bytes_read) { decompress(to + bytes_read, size_decompressed, size_compressed_without_checksum); bytes_read += size_decompressed; @@ -49,7 +49,7 @@ size_t CompressedReadBuffer::readBig(char * to, size_t n) else { bytes += offset(); - memory.resize(size_decompressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); + memory.resize(size_decompressed + codec->getAdditionalSizeAtTheEndOfBuffer()); working_buffer = Buffer(memory.data(), &memory[size_decompressed]); pos = working_buffer.begin(); diff --git a/dbms/src/IO/CompressedReadBufferBase.cpp b/dbms/src/IO/CompressedReadBufferBase.cpp index 689924919a1..65b7af852ab 100644 --- a/dbms/src/IO/CompressedReadBufferBase.cpp +++ b/dbms/src/IO/CompressedReadBufferBase.cpp @@ -11,6 +11,8 @@ #include #include #include +#include +#include #include #include #include @@ -35,6 +37,7 @@ namespace ErrorCodes extern const int CANNOT_DECOMPRESS; } +static constexpr auto CHECKSUM_SIZE{sizeof(CityHash_v1_0_2::uint128)}; /// Read compressed data into compressed_buffer. Get size of decompressed data from block header. Checksum if need. /// Returns number of compressed bytes read. @@ -44,57 +47,55 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, return 0; CityHash_v1_0_2::uint128 checksum; - compressed_in->readStrict(reinterpret_cast(&checksum), sizeof(checksum)); + compressed_in->readStrict(reinterpret_cast(&checksum), CHECKSUM_SIZE); - own_compressed_buffer.resize(COMPRESSED_BLOCK_HEADER_SIZE); - compressed_in->readStrict(own_compressed_buffer.data(), COMPRESSED_BLOCK_HEADER_SIZE); + UInt8 header_size = ICompressionCodec::getHeaderSize(); + own_compressed_buffer.resize(header_size); + compressed_in->readStrict(own_compressed_buffer.data(), header_size); - UInt8 method = own_compressed_buffer[0]; /// See CompressedWriteBuffer.h + UInt8 method = ICompressionCodec::readMethod(own_compressed_buffer.data()); - size_t & size_compressed = size_compressed_without_checksum; + if (!codec) + codec = CompressionCodecFactory::instance().get(method); + else if (method != codec->getMethodByte()) + throw Exception("Data compressed with different method, given method byte " + getHexUIntLowercase(method) + ", previous method byte " + getHexUIntLowercase(codec->getMethodByte()), ErrorCodes::CANNOT_DECOMPRESS); - if (method == static_cast(CompressionMethodByte::LZ4) || - method == static_cast(CompressionMethodByte::ZSTD) || - method == static_cast(CompressionMethodByte::NONE)) + size_compressed_without_checksum = ICompressionCodec::readCompressedBlockSize(own_compressed_buffer.data()); + size_decompressed = ICompressionCodec::readDecompressedBlockSize(own_compressed_buffer.data()); + + if (size_compressed_without_checksum > DBMS_MAX_COMPRESSED_SIZE) + throw Exception("Too large size_compressed_without_checksum: " + toString(size_compressed_without_checksum) + ". Most likely corrupted data.", ErrorCodes::TOO_LARGE_SIZE_COMPRESSED); + + ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + CHECKSUM_SIZE); + + /// Is whole compressed block located in 'compressed_in->' buffer? + if (compressed_in->offset() >= header_size && + compressed_in->position() + size_compressed_without_checksum + codec->getAdditionalSizeAtTheEndOfBuffer() - header_size <= compressed_in->buffer().end()) { - size_compressed = unalignedLoad(&own_compressed_buffer[1]); - size_decompressed = unalignedLoad(&own_compressed_buffer[5]); - } - else - throw Exception("Unknown compression method: " + toString(method), ErrorCodes::UNKNOWN_COMPRESSION_METHOD); - - if (size_compressed > DBMS_MAX_COMPRESSED_SIZE) - throw Exception("Too large size_compressed: " + toString(size_compressed) + ". Most likely corrupted data.", ErrorCodes::TOO_LARGE_SIZE_COMPRESSED); - - ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed + sizeof(checksum)); - - /// Is whole compressed block located in 'compressed_in' buffer? - if (compressed_in->offset() >= COMPRESSED_BLOCK_HEADER_SIZE && - compressed_in->position() + size_compressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER - COMPRESSED_BLOCK_HEADER_SIZE <= compressed_in->buffer().end()) - { - compressed_in->position() -= COMPRESSED_BLOCK_HEADER_SIZE; + compressed_in->position() -= header_size; compressed_buffer = compressed_in->position(); - compressed_in->position() += size_compressed; + compressed_in->position() += size_compressed_without_checksum; } else { - own_compressed_buffer.resize(size_compressed + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); + own_compressed_buffer.resize(size_compressed_without_checksum + codec->getAdditionalSizeAtTheEndOfBuffer()); compressed_buffer = own_compressed_buffer.data(); - compressed_in->readStrict(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, size_compressed - COMPRESSED_BLOCK_HEADER_SIZE); + compressed_in->readStrict(compressed_buffer + header_size, size_compressed_without_checksum - header_size); } if (!disable_checksum) { - auto checksum_calculated = CityHash_v1_0_2::CityHash128(compressed_buffer, size_compressed); + auto checksum_calculated = CityHash_v1_0_2::CityHash128(compressed_buffer, size_compressed_without_checksum); if (checksum != checksum_calculated) throw Exception("Checksum doesn't match: corrupted data." - " Reference: " + getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second) - + ". Actual: " + getHexUIntLowercase(checksum_calculated.first) + getHexUIntLowercase(checksum_calculated.second) - + ". Size of compressed block: " + toString(size_compressed) + ".", - ErrorCodes::CHECKSUM_DOESNT_MATCH); + " Reference: " + getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second) + + ". Actual: " + getHexUIntLowercase(checksum_calculated.first) + getHexUIntLowercase(checksum_calculated.second) + + ". Size of compressed block: " + toString(size_compressed_without_checksum), + ErrorCodes::CHECKSUM_DOESNT_MATCH); } - return size_compressed + sizeof(checksum); + + return size_compressed_without_checksum + sizeof(checksum); } @@ -103,33 +104,20 @@ void CompressedReadBufferBase::decompress(char * to, size_t size_decompressed, s ProfileEvents::increment(ProfileEvents::CompressedReadBufferBlocks); ProfileEvents::increment(ProfileEvents::CompressedReadBufferBytes, size_decompressed); - UInt8 method = compressed_buffer[0]; /// See CompressedWriteBuffer.h + UInt8 method = ICompressionCodec::readMethod(compressed_buffer); - if (method == static_cast(CompressionMethodByte::LZ4)) - { - LZ4::decompress(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, to, size_compressed_without_checksum, size_decompressed, lz4_stat); - } - else if (method == static_cast(CompressionMethodByte::ZSTD)) - { - size_t res = ZSTD_decompress( - to, size_decompressed, - compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, size_compressed_without_checksum - COMPRESSED_BLOCK_HEADER_SIZE); + if (!codec) + codec = CompressionCodecFactory::instance().get(method); + else if (codec->getMethodByte() != method) + throw Exception("Data compressed with different method, given method byte " + getHexUIntLowercase(method) + ", previous method byte " + getHexUIntLowercase(codec->getMethodByte()), ErrorCodes::CANNOT_DECOMPRESS); - if (ZSTD_isError(res)) - throw Exception("Cannot ZSTD_decompress: " + std::string(ZSTD_getErrorName(res)), ErrorCodes::CANNOT_DECOMPRESS); - } - else if (method == static_cast(CompressionMethodByte::NONE)) - { - memcpy(to, &compressed_buffer[COMPRESSED_BLOCK_HEADER_SIZE], size_decompressed); - } - else - throw Exception("Unknown compression method: " + toString(method), ErrorCodes::UNKNOWN_COMPRESSION_METHOD); + codec->decompress(compressed_buffer, size_compressed_without_checksum, to); } /// 'compressed_in' could be initialized lazily, but before first call of 'readCompressedData'. CompressedReadBufferBase::CompressedReadBufferBase(ReadBuffer * in) - : compressed_in(in), own_compressed_buffer(COMPRESSED_BLOCK_HEADER_SIZE) + : compressed_in(in), own_compressed_buffer(0) { } diff --git a/dbms/src/IO/CompressedReadBufferBase.h b/dbms/src/IO/CompressedReadBufferBase.h index 5b6d0e0ca47..34ff798a8f1 100644 --- a/dbms/src/IO/CompressedReadBufferBase.h +++ b/dbms/src/IO/CompressedReadBufferBase.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -25,9 +26,6 @@ protected: /// Don't checksum on decompressing. bool disable_checksum = false; - LZ4::PerformanceStatistics lz4_stat; - - /// Read compressed data into compressed_buffer. Get size of decompressed data from block header. Checksum if need. /// Returns number of compressed bytes read. size_t readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum); @@ -47,6 +45,9 @@ public: { disable_checksum = true; } + +public: + CompressionCodecPtr codec; }; } diff --git a/dbms/src/IO/CompressedWriteBuffer.cpp b/dbms/src/IO/CompressedWriteBuffer.cpp index 77ba1515a5b..a7638497a5d 100644 --- a/dbms/src/IO/CompressedWriteBuffer.cpp +++ b/dbms/src/IO/CompressedWriteBuffer.cpp @@ -9,6 +9,7 @@ #include #include +#include namespace DB @@ -20,120 +21,37 @@ namespace ErrorCodes extern const int UNKNOWN_COMPRESSION_METHOD; } +static constexpr auto CHECKSUM_SIZE{sizeof(CityHash_v1_0_2::uint128)}; void CompressedWriteBuffer::nextImpl() { if (!offset()) return; - size_t uncompressed_size = offset(); - size_t compressed_size = 0; - char * compressed_buffer_ptr = nullptr; + size_t decompressed_size = offset(); + UInt32 compressed_reserve_size = codec->getCompressedReserveSize(decompressed_size); + compressed_buffer.resize(compressed_reserve_size); + UInt32 compressed_size = codec->compress(working_buffer.begin(), decompressed_size, compressed_buffer.data()); - /** The format of compressed block - see CompressedStream.h - */ - - switch (compression_settings.method) - { - case CompressionMethod::LZ4: - case CompressionMethod::LZ4HC: - { - static constexpr size_t header_size = 1 + sizeof(UInt32) + sizeof(UInt32); - -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wold-style-cast" - compressed_buffer.resize(header_size + LZ4_COMPRESSBOUND(uncompressed_size)); -#pragma GCC diagnostic pop - - compressed_buffer[0] = static_cast(CompressionMethodByte::LZ4); - - if (compression_settings.method == CompressionMethod::LZ4) - compressed_size = header_size + LZ4_compress_default( - working_buffer.begin(), - &compressed_buffer[header_size], - uncompressed_size, - LZ4_COMPRESSBOUND(uncompressed_size)); - else - compressed_size = header_size + LZ4_compress_HC( - working_buffer.begin(), - &compressed_buffer[header_size], - uncompressed_size, - LZ4_COMPRESSBOUND(uncompressed_size), - compression_settings.level); - - UInt32 compressed_size_32 = compressed_size; - UInt32 uncompressed_size_32 = uncompressed_size; - - unalignedStore(&compressed_buffer[1], compressed_size_32); - unalignedStore(&compressed_buffer[5], uncompressed_size_32); - - compressed_buffer_ptr = compressed_buffer.data(); - break; - } - case CompressionMethod::ZSTD: - { - static constexpr size_t header_size = 1 + sizeof(UInt32) + sizeof(UInt32); - - compressed_buffer.resize(header_size + ZSTD_compressBound(uncompressed_size)); - - compressed_buffer[0] = static_cast(CompressionMethodByte::ZSTD); - - size_t res = ZSTD_compress( - &compressed_buffer[header_size], - compressed_buffer.size() - header_size, - working_buffer.begin(), - uncompressed_size, - compression_settings.level); - - if (ZSTD_isError(res)) - throw Exception("Cannot compress block with ZSTD: " + std::string(ZSTD_getErrorName(res)), ErrorCodes::CANNOT_COMPRESS); - - compressed_size = header_size + res; - - UInt32 compressed_size_32 = compressed_size; - UInt32 uncompressed_size_32 = uncompressed_size; - - unalignedStore(&compressed_buffer[1], compressed_size_32); - unalignedStore(&compressed_buffer[5], uncompressed_size_32); - - compressed_buffer_ptr = compressed_buffer.data(); - break; - } - case CompressionMethod::NONE: - { - static constexpr size_t header_size = 1 + sizeof (UInt32) + sizeof (UInt32); - - compressed_size = header_size + uncompressed_size; - UInt32 uncompressed_size_32 = uncompressed_size; - UInt32 compressed_size_32 = compressed_size; - - compressed_buffer.resize(compressed_size); - - compressed_buffer[0] = static_cast(CompressionMethodByte::NONE); - - unalignedStore(&compressed_buffer[1], compressed_size_32); - unalignedStore(&compressed_buffer[5], uncompressed_size_32); - memcpy(&compressed_buffer[9], working_buffer.begin(), uncompressed_size); - - compressed_buffer_ptr = compressed_buffer.data(); - break; - } - default: - throw Exception("Unknown compression method", ErrorCodes::UNKNOWN_COMPRESSION_METHOD); - } - - CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(compressed_buffer_ptr, compressed_size); - out.write(reinterpret_cast(&checksum), sizeof(checksum)); - - out.write(compressed_buffer_ptr, compressed_size); + CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(compressed_buffer.data(), compressed_size); + out.write(reinterpret_cast(&checksum), CHECKSUM_SIZE); + out.write(compressed_buffer.data(), compressed_size); } CompressedWriteBuffer::CompressedWriteBuffer( WriteBuffer & out_, - CompressionSettings compression_settings_, + CompressionCodecPtr codec_, size_t buf_size) - : BufferWithOwnMemory(buf_size), out(out_), compression_settings(compression_settings_) + : BufferWithOwnMemory(buf_size), out(out_), codec(codec_) +{ +} + +CompressedWriteBuffer::CompressedWriteBuffer( + WriteBuffer & out_, + CompressionSettings compression_settings, + size_t buf_size) + : BufferWithOwnMemory(buf_size), out(out_), codec(CompressionCodecFactory::instance().get(compression_settings)) { } diff --git a/dbms/src/IO/CompressedWriteBuffer.h b/dbms/src/IO/CompressedWriteBuffer.h index 05996b18752..009351f8747 100644 --- a/dbms/src/IO/CompressedWriteBuffer.h +++ b/dbms/src/IO/CompressedWriteBuffer.h @@ -7,6 +7,8 @@ #include #include #include +#include +#include namespace DB @@ -16,7 +18,7 @@ class CompressedWriteBuffer : public BufferWithOwnMemory { private: WriteBuffer & out; - CompressionSettings compression_settings; + CompressionCodecPtr codec; PODArray compressed_buffer; @@ -24,10 +26,16 @@ private: public: CompressedWriteBuffer( + WriteBuffer & out_, + CompressionCodecPtr codec_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + + explicit CompressedWriteBuffer( WriteBuffer & out_, CompressionSettings compression_settings = CompressionSettings(), size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + /// The amount of compressed data size_t getCompressedBytes() { diff --git a/dbms/src/IO/tests/cached_compressed_read_buffer.cpp b/dbms/src/IO/tests/cached_compressed_read_buffer.cpp index 32c6fb16fb7..df30a3bc064 100644 --- a/dbms/src/IO/tests/cached_compressed_read_buffer.cpp +++ b/dbms/src/IO/tests/cached_compressed_read_buffer.cpp @@ -32,7 +32,7 @@ int main(int argc, char ** argv) { Stopwatch watch; - CachedCompressedReadBuffer in(path, &cache, CompressionCodecFactory::instance().getDefaultCodec(), 0, 0); + CachedCompressedReadBuffer in(path, &cache, 0, 0); WriteBufferFromFile out("/dev/null"); copyData(in, out); @@ -44,7 +44,7 @@ int main(int argc, char ** argv) { Stopwatch watch; - CachedCompressedReadBuffer in(path, &cache, CompressionCodecFactory::instance().getDefaultCodec(), 0, 0); + CachedCompressedReadBuffer in(path, &cache, 0, 0); WriteBufferFromFile out("/dev/null"); copyData(in, out); diff --git a/dbms/src/Storages/ColumnsDescription.cpp b/dbms/src/Storages/ColumnsDescription.cpp index 726d488f957..80f67db812b 100644 --- a/dbms/src/Storages/ColumnsDescription.cpp +++ b/dbms/src/Storages/ColumnsDescription.cpp @@ -196,14 +196,15 @@ void parseColumn(ReadBufferFromString & buf, ColumnsDescription & result, const assertChar('\n', buf); } -CompressionCodecPtr ColumnsDescription::getCodec(const String & column_name, const CompressionSettings & /*compression_settings*/) const +CompressionCodecPtr ColumnsDescription::getCodec(const String & column_name, const CompressionSettings & compression_settings) const { const auto codec = codecs.find(column_name); - /// TODO get if (codec == codecs.end()) + { return CompressionCodecFactory::instance().getDefaultCodec(); -// return CompressionCodecFactory::instance().get(compression_settings.method, compression_settings.level); + //return CompressionCodecFactory::instance().get(compression_settings.method, compression_settings.level); + } return codec->second; } diff --git a/dbms/src/Storages/ColumnsDescription.h b/dbms/src/Storages/ColumnsDescription.h index c06418cf89a..23b7c56e001 100644 --- a/dbms/src/Storages/ColumnsDescription.h +++ b/dbms/src/Storages/ColumnsDescription.h @@ -20,8 +20,8 @@ struct ColumnsDescription NamesAndTypesList materialized; NamesAndTypesList aliases; ColumnDefaults defaults; - ColumnCodecs codecs; ColumnComments comments; + ColumnCodecs codecs; ColumnsDescription() = default; @@ -30,12 +30,14 @@ struct ColumnsDescription NamesAndTypesList materialized_, NamesAndTypesList aliases_, ColumnDefaults defaults_, - ColumnComments comments_) + ColumnComments comments_, + ColumnCodecs codecs_) : ordinary(std::move(ordinary_)) , materialized(std::move(materialized_)) , aliases(std::move(aliases_)) , defaults(std::move(defaults_)) , comments(std::move(comments_)) + , codecs(std::move(codecs_)) {} explicit ColumnsDescription(NamesAndTypesList ordinary_) : ordinary(std::move(ordinary_)) {} @@ -46,7 +48,8 @@ struct ColumnsDescription && materialized == other.materialized && aliases == other.aliases && defaults == other.defaults - && comments == other.comments; + && comments == other.comments + && codecs == other.codecs; } bool operator!=(const ColumnsDescription & other) const { return !(*this == other); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index a36741ba143..4cecf31740b 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -20,8 +20,6 @@ #include #include #include -#include -#include #include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index 39e3bb3c725..b8a031655c3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -54,8 +53,7 @@ MergeTreeReader::MergeTreeReader(const String & path, for (const NameAndTypePair & column : columns) { - CompressionCodecPtr codec = columns_desc.getCodec(column.name, {}); - addStreams(column.name, *column.type, codec, all_mark_ranges, profile_callback, clock_type); + addStreams(column.name, *column.type, all_mark_ranges, profile_callback, clock_type); } } catch (...) @@ -164,7 +162,6 @@ size_t MergeTreeReader::readRows(size_t from_mark, bool continue_reading, size_t MergeTreeReader::Stream::Stream( const String & path_prefix_, const String & extension_, size_t marks_count_, - const CompressionCodecPtr & codec, const MarkRanges & all_mark_ranges, MarkCache * mark_cache_, bool save_marks_in_cache_, UncompressedCache * uncompressed_cache, @@ -236,8 +233,8 @@ MergeTreeReader::Stream::Stream( /// Initialize the objects that shall be used to perform read operations. if (uncompressed_cache) { - auto buffer = std::make_shared( - path_prefix + extension, uncompressed_cache, codec, estimated_size, aio_threshold, buffer_size); + auto buffer = std::make_unique( + path_prefix + extension, uncompressed_cache, estimated_size, aio_threshold, buffer_size); if (profile_callback) buffer->setProfileCallback(profile_callback, clock_type); @@ -247,13 +244,13 @@ MergeTreeReader::Stream::Stream( } else { - file_in = createReadBufferFromFileBase(path_prefix + extension, estimated_size, aio_threshold, buffer_size); + auto buffer = std::make_unique( + path_prefix + extension, estimated_size, aio_threshold, buffer_size); if (profile_callback) - file_in->setProfileCallback(profile_callback, clock_type); + buffer->setProfileCallback(profile_callback, clock_type); - const auto compressed_buffer = liftCompressed(codec, *file_in); - non_cached_buffer = compressed_buffer; + non_cached_buffer = std::move(buffer); data_buffer = non_cached_buffer.get(); } } @@ -361,7 +358,7 @@ void MergeTreeReader::Stream::seekToStart() } -void MergeTreeReader::addStreams(const String & name, const IDataType & type, const CompressionCodecPtr & codec, +void MergeTreeReader::addStreams(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type) { IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path) @@ -381,7 +378,7 @@ void MergeTreeReader::addStreams(const String & name, const IDataType & type, co streams.emplace(stream_name, std::make_unique( path + stream_name, DATA_FILE_EXTENSION, data_part->marks_count, - codec, all_mark_ranges, mark_cache, save_marks_in_cache, + all_mark_ranges, mark_cache, save_marks_in_cache, uncompressed_cache, aio_threshold, max_read_buffer_size, profile_callback, clock_type)); }; diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index 73fcb1977e9..85b627802fd 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -4,17 +4,16 @@ #include #include #include +#include #include #include +class CachedCompressedReadBuffer; namespace DB { class IDataType; -class CachedCompressedReadBuffer; -class CompressedReadBufferFromFile; - /// Reads the data between pairs of marks in the same part. When reading consecutive ranges, avoids unnecessary seeks. /// When ranges are almost consecutive, seeks are fast because they are performed inside the buffer. @@ -63,7 +62,6 @@ private: public: Stream( const String & path_prefix_, const String & extension_, size_t marks_count_, - const CompressionCodecPtr & codec, const MarkRanges & all_mark_ranges, MarkCache * mark_cache, bool save_marks_in_cache, UncompressedCache * uncompressed_cache, @@ -92,9 +90,9 @@ private: bool save_marks_in_cache; MarkCache::MappedPtr marks; - std::unique_ptr file_in; - std::shared_ptr cached_buffer; - std::shared_ptr non_cached_buffer; + + std::unique_ptr cached_buffer; + std::unique_ptr non_cached_buffer; }; using FileStreams = std::map>; @@ -123,7 +121,7 @@ private: size_t max_read_buffer_size; size_t index_granularity; - void addStreams(const String & name, const IDataType & type, const CompressionCodecPtr & codec, + void addStreams(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); void readData( diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 6d2d5c132b5..911fa019e2a 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -192,7 +192,7 @@ IMergedBlockOutputStream::ColumnStream::ColumnStream( data_file_extension{data_file_extension_}, marks_file_extension{marks_file_extension_}, plain_file(createWriteBufferFromFileBase(data_path + data_file_extension, estimated_size, aio_threshold, max_compress_block_size)), - plain_hashing(*plain_file), compressed_buf(liftCompressed(compression_codec, plain_hashing)), compressed(*compressed_buf.get()), + plain_hashing(*plain_file), compressed_buf(plain_hashing, compression_codec), compressed(compressed_buf), marks_file(marks_path + marks_file_extension, 4096, O_TRUNC | O_CREAT | O_WRONLY), marks(marks_file) { } diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h index 956aa15cc51..32eea13413e 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -49,7 +49,7 @@ protected: /// compressed -> compressed_buf -> plain_hashing -> plain_file std::unique_ptr plain_file; HashingWriteBuffer plain_hashing; - WriteBufferPtr compressed_buf; + CompressedWriteBuffer compressed_buf; HashingWriteBuffer compressed; /// marks -> marks_file diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 0229a8ff2bb..462f20ad582 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -146,7 +146,7 @@ private: { Stream(const std::string & data_path, size_t max_compress_block_size) : plain(data_path, max_compress_block_size, O_APPEND | O_CREAT | O_WRONLY), - compressed(plain, CompressionSettings(CompressionMethod::LZ4), max_compress_block_size) + compressed(plain, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size) { plain_offset = Poco::File(data_path).getSize(); } diff --git a/dbms/src/Storages/System/StorageSystemPartsBase.cpp b/dbms/src/Storages/System/StorageSystemPartsBase.cpp index 5d888b5bd10..8b7a016dd20 100644 --- a/dbms/src/Storages/System/StorageSystemPartsBase.cpp +++ b/dbms/src/Storages/System/StorageSystemPartsBase.cpp @@ -307,7 +307,7 @@ StorageSystemPartsBase::StorageSystemPartsBase(std::string name_, NamesAndTypesL add_alias("bytes", "bytes_on_disk"); add_alias("marks_size", "marks_bytes"); - setColumns(ColumnsDescription(std::move(columns_), {}, std::move(aliases), std::move(defaults), ColumnComments{})); + setColumns(ColumnsDescription(std::move(columns_), {}, std::move(aliases), std::move(defaults), ColumnComments{}, ColumnCodecs{})); } } From 7c11455bd3cd371df069cf0c6adc128e02bf2cf1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 21 Dec 2018 15:17:30 +0300 Subject: [PATCH 043/230] Remove compression settings --- dbms/programs/compressor/Compressor.cpp | 19 +-- dbms/programs/server/TCPHandler.cpp | 24 +++- dbms/src/Client/Connection.cpp | 25 +++- dbms/src/Client/Connection.h | 5 +- dbms/src/Compression/CompressionCodecZSTD.cpp | 4 +- dbms/src/Compression/CompressionCodecZSTD.h | 2 + dbms/src/Compression/CompressionFactory.cpp | 24 ++-- dbms/src/Compression/CompressionFactory.h | 6 +- dbms/src/IO/CompressedWriteBuffer.cpp | 9 -- dbms/src/IO/CompressedWriteBuffer.h | 9 +- dbms/src/IO/CompressionSettings.cpp | 37 ------ dbms/src/IO/CompressionSettings.h | 38 ------ dbms/src/Interpreters/Context.cpp | 15 +-- dbms/src/Interpreters/Context.h | 6 +- dbms/src/Storages/ColumnsDescription.cpp | 7 +- dbms/src/Storages/ColumnsDescription.h | 3 +- .../Storages/CompressionSettingsSelector.h | 109 ------------------ dbms/src/Storages/MergeTree/MergeTreeData.cpp | 4 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 12 +- .../MergeTree/MergeTreeDataPartChecksum.cpp | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 4 +- .../MergeTree/MergedBlockOutputStream.cpp | 22 ++-- .../MergeTree/MergedBlockOutputStream.h | 10 +- dbms/src/Storages/StorageStripeLog.cpp | 2 +- dbms/src/Storages/StorageTinyLog.cpp | 3 +- 25 files changed, 123 insertions(+), 278 deletions(-) delete mode 100644 dbms/src/IO/CompressionSettings.cpp delete mode 100644 dbms/src/IO/CompressionSettings.h delete mode 100644 dbms/src/Storages/CompressionSettingsSelector.h diff --git a/dbms/programs/compressor/Compressor.cpp b/dbms/programs/compressor/Compressor.cpp index 544238bf581..e021893dfdf 100644 --- a/dbms/programs/compressor/Compressor.cpp +++ b/dbms/programs/compressor/Compressor.cpp @@ -10,6 +10,7 @@ #include #include +#include namespace DB { @@ -85,18 +86,20 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) bool use_none = options.count("none"); unsigned block_size = options["block-size"].as(); - DB::CompressionMethod method = DB::CompressionMethod::LZ4; + std::string method_family = "LZ4"; if (use_lz4hc) - method = DB::CompressionMethod::LZ4HC; + method_family = "LZ4HC"; else if (use_zstd) - method = DB::CompressionMethod::ZSTD; + method_family = "ZSTD"; else if (use_none) - method = DB::CompressionMethod::NONE; + method_family = "NONE"; - DB::CompressionSettings settings(method, options.count("level") - ? options["level"].as() - : DB::CompressionSettings::getDefaultLevel(method)); + std::optional level; + if (options.count("level")) + level = options["level"].as(); + + DB::CompressionCodecPtr codec = DB::CompressionCodecFactory::instance().get(method_family, level); DB::ReadBufferFromFileDescriptor rb(STDIN_FILENO); DB::WriteBufferFromFileDescriptor wb(STDOUT_FILENO); @@ -115,7 +118,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) else { /// Compression - DB::CompressedWriteBuffer to(wb, settings, block_size); + DB::CompressedWriteBuffer to(wb, codec, block_size); DB::copyData(rb, to); } } diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index dc36a060a80..83630bd492a 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -32,6 +31,7 @@ #include #include #include +#include #include "TCPHandler.h" @@ -772,9 +772,29 @@ void TCPHandler::initBlockOutput(const Block & block) { if (!state.maybe_compressed_out) { + /// crutch TODO(aleap) + + std::string method = "LZ4"; + std::optional level; + switch(query_context.getSettingsRef().network_compression_method) + { + case CompressionMethod::ZSTD: + level = query_context.getSettingsRef().network_zstd_compression_level; + method = "ZSTD"; + break; + case CompressionMethod::LZ4HC: + method = "LZ4HC"; + break; + case CompressionMethod::NONE: + method = "NONE"; + break; + default: + break; + } + if (state.compression == Protocol::Compression::Enable) state.maybe_compressed_out = std::make_shared( - *out, CompressionSettings(query_context.getSettingsRef())); + *out, CompressionCodecFactory::instance().get(method, level)); else state.maybe_compressed_out = out; } diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 923c8179ca1..0587d06b31e 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #if USE_POCO_NETSSL @@ -353,7 +354,27 @@ void Connection::sendQuery( if (!connected) connect(); - compression_settings = settings ? CompressionSettings(*settings) : CompressionSettings(CompressionMethod::LZ4); + std::string method = "LZ4"; + std::optional level; + if (settings) { + switch(settings->network_compression_method) + { + case CompressionMethod::ZSTD: + level = settings->network_zstd_compression_level; + method = "ZSTD"; + break; + case CompressionMethod::LZ4HC: + method = "LZ4HC"; + break; + case CompressionMethod::NONE: + method = "NONE"; + break; + default: + break; + } + } + + compression_codec = CompressionCodecFactory::instance().get(method, level); query_id = query_id_; @@ -426,7 +447,7 @@ void Connection::sendData(const Block & block, const String & name) if (!block_out) { if (compression == Protocol::Compression::Enable) - maybe_compressed_out = std::make_shared(*out, compression_settings); + maybe_compressed_out = std::make_shared(*out, compression_codec); else maybe_compressed_out = out; diff --git a/dbms/src/Client/Connection.h b/dbms/src/Client/Connection.h index 2c72e7e0a43..5185cafe86b 100644 --- a/dbms/src/Client/Connection.h +++ b/dbms/src/Client/Connection.h @@ -18,12 +18,13 @@ #include #include -#include #include #include #include +#include + #include #include @@ -205,7 +206,7 @@ private: Protocol::Secure secure; /// Enable data encryption for communication. /// What compression settings to use while sending data for INSERT queries and external tables. - CompressionSettings compression_settings; + CompressionCodecPtr compression_codec; /** If not nullptr, used to limit network traffic. * Only traffic for transferring blocks is accounted. Other packets don't. diff --git a/dbms/src/Compression/CompressionCodecZSTD.cpp b/dbms/src/Compression/CompressionCodecZSTD.cpp index b42d2f54620..7a9d6c90b89 100644 --- a/dbms/src/Compression/CompressionCodecZSTD.cpp +++ b/dbms/src/Compression/CompressionCodecZSTD.cpp @@ -66,10 +66,10 @@ void registerCodecZSTD(CompressionCodecFactory & factory) UInt8 method_code = static_cast(CompressionMethodByte::ZSTD); factory.registerCompressionCodec("ZSTD", method_code, [&](const ASTPtr & arguments) -> CompressionCodecPtr { - int level = 0; + int level = CompressionCodecZSTD::ZSTD_DEFAULT_LEVEL; if (arguments && !arguments->children.empty()) { - if (arguments->children.size() != 1) + if (arguments->children.size() > 1) throw Exception("ZSTD codec must have 1 parameter, given " + std::to_string(arguments->children.size()), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); const auto children = arguments->children; diff --git a/dbms/src/Compression/CompressionCodecZSTD.h b/dbms/src/Compression/CompressionCodecZSTD.h index ea2c1a9b17a..f82d25d58d5 100644 --- a/dbms/src/Compression/CompressionCodecZSTD.h +++ b/dbms/src/Compression/CompressionCodecZSTD.h @@ -11,6 +11,8 @@ namespace DB class CompressionCodecZSTD : public ICompressionCodec { public: + static constexpr auto ZSTD_DEFAULT_LEVEL = 1; + CompressionCodecZSTD(int level_); UInt8 getMethodByte() const override; diff --git a/dbms/src/Compression/CompressionFactory.cpp b/dbms/src/Compression/CompressionFactory.cpp index 58b72012755..eca19eafea9 100644 --- a/dbms/src/Compression/CompressionFactory.cpp +++ b/dbms/src/Compression/CompressionFactory.cpp @@ -30,10 +30,19 @@ CompressionCodecPtr CompressionCodecFactory::getDefaultCodec() const } -CompressionCodecPtr CompressionCodecFactory::get(const CompressionSettings & settings) const +CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std::optional level) const { - ///TODO implement - return default_codec; + if (level) + { + auto identifier = std::make_shared(family_name); + auto literal = std::make_shared(*level); + return get(makeASTFunction("CODEC", identifier, literal)); + } + else + { + auto identifier = std::make_shared(family_name); + return get(makeASTFunction("CODEC", identifier)); + } } CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast) const @@ -73,7 +82,7 @@ CompressionCodecPtr CompressionCodecFactory::get(const UInt8 byte_code) const CompressionCodecPtr CompressionCodecFactory::getImpl(const String & family_name, const ASTPtr & arguments) const { - if (family_name == "MULTIPLE") + if (family_name == "Multiple") throw Exception("Codec MULTIPLE cannot be specified directly", ErrorCodes::UNKNOWN_CODEC); const auto family_and_creator = family_name_with_codec.find(family_name); @@ -109,13 +118,6 @@ void CompressionCodecFactory::registerSimpleCompressionCodec(const String & fami } -ASTPtr CompressionCodecFactory::convertSettingsToAst(const CompressionSettings & settings) const -{ - ///TODO Implement - return nullptr; - //makeASTFunction(); -} - void registerCodecLZ4(CompressionCodecFactory & factory); void registerCodecNone(CompressionCodecFactory & factory); void registerCodecZSTD(CompressionCodecFactory & factory); diff --git a/dbms/src/Compression/CompressionFactory.h b/dbms/src/Compression/CompressionFactory.h index b474aa887ee..7b767560672 100644 --- a/dbms/src/Compression/CompressionFactory.h +++ b/dbms/src/Compression/CompressionFactory.h @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB { @@ -37,7 +36,8 @@ public: CompressionCodecPtr get(const UInt8 byte_code) const; - CompressionCodecPtr get(const CompressionSettings & settings) const; + /// For backward compatibility with config settings + CompressionCodecPtr get(const String & family_name, std::optional level) const; void registerCompressionCodec(const String & family_name, UInt8 byte_code, Creator creator); @@ -51,8 +51,6 @@ private: CompressionCodecsCodeDictionary family_code_with_codec; CompressionCodecPtr default_codec; - ASTPtr convertSettingsToAst(const CompressionSettings & settings) const; - CompressionCodecFactory(); friend class ext::singleton; diff --git a/dbms/src/IO/CompressedWriteBuffer.cpp b/dbms/src/IO/CompressedWriteBuffer.cpp index a7638497a5d..24a2021555f 100644 --- a/dbms/src/IO/CompressedWriteBuffer.cpp +++ b/dbms/src/IO/CompressedWriteBuffer.cpp @@ -47,15 +47,6 @@ CompressedWriteBuffer::CompressedWriteBuffer( { } -CompressedWriteBuffer::CompressedWriteBuffer( - WriteBuffer & out_, - CompressionSettings compression_settings, - size_t buf_size) - : BufferWithOwnMemory(buf_size), out(out_), codec(CompressionCodecFactory::instance().get(compression_settings)) -{ -} - - CompressedWriteBuffer::~CompressedWriteBuffer() { try diff --git a/dbms/src/IO/CompressedWriteBuffer.h b/dbms/src/IO/CompressedWriteBuffer.h index 009351f8747..a9612b463a5 100644 --- a/dbms/src/IO/CompressedWriteBuffer.h +++ b/dbms/src/IO/CompressedWriteBuffer.h @@ -6,7 +6,6 @@ #include #include -#include #include #include @@ -27,15 +26,9 @@ private: public: CompressedWriteBuffer( WriteBuffer & out_, - CompressionCodecPtr codec_, + CompressionCodecPtr codec_ = CompressionCodecFactory::instance().getDefaultCodec(), size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); - explicit CompressedWriteBuffer( - WriteBuffer & out_, - CompressionSettings compression_settings = CompressionSettings(), - size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); - - /// The amount of compressed data size_t getCompressedBytes() { diff --git a/dbms/src/IO/CompressionSettings.cpp b/dbms/src/IO/CompressionSettings.cpp deleted file mode 100644 index 6fd7d96c873..00000000000 --- a/dbms/src/IO/CompressionSettings.cpp +++ /dev/null @@ -1,37 +0,0 @@ -#include - -#include "CompressionSettings.h" - - -namespace DB -{ - -CompressionSettings::CompressionSettings(const Settings & settings) -{ - method = settings.network_compression_method; - switch (method) - { - case CompressionMethod::ZSTD: - level = settings.network_zstd_compression_level; - break; - default: - level = getDefaultLevel(method); - } -} - -int CompressionSettings::getDefaultLevel(CompressionMethod method) -{ - switch (method) - { - case CompressionMethod::LZ4: - return -1; - case CompressionMethod::LZ4HC: - return 0; - case CompressionMethod::ZSTD: - return 1; - default: - return -1; - } -} - -} diff --git a/dbms/src/IO/CompressionSettings.h b/dbms/src/IO/CompressionSettings.h deleted file mode 100644 index 2b3cd731183..00000000000 --- a/dbms/src/IO/CompressionSettings.h +++ /dev/null @@ -1,38 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -struct Settings; - -struct CompressionSettings -{ - CompressionMethod method; - int level; - - CompressionSettings() - : CompressionSettings(CompressionMethod::LZ4) - { - } - - CompressionSettings(CompressionMethod method_) - : method(method_) - , level(getDefaultLevel(method)) - { - } - - CompressionSettings(CompressionMethod method_, int level_) - : method(method_) - , level(level_) - { - } - - CompressionSettings(const Settings & settings); - - static int getDefaultLevel(CompressionMethod method); -}; - -} diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 1e33c90be2c..c890da8e8fe 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -20,7 +21,7 @@ #include #include #include -#include +#include #include #include #include @@ -143,7 +144,7 @@ struct ContextShared std::unique_ptr compiler; /// Used for dynamic compilation of queries' parts if it necessary. std::shared_ptr ddl_worker; /// Process ddl commands from zk. /// Rules for selecting the compression settings, depending on the size of the part. - mutable std::unique_ptr compression_settings_selector; + mutable std::unique_ptr compression_codec_selector; std::unique_ptr merge_tree_settings; /// Settings of MergeTree* engines. size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default) size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default) @@ -1585,22 +1586,22 @@ PartLog * Context::getPartLog(const String & part_database) } -CompressionSettings Context::chooseCompressionSettings(size_t part_size, double part_size_ratio) const +CompressionCodecPtr Context::chooseCompressionCodec(size_t part_size, double part_size_ratio) const { auto lock = getLock(); - if (!shared->compression_settings_selector) + if (!shared->compression_codec_selector) { constexpr auto config_name = "compression"; auto & config = getConfigRef(); if (config.has(config_name)) - shared->compression_settings_selector = std::make_unique(config, "compression"); + shared->compression_codec_selector = std::make_unique(config, "compression"); else - shared->compression_settings_selector = std::make_unique(); + shared->compression_codec_selector = std::make_unique(); } - return shared->compression_settings_selector->choose(part_size, part_size_ratio); + return shared->compression_codec_selector->choose(part_size, part_size_ratio); } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 6e38e056a0f..2609c4ff3e6 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -17,7 +17,6 @@ #include #include #include -#include namespace Poco @@ -80,6 +79,7 @@ using SystemLogsPtr = std::shared_ptr; class ActionLocksManager; using ActionLocksManagerPtr = std::shared_ptr; class ShellCommand; +class ICompressionCodec; #if USE_EMBEDDED_COMPILER @@ -406,8 +406,8 @@ public: void setMaxPartitionSizeToDrop(size_t max_size); void checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size); - /// Lets you select the compression settings according to the conditions described in the configuration file. - CompressionSettings chooseCompressionSettings(size_t part_size, double part_size_ratio) const; + /// Lets you select the compression codec according to the conditions described in the configuration file. + std::shared_ptr chooseCompressionCodec(size_t part_size, double part_size_ratio) const; /// Get the server uptime in seconds. time_t getUptimeSeconds() const; diff --git a/dbms/src/Storages/ColumnsDescription.cpp b/dbms/src/Storages/ColumnsDescription.cpp index 80f67db812b..b230cb0e294 100644 --- a/dbms/src/Storages/ColumnsDescription.cpp +++ b/dbms/src/Storages/ColumnsDescription.cpp @@ -196,15 +196,12 @@ void parseColumn(ReadBufferFromString & buf, ColumnsDescription & result, const assertChar('\n', buf); } -CompressionCodecPtr ColumnsDescription::getCodec(const String & column_name, const CompressionSettings & compression_settings) const +CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_name, CompressionCodecPtr default_codec) const { const auto codec = codecs.find(column_name); if (codec == codecs.end()) - { - return CompressionCodecFactory::instance().getDefaultCodec(); - //return CompressionCodecFactory::instance().get(compression_settings.method, compression_settings.level); - } + return default_codec; return codec->second; } diff --git a/dbms/src/Storages/ColumnsDescription.h b/dbms/src/Storages/ColumnsDescription.h index 23b7c56e001..edeabf63dfb 100644 --- a/dbms/src/Storages/ColumnsDescription.h +++ b/dbms/src/Storages/ColumnsDescription.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB @@ -68,7 +67,7 @@ struct ColumnsDescription String toString() const; - CompressionCodecPtr getCodec(const String & column_name, const CompressionSettings & compression_settings) const; + CompressionCodecPtr getCodecOrDefault(const String & column_name, CompressionCodecPtr default_codec) const; static ColumnsDescription parse(const String & str); diff --git a/dbms/src/Storages/CompressionSettingsSelector.h b/dbms/src/Storages/CompressionSettingsSelector.h deleted file mode 100644 index 2aa6f919fca..00000000000 --- a/dbms/src/Storages/CompressionSettingsSelector.h +++ /dev/null @@ -1,109 +0,0 @@ -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int UNKNOWN_COMPRESSION_METHOD; - extern const int UNKNOWN_ELEMENT_IN_CONFIG; -} - - -/** Allows you to select the compression settings for the conditions specified in the configuration file. - * The config looks like this - - - - - - - - 10000000000 - 0.01 - - - zstd - 2 - - - - ... - - - */ -class CompressionSettingsSelector -{ -private: - struct Element - { - size_t min_part_size = 0; - double min_part_size_ratio = 0; - CompressionSettings settings = CompressionSettings(CompressionMethod::LZ4); - - static CompressionMethod compressionMethodFromString(const std::string & name) - { - if (name == "lz4") - return CompressionMethod::LZ4; - else if (name == "zstd") - return CompressionMethod::ZSTD; - else if (name == "none") - return CompressionMethod::NONE; - else - throw Exception("Unknown compression method " + name, ErrorCodes::UNKNOWN_COMPRESSION_METHOD); - } - - Element(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) - { - min_part_size = config.getUInt64(config_prefix + ".min_part_size", 0); - min_part_size_ratio = config.getDouble(config_prefix + ".min_part_size_ratio", 0); - - CompressionMethod method = compressionMethodFromString(config.getString(config_prefix + ".method")); - int level = config.getInt64(config_prefix + ".level", CompressionSettings::getDefaultLevel(method)); - settings = CompressionSettings(method, level); - } - - bool check(size_t part_size, double part_size_ratio) const - { - return part_size >= min_part_size - && part_size_ratio >= min_part_size_ratio; - } - }; - - std::vector elements; - -public: - CompressionSettingsSelector() {} /// Always returns the default method. - - CompressionSettingsSelector(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) - { - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(config_prefix, keys); - - for (const auto & name : keys) - { - if (!startsWith(name.data(), "case")) - throw Exception("Unknown element in config: " + config_prefix + "." + name + ", must be 'case'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - - elements.emplace_back(config, config_prefix + "." + name); - } - } - - CompressionSettings choose(size_t part_size, double part_size_ratio) const - { - CompressionSettings res = CompressionSettings(CompressionMethod::LZ4); - - for (const auto & element : elements) - if (element.check(part_size, part_size_ratio)) - res = element.settings; - - return res; - } -}; - -} diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index b65d23f47e1..f56fef5a6ee 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1301,7 +1301,7 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( BlockInputStreamPtr part_in = std::make_shared( *this, part, expression->getRequiredColumns(), false, /* take_column_types_from_storage = */ false); - auto compression_settings = this->context.chooseCompressionSettings( + auto compression_codec = this->context.chooseCompressionCodec( part->bytes_on_disk, static_cast(part->bytes_on_disk) / this->getTotalActiveSizeInBytes()); ExpressionBlockInputStream in(part_in, expression); @@ -1315,7 +1315,7 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( */ IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets; MergedColumnOnlyOutputStream out( - *this, in.getHeader(), full_path + part->name + '/', true /* sync */, compression_settings, true /* skip_offsets */, unused_written_offsets); + *this, in.getHeader(), full_path + part->name + '/', true /* sync */, compression_codec, true /* skip_offsets */, unused_written_offsets); in.readPrefix(); out.writePrefix(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 4cecf31740b..43748241c47 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -567,7 +567,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor /// (which is locked in shared mode when input streams are created) and when inserting new data /// the order is reverse. This annoys TSan even though one lock is locked in shared mode and thus /// deadlock is impossible. - auto compression_settings = data.context.chooseCompressionSettings( + auto compression_codec = data.context.chooseCompressionCodec( merge_entry->total_size_bytes_compressed, static_cast (merge_entry->total_size_bytes_compressed) / data.getTotalActiveSizeInBytes()); @@ -693,7 +693,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor merged_stream = std::make_shared(merged_stream, SizeLimits(), 0 /*limit_hint*/, Names()); MergedBlockOutputStream to{ - data, new_part_tmp_path, merging_columns, compression_settings, merged_column_to_size, data.settings.min_merge_bytes_to_use_direct_io}; + data, new_part_tmp_path, merging_columns, compression_codec, merged_column_to_size, data.settings.min_merge_bytes_to_use_direct_io}; merged_stream->readPrefix(); to.writePrefix(); @@ -778,7 +778,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor rows_sources_read_buf.seek(0, 0); ColumnGathererStream column_gathered_stream(column_name, column_part_streams, rows_sources_read_buf); MergedColumnOnlyOutputStream column_to( - data, column_gathered_stream.getHeader(), new_part_tmp_path, false, compression_settings, false, written_offset_columns); + data, column_gathered_stream.getHeader(), new_part_tmp_path, false, compression_codec, false, written_offset_columns); size_t column_elems_written = 0; column_to.writePrefix(); @@ -883,7 +883,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor /// (which is locked in shared mode when input streams are created) and when inserting new data /// the order is reverse. This annoys TSan even though one lock is locked in shared mode and thus /// deadlock is impossible. - auto compression_settings = context.chooseCompressionSettings( + auto compression_codec = context.chooseCompressionCodec( source_part->bytes_on_disk, static_cast(source_part->bytes_on_disk) / data.getTotalActiveSizeInBytes()); @@ -904,7 +904,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor MergeTreeDataPart::MinMaxIndex minmax_idx; - MergedBlockOutputStream out(data, new_part_tmp_path, all_columns, compression_settings); + MergedBlockOutputStream out(data, new_part_tmp_path, all_columns, compression_codec); in->readPrefix(); out.writePrefix(); @@ -955,7 +955,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets; MergedColumnOnlyOutputStream out( - data, in_header, new_part_tmp_path, /* sync = */ false, compression_settings, /* skip_offsets = */ false, unused_written_offsets); + data, in_header, new_part_tmp_path, /* sync = */ false, compression_codec, /* skip_offsets = */ false, unused_written_offsets); in->readPrefix(); out.writePrefix(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 98627bd3c34..2b34413b6b9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -201,7 +201,7 @@ void MergeTreeDataPartChecksums::write(WriteBuffer & to) const { writeString("checksums format version: 4\n", to); - CompressedWriteBuffer out{to, CompressionSettings(CompressionMethod::LZ4), 1 << 16}; + CompressedWriteBuffer out{to, CompressionCodecFactory::instance().getDefaultCodec(), 1 << 16}; writeVarUInt(files.size(), out); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index d6c287dd01f..3399e162f0d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -209,10 +209,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa /// This effectively chooses minimal compression method: /// either default lz4 or compression method with zero thresholds on absolute and relative part size. - auto compression_settings = data.context.chooseCompressionSettings(0, 0); + auto compression_codec = data.context.chooseCompressionCodec(0, 0); NamesAndTypesList columns = data.getColumns().getAllPhysical().filter(block.getNames()); - MergedBlockOutputStream out(data, new_data_part->getFullPath(), columns, compression_settings); + MergedBlockOutputStream out(data, new_data_part->getFullPath(), columns, compression_codec); out.writePrefix(); out.writeWithPermutation(block, perm_ptr); diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 911fa019e2a..9d98e9f4555 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -26,13 +26,13 @@ IMergedBlockOutputStream::IMergedBlockOutputStream( MergeTreeData & storage_, size_t min_compress_block_size_, size_t max_compress_block_size_, - CompressionSettings compression_settings_, + CompressionCodecPtr codec_, size_t aio_threshold_) : storage(storage_), min_compress_block_size(min_compress_block_size_), max_compress_block_size(max_compress_block_size_), aio_threshold(aio_threshold_), - compression_settings(compression_settings_) + codec(codec_) { } @@ -231,10 +231,10 @@ MergedBlockOutputStream::MergedBlockOutputStream( MergeTreeData & storage_, String part_path_, const NamesAndTypesList & columns_list_, - CompressionSettings compression_settings) + CompressionCodecPtr default_codec_) : IMergedBlockOutputStream( storage_, storage_.context.getSettings().min_compress_block_size, - storage_.context.getSettings().max_compress_block_size, compression_settings, + storage_.context.getSettings().max_compress_block_size, default_codec_, storage_.context.getSettings().min_bytes_to_use_direct_io), columns_list(columns_list_), part_path(part_path_) { @@ -242,7 +242,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( for (const auto & it : columns_list) { const auto columns = storage.getColumns(); - addStreams(part_path, it.name, *it.type, columns.getCodec(it.name, compression_settings), 0, false); + addStreams(part_path, it.name, *it.type, columns.getCodecOrDefault(it.name, default_codec_), 0, false); } } @@ -250,12 +250,12 @@ MergedBlockOutputStream::MergedBlockOutputStream( MergeTreeData & storage_, String part_path_, const NamesAndTypesList & columns_list_, - CompressionSettings compression_settings, + CompressionCodecPtr default_codec_, const MergeTreeData::DataPart::ColumnToSize & merged_column_to_size_, size_t aio_threshold_) : IMergedBlockOutputStream( storage_, storage_.context.getSettings().min_compress_block_size, - storage_.context.getSettings().max_compress_block_size, compression_settings, + storage_.context.getSettings().max_compress_block_size, default_codec_, aio_threshold_), columns_list(columns_list_), part_path(part_path_) { @@ -276,7 +276,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( for (const auto & it : columns_list) { const auto columns = storage.getColumns(); - addStreams(part_path, it.name, *it.type, columns.getCodec(it.name, compression_settings), total_size, false); + addStreams(part_path, it.name, *it.type, columns.getCodecOrDefault(it.name, default_codec_), total_size, false); } } @@ -514,11 +514,11 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, - CompressionSettings compression_settings, bool skip_offsets_, + CompressionCodecPtr default_codec_, bool skip_offsets_, WrittenOffsetColumns & already_written_offset_columns) : IMergedBlockOutputStream( storage_, storage_.context.getSettings().min_compress_block_size, - storage_.context.getSettings().max_compress_block_size, compression_settings, + storage_.context.getSettings().max_compress_block_size, default_codec_, storage_.context.getSettings().min_bytes_to_use_direct_io), header(header_), part_path(part_path_), sync(sync_), skip_offsets(skip_offsets_), already_written_offset_columns(already_written_offset_columns) @@ -540,7 +540,7 @@ void MergedColumnOnlyOutputStream::write(const Block & block) const auto & col = block.safeGetByPosition(i); const auto columns = storage.getColumns(); - addStreams(part_path, col.name, *col.type, columns.getCodec(col.name, compression_settings), 0, skip_offsets); + addStreams(part_path, col.name, *col.type, columns.getCodecOrDefault(col.name, codec), 0, skip_offsets); serialization_states.emplace_back(nullptr); settings.getter = createStreamGetter(col.name, tmp_offset_columns, false); col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back()); diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h index 32eea13413e..b09abfc88a9 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -20,7 +20,7 @@ public: MergeTreeData & storage_, size_t min_compress_block_size_, size_t max_compress_block_size_, - CompressionSettings compression_settings_, + CompressionCodecPtr default_codec_, size_t aio_threshold_); using WrittenOffsetColumns = std::set; @@ -87,7 +87,7 @@ protected: size_t aio_threshold; - CompressionSettings compression_settings; + CompressionCodecPtr codec; }; @@ -101,13 +101,13 @@ public: MergeTreeData & storage_, String part_path_, const NamesAndTypesList & columns_list_, - CompressionSettings compression_settings); + CompressionCodecPtr default_codec_); MergedBlockOutputStream( MergeTreeData & storage_, String part_path_, const NamesAndTypesList & columns_list_, - CompressionSettings compression_settings, + CompressionCodecPtr default_codec_, const MergeTreeData::DataPart::ColumnToSize & merged_column_to_size_, size_t aio_threshold_); @@ -161,7 +161,7 @@ public: /// if you want to serialize elements of Nested data structure in different instances of MergedColumnOnlyOutputStream. MergedColumnOnlyOutputStream( MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, - CompressionSettings compression_settings, bool skip_offsets_, + CompressionCodecPtr default_codec_, bool skip_offsets_, WrittenOffsetColumns & already_written_offset_columns); Block getHeader() const override { return header; } diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index b86731f6f3f..9cf512880fe 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -136,7 +136,7 @@ public: explicit StripeLogBlockOutputStream(StorageStripeLog & storage_) : storage(storage_), lock(storage.rwlock), data_out_compressed(storage.full_path() + "data.bin", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT), - data_out(data_out_compressed, CompressionSettings(CompressionMethod::LZ4), storage.max_compress_block_size), + data_out(data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), storage.max_compress_block_size), index_out_compressed(storage.full_path() + "index.mrk", INDEX_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT), index_out(index_out_compressed), block_out(data_out, 0, storage.getSampleBlock(), false, &index_out, Poco::File(storage.full_path() + "data.bin").getSize()) diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index affdbc4eb04..5f66f0f5049 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -26,6 +26,7 @@ #include #include +#include #include @@ -136,7 +137,7 @@ private: { Stream(const std::string & data_path, size_t max_compress_block_size) : plain(data_path, max_compress_block_size, O_APPEND | O_CREAT | O_WRONLY), - compressed(plain, CompressionSettings(CompressionMethod::LZ4), max_compress_block_size) + compressed(plain, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size) { } From 6f7333840ae01a407adbb09460d12f8e9b67f842 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 21 Dec 2018 16:25:39 +0300 Subject: [PATCH 044/230] Move header with compression info to another place --- dbms/programs/server/TCPHandler.cpp | 19 +--- dbms/src/Client/Connection.cpp | 30 ++---- dbms/src/Compression/CompressionCodecLZ4.cpp | 2 +- .../Compression/CompressionCodecMultiple.cpp | 2 +- dbms/src/Compression/CompressionCodecNone.cpp | 2 +- dbms/src/Compression/CompressionCodecZSTD.cpp | 2 +- dbms/src/Compression/CompressionFactory.h | 2 +- .../CompressionInfo.h} | 10 -- dbms/src/Compression/ICompressionCodec.cpp | 1 - dbms/src/Compression/ICompressionCodec.h | 4 +- dbms/src/IO/CachedCompressedReadBuffer.cpp | 2 +- dbms/src/IO/CompressedReadBuffer.cpp | 2 +- dbms/src/IO/CompressedReadBufferBase.cpp | 2 +- dbms/src/IO/CompressedReadBufferFromFile.cpp | 2 +- dbms/src/IO/CompressionCodecWriteBuffer.h | 0 dbms/src/Interpreters/Settings.h | 2 +- dbms/src/Interpreters/SettingsCommon.cpp | 52 --------- dbms/src/Interpreters/SettingsCommon.h | 26 +---- dbms/src/Storages/CompressionCodecSelector.h | 101 ++++++++++++++++++ .../Storages/MergeTree/MergeTreeDataPart.cpp | 2 +- 20 files changed, 129 insertions(+), 136 deletions(-) rename dbms/src/{IO/CompressedStream.h => Compression/CompressionInfo.h} (80%) create mode 100644 dbms/src/IO/CompressionCodecWriteBuffer.h create mode 100644 dbms/src/Storages/CompressionCodecSelector.h diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 83630bd492a..3bfdedaeb58 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -772,25 +772,10 @@ void TCPHandler::initBlockOutput(const Block & block) { if (!state.maybe_compressed_out) { - /// crutch TODO(aleap) - - std::string method = "LZ4"; + std::string method = query_context.getSettingsRef().network_compression_method; std::optional level; - switch(query_context.getSettingsRef().network_compression_method) - { - case CompressionMethod::ZSTD: + if (method == "ZSTD") level = query_context.getSettingsRef().network_zstd_compression_level; - method = "ZSTD"; - break; - case CompressionMethod::LZ4HC: - method = "LZ4HC"; - break; - case CompressionMethod::NONE: - method = "NONE"; - break; - default: - break; - } if (state.compression == Protocol::Compression::Enable) state.maybe_compressed_out = std::make_shared( diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 0587d06b31e..f72eadcaec4 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -354,27 +354,19 @@ void Connection::sendQuery( if (!connected) connect(); - std::string method = "LZ4"; - std::optional level; - if (settings) { - switch(settings->network_compression_method) + if (settings) { - case CompressionMethod::ZSTD: - level = settings->network_zstd_compression_level; - method = "ZSTD"; - break; - case CompressionMethod::LZ4HC: - method = "LZ4HC"; - break; - case CompressionMethod::NONE: - method = "NONE"; - break; - default: - break; - } - } + std::optional level; + std::string method = settings->network_compression_method; - compression_codec = CompressionCodecFactory::instance().get(method, level); + /// Bad custom logic + if (method == "ZSTD") + level = settings->network_zstd_compression_level; + + compression_codec = CompressionCodecFactory::instance().get(method, level); + } + else + compression_codec = CompressionCodecFactory::instance().getDefaultCodec(); query_id = query_id_; diff --git a/dbms/src/Compression/CompressionCodecLZ4.cpp b/dbms/src/Compression/CompressionCodecLZ4.cpp index 8c44ebfac43..a9e54b36047 100644 --- a/dbms/src/Compression/CompressionCodecLZ4.cpp +++ b/dbms/src/Compression/CompressionCodecLZ4.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include "CompressionCodecLZ4.h" diff --git a/dbms/src/Compression/CompressionCodecMultiple.cpp b/dbms/src/Compression/CompressionCodecMultiple.cpp index 77514d6421e..93a38d43a84 100644 --- a/dbms/src/Compression/CompressionCodecMultiple.cpp +++ b/dbms/src/Compression/CompressionCodecMultiple.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/dbms/src/Compression/CompressionCodecNone.cpp b/dbms/src/Compression/CompressionCodecNone.cpp index e090c09f9c1..fbb0e9cd031 100644 --- a/dbms/src/Compression/CompressionCodecNone.cpp +++ b/dbms/src/Compression/CompressionCodecNone.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include diff --git a/dbms/src/Compression/CompressionCodecZSTD.cpp b/dbms/src/Compression/CompressionCodecZSTD.cpp index 7a9d6c90b89..d3f96cc7e06 100644 --- a/dbms/src/Compression/CompressionCodecZSTD.cpp +++ b/dbms/src/Compression/CompressionCodecZSTD.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/dbms/src/Compression/CompressionFactory.h b/dbms/src/Compression/CompressionFactory.h index 7b767560672..43621ba4d78 100644 --- a/dbms/src/Compression/CompressionFactory.h +++ b/dbms/src/Compression/CompressionFactory.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/dbms/src/IO/CompressedStream.h b/dbms/src/Compression/CompressionInfo.h similarity index 80% rename from dbms/src/IO/CompressedStream.h rename to dbms/src/Compression/CompressionInfo.h index be8fb254fab..fc7d501a80c 100644 --- a/dbms/src/IO/CompressedStream.h +++ b/dbms/src/Compression/CompressionInfo.h @@ -8,19 +8,9 @@ #define COMPRESSED_BLOCK_HEADER_SIZE 9 - namespace DB { -/** Compression method */ -enum class CompressionMethod -{ - LZ4 = 1, - LZ4HC = 2, /// The format is the same as for LZ4. The difference is only in compression. - ZSTD = 3, /// Experimental algorithm: https://github.com/Cyan4973/zstd - NONE = 4, /// No compression -}; - /** The compressed block format is as follows: * * The first 16 bytes are the checksum from all other bytes of the block. Now only CityHash128 is used. diff --git a/dbms/src/Compression/ICompressionCodec.cpp b/dbms/src/Compression/ICompressionCodec.cpp index 99aaf4d9ef2..45ed8250cd7 100644 --- a/dbms/src/Compression/ICompressionCodec.cpp +++ b/dbms/src/Compression/ICompressionCodec.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/src/Compression/ICompressionCodec.h b/dbms/src/Compression/ICompressionCodec.h index 8a4c05b7210..ef104d07eca 100644 --- a/dbms/src/Compression/ICompressionCodec.h +++ b/dbms/src/Compression/ICompressionCodec.h @@ -11,6 +11,8 @@ #include #include +#include + namespace DB { @@ -39,7 +41,7 @@ public: virtual UInt32 getAdditionalSizeAtTheEndOfBuffer() const { return 0; } - static UInt8 getHeaderSize() { return 1 + 8; } + static UInt8 getHeaderSize() { return COMPRESSED_BLOCK_HEADER_SIZE; } static UInt32 readCompressedBlockSize(const char * source); diff --git a/dbms/src/IO/CachedCompressedReadBuffer.cpp b/dbms/src/IO/CachedCompressedReadBuffer.cpp index 7d8d809a254..a2a5614ef43 100644 --- a/dbms/src/IO/CachedCompressedReadBuffer.cpp +++ b/dbms/src/IO/CachedCompressedReadBuffer.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include "CachedCompressedReadBuffer.h" diff --git a/dbms/src/IO/CompressedReadBuffer.cpp b/dbms/src/IO/CompressedReadBuffer.cpp index 0df4664ba5e..cc540161c92 100644 --- a/dbms/src/IO/CompressedReadBuffer.cpp +++ b/dbms/src/IO/CompressedReadBuffer.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include diff --git a/dbms/src/IO/CompressedReadBufferBase.cpp b/dbms/src/IO/CompressedReadBufferBase.cpp index 65b7af852ab..1932daa6e30 100644 --- a/dbms/src/IO/CompressedReadBufferBase.cpp +++ b/dbms/src/IO/CompressedReadBufferBase.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include diff --git a/dbms/src/IO/CompressedReadBufferFromFile.cpp b/dbms/src/IO/CompressedReadBufferFromFile.cpp index 1039fef66cd..25008c205b5 100644 --- a/dbms/src/IO/CompressedReadBufferFromFile.cpp +++ b/dbms/src/IO/CompressedReadBufferFromFile.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include diff --git a/dbms/src/IO/CompressionCodecWriteBuffer.h b/dbms/src/IO/CompressionCodecWriteBuffer.h new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 5b55ebba908..3f4b99078c4 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -111,7 +111,7 @@ struct Settings \ M(SettingFloat, max_streams_to_max_threads_ratio, 1, "Allows you to use more sources than the number of threads - to more evenly distribute work across threads. It is assumed that this is a temporary solution, since it will be possible in the future to make the number of sources equal to the number of threads, but for each source to dynamically select available work for itself.") \ \ - M(SettingCompressionMethod, network_compression_method, CompressionMethod::LZ4, "Allows you to select the method of data compression when writing.") \ + M(SettingString, network_compression_method, "LZ4", "Allows you to select the method of data compression when writing.") \ \ M(SettingInt64, network_zstd_compression_level, 1, "Allows you to select the level of ZSTD compression.") \ \ diff --git a/dbms/src/Interpreters/SettingsCommon.cpp b/dbms/src/Interpreters/SettingsCommon.cpp index b65097a0f42..b79d8650da0 100644 --- a/dbms/src/Interpreters/SettingsCommon.cpp +++ b/dbms/src/Interpreters/SettingsCommon.cpp @@ -458,58 +458,6 @@ void SettingOverflowMode::write(WriteBuffer & buf) const template struct SettingOverflowMode; template struct SettingOverflowMode; - -CompressionMethod SettingCompressionMethod::getCompressionMethod(const String & s) -{ - if (s == "lz4") - return CompressionMethod::LZ4; - if (s == "lz4hc") - return CompressionMethod::LZ4HC; - if (s == "zstd") - return CompressionMethod::ZSTD; - - throw Exception("Unknown compression method: '" + s + "', must be one of 'lz4', 'lz4hc', 'zstd'", ErrorCodes::UNKNOWN_COMPRESSION_METHOD); -} - -String SettingCompressionMethod::toString() const -{ - const char * strings[] = { nullptr, "lz4", "lz4hc", "zstd" }; - - if (value < CompressionMethod::LZ4 || value > CompressionMethod::ZSTD) - throw Exception("Unknown compression method", ErrorCodes::UNKNOWN_COMPRESSION_METHOD); - - return strings[static_cast(value)]; -} - -void SettingCompressionMethod::set(CompressionMethod x) -{ - value = x; - changed = true; -} - -void SettingCompressionMethod::set(const Field & x) -{ - set(safeGet(x)); -} - -void SettingCompressionMethod::set(const String & x) -{ - set(getCompressionMethod(x)); -} - -void SettingCompressionMethod::set(ReadBuffer & buf) -{ - String x; - readBinary(x, buf); - set(x); -} - -void SettingCompressionMethod::write(WriteBuffer & buf) const -{ - writeBinary(toString(), buf); -} - - DistributedProductMode SettingDistributedProductMode::getDistributedProductMode(const String & s) { if (s == "deny") return DistributedProductMode::DENY; diff --git a/dbms/src/Interpreters/SettingsCommon.h b/dbms/src/Interpreters/SettingsCommon.h index bfc0f30f8e5..9ed9e2f2746 100644 --- a/dbms/src/Interpreters/SettingsCommon.h +++ b/dbms/src/Interpreters/SettingsCommon.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include @@ -281,30 +281,6 @@ struct SettingOverflowMode void write(WriteBuffer & buf) const; }; - -struct SettingCompressionMethod -{ - CompressionMethod value; - bool changed = false; - - SettingCompressionMethod(CompressionMethod x = CompressionMethod::LZ4) : value(x) {} - - operator CompressionMethod() const { return value; } - SettingCompressionMethod & operator= (CompressionMethod x) { set(x); return *this; } - - static CompressionMethod getCompressionMethod(const String & s); - - String toString() const; - - void set(CompressionMethod x); - void set(const Field & x); - void set(const String & x); - void set(ReadBuffer & buf); - - void write(WriteBuffer & buf) const; -}; - - /// The setting for executing distributed subqueries inside IN or JOIN sections. enum class DistributedProductMode { diff --git a/dbms/src/Storages/CompressionCodecSelector.h b/dbms/src/Storages/CompressionCodecSelector.h new file mode 100644 index 00000000000..d005a249a1a --- /dev/null +++ b/dbms/src/Storages/CompressionCodecSelector.h @@ -0,0 +1,101 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_COMPRESSION_METHOD; + extern const int UNKNOWN_ELEMENT_IN_CONFIG; +} + + +/** Allows you to select the compression settings for the conditions specified in the configuration file. + * The config looks like this + + + + + + + + 10000000000 + 0.01 + + + zstd + 2 + + + + ... + + + */ +class CompressionCodecSelector +{ +private: + struct Element + { + size_t min_part_size = 0; + double min_part_size_ratio = 0; + std::string family_name; + std::optional level; + + + Element(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) + { + min_part_size = config.getUInt64(config_prefix + ".min_part_size", 0); + min_part_size_ratio = config.getDouble(config_prefix + ".min_part_size_ratio", 0); + + family_name = config.getString(config_prefix + ".method", "lz4"); + if (config.has(config_prefix + ".level")) + level = config.getInt64(config_prefix + ".level"); + } + + bool check(size_t part_size, double part_size_ratio) const + { + return part_size >= min_part_size + && part_size_ratio >= min_part_size_ratio; + } + }; + + std::vector elements; + +public: + CompressionCodecSelector() {} /// Always returns the default method. + + CompressionCodecSelector(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) + { + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_prefix, keys); + + for (const auto & name : keys) + { + if (!startsWith(name.data(), "case")) + throw Exception("Unknown element in config: " + config_prefix + "." + name + ", must be 'case'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + + elements.emplace_back(config, config_prefix + "." + name); + } + } + + CompressionCodecPtr choose(size_t part_size, double part_size_ratio) const + { + const auto & factory = CompressionCodecFactory::instance(); + CompressionCodecPtr res = factory.getDefaultCodec(); + + for (const auto & element : elements) + if (element.check(part_size, part_size_ratio)) + res = factory.get(element.family_name, element.level); + + return res; + } +}; + +} diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp index 77cb180af65..aca955fad7c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -14,6 +13,7 @@ #include #include #include +#include #include #include From 42b97357608d8c8de9b02df613899bd46178302d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 21 Dec 2018 17:03:53 +0300 Subject: [PATCH 045/230] Add LZ4HC --- dbms/src/Compression/CompressionCodecLZ4.cpp | 60 ++++++++++++++++++- dbms/src/Compression/CompressionCodecLZ4.h | 18 +++++- dbms/src/Compression/CompressionFactory.cpp | 11 ++-- dbms/src/Compression/CompressionFactory.h | 4 +- ...4_test_custom_compression_codecs.reference | 6 +- .../00804_test_custom_compression_codecs.sql | 41 +++++++------ utils/compressor/decompress_perf.cpp | 2 +- utils/compressor/mutator.cpp | 2 +- 8 files changed, 112 insertions(+), 32 deletions(-) diff --git a/dbms/src/Compression/CompressionCodecLZ4.cpp b/dbms/src/Compression/CompressionCodecLZ4.cpp index a9e54b36047..f5d63575720 100644 --- a/dbms/src/Compression/CompressionCodecLZ4.cpp +++ b/dbms/src/Compression/CompressionCodecLZ4.cpp @@ -5,11 +5,22 @@ #include #include #include "CompressionCodecLZ4.h" +#include +#include + namespace DB { +namespace ErrorCodes +{ +extern const int CANNOT_COMPRESS; +extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; +extern const int ILLEGAL_CODEC_PARAMETER; +} + + UInt8 CompressionCodecLZ4::getMethodByte() const { return static_cast(CompressionMethodByte::LZ4); @@ -37,9 +48,56 @@ void CompressionCodecLZ4::doDecompressData(const char * source, UInt32 source_si void registerCodecLZ4(CompressionCodecFactory & factory) { - factory.registerSimpleCompressionCodec("LZ4", static_cast(CompressionMethodByte::LZ4), [&](){ + factory.registerSimpleCompressionCodec("LZ4", static_cast(CompressionMethodByte::LZ4), [&] () { return std::make_shared(); }); } + +String CompressionCodecLZ4HC::getCodecDesc() const +{ + return "LZ4HC"; } + +UInt32 CompressionCodecLZ4HC::doCompressData(const char * source, UInt32 source_size, char * dest) const +{ + auto success = LZ4_compress_HC(source, dest, source_size, LZ4_COMPRESSBOUND(source_size), level); + + if (!success) + throw Exception("Cannot LZ4_compress_HC", ErrorCodes::CANNOT_COMPRESS); + + return success; +} + +void registerCodecLZ4HC(CompressionCodecFactory & factory) +{ + factory.registerCompressionCodec("LZ4HC", {}, [&](const ASTPtr & arguments) -> CompressionCodecPtr + { + int level = LZ4HC_CLEVEL_DEFAULT; + + if (arguments && !arguments->children.empty()) + { + if (arguments->children.size() > 1) + throw Exception("LZ4HC codec must have 1 parameter, given " + std::to_string(arguments->children.size()), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); + + const auto children = arguments->children; + const ASTLiteral * literal = static_cast(children[0].get()); + level = literal->value.safeGet(); + if (level > LZ4HC_CLEVEL_MAX || level < LZ4HC_CLEVEL_MIN) + throw Exception("LZ4HC codec can't have level more than " + + std::to_string(LZ4HC_CLEVEL_MAX) + " and less than " + + std::to_string(LZ4HC_CLEVEL_MIN) + ", given " + + std::to_string(level), ErrorCodes::ILLEGAL_CODEC_PARAMETER); + } + + return std::make_shared(level); + }); +} + +CompressionCodecLZ4HC::CompressionCodecLZ4HC(int level_) + : level(level_) +{ +} + +} + diff --git a/dbms/src/Compression/CompressionCodecLZ4.h b/dbms/src/Compression/CompressionCodecLZ4.h index 6af959ef4d8..fe3339b40a1 100644 --- a/dbms/src/Compression/CompressionCodecLZ4.h +++ b/dbms/src/Compression/CompressionCodecLZ4.h @@ -17,10 +17,11 @@ public: String getCodecDesc() const override; UInt32 getAdditionalSizeAtTheEndOfBuffer() const override { return LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER; } -private: +protected: UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; +private: void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const override; UInt32 getCompressedDataSize(UInt32 uncompressed_size) const override; @@ -28,4 +29,19 @@ private: mutable LZ4::PerformanceStatistics lz4_stat; }; +class CompressionCodecLZ4HC : public CompressionCodecLZ4 +{ +public: + + CompressionCodecLZ4HC(int level_); + + String getCodecDesc() const override; + +protected: + UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const override; + +private: + int level; +}; + } diff --git a/dbms/src/Compression/CompressionFactory.cpp b/dbms/src/Compression/CompressionFactory.cpp index eca19eafea9..53637b2259c 100644 --- a/dbms/src/Compression/CompressionFactory.cpp +++ b/dbms/src/Compression/CompressionFactory.cpp @@ -93,7 +93,7 @@ CompressionCodecPtr CompressionCodecFactory::getImpl(const String & family_name, return family_and_creator->second(arguments); } -void CompressionCodecFactory::registerCompressionCodec(const String & family_name, UInt8 byte_code, Creator creator) +void CompressionCodecFactory::registerCompressionCodec(const String & family_name, std::optional byte_code, Creator creator) { if (creator == nullptr) throw Exception("CompressionCodecFactory: the codec family " + family_name + " has been provided a null constructor", @@ -102,11 +102,12 @@ void CompressionCodecFactory::registerCompressionCodec(const String & family_nam if (!family_name_with_codec.emplace(family_name, creator).second) throw Exception("CompressionCodecFactory: the codec family name '" + family_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); - if (!family_code_with_codec.emplace(byte_code, creator).second) - throw Exception("CompressionCodecFactory: the codec family name '" + family_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); + if (byte_code) + if (!family_code_with_codec.emplace(*byte_code, creator).second) + throw Exception("CompressionCodecFactory: the codec family name '" + family_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); } -void CompressionCodecFactory::registerSimpleCompressionCodec(const String & family_name, UInt8 byte_code, +void CompressionCodecFactory::registerSimpleCompressionCodec(const String & family_name, std::optional byte_code, std::function creator) { registerCompressionCodec(family_name, byte_code, [family_name, creator](const ASTPtr & ast) @@ -122,6 +123,7 @@ void registerCodecLZ4(CompressionCodecFactory & factory); void registerCodecNone(CompressionCodecFactory & factory); void registerCodecZSTD(CompressionCodecFactory & factory); void registerCodecMultiple(CompressionCodecFactory & factory); +void registerCodecLZ4HC(CompressionCodecFactory & factory); //void registerCodecDelta(CompressionCodecFactory & factory); CompressionCodecFactory::CompressionCodecFactory() @@ -131,6 +133,7 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecNone(*this); registerCodecZSTD(*this); registerCodecMultiple(*this); + registerCodecLZ4HC(*this); // registerCodecDelta(*this); } diff --git a/dbms/src/Compression/CompressionFactory.h b/dbms/src/Compression/CompressionFactory.h index 43621ba4d78..1aef17de628 100644 --- a/dbms/src/Compression/CompressionFactory.h +++ b/dbms/src/Compression/CompressionFactory.h @@ -39,9 +39,9 @@ public: /// For backward compatibility with config settings CompressionCodecPtr get(const String & family_name, std::optional level) const; - void registerCompressionCodec(const String & family_name, UInt8 byte_code, Creator creator); + void registerCompressionCodec(const String & family_name, std::optional byte_code, Creator creator); - void registerSimpleCompressionCodec(const String & family_name, UInt8 byte_code, SimpleCreator creator); + void registerSimpleCompressionCodec(const String & family_name, std::optional byte_code, SimpleCreator creator); protected: CompressionCodecPtr getImpl(const String & family_name, const ASTPtr & arguments) const; diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference index 27037ff5641..ee64e2e922f 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference @@ -1,6 +1,6 @@ -1 hello 2018-12-14 1.1 -2 world 2018-12-15 2.2 -3 ! 2018-12-16 3.3 +1 hello 2018-12-14 1.1 aaa +2 world 2018-12-15 2.2 bbb +3 ! 2018-12-16 3.3 ccc 2 1 world 2018-10-05 1.1 2 hello 2018-10-01 2.2 diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 7b7c9977555..00d3afbb5cc 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -2,17 +2,17 @@ SET send_logs_level = 'none'; DROP TABLE IF EXISTS test.compression_codec; -CREATE TABLE test.compression_codec(id UInt64 CODEC(LZ4), data String CODEC(ZSTD), ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2))) ENGINE = MergeTree() ORDER BY tuple(); +CREATE TABLE test.compression_codec(id UInt64 CODEC(LZ4), data String CODEC(ZSTD), ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7))) ENGINE = MergeTree() ORDER BY tuple(); -INSERT INTO test.compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1); -INSERT INTO test.compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2); -INSERT INTO test.compression_codec VALUES(3, '!', toDate('2018-12-16'), 3.3); +INSERT INTO test.compression_codec VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa'); +INSERT INTO test.compression_codec VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb'); +INSERT INTO test.compression_codec VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc'); SELECT * FROM test.compression_codec ORDER BY id; OPTIMIZE TABLE test.compression_codec FINAL; -INSERT INTO test.compression_codec VALUES(2, '', toDate('2018-12-13'), 4.4); +INSERT INTO test.compression_codec VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd'); SELECT count(*) FROM test.compression_codec WHERE id = 2 GROUP BY id; @@ -22,27 +22,30 @@ DROP TABLE IF EXISTS test.bad_codec; DROP TABLE IF EXISTS test.params_when_no_params; DROP TABLE IF EXISTS test.params_zstd; DROP TABLE IF EXISTS test.too_many_params; -DROP TABLE IF EXISTS test.codec_multiple_direct_specification; +DROP TABLE IF EXISTS test.codec_multiple_direct_specification_1; +DROP TABLE IF EXISTS test.codec_multiple_direct_specification_2; CREATE TABLE test.bad_codec(id UInt64 CODEC(adssadads)) ENGINE = MergeTree() order by tuple(); -- { serverError 430 } CREATE TABLE test.too_many_params(id UInt64 CODEC(ZSTD(2,3,4,5))) ENGINE = MergeTree() order by tuple(); -- { serverError 429 } CREATE TABLE test.params_when_no_params(id UInt64 CODEC(LZ4(1))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 378 } -CREATE TABLE test.params_zstd(id UInt64 CODEC(ZSTD(33))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 431 } -CREATE TABLE test.codec_multiple_direct_specification(id UInt64 CODEC(MULTIPLE(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 430 } +CREATE TABLE test.params_zstd(id UInt64 CODEC(LZ4HC(0))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 431 } +CREATE TABLE test.codec_multiple_direct_specification_1(id UInt64 CODEC(MULTIPLE(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 430 } +CREATE TABLE test.codec_multiple_direct_specification_2(id UInt64 CODEC(multiple(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 430 } DROP TABLE IF EXISTS test.bad_codec; DROP TABLE IF EXISTS test.params_when_no_params; DROP TABLE IF EXISTS test.params_zstd; DROP TABLE IF EXISTS test.too_many_params; -DROP TABLE IF EXISTS test.codec_multiple_direct_specification; +DROP TABLE IF EXISTS test.codec_multiple_direct_specification_1; +DROP TABLE IF EXISTS test.codec_multiple_direct_specification_2; DROP TABLE IF EXISTS test.compression_codec_multiple; CREATE TABLE test.compression_codec_multiple ( - id UInt64 CODEC(LZ4, ZSTD, NONE), - data String CODEC(ZSTD(2), NONE, LZ4, LZ4), - ddd Date CODEC(NONE, NONE, NONE, LZ4, ZSTD), - somenum Float64 CODEC(LZ4, LZ4, ZSTD(2), ZSTD(3), ZSTD) + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC), + data String CODEC(ZSTD(2), NONE, LZ4HC, LZ4, LZ4), + ddd Date CODEC(NONE, NONE, NONE, LZ4, ZSTD, LZ4HC, LZ4HC), + somenum Float64 CODEC(LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD) ) ENGINE = MergeTree() ORDER BY tuple(); INSERT INTO test.compression_codec_multiple VALUES (1, 'world', toDate('2018-10-05'), 1.1), (2, 'hello', toDate('2018-10-01'), 2.2), (3, 'buy', toDate('2018-10-11'), 3.3); @@ -66,9 +69,9 @@ SELECT sum(cityHash64(*)) FROM test.compression_codec_multiple; DROP TABLE IF EXISTS test.compression_codec_multiple_more_types; CREATE TABLE test.compression_codec_multiple_more_types ( - id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD), - data FixedString(12) CODEC(ZSTD, ZSTD, NONE, NONE, NONE), - ddd Nested (age UInt8, Name String) CODEC(LZ4, NONE, NONE, NONE, ZSTD) + id Decimal128(13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, LZ4HC), + data FixedString(12) CODEC(ZSTD, ZSTD, NONE, NONE, NONE, LZ4HC), + ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD) ) ENGINE = MergeTree() ORDER BY tuple(); INSERT INTO test.compression_codec_multiple_more_types VALUES(1.5555555555555, 'hello world!', [77], ['John']); @@ -79,9 +82,9 @@ SELECT * FROM test.compression_codec_multiple_more_types order by id; DROP TABLE IF EXISTS test.compression_codec_multiple_with_key; CREATE TABLE test.compression_codec_multiple_with_key ( - somedate Date CODEC(ZSTD, ZSTD, ZSTD(12)), - id UInt64 CODEC(LZ4, ZSTD, NONE), - data String CODEC(ZSTD(2), NONE, LZ4, LZ4) + somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12)), + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC), + data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4) ) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; diff --git a/utils/compressor/decompress_perf.cpp b/utils/compressor/decompress_perf.cpp index 259d812fb82..91c2128cef1 100644 --- a/utils/compressor/decompress_perf.cpp +++ b/utils/compressor/decompress_perf.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/utils/compressor/mutator.cpp b/utils/compressor/mutator.cpp index 89a5d1d45c8..c8cca3e6ecf 100644 --- a/utils/compressor/mutator.cpp +++ b/utils/compressor/mutator.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include From 74ea83124084c26bf90f9604a935f7693e19d096 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 21 Dec 2018 17:06:49 +0300 Subject: [PATCH 046/230] Remove commented codec --- .../src/Compression/CompressionCodecDelta.cpp | 57 ------------------- dbms/src/Compression/CompressionCodecDelta.h | 26 --------- 2 files changed, 83 deletions(-) delete mode 100644 dbms/src/Compression/CompressionCodecDelta.cpp delete mode 100644 dbms/src/Compression/CompressionCodecDelta.h diff --git a/dbms/src/Compression/CompressionCodecDelta.cpp b/dbms/src/Compression/CompressionCodecDelta.cpp deleted file mode 100644 index 25f048a2dbe..00000000000 --- a/dbms/src/Compression/CompressionCodecDelta.cpp +++ /dev/null @@ -1,57 +0,0 @@ -//#include -//#include -//#include -// -// -//namespace DB -//{ -// -//char CompressionCodecDelta::getMethodByte() -//{ -// return static_cast(CompressionMethodByte::LZ4); -//} -// -//void CompressionCodecDelta::getCodecDesc(String & codec_desc) -//{ -// codec_desc = "DELTA"; -//} -// -//size_t CompressionCodecDelta::compress(char * source, size_t source_size, char * dest) -//{ -// /// TODO: use SIMD -// return LZ4_compress_default(source, dest, source_size, LZ4_COMPRESSBOUND(source_size)); -//} -// -//size_t CompressionCodecDelta::decompress(char * source, size_t source_size, char * dest, size_t size_decompressed) -//{ -// LZ4::decompress(source, dest, source_size, size_decompressed, lz4_stat); -// return size_decompressed; -//} -// -//void registerCodecLZ4(CompressionCodecFactory & factory) -//{ -// factory.registerCompressionCodec("DELTA", static_cast(CompressionMethodByte::DELTA), [&](ASTPtr & parameters) -// { -// int width = 1; -// -// if (arguments && !arguments->children.empty()) -// { -// const auto children = arguments->children; -// const ASTIdentifier * identifier = static_cast(children[0].get()); -// -// String delta_type = identifier->name; -// if (delta_type == "Int8" || delta_type == "UInt8") -// width = 1; -// else if (delta_type == "Int16" || delta_type == "UInt16") -// width = 2; -// else if (delta_type == "Int32" || delta_type == "UInt32") -// width = 4; -// else if (delta_type == "Int64" || delta_type == "UInt64") -// width = 8; -// } -// -// return std::make_shared(width); -// }); -//} -// -//} diff --git a/dbms/src/Compression/CompressionCodecDelta.h b/dbms/src/Compression/CompressionCodecDelta.h deleted file mode 100644 index d35edb46f68..00000000000 --- a/dbms/src/Compression/CompressionCodecDelta.h +++ /dev/null @@ -1,26 +0,0 @@ -//#pragma once -// -//#include -//#include -//#include -//#include -// -//namespace DB -//{ -// -//class CompressionCodecDelta : public ICompressionCodec -//{ -//public: -// char getMethodByte() override; -// -// void getCodecDesc(String & codec_desc) override; -// -// size_t compress(char * source, size_t source_size, char * dest) override; -// -// size_t getCompressedReserveSize(size_t uncompressed_size) override; -// -// size_t decompress(char * source, size_t source_size, char * dest, size_t decompressed_size) override; -// -//}; -// -//} \ No newline at end of file From a0240d8a0e507dc800a25f8cea98e6b8df159f3e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 21 Dec 2018 17:19:49 +0300 Subject: [PATCH 047/230] Remove accident change --- .../dictionaries/postgres_odbc_hashed_dictionary.xml | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml b/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml index 4871f268e9c..1c293f66761 100644 --- a/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml +++ b/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml @@ -13,15 +13,19 @@ 5 - + - + + column1 + + + column1 Int64 1 - + column2 From 661a117b91f2a296d97d4fde93f451e45306323c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 21 Dec 2018 19:03:40 +0300 Subject: [PATCH 048/230] Added assertion [#CLICKHOUSE-2] --- dbms/src/Storages/Kafka/StorageKafka.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 3d15259efcd..e125c694a6f 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -195,6 +195,9 @@ public: if (isCancelledOrThrowIfKilled() || !hasClaimed()) return {}; + if (!reader) + throw Exception("Logical error: reader is not initialized", ErrorCodes::LOGICAL_ERROR); + return reader->read(); } @@ -239,7 +242,7 @@ private: size_t max_block_size; Block sample_block; std::unique_ptr read_buf; - BlockInputStreamPtr reader = nullptr; + BlockInputStreamPtr reader; bool finalized = false; // Return true if consumer has been claimed by the stream From 2c6e49c3f36dca8a37013954fad9507881056ef6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 21 Dec 2018 19:24:47 +0300 Subject: [PATCH 049/230] More checks [#CLICKHOUSE-2] --- dbms/src/Storages/Kafka/StorageKafka.cpp | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index e125c694a6f..fd129b38519 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -63,10 +63,10 @@ static const String CONFIG_PREFIX = "kafka"; class ReadBufferFromKafkaConsumer : public ReadBuffer { rd_kafka_t * consumer; - rd_kafka_message_t * current; - bool current_pending; + rd_kafka_message_t * current = nullptr; + bool current_pending = false; /// We've fetched "current" message and need to process it on the next iteration. Poco::Logger * log; - size_t read_messages; + size_t read_messages = 0; char row_delimiter; bool nextImpl() override @@ -97,6 +97,10 @@ class ReadBufferFromKafkaConsumer : public ReadBuffer rd_kafka_message_destroy(msg); return nextImpl(); } + + if (msg->len && !msg->payload) + throw Exception("Logical error: nullptr message returned with non-zero length", ErrorCodes::LOGICAL_ERROR); + ++read_messages; // Now we've received a new message. Check if we need to produce a delimiter @@ -129,8 +133,7 @@ class ReadBufferFromKafkaConsumer : public ReadBuffer public: ReadBufferFromKafkaConsumer(rd_kafka_t * consumer_, Poco::Logger * log_, char row_delimiter_) - : ReadBuffer(nullptr, 0), consumer(consumer_), current(nullptr), - current_pending(false), log(log_), read_messages(0), row_delimiter(row_delimiter_) + : ReadBuffer(nullptr, 0), consumer(consumer_), log(log_), row_delimiter(row_delimiter_) { if (row_delimiter != '\0') LOG_TRACE(log, "Row delimiter is: " << row_delimiter); @@ -156,9 +159,8 @@ public: class KafkaBlockInputStream : public IProfilingBlockInputStream { public: - KafkaBlockInputStream(StorageKafka & storage_, const Context & context_, const String & schema, size_t max_block_size_) - : storage(storage_), consumer(nullptr), context(context_), max_block_size(max_block_size_) + : storage(storage_), context(context_), max_block_size(max_block_size_) { // Always skip unknown fields regardless of the context (JSON or TSKV) context.setSetting("input_format_skip_unknown_fields", 1u); From 71fad54eb26a14f4f00c8fd3219e69e41f900ed4 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 23 Dec 2018 16:21:55 +0800 Subject: [PATCH 050/230] Fix UB. --- dbms/src/Storages/StorageMerge.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 2a2471767e5..e7acd420dec 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -236,7 +236,7 @@ BlockInputStreams StorageMerge::read( else { source_streams.emplace_back(std::make_shared( - header, [=, &real_column_names, &modified_context]() -> BlockInputStreamPtr + header, [=]() mutable -> BlockInputStreamPtr { BlockInputStreams streams = createSourceStreams(query_info, processed_stage, max_block_size, header, storage, struct_lock, real_column_names, From 86c2d0f3290280c38b5fbbcba0f881d359ea6b78 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Dec 2018 23:01:17 +0300 Subject: [PATCH 051/230] Fixed buffer overflow in function addDays [#CLICKHOUSE-2] --- libs/libcommon/include/common/DateLUTImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h index 55a94f3733a..aa3fb4b6f1d 100644 --- a/libs/libcommon/include/common/DateLUTImpl.h +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -517,7 +517,7 @@ public: inline time_t addDays(time_t t, Int64 delta) const { - size_t index = findIndex(t); + UInt16 index = findIndex(t); /// Using UInt16 to possibly overflow within valid range. time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t); index += delta; From 986f96ada967c4c1459d3d777efc2a9db81c91a7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 00:37:42 +0300 Subject: [PATCH 052/230] Removed redundand code #3785 --- dbms/programs/server/Server.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index eda18809d66..499f233ff28 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -195,7 +195,6 @@ int Server::main(const std::vector & /*args*/) /// Check that the process' user id matches the owner of the data. const auto effective_user_id = geteuid(); struct stat statbuf; - const auto effective_user = getUserName(effective_user_id); if (stat(path.c_str(), &statbuf) == 0 && effective_user_id != statbuf.st_uid) { const auto effective_user = getUserName(effective_user_id); From 8367c99720415e672051f9f05ce230b6c7345b68 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 00:38:19 +0300 Subject: [PATCH 053/230] DateLUT: simpler, safer and more efficient [#CLICKHOUSE-2] --- libs/libcommon/include/common/DateLUTImpl.h | 58 +++++++++------------ libs/libcommon/src/DateLUTImpl.cpp | 6 +-- 2 files changed, 27 insertions(+), 37 deletions(-) diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h index aa3fb4b6f1d..45948b55425 100644 --- a/libs/libcommon/include/common/DateLUTImpl.h +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -71,26 +71,22 @@ private: /// We can correctly process only timestamps that less DATE_LUT_MAX (i.e. up to 2105 year inclusively) - inline size_t findIndex(time_t t) const + /// We don't care about overflow. + inline DayNum findIndex(time_t t) const { /// First guess. - size_t guess = t / 86400; - if (guess >= DATE_LUT_MAX_DAY_NUM) - return 0; - if (t >= lut[guess].date && t < lut[guess + 1].date) + DayNum guess(t / 86400); + + /// UTC offset is from -12 to +14 in all known time zones. This requires checking only three indices. + + if (t >= lut[guess].date && t < lut[DayNum(guess + 1)].date) return guess; - for (size_t i = 1;; ++i) - { - if (guess + i >= DATE_LUT_MAX_DAY_NUM) - return 0; - if (t >= lut[guess + i].date && t < lut[guess + i + 1].date) - return guess + i; - if (guess < i) - return 0; - if (t >= lut[guess - i].date && t < lut[guess - i + 1].date) - return guess - i; - } + /// Time zones that have offset 0 from UTC do daylight saving time change (if any) towards increasing UTC offset (example: British Standard Time). + if (offset_at_start_of_epoch >= 0) + return DayNum(guess + 1); + + return DayNum(guess - 1); } inline const Values & find(time_t t) const @@ -113,8 +109,8 @@ public: /// Round down to start of monday. inline time_t toFirstDayOfWeek(time_t t) const { - size_t index = findIndex(t); - return lut[index - (lut[index].day_of_week - 1)].date; + DayNum index = findIndex(t); + return lut[DayNum(index - (lut[index].day_of_week - 1))].date; } inline DayNum toFirstDayNumOfWeek(DayNum d) const @@ -130,7 +126,7 @@ public: /// Round down to start of month. inline time_t toFirstDayOfMonth(time_t t) const { - size_t index = findIndex(t); + DayNum index = findIndex(t); return lut[index - (lut[index].day_of_month - 1)].date; } @@ -147,13 +143,13 @@ public: /// Round down to start of quarter. inline DayNum toFirstDayNumOfQuarter(DayNum d) const { - size_t index = d; + DayNum index = d; size_t month_inside_quarter = (lut[index].month - 1) % 3; - index = index - lut[index].day_of_month; + index -= lut[index].day_of_month; while (month_inside_quarter) { - index = index - lut[index].day_of_month; + index -= lut[index].day_of_month; --month_inside_quarter; } @@ -188,14 +184,14 @@ public: inline time_t toFirstDayOfNextMonth(time_t t) const { - size_t index = findIndex(t); + DayNum index = findIndex(t); index += 32 - lut[index].day_of_month; return lut[index - (lut[index].day_of_month - 1)].date; } inline time_t toFirstDayOfPrevMonth(time_t t) const { - size_t index = findIndex(t); + DayNum index = findIndex(t); index -= lut[index].day_of_month; return lut[index - (lut[index].day_of_month - 1)].date; } @@ -213,7 +209,7 @@ public: inline UInt8 daysInMonth(UInt16 year, UInt8 month) const { /// 32 makes arithmetic more simple. - auto any_day_of_month = years_lut[year - DATE_LUT_MIN_YEAR] + 32 * (month - 1); + DayNum any_day_of_month = DayNum(years_lut[year - DATE_LUT_MIN_YEAR] + 32 * (month - 1)); return lut[any_day_of_month].days_in_month; } @@ -221,12 +217,12 @@ public: */ inline time_t toDateAndShift(time_t t, Int32 days) const { - return lut[findIndex(t) + days].date; + return lut[DayNum(findIndex(t) + days)].date; } inline time_t toTime(time_t t) const { - size_t index = findIndex(t); + DayNum index = findIndex(t); if (unlikely(index == 0)) return t + offset_at_start_of_epoch; @@ -241,7 +237,7 @@ public: inline unsigned toHour(time_t t) const { - size_t index = findIndex(t); + DayNum index = findIndex(t); /// If it is not 1970 year (findIndex found nothing appropriate), /// than limit number of hours to avoid insane results like 1970-01-01 89:28:15 @@ -301,7 +297,7 @@ public: * because the same calendar day starts/ends at different timestamps in different time zones) */ - inline DayNum toDayNum(time_t t) const { return static_cast(findIndex(t)); } + inline DayNum toDayNum(time_t t) const { return findIndex(t); } inline time_t fromDayNum(DayNum d) const { return lut[d].date; } inline time_t toDate(DayNum d) const { return lut[d].date; } @@ -517,7 +513,7 @@ public: inline time_t addDays(time_t t, Int64 delta) const { - UInt16 index = findIndex(t); /// Using UInt16 to possibly overflow within valid range. + DayNum index = findIndex(t); time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t); index += delta; @@ -687,6 +683,4 @@ public: return s; } - - inline bool isOffsetWholeNumberOfHoursEveryTime() const { return offset_is_whole_number_of_hours_everytime; } }; diff --git a/libs/libcommon/src/DateLUTImpl.cpp b/libs/libcommon/src/DateLUTImpl.cpp index 76afcd548c2..dfe0cdc0760 100644 --- a/libs/libcommon/src/DateLUTImpl.cpp +++ b/libs/libcommon/src/DateLUTImpl.cpp @@ -56,7 +56,6 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) cctz::time_zone::absolute_lookup start_of_epoch_lookup = cctz_time_zone.lookup(std::chrono::system_clock::from_time_t(start_of_day)); offset_at_start_of_epoch = start_of_epoch_lookup.offset; - offset_is_whole_number_of_hours_everytime = true; cctz::civil_day date{1970, 1, 1}; @@ -84,9 +83,6 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) values.time_at_offset_change = 0; values.amount_of_offset_change = 0; - if (start_of_day % 3600) - offset_is_whole_number_of_hours_everytime = false; - /// If UTC offset was changed in previous day. if (i != 0) { @@ -129,7 +125,7 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) /// Fill excessive part of lookup table. This is needed only to simplify handling of overflow cases. while (i < DATE_LUT_SIZE) { - lut[i] = lut[0]; + lut[i] = lut[DATE_LUT_MAX_DAY_NUM]; ++i; } From 344d610ef04c3a3b7d31ea3c6901c91850307537 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 00:45:28 +0300 Subject: [PATCH 054/230] DateLUT: simpler, safer and more efficient [#CLICKHOUSE-2] --- libs/libcommon/src/DateLUTImpl.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/libs/libcommon/src/DateLUTImpl.cpp b/libs/libcommon/src/DateLUTImpl.cpp index dfe0cdc0760..3f812accb48 100644 --- a/libs/libcommon/src/DateLUTImpl.cpp +++ b/libs/libcommon/src/DateLUTImpl.cpp @@ -56,6 +56,7 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) cctz::time_zone::absolute_lookup start_of_epoch_lookup = cctz_time_zone.lookup(std::chrono::system_clock::from_time_t(start_of_day)); offset_at_start_of_epoch = start_of_epoch_lookup.offset; + offset_is_whole_number_of_hours_everytime = true; cctz::civil_day date{1970, 1, 1}; @@ -83,6 +84,9 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) values.time_at_offset_change = 0; values.amount_of_offset_change = 0; + if (start_of_day % 3600) + offset_is_whole_number_of_hours_everytime = false; + /// If UTC offset was changed in previous day. if (i != 0) { From f09da489f7e6e92d730515ee08c679bdef87d38d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 01:10:48 +0300 Subject: [PATCH 055/230] Added test #3913 --- .../queries/0_stateless/00809_add_days_segfault.reference | 5 +++++ dbms/tests/queries/0_stateless/00809_add_days_segfault.sql | 6 ++++++ 2 files changed, 11 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00809_add_days_segfault.reference create mode 100644 dbms/tests/queries/0_stateless/00809_add_days_segfault.sql diff --git a/dbms/tests/queries/0_stateless/00809_add_days_segfault.reference b/dbms/tests/queries/0_stateless/00809_add_days_segfault.reference new file mode 100644 index 00000000000..229972f2924 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00809_add_days_segfault.reference @@ -0,0 +1,5 @@ +0 +0 +0 +0 +0 diff --git a/dbms/tests/queries/0_stateless/00809_add_days_segfault.sql b/dbms/tests/queries/0_stateless/00809_add_days_segfault.sql new file mode 100644 index 00000000000..8ddc9ba1a27 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00809_add_days_segfault.sql @@ -0,0 +1,6 @@ +SELECT ignore(addDays(toDateTime(0), -1)); +SELECT ignore(subtractDays(toDateTime(0), 1)); +SELECT ignore(addDays((CAST((96.338) AS DateTime)), -3)); + +SELECT ignore(addDays(toDate(0), -1)); +SELECT ignore(subtractDays(toDate(0), 1)); From ce7c35ac7d7213850b6864e56b4399bf41200a13 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 01:11:29 +0300 Subject: [PATCH 056/230] Updated documentation about ClickHouse testing [#CLICKHOUSE-2] --- docs/en/development/tests.md | 27 +++++++++++++++++---------- 1 file changed, 17 insertions(+), 10 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 5455a234ae3..d9a44f78ea3 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -9,13 +9,13 @@ Each functional test sends one or multiple queries to the running ClickHouse ser Tests are located in `dbms/src/tests/queries` directory. There are two subdirectories: `stateless` and `stateful`. Stateless tests run queries without any preloaded test data - they often create small synthetic datasets on the fly, within the test itself. Stateful tests require preloaded test data from Yandex.Metrica and not available to general public. We tend to use only `stateless` tests and avoid adding new `stateful` tests. -Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery`. `.sh` test is a script that is run by itself. +Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery --testmode`. `.sh` test is a script that is run by itself. To run all tests, use `dbms/tests/clickhouse-test` tool. Look `--help` for the list of possible options. You can simply run all tests or run subset of tests filtered by substring in test name: `./clickhouse-test substring`. The most simple way to invoke functional tests is to copy `clickhouse-client` to `/usr/bin/`, run `clickhouse-server` and then run `./clickhouse-test` from its own directory. -To add new test, create a `.sql` or `.sh` file in `dbms/src/tests/queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client -n < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. +To add new test, create a `.sql` or `.sh` file in `dbms/src/tests/queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client -n --testmode < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables. @@ -24,6 +24,11 @@ If you want to use distributed queries in functional tests, you can leverage `re Some tests are marked with `zookeeper`, `shard` or `long` in their names. `zookeeper` is for tests that are using ZooKeeper; `shard` is for tests that requires server to listen `127.0.0.*`; `long` is for tests that run slightly longer that one second. +## Known bugs + +If we know some bugs that can be easily reproduced by functional tests, we place prepared functional tests in `dbms/src/tests/queries/bugs` directory. These tests will be moved to `dbms/src/tests/queries/0_stateless` when bugs are fixed. + + ## Integration Tests Integration tests allow to test ClickHouse in clustered configuration and ClickHouse interaction with other servers like MySQL, Postgres, MongoDB. They are useful to emulate network splits, packet drops, etc. These tests are run under Docker and create multiple containers with various software. @@ -55,7 +60,7 @@ Performance tests are not run on per-commit basis. Results of performance tests Some programs in `tests` directory are not prepared tests, but are test tools. For example, for `Lexer` there is a tool `dbms/src/Parsers/tests/lexer` that just do tokenization of stdin and writes colorized result to stdout. You can use these kind of tools as a code examples and for exploration and manual testing. -You can also place pair of files `.sh` and `.reference` along with the tool to run it on some predefined input - then script result can be compared to `.reference` file. There kind of tests are not automated. +You can also place pair of files `.sh` and `.reference` along with the tool to run it on some predefined input - then script result can be compared to `.reference` file. These kind of tests are not automated. ## Miscellanous Tests @@ -173,7 +178,7 @@ For production builds, gcc is used (it still generates slightly more efficient c ## Sanitizers **Address sanitizer**. -We run functional tests under ASan on per-commit basis. +We run functional and integration tests under ASan on per-commit basis. **Valgrind (Memcheck)**. We run functional tests under Valgrind overnight. It takes multiple hours. Currently there is one known false positive in `re2` library, see [this article](https://research.swtch.com/sparse). @@ -185,7 +190,7 @@ We run functional tests under TSan. ClickHouse must pass all tests. Run under TS Currently we still don't use MSan. **Undefined behaviour sanitizer.** -We still don't use UBSan. The only thing to fix is unaligned placement of structs in Arena during aggregation. This is totally fine, we only have to force alignment under UBSan. +We still don't use UBSan on per commit basis. There are some places to fix. **Debug allocator.** You can enable debug version of `tcmalloc` with `DEBUG_TCMALLOC` CMake option. We run tests with debug allocator on per-commit basis. @@ -195,7 +200,9 @@ You will find some additional details in `dbms/tests/instructions/sanitizers.txt ## Fuzzing -As of July 2018 we don't use fuzzing. +We use simple fuzz test to generate random SQL queries and to check that the server doesn't die. Fuzz testing is performed with Address sanitizer. You can find it in `00746_sql_fuzzy.pl`. This test should be run continuously (overnight and longer). + +As of December 2018, we still don't use isolated fuzz testing of library code. ## Security Audit @@ -242,12 +249,12 @@ As of July 2018 we don't track test coverage. ## Test Automation -We run tests with Travis CI (available for general public) and Jenkins (available inside Yandex). +We run tests with Yandex internal CI and job automation system named "Sandbox". We also continue to use Jenkins (available inside Yandex). -In Travis CI due to limit on time and computational power we can afford only subset of functional tests that are run with limited build of ClickHouse (debug version with cut off most of libraries). In about half of runs it still fails to finish in 50 minutes timeout. The only advantage - test results are visible for all external contributors. +Build jobs and tests are run in Sandbox on per commit basis. Resulting packages and test results are published in GitHub and can be downloaded by direct links. Artifacts are stored eternally. When you send a pull request on GitHub, we tag it as "can be tested" and our CI system will build ClickHouse packages (release, debug, with address sanitizer, etc) for you. -In Jenkins we run functional tests for each commit and for each pull request from trusted users; the same under ASan; we also run quorum tests, dictionary tests, Metrica B2B tests. We use Jenkins to prepare and publish releases. Worth to note that we are not happy with Jenkins at all. +We don't use Travis CI due to the limit on time and computational power. -One of our goals is to provide reliable testing infrastructure that will be available to community. +In Jenkins we run dictionary tests, Metrica B2B tests. We use Jenkins to prepare and publish releases. Jenkins is a legacy technology and all jobs will be moved to Sandbox. [Original article](https://clickhouse.yandex/docs/en/development/tests/) From 5ccf9e1d54151aca59dd8d9205f312e3ee2c99bf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 01:16:04 +0300 Subject: [PATCH 057/230] Updated test #3913 --- .../queries/0_stateless/00809_add_days_segfault.reference | 1 + .../tests/queries/0_stateless/00809_add_days_segfault.sql | 8 +++++++- dbms/tests/queries/bugs/fuzzy.sql | 4 ---- 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00809_add_days_segfault.reference b/dbms/tests/queries/0_stateless/00809_add_days_segfault.reference index 229972f2924..f7eb44d66e0 100644 --- a/dbms/tests/queries/0_stateless/00809_add_days_segfault.reference +++ b/dbms/tests/queries/0_stateless/00809_add_days_segfault.reference @@ -3,3 +3,4 @@ 0 0 0 +0 diff --git a/dbms/tests/queries/0_stateless/00809_add_days_segfault.sql b/dbms/tests/queries/0_stateless/00809_add_days_segfault.sql index 8ddc9ba1a27..b087f7bbde5 100644 --- a/dbms/tests/queries/0_stateless/00809_add_days_segfault.sql +++ b/dbms/tests/queries/0_stateless/00809_add_days_segfault.sql @@ -1,6 +1,12 @@ SELECT ignore(addDays(toDateTime(0), -1)); SELECT ignore(subtractDays(toDateTime(0), 1)); -SELECT ignore(addDays((CAST((96.338) AS DateTime)), -3)); SELECT ignore(addDays(toDate(0), -1)); SELECT ignore(subtractDays(toDate(0), 1)); + +SET send_logs_level = 'none'; + +SELECT ignore(addDays((CAST((96.338) AS DateTime)), -3)); +SELECT ignore(subtractDays((CAST((-5263074.47) AS DateTime)), -737895)); +SELECT quantileDeterministic([], findClusterIndex(( SELECT subtractDays((CAST((566450.398706) AS DateTime)), 54) ) )), '\0', []; -- { serverError 42 } +SELECT sequenceCount((CAST((( SELECT NULL ) AS rg, ( SELECT ( SELECT [], 'A') AS String))]]); SELECT truncate(895, -16); SELECT notIn([['']], [[NULL]]); -SELECT subtractDays((CAST((-5263074.47) AS DateTime)), -737895); -SELECT quantileDeterministic([], findClusterIndex(( SELECT subtractDays((CAST((566450.398706) AS DateTime)), 54) ) )), '\0', []; -SELECT addDays((CAST((96.338) AS DateTime)), -3); From c828afd3db892ef1a4bd86b2b185e2fdcb969a55 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 01:20:44 +0300 Subject: [PATCH 058/230] Added a test for already fixed bug [#CLICKHOUSE-2] --- .../0_stateless/00810_in_operators_segfault.reference | 0 .../queries/0_stateless/00810_in_operators_segfault.sql | 5 +++++ dbms/tests/queries/bugs/fuzzy.sql | 3 ++- 3 files changed, 7 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/00810_in_operators_segfault.reference create mode 100644 dbms/tests/queries/0_stateless/00810_in_operators_segfault.sql diff --git a/dbms/tests/queries/0_stateless/00810_in_operators_segfault.reference b/dbms/tests/queries/0_stateless/00810_in_operators_segfault.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00810_in_operators_segfault.sql b/dbms/tests/queries/0_stateless/00810_in_operators_segfault.sql new file mode 100644 index 00000000000..1fa525eaccc --- /dev/null +++ b/dbms/tests/queries/0_stateless/00810_in_operators_segfault.sql @@ -0,0 +1,5 @@ +SET send_logs_level = 'none'; + +SELECT globalNotIn(['"wh'], [NULL]); -- { serverError 53 } +SELECT globalIn([''], [NULL]); -- { serverError 53 } +SELECT notIn([['']], [[NULL]]); -- { serverError 53 } diff --git a/dbms/tests/queries/bugs/fuzzy.sql b/dbms/tests/queries/bugs/fuzzy.sql index ff82baca47a..541d88eab32 100644 --- a/dbms/tests/queries/bugs/fuzzy.sql +++ b/dbms/tests/queries/bugs/fuzzy.sql @@ -1,5 +1,6 @@ SELECT globalNotIn(['"wh'], [NULL]); SELECT globalIn([''], [NULL]) +SELECT notIn([['']], [[NULL]]); + SELECT ( SELECT toDecimal128([], rowNumberInBlock()) ) , lcm('', [[(CAST(('>A') AS String))]]); SELECT truncate(895, -16); -SELECT notIn([['']], [[NULL]]); From 80f155cfda80cf09086825236e3497a1d3afc8c9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 02:12:30 +0300 Subject: [PATCH 059/230] Fixed overflow in rounding functions with integer argument and large negative scale [#CLICKHOUSE-2] --- dbms/src/Functions/FunctionsRound.h | 4 ++-- dbms/tests/queries/0_stateless/00811_garbage.reference | 1 + dbms/tests/queries/0_stateless/00811_garbage.sql | 4 ++++ dbms/tests/queries/bugs/fuzzy.sql | 2 -- libs/libcommon/include/common/intExp.h | 5 +++-- 5 files changed, 10 insertions(+), 6 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00811_garbage.reference create mode 100644 dbms/tests/queries/0_stateless/00811_garbage.sql diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 3c9be5f15b9..db41a72eadb 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -87,7 +87,7 @@ struct IntegerRoundingComputation return scale; } - static ALWAYS_INLINE T computeImpl(T x, T scale) + static ALWAYS_INLINE T computeImpl(T x, size_t scale) { switch (rounding_mode) { @@ -122,7 +122,7 @@ struct IntegerRoundingComputation } } - static ALWAYS_INLINE T compute(T x, T scale) + static ALWAYS_INLINE T compute(T x, size_t scale) { switch (scale_mode) { diff --git a/dbms/tests/queries/0_stateless/00811_garbage.reference b/dbms/tests/queries/0_stateless/00811_garbage.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00811_garbage.reference @@ -0,0 +1 @@ +0 diff --git a/dbms/tests/queries/0_stateless/00811_garbage.sql b/dbms/tests/queries/0_stateless/00811_garbage.sql new file mode 100644 index 00000000000..65db1c736fa --- /dev/null +++ b/dbms/tests/queries/0_stateless/00811_garbage.sql @@ -0,0 +1,4 @@ +SET send_logs_level = 'none'; + +SELECT truncate(895, -16); +SELECT ( SELECT toDecimal128([], rowNumberInBlock()) ) , lcm('', [[(CAST(('>A') AS String))]]); -- { serverError 44 } diff --git a/dbms/tests/queries/bugs/fuzzy.sql b/dbms/tests/queries/bugs/fuzzy.sql index 1468ed648b2..2830fe7c2f9 100644 --- a/dbms/tests/queries/bugs/fuzzy.sql +++ b/dbms/tests/queries/bugs/fuzzy.sql @@ -1,6 +1,4 @@ SELECT sequenceCount((CAST((( SELECT NULL ) AS rg, ( SELECT ( SELECT [], 'A') AS String))]]); -SELECT truncate(895, -16); SELECT subtractDays((CAST((-5263074.47) AS DateTime)), -737895); SELECT quantileDeterministic([], findClusterIndex(( SELECT subtractDays((CAST((566450.398706) AS DateTime)), 54) ) )), '\0', []; SELECT addDays((CAST((96.338) AS DateTime)), -3); diff --git a/libs/libcommon/include/common/intExp.h b/libs/libcommon/include/common/intExp.h index 9ecd3f07dd1..8c46d9f26c2 100644 --- a/libs/libcommon/include/common/intExp.h +++ b/libs/libcommon/include/common/intExp.h @@ -32,7 +32,8 @@ inline uint64_t intExp10(int x) return table[x]; } -namespace common { +namespace common +{ inline int exp10_i32(int x) { @@ -123,4 +124,4 @@ inline __int128 exp10_i128(int x) return values[x]; } -} // common +} From 0b760ee9e92dd9e834a62d956376d857231803ff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 03:47:11 +0300 Subject: [PATCH 060/230] Fixed overflow in rounding functions with integer argument and large negative scale [#CLICKHOUSE-2] --- dbms/src/Functions/FunctionsRound.h | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index db41a72eadb..4d8f5edccc3 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -87,7 +87,7 @@ struct IntegerRoundingComputation return scale; } - static ALWAYS_INLINE T computeImpl(T x, size_t scale) + static ALWAYS_INLINE T computeImpl(T x, T scale) { switch (rounding_mode) { @@ -122,7 +122,7 @@ struct IntegerRoundingComputation } } - static ALWAYS_INLINE T compute(T x, size_t scale) + static ALWAYS_INLINE T compute(T x, T scale) { switch (scale_mode) { @@ -139,7 +139,10 @@ struct IntegerRoundingComputation static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) { - *out = compute(*in, scale); + if (scale > size_t(std::numeric_limits::max())) + *out = 0; + else + *out = compute(*in, scale); } }; From 4f9630fd03fee72208d5dce4103b3157d19f6a6a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 05:57:22 +0300 Subject: [PATCH 061/230] Fixed error #3913 --- libs/libcommon/include/common/DateLUTImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h index 45948b55425..8fd015afa15 100644 --- a/libs/libcommon/include/common/DateLUTImpl.h +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -79,7 +79,7 @@ private: /// UTC offset is from -12 to +14 in all known time zones. This requires checking only three indices. - if (t >= lut[guess].date && t < lut[DayNum(guess + 1)].date) + if ((guess == 0 || t >= lut[guess].date) && t < lut[DayNum(guess + 1)].date) return guess; /// Time zones that have offset 0 from UTC do daylight saving time change (if any) towards increasing UTC offset (example: British Standard Time). From 416d6aa788685388104afc83227fcb732ab01a91 Mon Sep 17 00:00:00 2001 From: qianlixiang Date: Mon, 24 Dec 2018 16:17:22 +0800 Subject: [PATCH 062/230] Fixed core dump caused by kill query sync --- dbms/src/Interpreters/InterpreterKillQueryQuery.cpp | 7 +++++-- dbms/src/Parsers/ASTKillQueryQuery.h | 8 ++++++-- dbms/src/Parsers/ParserKillQueryQuery.cpp | 3 ++- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp index 7e8d783836a..31535bc4bbe 100644 --- a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -221,8 +221,11 @@ BlockIO InterpreterKillQueryQuery::execute() Block InterpreterKillQueryQuery::getSelectFromSystemProcessesResult() { - String system_processes_query = "SELECT query_id, user, query FROM system.processes WHERE " - + queryToString(static_cast(*query_ptr).where_expression); + String system_processes_query = "SELECT query_id, user, query FROM system.processes"; + auto & where_expression = static_cast(*query_ptr).where_expression; + if (where_expression) + system_processes_query += " WHERE " + queryToString(where_expression); + BlockIO system_processes_io = executeQuery(system_processes_query, context, true); Block res = system_processes_io.in->read(); diff --git a/dbms/src/Parsers/ASTKillQueryQuery.h b/dbms/src/Parsers/ASTKillQueryQuery.h index 7099fbccece..34fdba051d1 100644 --- a/dbms/src/Parsers/ASTKillQueryQuery.h +++ b/dbms/src/Parsers/ASTKillQueryQuery.h @@ -15,8 +15,12 @@ public: ASTPtr clone() const override { auto clone = std::make_shared(*this); - clone->where_expression = where_expression->clone(); - clone->children = {clone->where_expression}; + if (where_expression) + { + clone->where_expression = where_expression->clone(); + clone->children = {clone->where_expression}; + } + return clone; } diff --git a/dbms/src/Parsers/ParserKillQueryQuery.cpp b/dbms/src/Parsers/ParserKillQueryQuery.cpp index 5e674d9da83..56b0d3cd557 100644 --- a/dbms/src/Parsers/ParserKillQueryQuery.cpp +++ b/dbms/src/Parsers/ParserKillQueryQuery.cpp @@ -39,7 +39,8 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect query->test = true; query->cluster = cluster_str; - query->children.emplace_back(query->where_expression); + if (query->where_expression) + query->children.emplace_back(query->where_expression); node = std::move(query); return true; } From 184c36d78d5f4949038bc7107449ec067628ed64 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 14:06:35 +0300 Subject: [PATCH 063/230] Fixed error in KILL QUERY #3916 --- dbms/src/Parsers/ASTKillQueryQuery.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Parsers/ASTKillQueryQuery.cpp b/dbms/src/Parsers/ASTKillQueryQuery.cpp index 9e7631eacc3..0c9dba01219 100644 --- a/dbms/src/Parsers/ASTKillQueryQuery.cpp +++ b/dbms/src/Parsers/ASTKillQueryQuery.cpp @@ -13,10 +13,12 @@ void ASTKillQueryQuery::formatQueryImpl(const FormatSettings & settings, FormatS settings.ostr << (settings.hilite ? hilite_keyword : "") << "KILL QUERY"; formatOnCluster(settings); - settings.ostr << " WHERE " << (settings.hilite ? hilite_none : ""); if (where_expression) + { + settings.ostr << " WHERE " << (settings.hilite ? hilite_none : ""); where_expression->formatImpl(settings, state, frame); + } settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << (test ? "TEST" : (sync ? "SYNC" : "ASYNC")) << (settings.hilite ? hilite_none : ""); } From 7cfa7c3dbe1865d22dde63d0a434806e4597b922 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 14:08:09 +0300 Subject: [PATCH 064/230] KILL QUERY: make WHERE clause mandatory #3916 --- dbms/src/Parsers/ParserKillQueryQuery.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dbms/src/Parsers/ParserKillQueryQuery.cpp b/dbms/src/Parsers/ParserKillQueryQuery.cpp index 56b0d3cd557..29e0b572f83 100644 --- a/dbms/src/Parsers/ParserKillQueryQuery.cpp +++ b/dbms/src/Parsers/ParserKillQueryQuery.cpp @@ -28,7 +28,7 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect if (p_on.ignore(pos, expected) && !ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) return false; - if (p_where.ignore(pos, expected) && !p_where_expression.parse(pos, query->where_expression, expected)) + if (!p_where.ignore(pos, expected) || !p_where_expression.parse(pos, query->where_expression, expected)) return false; if (p_sync.ignore(pos, expected)) @@ -39,8 +39,7 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect query->test = true; query->cluster = cluster_str; - if (query->where_expression) - query->children.emplace_back(query->where_expression); + query->children.emplace_back(query->where_expression); node = std::move(query); return true; } From 67ad598582646f568a35bc3b3c9898cd59ba5918 Mon Sep 17 00:00:00 2001 From: mf5137 Date: Mon, 24 Dec 2018 12:38:07 +0100 Subject: [PATCH 065/230] Performance tests for hash functions --- .../consistent_hashes.xml | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/performance/{consistent_hashes => functions_hashing}/consistent_hashes.xml (100%) diff --git a/dbms/tests/performance/consistent_hashes/consistent_hashes.xml b/dbms/tests/performance/functions_hashing/consistent_hashes.xml similarity index 100% rename from dbms/tests/performance/consistent_hashes/consistent_hashes.xml rename to dbms/tests/performance/functions_hashing/consistent_hashes.xml From 1070702168f29704080b94f7a808f4d0a16ab2b4 Mon Sep 17 00:00:00 2001 From: KochetovNicolai Date: Mon, 24 Dec 2018 15:35:46 +0300 Subject: [PATCH 066/230] Update CatBoostModel.cpp Do not evaluate CatBoost model on empty dataset. --- dbms/src/Interpreters/CatBoostModel.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/src/Interpreters/CatBoostModel.cpp b/dbms/src/Interpreters/CatBoostModel.cpp index 61fc1d19785..bd7c07813d0 100644 --- a/dbms/src/Interpreters/CatBoostModel.cpp +++ b/dbms/src/Interpreters/CatBoostModel.cpp @@ -331,6 +331,9 @@ private: auto result = ColumnFloat64::create(column_size); auto result_buf = result->getData().data(); + if (!column_size) + return result; + /// Prepare float features. PODArray float_features(column_size); auto float_features_buf = float_features.data(); From 99c73209a942db1d5a1c76a9c052dbb637768324 Mon Sep 17 00:00:00 2001 From: KochetovNicolai Date: Mon, 24 Dec 2018 15:37:46 +0300 Subject: [PATCH 067/230] Update build_catboost.sh Fix build script for CatBoost models. --- dbms/tests/external_models/catboost/data/build_catboost.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/external_models/catboost/data/build_catboost.sh b/dbms/tests/external_models/catboost/data/build_catboost.sh index 080c5bcb245..50a3fb43ef8 100755 --- a/dbms/tests/external_models/catboost/data/build_catboost.sh +++ b/dbms/tests/external_models/catboost/data/build_catboost.sh @@ -12,6 +12,6 @@ cd $DIR ln -sf "${DIR}/build/lib/catboost/libs/model_interface/libcatboostmodel.so" libcatboostmodel.so cd "${DIR}/catboost/catboost/python-package/catboost" -../../../ya make -r -DUSE_ARCADIA_PYTHON=no -DPYTHON_CONFIG=python2-config -j4 +../../../ya make -r -DUSE_ARCADIA_PYTHON=no -DOS_SDK=local -DPYTHON_CONFIG=python2-config -j4 cd $DIR ln -sf "${DIR}/catboost/catboost/python-package" python-package From a380cb11f876125ebed8734871d1fba915b8c296 Mon Sep 17 00:00:00 2001 From: proller Date: Mon, 24 Dec 2018 15:45:50 +0300 Subject: [PATCH 068/230] CLICKHOUSE-4223 Fix .deb install bad message --- debian/clickhouse-server.postinst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/debian/clickhouse-server.postinst b/debian/clickhouse-server.postinst index e17c47df6dd..80b25d47f62 100644 --- a/debian/clickhouse-server.postinst +++ b/debian/clickhouse-server.postinst @@ -76,7 +76,7 @@ Please fix this and reinstall this package." >&2 fi if [ -d ${CLICKHOUSE_CONFDIR} ]; then - rm -v ${CLICKHOUSE_CONFDIR}/*-preprocessed.xml ||: + rm -fv ${CLICKHOUSE_CONFDIR}/*-preprocessed.xml ||: fi ln -s ${CLICKHOUSE_DATADIR}/preprocessed_configs ${CLICKHOUSE_CONFDIR}/preprocessed ||: From 1dffa56073fc835bbfa05de8da9fdb84f16d000c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 15:47:24 +0300 Subject: [PATCH 069/230] Fixed test #3913 --- .../0_stateless/00569_parse_date_time_best_effort.reference | 1 - .../queries/0_stateless/00569_parse_date_time_best_effort.sql | 1 - 2 files changed, 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference b/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference index 4b209d6a90f..8638c0b707f 100644 --- a/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference +++ b/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference @@ -7,7 +7,6 @@ s a b 02/01/17 010203 MSK 2017-01-01 22:02:03 2017-01-01 22:02:03 02/01/17 010203 MSK+0100 2017-01-01 21:02:03 2017-01-01 21:02:03 02/01/17 010203 UTC+0300 2017-01-01 22:02:03 2017-01-01 22:02:03 -020117 010203 UTC+0300 1970-01-01 04:30:19 1970-01-01 04:30:19 02/01/17 010203Z 2017-01-02 01:02:03 2017-01-02 01:02:03 02/01/1970 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 02/01/70 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 diff --git a/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql b/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql index 35e0d248585..5f71efa1485 100644 --- a/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql +++ b/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql @@ -12,7 +12,6 @@ FROM '02/01/17 010203 MSK', '02/01/17 010203 MSK+0100', '02/01/17 010203 UTC+0300', -'020117 010203 UTC+0300', '02/01/17 010203Z', '02/01/1970 010203Z', '02/01/70 010203Z', From 29ad7f9fb1ad4756eca23c8b474400dfcdf91a7c Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 24 Dec 2018 16:19:24 +0300 Subject: [PATCH 070/230] Remove redundant changes and shift error codes --- dbms/src/Storages/MergeTree/MergeTreeReader.cpp | 9 ++------- dbms/src/Storages/MergeTree/MergeTreeReader.h | 4 ++-- .../00804_test_custom_compression_codecs.sql | 10 +++++----- 3 files changed, 9 insertions(+), 14 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index b8a031655c3..537a4d82722 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -8,7 +8,6 @@ #include #include #include -#include namespace DB @@ -49,12 +48,8 @@ MergeTreeReader::MergeTreeReader(const String & path, if (!Poco::File(path).exists()) throw Exception("Part " + path + " is missing", ErrorCodes::NOT_FOUND_EXPECTED_DATA_PART); - const auto columns_desc = storage.getColumns(); - for (const NameAndTypePair & column : columns) - { addStreams(column.name, *column.type, all_mark_ranges, profile_callback, clock_type); - } } catch (...) { @@ -358,8 +353,8 @@ void MergeTreeReader::Stream::seekToStart() } -void MergeTreeReader::addStreams(const String & name, const IDataType & type, - const MarkRanges & all_mark_ranges, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type) +void MergeTreeReader::addStreams(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges, + const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type) { IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path) { diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index 85b627802fd..00973592c9e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -121,8 +121,8 @@ private: size_t max_read_buffer_size; size_t index_granularity; - void addStreams(const String & name, const IDataType & type, - const MarkRanges & all_mark_ranges, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); + void addStreams(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges, + const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); void readData( const String & name, const IDataType & type, IColumn & column, diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 00d3afbb5cc..0f2e813a64b 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -25,12 +25,12 @@ DROP TABLE IF EXISTS test.too_many_params; DROP TABLE IF EXISTS test.codec_multiple_direct_specification_1; DROP TABLE IF EXISTS test.codec_multiple_direct_specification_2; -CREATE TABLE test.bad_codec(id UInt64 CODEC(adssadads)) ENGINE = MergeTree() order by tuple(); -- { serverError 430 } -CREATE TABLE test.too_many_params(id UInt64 CODEC(ZSTD(2,3,4,5))) ENGINE = MergeTree() order by tuple(); -- { serverError 429 } +CREATE TABLE test.bad_codec(id UInt64 CODEC(adssadads)) ENGINE = MergeTree() order by tuple(); -- { serverError 432 } +CREATE TABLE test.too_many_params(id UInt64 CODEC(ZSTD(2,3,4,5))) ENGINE = MergeTree() order by tuple(); -- { serverError 431 } CREATE TABLE test.params_when_no_params(id UInt64 CODEC(LZ4(1))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 378 } -CREATE TABLE test.params_zstd(id UInt64 CODEC(LZ4HC(0))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 431 } -CREATE TABLE test.codec_multiple_direct_specification_1(id UInt64 CODEC(MULTIPLE(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 430 } -CREATE TABLE test.codec_multiple_direct_specification_2(id UInt64 CODEC(multiple(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 430 } +CREATE TABLE test.params_zstd(id UInt64 CODEC(LZ4HC(0))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 433 } +CREATE TABLE test.codec_multiple_direct_specification_1(id UInt64 CODEC(MULTIPLE(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 432 } +CREATE TABLE test.codec_multiple_direct_specification_2(id UInt64 CODEC(multiple(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 432 } DROP TABLE IF EXISTS test.bad_codec; DROP TABLE IF EXISTS test.params_when_no_params; From ce1067410718ca9c818d6b5705a875addcaee5b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 16:25:48 +0300 Subject: [PATCH 071/230] Fixed weird error in PR #3870 --- dbms/src/Functions/FunctionsConversion.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 0e0bf218db3..5ca141a64a0 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -1161,6 +1161,10 @@ struct ToIntMonotonicity checkAndGetDataType>(&type)) return { true, true, true }; + /// In other cases, if range is unbounded, we don't know, whether function is monotonic or not. + if (left.isNull() || right.isNull()) + return {}; + /// If converting from float, for monotonicity, arguments must fit in range of result type. if (WhichDataType(type).isFloat()) { From c07b0659fdd34d52949b6b3488f91bdf17d902c4 Mon Sep 17 00:00:00 2001 From: mf5137 Date: Mon, 24 Dec 2018 15:09:54 +0100 Subject: [PATCH 072/230] Performance tests for hash functions2 --- .../cryptographic_hashes.xml | 50 +++++++++++++++++ .../general_purpose_hashes.xml | 55 +++++++++++++++++++ 2 files changed, 105 insertions(+) create mode 100644 dbms/tests/performance/functions_hashing/cryptographic_hashes.xml create mode 100644 dbms/tests/performance/functions_hashing/general_purpose_hashes.xml diff --git a/dbms/tests/performance/functions_hashing/cryptographic_hashes.xml b/dbms/tests/performance/functions_hashing/cryptographic_hashes.xml new file mode 100644 index 00000000000..5dffe4e0cec --- /dev/null +++ b/dbms/tests/performance/functions_hashing/cryptographic_hashes.xml @@ -0,0 +1,50 @@ + + cryptographic_hashes + once + + + + 10000 + + + 5000 + 20000 + + + + + + + + + + crypto_hash_func + + MD5 + SHA1 + SHA224 + SHA256 + halfMD5 + sipHash64 + sipHash128 + + + + string + + materialize('') + toString(1000000000+number) + materialize('Lorem ipsum dolor sit amet, consectetur adipiscing elit. Mauris sollicitudin nisi ac erat mollis dapibus. Maecenas leo purus, bibendum eu erat eget, iaculis molestie tortor. Phasellus maximus odio nec mauris ultrices dictum. Morbi efficitur nisl eget congue mollis. Vestibulum pharetra diam vitae urna interdum, eget ultricies justo sollicitudin. Nunc sit amet purus id leo tempus dignissim. Donec ac lacus ut orci tempus scelerisque quis ultricies nibh. Nullam lobortis, erat ac ullamcorper interdum, odio nisl elementum quam, ut malesuada massa nunc eget quam. Nam suscipit neque quis sapien ultricies imperdiet. Maecenas augue libero, finibus tristique sagittis et, semper nec arcu. Morbi non tortor ultrices, sollicitudin justo sed, accumsan ligula. Nullam at ipsum in nibh auctor ullamcorper. Nullam laoreet neque id lorem condimentum tincidunt. Nullam vel orci nibh. Ut sit amet sem faucibus, fringilla orci at, lacinia enim. Mauris imperdiet ex id scelerisque eleifend. Ut tincidunt massa nibh, viverra pharetra metus') + + + + table + + numbers + numbers_mt + + + + + SELECT ignore({crypto_hash_func}({string})) FROM system.{table} LIMIT 10000000 + diff --git a/dbms/tests/performance/functions_hashing/general_purpose_hashes.xml b/dbms/tests/performance/functions_hashing/general_purpose_hashes.xml new file mode 100644 index 00000000000..3469fcc4969 --- /dev/null +++ b/dbms/tests/performance/functions_hashing/general_purpose_hashes.xml @@ -0,0 +1,55 @@ + + general_purpose_hashes + once + + + + 10000 + + + 3000 + 20000 + + + + + + + + + + gp_hash_func + + cityHash64 + farmHash64 + metroHash64 + murmurHash2_32 + murmurHash2_64 + murmurHash3_32 + murmurHash3_64 + murmurHash3_128 + javaHash + hiveHash + xxHash32 + xxHash64 + + + + string + + materialize('') + toString(1000000000+number) + materialize('Lorem ipsum dolor sit amet, consectetur adipiscing elit. Mauris sollicitudin nisi ac erat mollis dapibus. Maecenas leo purus, bibendum eu erat eget, iaculis molestie tortor. Phasellus maximus odio nec mauris ultrices dictum. Morbi efficitur nisl eget congue mollis. Vestibulum pharetra diam vitae urna interdum, eget ultricies justo sollicitudin. Nunc sit amet purus id leo tempus dignissim. Donec ac lacus ut orci tempus scelerisque quis ultricies nibh. Nullam lobortis, erat ac ullamcorper interdum, odio nisl elementum quam, ut malesuada massa nunc eget quam. Nam suscipit neque quis sapien ultricies imperdiet. Maecenas augue libero, finibus tristique sagittis et, semper nec arcu. Morbi non tortor ultrices, sollicitudin justo sed, accumsan ligula. Nullam at ipsum in nibh auctor ullamcorper. Nullam laoreet neque id lorem condimentum tincidunt. Nullam vel orci nibh. Ut sit amet sem faucibus, fringilla orci at, lacinia enim. Mauris imperdiet ex id scelerisque eleifend. Ut tincidunt massa nibh, viverra pharetra metus') + + + + table + + numbers + numbers_mt + + + + + SELECT ignore({gp_hash_func}({string})) FROM system.{table} LIMIT 10000000 + From f2fd1e4da6bc7e3adb41d16f15fa899dcf6fe6b9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 24 Dec 2018 17:10:37 +0300 Subject: [PATCH 073/230] Fix style and includes --- dbms/src/Compression/CompressionCodecLZ4.cpp | 10 +++------- dbms/src/Compression/CompressionCodecMultiple.cpp | 3 ++- dbms/src/Compression/CompressionCodecNone.cpp | 3 ++- dbms/src/Compression/CompressionFactory.h | 1 + 4 files changed, 8 insertions(+), 9 deletions(-) diff --git a/dbms/src/Compression/CompressionCodecLZ4.cpp b/dbms/src/Compression/CompressionCodecLZ4.cpp index f5d63575720..a2f9262ae06 100644 --- a/dbms/src/Compression/CompressionCodecLZ4.cpp +++ b/dbms/src/Compression/CompressionCodecLZ4.cpp @@ -48,7 +48,8 @@ void CompressionCodecLZ4::doDecompressData(const char * source, UInt32 source_si void registerCodecLZ4(CompressionCodecFactory & factory) { - factory.registerSimpleCompressionCodec("LZ4", static_cast(CompressionMethodByte::LZ4), [&] () { + factory.registerSimpleCompressionCodec("LZ4", static_cast(CompressionMethodByte::LZ4), [&] () + { return std::make_shared(); }); } @@ -73,7 +74,7 @@ void registerCodecLZ4HC(CompressionCodecFactory & factory) { factory.registerCompressionCodec("LZ4HC", {}, [&](const ASTPtr & arguments) -> CompressionCodecPtr { - int level = LZ4HC_CLEVEL_DEFAULT; + int level = 0; if (arguments && !arguments->children.empty()) { @@ -83,11 +84,6 @@ void registerCodecLZ4HC(CompressionCodecFactory & factory) const auto children = arguments->children; const ASTLiteral * literal = static_cast(children[0].get()); level = literal->value.safeGet(); - if (level > LZ4HC_CLEVEL_MAX || level < LZ4HC_CLEVEL_MIN) - throw Exception("LZ4HC codec can't have level more than " - + std::to_string(LZ4HC_CLEVEL_MAX) + " and less than " - + std::to_string(LZ4HC_CLEVEL_MIN) + ", given " - + std::to_string(level), ErrorCodes::ILLEGAL_CODEC_PARAMETER); } return std::make_shared(level); diff --git a/dbms/src/Compression/CompressionCodecMultiple.cpp b/dbms/src/Compression/CompressionCodecMultiple.cpp index 93a38d43a84..c7418592e3b 100644 --- a/dbms/src/Compression/CompressionCodecMultiple.cpp +++ b/dbms/src/Compression/CompressionCodecMultiple.cpp @@ -106,7 +106,8 @@ void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 sour void registerCodecMultiple(CompressionCodecFactory & factory) { - factory.registerSimpleCompressionCodec("Multiple", static_cast(CompressionMethodByte::Multiple), [&](){ + factory.registerSimpleCompressionCodec("Multiple", static_cast(CompressionMethodByte::Multiple), [&] () + { return std::make_shared(); }); } diff --git a/dbms/src/Compression/CompressionCodecNone.cpp b/dbms/src/Compression/CompressionCodecNone.cpp index fbb0e9cd031..7c5a4628539 100644 --- a/dbms/src/Compression/CompressionCodecNone.cpp +++ b/dbms/src/Compression/CompressionCodecNone.cpp @@ -29,7 +29,8 @@ void CompressionCodecNone::doDecompressData(const char * source, UInt32 /*source void registerCodecNone(CompressionCodecFactory & factory) { - factory.registerSimpleCompressionCodec("NONE", static_cast(CompressionMethodByte::NONE), [&](){ + factory.registerSimpleCompressionCodec("NONE", static_cast(CompressionMethodByte::NONE), [&] () + { return std::make_shared(); }); } diff --git a/dbms/src/Compression/CompressionFactory.h b/dbms/src/Compression/CompressionFactory.h index 1aef17de628..e7daf9648b8 100644 --- a/dbms/src/Compression/CompressionFactory.h +++ b/dbms/src/Compression/CompressionFactory.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include From b624add2e782ee7110c9575c917e78615c072f00 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 18:46:55 +0300 Subject: [PATCH 074/230] Rewrite code to calculate integer conversion function monotonicity; fixed test #3870 --- dbms/src/Functions/FunctionsConversion.h | 107 +++++++++++++----- .../00653_verification_monotonic_data_load.sh | 4 +- 2 files changed, 80 insertions(+), 31 deletions(-) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 5ca141a64a0..2762d9271ee 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -1142,32 +1142,24 @@ struct ToIntMonotonicity if (!type.isValueRepresentedByNumber()) return {}; - size_t size_of_type = type.getSizeOfValueInMemory(); - - /// If type is expanding - if (sizeof(T) > size_of_type) - { - /// If convert signed -> signed or unsigned -> signed, then function is monotonic. - if (std::is_signed_v || type.isValueRepresentedByUnsignedInteger()) - return {true, true, true}; - - /// If arguments from the same half, then function is monotonic. - if ((left.get() >= 0) == (right.get() >= 0)) - return {true, true, true}; - } - - /// If type is same, too. (Enum has separate case, because it is different data type) + /// If type is same, the conversion is always monotonic. + /// (Enum has separate case, because it is different data type) if (checkAndGetDataType>(&type) || checkAndGetDataType>(&type)) return { true, true, true }; - /// In other cases, if range is unbounded, we don't know, whether function is monotonic or not. - if (left.isNull() || right.isNull()) - return {}; + /// Float cases. - /// If converting from float, for monotonicity, arguments must fit in range of result type. + /// When converting to Float, the conversion is always monotonic. + if (std::is_floating_point_v) + return {true, true, true}; + + /// If converting from Float, for monotonicity, arguments must fit in range of result type. if (WhichDataType(type).isFloat()) { + if (left.isNull() || right.isNull()) + return {}; + Float64 left_float = left.get(); Float64 right_float = right.get(); @@ -1178,18 +1170,75 @@ struct ToIntMonotonicity return {}; } - /// If signedness of type is changing, or converting from Date, DateTime, then arguments must be from same half, - /// and after conversion, resulting values must be from same half. - /// Just in case, it is required in rest of cases too. - if ((left.get() >= 0) != (right.get() >= 0) - || (T(left.get()) >= 0) != (T(right.get()) >= 0)) - return {}; + /// Integer cases. - /// If type is shrinked, then for monotonicity, all bits other than that fits, must be same. - if (divideByRangeOfType(left.get()) != divideByRangeOfType(right.get())) - return {}; + bool from_is_unsigned = type.isValueRepresentedByUnsignedInteger(); + bool to_is_unsigned = std::is_unsigned_v; - return { true }; + size_t size_of_from = type.getSizeOfValueInMemory(); + size_t size_of_to = sizeof(T); + + bool left_in_first_half = left.isNull() + ? from_is_unsigned + : left.get() >= 0; + + bool right_in_first_half = right.isNull() + ? !from_is_unsigned + : right.get() >= 0; + + /// Size of type is the same. + if (size_of_from == size_of_to) + { + if (from_is_unsigned == to_is_unsigned) + return {true, true, true}; + + if (left_in_first_half == right_in_first_half) + return {true}; + + return {}; + } + + /// Size of type is expanded. + if (size_of_from < size_of_to) + { + if (from_is_unsigned == to_is_unsigned) + return {true, true, true}; + + if (!to_is_unsigned) + return {true, true, true}; + + /// signed -> unsigned. If arguments from the same half, then function is monotonic. + if (left_in_first_half == right_in_first_half) + return {true}; + } + + /// Size of type is shrinked. + if (size_of_from > size_of_to) + { + /// Function cannot be monotonic on unbounded ranges. + if (left.isNull() || right.isNull()) + return {}; + + if (from_is_unsigned == to_is_unsigned) + { + /// all bits other than that fits, must be same. + if (divideByRangeOfType(left.get()) == divideByRangeOfType(right.get())) + return {true}; + + return {}; + } + else + { + /// When signedness is changed, it's also required for arguments to be from the same half. + if (left_in_first_half == right_in_first_half + && divideByRangeOfType(left.get()) == divideByRangeOfType(right.get())) + return {true}; + + return {}; + } + } + + __builtin_unreachable(); } }; diff --git a/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh b/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh index 325f19dc9ec..c0a81f16f10 100755 --- a/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh +++ b/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh @@ -52,7 +52,7 @@ ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.signed_integer_test_table ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> Int64" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> UInt64" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> Int32" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: UInt32 -> Int32" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> UInt32" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "monotonic int case: UInt32 -> Int16" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "monotonic int case: UInt32 -> UInt16" @@ -68,7 +68,7 @@ ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.enum_test_table WHERE toU ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> Int32" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> UInt32" -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> Int16" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Date -> Int16" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> UInt16" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Date -> Int8" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Date -> UInt8" From 36fb2d16cd9217c0a68403af6fea658689908574 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 24 Dec 2018 18:50:02 +0300 Subject: [PATCH 075/230] Add test with non default config --- dbms/CMakeLists.txt | 3 - dbms/src/Compression/CompressionFactory.cpp | 7 +- dbms/src/Storages/CompressionCodecSelector.h | 2 +- .../test_non_default_compression/__init__.py | 0 .../configs/compression_config.xml | 10 +++ .../configs/custom_compression_by_default.xml | 18 +++++ .../configs/lz4hc_compression_by_default.xml | 12 ++++ .../configs/zstd_compression_by_default.xml | 12 ++++ .../test_non_default_compression/test.py | 70 +++++++++++++++++++ .../00804_test_custom_compression_codecs.sql | 3 - 10 files changed, 126 insertions(+), 11 deletions(-) create mode 100644 dbms/tests/integration/test_non_default_compression/__init__.py create mode 100644 dbms/tests/integration/test_non_default_compression/configs/compression_config.xml create mode 100644 dbms/tests/integration/test_non_default_compression/configs/custom_compression_by_default.xml create mode 100644 dbms/tests/integration/test_non_default_compression/configs/lz4hc_compression_by_default.xml create mode 100644 dbms/tests/integration/test_non_default_compression/configs/zstd_compression_by_default.xml create mode 100644 dbms/tests/integration/test_non_default_compression/test.py diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index f78bebade2f..28230c72def 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -179,9 +179,6 @@ target_link_libraries (clickhouse_common_io target_link_libraries (dbms PRIVATE - ${LZ4_LIBRARY} - ${ZSTD_LIBRARY} - PRIVATE clickhouse_parsers clickhouse_common_config PUBLIC diff --git a/dbms/src/Compression/CompressionFactory.cpp b/dbms/src/Compression/CompressionFactory.cpp index 53637b2259c..c70917e0cb1 100644 --- a/dbms/src/Compression/CompressionFactory.cpp +++ b/dbms/src/Compression/CompressionFactory.cpp @@ -34,13 +34,12 @@ CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std { if (level) { - auto identifier = std::make_shared(family_name); - auto literal = std::make_shared(*level); - return get(makeASTFunction("CODEC", identifier, literal)); + auto literal = std::make_shared(static_cast(*level)); + return get(makeASTFunction("CODEC", makeASTFunction(Poco::toUpper(family_name), literal))); } else { - auto identifier = std::make_shared(family_name); + auto identifier = std::make_shared(Poco::toUpper(family_name)); return get(makeASTFunction("CODEC", identifier)); } } diff --git a/dbms/src/Storages/CompressionCodecSelector.h b/dbms/src/Storages/CompressionCodecSelector.h index d005a249a1a..4c7ad8ba973 100644 --- a/dbms/src/Storages/CompressionCodecSelector.h +++ b/dbms/src/Storages/CompressionCodecSelector.h @@ -28,7 +28,7 @@ namespace ErrorCodes 10000000000 0.01 - + zstd 2 diff --git a/dbms/tests/integration/test_non_default_compression/__init__.py b/dbms/tests/integration/test_non_default_compression/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_non_default_compression/configs/compression_config.xml b/dbms/tests/integration/test_non_default_compression/configs/compression_config.xml new file mode 100644 index 00000000000..71c92738b9c --- /dev/null +++ b/dbms/tests/integration/test_non_default_compression/configs/compression_config.xml @@ -0,0 +1,10 @@ + + + + 0 + 0 + + zstd + 2 + + diff --git a/dbms/tests/integration/test_non_default_compression/configs/custom_compression_by_default.xml b/dbms/tests/integration/test_non_default_compression/configs/custom_compression_by_default.xml new file mode 100644 index 00000000000..5c0216db3e0 --- /dev/null +++ b/dbms/tests/integration/test_non_default_compression/configs/custom_compression_by_default.xml @@ -0,0 +1,18 @@ + + + + + 10000 + + lz4hc + 10 + + + + 100000 + + zstd + 20 + + + diff --git a/dbms/tests/integration/test_non_default_compression/configs/lz4hc_compression_by_default.xml b/dbms/tests/integration/test_non_default_compression/configs/lz4hc_compression_by_default.xml new file mode 100644 index 00000000000..d4bc426b1ac --- /dev/null +++ b/dbms/tests/integration/test_non_default_compression/configs/lz4hc_compression_by_default.xml @@ -0,0 +1,12 @@ + + + + + 0 + 0 + + lz4hc + 10 + + + diff --git a/dbms/tests/integration/test_non_default_compression/configs/zstd_compression_by_default.xml b/dbms/tests/integration/test_non_default_compression/configs/zstd_compression_by_default.xml new file mode 100644 index 00000000000..27a55b5cf08 --- /dev/null +++ b/dbms/tests/integration/test_non_default_compression/configs/zstd_compression_by_default.xml @@ -0,0 +1,12 @@ + + + + + 0 + 0 + + zstd + 2 + + + diff --git a/dbms/tests/integration/test_non_default_compression/test.py b/dbms/tests/integration/test_non_default_compression/test.py new file mode 100644 index 00000000000..5c4ff833b52 --- /dev/null +++ b/dbms/tests/integration/test_non_default_compression/test.py @@ -0,0 +1,70 @@ +import time +import pytest +import string +import random + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1', main_configs=['configs/zstd_compression_by_default.xml']) +node2 = cluster.add_instance('node2', main_configs=['configs/lz4hc_compression_by_default.xml']) +node3 = cluster.add_instance('node3', main_configs=['configs/custom_compression_by_default.xml']) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_preconfigured_default_codec(start_cluster): + for node in [node1, node2]: + node.query(""" + CREATE TABLE compression_codec_multiple_with_key ( + somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12)), + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC), + data String CODEC(ZSTD(2), LZ4HC, NONE, LZ4, LZ4), + somecolumn Float64 + ) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; + """) + node.query("INSERT INTO compression_codec_multiple_with_key VALUES(toDate('2018-10-12'), 100000, 'hello', 88.88), (toDate('2018-10-12'), 100002, 'world', 99.99), (toDate('2018-10-12'), 1111, '!', 777.777)") + assert node.query("SELECT COUNT(*) FROM compression_codec_multiple_with_key WHERE id % 2 == 0") == "2\n" + assert node.query("SELECT DISTINCT somecolumn FROM compression_codec_multiple_with_key ORDER BY id") == "777.777\n88.88\n99.99\n" + assert node.query("SELECT data FROM compression_codec_multiple_with_key WHERE id >= 1112 AND somedate = toDate('2018-10-12') AND somecolumn <= 100") == "hello\nworld\n" + + node.query("INSERT INTO compression_codec_multiple_with_key SELECT toDate('2018-10-12'), number, toString(number), 1.0 FROM system.numbers LIMIT 10000") + + assert node.query("SELECT COUNT(id) FROM compression_codec_multiple_with_key WHERE id % 10 == 0") == "1001\n" + assert node.query("SELECT SUM(somecolumn) FROM compression_codec_multiple_with_key") == str(777.777 + 88.88 + 99.99 + 1.0 * 10000) + "\n" + assert node.query("SELECT count(*) FROM compression_codec_multiple_with_key GROUP BY somedate") == "10003\n" + +def test_preconfigured_custom_codec(start_cluster): + node3.query(""" + CREATE TABLE compression_codec_multiple_with_key ( + somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12)), + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC), + data String, + somecolumn Float64 CODEC(ZSTD(2), LZ4HC, NONE, NONE, NONE, LZ4HC(5)) + ) ENGINE = MergeTree() PARTITION BY somedate ORDER BY id SETTINGS index_granularity = 2; + """) + + node3.query("INSERT INTO compression_codec_multiple_with_key VALUES(toDate('2018-10-12'), 100000, 'hello', 88.88), (toDate('2018-10-12'), 100002, 'world', 99.99), (toDate('2018-10-12'), 1111, '!', 777.777)") + assert node3.query("SELECT COUNT(*) FROM compression_codec_multiple_with_key WHERE id % 2 == 0") == "2\n" + assert node3.query("SELECT DISTINCT somecolumn FROM compression_codec_multiple_with_key ORDER BY id") == "777.777\n88.88\n99.99\n" + assert node3.query("SELECT data FROM compression_codec_multiple_with_key WHERE id >= 1112 AND somedate = toDate('2018-10-12') AND somecolumn <= 100") == "hello\nworld\n" + + node3.query("INSERT INTO compression_codec_multiple_with_key VALUES(toDate('2018-10-12'), 100000, '{}', 88.88)".format(''.join(random.choice(string.ascii_uppercase + string.digits) for _ in range(10000)))) + + node3.query("OPTIMIZE TABLE compression_codec_multiple_with_key FINAL") + assert node3.query("SELECT max(length(data)) from compression_codec_multiple_with_key GROUP BY data ORDER BY max(length(data)) DESC LIMIT 1") == "10000\n" + + for i in xrange(10): + node3.query("INSERT INTO compression_codec_multiple_with_key VALUES(toDate('2018-10-12'), {}, '{}', 88.88)".format(i, ''.join(random.choice(string.ascii_uppercase + string.digits) for _ in range(10000)))) + + node3.query("OPTIMIZE TABLE compression_codec_multiple_with_key FINAL") + + assert node3.query("SELECT COUNT(*) from compression_codec_multiple_with_key WHERE length(data) = 10000") == "11\n" diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 0f2e813a64b..355e85785c9 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -20,7 +20,6 @@ DROP TABLE IF EXISTS test.compression_codec; DROP TABLE IF EXISTS test.bad_codec; DROP TABLE IF EXISTS test.params_when_no_params; -DROP TABLE IF EXISTS test.params_zstd; DROP TABLE IF EXISTS test.too_many_params; DROP TABLE IF EXISTS test.codec_multiple_direct_specification_1; DROP TABLE IF EXISTS test.codec_multiple_direct_specification_2; @@ -28,13 +27,11 @@ DROP TABLE IF EXISTS test.codec_multiple_direct_specification_2; CREATE TABLE test.bad_codec(id UInt64 CODEC(adssadads)) ENGINE = MergeTree() order by tuple(); -- { serverError 432 } CREATE TABLE test.too_many_params(id UInt64 CODEC(ZSTD(2,3,4,5))) ENGINE = MergeTree() order by tuple(); -- { serverError 431 } CREATE TABLE test.params_when_no_params(id UInt64 CODEC(LZ4(1))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 378 } -CREATE TABLE test.params_zstd(id UInt64 CODEC(LZ4HC(0))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 433 } CREATE TABLE test.codec_multiple_direct_specification_1(id UInt64 CODEC(MULTIPLE(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 432 } CREATE TABLE test.codec_multiple_direct_specification_2(id UInt64 CODEC(multiple(LZ4, ZSTD))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 432 } DROP TABLE IF EXISTS test.bad_codec; DROP TABLE IF EXISTS test.params_when_no_params; -DROP TABLE IF EXISTS test.params_zstd; DROP TABLE IF EXISTS test.too_many_params; DROP TABLE IF EXISTS test.codec_multiple_direct_specification_1; DROP TABLE IF EXISTS test.codec_multiple_direct_specification_2; From 8736a3908f715c6e8920dcc86eca0f9340d7ab22 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 24 Dec 2018 22:26:38 +0800 Subject: [PATCH 076/230] left pad --- .../AggregateFunctionArray.h | 2 +- .../AggregateFunctionForEach.h | 2 +- .../AggregateFunctionGroupArray.h | 4 +- .../AggregateFunctionGroupUniqArray.h | 4 +- .../AggregateFunctionQuantile.h | 2 +- .../AggregateFunctionSumMap.h | 4 +- .../AggregateFunctionTopK.h | 4 +- dbms/src/Columns/ColumnArray.cpp | 10 ++-- dbms/src/Columns/ColumnArray.h | 4 +- dbms/src/Columns/ColumnString.cpp | 4 +- dbms/src/Columns/ColumnString.h | 6 +-- dbms/src/Columns/IColumnDummy.h | 2 +- dbms/src/Common/PODArray.cpp | 8 +++ dbms/src/Common/PODArray.h | 49 ++++++++++++------- dbms/src/DataTypes/DataTypeArray.cpp | 10 ++-- dbms/src/Functions/arrayReverse.cpp | 2 +- dbms/src/Interpreters/Aggregator.h | 2 +- dbms/src/Interpreters/Join.h | 2 +- dbms/src/Interpreters/SetVariants.h | 2 +- dbms/tests/performance/leftpad/leftpad.xml | 37 ++++++++++++++ 20 files changed, 108 insertions(+), 52 deletions(-) create mode 100644 dbms/src/Common/PODArray.cpp create mode 100644 dbms/tests/performance/leftpad/leftpad.xml diff --git a/dbms/src/AggregateFunctions/AggregateFunctionArray.h b/dbms/src/AggregateFunctions/AggregateFunctionArray.h index ccf5f787309..5dfebf13d52 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionArray.h @@ -85,7 +85,7 @@ public: const ColumnArray & first_array_column = static_cast(*columns[0]); const IColumn::Offsets & offsets = first_array_column.getOffsets(); - size_t begin = row_num == 0 ? 0 : offsets[row_num - 1]; + size_t begin = offsets[row_num - 1]; size_t end = offsets[row_num]; /// Sanity check. NOTE We can implement specialization for a case with single argument, if the check will hurt performance. diff --git a/dbms/src/AggregateFunctions/AggregateFunctionForEach.h b/dbms/src/AggregateFunctions/AggregateFunctionForEach.h index ddd94f8c918..79932c7744f 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionForEach.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionForEach.h @@ -146,7 +146,7 @@ public: const ColumnArray & first_array_column = static_cast(*columns[0]); const IColumn::Offsets & offsets = first_array_column.getOffsets(); - size_t begin = row_num == 0 ? 0 : offsets[row_num - 1]; + size_t begin = offsets[row_num - 1]; size_t end = offsets[row_num]; /// Sanity check. NOTE We can implement specialization for a case with single argument, if the check will hurt performance. diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h index c6a347aaf3b..094cae05615 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -119,7 +119,7 @@ public: ColumnArray & arr_to = static_cast(to); ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); - offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + size); + offsets_to.push_back(offsets_to.back() + size); typename ColumnVector::Container & data_to = static_cast &>(arr_to.getData()).getData(); data_to.insert(this->data(place).value.begin(), this->data(place).value.end()); @@ -370,7 +370,7 @@ public: auto & column_array = static_cast(to); auto & offsets = column_array.getOffsets(); - offsets.push_back((offsets.size() == 0 ? 0 : offsets.back()) + data(place).elems); + offsets.push_back(offsets.back() + data(place).elems); auto & column_data = column_array.getData(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h index e24f20b1769..9a96425a240 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h @@ -83,7 +83,7 @@ public: const typename State::Set & set = this->data(place).value; size_t size = set.size(); - offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + size); + offsets_to.push_back(offsets_to.back() + size); typename ColumnVector::Container & data_to = static_cast &>(arr_to.getData()).getData(); size_t old_size = data_to.size(); @@ -207,7 +207,7 @@ public: IColumn & data_to = arr_to.getData(); auto & set = this->data(place).value; - offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + set.size()); + offsets_to.push_back(offsets_to.back() + set.size()); for (auto & elem : set) { diff --git a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h index 21d871c8e79..3d420d13af0 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -138,7 +138,7 @@ public: ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); size_t size = levels.size(); - offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + size); + offsets_to.push_back(offsets_to.back() + size); if (!size) return; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index b79670eb3de..c3997b37192 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -83,7 +83,7 @@ public: const ColumnArray & array_column = static_cast(*columns[0]); const IColumn::Offsets & offsets = array_column.getOffsets(); const auto & keys_vec = static_cast(array_column.getData()); - const size_t keys_vec_offset = row_num == 0 ? 0 : offsets[row_num - 1]; + const size_t keys_vec_offset = offsets[row_num - 1]; const size_t keys_vec_size = (offsets[row_num] - keys_vec_offset); // Columns 1..n contain arrays of numeric values to sum @@ -93,7 +93,7 @@ public: Field value; const ColumnArray & array_column = static_cast(*columns[col + 1]); const IColumn::Offsets & offsets = array_column.getOffsets(); - const size_t values_vec_offset = row_num == 0 ? 0 : offsets[row_num - 1]; + const size_t values_vec_offset = offsets[row_num - 1]; const size_t values_vec_size = (offsets[row_num] - values_vec_offset); // Expect key and value arrays to be of same length diff --git a/dbms/src/AggregateFunctions/AggregateFunctionTopK.h b/dbms/src/AggregateFunctions/AggregateFunctionTopK.h index ac20f48dd12..4d1238a12cc 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionTopK.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionTopK.h @@ -93,7 +93,7 @@ public: auto result_vec = set.topK(threshold); size_t size = result_vec.size(); - offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + size); + offsets_to.push_back(offsets_to.back() + size); typename ColumnVector::Container & data_to = static_cast &>(arr_to.getData()).getData(); size_t old_size = data_to.size(); @@ -212,7 +212,7 @@ public: IColumn & data_to = arr_to.getData(); auto result_vec = this->data(place).value.topK(threshold); - offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + result_vec.size()); + offsets_to.push_back(offsets_to.back() + result_vec.size()); for (auto & elem : result_vec) { diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index eb497ea8f31..82830281373 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -166,7 +166,7 @@ void ColumnArray::insertData(const char * pos, size_t length) if (pos != end) throw Exception("Incorrect length argument for method ColumnArray::insertData", ErrorCodes::BAD_ARGUMENTS); - getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + elems); + getOffsets().push_back(getOffsets().back() + elems); } @@ -194,7 +194,7 @@ const char * ColumnArray::deserializeAndInsertFromArena(const char * pos) for (size_t i = 0; i < array_size; ++i) pos = getData().deserializeAndInsertFromArena(pos); - getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + array_size); + getOffsets().push_back(getOffsets().back() + array_size); return pos; } @@ -216,7 +216,7 @@ void ColumnArray::insert(const Field & x) size_t size = array.size(); for (size_t i = 0; i < size; ++i) getData().insert(array[i]); - getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + size); + getOffsets().push_back(getOffsets().back() + size); } @@ -227,13 +227,13 @@ void ColumnArray::insertFrom(const IColumn & src_, size_t n) size_t offset = src.offsetAt(n); getData().insertRangeFrom(src.getData(), offset, size); - getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + size); + getOffsets().push_back(getOffsets().back() + size); } void ColumnArray::insertDefault() { - getOffsets().push_back(getOffsets().size() == 0 ? 0 : getOffsets().back()); + getOffsets().push_back(getOffsets().back()); } diff --git a/dbms/src/Columns/ColumnArray.h b/dbms/src/Columns/ColumnArray.h index c2c17c17ed7..0caf6cbb87e 100644 --- a/dbms/src/Columns/ColumnArray.h +++ b/dbms/src/Columns/ColumnArray.h @@ -124,8 +124,8 @@ private: ColumnPtr data; ColumnPtr offsets; - size_t ALWAYS_INLINE offsetAt(size_t i) const { return i == 0 ? 0 : getOffsets()[i - 1]; } - size_t ALWAYS_INLINE sizeAt(size_t i) const { return i == 0 ? getOffsets()[0] : (getOffsets()[i] - getOffsets()[i - 1]); } + size_t ALWAYS_INLINE offsetAt(size_t i) const { return getOffsets()[i - 1]; } + size_t ALWAYS_INLINE sizeAt(size_t i) const { return getOffsets()[i] - getOffsets()[i - 1]; } /// Multiply values if the nested column is ColumnVector. diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index a92f62a2b08..86521e3dfb5 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -148,7 +148,7 @@ ColumnPtr ColumnString::permute(const Permutation & perm, size_t limit) const for (size_t i = 0; i < limit; ++i) { size_t j = perm[i]; - size_t string_offset = j == 0 ? 0 : offsets[j - 1]; + size_t string_offset = offsets[j - 1]; size_t string_size = offsets[j] - string_offset; memcpySmallAllowReadWriteOverflow15(&res_chars[current_new_offset], &chars[string_offset], string_size); @@ -219,7 +219,7 @@ ColumnPtr ColumnString::indexImpl(const PaddedPODArray & indexes, size_t l for (size_t i = 0; i < limit; ++i) { size_t j = indexes[i]; - size_t string_offset = j == 0 ? 0 : offsets[j - 1]; + size_t string_offset = offsets[j - 1]; size_t string_size = offsets[j] - string_offset; memcpySmallAllowReadWriteOverflow15(&res_chars[current_new_offset], &chars[string_offset], string_size); diff --git a/dbms/src/Columns/ColumnString.h b/dbms/src/Columns/ColumnString.h index 863db4f4eff..645f9dcac47 100644 --- a/dbms/src/Columns/ColumnString.h +++ b/dbms/src/Columns/ColumnString.h @@ -31,10 +31,10 @@ private: /// For convenience, every string ends with terminating zero byte. Note that strings could contain zero bytes in the middle. Chars chars; - size_t ALWAYS_INLINE offsetAt(size_t i) const { return i == 0 ? 0 : offsets[i - 1]; } + size_t ALWAYS_INLINE offsetAt(size_t i) const { return offsets[i - 1]; } /// Size of i-th element, including terminating zero. - size_t ALWAYS_INLINE sizeAt(size_t i) const { return i == 0 ? offsets[0] : (offsets[i] - offsets[i - 1]); } + size_t ALWAYS_INLINE sizeAt(size_t i) const { return offsets[i] - offsets[i - 1]; } template struct less; @@ -203,7 +203,7 @@ public: void insertDefault() override { chars.push_back(0); - offsets.push_back(offsets.size() == 0 ? 1 : (offsets.back() + 1)); + offsets.push_back(offsets.back() + 1); } int compareAt(size_t n, size_t m, const IColumn & rhs_, int /*nan_direction_hint*/) const override diff --git a/dbms/src/Columns/IColumnDummy.h b/dbms/src/Columns/IColumnDummy.h index 576b2d70459..beb0d101122 100644 --- a/dbms/src/Columns/IColumnDummy.h +++ b/dbms/src/Columns/IColumnDummy.h @@ -107,7 +107,7 @@ public: if (s != offsets.size()) throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - return cloneDummy(s == 0 ? 0 : offsets.back()); + return cloneDummy(offsets.back()); } MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override diff --git a/dbms/src/Common/PODArray.cpp b/dbms/src/Common/PODArray.cpp new file mode 100644 index 00000000000..95810551ac4 --- /dev/null +++ b/dbms/src/Common/PODArray.cpp @@ -0,0 +1,8 @@ +#include + +namespace DB +{ +/// Used for left padding of PODArray when empty +const char EmptyPODArray[EmptyPODArraySize]{}; + +} diff --git a/dbms/src/Common/PODArray.h b/dbms/src/Common/PODArray.h index a80dc74de76..03771f4a8ec 100644 --- a/dbms/src/Common/PODArray.h +++ b/dbms/src/Common/PODArray.h @@ -49,16 +49,22 @@ namespace DB * TODO Pass alignment to Allocator. * TODO Allow greater alignment than alignof(T). Example: array of char aligned to page size. */ -template , size_t pad_right_ = 0> +static constexpr size_t EmptyPODArraySize = 1024; +extern const char EmptyPODArray[EmptyPODArraySize]; +template , size_t pad_right_ = 0, size_t pad_left_ = 0> class PODArray : private boost::noncopyable, private TAllocator /// empty base optimization { protected: /// Round padding up to an whole number of elements to simplify arithmetic. static constexpr size_t pad_right = (pad_right_ + sizeof(T) - 1) / sizeof(T) * sizeof(T); + static constexpr size_t pad_left_unaligned = (pad_left_ + sizeof(T) - 1) / sizeof(T) * sizeof(T); + static constexpr size_t pad_left = pad_left_unaligned ? pad_left_unaligned + 15 - (pad_left_unaligned - 1) % 16 : 0; + static constexpr char * null = pad_left ? const_cast(EmptyPODArray) + EmptyPODArraySize : nullptr; + static_assert(pad_left <= EmptyPODArraySize && "Left Padding exceeds EmptyPODArraySize. Element size too large?"); - char * c_start = nullptr; - char * c_end = nullptr; - char * c_end_of_storage = nullptr; /// Does not include pad_right. + char * c_start = null; /// Does not include pad_left. + char * c_end = null; + char * c_end_of_storage = null; /// Does not include pad_right. T * t_start() { return reinterpret_cast(c_start); } T * t_end() { return reinterpret_cast(c_end); } @@ -72,7 +78,7 @@ protected: static size_t byte_size(size_t num_elements) { return num_elements * sizeof(T); } /// Minimum amount of memory to allocate for num_elements, including padding. - static size_t minimum_memory_for_elements(size_t num_elements) { return byte_size(num_elements) + pad_right; } + static size_t minimum_memory_for_elements(size_t num_elements) { return byte_size(num_elements) + pad_right + pad_left; } void alloc_for_num_elements(size_t num_elements) { @@ -82,22 +88,24 @@ protected: template void alloc(size_t bytes, TAllocatorParams &&... allocator_params) { - c_start = c_end = reinterpret_cast(TAllocator::alloc(bytes, std::forward(allocator_params)...)); - c_end_of_storage = c_start + bytes - pad_right; + c_start = c_end = reinterpret_cast(TAllocator::alloc(bytes, std::forward(allocator_params)...)) + pad_left; + c_end_of_storage = c_start + bytes - pad_right - pad_left; + if (pad_left) + *(t_start() - 1) = {}; } void dealloc() { - if (c_start == nullptr) + if (c_start == null) return; - TAllocator::free(c_start, allocated_bytes()); + TAllocator::free(c_start - pad_left, allocated_bytes()); } template void realloc(size_t bytes, TAllocatorParams &&... allocator_params) { - if (c_start == nullptr) + if (c_start == null) { alloc(bytes, std::forward(allocator_params)...); return; @@ -105,15 +113,18 @@ protected: ptrdiff_t end_diff = c_end - c_start; - c_start = reinterpret_cast(TAllocator::realloc(c_start, allocated_bytes(), bytes, std::forward(allocator_params)...)); - + c_start = reinterpret_cast( + TAllocator::realloc(c_start - pad_left, allocated_bytes(), bytes, std::forward(allocator_params)...)) + + pad_left; c_end = c_start + end_diff; - c_end_of_storage = c_start + bytes - pad_right; + c_end_of_storage = c_start + bytes - pad_right - pad_left; + if (pad_left) + *(t_start() - 1) = {}; } bool isInitialized() const { - return (c_start != nullptr) && (c_end != nullptr) && (c_end_of_storage != nullptr); + return (c_start != null) && (c_end != null) && (c_end_of_storage != null); } bool isAllocatedFromStack() const @@ -139,7 +150,7 @@ protected: public: using value_type = T; - size_t allocated_bytes() const { return c_end_of_storage - c_start + pad_right; } + size_t allocated_bytes() const { return c_end_of_storage - c_start + pad_right + pad_left; } /// You can not just use `typedef`, because there is ambiguity for the constructors and `assign` functions. struct iterator : public boost::iterator_adaptor @@ -378,9 +389,9 @@ public: memcpy(dest.c_start, src.c_start, byte_size(src.size())); dest.c_end = dest.c_start + (src.c_end - src.c_start); - src.c_start = nullptr; - src.c_end = nullptr; - src.c_end_of_storage = nullptr; + src.c_start = null; + src.c_end = null; + src.c_end_of_storage = null; } else { @@ -504,7 +515,7 @@ void swap(PODArray & lhs, PODArray> -using PaddedPODArray = PODArray; +using PaddedPODArray = PODArray; inline constexpr size_t integerRound(size_t value, size_t dividend) diff --git a/dbms/src/DataTypes/DataTypeArray.cpp b/dbms/src/DataTypes/DataTypeArray.cpp index 1b739a040ae..3dd4db815f2 100644 --- a/dbms/src/DataTypes/DataTypeArray.cpp +++ b/dbms/src/DataTypes/DataTypeArray.cpp @@ -59,7 +59,7 @@ void DataTypeArray::serializeBinary(const IColumn & column, size_t row_num, Writ const ColumnArray & column_array = static_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); - size_t offset = row_num == 0 ? 0 : offsets[row_num - 1]; + size_t offset = offsets[row_num - 1]; size_t next_offset = offsets[row_num]; size_t size = next_offset - offset; @@ -113,7 +113,7 @@ namespace ? offset + limit : size; - ColumnArray::Offset prev_offset = offset == 0 ? 0 : offset_values[offset - 1]; + ColumnArray::Offset prev_offset = offset_values[offset - 1]; for (size_t i = offset; i < end; ++i) { ColumnArray::Offset current_offset = offset_values[i]; @@ -280,7 +280,7 @@ static void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffe const ColumnArray & column_array = static_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); - size_t offset = row_num == 0 ? 0 : offsets[row_num - 1]; + size_t offset = offsets[row_num - 1]; size_t next_offset = offsets[row_num]; const IColumn & nested_column = column_array.getData(); @@ -369,7 +369,7 @@ void DataTypeArray::serializeTextJSON(const IColumn & column, size_t row_num, Wr const ColumnArray & column_array = static_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); - size_t offset = row_num == 0 ? 0 : offsets[row_num - 1]; + size_t offset = offsets[row_num - 1]; size_t next_offset = offsets[row_num]; const IColumn & nested_column = column_array.getData(); @@ -396,7 +396,7 @@ void DataTypeArray::serializeTextXML(const IColumn & column, size_t row_num, Wri const ColumnArray & column_array = static_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); - size_t offset = row_num == 0 ? 0 : offsets[row_num - 1]; + size_t offset = offsets[row_num - 1]; size_t next_offset = offsets[row_num]; const IColumn & nested_column = column_array.getData(); diff --git a/dbms/src/Functions/arrayReverse.cpp b/dbms/src/Functions/arrayReverse.cpp index 19d294b97cd..c6a9248c2e4 100644 --- a/dbms/src/Functions/arrayReverse.cpp +++ b/dbms/src/Functions/arrayReverse.cpp @@ -225,7 +225,7 @@ bool FunctionArrayReverse::executeString(const IColumn & src_data, const ColumnA { size_t j_reversed = array_size - j - 1; - auto src_pos = src_array_prev_offset + j_reversed == 0 ? 0 : src_string_offsets[src_array_prev_offset + j_reversed - 1]; + auto src_pos = src_string_offsets[src_array_prev_offset + j_reversed - 1]; size_t string_size = src_string_offsets[src_array_prev_offset + j_reversed] - src_pos; memcpySmallAllowReadWriteOverflow15(&res_chars[res_string_prev_offset], &src_data[src_pos], string_size); diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index a52fe389549..f3abe782134 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -271,7 +271,7 @@ struct AggregationMethodString Arena & /*pool*/) const { return StringRef( - &(*chars)[i == 0 ? 0 : (*offsets)[i - 1]], + &(*chars)[(*offsets)[i - 1]], (i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1); } }; diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 059eb3de2f1..e7bc3b1dfef 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -75,7 +75,7 @@ struct JoinKeyGetterString const Sizes &) const { return StringRef( - &(*chars)[i == 0 ? 0 : (*offsets)[i - 1]], + &(*chars)[(*offsets)[i - 1]], (i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1); } diff --git a/dbms/src/Interpreters/SetVariants.h b/dbms/src/Interpreters/SetVariants.h index 0bb5cd6a32b..8023282a331 100644 --- a/dbms/src/Interpreters/SetVariants.h +++ b/dbms/src/Interpreters/SetVariants.h @@ -86,7 +86,7 @@ struct SetMethodString const Sizes &) const { return StringRef( - &(*chars)[i == 0 ? 0 : (*offsets)[i - 1]], + &(*chars)[(*offsets)[i - 1]], (i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1); } }; diff --git a/dbms/tests/performance/leftpad/leftpad.xml b/dbms/tests/performance/leftpad/leftpad.xml new file mode 100644 index 00000000000..0dcb09cbbb8 --- /dev/null +++ b/dbms/tests/performance/leftpad/leftpad.xml @@ -0,0 +1,37 @@ + + left pad test + + + string + + + + hashfile + + + loop + + + + 5 + 10000 + + + 50 + 60000 + + + + + + + + + + + + + + + + From f991232fe3103538ca984c4a61de0addb7d7b588 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 24 Dec 2018 18:52:12 +0300 Subject: [PATCH 077/230] Add custom network compression settings to test --- .../0_stateless/00804_test_custom_compression_codecs.sql | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 355e85785c9..0de1fa82410 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -38,6 +38,8 @@ DROP TABLE IF EXISTS test.codec_multiple_direct_specification_2; DROP TABLE IF EXISTS test.compression_codec_multiple; +SET network_compression_method = 'lz4hc'; + CREATE TABLE test.compression_codec_multiple ( id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC), data String CODEC(ZSTD(2), NONE, LZ4HC, LZ4, LZ4), @@ -78,6 +80,9 @@ SELECT * FROM test.compression_codec_multiple_more_types order by id; DROP TABLE IF EXISTS test.compression_codec_multiple_with_key; +SET network_compression_method = 'zstd'; +SET network_zstd_compression_level = 5; + CREATE TABLE test.compression_codec_multiple_with_key ( somedate Date CODEC(ZSTD, ZSTD, ZSTD(12), LZ4HC(12)), id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC), From f36c9a174728fc9720c05eb994177cd038693e6c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 18:56:38 +0300 Subject: [PATCH 078/230] Fixed test #3921 --- .../0_stateless/00653_verification_monotonic_data_load.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh b/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh index c0a81f16f10..db533844fe4 100755 --- a/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh +++ b/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh @@ -38,7 +38,7 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO test.unsigned_integer_test_table VALUE ${CLICKHOUSE_CLIENT} --query="INSERT INTO test.enum_test_table VALUES ('hello'), ('world'), ('world'), ('yandex'), ('clickhouse'), ('clickhouse');" ${CLICKHOUSE_CLIENT} --query="INSERT INTO test.date_test_table VALUES (1), (2), (2), (256), (257), (257);" -export CLICKHOUSE_CLIENT=`echo ${CLICKHOUSE_CLIENT} |sed 's/'"${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/debug/g'` +export CLICKHOUSE_CLIENT=`echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g'` ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: String -> UInt64" ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.fixed_string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: FixedString -> UInt64" From f7e06cf9acef34fb12df381f6740b10495cf965f Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 24 Dec 2018 19:03:22 +0300 Subject: [PATCH 079/230] Remove random changes --- dbms/src/Storages/MergeTree/MergeTreeReader.h | 3 +-- .../configs/compression_config.xml | 10 ---------- 2 files changed, 1 insertion(+), 12 deletions(-) delete mode 100644 dbms/tests/integration/test_non_default_compression/configs/compression_config.xml diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index 00973592c9e..988ca570ddd 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -8,12 +8,12 @@ #include #include -class CachedCompressedReadBuffer; namespace DB { class IDataType; +class CachedCompressedReadBuffer; /// Reads the data between pairs of marks in the same part. When reading consecutive ranges, avoids unnecessary seeks. /// When ranges are almost consecutive, seeks are fast because they are performed inside the buffer. @@ -90,7 +90,6 @@ private: bool save_marks_in_cache; MarkCache::MappedPtr marks; - std::unique_ptr cached_buffer; std::unique_ptr non_cached_buffer; }; diff --git a/dbms/tests/integration/test_non_default_compression/configs/compression_config.xml b/dbms/tests/integration/test_non_default_compression/configs/compression_config.xml deleted file mode 100644 index 71c92738b9c..00000000000 --- a/dbms/tests/integration/test_non_default_compression/configs/compression_config.xml +++ /dev/null @@ -1,10 +0,0 @@ - - - - 0 - 0 - - zstd - 2 - - From bbdfc77f210be0a9ed884f4b0ff49ac7ba339c06 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 24 Dec 2018 19:12:25 +0300 Subject: [PATCH 080/230] Update CachedCompressedReadBuffer.h --- dbms/src/IO/CachedCompressedReadBuffer.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/IO/CachedCompressedReadBuffer.h b/dbms/src/IO/CachedCompressedReadBuffer.h index 9135f0482ca..327ee8a0940 100644 --- a/dbms/src/IO/CachedCompressedReadBuffer.h +++ b/dbms/src/IO/CachedCompressedReadBuffer.h @@ -19,7 +19,7 @@ namespace DB * Disadvantages: * - in case you need to read a lot of data in a row, but of them only a part is cached, you have to do seek-and. */ -class CachedCompressedReadBuffer : public CompressedReadBufferBase, public ReadBuffer +class CachedCompressedReadBuffer : public CompressedReadBufferBase, public ReadBuffer { private: const std::string path; From 5e8035f16676ba2b187f791c58e49046408fc92a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 19:13:04 +0300 Subject: [PATCH 081/230] Fixed error #3921 --- dbms/src/Functions/FunctionsConversion.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 2762d9271ee..178df33853f 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -1230,7 +1230,9 @@ struct ToIntMonotonicity else { /// When signedness is changed, it's also required for arguments to be from the same half. + /// And they must be in the same half after converting to the result type. if (left_in_first_half == right_in_first_half + && (T(left.get()) >= 0) == (T(right.get()) >= 0) && divideByRangeOfType(left.get()) == divideByRangeOfType(right.get())) return {true}; From a82dfef89366efe3ea3b36018f1713b4a31d8bca Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 24 Dec 2018 20:12:14 +0300 Subject: [PATCH 082/230] Fix in parser --- dbms/src/Parsers/ExpressionElementParsers.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 59c3685c3d4..ad2031a1b51 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -306,6 +306,7 @@ bool ParserCodec::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ++pos; auto function_node = std::make_shared(); + function_node->name = "CODEC"; function_node->arguments = expr_list_args; function_node->children.push_back(function_node->arguments); From 66a0c1691db8f830402289dbae01108fd5b4e046 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 21:28:41 +0300 Subject: [PATCH 083/230] Added more formats to "parseDateTimeBestEffort" function [#CLICKHOUSE-2] --- dbms/src/IO/parseDateTimeBestEffort.cpp | 56 ++++++++++++++----- dbms/src/IO/parseDateTimeBestEffort.h | 2 +- ...parse_date_time_best_effort_more.reference | 21 +++++++ ...00813_parse_date_time_best_effort_more.sql | 28 ++++++++++ 4 files changed, 91 insertions(+), 16 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference create mode 100644 dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql diff --git a/dbms/src/IO/parseDateTimeBestEffort.cpp b/dbms/src/IO/parseDateTimeBestEffort.cpp index ff9b8d067a9..4aea4d621c2 100644 --- a/dbms/src/IO/parseDateTimeBestEffort.cpp +++ b/dbms/src/IO/parseDateTimeBestEffort.cpp @@ -86,6 +86,25 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date bool is_pm = false; + auto read_alpha_month = [&month] (const auto & alpha) + { + if (0 == strncasecmp(alpha, "Jan", 3)) month = 1; + else if (0 == strncasecmp(alpha, "Feb", 3)) month = 2; + else if (0 == strncasecmp(alpha, "Mar", 3)) month = 3; + else if (0 == strncasecmp(alpha, "Apr", 3)) month = 4; + else if (0 == strncasecmp(alpha, "May", 3)) month = 5; + else if (0 == strncasecmp(alpha, "Jun", 3)) month = 6; + else if (0 == strncasecmp(alpha, "Jul", 3)) month = 7; + else if (0 == strncasecmp(alpha, "Aug", 3)) month = 8; + else if (0 == strncasecmp(alpha, "Sep", 3)) month = 9; + else if (0 == strncasecmp(alpha, "Oct", 3)) month = 10; + else if (0 == strncasecmp(alpha, "Nov", 3)) month = 11; + else if (0 == strncasecmp(alpha, "Dec", 3)) month = 12; + else + return false; + return true; + }; + while (!in.eof()) { char digits[14]; @@ -205,6 +224,10 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date /// hh - only if already have day of month /// DD/MM/YYYY /// DD/MM/YY + /// DD.MM.YYYY + /// DD.MM.YY + /// DD-MM-YYYY + /// DD-MM-YY /// DD UInt8 hour_or_day_of_month = 0; @@ -244,7 +267,7 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date return on_error("Cannot read DateTime: unexpected number of decimal digits after hour and minute: " + toString(num_digits), ErrorCodes::CANNOT_PARSE_DATETIME); } } - else if (checkChar('/', in)) + else if (checkChar('/', in) || checkChar('.', in) || checkChar('-', in)) { if (day_of_month) return on_error("Cannot read DateTime: day of month is duplicated", ErrorCodes::CANNOT_PARSE_DATETIME); @@ -260,10 +283,23 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date readDecimalNumber<2>(month, digits); else if (num_digits == 1) readDecimalNumber<1>(month, digits); + else if (num_digits == 0) + { + /// Month in alphabetical form + + char alpha[9]; /// The longest month name: September + size_t num_alpha = readAlpha(alpha, sizeof(alpha), in); + + if (num_alpha < 3) + return on_error("Cannot read DateTime: unexpected number of alphabetical characters after day of month: " + toString(num_alpha), ErrorCodes::CANNOT_PARSE_DATETIME); + + if (!read_alpha_month(alpha)) + return on_error("Cannot read DateTime: alphabetical characters after day of month don't look like month: " + std::string(alpha, 3), ErrorCodes::CANNOT_PARSE_DATETIME); + } else return on_error("Cannot read DateTime: unexpected number of decimal digits after day of month: " + toString(num_digits), ErrorCodes::CANNOT_PARSE_DATETIME); - if (checkChar('/', in)) + if (checkChar('/', in) || checkChar('.', in) || checkChar('-', in)) { if (year) return on_error("Cannot read DateTime: year component is duplicated", ErrorCodes::CANNOT_PARSE_DATETIME); @@ -401,19 +437,9 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date { bool has_day_of_week = false; - if (0 == strncasecmp(alpha, "Jan", 3)) month = 1; - else if (0 == strncasecmp(alpha, "Feb", 3)) month = 2; - else if (0 == strncasecmp(alpha, "Mar", 3)) month = 3; - else if (0 == strncasecmp(alpha, "Apr", 3)) month = 4; - else if (0 == strncasecmp(alpha, "May", 3)) month = 5; - else if (0 == strncasecmp(alpha, "Jun", 3)) month = 6; - else if (0 == strncasecmp(alpha, "Jul", 3)) month = 7; - else if (0 == strncasecmp(alpha, "Aug", 3)) month = 8; - else if (0 == strncasecmp(alpha, "Sep", 3)) month = 9; - else if (0 == strncasecmp(alpha, "Oct", 3)) month = 10; - else if (0 == strncasecmp(alpha, "Nov", 3)) month = 11; - else if (0 == strncasecmp(alpha, "Dec", 3)) month = 12; - + if (read_alpha_month(alpha)) + { + } else if (0 == strncasecmp(alpha, "UTC", 3)) has_time_zone_offset = true; else if (0 == strncasecmp(alpha, "GMT", 3)) has_time_zone_offset = true; else if (0 == strncasecmp(alpha, "MSK", 3)) { has_time_zone_offset = true; time_zone_offset_hour = 3; } diff --git a/dbms/src/IO/parseDateTimeBestEffort.h b/dbms/src/IO/parseDateTimeBestEffort.h index 65c822ff331..dc416a750c5 100644 --- a/dbms/src/IO/parseDateTimeBestEffort.h +++ b/dbms/src/IO/parseDateTimeBestEffort.h @@ -34,7 +34,7 @@ class ReadBuffer; * YYYYMM - 6 digits is a year, month if year was not already read * hhmmss - 6 digits is a time if year was already read * - * .nnnnnnn - any number of digits after point is fractional part of second, if it is not YYYY.MM.DD + * .nnnnnnn - any number of digits after point is fractional part of second, if it is not YYYY.MM.DD or DD.MM.YYYY * * T - means that time will follow * diff --git a/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference new file mode 100644 index 00000000000..3ec7da1c47e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference @@ -0,0 +1,21 @@ +s a b + +24.12.2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +24-12-2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +24.12.18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24-12-18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24-Dec-18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24/DEC/18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24/DEC/2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +01-OCT-2015 2015-10-01 00:00:00 2015-10-01 00:00:00 +24.12.2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +24-12-2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +24.12.18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24-12-18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24-Dec-18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24/DEC/18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24/DEC/2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +01-OCT-2015 2015-10-01 00:00:00 2015-10-01 00:00:00 +24.12.18 010203 2018-12-24 01:02:03 2018-12-24 01:02:03 +24.12.18 01:02:03 2018-12-24 01:02:03 2018-12-24 01:02:03 +24.DEC.18T01:02:03.000+0300 2018-12-23 22:02:03 2018-12-23 22:02:03 diff --git a/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql new file mode 100644 index 00000000000..4a24d73f55a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql @@ -0,0 +1,28 @@ +SELECT + s, + parseDateTimeBestEffortOrNull(s, 'UTC') AS a, + parseDateTimeBestEffortOrZero(s, 'UTC') AS b +FROM +( + SELECT arrayJoin([ +'24.12.2018', +'24-12-2018', +'24.12.18', +'24-12-18', +'24-Dec-18', +'24/DEC/18', +'24/DEC/2018', +'01-OCT-2015', +'24.12.2018', +'24-12-2018', +'24.12.18', +'24-12-18', +'24-Dec-18', +'24/DEC/18', +'24/DEC/2018', +'01-OCT-2015', +'24.12.18 010203', +'24.12.18 01:02:03', +'24.DEC.18T01:02:03.000+0300' +]) AS s) +FORMAT PrettySpaceNoEscapes; From 41e3bf55180dc735dd1b7ad202501ae109bcad45 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 21:31:43 +0300 Subject: [PATCH 084/230] Added more formats to "parseDateTimeBestEffort" function [#CLICKHOUSE-2] --- .../00813_parse_date_time_best_effort_more.reference | 1 + .../0_stateless/00813_parse_date_time_best_effort_more.sql | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference index 3ec7da1c47e..921400838d0 100644 --- a/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference +++ b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference @@ -19,3 +19,4 @@ s a b 24.12.18 010203 2018-12-24 01:02:03 2018-12-24 01:02:03 24.12.18 01:02:03 2018-12-24 01:02:03 2018-12-24 01:02:03 24.DEC.18T01:02:03.000+0300 2018-12-23 22:02:03 2018-12-23 22:02:03 +01-September-2018 11:22 2018-09-01 11:22:00 2018-09-01 11:22:00 diff --git a/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql index 4a24d73f55a..1e3b24e60c0 100644 --- a/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql +++ b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql @@ -23,6 +23,7 @@ FROM '01-OCT-2015', '24.12.18 010203', '24.12.18 01:02:03', -'24.DEC.18T01:02:03.000+0300' +'24.DEC.18T01:02:03.000+0300', +'01-September-2018 11:22' ]) AS s) FORMAT PrettySpaceNoEscapes; From 37075ba1a3b689477927092eabb54ebdc2a5253e Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 25 Dec 2018 11:43:07 +0300 Subject: [PATCH 085/230] add lost else --- docs/tools/mdx_clickhouse.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/tools/mdx_clickhouse.py b/docs/tools/mdx_clickhouse.py index ae57d1309e6..a2d71b0ea58 100755 --- a/docs/tools/mdx_clickhouse.py +++ b/docs/tools/mdx_clickhouse.py @@ -47,6 +47,9 @@ class ClickHousePreprocessor(markdown.util.Processor): for line in lines: if '' not in line: yield line + else: + for line in lines: + yield line class ClickHouseMarkdown(markdown.extensions.Extension): From 4309775fc83ccb14c1cdde1a3896e9a337c6029c Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 25 Dec 2018 13:04:38 +0300 Subject: [PATCH 086/230] Add coments and fix compression for nested types --- dbms/src/Compression/CompressionFactory.h | 5 +++ dbms/src/Compression/CompressionInfo.h | 3 +- dbms/src/Compression/ICompressionCodec.h | 15 ++++++++- dbms/src/DataTypes/NestedUtils.cpp | 16 ++++++++- dbms/src/DataTypes/NestedUtils.h | 2 ++ .../Interpreters/InterpreterCreateQuery.cpp | 15 ++++++++- .../Interpreters/InterpreterDescribeQuery.cpp | 15 +++++++++ dbms/src/Storages/ColumnsDescription.cpp | 33 +++++++++++++++++++ ...4_test_custom_compression_codecs.reference | 1 + .../00804_test_custom_compression_codecs.sql | 2 ++ 10 files changed, 103 insertions(+), 4 deletions(-) diff --git a/dbms/src/Compression/CompressionFactory.h b/dbms/src/Compression/CompressionFactory.h index e7daf9648b8..e1019cb983a 100644 --- a/dbms/src/Compression/CompressionFactory.h +++ b/dbms/src/Compression/CompressionFactory.h @@ -31,17 +31,22 @@ protected: using CompressionCodecsCodeDictionary = std::unordered_map; public: + /// Return default codec (currently LZ4) CompressionCodecPtr getDefaultCodec() const; + /// Get codec by AST CompressionCodecPtr get(const ASTPtr & ast) const; + /// Get codec by method byte (no params available) CompressionCodecPtr get(const UInt8 byte_code) const; /// For backward compatibility with config settings CompressionCodecPtr get(const String & family_name, std::optional level) const; + /// Register codec with parameters void registerCompressionCodec(const String & family_name, std::optional byte_code, Creator creator); + /// Register codec without parameters void registerSimpleCompressionCodec(const String & family_name, std::optional byte_code, SimpleCreator creator); protected: diff --git a/dbms/src/Compression/CompressionInfo.h b/dbms/src/Compression/CompressionInfo.h index fc7d501a80c..b5aeef69ead 100644 --- a/dbms/src/Compression/CompressionInfo.h +++ b/dbms/src/Compression/CompressionInfo.h @@ -2,10 +2,11 @@ #include -/** Common Defines */ +/** Common defines for compression */ #define DBMS_MAX_COMPRESSED_SIZE 0x40000000ULL /// 1GB +/** one byte for method, 4 bytes for compressed size, 4 bytes for uncompressed size */ #define COMPRESSED_BLOCK_HEADER_SIZE 9 namespace DB diff --git a/dbms/src/Compression/ICompressionCodec.h b/dbms/src/Compression/ICompressionCodec.h index ef104d07eca..eaf877b479f 100644 --- a/dbms/src/Compression/ICompressionCodec.h +++ b/dbms/src/Compression/ICompressionCodec.h @@ -22,39 +22,52 @@ using CompressionCodecPtr = std::shared_ptr; using Codecs = std::vector; /** -* +* Represents interface for compression codecs like LZ4, ZSTD, etc. */ class ICompressionCodec : private boost::noncopyable { public: virtual ~ICompressionCodec() = default; + /// Byte which indicates codec in compressed file virtual UInt8 getMethodByte() const = 0; + /// Codec description, for example "ZSTD(2)" or "LZ4,LZ4HC(5)" virtual String getCodecDesc() const = 0; + /// Compressed bytes from uncompressed source to dest. Dest should preallocate memory virtual UInt32 compress(char * source, UInt32 source_size, char * dest) const; + /// Decompress bytes from compressed source to dest. Dest should preallocate memory virtual UInt32 decompress(char * source, UInt32 source_size, char * dest) const; + /// Number of bytes, that will be used to compress uncompressed_size bytes with current codec virtual UInt32 getCompressedReserveSize(UInt32 uncompressed_size) const { return getHeaderSize() + getCompressedDataSize(uncompressed_size); } + /// Some codecs (LZ4, for example) require additional bytes at end of buffer virtual UInt32 getAdditionalSizeAtTheEndOfBuffer() const { return 0; } + /// Size of header in compressed data on disk static UInt8 getHeaderSize() { return COMPRESSED_BLOCK_HEADER_SIZE; } + /// Read size of compressed block from compressed source static UInt32 readCompressedBlockSize(const char * source); + /// Read size of decompressed block from compressed source static UInt32 readDecompressedBlockSize(const char * source); + /// Read method byte from compressed source static UInt8 readMethod(const char * source); protected: + /// Return size of compressed data without header virtual UInt32 getCompressedDataSize(UInt32 uncompressed_size) const { return uncompressed_size; } + /// Actually compress data, without header virtual UInt32 doCompressData(const char * source, UInt32 source_size, char * dest) const = 0; + /// Actually decompress data without header virtual void doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const = 0; }; diff --git a/dbms/src/DataTypes/NestedUtils.cpp b/dbms/src/DataTypes/NestedUtils.cpp index 92d76fd474d..d9a2ba2dc94 100644 --- a/dbms/src/DataTypes/NestedUtils.cpp +++ b/dbms/src/DataTypes/NestedUtils.cpp @@ -78,6 +78,13 @@ std::string extractTableName(const std::string & nested_name) NamesAndTypesList flatten(const NamesAndTypesList & names_and_types) +{ + std::unordered_map> dummy; + return flattenWithMapping(names_and_types, dummy); +} + + +NamesAndTypesList flattenWithMapping(const NamesAndTypesList & names_and_types, std::unordered_map> & mapping) { NamesAndTypesList res; @@ -94,19 +101,26 @@ NamesAndTypesList flatten(const NamesAndTypesList & names_and_types) for (size_t i = 0; i < tuple_size; ++i) { String nested_name = concatenateName(name_type.name, names[i]); + mapping[name_type.name].push_back(nested_name); res.emplace_back(nested_name, std::make_shared(elements[i])); } } else + { + mapping[name_type.name].push_back(name_type.name); res.push_back(name_type); + } } else + { + mapping[name_type.name].push_back(name_type.name); res.push_back(name_type); + } } return res; -} +} Block flatten(const Block & block) { diff --git a/dbms/src/DataTypes/NestedUtils.h b/dbms/src/DataTypes/NestedUtils.h index 7a3a4b4f41f..0d1ed025281 100644 --- a/dbms/src/DataTypes/NestedUtils.h +++ b/dbms/src/DataTypes/NestedUtils.h @@ -19,6 +19,8 @@ namespace Nested /// Replace Array(Tuple(...)) columns to a multiple of Array columns in a form of `column_name.element_name`. NamesAndTypesList flatten(const NamesAndTypesList & names_and_types); Block flatten(const Block & block); + /// Works as normal flatten, but provide information about flattened columns + NamesAndTypesList flattenWithMapping(const NamesAndTypesList & names_and_types, std::unordered_map> & mapping); /// Collect Array columns in a form of `column_name.element_name` to single Array(Tuple(...)) column. NamesAndTypesList collect(const NamesAndTypesList & names_and_types); diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 1f90d8048c0..944bee63935 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -287,7 +287,20 @@ static ColumnsDeclarationAndModifiers parseColumns(const ASTExpressionList & col } } - return {Nested::flatten(columns), defaults, codecs, comments}; + std::unordered_map> mapping; + auto new_columns = Nested::flattenWithMapping(columns, mapping); + for (const auto & [old_name, new_names] : mapping) + { + if (new_names.size() == 1 && old_name == new_names.back()) + continue; + + auto codec = codecs[old_name]; + codecs.erase(old_name); + for (const auto & new_name : new_names) + codecs.emplace(new_name, codec); + } + + return {new_columns, defaults, codecs, comments}; } diff --git a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp index b5e62b9b92d..9aec49f27f1 100644 --- a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp @@ -48,6 +48,9 @@ Block InterpreterDescribeQuery::getSampleBlock() col.name = "comment_expression"; block.insert(col); + col.name = "codec_expression"; + block.insert(col); + return block; } @@ -59,6 +62,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() NamesAndTypesList columns; ColumnDefaults column_defaults; ColumnComments column_comments; + ColumnCodecs column_codecs; StoragePtr table; auto table_expression = typeid_cast(ast.table_expression.get()); @@ -106,6 +110,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() columns = table->getColumns().getAll(); column_defaults = table->getColumns().defaults; column_comments = table->getColumns().comments; + column_codecs = table->getColumns().codecs; } Block sample_block = getSampleBlock(); @@ -137,6 +142,16 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() { res_columns[4]->insert(comments_it->second); } + + const auto codecs_it = column_codecs.find(column.name); + if (codecs_it == std::end(column_codecs)) + { + res_columns[5]->insertDefault(); + } + else + { + res_columns[5]->insert(codecs_it->second->getCodecDesc()); + } } return std::make_shared(sample_block.cloneWithColumns(std::move(res_columns))); diff --git a/dbms/src/Storages/ColumnsDescription.cpp b/dbms/src/Storages/ColumnsDescription.cpp index b230cb0e294..2ef5fa07a9d 100644 --- a/dbms/src/Storages/ColumnsDescription.cpp +++ b/dbms/src/Storages/ColumnsDescription.cpp @@ -85,12 +85,14 @@ String ColumnsDescription::toString() const { const auto defaults_it = defaults.find(column.name); const auto comments_it = comments.find(column.name); + const auto codec_it = codecs.find(column.name); writeBackQuotedString(column.name, buf); writeChar(' ', buf); writeText(column.type->getName(), buf); const bool exist_comment = comments_it != std::end(comments); + const bool exist_codec = codec_it != std::end(codecs); if (defaults_it != std::end(defaults)) { writeChar('\t', buf); @@ -109,6 +111,14 @@ String ColumnsDescription::toString() const writeText(queryToString(ASTLiteral(Field(comments_it->second))), buf); } + if (exist_codec) + { + writeChar('\t', buf); + writeText("CODEC(", buf); + writeText(codec_it->second->getCodecDesc(), buf); + writeText(")", buf); + } + writeChar('\n', buf); } }; @@ -153,6 +163,23 @@ String parseComment(ReadBufferFromString& buf) return typeid_cast(*comment_expr).value.get(); } +CompressionCodecPtr parseCodec(ReadBufferFromString& buf) +{ + if (*buf.position() == '\n') + return {}; + + assertChar('\t', buf); + ParserCodec codec_parser; + String codec_expr_str; + readText(codec_expr_str, buf); + ASTPtr codec_expr = parseQuery(codec_parser, codec_expr_str, "codec expression", 0); + if (codec_expr) + return CompressionCodecFactory::instance().get(codec_expr); + else + return nullptr; +} + + void parseColumn(ReadBufferFromString & buf, ColumnsDescription & result, const DataTypeFactory & data_type_factory) { String column_name; @@ -187,6 +214,12 @@ void parseColumn(ReadBufferFromString & buf, ColumnsDescription & result, const result.defaults.emplace(column_name, std::move(*column_default)); } + auto codec = parseCodec(buf); + if (codec) + { + result.codecs.emplace(column_name, std::move(codec)); + } + auto comment = parseComment(buf); if (!comment.empty()) { diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference index ee64e2e922f..c5e6e9a809f 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.reference @@ -9,6 +9,7 @@ 10003 274972506.6 9175437371954010821 +CREATE TABLE test.compression_codec_multiple_more_types ( id Decimal(38, 13) CODEC(ZSTD, LZ4, ZSTD, ZSTD, LZ4HC), data FixedString(12) CODEC(ZSTD, ZSTD, NONE, NONE, NONE, LZ4HC), `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD), `ddd.Name` Array(String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD)) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192 1.5555555555555 hello world! [77] ['John'] 7.1000000000000 xxxxxxxxxxxx [127] ['Henry'] ! diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql index 0de1fa82410..cea8691f2fd 100644 --- a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codecs.sql @@ -73,6 +73,8 @@ CREATE TABLE test.compression_codec_multiple_more_types ( ddd Nested (age UInt8, Name String) CODEC(LZ4, LZ4HC, NONE, NONE, NONE, ZSTD) ) ENGINE = MergeTree() ORDER BY tuple(); +SHOW CREATE TABLE test.compression_codec_multiple_more_types; + INSERT INTO test.compression_codec_multiple_more_types VALUES(1.5555555555555, 'hello world!', [77], ['John']); INSERT INTO test.compression_codec_multiple_more_types VALUES(7.1, 'xxxxxxxxxxxx', [127], ['Henry']); From c91687f9d7448bf8903fff8a1d8be53ee9ffaad8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 25 Dec 2018 13:35:46 +0300 Subject: [PATCH 087/230] Small refactoring --- dbms/src/IO/CachedCompressedReadBuffer.cpp | 5 +++-- dbms/src/IO/CachedCompressedReadBuffer.h | 4 ++-- dbms/src/IO/CompressedReadBufferBase.cpp | 19 +++++++++++++++---- 3 files changed, 20 insertions(+), 8 deletions(-) diff --git a/dbms/src/IO/CachedCompressedReadBuffer.cpp b/dbms/src/IO/CachedCompressedReadBuffer.cpp index a2a5614ef43..50c97edf1a3 100644 --- a/dbms/src/IO/CachedCompressedReadBuffer.cpp +++ b/dbms/src/IO/CachedCompressedReadBuffer.cpp @@ -72,13 +72,14 @@ bool CachedCompressedReadBuffer::nextImpl() CachedCompressedReadBuffer::CachedCompressedReadBuffer( - const std::string & path_, UncompressedCache * cache_, - size_t estimated_size_, size_t aio_threshold_, size_t buf_size_) + const std::string & path_, UncompressedCache * cache_, size_t estimated_size_, size_t aio_threshold_, + size_t buf_size_) : ReadBuffer(nullptr, 0), path(path_), cache(cache_), buf_size(buf_size_), estimated_size(estimated_size_), aio_threshold(aio_threshold_), file_pos(0) { } + void CachedCompressedReadBuffer::seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block) { if (owned_cell && diff --git a/dbms/src/IO/CachedCompressedReadBuffer.h b/dbms/src/IO/CachedCompressedReadBuffer.h index 327ee8a0940..1b5e41972f3 100644 --- a/dbms/src/IO/CachedCompressedReadBuffer.h +++ b/dbms/src/IO/CachedCompressedReadBuffer.h @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB @@ -43,7 +42,8 @@ private: public: CachedCompressedReadBuffer( - const std::string & path_, UncompressedCache * cache_, size_t estimated_size_, size_t aio_threshold_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); + const std::string & path_, UncompressedCache * cache_, size_t estimated_size_, size_t aio_threshold_, + size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); diff --git a/dbms/src/IO/CompressedReadBufferBase.cpp b/dbms/src/IO/CompressedReadBufferBase.cpp index 1932daa6e30..5ac795a82e1 100644 --- a/dbms/src/IO/CompressedReadBufferBase.cpp +++ b/dbms/src/IO/CompressedReadBufferBase.cpp @@ -58,13 +58,20 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, if (!codec) codec = CompressionCodecFactory::instance().get(method); else if (method != codec->getMethodByte()) - throw Exception("Data compressed with different method, given method byte " + getHexUIntLowercase(method) + ", previous method byte " + getHexUIntLowercase(codec->getMethodByte()), ErrorCodes::CANNOT_DECOMPRESS); + throw Exception("Data compressed with different methods, given method byte " + + getHexUIntLowercase(method) + + ", previous method byte " + + getHexUIntLowercase(codec->getMethodByte()), + ErrorCodes::CANNOT_DECOMPRESS); size_compressed_without_checksum = ICompressionCodec::readCompressedBlockSize(own_compressed_buffer.data()); size_decompressed = ICompressionCodec::readDecompressedBlockSize(own_compressed_buffer.data()); if (size_compressed_without_checksum > DBMS_MAX_COMPRESSED_SIZE) - throw Exception("Too large size_compressed_without_checksum: " + toString(size_compressed_without_checksum) + ". Most likely corrupted data.", ErrorCodes::TOO_LARGE_SIZE_COMPRESSED); + throw Exception("Too large size_compressed_without_checksum: " + + toString(size_compressed_without_checksum) + + ". Most likely corrupted data.", + ErrorCodes::TOO_LARGE_SIZE_COMPRESSED); ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + CHECKSUM_SIZE); @@ -95,7 +102,7 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, } - return size_compressed_without_checksum + sizeof(checksum); + return size_compressed_without_checksum + CHECKSUM_SIZE; } @@ -109,7 +116,11 @@ void CompressedReadBufferBase::decompress(char * to, size_t size_decompressed, s if (!codec) codec = CompressionCodecFactory::instance().get(method); else if (codec->getMethodByte() != method) - throw Exception("Data compressed with different method, given method byte " + getHexUIntLowercase(method) + ", previous method byte " + getHexUIntLowercase(codec->getMethodByte()), ErrorCodes::CANNOT_DECOMPRESS); + throw Exception("Data compressed with different methods, given method byte " + + getHexUIntLowercase(method) + + ", previous method byte " + + getHexUIntLowercase(codec->getMethodByte()), + ErrorCodes::CANNOT_DECOMPRESS); codec->decompress(compressed_buffer, size_compressed_without_checksum, to); } From 1228645a7905eeb757fc7e10b9c9e2c9b3a023c7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 25 Dec 2018 14:33:54 +0300 Subject: [PATCH 088/230] Fix segfault and add tabs to ton of tests (desc table) --- .../Interpreters/InterpreterCreateQuery.cpp | 7 +- dbms/src/Storages/ColumnsDescription.cpp | 11 +- .../0_stateless/00030_alter_table.reference | 52 ++++----- .../00061_merge_tree_alter.reference | 106 +++++++++--------- .../00079_defaulted_columns.reference | 54 ++++----- .../00168_buffer_defaults.reference | 12 +- .../0_stateless/00415_into_outfile.reference | 2 +- .../00557_alter_null_storage_tables.reference | 6 +- .../queries/0_stateless/00642_cast.reference | 4 +- ...ult_database_on_client_reconnect.reference | 2 +- .../00700_decimal_defaults.reference | 12 +- .../00725_comment_columns.reference | 16 +-- .../00753_comment_columns_zookeeper.reference | 4 +- 13 files changed, 145 insertions(+), 143 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 944bee63935..c5d546d11ca 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -291,11 +291,12 @@ static ColumnsDeclarationAndModifiers parseColumns(const ASTExpressionList & col auto new_columns = Nested::flattenWithMapping(columns, mapping); for (const auto & [old_name, new_names] : mapping) { - if (new_names.size() == 1 && old_name == new_names.back()) + auto codec_it = codecs.find(old_name); + if ((new_names.size() == 1 && old_name == new_names.back()) || codec_it == codecs.end()) continue; - auto codec = codecs[old_name]; - codecs.erase(old_name); + auto codec = codec_it->second; + codecs.erase(codec_it); for (const auto & new_name : new_names) codecs.emplace(new_name, codec); } diff --git a/dbms/src/Storages/ColumnsDescription.cpp b/dbms/src/Storages/ColumnsDescription.cpp index 2ef5fa07a9d..c64f78b330d 100644 --- a/dbms/src/Storages/ColumnsDescription.cpp +++ b/dbms/src/Storages/ColumnsDescription.cpp @@ -214,17 +214,18 @@ void parseColumn(ReadBufferFromString & buf, ColumnsDescription & result, const result.defaults.emplace(column_name, std::move(*column_default)); } + auto comment = parseComment(buf); + if (!comment.empty()) + { + result.comments.emplace(column_name, std::move(comment)); + } + auto codec = parseCodec(buf); if (codec) { result.codecs.emplace(column_name, std::move(codec)); } - auto comment = parseComment(buf); - if (!comment.empty()) - { - result.comments.emplace(column_name, std::move(comment)); - } assertChar('\n', buf); } diff --git a/dbms/tests/queries/0_stateless/00030_alter_table.reference b/dbms/tests/queries/0_stateless/00030_alter_table.reference index 05022b485f7..24a54b6e2a0 100644 --- a/dbms/tests/queries/0_stateless/00030_alter_table.reference +++ b/dbms/tests/queries/0_stateless/00030_alter_table.reference @@ -1,27 +1,27 @@ -CounterID UInt32 -StartDate Date -UserID UInt32 -VisitID UInt32 -NestedColumn.A Array(UInt8) -NestedColumn.S Array(String) -ToDrop UInt32 -Added0 UInt32 -Added1 UInt32 -Added2 UInt32 -AddedNested1.A Array(UInt32) -AddedNested1.B Array(UInt64) -AddedNested1.C Array(String) -AddedNested2.A Array(UInt32) -AddedNested2.B Array(UInt64) -CounterID UInt32 -StartDate Date -UserID UInt32 -VisitID UInt32 -Added0 String -Added1 UInt32 -Added2 UInt32 -AddedNested1.A Array(UInt32) -AddedNested1.C Array(String) -AddedNested2.A Array(UInt32) -AddedNested2.B Array(UInt64) +CounterID UInt32 +StartDate Date +UserID UInt32 +VisitID UInt32 +NestedColumn.A Array(UInt8) +NestedColumn.S Array(String) +ToDrop UInt32 +Added0 UInt32 +Added1 UInt32 +Added2 UInt32 +AddedNested1.A Array(UInt32) +AddedNested1.B Array(UInt64) +AddedNested1.C Array(String) +AddedNested2.A Array(UInt32) +AddedNested2.B Array(UInt64) +CounterID UInt32 +StartDate Date +UserID UInt32 +VisitID UInt32 +Added0 String +Added1 UInt32 +Added2 UInt32 +AddedNested1.A Array(UInt32) +AddedNested1.C Array(String) +AddedNested2.A Array(UInt32) +AddedNested2.B Array(UInt64) 1 2014-01-01 2 3 0 0 [] [] [] [] diff --git a/dbms/tests/queries/0_stateless/00061_merge_tree_alter.reference b/dbms/tests/queries/0_stateless/00061_merge_tree_alter.reference index 9545f7b4200..21d00634771 100644 --- a/dbms/tests/queries/0_stateless/00061_merge_tree_alter.reference +++ b/dbms/tests/queries/0_stateless/00061_merge_tree_alter.reference @@ -1,56 +1,56 @@ -d Date -k UInt64 -i32 Int32 +d Date +k UInt64 +i32 Int32 CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32) ENGINE = MergeTree(d, k, 8192) 2015-01-01 10 42 -d Date -k UInt64 -i32 Int32 -n.ui8 Array(UInt8) -n.s Array(String) +d Date +k UInt64 +i32 Int32 +n.ui8 Array(UInt8) +n.s Array(String) CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = MergeTree(d, k, 8192) 2015-01-01 8 40 [1,2,3] ['12','13','14'] 2015-01-01 10 42 [] [] -d Date -k UInt64 -i32 Int32 -n.ui8 Array(UInt8) -n.s Array(String) -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +n.ui8 Array(UInt8) +n.s Array(String) +n.d Array(Date) CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192) 2015-01-01 7 39 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 2015-01-01 8 40 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 10 42 [] [] [] -d Date -k UInt64 -i32 Int32 -n.ui8 Array(UInt8) -n.s Array(String) -n.d Array(Date) -s String DEFAULT \'0\' +d Date +k UInt64 +i32 Int32 +n.ui8 Array(UInt8) +n.s Array(String) +n.d Array(Date) +s String DEFAULT \'0\' CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), s String DEFAULT \'0\') ENGINE = MergeTree(d, k, 8192) 2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] ['2000-01-01','2000-01-01','2000-01-03'] 100500 2015-01-01 7 39 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 0 2015-01-01 8 40 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00'] 0 2015-01-01 10 42 [] [] [] 0 -d Date -k UInt64 -i32 Int32 -n.ui8 Array(UInt8) -n.s Array(String) -s Int64 +d Date +k UInt64 +i32 Int32 +n.ui8 Array(UInt8) +n.s Array(String) +s Int64 CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String), s Int64) ENGINE = MergeTree(d, k, 8192) 2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] 100500 2015-01-01 7 39 [10,20,30] ['120','130','140'] 0 2015-01-01 8 40 [1,2,3] ['12','13','14'] 0 2015-01-01 10 42 [] [] 0 -d Date -k UInt64 -i32 Int32 -n.ui8 Array(UInt8) -n.s Array(String) -s UInt32 -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +n.ui8 Array(UInt8) +n.s Array(String) +s UInt32 +n.d Array(Date) CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String), s UInt32, `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192) 2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] 100500 ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 7 39 [10,20,30] ['120','130','140'] 0 ['0000-00-00','0000-00-00','0000-00-00'] @@ -60,40 +60,40 @@ CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), 2015-01-01 7 39 [10,20,30] ['120','130','140'] 0 ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 8 40 [1,2,3] ['12','13','14'] 0 ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 10 42 [] [] 0 [] -d Date -k UInt64 -i32 Int32 -n.s Array(String) -s UInt32 +d Date +k UInt64 +i32 Int32 +n.s Array(String) +s UInt32 CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.s` Array(String), s UInt32) ENGINE = MergeTree(d, k, 8192) 2015-01-01 6 38 ['asd','qwe','qwe'] 100500 2015-01-01 7 39 ['120','130','140'] 0 2015-01-01 8 40 ['12','13','14'] 0 2015-01-01 10 42 [] 0 -d Date -k UInt64 -i32 Int32 -s UInt32 +d Date +k UInt64 +i32 Int32 +s UInt32 CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, s UInt32) ENGINE = MergeTree(d, k, 8192) 2015-01-01 6 38 100500 2015-01-01 7 39 0 2015-01-01 8 40 0 2015-01-01 10 42 0 -d Date -k UInt64 -i32 Int32 -s UInt32 -n.s Array(String) -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +s UInt32 +n.s Array(String) +n.d Array(Date) CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, s UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192) 2015-01-01 6 38 100500 [] [] 2015-01-01 7 39 0 [] [] 2015-01-01 8 40 0 [] [] 2015-01-01 10 42 0 [] [] -d Date -k UInt64 -i32 Int32 -s UInt32 +d Date +k UInt64 +i32 Int32 +s UInt32 CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, s UInt32) ENGINE = MergeTree(d, k, 8192) 2015-01-01 6 38 100500 2015-01-01 7 39 0 diff --git a/dbms/tests/queries/0_stateless/00079_defaulted_columns.reference b/dbms/tests/queries/0_stateless/00079_defaulted_columns.reference index 135da6b3681..8ae430dbd25 100644 --- a/dbms/tests/queries/0_stateless/00079_defaulted_columns.reference +++ b/dbms/tests/queries/0_stateless/00079_defaulted_columns.reference @@ -1,42 +1,42 @@ -col1 UInt8 DEFAULT 0 -col1 UInt32 -col2 UInt64 DEFAULT col1 + 1 -col3 UInt64 MATERIALIZED col1 + 2 -col4 UInt64 ALIAS col1 + 3 +col1 UInt8 DEFAULT 0 +col1 UInt32 +col2 UInt64 DEFAULT col1 + 1 +col3 UInt64 MATERIALIZED col1 + 2 +col4 UInt64 ALIAS col1 + 3 10 11 12 13 99 -payload String -date Date MATERIALIZED today() -key UInt64 MATERIALIZED 0 * rand() +payload String +date Date MATERIALIZED today() +key UInt64 MATERIALIZED 0 * rand() hello clickhouse -payload String -date Date MATERIALIZED today() -key UInt64 MATERIALIZED 0 * rand() -payload_length UInt64 MATERIALIZED length(payload) +payload String +date Date MATERIALIZED today() +key UInt64 MATERIALIZED 0 * rand() +payload_length UInt64 MATERIALIZED length(payload) hello clickhouse 16 hello clickhouse 16 some string 11 hello clickhouse 16 some string 11 -payload String -payload_length UInt64 DEFAULT length(payload) -date Date MATERIALIZED today() -key UInt64 MATERIALIZED 0 * rand() +payload String +payload_length UInt64 DEFAULT length(payload) +date Date MATERIALIZED today() +key UInt64 MATERIALIZED 0 * rand() hello clickhouse 16 some string 11 -payload String -payload_length UInt16 DEFAULT length(payload) % 65535 -date Date MATERIALIZED today() -key UInt64 MATERIALIZED 0 * rand() +payload String +payload_length UInt16 DEFAULT length(payload) % 65535 +date Date MATERIALIZED today() +key UInt64 MATERIALIZED 0 * rand() hello clickhouse 16 some string 11 -payload String -payload_length UInt16 DEFAULT CAST(length(payload), \'UInt16\') -date Date MATERIALIZED today() -key UInt64 MATERIALIZED 0 * rand() -payload String -date Date MATERIALIZED today() -key UInt64 MATERIALIZED 0 * rand() +payload String +payload_length UInt16 DEFAULT CAST(length(payload), \'UInt16\') +date Date MATERIALIZED today() +key UInt64 MATERIALIZED 0 * rand() +payload String +date Date MATERIALIZED today() +key UInt64 MATERIALIZED 0 * rand() hello clickhouse some string diff --git a/dbms/tests/queries/0_stateless/00168_buffer_defaults.reference b/dbms/tests/queries/0_stateless/00168_buffer_defaults.reference index aa378b78ba9..2667191e518 100644 --- a/dbms/tests/queries/0_stateless/00168_buffer_defaults.reference +++ b/dbms/tests/queries/0_stateless/00168_buffer_defaults.reference @@ -1,9 +1,9 @@ -EventDate Date -UTCEventTime DateTime -MoscowEventDate Date DEFAULT toDate(UTCEventTime) -EventDate Date -UTCEventTime DateTime -MoscowEventDate Date DEFAULT toDate(UTCEventTime) +EventDate Date +UTCEventTime DateTime +MoscowEventDate Date DEFAULT toDate(UTCEventTime) +EventDate Date +UTCEventTime DateTime +MoscowEventDate Date DEFAULT toDate(UTCEventTime) 2015-06-09 2015-06-09 01:02:03 2015-06-09 2015-06-09 2015-06-09 01:02:03 2015-06-09 2015-06-09 2015-06-09 01:02:03 2015-06-09 diff --git a/dbms/tests/queries/0_stateless/00415_into_outfile.reference b/dbms/tests/queries/0_stateless/00415_into_outfile.reference index 1fc1ceac892..967f7615550 100644 --- a/dbms/tests/queries/0_stateless/00415_into_outfile.reference +++ b/dbms/tests/queries/0_stateless/00415_into_outfile.reference @@ -6,7 +6,7 @@ performing test: union_all performing test: bad_union_all query failed performing test: describe_table -dummy UInt8 +dummy UInt8 performing test: clickhouse-local 2 3 performing test: http diff --git a/dbms/tests/queries/0_stateless/00557_alter_null_storage_tables.reference b/dbms/tests/queries/0_stateless/00557_alter_null_storage_tables.reference index 35790c754de..0f20962a568 100644 --- a/dbms/tests/queries/0_stateless/00557_alter_null_storage_tables.reference +++ b/dbms/tests/queries/0_stateless/00557_alter_null_storage_tables.reference @@ -1,3 +1,3 @@ -x UInt8 -x Int64 DEFAULT toInt64(y) -y String +x UInt8 +x Int64 DEFAULT toInt64(y) +y String diff --git a/dbms/tests/queries/0_stateless/00642_cast.reference b/dbms/tests/queries/0_stateless/00642_cast.reference index 56f79769cc7..58518327779 100644 --- a/dbms/tests/queries/0_stateless/00642_cast.reference +++ b/dbms/tests/queries/0_stateless/00642_cast.reference @@ -8,6 +8,6 @@ hello hello 1970-01-01 00:00:01 CREATE TABLE test.cast ( x UInt8, e Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')) ENGINE = MergeTree ORDER BY e SETTINGS index_granularity = 8192 -x UInt8 -e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\') +x UInt8 +e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\') 1 hello diff --git a/dbms/tests/queries/0_stateless/00651_default_database_on_client_reconnect.reference b/dbms/tests/queries/0_stateless/00651_default_database_on_client_reconnect.reference index 8ed1dd9ba87..4a5f18ade7d 100644 --- a/dbms/tests/queries/0_stateless/00651_default_database_on_client_reconnect.reference +++ b/dbms/tests/queries/0_stateless/00651_default_database_on_client_reconnect.reference @@ -1 +1 @@ -val UInt64 +val UInt64 diff --git a/dbms/tests/queries/0_stateless/00700_decimal_defaults.reference b/dbms/tests/queries/0_stateless/00700_decimal_defaults.reference index 635f40a11fe..c0c5de4d8a9 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_defaults.reference +++ b/dbms/tests/queries/0_stateless/00700_decimal_defaults.reference @@ -1,9 +1,9 @@ -a Decimal(9, 4) DEFAULT CAST(0, \'Decimal(9, 4)\') -b Decimal(18, 4) DEFAULT CAST(a / 2, \'Decimal(18, 4)\') -c Decimal(38, 4) DEFAULT CAST(b / 3, \'Decimal(38, 4)\') -d Decimal(9, 4) MATERIALIZED a + toDecimal32(\'0.2\', 1) -e Decimal(18, 4) ALIAS b * 2 -f Decimal(38, 4) ALIAS c * 6 +a Decimal(9, 4) DEFAULT CAST(0, \'Decimal(9, 4)\') +b Decimal(18, 4) DEFAULT CAST(a / 2, \'Decimal(18, 4)\') +c Decimal(38, 4) DEFAULT CAST(b / 3, \'Decimal(38, 4)\') +d Decimal(9, 4) MATERIALIZED a + toDecimal32(\'0.2\', 1) +e Decimal(18, 4) ALIAS b * 2 +f Decimal(38, 4) ALIAS c * 6 0.0000 0.0000 0.0000 1.0000 0.5000 0.1666 2.0000 1.0000 0.3333 diff --git a/dbms/tests/queries/0_stateless/00725_comment_columns.reference b/dbms/tests/queries/0_stateless/00725_comment_columns.reference index ca4edfb3122..5d5f9422bd5 100644 --- a/dbms/tests/queries/0_stateless/00725_comment_columns.reference +++ b/dbms/tests/queries/0_stateless/00725_comment_columns.reference @@ -1,9 +1,9 @@ CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'comment 1\', fourth_column UInt8 COMMENT \'comment 4\', fifth_column UInt8, second_column UInt8 MATERIALIZED first_column COMMENT \'comment 2\', third_column UInt8 ALIAS second_column COMMENT \'comment 3\') ENGINE = TinyLog -first_column UInt8 DEFAULT 1 comment 1 -fourth_column UInt8 comment 4 -fifth_column UInt8 -second_column UInt8 MATERIALIZED first_column comment 2 -third_column UInt8 ALIAS second_column comment 3 +first_column UInt8 DEFAULT 1 comment 1 +fourth_column UInt8 comment 4 +fifth_column UInt8 +second_column UInt8 MATERIALIZED first_column comment 2 +third_column UInt8 ALIAS second_column comment 3 ┌─table──────────────────────┬─name──────────┬─comment───┐ │ check_query_comment_column │ first_column │ comment 1 │ │ check_query_comment_column │ fourth_column │ comment 4 │ @@ -21,9 +21,9 @@ CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMM └────────────────────────────┴───────────────┴─────────────┘ CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'comment 1_2\', fourth_column UInt8 COMMENT \'comment 4_2\', fifth_column UInt8 COMMENT \'comment 5_2\', second_column UInt8 MATERIALIZED first_column COMMENT \'comment 2_2\', third_column UInt8 ALIAS second_column COMMENT \'comment 3_2\') ENGINE = TinyLog CREATE TABLE test.check_query_comment_column ( first_column UInt8 COMMENT \'comment 1\', second_column UInt8 COMMENT \'comment 2\', third_column UInt8 COMMENT \'comment 3\') ENGINE = MergeTree() PARTITION BY second_column ORDER BY first_column SAMPLE BY first_column SETTINGS index_granularity = 8192 -first_column UInt8 comment 1 -second_column UInt8 comment 2 -third_column UInt8 comment 3 +first_column UInt8 comment 1 +second_column UInt8 comment 2 +third_column UInt8 comment 3 ┌─table──────────────────────┬─name──────────┬─comment───┐ │ check_query_comment_column │ first_column │ comment 1 │ │ check_query_comment_column │ second_column │ comment 2 │ diff --git a/dbms/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference b/dbms/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference index 3c344a523ae..25f86684c28 100644 --- a/dbms/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference @@ -1,4 +1,4 @@ CREATE TABLE test.check_comments ( column_name1 UInt8 DEFAULT 1 COMMENT \'comment\') ENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_comments\', \'r1\') ORDER BY column_name1 SETTINGS index_granularity = 8192 -column_name1 UInt8 DEFAULT 1 comment +column_name1 UInt8 DEFAULT 1 comment CREATE TABLE test.check_comments ( column_name1 UInt8 DEFAULT 1 COMMENT \'another comment\') ENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_comments\', \'r1\') ORDER BY column_name1 SETTINGS index_granularity = 8192 -column_name1 UInt8 DEFAULT 1 another comment +column_name1 UInt8 DEFAULT 1 another comment From 8abcdc70f133b6b209a3e46935aac09a19d7a481 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 25 Dec 2018 15:56:11 +0300 Subject: [PATCH 089/230] More tabs in desc table tests --- ...cated_merge_tree_alter_zookeeper.reference | 284 +++++++++--------- .../0_stateless/00294_shard_enums.reference | 40 +-- ...c_table_functions_and_subqueries.reference | 46 +-- .../00643_cast_zookeeper.reference | 4 +- 4 files changed, 187 insertions(+), 187 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference b/dbms/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference index cd371656965..4d45c186220 100644 --- a/dbms/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference @@ -1,213 +1,213 @@ -d Date -k UInt64 -i32 Int32 +d Date +k UInt64 +i32 Int32 CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 +d Date +k UInt64 +i32 Int32 CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 10 42 -d Date -k UInt64 -i32 Int32 -dt DateTime +d Date +k UInt64 +i32 Int32 +dt DateTime CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime +d Date +k UInt64 +i32 Int32 +dt DateTime CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 9 41 1992-01-01 08:00:00 2015-01-01 10 42 0000-00-00 00:00:00 -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 2015-01-01 9 41 1992-01-01 08:00:00 [] [] 2015-01-01 10 42 0000-00-00 00:00:00 [] [] -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +n.d Array(Date) CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +n.d Array(Date) CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 9 41 1992-01-01 08:00:00 [] [] [] 2015-01-01 10 42 0000-00-00 00:00:00 [] [] [] -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -n.d Array(Date) -s String DEFAULT \'0\' +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +n.d Array(Date) +s String DEFAULT \'0\' CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), s String DEFAULT \'0\') ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -n.d Array(Date) -s String DEFAULT \'0\' +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +n.d Array(Date) +s String DEFAULT \'0\' CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), s String DEFAULT \'0\') ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] ['2000-01-01','2000-01-01','2000-01-03'] 100500 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 0 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00'] 0 2015-01-01 9 41 1992-01-01 08:00:00 [] [] [] 0 2015-01-01 10 42 0000-00-00 00:00:00 [] [] [] 0 -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -s Int64 +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +s Int64 CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), s Int64) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -s Int64 +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +s Int64 CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), s Int64) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0 2015-01-01 9 41 1992-01-01 08:00:00 [] [] 0 2015-01-01 10 42 0000-00-00 00:00:00 [] [] 0 -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -s UInt32 -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +s UInt32 +n.d Array(Date) CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), s UInt32, `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -s UInt32 -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +s UInt32 +n.d Array(Date) CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), s UInt32, `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500 ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0 ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0 ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 9 41 1992-01-01 08:00:00 [] [] 0 [] 2015-01-01 10 42 0000-00-00 00:00:00 [] [] 0 [] -d Date -k UInt64 -i32 Int32 -dt DateTime -n.s Array(String) -s UInt32 +d Date +k UInt64 +i32 Int32 +dt DateTime +n.s Array(String) +s UInt32 CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.s` Array(String), s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -n.s Array(String) -s UInt32 +d Date +k UInt64 +i32 Int32 +dt DateTime +n.s Array(String) +s UInt32 CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.s` Array(String), s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 ['asd','qwe','qwe'] 100500 2015-01-01 7 39 2014-07-14 13:26:50 ['120','130','140'] 0 2015-01-01 8 40 2012-12-12 12:12:12 ['12','13','14'] 0 2015-01-01 9 41 1992-01-01 08:00:00 [] 0 2015-01-01 10 42 0000-00-00 00:00:00 [] 0 -d Date -k UInt64 -i32 Int32 -dt DateTime -s UInt32 +d Date +k UInt64 +i32 Int32 +dt DateTime +s UInt32 CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -s UInt32 +d Date +k UInt64 +i32 Int32 +dt DateTime +s UInt32 CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 2015-01-01 7 39 2014-07-14 13:26:50 0 2015-01-01 8 40 2012-12-12 12:12:12 0 2015-01-01 9 41 1992-01-01 08:00:00 0 2015-01-01 10 42 0000-00-00 00:00:00 0 -d Date -k UInt64 -i32 Int32 -dt DateTime -s UInt32 -n.s Array(String) -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +dt DateTime +s UInt32 +n.s Array(String) +n.d Array(Date) CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -s UInt32 -n.s Array(String) -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +dt DateTime +s UInt32 +n.s Array(String) +n.d Array(Date) CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 [] [] 2015-01-01 7 39 2014-07-14 13:26:50 0 [] [] 2015-01-01 8 40 2012-12-12 12:12:12 0 [] [] 2015-01-01 9 41 1992-01-01 08:00:00 0 [] [] 2015-01-01 10 42 0000-00-00 00:00:00 0 [] [] -d Date -k UInt64 -i32 Int32 -dt DateTime -s UInt32 +d Date +k UInt64 +i32 Int32 +dt DateTime +s UInt32 CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -s UInt32 +d Date +k UInt64 +i32 Int32 +dt DateTime +s UInt32 CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 2015-01-01 7 39 2014-07-14 13:26:50 0 2015-01-01 8 40 2012-12-12 12:12:12 0 2015-01-01 9 41 1992-01-01 08:00:00 0 2015-01-01 10 42 0000-00-00 00:00:00 0 -d Date -k UInt64 -i32 Int32 -dt Date -s DateTime +d Date +k UInt64 +i32 Int32 +dt Date +s DateTime CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt Date, s DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt Date -s DateTime +d Date +k UInt64 +i32 Int32 +dt Date +s DateTime CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt Date, s DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 1970-01-02 06:55:00 2015-01-01 7 39 2014-07-14 0000-00-00 00:00:00 diff --git a/dbms/tests/queries/0_stateless/00294_shard_enums.reference b/dbms/tests/queries/0_stateless/00294_shard_enums.reference index 28dc7781253..06528e21138 100644 --- a/dbms/tests/queries/0_stateless/00294_shard_enums.reference +++ b/dbms/tests/queries/0_stateless/00294_shard_enums.reference @@ -1,28 +1,28 @@ -d Date DEFAULT CAST(\'2015-12-29\', \'Date\') -k UInt8 DEFAULT 0 -e Enum8(\'hello\' = 1, \'world\' = 2) -sign Enum8(\'minus\' = -1, \'plus\' = 1) -letter Enum16(\'*\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) +d Date DEFAULT CAST(\'2015-12-29\', \'Date\') +k UInt8 DEFAULT 0 +e Enum8(\'hello\' = 1, \'world\' = 2) +sign Enum8(\'minus\' = -1, \'plus\' = 1) +letter Enum16(\'*\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) 2015-12-29 0 hello minus * -d Date DEFAULT CAST(\'2015-12-29\', \'Date\') -k UInt8 DEFAULT 0 -e Enum8(\'hello\' = 1, \'world\' = 2, \'!\' = 3) -sign Enum8(\'minus\' = -1, \'plus\' = 1) -letter Enum16(\'*\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) +d Date DEFAULT CAST(\'2015-12-29\', \'Date\') +k UInt8 DEFAULT 0 +e Enum8(\'hello\' = 1, \'world\' = 2, \'!\' = 3) +sign Enum8(\'minus\' = -1, \'plus\' = 1) +letter Enum16(\'*\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) 2015-12-29 0 hello minus * 2015-12-29 0 ! plus b -d Date DEFAULT CAST(\'2015-12-29\', \'Date\') -k UInt8 DEFAULT 0 -e Enum16(\'hello\' = 1, \'world\' = 2, \'!\' = 3) -sign Enum16(\'minus\' = -1, \'plus\' = 1) -letter Enum16(\'no letter\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) +d Date DEFAULT CAST(\'2015-12-29\', \'Date\') +k UInt8 DEFAULT 0 +e Enum16(\'hello\' = 1, \'world\' = 2, \'!\' = 3) +sign Enum16(\'minus\' = -1, \'plus\' = 1) +letter Enum16(\'no letter\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) 2015-12-29 0 hello minus no letter 2015-12-29 0 ! plus b -d Date DEFAULT CAST(\'2015-12-29\', \'Date\') -k UInt8 DEFAULT 0 -e Enum8(\'hello\' = 1, \'world\' = 2, \'!\' = 3) -sign Enum8(\'minus\' = -1, \'plus\' = 1) -letter Enum16(\'no letter\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) +d Date DEFAULT CAST(\'2015-12-29\', \'Date\') +k UInt8 DEFAULT 0 +e Enum8(\'hello\' = 1, \'world\' = 2, \'!\' = 3) +sign Enum8(\'minus\' = -1, \'plus\' = 1) +letter Enum16(\'no letter\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) 2015-12-29 0 hello minus no letter 2015-12-29 0 ! plus b 2015-12-29 0 world minus c diff --git a/dbms/tests/queries/0_stateless/00515_shard_desc_table_functions_and_subqueries.reference b/dbms/tests/queries/0_stateless/00515_shard_desc_table_functions_and_subqueries.reference index 5ccb1694c92..0410fde9b62 100644 --- a/dbms/tests/queries/0_stateless/00515_shard_desc_table_functions_and_subqueries.reference +++ b/dbms/tests/queries/0_stateless/00515_shard_desc_table_functions_and_subqueries.reference @@ -1,30 +1,30 @@ -date Date -val UInt64 -val2 UInt8 DEFAULT 42 -val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') -val4 UInt64 ALIAS val +date Date +val UInt64 +val2 UInt8 DEFAULT 42 +val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') +val4 UInt64 ALIAS val - -date Date -val UInt64 -val2 UInt8 DEFAULT 42 -val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') -val4 UInt64 ALIAS val +date Date +val UInt64 +val2 UInt8 DEFAULT 42 +val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') +val4 UInt64 ALIAS val - -date Date -val UInt64 -val2 UInt8 DEFAULT 42 -val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') -val4 UInt64 ALIAS val +date Date +val UInt64 +val2 UInt8 DEFAULT 42 +val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') +val4 UInt64 ALIAS val - -date Date -val UInt64 -val2 UInt8 DEFAULT 42 -val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') -val4 UInt64 ALIAS val +date Date +val UInt64 +val2 UInt8 DEFAULT 42 +val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') +val4 UInt64 ALIAS val - -1 UInt8 +1 UInt8 - -1 UInt8 +1 UInt8 - -number UInt64 +number UInt64 - diff --git a/dbms/tests/queries/0_stateless/00643_cast_zookeeper.reference b/dbms/tests/queries/0_stateless/00643_cast_zookeeper.reference index f3b3ba04d1c..abdca87a873 100644 --- a/dbms/tests/queries/0_stateless/00643_cast_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00643_cast_zookeeper.reference @@ -1,5 +1,5 @@ CREATE TABLE test.cast1 ( x UInt8, e Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_cast', 'r1') ORDER BY e SETTINGS index_granularity = 8192 -x UInt8 -e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\') +x UInt8 +e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\') 1 hello 1 hello From abae86f7a68822b2499a2c55dd47b29cb8dacb5c Mon Sep 17 00:00:00 2001 From: ogorbacheva Date: Tue, 25 Dec 2018 18:25:43 +0300 Subject: [PATCH 090/230] Doc fixes: remove double placeholders; add them where missing. (#3923) * Doc fix: add spaces where missing * Doc fixes: rm double spaces * Doc fixes: edit spaces * Doc fixes: rm double spaces in /fa * Revert "Doc fixes: rm double spaces in /fa" This reverts commit bb879a62ef5fa965d989fea3b1b2a693d2016a2d. * Doc fix: resolve all problems with double spaces in /fa * Doc fix: add spaces for readability * Doc fix: add spaces * Fix spaces --- docs/en/data_types/array.md | 2 +- docs/en/data_types/enum.md | 2 +- docs/en/data_types/float.md | 2 +- .../aggregatefunction.md | 2 +- docs/en/development/style.md | 10 ++++----- docs/en/development/tests.md | 2 +- .../example_datasets/ontime.md | 2 +- .../example_datasets/star_schema.md | 4 ++-- docs/en/interfaces/cli.md | 4 ++-- docs/en/interfaces/formats.md | 6 ++--- docs/en/introduction/ya_metrika_task.md | 2 +- docs/en/operations/configuration_files.md | 2 +- .../en/operations/server_settings/settings.md | 6 ++--- docs/en/operations/settings/settings.md | 16 +++++++------- .../operations/settings/settings_profiles.md | 6 ++--- docs/en/operations/system_tables.md | 4 ++-- .../table_engines/custom_partitioning_key.md | 2 +- docs/en/operations/table_engines/kafka.md | 4 ++-- docs/en/operations/table_engines/merge.md | 14 ++++++------ docs/en/operations/table_engines/mergetree.md | 2 +- .../table_engines/replacingmergetree.md | 4 ++-- .../table_engines/summingmergetree.md | 8 +++---- docs/en/operations/tips.md | 2 +- .../agg_functions/parametric_functions.md | 2 +- .../query_language/agg_functions/reference.md | 8 +++---- docs/en/query_language/alter.md | 14 ++++++------ .../dicts/external_dicts_dict_layout.md | 2 +- .../dicts/external_dicts_dict_lifetime.md | 2 +- .../functions/array_functions.md | 4 ++-- .../functions/ext_dict_functions.md | 2 +- .../functions/functions_for_nulls.md | 4 ++-- docs/en/query_language/functions/geo.md | 2 +- .../functions/other_functions.md | 4 ++-- .../functions/string_search_functions.md | 2 +- .../query_language/functions/url_functions.md | 4 ++-- docs/en/query_language/insert_into.md | 2 +- docs/en/query_language/misc.md | 2 +- docs/en/query_language/operators.md | 4 ++-- docs/en/query_language/select.md | 14 ++++++------ docs/en/query_language/syntax.md | 2 +- docs/en/query_language/table_functions/url.md | 2 +- docs/fa/data_types/float.md | 2 +- .../aggregatefunction.md | 2 +- .../nested_data_structures/nested.md | 2 +- .../example_datasets/amplab_benchmark.md | 2 +- .../example_datasets/nyc_taxi.md | 8 +++---- .../example_datasets/ontime.md | 2 +- .../example_datasets/star_schema.md | 2 +- docs/fa/getting_started/index.md | 2 +- docs/fa/interfaces/cli.md | 6 ++--- docs/fa/interfaces/formats.md | 8 +++---- docs/fa/interfaces/http.md | 4 ++-- docs/fa/introduction/distinctive_features.md | 2 +- docs/ru/data_types/tuple.md | 2 +- docs/ru/development/style.md | 16 +++++++------- .../example_datasets/nyc_taxi.md | 2 +- .../example_datasets/ontime.md | 2 +- docs/ru/index.md | 6 ++--- docs/ru/interfaces/cli.md | 10 ++++----- docs/ru/interfaces/http.md | 6 ++--- docs/ru/introduction/distinctive_features.md | 6 ++--- .../ru/operations/server_settings/settings.md | 2 +- .../settings/permissions_for_queries.md | 4 ++-- docs/ru/operations/settings/settings.md | 2 +- .../table_engines/collapsingmergetree.md | 4 ++-- .../table_engines/graphitemergetree.md | 2 +- docs/ru/operations/table_engines/kafka.md | 2 +- .../operations/table_engines/replication.md | 8 +++---- docs/ru/query_language/agg_functions/index.md | 4 ++-- .../query_language/agg_functions/reference.md | 2 +- docs/ru/query_language/alter.md | 8 +++---- docs/ru/query_language/create.md | 2 +- .../dicts/external_dicts_dict.md | 10 ++++----- .../dicts/external_dicts_dict_layout.md | 8 +++---- .../functions/array_functions.md | 6 ++--- .../functions/json_functions.md | 8 +++---- .../functions/other_functions.md | 4 ++-- docs/ru/query_language/misc.md | 6 ++--- docs/ru/query_language/select.md | 18 +++++++-------- docs/zh/data_types/array.md | 2 +- docs/zh/data_types/float.md | 4 ++-- docs/zh/development/style.md | 6 ++--- docs/zh/faq/general.md | 2 +- .../example_datasets/nyc_taxi.md | 2 +- .../example_datasets/ontime.md | 2 +- docs/zh/interfaces/cli.md | 4 ++-- docs/zh/interfaces/formats.md | 10 ++++----- docs/zh/interfaces/http.md | 2 +- docs/zh/interfaces/jdbc.md | 2 +- docs/zh/interfaces/odbc.md | 2 +- docs/zh/interfaces/tcp.md | 2 +- docs/zh/interfaces/third-party/gui.md | 8 +++---- docs/zh/interfaces/third-party/proxy.md | 6 ++--- docs/zh/introduction/distinctive_features.md | 2 +- .../features_considered_disadvantages.md | 2 +- docs/zh/introduction/performance.md | 4 ++-- docs/zh/introduction/ya_metrika_task.md | 2 +- docs/zh/operations/configuration_files.md | 2 +- .../zh/operations/server_settings/settings.md | 6 ++--- docs/zh/operations/settings/settings.md | 14 ++++++------ .../operations/settings/settings_profiles.md | 6 ++--- docs/zh/operations/system_tables.md | 4 ++-- .../table_engines/custom_partitioning_key.md | 2 +- docs/zh/operations/table_engines/kafka.md | 2 +- docs/zh/operations/table_engines/merge.md | 14 ++++++------ docs/zh/operations/table_engines/mergetree.md | 2 +- .../table_engines/replacingmergetree.md | 4 ++-- .../table_engines/summingmergetree.md | 8 +++---- docs/zh/operations/tips.md | 2 +- docs/zh/query_language/create.md | 2 +- docs/zh/query_language/insert_into.md | 4 ++-- docs/zh/query_language/select.md | 22 +++++++++---------- 112 files changed, 267 insertions(+), 267 deletions(-) diff --git a/docs/en/data_types/array.md b/docs/en/data_types/array.md index 5e2b2f55127..552c549b622 100644 --- a/docs/en/data_types/array.md +++ b/docs/en/data_types/array.md @@ -48,7 +48,7 @@ SELECT ## Working with data types -When creating an array on the fly, ClickHouse automatically defines the argument type as the narrowest data type that can store all the listed arguments. If there are any [NULL](../query_language/syntax.md#null-literal) or [Nullable](nullable.md#data_type-nullable) type arguments, the type of array elements is [Nullable](nullable.md). +When creating an array on the fly, ClickHouse automatically defines the argument type as the narrowest data type that can store all the listed arguments. If there are any [NULL](../query_language/syntax.md#null-literal) or [Nullable](nullable.md#data_type-nullable) type arguments, the type of array elements is [Nullable](nullable.md). If ClickHouse couldn't determine the data type, it will generate an exception. For instance, this will happen when trying to create an array with strings and numbers simultaneously (`SELECT array(1, 'a')`). diff --git a/docs/en/data_types/enum.md b/docs/en/data_types/enum.md index 5ba82e71ec5..754b6651f56 100644 --- a/docs/en/data_types/enum.md +++ b/docs/en/data_types/enum.md @@ -8,7 +8,7 @@ Includes the `Enum8` and `Enum16` types. `Enum` saves the finite set of pairs of ## Usage examples -Here we create a table with an `Enum8('hello' = 1, 'world' = 2)` type column: +Here we create a table with an `Enum8('hello' = 1, 'world' = 2)` type column: ``` CREATE TABLE t_enum diff --git a/docs/en/data_types/float.md b/docs/en/data_types/float.md index 894b50d8237..1c908819c1f 100644 --- a/docs/en/data_types/float.md +++ b/docs/en/data_types/float.md @@ -5,7 +5,7 @@ Types are equivalent to types of C: - `Float32` - `float` -- `Float64` - `double` +- `Float64` - `double` We recommend that you store data in integer form whenever possible. For example, convert fixed precision numbers to integer values, such as monetary amounts or page load times in milliseconds. diff --git a/docs/en/data_types/nested_data_structures/aggregatefunction.md b/docs/en/data_types/nested_data_structures/aggregatefunction.md index 4a02c5c7ad3..95ed4d751e6 100644 --- a/docs/en/data_types/nested_data_structures/aggregatefunction.md +++ b/docs/en/data_types/nested_data_structures/aggregatefunction.md @@ -40,7 +40,7 @@ quantilesState(0.5, 0.9)(SendTiming) In contrast to the corresponding functions `uniq` and `quantiles`, `-State`- functions return the state, instead the final value. In other words, they return a value of `AggregateFunction` type. -In the results of `SELECT` query the values of `AggregateFunction` type have implementation-specific binary representation for all of the ClickHouse output formats. If dump data into, for example, `TabSeparated` format with `SELECT` query then this dump can be loaded back using `INSERT` query. +In the results of `SELECT` query the values of `AggregateFunction` type have implementation-specific binary representation for all of the ClickHouse output formats. If dump data into, for example, `TabSeparated` format with `SELECT` query then this dump can be loaded back using `INSERT` query. ### Data Selection diff --git a/docs/en/development/style.md b/docs/en/development/style.md index edfda73a7eb..1af7814a97f 100644 --- a/docs/en/development/style.md +++ b/docs/en/development/style.md @@ -44,7 +44,7 @@ void reinsert(const Value & x) memcpy(&buf[place_value], &x, sizeof(x)); ``` -**6.** In `if`, `for`, `while` and other expressions, a space is inserted in front of the opening bracket (as opposed to function calls). +**6.** In `if`, `for`, `while` and other expressions, a space is inserted in front of the opening bracket (as opposed to function calls). ```cpp for (size_t i = 0; i < rows; i += storage.index_granularity) @@ -93,7 +93,7 @@ struct AggregatedStatElement {} ``` -**15.** In classes and structures, write `public`, `private`, and `protected` on the same level as `class/struct`, and indent the rest of the code. +**15.** In classes and structures, write `public`, `private`, and `protected` on the same level as `class/struct`, and indent the rest of the code. ```cpp template @@ -261,7 +261,7 @@ The example is borrowed from the resource [http://home.tamk.fi/~jaalto/course/co **7.** Do not write garbage comments (author, creation date ..) at the beginning of each file. -**8.** Single-line comments begin with three slashes: `///` and multi-line comments begin with `/**`. These comments are considered "documentation". +**8.** Single-line comments begin with three slashes: `///` and multi-line comments begin with `/**`. These comments are considered "documentation". Note: You can use Doxygen to generate documentation from these comments. But Doxygen is not generally used because it is more convenient to navigate the code in the IDE. @@ -299,7 +299,7 @@ Note: You can use Doxygen to generate documentation from these comments. But Dox ## Names -**1.** Use lowercase letters with underscores in the names of variables and class members. +**1.** Use lowercase letters with underscores in the names of variables and class members. ```cpp size_t max_block_size; @@ -349,7 +349,7 @@ In all other cases, use a name that describes the meaning. bool info_successfully_loaded = false; ``` -**9.** Names of `define`s and global constants use ALL_CAPS with underscores. +**9.** Names of `define`s and global constants use ALL_CAPS with underscores. ```cpp #define MAX_SRC_TABLE_NAMES_TO_STORE 1000 diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index d9a44f78ea3..2e2e1660270 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -15,7 +15,7 @@ To run all tests, use `dbms/tests/clickhouse-test` tool. Look `--help` for the l The most simple way to invoke functional tests is to copy `clickhouse-client` to `/usr/bin/`, run `clickhouse-server` and then run `./clickhouse-test` from its own directory. -To add new test, create a `.sql` or `.sh` file in `dbms/src/tests/queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client -n --testmode < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. +To add new test, create a `.sql` or `.sh` file in `dbms/src/tests/queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client -n --testmode < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables. diff --git a/docs/en/getting_started/example_datasets/ontime.md b/docs/en/getting_started/example_datasets/ontime.md index bd77e838e90..c4dcaf9575e 100644 --- a/docs/en/getting_started/example_datasets/ontime.md +++ b/docs/en/getting_started/example_datasets/ontime.md @@ -166,7 +166,7 @@ SELECT Origin, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Q4. The number of delays by carrier for 2007 ``` sql -SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC +SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC ``` Q5. The percentage of delays by carrier for 2007 diff --git a/docs/en/getting_started/example_datasets/star_schema.md b/docs/en/getting_started/example_datasets/star_schema.md index 5606cf71dd1..a6281b88b92 100644 --- a/docs/en/getting_started/example_datasets/star_schema.md +++ b/docs/en/getting_started/example_datasets/star_schema.md @@ -10,7 +10,7 @@ make There will be some warnings during the process, but this is normal. -Place `dbgen` and `dists.dss` in any location with 800 GB of free disk space. +Place `dbgen` and `dists.dss` in any location with 800 GB of free disk space. Generating data: @@ -73,7 +73,7 @@ CREATE TABLE partd AS part ENGINE = Distributed(perftest_3shards_1replicas, defa ``` For testing on a single server, just use MergeTree tables. -For distributed testing, you need to configure the `perftest_3shards_1replicas` cluster in the config file. +For distributed testing, you need to configure the `perftest_3shards_1replicas` cluster in the config file. Next, create MergeTree tables on each server and a Distributed above them. Downloading data (change 'customer' to 'customerd' in the distributed version): diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index caffafe61ca..b3c3dd97b24 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -79,7 +79,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va ### Command Line Options -- `--host, -h` -– The server name, 'localhost' by default. You can use either the name or the IPv4 or IPv6 address. +- `--host, -h` -– The server name, 'localhost' by default. You can use either the name or the IPv4 or IPv6 address. - `--port` – The port to connect to. Default value: 9000. Note that the HTTP interface and the native interface use different ports. - `--user, -u` – The username. Default value: default. - `--password` – The password. Default value: empty string. @@ -95,7 +95,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va ### Configuration Files -`clickhouse-client` uses the first existing file of the following: +`clickhouse-client` uses the first existing file of the following: - Defined in the `-config-file` parameter. - `./clickhouse-client.xml` diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 83a0a320f18..c47e75142df 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -74,7 +74,7 @@ During a read operation, incorrect dates and dates with times can be parsed with As an exception, parsing dates with times is also supported in Unix timestamp format, if it consists of exactly 10 decimal digits. The result is not time zone-dependent. The formats YYYY-MM-DD hh:mm:ss and NNNNNNNNNN are differentiated automatically. -Strings are output with backslash-escaped special characters. The following escape sequences are used for output: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\'`, `\\`. Parsing also supports the sequences `\a`, `\v`, and `\xHH` (hex escape sequences) and any `\c` sequences, where `c` is any character (these sequences are converted to `c`). Thus, reading data supports formats where a line feed can be written as `\n` or `\`, or as a line feed. For example, the string `Hello world` with a line feed between the words instead of a space can be parsed in any of the following variations: +Strings are output with backslash-escaped special characters. The following escape sequences are used for output: `\b`, `\f`, `\r`, `\n`, `\t`, `\0`, `\'`, `\\`. Parsing also supports the sequences `\a`, `\v`, and `\xHH` (hex escape sequences) and any `\c` sequences, where `c` is any character (these sequences are converted to `c`). Thus, reading data supports formats where a line feed can be written as `\n` or `\`, or as a line feed. For example, the string `Hello world` with a line feed between the words instead of a space can be parsed in any of the following variations: ``` Hello\nworld @@ -242,7 +242,7 @@ SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase WITH TOTA } ``` -The JSON is compatible with JavaScript. To ensure this, some characters are additionally escaped: the slash `/` is escaped as `\/`; alternative line breaks `U+2028` and `U+2029`, which break some browsers, are escaped as `\uXXXX`. ASCII control characters are escaped: backspace, form feed, line feed, carriage return, and horizontal tab are replaced with `\b`, `\f`, `\n`, `\r`, `\t` , as well as the remaining bytes in the 00-1F range using `\uXXXX` sequences. Invalid UTF-8 sequences are changed to the replacement character � so the output text will consist of valid UTF-8 sequences. For compatibility with JavaScript, Int64 and UInt64 integers are enclosed in double quotes by default. To remove the quotes, you can set the configuration parameter output_format_json_quote_64bit_integers to 0. +The JSON is compatible with JavaScript. To ensure this, some characters are additionally escaped: the slash `/` is escaped as `\/`; alternative line breaks `U+2028` and `U+2029`, which break some browsers, are escaped as `\uXXXX`. ASCII control characters are escaped: backspace, form feed, line feed, carriage return, and horizontal tab are replaced with `\b`, `\f`, `\n`, `\r`, `\t` , as well as the remaining bytes in the 00-1F range using `\uXXXX` sequences. Invalid UTF-8 sequences are changed to the replacement character � so the output text will consist of valid UTF-8 sequences. For compatibility with JavaScript, Int64 and UInt64 integers are enclosed in double quotes by default. To remove the quotes, you can set the configuration parameter output_format_json_quote_64bit_integers to 0. `rows` – The total number of output rows. @@ -255,7 +255,7 @@ If the query contains GROUP BY, rows_before_limit_at_least is the exact number o This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). -ClickHouse supports [NULL](../query_language/syntax.md), which is displayed as `null` in the JSON output. +ClickHouse supports [NULL](../query_language/syntax.md), which is displayed as `null` in the JSON output. See also the JSONEachRow format. diff --git a/docs/en/introduction/ya_metrika_task.md b/docs/en/introduction/ya_metrika_task.md index 9acbdc0989e..41b33eff581 100644 --- a/docs/en/introduction/ya_metrika_task.md +++ b/docs/en/introduction/ya_metrika_task.md @@ -1,6 +1,6 @@ # Yandex.Metrica Use Case -ClickHouse was originally developed to power [Yandex.Metrica](https://metrica.yandex.com/), [the second largest web analytics platform in the world](http://w3techs.com/technologies/overview/traffic_analysis/all), and continues to be the core component of this system. With more than 13 trillion records in the database and more than 20 billion events daily, ClickHouse allows generating custom reports on the fly directly from non-aggregated data. This article briefly covers the goals of ClickHouse in the early stages of its development. +ClickHouse was originally developed to power [Yandex.Metrica](https://metrica.yandex.com/), [the second largest web analytics platform in the world](http://w3techs.com/technologies/overview/traffic_analysis/all), and continues to be the core component of this system. With more than 13 trillion records in the database and more than 20 billion events daily, ClickHouse allows generating custom reports on the fly directly from non-aggregated data. This article briefly covers the goals of ClickHouse in the early stages of its development. Yandex.Metrica builds customized reports on the fly based on hits and sessions, with arbitrary segments defined by the user. This often requires building complex aggregates, such as the number of unique users. New data for building a report is received in real time. diff --git a/docs/en/operations/configuration_files.md b/docs/en/operations/configuration_files.md index 43a062e262f..2184bb95122 100644 --- a/docs/en/operations/configuration_files.md +++ b/docs/en/operations/configuration_files.md @@ -12,7 +12,7 @@ If `replace` is specified, it replaces the entire element with the specified one If `remove` is specified, it deletes the element. -The config can also define "substitutions". If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](server_settings/settings.md#server_settings-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros](server_settings/settings.md)). +The config can also define "substitutions". If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](server_settings/settings.md#server_settings-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros](server_settings/settings.md)). Substitutions can also be performed from ZooKeeper. To do this, specify the attribute `from_zk = "/path/to/node"`. The element value is replaced with the contents of the node at `/path/to/node` in ZooKeeper. You can also put an entire XML subtree on the ZooKeeper node and it will be fully inserted into the source element. diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index 6a7ff2c0e52..76fe4a62bb1 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -42,7 +42,7 @@ Block field ``: - ``min_part_size_ratio`` – The ratio of the minimum size of a table part to the full size of the table. - ``method`` – Compression method. Acceptable values ​: ``lz4`` or ``zstd``(experimental). -ClickHouse checks `min_part_size` and `min_part_size_ratio` and processes the `case` blocks that match these conditions. If none of the `` matches, ClickHouse applies the `lz4` compression algorithm. +ClickHouse checks `min_part_size` and `min_part_size_ratio` and processes the `case` blocks that match these conditions. If none of the `` matches, ClickHouse applies the `lz4` compression algorithm. **Example** @@ -456,7 +456,7 @@ Keys for server/client settings: - requireTLSv1 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips – Activates OpenSSL FIPS mode. Supported if the library's OpenSSL version supports FIPS. - privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ````, ``KeyFileHandler``, ``test``, ````. -- invalidCertificateHandler – Class (subclass of CertificateHandler) for verifying invalid certificates. For example: `` ConsoleCertificateHandler `` . +- invalidCertificateHandler – Class (subclass of CertificateHandler) for verifying invalid certificates. For example: `` ConsoleCertificateHandler `` . - disableProtocols – Protocols that are not allowed to use. - preferServerCiphers – Preferred server ciphers on the client. @@ -641,7 +641,7 @@ The uncompressed cache is advantageous for very short queries in individual case ## user_files_path {#server_settings-user_files_path} -The directory with user files. Used in the table function [file()](../../query_language/table_functions/file.md). +The directory with user files. Used in the table function [file()](../../query_language/table_functions/file.md). **Example** diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 59ba8189261..add73212c5d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -28,7 +28,7 @@ Forces a query to an out-of-date replica if updated data is not available. See " ClickHouse selects the most relevant from the outdated replicas of the table. -Used when performing `SELECT` from a distributed table that points to replicated tables. +Used when performing `SELECT` from a distributed table that points to replicated tables. By default, 1 (enabled). @@ -38,7 +38,7 @@ Disables query execution if the index can't be used by date. Works with tables in the MergeTree family. -If `force_index_by_date=1`, ClickHouse checks whether the query has a date key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For example, the condition `Date != ' 2000-01-01 '` is acceptable even when it matches all the data in the table (i.e., running the query requires a full scan). For more information about ranges of data in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". +If `force_index_by_date=1`, ClickHouse checks whether the query has a date key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For example, the condition `Date != ' 2000-01-01 '` is acceptable even when it matches all the data in the table (i.e., running the query requires a full scan). For more information about ranges of data in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". ## force_primary_key @@ -47,7 +47,7 @@ Disables query execution if indexing by the primary key is not possible. Works with tables in the MergeTree family. -If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For more information about data ranges in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". +If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For more information about data ranges in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". ## fsync_metadata @@ -141,7 +141,7 @@ Sets the time in seconds. If a replica lags more than the set value, this replic Default value: 0 (off). -Used when performing `SELECT` from a distributed table that points to replicated tables. +Used when performing `SELECT` from a distributed table that points to replicated tables. ## max_threads {#settings-max_threads} @@ -267,7 +267,7 @@ This parameter is useful when you are using formats that require a schema defini ## stream_flush_interval_ms -Works for tables with streaming in the case of a timeout, or when a thread generates[max_insert_block_size](#settings-max_insert_block_size) rows. +Works for tables with streaming in the case of a timeout, or when a thread generates [max_insert_block_size](#settings-max_insert_block_size) rows. The default value is 7500. @@ -326,7 +326,7 @@ Replica lag is not controlled. Enable compilation of queries. By default, 0 (disabled). Compilation is only used for part of the query-processing pipeline: for the first stage of aggregation (GROUP BY). -If this portion of the pipeline was compiled, the query may run faster due to deployment of short cycles and inlining aggregate function calls. The maximum performance improvement (up to four times faster in rare cases) is seen for queries with multiple simple aggregate functions. Typically, the performance gain is insignificant. In very rare cases, it may slow down query execution. +If this portion of the pipeline was compiled, the query may run faster due to deployment of short cycles and inlining aggregate function calls. The maximum performance improvement (up to four times faster in rare cases) is seen for queries with multiple simple aggregate functions. Typically, the performance gain is insignificant. In very rare cases, it may slow down query execution. ## min_count_to_compile @@ -344,7 +344,7 @@ It works for JSONEachRow and TSKV formats. ## output_format_json_quote_64bit_integers -If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes. +If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes. ## format_csv_delimiter {#settings-format_csv_delimiter} @@ -373,7 +373,7 @@ The default value is 0. All the replicas in the quorum are consistent, i.e., they contain data from all previous `INSERT` queries. The `INSERT` sequence is linearized. -When reading the data written from the `insert_quorum`, you can use the[select_sequential_consistency](#select-sequential-consistency) option. +When reading the data written from the `insert_quorum`, you can use the [select_sequential_consistency](#select-sequential-consistency) option. **ClickHouse generates an exception** diff --git a/docs/en/operations/settings/settings_profiles.md b/docs/en/operations/settings/settings_profiles.md index 3953ada8a83..c335e249212 100644 --- a/docs/en/operations/settings/settings_profiles.md +++ b/docs/en/operations/settings/settings_profiles.md @@ -2,7 +2,7 @@ # Settings profiles A settings profile is a collection of settings grouped under the same name. Each ClickHouse user has a profile. -To apply all the settings in a profile, set the `profile` setting. +To apply all the settings in a profile, set the `profile` setting. Example: @@ -58,9 +58,9 @@ Example: ``` -The example specifies two profiles: `default` and `web`. The `default` profile has a special purpose: it must always be present and is applied when starting the server. In other words, the `default` profile contains default settings. The `web` profile is a regular profile that can be set using the `SET` query or using a URL parameter in an HTTP query. +The example specifies two profiles: `default` and `web`. The `default` profile has a special purpose: it must always be present and is applied when starting the server. In other words, the `default` profile contains default settings. The `web` profile is a regular profile that can be set using the `SET` query or using a URL parameter in an HTTP query. -Settings profiles can inherit from each other. To use inheritance, indicate the `profile` setting before the other settings that are listed in the profile. +Settings profiles can inherit from each other. To use inheritance, indicate the `profile` setting before the other settings that are listed in the profile. [Original article](https://clickhouse.yandex/docs/en/operations/settings/settings_profiles/) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index b41f7f02b95..d15d392d5f9 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -141,7 +141,7 @@ Formats: - active (UInt8) – Indicates whether the part is active. If a part is active, it is used in a table; otherwise, it will be deleted. Inactive data parts remain after merging. -- marks (UInt64) – The number of marks. To get the approximate number of rows in a data part, multiply ``marks`` by the index granularity (usually 8192). +- marks (UInt64) – The number of marks. To get the approximate number of rows in a data part, multiply ``marks`` by the index granularity (usually 8192). - marks_size (UInt64) – The size of the file with marks. @@ -373,7 +373,7 @@ If the path specified in 'path' doesn't exist, an exception will be thrown. Columns: -- `name String` — The name of the node. +- `name String` — The name of the node. - `path String` — The path to the node. - `value String` — Node value. - `dataLength Int32` — Size of the value. diff --git a/docs/en/operations/table_engines/custom_partitioning_key.md b/docs/en/operations/table_engines/custom_partitioning_key.md index a674bc5533f..23e275b7925 100644 --- a/docs/en/operations/table_engines/custom_partitioning_key.md +++ b/docs/en/operations/table_engines/custom_partitioning_key.md @@ -36,7 +36,7 @@ In the `system.parts` table, the `partition` column specifies the value of the p Old: `20140317_20140323_2_2_0` (minimum date - maximum date - minimum block number - maximum block number - level). -Now: `201403_2_2_0` (partition ID - minimum block number - maximum block number - level). +Now: `201403_2_2_0` (partition ID - minimum block number - maximum block number - level). The partition ID is its string identifier (human-readable, if possible) that is used for the names of data parts in the file system and in ZooKeeper. You can specify it in ALTER queries in place of the partition key. Example: Partition key `toYYYYMM(EventDate)`; ALTER can specify either `PARTITION 201710` or `PARTITION ID '201710'`. diff --git a/docs/en/operations/table_engines/kafka.md b/docs/en/operations/table_engines/kafka.md index d668def503e..49c42fb0a16 100644 --- a/docs/en/operations/table_engines/kafka.md +++ b/docs/en/operations/table_engines/kafka.md @@ -39,7 +39,7 @@ Required parameters: Optional parameters: - `kafka_row_delimiter` - Character-delimiter of records (rows), which ends the message. -- `kafka_schema` – An optional parameter that must be used if the format requires a schema definition. For example, [Cap'n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object. +- `kafka_schema` – An optional parameter that must be used if the format requires a schema definition. For example, [Cap'n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object. - `kafka_num_consumers` – The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. The total number of consumers should not exceed the number of partitions in the topic, since only one consumer can be assigned per partition. Examples: @@ -122,7 +122,7 @@ If you want to change the target table by using `ALTER`, we recommend disabling Similar to GraphiteMergeTree, the Kafka engine supports extended configuration using the ClickHouse config file. There are two configuration keys that you can use: global (`kafka`) and topic-level (`kafka_*`). The global configuration is applied first, and then the topic-level configuration is applied (if it exists). ```xml - + cgrp smallest diff --git a/docs/en/operations/table_engines/merge.md b/docs/en/operations/table_engines/merge.md index e8de53bc286..366a5459bf8 100644 --- a/docs/en/operations/table_engines/merge.md +++ b/docs/en/operations/table_engines/merge.md @@ -1,8 +1,8 @@ # Merge -The `Merge` engine (not to be confused with `MergeTree`) does not store data itself, but allows reading from any number of other tables simultaneously. +The `Merge` engine (not to be confused with `MergeTree`) does not store data itself, but allows reading from any number of other tables simultaneously. Reading is automatically parallelized. Writing to a table is not supported. When reading, the indexes of tables that are actually being read are used, if they exist. -The `Merge` engine accepts parameters: the database name and a regular expression for tables. +The `Merge` engine accepts parameters: the database name and a regular expression for tables. Example: @@ -10,17 +10,17 @@ Example: Merge(hits, '^WatchLog') ``` -Data will be read from the tables in the `hits` database that have names that match the regular expression '`^WatchLog`'. +Data will be read from the tables in the `hits` database that have names that match the regular expression '`^WatchLog`'. Instead of the database name, you can use a constant expression that returns a string. For example, `currentDatabase()`. Regular expressions — [re2](https://github.com/google/re2) (supports a subset of PCRE), case-sensitive. See the notes about escaping symbols in regular expressions in the "match" section. -When selecting tables to read, the `Merge` table itself will not be selected, even if it matches the regex. This is to avoid loops. -It is possible to create two `Merge` tables that will endlessly try to read each others' data, but this is not a good idea. +When selecting tables to read, the `Merge` table itself will not be selected, even if it matches the regex. This is to avoid loops. +It is possible to create two `Merge` tables that will endlessly try to read each others' data, but this is not a good idea. -The typical way to use the `Merge` engine is for working with a large number of `TinyLog` tables as if with a single table. +The typical way to use the `Merge` engine is for working with a large number of `TinyLog` tables as if with a single table. Example 2: @@ -63,7 +63,7 @@ Virtual columns differ from normal columns in the following ways: The `Merge` type table contains a virtual `_table` column of the `String` type. (If the table already has a `_table` column, the virtual column is called `_table1`; if you already have `_table1`, it's called `_table2`, and so on.) It contains the name of the table that data was read from. -If the `WHERE/PREWHERE` clause contains conditions for the `_table` column that do not depend on other table columns (as one of the conjunction elements, or as an entire expression), these conditions are used as an index. The conditions are performed on a data set of table names to read data from, and the read operation will be performed from only those tables that the condition was triggered on. +If the `WHERE/PREWHERE` clause contains conditions for the `_table` column that do not depend on other table columns (as one of the conjunction elements, or as an entire expression), these conditions are used as an index. The conditions are performed on a data set of table names to read data from, and the read operation will be performed from only those tables that the condition was triggered on. [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/merge/) diff --git a/docs/en/operations/table_engines/mergetree.md b/docs/en/operations/table_engines/mergetree.md index 14e5fd19fe2..abac921f9df 100644 --- a/docs/en/operations/table_engines/mergetree.md +++ b/docs/en/operations/table_engines/mergetree.md @@ -61,7 +61,7 @@ For a description of request parameters, see [request description](../../query_l By default the primary key is the same as the sorting key (which is specified by the `ORDER BY` clause). Thus in most cases it is unnecessary to specify a separate `PRIMARY KEY` clause. -- `SAMPLE BY` — An expression for sampling. +- `SAMPLE BY` — An expression for sampling. If a sampling expression is used, the primary key must contain it. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. diff --git a/docs/en/operations/table_engines/replacingmergetree.md b/docs/en/operations/table_engines/replacingmergetree.md index a147e9c8dd6..53b9a47c5c2 100644 --- a/docs/en/operations/table_engines/replacingmergetree.md +++ b/docs/en/operations/table_engines/replacingmergetree.md @@ -4,7 +4,7 @@ The engine differs from [MergeTree](mergetree.md#table_engines-mergetree) in tha Data deduplication occurs only during a merge. Merging occurs in the background at an unknown time, so you can't plan for it. Some of the data may remain unprocessed. Although you can run an unscheduled merge using the `OPTIMIZE` query, don't count on using it, because the `OPTIMIZE` query will read and write a large amount of data. -Thus, `ReplacingMergeTree` is suitable for clearing out duplicate data in the background in order to save space, but it doesn't guarantee the absence of duplicates. +Thus, `ReplacingMergeTree` is suitable for clearing out duplicate data in the background in order to save space, but it doesn't guarantee the absence of duplicates. ## Creating a Table @@ -33,7 +33,7 @@ For a description of request parameters, see [request description](../../query_l **Query clauses** -When creating a `ReplacingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table. +When creating a `ReplacingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table.
Deprecated Method for Creating a Table diff --git a/docs/en/operations/table_engines/summingmergetree.md b/docs/en/operations/table_engines/summingmergetree.md index efea8da74be..32444b61c1a 100644 --- a/docs/en/operations/table_engines/summingmergetree.md +++ b/docs/en/operations/table_engines/summingmergetree.md @@ -1,9 +1,9 @@ # SummingMergeTree -The engine inherits from [MergeTree](mergetree.md#table_engines-mergetree). The difference is that when merging data parts for `SummingMergeTree` tables ClickHouse replaces all the rows with the same primary key (or more accurately, with the same [sorting key](mergetree.md)) with one row which contains summarized values for the columns with the numeric data type. If the sorting key is composed in a way that a single key value corresponds to large number of rows, this significantly reduces storage volume and speeds up data selection. +The engine inherits from [MergeTree](mergetree.md#table_engines-mergetree). The difference is that when merging data parts for `SummingMergeTree` tables ClickHouse replaces all the rows with the same primary key (or more accurately, with the same [sorting key](mergetree.md)) with one row which contains summarized values for the columns with the numeric data type. If the sorting key is composed in a way that a single key value corresponds to large number of rows, this significantly reduces storage volume and speeds up data selection. -We recommend to use the engine together with `MergeTree`. Store complete data in `MergeTree` table, and use `SummingMergeTree` for aggregated data storing, for example, when preparing reports. Such an approach will prevent you from losing valuable data due to an incorrectly composed primary key. +We recommend to use the engine together with `MergeTree`. Store complete data in `MergeTree` table, and use `SummingMergeTree` for aggregated data storing, for example, when preparing reports. Such an approach will prevent you from losing valuable data due to an incorrectly composed primary key. ## Creating a Table @@ -31,7 +31,7 @@ The columns must be of a numeric type and must not be in the primary key. **Query clauses** -When creating a `SummingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table. +When creating a `SummingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table.
Deprecated Method for Creating a Table @@ -72,7 +72,7 @@ Insert data to it: :) INSERT INTO summtt Values(1,1),(1,2),(2,1) ``` -ClickHouse may sum all the rows not completely ([see below](#data-processing)), so we use an aggregate function `sum` and `GROUP BY` clause in the query. +ClickHouse may sum all the rows not completely ([see below](#data-processing)), so we use an aggregate function `sum` and `GROUP BY` clause in the query. ```sql SELECT key, sum(value) FROM summtt GROUP BY key diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index bd058afb6dd..e3b66a639ba 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -18,7 +18,7 @@ You can use `turbostat` to view the CPU's actual clock rate under a load. ## CPU Scaling Governor -Always use the `performance` scaling governor. The `on-demand` scaling governor works much worse with constantly high demand. +Always use the `performance` scaling governor. The `on-demand` scaling governor works much worse with constantly high demand. ```bash sudo echo 'performance' | tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor diff --git a/docs/en/query_language/agg_functions/parametric_functions.md b/docs/en/query_language/agg_functions/parametric_functions.md index 5a49b077089..1cbe784e621 100644 --- a/docs/en/query_language/agg_functions/parametric_functions.md +++ b/docs/en/query_language/agg_functions/parametric_functions.md @@ -37,7 +37,7 @@ Pattern syntax: Any quantity of any type of events is allowed over the specified time. -Instead of `>=`, the following operators can be used:`<`, `>`, `<=`. +Instead of `>=`, the following operators can be used:`<`, `>`, `<=`. Any number may be specified in place of 1800. diff --git a/docs/en/query_language/agg_functions/reference.md b/docs/en/query_language/agg_functions/reference.md index 882371dedc0..782be3374b1 100644 --- a/docs/en/query_language/agg_functions/reference.md +++ b/docs/en/query_language/agg_functions/reference.md @@ -295,7 +295,7 @@ Uses the [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) algorithm to a The result is determinate (it doesn't depend on the order of query processing). -We don't recommend using this function. In most cases, use the `uniq` or `uniqCombined` function. +We don't recommend using this function. In most cases, use the `uniq` or `uniqCombined` function. ## uniqExact(x) @@ -382,8 +382,8 @@ For its purpose (calculating quantiles of page loading times), using this functi ## quantileTimingWeighted(level)(x, weight) -Differs from the `quantileTiming` function in that it has a second argument, "weights". Weight is a non-negative integer. -The result is calculated as if the `x` value were passed `weight` number of times to the `quantileTiming` function. +Differs from the `quantileTiming` function in that it has a second argument, "weights". Weight is a non-negative integer. +The result is calculated as if the `x` value were passed `weight` number of times to the `quantileTiming` function. ## quantileExact(level)(x) @@ -437,7 +437,7 @@ The result is equal to the square root of `varPop(x)`. Returns an array of the most frequent values in the specified column. The resulting array is sorted in descending order of frequency of values (not by the values themselves). -Implements the [ Filtered Space-Saving](http://www.l2f.inesc-id.pt/~fmmb/wiki/uploads/Work/misnis.ref0a.pdf) algorithm for analyzing TopK, based on the reduce-and-combine algorithm from [Parallel Space Saving](https://arxiv.org/pdf/1401.0702.pdf). +Implements the [ Filtered Space-Saving](http://www.l2f.inesc-id.pt/~fmmb/wiki/uploads/Work/misnis.ref0a.pdf) algorithm for analyzing TopK, based on the reduce-and-combine algorithm from [Parallel Space Saving](https://arxiv.org/pdf/1401.0702.pdf). ``` topK(N)(column) diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index 4d146f55cbf..c3d504f07bb 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -54,7 +54,7 @@ If there is a failure during one of the successive stages, data can be restored The `ALTER` query lets you create and delete separate elements (columns) in nested data structures, but not whole nested data structures. To add a nested data structure, you can add columns with a name like `name.nested_name` and the type `Array(T)`. A nested data structure is equivalent to multiple array columns with a name that has the same prefix before the dot. -There is no support for deleting columns in the primary key or the sampling key (columns that are in the `ENGINE` expression). Changing the type for columns that are included in the primary key is only possible if this change does not cause the data to be modified (for example, it is allowed to add values to an Enum or change a type with `DateTime` to `UInt32`). +There is no support for deleting columns in the primary key or the sampling key (columns that are in the `ENGINE` expression). Changing the type for columns that are included in the primary key is only possible if this change does not cause the data to be modified (for example, it is allowed to add values to an Enum or change a type with `DateTime` to `UInt32`). If the `ALTER` query is not sufficient for making the table changes you need, you can create a new table, copy the data to it using the `INSERT SELECT` query, then switch the tables using the `RENAME` query and delete the old table. @@ -115,10 +115,10 @@ Data directory: `/var/lib/clickhouse/data/database/table/`,where `/var/lib/click ```bash $ ls -l /var/lib/clickhouse/data/test/visits/ total 48 -drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_2_2_0 -drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_4_4_0 -drwxrwxrwx 2 clickhouse clickhouse 4096 May 5 02:55 detached --rw-rw-rw- 1 clickhouse clickhouse 2 May 5 02:58 increment.txt +drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_2_2_0 +drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_4_4_0 +drwxrwxrwx 2 clickhouse clickhouse 4096 May 5 02:55 detached +-rw-rw-rw- 1 clickhouse clickhouse 2 May 5 02:58 increment.txt ``` Here, `20140317_20140323_2_2_0` and ` 20140317_20140323_4_4_0` are the directories of data parts. @@ -193,7 +193,7 @@ The `ALTER ... FREEZE PARTITION` query is not replicated. A local backup is only As an alternative, you can manually copy data from the `/var/lib/clickhouse/data/database/table` directory. But if you do this while the server is running, race conditions are possible when copying directories with files being added or changed, and the backup may be inconsistent. You can do this if the server isn't running – then the resulting data will be the same as after the `ALTER TABLE t FREEZE PARTITION` query. -`ALTER TABLE ... FREEZE PARTITION` only copies data, not table metadata. To make a backup of table metadata, copy the file `/var/lib/clickhouse/metadata/database/table.sql` +`ALTER TABLE ... FREEZE PARTITION` only copies data, not table metadata. To make a backup of table metadata, copy the file `/var/lib/clickhouse/metadata/database/table.sql` To restore from a backup: @@ -216,7 +216,7 @@ Although the query is called `ALTER TABLE`, it does not change the table structu Data is placed in the `detached` directory. You can use the `ALTER TABLE ... ATTACH` query to attach the data. -The ` FROM` clause specifies the path in ` ZooKeeper`. For example, `/clickhouse/tables/01-01/visits`. +The ` FROM` clause specifies the path in ` ZooKeeper`. For example, `/clickhouse/tables/01-01/visits`. Before downloading, the system checks that the partition exists and the table structure matches. The most appropriate replica is selected automatically from the healthy replicas. The `ALTER ... FETCH PARTITION` query is not replicated. The partition will be downloaded to the 'detached' directory only on the local server. Note that if after this you use the `ALTER TABLE ... ATTACH` query to add data to the table, the data will be added on all replicas (on one of the replicas it will be added from the 'detached' directory, and on the rest it will be loaded from neighboring replicas). diff --git a/docs/en/query_language/dicts/external_dicts_dict_layout.md b/docs/en/query_language/dicts/external_dicts_dict_layout.md index e3b6498c9a9..03279688d6c 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_layout.md +++ b/docs/en/query_language/dicts/external_dicts_dict_layout.md @@ -95,7 +95,7 @@ Configuration example: The dictionary is stored in memory in the form of a hash table with an ordered array of ranges and their corresponding values. -This storage method works the same way as hashed and allows using date/time ranges in addition to the key, if they appear in the dictionary. +This storage method works the same way as hashed and allows using date/time ranges in addition to the key, if they appear in the dictionary. Example: The table contains discounts for each advertiser in the format: diff --git a/docs/en/query_language/dicts/external_dicts_dict_lifetime.md b/docs/en/query_language/dicts/external_dicts_dict_lifetime.md index 4ab8ba4ea20..cbd78da16ad 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_lifetime.md +++ b/docs/en/query_language/dicts/external_dicts_dict_lifetime.md @@ -15,7 +15,7 @@ Example of settings: ``` -Setting ` 0 ` prevents updating dictionaries. +Setting ` 0 ` prevents updating dictionaries. You can set a time interval for upgrades, and ClickHouse will choose a uniformly random time within this range. This is necessary in order to distribute the load on the dictionary source when upgrading on a large number of servers. diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index 656abc1ea01..3a16db67e8c 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -243,7 +243,7 @@ This function can also be used in higher-order functions. For example, you can u ## arrayEnumerateUniq(arr, ...) Returns an array the same size as the source array, indicating for each element what its position is among elements with the same value. -For example: arrayEnumerateUniq(\[10, 20, 10, 30\]) = \[1, 1, 2, 1\]. +For example: arrayEnumerateUniq(\[10, 20, 10, 30\]) = \[1, 1, 2, 1\]. This function is useful when using ARRAY JOIN and aggregation of array elements. Example: @@ -378,7 +378,7 @@ arrayPushFront(array, single_value) **Parameters** - `array` – Array. -- `single_value` – A single value. Only numbers can be added to an array with numbers, and only strings can be added to an array of strings. When adding numbers, ClickHouse automatically sets the `single_value` type for the data type of the array. For more information about the types of data in ClickHouse, see "[Data types](../../data_types/index.md#data_types)". Can be `NULL`. The function adds a `NULL` element to an array, and the type of array elements converts to `Nullable`. +- `single_value` – A single value. Only numbers can be added to an array with numbers, and only strings can be added to an array of strings. When adding numbers, ClickHouse automatically sets the `single_value` type for the data type of the array. For more information about the types of data in ClickHouse, see "[Data types](../../data_types/index.md#data_types)". Can be `NULL`. The function adds a `NULL` element to an array, and the type of array elements converts to `Nullable`. **Example** diff --git a/docs/en/query_language/functions/ext_dict_functions.md b/docs/en/query_language/functions/ext_dict_functions.md index 6d0a8bb2615..99059d79016 100644 --- a/docs/en/query_language/functions/ext_dict_functions.md +++ b/docs/en/query_language/functions/ext_dict_functions.md @@ -16,7 +16,7 @@ For information on connecting and configuring external dictionaries, see "[Exter `dictGetT('dict_name', 'attr_name', id)` -- Get the value of the attr_name attribute from the dict_name dictionary using the 'id' key.`dict_name` and `attr_name` are constant strings.`id`must be UInt64. +- Get the value of the attr_name attribute from the dict_name dictionary using the 'id' key.`dict_name` and `attr_name` are constant strings.`id`must be UInt64. If there is no `id` key in the dictionary, it returns the default value specified in the dictionary description. ## dictGetTOrDefault diff --git a/docs/en/query_language/functions/functions_for_nulls.md b/docs/en/query_language/functions/functions_for_nulls.md index dcbc745066a..dd0594b8185 100644 --- a/docs/en/query_language/functions/functions_for_nulls.md +++ b/docs/en/query_language/functions/functions_for_nulls.md @@ -102,7 +102,7 @@ coalesce(x,...) **Returned values** -- The first non-`NULL` argument. +- The first non-`NULL` argument. - `NULL`, if all arguments are `NULL`. **Example** @@ -207,7 +207,7 @@ SELECT nullIf(1, 2) ## assumeNotNull -Results in a value of type [Nullable](../../data_types/nullable.md)i for a non- `Nullable`, if the value is not `NULL`. +Results in a value of type [Nullable](../../data_types/nullable.md) for a non- `Nullable`, if the value is not `NULL`. ``` assumeNotNull(x) diff --git a/docs/en/query_language/functions/geo.md b/docs/en/query_language/functions/geo.md index 13fc8673746..2c9a3aac38f 100644 --- a/docs/en/query_language/functions/geo.md +++ b/docs/en/query_language/functions/geo.md @@ -77,7 +77,7 @@ pointInPolygon((x, y), [(a, b), (c, d) ...], ...) **Input values** -- `(x, y)` — Coordinates of a point on the plane. Data type — [Tuple](../../data_types/tuple.md) — A tuple of two numbers. +- `(x, y)` — Coordinates of a point on the plane. Data type — [Tuple](../../data_types/tuple.md) — A tuple of two numbers. - `[(a, b), (c, d) ...]` — Polygon vertices. Data type — [Array](../../data_types/array.md). Each vertex is represented by a pair of coordinates `(a, b)`. Vertices should be specified in a clockwise or counterclockwise order. The minimum number of vertices is 3. The polygon must be constant. - The function also supports polygons with holes (cut out sections). In this case, add polygons that define the cut out sections using additional arguments of the function. The function does not support non-simply-connected polygons. diff --git a/docs/en/query_language/functions/other_functions.md b/docs/en/query_language/functions/other_functions.md index 85cd1dfe5c5..e49bedd8199 100644 --- a/docs/en/query_language/functions/other_functions.md +++ b/docs/en/query_language/functions/other_functions.md @@ -336,7 +336,7 @@ toColumnTypeName(value) **Returned values** -- A string with the name of the class that is used for representing the `value` data type in RAM. +- A string with the name of the class that is used for representing the `value` data type in RAM. **Example of the difference between` toTypeName ' and ' toColumnTypeName`** @@ -376,7 +376,7 @@ dumpColumnStructure(value) **Returned values** -- A string describing the structure that is used for representing the `value` data type in RAM. +- A string describing the structure that is used for representing the `value` data type in RAM. **Example** diff --git a/docs/en/query_language/functions/string_search_functions.md b/docs/en/query_language/functions/string_search_functions.md index 22af3c8550d..ea5b1ecf8ff 100644 --- a/docs/en/query_language/functions/string_search_functions.md +++ b/docs/en/query_language/functions/string_search_functions.md @@ -18,7 +18,7 @@ For a case-insensitive search, use the function `positionCaseInsensitiveUTF8`. ## match(haystack, pattern) -Checks whether the string matches the `pattern` regular expression. A `re2` regular expression. The [syntax](https://github.com/google/re2/wiki/Syntax) of the `re2` regular expressions is more limited than the syntax of the Perl regular expressions. +Checks whether the string matches the `pattern` regular expression. A `re2` regular expression. The [syntax](https://github.com/google/re2/wiki/Syntax) of the `re2` regular expressions is more limited than the syntax of the Perl regular expressions. Returns 0 if it doesn't match, or 1 if it matches. diff --git a/docs/en/query_language/functions/url_functions.md b/docs/en/query_language/functions/url_functions.md index efe48970338..d32c05aff57 100644 --- a/docs/en/query_language/functions/url_functions.md +++ b/docs/en/query_language/functions/url_functions.md @@ -34,7 +34,7 @@ For example, `cutToFirstSignificantSubdomain('https://news.yandex.com.tr/') = 'y ### path -Returns the path. Example: `/top/news.html` The path does not include the query string. +Returns the path. Example: `/top/news.html` The path does not include the query string. ### pathFull @@ -42,7 +42,7 @@ The same as above, but including query string and fragment. Example: /top/news.h ### queryString -Returns the query string. Example: page=1&lr=213. query-string does not include the initial question mark, as well as \# and everything after \#. +Returns the query string. Example: page=1&lr=213. query-string does not include the initial question mark, as well as \# and everything after \#. ### fragment diff --git a/docs/en/query_language/insert_into.md b/docs/en/query_language/insert_into.md index fc9421d3479..1d6d268e631 100644 --- a/docs/en/query_language/insert_into.md +++ b/docs/en/query_language/insert_into.md @@ -49,7 +49,7 @@ INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... Columns are mapped according to their position in the SELECT clause. However, their names in the SELECT expression and the table for INSERT may differ. If necessary, type casting is performed. -None of the data formats except Values allow setting values to expressions such as `now()`, `1 + 2`, and so on. The Values format allows limited use of expressions, but this is not recommended, because in this case inefficient code is used for their execution. +None of the data formats except Values allow setting values to expressions such as `now()`, `1 + 2`, and so on. The Values format allows limited use of expressions, but this is not recommended, because in this case inefficient code is used for their execution. Other queries for modifying data parts are not supported: `UPDATE`, `DELETE`, `REPLACE`, `MERGE`, `UPSERT`, `INSERT UPDATE`. However, you can delete old data using `ALTER TABLE ... DROP PARTITION`. diff --git a/docs/en/query_language/misc.md b/docs/en/query_language/misc.md index 01cf2c934c2..89ad8f3bca8 100644 --- a/docs/en/query_language/misc.md +++ b/docs/en/query_language/misc.md @@ -76,7 +76,7 @@ There is no `DETACH DATABASE` query. ## DROP -This query has two types: `DROP DATABASE` and `DROP TABLE`. +This query has two types: `DROP DATABASE` and `DROP TABLE`. ``` sql DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] diff --git a/docs/en/query_language/operators.md b/docs/en/query_language/operators.md index fc4d152bf40..61e008c2ba8 100644 --- a/docs/en/query_language/operators.md +++ b/docs/en/query_language/operators.md @@ -5,7 +5,7 @@ Groups of operators are listed in order of priority (the higher it is in the lis ## Access Operators -`a[N]` Access to an element of an array; ` arrayElement(a, N) function`. +`a[N]` Access to an element of an array; ` arrayElement(a, N) function`. `a.N` – Access to a tuble element; `tupleElement(a, N)` function. @@ -120,7 +120,7 @@ The following operators do not have a priority, since they are brackets: ## Associativity All binary operators have left associativity. For example, `1 + 2 + 3` is transformed to `plus(plus(1, 2), 3)`. -Sometimes this doesn't work the way you expect. For example, ` SELECT 4 > 2 > 3` will result in 0. +Sometimes this doesn't work the way you expect. For example, ` SELECT 4 > 2 > 3` will result in 0. For efficiency, the `and` and `or` functions accept any number of arguments. The corresponding chains of `AND` and `OR` operators are transformed to a single call of these functions. diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index 02a5cc3578d..a4aeec35ec9 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -575,11 +575,11 @@ When using `max_bytes_before_external_group_by`, we recommend that you set max_m For example, if `max_memory_usage` was set to 10000000000 and you want to use external aggregation, it makes sense to set `max_bytes_before_external_group_by` to 10000000000, and max_memory_usage to 20000000000. When external aggregation is triggered (if there was at least one dump of temporary data), maximum consumption of RAM is only slightly more than ` max_bytes_before_external_group_by`. -With distributed query processing, external aggregation is performed on remote servers. In order for the requestor server to use only a small amount of RAM, set ` distributed_aggregation_memory_efficient` to 1. +With distributed query processing, external aggregation is performed on remote servers. In order for the requestor server to use only a small amount of RAM, set ` distributed_aggregation_memory_efficient` to 1. When merging data flushed to the disk, as well as when merging results from remote servers when the ` distributed_aggregation_memory_efficient` setting is enabled, consumes up to 1/256 \* the number of threads from the total amount of RAM. -When external aggregation is enabled, if there was less than ` max_bytes_before_external_group_by` of data (i.e. data was not flushed), the query runs just as fast as without external aggregation. If any temporary data was flushed, the run time will be several times longer (approximately three times). +When external aggregation is enabled, if there was less than ` max_bytes_before_external_group_by` of data (i.e. data was not flushed), the query runs just as fast as without external aggregation. If any temporary data was flushed, the run time will be several times longer (approximately three times). If you have an ORDER BY with a small LIMIT after GROUP BY, then the ORDER BY CLAUSE will not use significant amounts of RAM. But if the ORDER BY doesn't have LIMIT, don't forget to enable external sorting (`max_bytes_before_external_sort`). @@ -693,7 +693,7 @@ The result will be the same as if GROUP BY were specified across all the fields DISTINCT is not supported if SELECT has at least one array column. -`DISTINCT` works with [NULL](syntax.md) as if `NULL` were a specific value, and `NULL=NULL`. In other words, in the `DISTINCT` results, different combinations with `NULL` only occur once. +`DISTINCT` works with [NULL](syntax.md) as if `NULL` were a specific value, and `NULL=NULL`. In other words, in the `DISTINCT` results, different combinations with `NULL` only occur once. ### LIMIT Clause @@ -813,7 +813,7 @@ A subquery in the IN clause is always run just one time on a single server. Ther #### NULL processing -During request processing, the IN operator assumes that the result of an operation with [NULL](syntax.md) is always equal to `0`, regardless of whether `NULL` is on the right or left side of the operator. `NULL` values are not included in any dataset, do not correspond to each other and cannot be compared. +During request processing, the IN operator assumes that the result of an operation with [NULL](syntax.md) is always equal to `0`, regardless of whether `NULL` is on the right or left side of the operator. `NULL` values are not included in any dataset, do not correspond to each other and cannot be compared. Here is an example with the `t_null` table: @@ -847,18 +847,18 @@ FROM t_null #### Distributed Subqueries {#select-distributed-subqueries} -There are two options for IN-s with subqueries (similar to JOINs): normal `IN` / `JOIN` and `GLOBAL IN` / `GLOBAL JOIN`. They differ in how they are run for distributed query processing. +There are two options for IN-s with subqueries (similar to JOINs): normal `IN` / `JOIN` and `GLOBAL IN` / `GLOBAL JOIN`. They differ in how they are run for distributed query processing. !!! attention Remember that the algorithms described below may work differently depending on the [settings](../operations/settings/settings.md) `distributed_product_mode` setting. When using the regular IN, the query is sent to remote servers, and each of them runs the subqueries in the `IN` or `JOIN` clause. -When using `GLOBAL IN` / `GLOBAL JOINs`, first all the subqueries are run for `GLOBAL IN` / `GLOBAL JOINs`, and the results are collected in temporary tables. Then the temporary tables are sent to each remote server, where the queries are run using this temporary data. +When using `GLOBAL IN` / `GLOBAL JOINs`, first all the subqueries are run for `GLOBAL IN` / `GLOBAL JOINs`, and the results are collected in temporary tables. Then the temporary tables are sent to each remote server, where the queries are run using this temporary data. For a non-distributed query, use the regular `IN` / `JOIN`. -Be careful when using subqueries in the `IN` / `JOIN` clauses for distributed query processing. +Be careful when using subqueries in the `IN` / `JOIN` clauses for distributed query processing. Let's look at some examples. Assume that each server in the cluster has a normal **local_table**. Each server also has a **distributed_table** table with the **Distributed** type, which looks at all the servers in the cluster. diff --git a/docs/en/query_language/syntax.md b/docs/en/query_language/syntax.md index d7307a64376..606df81a087 100644 --- a/docs/en/query_language/syntax.md +++ b/docs/en/query_language/syntax.md @@ -24,7 +24,7 @@ There may be any number of space symbols between syntactical constructions (incl SQL-style and C-style comments are supported. SQL-style comments: from `--` to the end of the line. The space after `--` can be omitted. -Comments in C-style: from `/*` to `*/`. These comments can be multiline. Spaces are not required here, either. +Comments in C-style: from `/*` to `*/`. These comments can be multiline. Spaces are not required here, either. ## Keywords diff --git a/docs/en/query_language/table_functions/url.md b/docs/en/query_language/table_functions/url.md index 662049aa0f9..edeabdc6902 100644 --- a/docs/en/query_language/table_functions/url.md +++ b/docs/en/query_language/table_functions/url.md @@ -4,7 +4,7 @@ `url(URL, format, structure)` - returns a table created from the `URL` with given `format` and `structure`. -URL - HTTP or HTTPS server address, which can accept `GET` and/or `POST` requests. +URL - HTTP or HTTPS server address, which can accept `GET` and/or `POST` requests. format - [format](../../interfaces/formats.md#formats) of the data. diff --git a/docs/fa/data_types/float.md b/docs/fa/data_types/float.md index 6c48b71d9a4..c83a4831ff3 100644 --- a/docs/fa/data_types/float.md +++ b/docs/fa/data_types/float.md @@ -7,7 +7,7 @@ Type های float در ClickHouse مشابه C می باشد: - `Float32` - `float` -- `Float64` - `double` +- `Float64` - `double` توصیه می کنیم که داده ها را هرزمان که امکان پذیره است به جای float به صورت int ذخیره کنید. برای مثال: تبدیل دقت اعداد به یک مقدار int، مثل سرعت page load در قالب میلی ثانیه. diff --git a/docs/fa/data_types/nested_data_structures/aggregatefunction.md b/docs/fa/data_types/nested_data_structures/aggregatefunction.md index d57a3eddba0..3ebcde1b6ae 100644 --- a/docs/fa/data_types/nested_data_structures/aggregatefunction.md +++ b/docs/fa/data_types/nested_data_structures/aggregatefunction.md @@ -2,7 +2,7 @@ # AggregateFunction(name, types_of_arguments...) -حالت متوسط از توابع aggregate. برای دریافت آن، از توابع aggregate به همراه پسوند '-State' استفاده کنید. برای اطلاعات بیشتر قسمت "AggregatingMergeTree" را ببینید. +حالت متوسط از توابع aggregate. برای دریافت آن، از توابع aggregate به همراه پسوند '-State' استفاده کنید. برای اطلاعات بیشتر قسمت "AggregatingMergeTree" را ببینید. [مقاله اصلی](https://clickhouse.yandex/docs/fa/data_types/nested_data_structures/aggregatefunction/) diff --git a/docs/fa/data_types/nested_data_structures/nested.md b/docs/fa/data_types/nested_data_structures/nested.md index 1f3b675eef1..a7a4be5b1ea 100644 --- a/docs/fa/data_types/nested_data_structures/nested.md +++ b/docs/fa/data_types/nested_data_structures/nested.md @@ -69,7 +69,7 @@ LIMIT 10
-ساده ترین راه برای فکر کردن به یک ساختار داده nestet این است که، یک nestet مجموعه ای از آرایه های چند ستونی با طول ثابت است. +ساده ترین راه برای فکر کردن به یک ساختار داده nestet این است که، یک nestet مجموعه ای از آرایه های چند ستونی با طول ثابت است. تنها جایی که یک دستور SELECT می تواند کل ساختار داده ی nested را به جای مشخص کردن ستون های آن قرار دهد، عبارت ARRAY JOIN است. برای اطلاعات بیشتر "ARRAY JOIN clouse" را ببینید. مثال: diff --git a/docs/fa/getting_started/example_datasets/amplab_benchmark.md b/docs/fa/getting_started/example_datasets/amplab_benchmark.md index 6ed6f97a344..6718b358f49 100644 --- a/docs/fa/getting_started/example_datasets/amplab_benchmark.md +++ b/docs/fa/getting_started/example_datasets/amplab_benchmark.md @@ -1,6 +1,6 @@
-# بنچمارک AMPLab Big Data +# بنچمارک AMPLab Big Data ببینید diff --git a/docs/fa/getting_started/example_datasets/nyc_taxi.md b/docs/fa/getting_started/example_datasets/nyc_taxi.md index 892b706e616..1f7c5c1a573 100644 --- a/docs/fa/getting_started/example_datasets/nyc_taxi.md +++ b/docs/fa/getting_started/example_datasets/nyc_taxi.md @@ -8,7 +8,7 @@ برای توضیحات بیشتر در ارتباط با دیتاست و موارد مربوط به دانلود به دو لینک و مراجعه کنید. -دانلود فایل ها حدود 277 گیگابایت داده ی غیرفشرده در قالب فایل های CSV می باشد. دانلود با استفاده ازبیش از یک کانکشن 1 Gbit نزدیک 1 ساعت طول می کشد (دانلود موازی از s3.amazonaws.com حداقل نصف کانال 1 Gbit رو جبران می کند). بعضی از فایل ها ممکن است به طول کامل دانلود نشوند. اندازه فایل ها را بررسی کنید و اگر فایلی مشکوک بود، مجددا دانلود کنید. +دانلود فایل ها حدود 277 گیگابایت داده ی غیرفشرده در قالب فایل های CSV می باشد. دانلود با استفاده ازبیش از یک کانکشن 1 Gbit نزدیک 1 ساعت طول می کشد (دانلود موازی از s3.amazonaws.com حداقل نصف کانال 1 Gbit رو جبران می کند). بعضی از فایل ها ممکن است به طول کامل دانلود نشوند. اندازه فایل ها را بررسی کنید و اگر فایلی مشکوک بود، مجددا دانلود کنید. بعضی از فایل ها ممکن است دارای سطرهای نامعتبر باشه. با اجرای دستورات زیر این موارد برطرف می شود: @@ -23,7 +23,7 @@ mv data/yellow_tripdata_2010-03.csv_ data/yellow_tripdata_2010-03.csv
-سپس داده ها باید در PostgreSQL پیش پردازش شوند. این کار نقاط انتخابی چند ضلعی را ایجاد می کند (برای مطابقت با نقاط بر روی نقشه با مناطق شهر نیویورک) و تمام داده ها را با استفاده از JOIN در یک جدول flat و denormal ترکیب می کند. برای این کار شما نیاز به نصب PostgreSQL با پشتیبانی از PostGIS دارید. +سپس داده ها باید در PostgreSQL پیش پردازش شوند. این کار نقاط انتخابی چند ضلعی را ایجاد می کند (برای مطابقت با نقاط بر روی نقشه با مناطق شهر نیویورک) و تمام داده ها را با استفاده از JOIN در یک جدول flat و denormal ترکیب می کند. برای این کار شما نیاز به نصب PostgreSQL با پشتیبانی از PostGIS دارید. در هنگام اجرای `initialize_database.sh` مراقب باشید و به صورت دستی مجددا تمام جداول را چک کنید. @@ -277,7 +277,7 @@ FROM trips
-این کار با سرعت 428 هزار رکورد در ثانیه و 3030 ثانیه طول خواهد کشید. برای load سریعتر، شما می توانید یک جدول با موتور `Log` به جای `MergeTree` بسازید. در این مورد، دانلود سریعتر از 200 ثانیه کار می کند. +این کار با سرعت 428 هزار رکورد در ثانیه و 3030 ثانیه طول خواهد کشید. برای load سریعتر، شما می توانید یک جدول با موتور `Log` به جای `MergeTree` بسازید. در این مورد، دانلود سریعتر از 200 ثانیه کار می کند. این جدول 126 گیابایت فضا بر روی دیسک اشغال می کند. @@ -355,7 +355,7 @@ Two Intel(R) Xeon(R) CPU E5-2650 v2 @ 2.60GHz, 16 physical kernels total,
``` -CREATE TABLE default.trips_mergetree_third ( trip_id UInt32, vendor_id Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14), pickup_date Date, pickup_datetime DateTime, dropoff_date Date, dropoff_datetime DateTime, store_and_fwd_flag UInt8, rate_code_id UInt8, pickup_longitude Float64, pickup_latitude Float64, dropoff_longitude Float64, dropoff_latitude Float64, passenger_count UInt8, trip_distance Float64, fare_amount Float32, extra Float32, mta_tax Float32, tip_amount Float32, tolls_amount Float32, ehail_fee Float32, improvement_surcharge Float32, total_amount Float32, payment_type_ Enum8('UNK' = 0, 'CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4), trip_type UInt8, pickup FixedString(25), dropoff FixedString(25), cab_type Enum8('yellow' = 1, 'green' = 2, 'uber' = 3), pickup_nyct2010_gid UInt8, pickup_ctlabel Float32, pickup_borocode UInt8, pickup_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), pickup_ct2010 FixedString(6), pickup_boroct2010 FixedString(7), pickup_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), pickup_ntacode FixedString(4), pickup_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), pickup_puma UInt16, dropoff_nyct2010_gid UInt8, dropoff_ctlabel Float32, dropoff_borocode UInt8, dropoff_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), dropoff_ct2010 FixedString(6), dropoff_boroct2010 FixedString(7), dropoff_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), dropoff_ntacode FixedString(4), dropoff_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), dropoff_puma UInt16) ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) +CREATE TABLE default.trips_mergetree_third ( trip_id UInt32, vendor_id Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14), pickup_date Date, pickup_datetime DateTime, dropoff_date Date, dropoff_datetime DateTime, store_and_fwd_flag UInt8, rate_code_id UInt8, pickup_longitude Float64, pickup_latitude Float64, dropoff_longitude Float64, dropoff_latitude Float64, passenger_count UInt8, trip_distance Float64, fare_amount Float32, extra Float32, mta_tax Float32, tip_amount Float32, tolls_amount Float32, ehail_fee Float32, improvement_surcharge Float32, total_amount Float32, payment_type_ Enum8('UNK' = 0, 'CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4), trip_type UInt8, pickup FixedString(25), dropoff FixedString(25), cab_type Enum8('yellow' = 1, 'green' = 2, 'uber' = 3), pickup_nyct2010_gid UInt8, pickup_ctlabel Float32, pickup_borocode UInt8, pickup_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), pickup_ct2010 FixedString(6), pickup_boroct2010 FixedString(7), pickup_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), pickup_ntacode FixedString(4), pickup_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), pickup_puma UInt16, dropoff_nyct2010_gid UInt8, dropoff_ctlabel Float32, dropoff_borocode UInt8, dropoff_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), dropoff_ct2010 FixedString(6), dropoff_boroct2010 FixedString(7), dropoff_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), dropoff_ntacode FixedString(4), dropoff_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), dropoff_puma UInt16) ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) ```
diff --git a/docs/fa/getting_started/example_datasets/ontime.md b/docs/fa/getting_started/example_datasets/ontime.md index 71c09531a8d..12f844f1181 100644 --- a/docs/fa/getting_started/example_datasets/ontime.md +++ b/docs/fa/getting_started/example_datasets/ontime.md @@ -197,7 +197,7 @@ Q4. تعداد تاخیرها براساس carrier در سال 78
``` sql -SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC +SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC ```
diff --git a/docs/fa/getting_started/example_datasets/star_schema.md b/docs/fa/getting_started/example_datasets/star_schema.md index 90bf5246e5c..fe7f861dd15 100644 --- a/docs/fa/getting_started/example_datasets/star_schema.md +++ b/docs/fa/getting_started/example_datasets/star_schema.md @@ -14,7 +14,7 @@ make
-در هنگام پردازش چند warnings نمایش داده می شود که مشکلی نیست و طبیعی است. +در هنگام پردازش چند warnings نمایش داده می شود که مشکلی نیست و طبیعی است. `dbgen` و ` dists.dss` را در یک جا با 800 گیگابایت فضای حالی دیسک قرار دهید. diff --git a/docs/fa/getting_started/index.md b/docs/fa/getting_started/index.md index 3fbe345b888..9189e0cabae 100644 --- a/docs/fa/getting_started/index.md +++ b/docs/fa/getting_started/index.md @@ -46,7 +46,7 @@ sudo apt-get install clickhouse-client clickhouse-server شما همچنین می توانید از طریق لینک زیر پکیج ClickHouse را به صورت دستی دانلود و نصب کنید: . -ClickHouse دارای تنظیمات محدودیت دسترسی می باشد. این تنظیمات در فایل 'users.xml' (کنار 'config.xml') می باشد. به صورت پیش فرض دسترسی برای کاربر 'default' از همه جا بدون نیاز به پسورد وجود دارد. 'user/default/networks' را مشاهده کنید. برای اطلاعات بیشتر قسمت "تنظیمات فایل ها" را مشاهده کنید. +ClickHouse دارای تنظیمات محدودیت دسترسی می باشد. این تنظیمات در فایل 'users.xml' (کنار 'config.xml') می باشد. به صورت پیش فرض دسترسی برای کاربر 'default' از همه جا بدون نیاز به پسورد وجود دارد. 'user/default/networks' را مشاهده کنید. برای اطلاعات بیشتر قسمت "تنظیمات فایل ها" را مشاهده کنید. ### نصب از طریق Source diff --git a/docs/fa/interfaces/cli.md b/docs/fa/interfaces/cli.md index 4b30ecbfb1b..8501f46ecd7 100644 --- a/docs/fa/interfaces/cli.md +++ b/docs/fa/interfaces/cli.md @@ -45,7 +45,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA در حالت intercative، شما یک command line برای درج query های خود دریافت می کنید. -اگر 'multiline' مشخص نشده باشد (به صورت پیش فرض): برای اجرای یک query، دکمه Enter را بزنید. سیمی کالن در انتهای query اجباری نیست. برای درج یک query چند خطی (multiline)، دکمه ی بک اسلش `\` را قبل از line feed فشار دهید. بعد از فشردن Enter، از شما برای درج خط بعدی query درخواست خواهد شد. +اگر 'multiline' مشخص نشده باشد (به صورت پیش فرض): برای اجرای یک query، دکمه Enter را بزنید. سیمی کالن در انتهای query اجباری نیست. برای درج یک query چند خطی (multiline)، دکمه ی بک اسلش `\` را قبل از line feed فشار دهید. بعد از فشردن Enter، از شما برای درج خط بعدی query درخواست خواهد شد. اگر چند خطی (multiline) مشخص شده باشد: برای اجرای query، در انتها سیمی کالن را وارد کنید و سپس Enter بزنید. اگر سیمی کالن از انتهای خط حذف می شد، از شما برای درج خط جدید query درخواست می شد. @@ -55,7 +55,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA command line برا پایه 'readline' (و 'history' یا 'libedit'، یه بدون کتابخانه بسته به build) می باشد. به عبارت دیگر، این محیط از shortcut های آشنا استفاده می کند و history دستورات را نگه می دار. history ها در فایل ~/.clickhouse-client-history نوشته می شوند. -به صورت پیش فرض فرمت خروجی PrettyCompact می باشد. شما میتوانید از طریق دستور FORMAT در یک query، یا با مشخص کردن `\G` در انتهای query، استفاده از آرگومان های `--format` یا `--vertical` یا از کانفیگ فایل کلاینت، فرمت خروجی را مشخص کنید. +به صورت پیش فرض فرمت خروجی PrettyCompact می باشد. شما میتوانید از طریق دستور FORMAT در یک query، یا با مشخص کردن `\G` در انتهای query، استفاده از آرگومان های `--format` یا `--vertical` یا از کانفیگ فایل کلاینت، فرمت خروجی را مشخص کنید. برای خروج از کلاینت، Ctrl-D (یا Ctrl+C) را فشار دهید؛ و یا یکی از دستورات زیر را به جای اجرای query اجرا کنید: "exit", "quit", "logout", "exit;", "quit;", "logout;", "q", "Q", ":q" @@ -68,7 +68,7 @@ command line برا پایه 'readline' (و 'history' یا 'libedit'، یه بد شما میتوانید query های طولانی را با فشردن Ctrl-C کنسل کنید. هر چند، بعد از این کار همچنان نیاز به انتظار چند ثانیه ای برای قطع کردن درخواست توسط سرور می باشید. امکان کنسل کردن یک query در مراحل خاص وجود ندارد. اگر شما صبر نکنید و برای بار دوم Ctrl+C را وارد کنید از client خارج می شوید. -کلاینت commant-line اجازه ی پاس دادن داده های external (جداول موقت external) را برای query ها می دهد. برای اطلاعات بیشتر به بخش "داده های External برای پردازش query" مراجعه کنید. +کلاینت commant-line اجازه ی پاس دادن داده های external (جداول موقت external) را برای query ها می دهد. برای اطلاعات بیشتر به بخش "داده های External برای پردازش query" مراجعه کنید. ## پیکربندی {#interfaces_cli_configuration} diff --git a/docs/fa/interfaces/formats.md b/docs/fa/interfaces/formats.md index 04e4a4ea3bf..8391352ea93 100644 --- a/docs/fa/interfaces/formats.md +++ b/docs/fa/interfaces/formats.md @@ -43,9 +43,9 @@ Format | INSERT | SELECT Dates با فرمت YYY-MM-DD نوشته می شوند و به همین حالت پارس می شوند، اما با هر کاراکتری به عنوان جدا کننده. Dates به همراه زمان با فرمت YYYY-MM-DD hh:mm:ss نوشته می شوند و با همین فرمت پارس می شوند، اما با هر کاراکتری به عنوان جداکننده. این در منطقه زمان سیستم در زمانی که کلاینت یا سرور شروع می شود (بسته به اینکه کدام یک از داده ها را تشکیل می دهد) رخ می دهد. برای تاریخ همراه با زمان DST مشخص نمی شود. پس اگر یک دامپ دارای زمان DST باشد، دامپ، داده ها را به طور غیرمستقیم مطابقت نمی دهد و پارسینگ، یکی از دو ساعت را انتخاب خواهد کرد. در طول عملیات خواندن، تاریخ ها و تاریخ و ساعت های نادرست می توانند به صورت null و یا natural overflow پارس شوند، بدون اینکه پیغام خطایی نمایش دهند. -به عنوان یک استثنا، پارس کردن تاریخ به همراه ساعت، اگر مقدار دقیقا شامل 10 عدد decimal باشد، به عنوان فرمت unix timestamp پشتیبانی خواهد کرد. خروجی وابسته به time-zone نمی باشد. فرمت های YYYY-MM-DD hh: mm: ss و NNNNNNNNNN به صورت خودکار تمایز می یابند. +به عنوان یک استثنا، پارس کردن تاریخ به همراه ساعت، اگر مقدار دقیقا شامل 10 عدد decimal باشد، به عنوان فرمت unix timestamp پشتیبانی خواهد کرد. خروجی وابسته به time-zone نمی باشد. فرمت های YYYY-MM-DD hh: mm: ss و NNNNNNNNNN به صورت خودکار تمایز می یابند. -رشته های دارای کاراکتر های ویژه backslash-escaped چاپ می شوند. escape های در ادامه برای خروجی استفاده می شوند: `\b`، `\f`، `\r`، `\n`، `\t`، `\0`, `\'`، `\\`. پارسر همچنین از `\a`، `\v`، و `\xHH` (hex escape) و هر `\c` پشتیبانی می کند. بدین ترتیب خواندن داده ها از فرمت line feed که می تواند به صورت `\n` یا `\` نوشته شود پشتیبانی می کند. برای مثال، رشته ی `Hello world` به همراه line feed بین کلمات به جای space می تواند به هر یک از حالات زیر پارس شود:: +رشته های دارای کاراکتر های ویژه backslash-escaped چاپ می شوند. escape های در ادامه برای خروجی استفاده می شوند: `\b`، `\f`، `\r`، `\n`، `\t`، `\0`, `\'`، `\\`. پارسر همچنین از `\a`، `\v`، و `\xHH` (hex escape) و هر `\c` پشتیبانی می کند. بدین ترتیب خواندن داده ها از فرمت line feed که می تواند به صورت `\n` یا `\` نوشته شود پشتیبانی می کند. برای مثال، رشته ی `Hello world` به همراه line feed بین کلمات به جای space می تواند به هر یک از حالات زیر پارس شود::
@@ -146,7 +146,7 @@ SearchPhrase=baku count()=1000 Comma Separated Values format ([RFC](https://tools.ietf.org/html/rfc4180)). -زمانی که از این روش برای فرمت استفاده می شود، سطر ها با دابل کوتیشن enclosed می شوند. دابل کوتیشن داخل یک رشته خروجی آن به صورت دو دابل کوتیشن در یک سطر است. قانون دیگری برای escape کردن کاراکترها وجود ندارد. تاریخ و تاریخ-ساعت در دابل کوتیشن ها enclosed می شوند. اعداد بدون دابل کوتیشن در خروجی می آیند. مقادیر با جدا کننده * مشخص می شوند. سطر ها با استفاده از line feed (LF) جدا می شوند. آرایه ها در csv به این صورت serialize می شوند: ابتدا آرایه به یک رشته با فرمت TabSeparate سریالایز می شوند، و سپس رشته ی حاصل در دابل کوتیشن برای csv ارسال می شود. Tuple ها در فرمت CSV در ستون های جدا سریالایز می شوند (به این ترتیب، nest ها در tuble از دست میروند) +زمانی که از این روش برای فرمت استفاده می شود، سطر ها با دابل کوتیشن enclosed می شوند. دابل کوتیشن داخل یک رشته خروجی آن به صورت دو دابل کوتیشن در یک سطر است. قانون دیگری برای escape کردن کاراکترها وجود ندارد. تاریخ و تاریخ-ساعت در دابل کوتیشن ها enclosed می شوند. اعداد بدون دابل کوتیشن در خروجی می آیند. مقادیر با جدا کننده * مشخص می شوند. سطر ها با استفاده از line feed (LF) جدا می شوند. آرایه ها در csv به این صورت serialize می شوند: ابتدا آرایه به یک رشته با فرمت TabSeparate سریالایز می شوند، و سپس رشته ی حاصل در دابل کوتیشن برای csv ارسال می شود. Tuple ها در فرمت CSV در ستون های جدا سریالایز می شوند (به این ترتیب، nest ها در tuble از دست میروند)
@@ -340,7 +340,7 @@ JSON با جاوااسکریپت سازگار است. برای اطمینان ا ## Pretty -خروجی داده ها به صورت جداول Unicode-art، همچنین استفاده از ANSI-escape برای تنظیم رنگ های ترمینال. یک جدول کامل کشیده می شود، و هر سطر دو خط از ترمینال را اشغال می کند. هر بلاکِ نتیجه، به عنوان یک جدول جدا چاپ می شود.پس بلاک ها می توانند بدون بافر کردن نتایج چاپ شوند (بافرینگ برای pre-calculate تمام مقادیر قابل مشاهده ضروری است). برای جلوگیری از دامپ زیاد داده ها در ترمینال، 10 هزار سطر اول چاپ می شوند. اگر تعداد سطر های بزرگتر مساوی 10 هزار باشد، پیغام " 10 هزار اول نمایش داده شد" چاپ می شود. این فرمت فقط مناسب خروجی نتایج query ها می باشد، نه برای پارس کردن (دریافت داده ها و درج آن در جدول). +خروجی داده ها به صورت جداول Unicode-art، همچنین استفاده از ANSI-escape برای تنظیم رنگ های ترمینال. یک جدول کامل کشیده می شود، و هر سطر دو خط از ترمینال را اشغال می کند. هر بلاکِ نتیجه، به عنوان یک جدول جدا چاپ می شود.پس بلاک ها می توانند بدون بافر کردن نتایج چاپ شوند (بافرینگ برای pre-calculate تمام مقادیر قابل مشاهده ضروری است). برای جلوگیری از دامپ زیاد داده ها در ترمینال، 10 هزار سطر اول چاپ می شوند. اگر تعداد سطر های بزرگتر مساوی 10 هزار باشد، پیغام " 10 هزار اول نمایش داده شد" چاپ می شود. این فرمت فقط مناسب خروجی نتایج query ها می باشد، نه برای پارس کردن (دریافت داده ها و درج آن در جدول). فرمت Pretty از total values (هنگام استفاده از WITH TOTALS) و extreme (هنگام که 'extremes' برابر با 1 است) برای خروجی پشتیبانی می کند. در این موارد، total values و extreme values بعد از نمایش داده های اصلی در جداول جدا، چاپ می شوند. مثال (برای فرمت PrettyCompact نمایش داده شده است): diff --git a/docs/fa/interfaces/http.md b/docs/fa/interfaces/http.md index 418a92f4daf..6462c68b224 100644 --- a/docs/fa/interfaces/http.md +++ b/docs/fa/interfaces/http.md @@ -87,7 +87,7 @@ $ echo 'SELECT 1 FORMAT Pretty' | curl 'http://localhost:8123/?' --data-binary @
-برای query های INSERT متد POST ضروری است. در این مورد، شما می توانید ابتدای query خود را در URL parameter بنویسید، و از POST برای پاس داده داده ها برای درج استفاده کنید. داده ی برای درج می تواند، برای مثال یک دامپ tab-separated شده از MySQL باشد. به این ترتیب، query INSERT جایگزین LOAD DATA LOCAL INFILE از MySQL می شود. +برای query های INSERT متد POST ضروری است. در این مورد، شما می توانید ابتدای query خود را در URL parameter بنویسید، و از POST برای پاس داده داده ها برای درج استفاده کنید. داده ی برای درج می تواند، برای مثال یک دامپ tab-separated شده از MySQL باشد. به این ترتیب، query INSERT جایگزین LOAD DATA LOCAL INFILE از MySQL می شود. مثال: ساخت جدول @@ -175,7 +175,7 @@ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- شما می توانید از فرمت فشرده سازی داخلی ClickHouse در هنگان انتقال داده ها استفاده کنید. این فشرده سازی داده، یک فرمت غیراستاندارد است، و شما باید از برنامه مخصوص فشرده سازی ClickHouse برای استفاده از آن استفاده کنید. (این برنامه در هنگام نصب پکیج clickhouse-client نصب شده است) -اگر شما در URL پارامتر 'compress=1' را قرار دهید، سرور داده های ارسالی به شما را فشرده سازی می کند. اگر شما پارامتر 'decompress=1' را در URL ست کنید، سرور داده های ارسالی توسط متد POST را decompress می کند. +اگر شما در URL پارامتر 'compress=1' را قرار دهید، سرور داده های ارسالی به شما را فشرده سازی می کند. اگر شما پارامتر 'decompress=1' را در URL ست کنید، سرور داده های ارسالی توسط متد POST را decompress می کند. همچنین استفاده از فشرده سازی استاندارد gzip در HTTP ممکن است. برای ارسال درخواست POST و فشرده سازی آن به صورت gzip، هدر `Content-Encoding: gzip` را به request خود اضافه کنید. برای اینکه ClickHouse، response فشرده شده به صورت gzip برای شما ارسال کند، ابتدا باید `enable_http_compression` را در تنظیمات ClickHouse فعال کنید و در ادامه هدر `Accept-Encoding: gzip` را به درخواست خود اضافه کنید. diff --git a/docs/fa/introduction/distinctive_features.md b/docs/fa/introduction/distinctive_features.md index 1eafdbd6863..0dbe60db7f3 100644 --- a/docs/fa/introduction/distinctive_features.md +++ b/docs/fa/introduction/distinctive_features.md @@ -4,7 +4,7 @@ ## مدیریت دیتابیس ستون گرای واقعی -در یک مدیریت دیتابیس ستون گرای واقعی، هیچ مقداری فضای اضافی برای ذخیره سازی ندارد. برای مثال، این به این معنیست که برای مقادیر، constant-length باید پشتیبانی شوند تا از ذخیره سازی طول مقدار به عنوان یه عدد integer کنار مقدار جلوگیری شود. در این مورد، یک میلیارد مقدار Uint8 باید در واقع در حالت غیرفشرده 1 گیگابایت فضا اشغال کند، در غیراین صورت به شدت بر عملکرد CPU تاثیر میگذارد. این خیلی مهم هست که داده ها به صورت compact ذخیره سازی شوند حتی زمانی که uncompressed هستند، از آنجا که سرعت سرعت decompress (CPU Usage) عمدتا به حجم داده های uncompress بستگی دارد. +در یک مدیریت دیتابیس ستون گرای واقعی، هیچ مقداری فضای اضافی برای ذخیره سازی ندارد. برای مثال، این به این معنیست که برای مقادیر، constant-length باید پشتیبانی شوند تا از ذخیره سازی طول مقدار به عنوان یه عدد integer کنار مقدار جلوگیری شود. در این مورد، یک میلیارد مقدار Uint8 باید در واقع در حالت غیرفشرده 1 گیگابایت فضا اشغال کند، در غیراین صورت به شدت بر عملکرد CPU تاثیر میگذارد. این خیلی مهم هست که داده ها به صورت compact ذخیره سازی شوند حتی زمانی که uncompressed هستند، از آنجا که سرعت سرعت decompress (CPU Usage) عمدتا به حجم داده های uncompress بستگی دارد. این بسیار قابل توجه است چون سیستم هایی وجود دارند که توانایی ذخیره سازی مقادیر ستون ها را به صورت جداگانه دارند، اما به دلیل بهینه سازی آنها برای دیگر سناریو ها، نمیتوانند به طور موثر پردازش های تحیلی انجام دهند. برای مثال HBase، BigTable، Cassandra و HyperTable. در این سیستم ها، شما توان عملیاتی حدود صدها هزار سطر در ثانیه را دارید، اما نه صدها میلیون سطر در ثانیه. diff --git a/docs/ru/data_types/tuple.md b/docs/ru/data_types/tuple.md index 1763967c052..56a195d6398 100644 --- a/docs/ru/data_types/tuple.md +++ b/docs/ru/data_types/tuple.md @@ -33,7 +33,7 @@ SELECT ## Особенности работы с типами данных -При создании кортежа "на лету" ClickHouse автоматически определяет тип каждого аргументов как минимальный из типов, который может сохранить значение аргумента. Если аргумент — [NULL](../query_language/syntax.md#null-literal), то тип элемента кортежа — [Nullable](nullable.md). +При создании кортежа "на лету" ClickHouse автоматически определяет тип каждого аргументов как минимальный из типов, который может сохранить значение аргумента. Если аргумент — [NULL](../query_language/syntax.md#null-literal), то тип элемента кортежа — [Nullable](nullable.md). Пример автоматического определения типа данных: diff --git a/docs/ru/development/style.md b/docs/ru/development/style.md index 4bcff77ca3a..047291d3341 100644 --- a/docs/ru/development/style.md +++ b/docs/ru/development/style.md @@ -44,13 +44,13 @@ void reinsert(const Value & x) memcpy(&buf[place_value], &x, sizeof(x)); ``` -**6.** В выражениях `if`, `for`, `while` и т.д. перед открывающей скобкой ставится пробел (в отличие от вызовов функций). +**6.** В выражениях `if`, `for`, `while` и т.д. перед открывающей скобкой ставится пробел (в отличие от вызовов функций). ```cpp for (size_t i = 0; i < rows; i += storage.index_granularity) ``` -**7.** Вокруг бинарных операторов (`+`, `-`, `*`, `/`, `%`, ...), а также тернарного оператора `?:` ставятся пробелы. +**7.** Вокруг бинарных операторов (`+`, `-`, `*`, `/`, `%`, ...), а также тернарного оператора `?:` ставятся пробелы. ```cpp UInt16 year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'); @@ -58,7 +58,7 @@ UInt8 month = (s[5] - '0') * 10 + (s[6] - '0'); UInt8 day = (s[8] - '0') * 10 + (s[9] - '0'); ``` -**8.** Если ставится перенос строки, то оператор пишется на новой строке, и перед ним увеличивается отступ. +**8.** Если ставится перенос строки, то оператор пишется на новой строке, и перед ним увеличивается отступ. ```cpp if (elapsed_ns) @@ -67,7 +67,7 @@ if (elapsed_ns) << bytes_read_on_server * 1000.0 / elapsed_ns << " MB/s.) "; ``` -**9.** Внутри строки можно, выполнять выравнивание с помощью пробелов. +**9.** Внутри строки можно, выполнять выравнивание с помощью пробелов. ```cpp dst.ClickLogID = click.LogID; @@ -85,7 +85,7 @@ dst.ClickGoodEvent = click.GoodEvent; **13.** Оператор `[]` не отделяется пробелами. -**14.** В выражении `template <...>`, между `template` и `<` ставится пробел, а после `<` и до `>` не ставится. +**14.** В выражении `template <...>`, между `template` и `<` ставится пробел, а после `<` и до `>` не ставится. ```cpp template @@ -257,7 +257,7 @@ void executeQuery( */ ``` -Пример взят с ресурса [http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/](http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/). +Пример взят с ресурса [http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/](http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/). **7.** Нельзя писать мусорные комментарии (автор, дата создания...) в начале каждого файла. @@ -305,7 +305,7 @@ void executeQuery( size_t max_block_size; ``` -**2.** Имена функций (методов) camelCase с маленькой буквы. +**2.** Имена функций (методов) camelCase с маленькой буквы. ```cpp std::string getName() const override { return "Memory"; } @@ -348,7 +348,7 @@ class IBlockInputStream bool info_successfully_loaded = false; ``` -**9.** В именах `define` и глобальных констант используется ALL_CAPS с подчёркиванием. +**9.** В именах `define` и глобальных констант используется ALL_CAPS с подчёркиванием. ```cpp #define MAX_SRC_TABLE_NAMES_TO_STORE 1000 diff --git a/docs/ru/getting_started/example_datasets/nyc_taxi.md b/docs/ru/getting_started/example_datasets/nyc_taxi.md index 3cfe7e439c5..cd86ff05b76 100644 --- a/docs/ru/getting_started/example_datasets/nyc_taxi.md +++ b/docs/ru/getting_started/example_datasets/nyc_taxi.md @@ -323,7 +323,7 @@ ORDER BY year, count(*) DESC На каждом сервере: ``` -CREATE TABLE default.trips_mergetree_third ( trip_id UInt32, vendor_id Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14), pickup_date Date, pickup_datetime DateTime, dropoff_date Date, dropoff_datetime DateTime, store_and_fwd_flag UInt8, rate_code_id UInt8, pickup_longitude Float64, pickup_latitude Float64, dropoff_longitude Float64, dropoff_latitude Float64, passenger_count UInt8, trip_distance Float64, fare_amount Float32, extra Float32, mta_tax Float32, tip_amount Float32, tolls_amount Float32, ehail_fee Float32, improvement_surcharge Float32, total_amount Float32, payment_type_ Enum8('UNK' = 0, 'CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4), trip_type UInt8, pickup FixedString(25), dropoff FixedString(25), cab_type Enum8('yellow' = 1, 'green' = 2, 'uber' = 3), pickup_nyct2010_gid UInt8, pickup_ctlabel Float32, pickup_borocode UInt8, pickup_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), pickup_ct2010 FixedString(6), pickup_boroct2010 FixedString(7), pickup_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), pickup_ntacode FixedString(4), pickup_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), pickup_puma UInt16, dropoff_nyct2010_gid UInt8, dropoff_ctlabel Float32, dropoff_borocode UInt8, dropoff_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), dropoff_ct2010 FixedString(6), dropoff_boroct2010 FixedString(7), dropoff_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), dropoff_ntacode FixedString(4), dropoff_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), dropoff_puma UInt16) ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) +CREATE TABLE default.trips_mergetree_third ( trip_id UInt32, vendor_id Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14), pickup_date Date, pickup_datetime DateTime, dropoff_date Date, dropoff_datetime DateTime, store_and_fwd_flag UInt8, rate_code_id UInt8, pickup_longitude Float64, pickup_latitude Float64, dropoff_longitude Float64, dropoff_latitude Float64, passenger_count UInt8, trip_distance Float64, fare_amount Float32, extra Float32, mta_tax Float32, tip_amount Float32, tolls_amount Float32, ehail_fee Float32, improvement_surcharge Float32, total_amount Float32, payment_type_ Enum8('UNK' = 0, 'CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4), trip_type UInt8, pickup FixedString(25), dropoff FixedString(25), cab_type Enum8('yellow' = 1, 'green' = 2, 'uber' = 3), pickup_nyct2010_gid UInt8, pickup_ctlabel Float32, pickup_borocode UInt8, pickup_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), pickup_ct2010 FixedString(6), pickup_boroct2010 FixedString(7), pickup_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), pickup_ntacode FixedString(4), pickup_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), pickup_puma UInt16, dropoff_nyct2010_gid UInt8, dropoff_ctlabel Float32, dropoff_borocode UInt8, dropoff_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), dropoff_ct2010 FixedString(6), dropoff_boroct2010 FixedString(7), dropoff_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), dropoff_ntacode FixedString(4), dropoff_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), dropoff_puma UInt16) ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) ``` На исходном сервере: diff --git a/docs/ru/getting_started/example_datasets/ontime.md b/docs/ru/getting_started/example_datasets/ontime.md index d08d40d6692..f138817b400 100644 --- a/docs/ru/getting_started/example_datasets/ontime.md +++ b/docs/ru/getting_started/example_datasets/ontime.md @@ -166,7 +166,7 @@ SELECT Origin, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Q4. Количество задержек по перевозчикам за 2007 год ``` sql -SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC +SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC ``` Q5. Процент задержек по перевозчикам за 2007 год diff --git a/docs/ru/index.md b/docs/ru/index.md index 24e97aaff83..2db293a36ff 100644 --- a/docs/ru/index.md +++ b/docs/ru/index.md @@ -72,9 +72,9 @@ ClickHouse - столбцовая система управления базам ### По вводу-выводу -1. Для выполнения аналитического запроса, требуется прочитать небольшое количество столбцов таблицы. В столбцовой БД для этого можно читать только нужные данные. Например, если вам требуется только 5 столбцов из 100, то следует рассчитывать на 20-кратное уменьшение ввода-вывода. -2. Так как данные читаются пачками, то их проще сжимать. Данные, лежащие по столбцам также лучше сжимаются. За счёт этого, дополнительно уменьшается объём ввода-вывода. -3. За счёт уменьшения ввода-вывода, больше данных влезает в системный кэш. +1. Для выполнения аналитического запроса, требуется прочитать небольшое количество столбцов таблицы. В столбцовой БД для этого можно читать только нужные данные. Например, если вам требуется только 5 столбцов из 100, то следует рассчитывать на 20-кратное уменьшение ввода-вывода. +2. Так как данные читаются пачками, то их проще сжимать. Данные, лежащие по столбцам также лучше сжимаются. За счёт этого, дополнительно уменьшается объём ввода-вывода. +3. За счёт уменьшения ввода-вывода, больше данных влезает в системный кэш. Например, для запроса "посчитать количество записей для каждой рекламной системы", требуется прочитать один столбец "идентификатор рекламной системы", который занимает 1 байт в несжатом виде. Если большинство переходов было не с рекламных систем, то можно рассчитывать хотя бы на десятикратное сжатие этого столбца. При использовании быстрого алгоритма сжатия, возможно разжатие данных со скоростью более нескольких гигабайт несжатых данных в секунду. То есть, такой запрос может выполняться со скоростью около нескольких миллиардов строк в секунду на одном сервере. На практике, такая скорость действительно достигается. diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index 8a0d954445a..eaf2cf132af 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -59,10 +59,10 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA При выполнении запроса, клиент показывает: -1. Прогресс выполнение запроса, который обновляется не чаще, чем 10 раз в секунду (по умолчанию). При быстрых запросах, прогресс может не успеть отобразиться. -2. Отформатированный запрос после его парсинга - для отладки. -3. Результат в заданном формате. -4. Количество строк результата, прошедшее время, а также среднюю скорость выполнения запроса. +1. Прогресс выполнение запроса, который обновляется не чаще, чем 10 раз в секунду (по умолчанию). При быстрых запросах, прогресс может не успеть отобразиться. +2. Отформатированный запрос после его парсинга - для отладки. +3. Результат в заданном формате. +4. Количество строк результата, прошедшее время, а также среднюю скорость выполнения запроса. Вы можете прервать длинный запрос, нажав Ctrl+C. При этом вам всё равно придётся чуть-чуть подождать, пока сервер остановит запрос. На некоторых стадиях выполнения, запрос невозможно прервать. Если вы не дождётесь и нажмёте Ctrl+C второй раз, то клиент будет завершён. @@ -82,7 +82,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA ### Параметры командной строки -- `--host, -h` - имя сервера, по умолчанию - localhost. Вы можете использовать как имя, так и IPv4 или IPv6 адрес. +- `--host, -h` - имя сервера, по умолчанию - localhost. Вы можете использовать как имя, так и IPv4 или IPv6 адрес. - `--port` - порт, к которому соединяться, по умолчанию - 9000. Замечу, что для HTTP и родного интерфейса используются разные порты. - `--user, -u` - имя пользователя, по умолчанию - default. - `--password` - пароль, по умолчанию - пустая строка. diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index 162eb494166..effd512ea3f 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -135,7 +135,7 @@ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- Если вы указали в URL decompress=1, то сервер будет разжимать те данные, которые вы передаёте ему POST-ом. Также имеется возможность использования стандартного сжатия HTTP, на основе gzip. Чтобы отправить POST-запрос, сжатый с помощью gzip, добавьте к запросу заголовок `Content-Encoding: gzip`. -Чтобы ClickHouse сжимал ответ на запрос с помощью gzip, необходимо добавить `Accept-Encoding: gzip` к заголовкам запроса, и включить настройку ClickHouse `enable_http_compression`. +Чтобы ClickHouse сжимал ответ на запрос с помощью gzip, необходимо добавить `Accept-Encoding: gzip` к заголовкам запроса, и включить настройку ClickHouse `enable_http_compression`. Это может быть использовано для уменьшения трафика по сети при передаче большого количества данных, а также для создания сразу сжатых дампов. @@ -159,13 +159,13 @@ $ echo 'SELECT number FROM numbers LIMIT 10' | curl 'http://localhost:8123/?data Имя пользователя и пароль могут быть указаны в одном из двух вариантов: -1. С использованием HTTP Basic Authentification. Пример: +1. С использованием HTTP Basic Authentification. Пример: ```bash echo 'SELECT 1' | curl 'http://user:password@localhost:8123/' -d @- ``` -2. В параметрах URL user и password. Пример: +2. В параметрах URL user и password. Пример: ```bash echo 'SELECT 1' | curl 'http://localhost:8123/?user=user&password=password' -d @- diff --git a/docs/ru/introduction/distinctive_features.md b/docs/ru/introduction/distinctive_features.md index 73d2826d16c..9a01388f630 100644 --- a/docs/ru/introduction/distinctive_features.md +++ b/docs/ru/introduction/distinctive_features.md @@ -51,9 +51,9 @@ ClickHouse поддерживает таблицы с первичным клю ClickHouse предоставляет различные способы разменять точность вычислений на производительность: -1. Система содержит агрегатные функции для приближённого вычисления количества различных значений, медианы и квантилей. -2. Поддерживается возможность выполнить запрос на основе части (выборки) данных и получить приближённый результат. При этом, с диска будет считано пропорционально меньше данных. -3. Поддерживается возможность выполнить агрегацию не для всех ключей, а для ограниченного количества первых попавшихся ключей. При выполнении некоторых условий на распределение ключей в данных, это позволяет получить достаточно точный результат с использованием меньшего количества ресурсов. +1. Система содержит агрегатные функции для приближённого вычисления количества различных значений, медианы и квантилей. +2. Поддерживается возможность выполнить запрос на основе части (выборки) данных и получить приближённый результат. При этом, с диска будет считано пропорционально меньше данных. +3. Поддерживается возможность выполнить агрегацию не для всех ключей, а для ограниченного количества первых попавшихся ключей. При выполнении некоторых условий на распределение ключей в данных, это позволяет получить достаточно точный результат с использованием меньшего количества ресурсов. ## Репликация данных и поддержка целостности diff --git a/docs/ru/operations/server_settings/settings.md b/docs/ru/operations/server_settings/settings.md index 9c855fa6ff0..68b7c4becc0 100644 --- a/docs/ru/operations/server_settings/settings.md +++ b/docs/ru/operations/server_settings/settings.md @@ -456,7 +456,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat - requireTLSv1_2 - Требование соединения TLSv1.2. Допустимые значения: ``true``, ``false``. - fips - Активация режима OpenSSL FIPS. Поддерживается, если версия OpenSSL, с которой собрана библиотека поддерживает fips. - privateKeyPassphraseHandler - Класс (подкласс PrivateKeyPassphraseHandler)запрашивающий кодовую фразу доступа к секретному ключу. Например, ````, ``KeyFileHandler``, ``test``, ````. -- invalidCertificateHandler - Класс (подкласс CertificateHandler) для подтвеждения невалидных сертификатов. Например, `` ConsoleCertificateHandler ``. +- invalidCertificateHandler - Класс (подкласс CertificateHandler) для подтвеждения невалидных сертификатов. Например, `` ConsoleCertificateHandler ``. - disableProtocols - Запрещенные к искользованию протоколы. - preferServerCiphers - Предпочтение серверных шифров на клиенте. diff --git a/docs/ru/operations/settings/permissions_for_queries.md b/docs/ru/operations/settings/permissions_for_queries.md index 69a00ab4f8e..6eebea9fdbe 100644 --- a/docs/ru/operations/settings/permissions_for_queries.md +++ b/docs/ru/operations/settings/permissions_for_queries.md @@ -29,7 +29,7 @@ После установки `readonly = 1`пользователь не может изменить настройки `readonly` и `allow_ddl` в текущей сессии. -При использовании метода `GET` в [HTTP интерфейсе](../../interfaces/http.md#http_interface), `readonly = 1` устанавливается автоматически. Для изменения данных используйте метод `POST`. +При использовании метода `GET` в [HTTP интерфейсе](../../interfaces/http.md#http_interface), `readonly = 1` устанавливается автоматически. Для изменения данных используйте метод `POST`. **Значение по умолчанию** @@ -46,7 +46,7 @@ - 0 — DDL запросы не разрешены. - 1 — DDL запросы разрешены. -Если `allow_ddl = 0`, то невозможно выполнить `SET allow_ddl = 1` для текущей сессии. +Если `allow_ddl = 0`, то невозможно выполнить `SET allow_ddl = 1` для текущей сессии. **Значение по умолчанию** diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index a06290f51f1..ed1395a1aaf 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -16,7 +16,7 @@ ClickHouse применяет настройку в тех случаях, ко Возможные значения: -- `deny` - (по умолчанию) запрещает использование таких подзапросов (При попытке использование вернет исключение "Double-distributed IN/JOIN subqueries is denied"); +- `deny` - (по умолчанию) запрещает использование таких подзапросов (При попытке использование вернет исключение "Double-distributed IN/JOIN subqueries is denied"); - `local` - заменит базу данных и таблицу в подзапросе на локальные для конечного сервера (шарда), оставив обычный `IN` / `JOIN`; - `global` - заменит запрос `IN` / `JOIN` на `GLOBAL IN` / `GLOBAL JOIN`; - `allow` - разрешает использование таких подзапросов. diff --git a/docs/ru/operations/table_engines/collapsingmergetree.md b/docs/ru/operations/table_engines/collapsingmergetree.md index 2a1d7f0d111..9b28deb880a 100644 --- a/docs/ru/operations/table_engines/collapsingmergetree.md +++ b/docs/ru/operations/table_engines/collapsingmergetree.md @@ -31,7 +31,7 @@ CollapsingMergeTree(EventDate, (CounterID, EventDate, intHash32(UniqID), VisitID Существует несколько способов получения полностью "схлопнутых" данных из таблицы типа `CollapsingMergeTree`: -1. Написать запрос с GROUP BY и агрегатными функциями, учитывающими знак. Например, чтобы посчитать количество, надо вместо count() написать sum(Sign); чтобы посчитать сумму чего-либо, надо вместо sum(x) написать sum(Sign \* x) и т. п., а также добавить HAVING sum(Sign) `>` 0. Не все величины можно посчитать подобным образом. Например, агрегатные функции min, max не могут быть переписаны. -2. Если необходимо вынимать данные без агрегации (например, проверить наличие строк, самые новые значения которых удовлетворяют некоторым условиям), можно использовать модификатор FINAL для секции FROM. Это вариант существенно менее эффективен. +1. Написать запрос с GROUP BY и агрегатными функциями, учитывающими знак. Например, чтобы посчитать количество, надо вместо count() написать sum(Sign); чтобы посчитать сумму чего-либо, надо вместо sum(x) написать sum(Sign \* x) и т. п., а также добавить HAVING sum(Sign) `>` 0. Не все величины можно посчитать подобным образом. Например, агрегатные функции min, max не могут быть переписаны. +2. Если необходимо вынимать данные без агрегации (например, проверить наличие строк, самые новые значения которых удовлетворяют некоторым условиям), можно использовать модификатор FINAL для секции FROM. Это вариант существенно менее эффективен. [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/collapsingmergetree/) diff --git a/docs/ru/operations/table_engines/graphitemergetree.md b/docs/ru/operations/table_engines/graphitemergetree.md index 9142a4023a6..52e0d98df81 100644 --- a/docs/ru/operations/table_engines/graphitemergetree.md +++ b/docs/ru/operations/table_engines/graphitemergetree.md @@ -40,7 +40,7 @@ default ... ``` -При обработке записи ClickHouse проверит правила в секции `pattern`. Если имя метрики соответствует шаблону `regexp`, то применяются правила из `pattern`, в противном случае из `default`. +При обработке записи ClickHouse проверит правила в секции `pattern`. Если имя метрики соответствует шаблону `regexp`, то применяются правила из `pattern`, в противном случае из `default`. Поля шаблона правил. diff --git a/docs/ru/operations/table_engines/kafka.md b/docs/ru/operations/table_engines/kafka.md index 73b4fb395ee..d4f0c4bdfb0 100644 --- a/docs/ru/operations/table_engines/kafka.md +++ b/docs/ru/operations/table_engines/kafka.md @@ -122,7 +122,7 @@ Kafka SETTINGS Аналогично GraphiteMergeTree, движок Kafka поддерживает расширенную конфигурацию с помощью конфигурационного файла ClickHouse. Существует два конфигурационных ключа, которые можно использовать - глобальный (`kafka`) и по топикам (`kafka_*`). Сначала применяется глобальная конфигурация, затем конфигурация по топикам (если она существует). ```xml - + cgrp smallest diff --git a/docs/ru/operations/table_engines/replication.md b/docs/ru/operations/table_engines/replication.md index 7eb9e1216c3..d80ef509986 100644 --- a/docs/ru/operations/table_engines/replication.md +++ b/docs/ru/operations/table_engines/replication.md @@ -159,10 +159,10 @@ sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data Если на одном из серверов исчезли все данные и метаданные, восстановление делается следующим образом: -1. Установите на сервер ClickHouse. Корректно пропишите подстановки в конфигурационном файле, отвечающие за идентификатор шарда и реплики, если вы их используете. -2. Если у вас были нереплицируемые таблицы, которые должны быть вручную продублированы на серверах, скопируйте их данные (в директории `/var/lib/clickhouse/data/db_name/table_name/`) с реплики. -3. Скопируйте с реплики определения таблиц, находящиеся в `/var/lib/clickhouse/metadata/`. Если в определениях таблиц, идентификатор шарда или реплики, прописаны в явном виде - исправьте их, чтобы они соответствовали данной реплике. (Альтернативный вариант - запустить сервер и сделать самостоятельно все запросы `ATTACH TABLE`, которые должны были бы быть в соответствующих .sql файлах в `/var/lib/clickhouse/metadata/`.) -4. Создайте в ZooKeeper узел `/path_to_table/replica_name/flags/force_restore_data` с любым содержимым или выполните команду для восстановления всех реплицируемых таблиц: `sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data` +1. Установите на сервер ClickHouse. Корректно пропишите подстановки в конфигурационном файле, отвечающие за идентификатор шарда и реплики, если вы их используете. +2. Если у вас были нереплицируемые таблицы, которые должны быть вручную продублированы на серверах, скопируйте их данные (в директории `/var/lib/clickhouse/data/db_name/table_name/`) с реплики. +3. Скопируйте с реплики определения таблиц, находящиеся в `/var/lib/clickhouse/metadata/`. Если в определениях таблиц, идентификатор шарда или реплики, прописаны в явном виде - исправьте их, чтобы они соответствовали данной реплике. (Альтернативный вариант - запустить сервер и сделать самостоятельно все запросы `ATTACH TABLE`, которые должны были бы быть в соответствующих .sql файлах в `/var/lib/clickhouse/metadata/`.) +4. Создайте в ZooKeeper узел `/path_to_table/replica_name/flags/force_restore_data` с любым содержимым или выполните команду для восстановления всех реплицируемых таблиц: `sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data` Затем запустите сервер (перезапустите, если уже запущен). Данные будут скачаны с реплик. diff --git a/docs/ru/query_language/agg_functions/index.md b/docs/ru/query_language/agg_functions/index.md index 5bcd7fa0301..138bae48fd3 100644 --- a/docs/ru/query_language/agg_functions/index.md +++ b/docs/ru/query_language/agg_functions/index.md @@ -4,8 +4,8 @@ ClickHouse поддерживает также: -- [Параметрические агрегатные функции](parametric_functions.md#aggregate_functions_parametric), которые помимо столбцов принимаю и другие параметры. -- [Комбинаторы](combinators.md#aggregate_functions_combinators), которые изменяют поведение агрегатных фунций. +- [Параметрические агрегатные функции](parametric_functions.md#aggregate_functions_parametric), которые помимо столбцов принимаю и другие параметры. +- [Комбинаторы](combinators.md#aggregate_functions_combinators), которые изменяют поведение агрегатных фунций. ## Обработка NULL diff --git a/docs/ru/query_language/agg_functions/reference.md b/docs/ru/query_language/agg_functions/reference.md index dafbcc07a5c..1a8aefc7e21 100644 --- a/docs/ru/query_language/agg_functions/reference.md +++ b/docs/ru/query_language/agg_functions/reference.md @@ -6,7 +6,7 @@ Считает количество строк. Принимает ноль аргументов, возвращает UInt64. Не поддерживается синтаксис `COUNT (DISTINCT x)`. Для этого существует агрегатная функция`uniq`. -Запрос вида `SELECT count() FROM table` не оптимизируется, так как количество записей в таблице нигде не хранится отдельно. Из таблицы будет выбран какой-нибудь достаточно маленький столбец, и будет посчитано количество значений в нём. +Запрос вида `SELECT count() FROM table` не оптимизируется, так как количество записей в таблице нигде не хранится отдельно. Из таблицы будет выбран какой-нибудь достаточно маленький столбец, и будет посчитано количество значений в нём. ## any(x) {#agg_function-any} diff --git a/docs/ru/query_language/alter.md b/docs/ru/query_language/alter.md index 945338d3126..f8dd65e8c45 100644 --- a/docs/ru/query_language/alter.md +++ b/docs/ru/query_language/alter.md @@ -112,10 +112,10 @@ SELECT * FROM system.parts WHERE active ```bash $ ls -l /var/lib/clickhouse/data/test/visits/ total 48 -drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_2_2_0 -drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_4_4_0 -drwxrwxrwx 2 clickhouse clickhouse 4096 May 5 02:55 detached --rw-rw-rw- 1 clickhouse clickhouse 2 May 5 02:58 increment.txt +drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_2_2_0 +drwxrwxrwx 2 clickhouse clickhouse 20480 May 5 02:58 20140317_20140323_4_4_0 +drwxrwxrwx 2 clickhouse clickhouse 4096 May 5 02:55 detached +-rw-rw-rw- 1 clickhouse clickhouse 2 May 5 02:58 increment.txt ``` Здесь `20140317_20140323_2_2_0`, `20140317_20140323_4_4_0` - директории кусков. diff --git a/docs/ru/query_language/create.md b/docs/ru/query_language/create.md index a0027d3abc1..9f4cd95863c 100644 --- a/docs/ru/query_language/create.md +++ b/docs/ru/query_language/create.md @@ -116,7 +116,7 @@ CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO[db.]name] [ENGINE = eng При создании материализованного представления, нужно обязательно указать ENGINE - движок таблицы для хранения данных. -Материализованное представление работает следующим образом: при вставлении данных в таблицу, указанную в SELECT, часть вставленных данных конвертируется запросом, а результат вставляется в представление. +Материализованное представление работает следующим образом: при вставлении данных в таблицу, указанную в SELECT, часть вставленных данных конвертируется запросом, а результат вставляется в представление. Обычные представления не хранят никаких данных, а всего лишь производят чтение из другой таблицы. То есть, обычное представление - не более чем сохранённый запрос. При чтении из представления, этот сохранённый запрос, используется в качестве подзапроса в секции FROM. diff --git a/docs/ru/query_language/dicts/external_dicts_dict.md b/docs/ru/query_language/dicts/external_dicts_dict.md index 776ffb24756..94d923a6a75 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict.md +++ b/docs/ru/query_language/dicts/external_dicts_dict.md @@ -24,10 +24,10 @@ ``` -- name - Идентификатор, под которым словарь будет доступен для использования. Используйте символы `[a-zA-Z0-9_\-]`. -- [source](external_dicts_dict_sources.md) - Источник словаря. -- [layout](external_dicts_dict_layout.md) - Размещение словаря в памяти. -- [structure](external_dicts_dict_structure.md) - Структура словаря. Ключ и атрибуты, которые можно получить по ключу. -- [lifetime](external_dicts_dict_lifetime.md) - Периодичность обновления словарей. +- name - Идентификатор, под которым словарь будет доступен для использования. Используйте символы `[a-zA-Z0-9_\-]`. +- [source](external_dicts_dict_sources.md) - Источник словаря. +- [layout](external_dicts_dict_layout.md) - Размещение словаря в памяти. +- [structure](external_dicts_dict_structure.md) - Структура словаря. Ключ и атрибуты, которые можно получить по ключу. +- [lifetime](external_dicts_dict_lifetime.md) - Периодичность обновления словарей. [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/dicts/external_dicts_dict/) diff --git a/docs/ru/query_language/dicts/external_dicts_dict_layout.md b/docs/ru/query_language/dicts/external_dicts_dict_layout.md index fbac880c403..aafcf531860 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_layout.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_layout.md @@ -202,10 +202,10 @@ Укажите достаточно большой размер кэша. Количество ячеек следует подобрать экспериментальным путём: -1. Выставить некоторое значение. -2. Запросами добиться полной заполненности кэша. -3. Оценить потребление оперативной памяти с помощью таблицы `system.dictionaries`. -4. Увеличивать/уменьшать количество ячеек до получения требуемого расхода оперативной памяти. +1. Выставить некоторое значение. +2. Запросами добиться полной заполненности кэша. +3. Оценить потребление оперативной памяти с помощью таблицы `system.dictionaries`. +4. Увеличивать/уменьшать количество ячеек до получения требуемого расхода оперативной памяти. !!! warning Не используйте в качестве источника ClickHouse, поскольку он медленно обрабатывает запросы со случайным чтением. diff --git a/docs/ru/query_language/functions/array_functions.md b/docs/ru/query_language/functions/array_functions.md index af470725f3e..7cc4cb41b8e 100644 --- a/docs/ru/query_language/functions/array_functions.md +++ b/docs/ru/query_language/functions/array_functions.md @@ -144,7 +144,7 @@ hasAny(array1, array2) **Возвращаемые значения** -- `1`, если `array1` и `array2` имеют хотя бы один одинаковый элемент. +- `1`, если `array1` и `array2` имеют хотя бы один одинаковый элемент. - `0`, в противном случае. **Особенности** @@ -378,7 +378,7 @@ arrayPushFront(array, single_value) **Параметры** - `array` - Массив. -- `single_value` - Одиночное значение. В массив с числам можно добавить только числа, в массив со строками только строки. При добавлении чисел ClickHouse автоматически приводит тип `single_value` к типу данных массива. Подробнее о типах данных в ClickHouse читайте в разделе "[Типы данных](../../data_types/index.md#data_types)". Может быть равно `NULL`. Функция добавит элемент `NULL` в массив, а тип элементов массива преобразует в `Nullable`. +- `single_value` - Одиночное значение. В массив с числам можно добавить только числа, в массив со строками только строки. При добавлении чисел ClickHouse автоматически приводит тип `single_value` к типу данных массива. Подробнее о типах данных в ClickHouse читайте в разделе "[Типы данных](../../data_types/index.md#data_types)". Может быть равно `NULL`. Функция добавит элемент `NULL` в массив, а тип элементов массива преобразует в `Nullable`. **Пример** @@ -442,7 +442,7 @@ arraySlice(array, offset[, length]) - `array` - Массив данных. - `offset` - Отступ от края массива. Положительное значение - отступ слева, отрицательное значение - отступ справа. Отсчет элементов массива начинается с 1. -- `length` - Длина необходимого среза. Если указать отрицательное значение, то функция вернёт открытый срез `[offset, array_length - length)`. Если не указать значение, то функция вернёт срез `[offset, the_end_of_array]`. +- `length` - Длина необходимого среза. Если указать отрицательное значение, то функция вернёт открытый срез `[offset, array_length - length)`. Если не указать значение, то функция вернёт срез `[offset, the_end_of_array]`. **Пример** diff --git a/docs/ru/query_language/functions/json_functions.md b/docs/ru/query_language/functions/json_functions.md index 6958f33e2ba..464b037ad3a 100644 --- a/docs/ru/query_language/functions/json_functions.md +++ b/docs/ru/query_language/functions/json_functions.md @@ -4,10 +4,10 @@ Делаются следующие допущения: -1. Имя поля (аргумент функции) должно быть константой; -2. Считается, что имя поля в JSON-е закодировано некоторым каноническим образом. Например, `visitParamHas('{"abc":"def"}', 'abc') = 1`, но `visitParamHas('{"\\u0061\\u0062\\u0063":"def"}', 'abc') = 0` -3. Поля ищутся на любом уровне вложенности, без разбора. Если есть несколько подходящих полей - берётся первое. -4. В JSON-е нет пробельных символов вне строковых литералов. +1. Имя поля (аргумент функции) должно быть константой; +2. Считается, что имя поля в JSON-е закодировано некоторым каноническим образом. Например, `visitParamHas('{"abc":"def"}', 'abc') = 1`, но `visitParamHas('{"\\u0061\\u0062\\u0063":"def"}', 'abc') = 0` +3. Поля ищутся на любом уровне вложенности, без разбора. Если есть несколько подходящих полей - берётся первое. +4. В JSON-е нет пробельных символов вне строковых литералов. ## visitParamHas(params, name) Проверить наличие поля с именем name. diff --git a/docs/ru/query_language/functions/other_functions.md b/docs/ru/query_language/functions/other_functions.md index eede1a97264..453147dd4af 100644 --- a/docs/ru/query_language/functions/other_functions.md +++ b/docs/ru/query_language/functions/other_functions.md @@ -116,7 +116,7 @@ ORDER BY h ASC Преобразовать значение согласно явно указанному отображению одних элементов на другие. Имеется два варианта функции: -1. `transform(x, array_from, array_to, default)` +1. `transform(x, array_from, array_to, default)` `x` - что преобразовывать. @@ -158,7 +158,7 @@ ORDER BY c DESC └───────────┴────────┘ ``` -2. `transform(x, array_from, array_to)` +2. `transform(x, array_from, array_to)` Отличается от первого варианта отсутствующим аргументом default. Если значение x равно одному из элементов массива array_from, то возвращает соответствующий (такой же по номеру) элемент массива array_to; иначе возвращает x. diff --git a/docs/ru/query_language/misc.md b/docs/ru/query_language/misc.md index 000daf746dc..050d092befa 100644 --- a/docs/ru/query_language/misc.md +++ b/docs/ru/query_language/misc.md @@ -201,9 +201,9 @@ Readonly-пользователи могут останавливать толь Синхронный вариант (`SYNC`) ожидает остановки всех запросов и построчно выводит информацию о процессах по ходу их остановки. Ответ содержит колонку `kill_status`, которая может принимать следующие значения: -1. 'finished' - запрос был успешно остановлен; -2. 'waiting' - запросу отправлен сигнал завершения, ожидается его остановка; -3. остальные значения описывают причину невозможности остановки запроса. +1. 'finished' - запрос был успешно остановлен; +2. 'waiting' - запросу отправлен сигнал завершения, ожидается его остановка; +3. остальные значения описывают причину невозможности остановки запроса. Тестовый вариант запроса (`TEST`) только проверяет права пользователя и выводит список запросов для остановки. diff --git a/docs/ru/query_language/select.md b/docs/ru/query_language/select.md index ce1fa6de9a0..2709b24f28b 100644 --- a/docs/ru/query_language/select.md +++ b/docs/ru/query_language/select.md @@ -434,7 +434,7 @@ LIMIT 10 #### Обработка NULL -На поведение JOIN влияет настройка [join_use_nulls](../operations/settings/settings.md). При `join_use_nulls=1` `JOIN` работает как в стандартном SQL. +На поведение JOIN влияет настройка [join_use_nulls](../operations/settings/settings.md). При `join_use_nulls=1` `JOIN` работает как в стандартном SQL. Если ключами JOIN выступают поля типа [Nullable](../data_types/nullable.md), то строки, где хотя бы один из ключей имеет значение [NULL](syntax.md), не соединяются. @@ -706,9 +706,9 @@ WHERE и HAVING отличаются тем, что WHERE выполняется `DISTINCT` не поддерживается, если в `SELECT` присутствует хотя бы один столбец типа массив. -`DISTINCT` работает с [NULL](syntax.md) как если бы `NULL` был конкретным значением, причём `NULL=NULL`. Т.е. в результате `DISTINCT` разные комбинации с `NULL` встретятся только по одному разу. +`DISTINCT` работает с [NULL](syntax.md) как если бы `NULL` был конкретным значением, причём `NULL=NULL`. Т.е. в результате `DISTINCT` разные комбинации с `NULL` встретятся только по одному разу. -`DISTINCT` работает с [NULL](syntax.md) как если бы `NULL` был конкретным значением, причём `NULL=NULL`. Т.е. в результате `DISTINCT` разные комбинации с `NULL` встретятся только по одному разу. +`DISTINCT` работает с [NULL](syntax.md) как если бы `NULL` был конкретным значением, причём `NULL=NULL`. Т.е. в результате `DISTINCT` разные комбинации с `NULL` встретятся только по одному разу. ### Секция LIMIT @@ -830,7 +830,7 @@ ORDER BY EventDate ASC #### Обработка NULL -При обработке запроса оператор IN будет считать, что результат операции с [NULL](syntax.md) всегда равен `0`, независимо от того, находится `NULL` в правой или левой части оператора. Значения `NULL` не входят ни в какое множество, не соответствуют друг другу и не могут сравниваться. +При обработке запроса оператор IN будет считать, что результат операции с [NULL](syntax.md) всегда равен `0`, независимо от того, находится `NULL` в правой или левой части оператора. Значения `NULL` не входят ни в какое множество, не соответствуют друг другу и не могут сравниваться. Рассмотрим для примера таблицу `t_null`: @@ -955,11 +955,11 @@ SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID GLOBAL Это гораздо более оптимально, чем при использовании обычного IN. Но при этом, следует помнить о нескольких вещах: -1. При создании временной таблицы данные не уникализируются. Чтобы уменьшить объём передаваемых по сети данных, укажите в подзапросе DISTINCT (для обычного IN-а этого делать не нужно). -2. Временная таблица будет передана на все удалённые серверы. Передача не учитывает топологию сети. Например, если 10 удалённых серверов расположены в удалённом относительно сервера-инициатора запроса датацентре, то по каналу в удалённый датацентр данные будет переданы 10 раз. Старайтесь не использовать большие множества при использовании GLOBAL IN. -3. При передаче данных на удалённые серверы не настраивается ограничение использования сетевой полосы. Вы можете перегрузить сеть. -4. Старайтесь распределять данные по серверам так, чтобы в GLOBAL IN-ах не было частой необходимости. -5. Если в GLOBAL IN есть частая необходимость, то спланируйте размещение кластера ClickHouse таким образом, чтобы в каждом датацентре была хотя бы одна реплика каждого шарда, и среди них была быстрая сеть - чтобы запрос целиком можно было бы выполнить, передавая данные в пределах одного датацентра. +1. При создании временной таблицы данные не уникализируются. Чтобы уменьшить объём передаваемых по сети данных, укажите в подзапросе DISTINCT (для обычного IN-а этого делать не нужно). +2. Временная таблица будет передана на все удалённые серверы. Передача не учитывает топологию сети. Например, если 10 удалённых серверов расположены в удалённом относительно сервера-инициатора запроса датацентре, то по каналу в удалённый датацентр данные будет переданы 10 раз. Старайтесь не использовать большие множества при использовании GLOBAL IN. +3. При передаче данных на удалённые серверы не настраивается ограничение использования сетевой полосы. Вы можете перегрузить сеть. +4. Старайтесь распределять данные по серверам так, чтобы в GLOBAL IN-ах не было частой необходимости. +5. Если в GLOBAL IN есть частая необходимость, то спланируйте размещение кластера ClickHouse таким образом, чтобы в каждом датацентре была хотя бы одна реплика каждого шарда, и среди них была быстрая сеть - чтобы запрос целиком можно было бы выполнить, передавая данные в пределах одного датацентра. В секции `GLOBAL IN` также имеет смысл указывать локальную таблицу - в случае, если эта локальная таблица есть только на сервере-инициаторе запроса, и вы хотите воспользоваться данными из неё на удалённых серверах. diff --git a/docs/zh/data_types/array.md b/docs/zh/data_types/array.md index b6764498183..e9e59f2ed6f 100644 --- a/docs/zh/data_types/array.md +++ b/docs/zh/data_types/array.md @@ -48,7 +48,7 @@ SELECT ## 使用数据类型 -ClickHouse会自动检测数组元素,并根据元素计算出存储这些元素最小的数据类型。如果在元素中存在[NULL](../query_language/syntax.md#null-literal)或存在[Nullable](nullable.md#data_type-nullable)类型元素,那么数组的元素类型将会变成[Nullable](nullable.md)。 +ClickHouse会自动检测数组元素,并根据元素计算出存储这些元素最小的数据类型。如果在元素中存在 [NULL](../query_language/syntax.md#null-literal) 或存在 [Nullable](nullable.md#data_type-nullable) 类型元素,那么数组的元素类型将会变成 [Nullable](nullable.md)。 如果 ClickHouse 无法确定数据类型,它将产生异常。当尝试同时创建一个包含字符串和数字的数组时会发生这种情况 (`SELECT array(1, 'a')`)。 diff --git a/docs/zh/data_types/float.md b/docs/zh/data_types/float.md index 7c71fc5e260..113e640e611 100644 --- a/docs/zh/data_types/float.md +++ b/docs/zh/data_types/float.md @@ -5,7 +5,7 @@ 类型与以下 C 语言中类型是相同的: - `Float32` - `float` -- `Float64` - `double` +- `Float64` - `double` 我们建议您尽可能以整数形式存储数据。例如,将固定精度的数字转换为整数值,例如货币数量或页面加载时间用毫秒为单位表示 @@ -69,4 +69,4 @@ SELECT 0 / 0 └──────────────┘ ``` -可以在[ORDER BY 子句](../query_language/select.md) 查看更多关于 ` NaN` 排序的规则。 +可以在 [ORDER BY 子句](../query_language/select.md) 查看更多关于 ` NaN` 排序的规则。 diff --git a/docs/zh/development/style.md b/docs/zh/development/style.md index d31cd450b6e..828ef44bfe9 100644 --- a/docs/zh/development/style.md +++ b/docs/zh/development/style.md @@ -261,7 +261,7 @@ void executeQuery( **7.** 不要在每个文件的开头写入垃圾注释(作者,创建日期...)。 -**8.** 单行注释用三个斜杆: `///` ,多行注释以 `/**`开始。 这些注释会当做文档。 +**8.** 单行注释用三个斜杆: `///` ,多行注释以 `/**`开始。 这些注释会当做文档。 注意:您可以使用 Doxygen 从这些注释中生成文档。 但是通常不使用 Doxygen,因为在 IDE 中导航代码更方便。 @@ -349,7 +349,7 @@ class IBlockInputStream bool info_successfully_loaded = false; ``` -**9.** `define` 和全局常量的名称使用带下划线的 `ALL_CAPS`。 +**9.** `define` 和全局常量的名称使用带下划线的 `ALL_CAPS`。 ```cpp #define MAX_SRC_TABLE_NAMES_TO_STORE 1000 @@ -564,7 +564,7 @@ Fork不用于并行化。 **12.** 数值类型。 -使用 `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, 以及 `Int64`, `size_t`, `ssize_t` 还有 `ptrdiff_t`。 +使用 `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, 以及 `Int64`, `size_t`, `ssize_t` 还有 `ptrdiff_t`。 不要使用这些类型:`signed / unsigned long`,`long long`,`short`,`signed / unsigned char`,`char`。 diff --git a/docs/zh/faq/general.md b/docs/zh/faq/general.md index adc0b5791b4..9a1aef5dde6 100644 --- a/docs/zh/faq/general.md +++ b/docs/zh/faq/general.md @@ -2,7 +2,7 @@ ## 为什么不使用MapReduce之类的产品呢? -我们可以将MapReduce这类的系统称为分布式计算系统,其reduce操作基于分布式排序。其中最常见的开源解决方案是[Apache Hadoop](http://hadoop.apache.org)。 Yandex使用他们的内部解决方案YT。 +我们可以将MapReduce这类的系统称为分布式计算系统,其reduce操作基于分布式排序。其中最常见的开源解决方案是 [Apache Hadoop](http://hadoop.apache.org)。 Yandex使用他们的内部解决方案YT。 这些系统不适合在线查询,因为它们的延迟高。换句话说,它们不能用作Web接口的后端服务。这些系统对于实时数据更新是没有用的。如果操作的结果和所有中间结果(如果有的话)位于单个服务器的内存中,则分布式排序不是执行reduce操作的最佳方式,但这通常是在线查询的情况。在这种情况下,哈希表是执行reduce操作的最佳方式。优化map-reduce任务的常用方法是使用内存中的哈希表进行预聚合(部分reduce),用户手动执行此优化操作。分布式排序是运行简单map-reduce任务时性能降低的主要原因之一。 diff --git a/docs/zh/getting_started/example_datasets/nyc_taxi.md b/docs/zh/getting_started/example_datasets/nyc_taxi.md index a5b2bd82dea..16adae18120 100644 --- a/docs/zh/getting_started/example_datasets/nyc_taxi.md +++ b/docs/zh/getting_started/example_datasets/nyc_taxi.md @@ -320,7 +320,7 @@ Two Intel(R) Xeon(R) CPU E5-2650 v2 @ 2.60GHz, 16 physical kernels total,128 GiB 在每台服务器中运行: ``` -CREATE TABLE default.trips_mergetree_third ( trip_id UInt32, vendor_id Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14), pickup_date Date, pickup_datetime DateTime, dropoff_date Date, dropoff_datetime DateTime, store_and_fwd_flag UInt8, rate_code_id UInt8, pickup_longitude Float64, pickup_latitude Float64, dropoff_longitude Float64, dropoff_latitude Float64, passenger_count UInt8, trip_distance Float64, fare_amount Float32, extra Float32, mta_tax Float32, tip_amount Float32, tolls_amount Float32, ehail_fee Float32, improvement_surcharge Float32, total_amount Float32, payment_type_ Enum8('UNK' = 0, 'CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4), trip_type UInt8, pickup FixedString(25), dropoff FixedString(25), cab_type Enum8('yellow' = 1, 'green' = 2, 'uber' = 3), pickup_nyct2010_gid UInt8, pickup_ctlabel Float32, pickup_borocode UInt8, pickup_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), pickup_ct2010 FixedString(6), pickup_boroct2010 FixedString(7), pickup_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), pickup_ntacode FixedString(4), pickup_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), pickup_puma UInt16, dropoff_nyct2010_gid UInt8, dropoff_ctlabel Float32, dropoff_borocode UInt8, dropoff_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), dropoff_ct2010 FixedString(6), dropoff_boroct2010 FixedString(7), dropoff_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), dropoff_ntacode FixedString(4), dropoff_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), dropoff_puma UInt16) ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) +CREATE TABLE default.trips_mergetree_third ( trip_id UInt32, vendor_id Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14), pickup_date Date, pickup_datetime DateTime, dropoff_date Date, dropoff_datetime DateTime, store_and_fwd_flag UInt8, rate_code_id UInt8, pickup_longitude Float64, pickup_latitude Float64, dropoff_longitude Float64, dropoff_latitude Float64, passenger_count UInt8, trip_distance Float64, fare_amount Float32, extra Float32, mta_tax Float32, tip_amount Float32, tolls_amount Float32, ehail_fee Float32, improvement_surcharge Float32, total_amount Float32, payment_type_ Enum8('UNK' = 0, 'CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4), trip_type UInt8, pickup FixedString(25), dropoff FixedString(25), cab_type Enum8('yellow' = 1, 'green' = 2, 'uber' = 3), pickup_nyct2010_gid UInt8, pickup_ctlabel Float32, pickup_borocode UInt8, pickup_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), pickup_ct2010 FixedString(6), pickup_boroct2010 FixedString(7), pickup_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), pickup_ntacode FixedString(4), pickup_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), pickup_puma UInt16, dropoff_nyct2010_gid UInt8, dropoff_ctlabel Float32, dropoff_borocode UInt8, dropoff_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), dropoff_ct2010 FixedString(6), dropoff_boroct2010 FixedString(7), dropoff_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), dropoff_ntacode FixedString(4), dropoff_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), dropoff_puma UInt16) ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) ``` 在之前的服务器中运行: diff --git a/docs/zh/getting_started/example_datasets/ontime.md b/docs/zh/getting_started/example_datasets/ontime.md index 49d48fb1660..b5f78352f4c 100644 --- a/docs/zh/getting_started/example_datasets/ontime.md +++ b/docs/zh/getting_started/example_datasets/ontime.md @@ -166,7 +166,7 @@ SELECT Origin, count(*) AS c FROM ontime WHERE DepDelay>10 AND Year >= 2000 AND Q4. 查询2007年各航空公司延误超过10分钟以上的次数 ```sql -SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC +SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY Carrier ORDER BY count(*) DESC ``` Q5. 查询2007年各航空公司延误超过10分钟以上的百分比 diff --git a/docs/zh/interfaces/cli.md b/docs/zh/interfaces/cli.md index 7a61a315b2b..933b43c9469 100644 --- a/docs/zh/interfaces/cli.md +++ b/docs/zh/interfaces/cli.md @@ -77,7 +77,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA ### 命令行参数 -- `--host, -h` -– 服务端的 host 名称, 默认是 'localhost'。 您可以选择使用 host 名称或者 IPv4 或 IPv6 地址。 +- `--host, -h` -– 服务端的 host 名称, 默认是 'localhost'。 您可以选择使用 host 名称或者 IPv4 或 IPv6 地址。 - `--port` – 连接的端口,默认值: 9000。注意 HTTP 接口以及 TCP 原生接口是使用不同端口的。 - `--user, -u` – 用户名。 默认值: default。 - `--password` – 密码。 默认值: 空字符串。 @@ -93,7 +93,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA ### 配置文件 -`clickhouse-client` 使用一下第一个存在的文件: +`clickhouse-client` 使用一下第一个存在的文件: - 通过 `-config-file` 参数指定的文件. - `./clickhouse-client.xml` diff --git a/docs/zh/interfaces/formats.md b/docs/zh/interfaces/formats.md index e10532174ff..80985542fac 100644 --- a/docs/zh/interfaces/formats.md +++ b/docs/zh/interfaces/formats.md @@ -159,7 +159,7 @@ x=1 y=\N clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv ``` -*默认情况下间隔符是 `,` ,在[format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter)中可以了解更多间隔符配置。 +*默认情况下间隔符是 `,` ,在 [format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter) 中可以了解更多间隔符配置。 解析的时候,可以使用或不使用引号来解析所有值。支持双引号和单引号。行也可以不用引号排列。 在这种情况下,它们被解析为逗号或换行符(CR 或 LF)。在解析不带引号的行时,若违反 RFC 规则,会忽略前导和尾随的空格和制表符。 对于换行,全部支持 Unix(LF),Windows(CR LF)和 Mac OS Classic(CR LF)。 @@ -256,7 +256,7 @@ JSON 与 JavaScript 兼容。为了确保这一点,一些字符被另外转义 该格式仅适用于输出查询结果,但不适用于解析输入(将数据插入到表中)。 -ClickHouse 支持 [NULL](../query_language/syntax.md), 在 JSON 格式中以 `null` 输出来表示. +ClickHouse 支持 [NULL](../query_language/syntax.md), 在 JSON 格式中以 `null` 输出来表示. 参考 JSONEachRow 格式。 @@ -428,10 +428,10 @@ watch -n1 "clickhouse-client --query='SELECT event, value FROM system.events FOR 整数使用固定长度的小端表示法。 例如,UInt64 使用8个字节。 DateTime 被表示为 UInt32 类型的Unix 时间戳值。 Date 被表示为 UInt16 对象,它的值为 1970-01-01以来的天数。 -字符串表示为 varint 长度(无符号[LEB128](https://en.wikipedia.org/wiki/LEB128)),后跟字符串的字节数。 +字符串表示为 varint 长度(无符号 [LEB128](https://en.wikipedia.org/wiki/LEB128)),后跟字符串的字节数。 FixedString 被简单地表示为一个字节序列。 -数组表示为 varint 长度(无符号[LEB128](https://en.wikipedia.org/wiki/LEB128)),后跟有序的数组元素。 +数组表示为 varint 长度(无符号 [LEB128](https://en.wikipedia.org/wiki/LEB128)),后跟有序的数组元素。 对于 [NULL](../query_language/syntax.md#null-literal) 的支持, 一个为 1 或 0 的字节会加在每个 [Nullable](../data_types/nullable.md) 值前面。如果为 1, 那么该值就是 `NULL`。 如果为 0,则不为 `NULL`。 @@ -587,7 +587,7 @@ struct Message { } ``` -格式文件存储的目录可以在服务配置中的[ format_schema_path ](../operations/server_settings/settings.md) 指定。 +格式文件存储的目录可以在服务配置中的 [ format_schema_path ](../operations/server_settings/settings.md) 指定。 Cap'n Proto 反序列化是很高效的,通常不会增加系统的负载。 diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index cbf2f868ce1..8700f0be3c9 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -10,7 +10,7 @@ $ curl 'http://localhost:8123/' Ok. ``` -通过 URL 中的 `query` 参数来发送请求,或者发送 POST 请求,或者将查询的开头部分放在 URL 的 `query` 参数中,其他部分放在 POST 中(我们会在后面解释为什么这样做是有必要的)。URL 的大小会限制在 16 KB,所以发送大型查询时要时刻记住这点。 +通过 URL 中的 `query` 参数来发送请求,或者发送 POST 请求,或者将查询的开头部分放在 URL 的 `query` 参数中,其他部分放在 POST 中(我们会在后面解释为什么这样做是有必要的)。URL 的大小会限制在 16 KB,所以发送大型查询时要时刻记住这点。 如果请求成功,将会收到 200 的响应状态码和响应主体中的结果。 如果发生了某个异常,将会收到 500 的响应状态码和响应主体中的异常描述信息。 diff --git a/docs/zh/interfaces/jdbc.md b/docs/zh/interfaces/jdbc.md index 41bf39ab2f7..61656c35217 100644 --- a/docs/zh/interfaces/jdbc.md +++ b/docs/zh/interfaces/jdbc.md @@ -1,6 +1,6 @@ # JDBC 驱动 -- ClickHouse官方有 JDBC 的驱动。 见[这里](https://github.com/yandex/clickhouse-jdbc)。 +- ClickHouse官方有 JDBC 的驱动。 见 [这里](https://github.com/yandex/clickhouse-jdbc)。 - 三方提供的 JDBC 驱动 [ClickHouse-Native-JDBC](https://github.com/housepower/ClickHouse-Native-JDBC). diff --git a/docs/zh/interfaces/odbc.md b/docs/zh/interfaces/odbc.md index e24e557f090..2de4bee64fc 100644 --- a/docs/zh/interfaces/odbc.md +++ b/docs/zh/interfaces/odbc.md @@ -1,5 +1,5 @@ # ODBC 驱动 -- ClickHouse官方有 ODBC 的驱动。 见[这里](https://github.com/yandex/clickhouse-odbc)。 +- ClickHouse官方有 ODBC 的驱动。 见 [这里](https://github.com/yandex/clickhouse-odbc)。 [来源文章](https://clickhouse.yandex/docs/zh/interfaces/odbc/) diff --git a/docs/zh/interfaces/tcp.md b/docs/zh/interfaces/tcp.md index c60fc2e09aa..7eab7dea1b1 100644 --- a/docs/zh/interfaces/tcp.md +++ b/docs/zh/interfaces/tcp.md @@ -1,5 +1,5 @@ # 原生客户端接口(TCP) -本机协议用于[命令行客户端](cli.md),用于分布式查询处理期间的服务器间通信,以及其他C ++程序。 不幸的是,本机ClickHouse协议还没有正式的规范,但它可以从ClickHouse源代码进行逆向工程[从这里开始](https://github.com/yandex/ClickHouse/tree/master/dbms/src/Client))和/或拦截和分析TCP流量。 +本机协议用于 [命令行客户端](cli.md),用于分布式查询处理期间的服务器间通信,以及其他C ++程序。 不幸的是,本机ClickHouse协议还没有正式的规范,但它可以从ClickHouse源代码进行逆向工程 [从这里开始](https://github.com/yandex/ClickHouse/tree/master/dbms/src/Client))和/或拦截和分析TCP流量。 [来源文章](https://clickhouse.yandex/docs/zh/interfaces/tcp/) diff --git a/docs/zh/interfaces/third-party/gui.md b/docs/zh/interfaces/third-party/gui.md index e2f4a50d336..35e4717a497 100644 --- a/docs/zh/interfaces/third-party/gui.md +++ b/docs/zh/interfaces/third-party/gui.md @@ -40,7 +40,7 @@ ClickHouse Web 界面 [Tabix](https://github.com/tabixio/tabix). ### LightHouse -[LightHouse](https://github.com/VKCOM/lighthouse)是ClickHouse的轻量级Web界面。 +[LightHouse](https://github.com/VKCOM/lighthouse) 是ClickHouse的轻量级Web界面。 特征: @@ -50,7 +50,7 @@ ClickHouse Web 界面 [Tabix](https://github.com/tabixio/tabix). ### clickhouse-cli -[clickhouse-cli](https://github.com/hatarist/clickhouse-cli)是ClickHouse的替代命令行客户端,用Python 3编写。 +[clickhouse-cli](https://github.com/hatarist/clickhouse-cli) 是ClickHouse的替代命令行客户端,用Python 3编写。 特征: - 自动完成。 @@ -62,7 +62,7 @@ ClickHouse Web 界面 [Tabix](https://github.com/tabixio/tabix). ### DBeaver -[DBeaver](https://dbeaver.io/)具有ClickHouse支持的通用桌面数据库客户端。 +[DBeaver](https://dbeaver.io/) 具有ClickHouse支持的通用桌面数据库客户端。 特征: @@ -72,7 +72,7 @@ ClickHouse Web 界面 [Tabix](https://github.com/tabixio/tabix). ### DataGrip -[DataGrip](https://www.jetbrains.com/datagrip/)是JetBrains的数据库IDE,专门支持ClickHouse。 它还嵌入到其他基于IntelliJ的工具中:PyCharm,IntelliJ IDEA,GoLand,PhpStorm等。 +[DataGrip](https://www.jetbrains.com/datagrip/) 是JetBrains的数据库IDE,专门支持ClickHouse。 它还嵌入到其他基于IntelliJ的工具中:PyCharm,IntelliJ IDEA,GoLand,PhpStorm等。 特征: diff --git a/docs/zh/interfaces/third-party/proxy.md b/docs/zh/interfaces/third-party/proxy.md index f4f7c528440..ee5ff42bbb9 100644 --- a/docs/zh/interfaces/third-party/proxy.md +++ b/docs/zh/interfaces/third-party/proxy.md @@ -1,6 +1,6 @@ # 来自第三方开发人员的代理服务器 -[chproxy](https://github.com/Vertamedia/chproxy)是ClickHouse数据库的http代理和负载均衡器。 +[chproxy](https://github.com/Vertamedia/chproxy) 是ClickHouse数据库的http代理和负载均衡器。 特征 @@ -12,7 +12,7 @@ ## KittenHouse -[KittenHouse](https://github.com/VKCOM/kittenhouse)设计为ClickHouse和应用程序服务器之间的本地代理,以防在应用程序端缓冲INSERT数据是不可能或不方便的。 +[KittenHouse](https://github.com/VKCOM/kittenhouse) 设计为ClickHouse和应用程序服务器之间的本地代理,以防在应用程序端缓冲INSERT数据是不可能或不方便的。 特征: @@ -24,7 +24,7 @@ ## ClickHouse-Bulk -[ClickHouse-Bulk](https://github.com/nikepan/clickhouse-bulk)是一个简单的ClickHouse插入收集器。 +[ClickHouse-Bulk](https://github.com/nikepan/clickhouse-bulk) 是一个简单的ClickHouse插入收集器。 特征: diff --git a/docs/zh/introduction/distinctive_features.md b/docs/zh/introduction/distinctive_features.md index 6a9557a301c..0308db9e765 100644 --- a/docs/zh/introduction/distinctive_features.md +++ b/docs/zh/introduction/distinctive_features.md @@ -59,6 +59,6 @@ ClickHouse提供各种各样在允许牺牲数据精度的情况下对查询进 ClickHouse使用异步的多主复制技术。当数据被写入任何一个可用副本后,系统会在后台将数据分发给其他副本,以保证系统在不同副本上保持相同的数据。在大多数情况下ClickHouse能在故障后自动恢复,在一些复杂的情况下需要少量的手动恢复。 -更多信息,参见[数据复制](../operations/table_engines/replication.md)。 +更多信息,参见 [数据复制](../operations/table_engines/replication.md)。 [来源文章](https://clickhouse.yandex/docs/en/introduction/distinctive_features/) diff --git a/docs/zh/introduction/features_considered_disadvantages.md b/docs/zh/introduction/features_considered_disadvantages.md index 7a1c2721584..4deea397c2f 100644 --- a/docs/zh/introduction/features_considered_disadvantages.md +++ b/docs/zh/introduction/features_considered_disadvantages.md @@ -1,7 +1,7 @@ # ClickHouse可以考虑缺点的功能 1. 没有完整的事物支持。 -2. 缺少高频率,低延迟的修改或删除已存在数据的能力。仅能用于批量删除或修改数据,但这符合[GDPR](https://gdpr-info.eu)。 +2. 缺少高频率,低延迟的修改或删除已存在数据的能力。仅能用于批量删除或修改数据,但这符合 [GDPR](https://gdpr-info.eu)。 3. 稀疏索引使得ClickHouse不适合通过其键检索单行的点查询。 [来源文章](https://clickhouse.yandex/docs/zh/introduction/features_considered_disadvantages/) diff --git a/docs/zh/introduction/performance.md b/docs/zh/introduction/performance.md index f06e6892187..a740db13995 100644 --- a/docs/zh/introduction/performance.md +++ b/docs/zh/introduction/performance.md @@ -1,8 +1,8 @@ # Performance -根据Yandex的内部测试结果,ClickHouse表现出了比同类可比较产品更优的性能。你可以在[这里](https://clickhouse.yandex/benchmark.html)查看具体的测试结果。 +根据Yandex的内部测试结果,ClickHouse表现出了比同类可比较产品更优的性能。你可以在 [这里](https://clickhouse.yandex/benchmark.html) 查看具体的测试结果。 -许多其他的测试也证实这一点。你可以使用互联网搜索到它们,或者你也可以从[我们收集的部分相关连接](https://clickhouse.yandex/#independent-benchmarks)中查看。 +许多其他的测试也证实这一点。你可以使用互联网搜索到它们,或者你也可以从 [我们收集的部分相关连接](https://clickhouse.yandex/#independent-benchmarks) 中查看。 ## 单个大查询的吞吐量 diff --git a/docs/zh/introduction/ya_metrika_task.md b/docs/zh/introduction/ya_metrika_task.md index 72d8f1bddd5..f4758ae4e99 100644 --- a/docs/zh/introduction/ya_metrika_task.md +++ b/docs/zh/introduction/ya_metrika_task.md @@ -1,6 +1,6 @@ # Yandex.Metrica的使用案例 -ClickHouse最初是为[Yandex.Metrica](https://metrica.yandex.com/)[世界第二大Web分析平台](http://w3techs.com/technologies/overview/traffic_analysis/all)而开发的。多年来一直作为该系统的核心组件被该系统持续使用着。目前为止,该系统在ClickHouse中有超过13万亿条记录,并且每天超过200多亿个事件被处理。它允许直接从原始数据中动态查询并生成报告。本文简要介绍了ClickHouse在其早期发展阶段的目标。 +ClickHouse最初是为 [Yandex.Metrica](https://metrica.yandex.com/) [世界第二大Web分析平台](http://w3techs.com/technologies/overview/traffic_analysis/all) 而开发的。多年来一直作为该系统的核心组件被该系统持续使用着。目前为止,该系统在ClickHouse中有超过13万亿条记录,并且每天超过200多亿个事件被处理。它允许直接从原始数据中动态查询并生成报告。本文简要介绍了ClickHouse在其早期发展阶段的目标。 Yandex.Metrica基于用户定义的字段,对实时访问、连接会话,生成实时的统计报表。这种需求往往需要复杂聚合方式,比如对访问用户进行去重。构建报表的数据,是实时接收存储的新数据。 diff --git a/docs/zh/operations/configuration_files.md b/docs/zh/operations/configuration_files.md index acbc37634d8..eafa5d34235 100644 --- a/docs/zh/operations/configuration_files.md +++ b/docs/zh/operations/configuration_files.md @@ -12,7 +12,7 @@ If `replace` is specified, it replaces the entire element with the specified one If `remove` is specified, it deletes the element. -The config can also define "substitutions". If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](server_settings/settings.md#server_settings-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros](#macros) server_settings/settings.md)). +The config can also define "substitutions". If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](server_settings/settings.md#server_settings-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in `incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros](#macros) server_settings/settings.md)). Substitutions can also be performed from ZooKeeper. To do this, specify the attribute `from_zk = "/path/to/node"`. The element value is replaced with the contents of the node at `/path/to/node` in ZooKeeper. You can also put an entire XML subtree on the ZooKeeper node and it will be fully inserted into the source element. diff --git a/docs/zh/operations/server_settings/settings.md b/docs/zh/operations/server_settings/settings.md index 0eadd7ebcec..5b86bc068c5 100644 --- a/docs/zh/operations/server_settings/settings.md +++ b/docs/zh/operations/server_settings/settings.md @@ -42,7 +42,7 @@ Block field ``: - ``min_part_size_ratio`` – The ratio of the minimum size of a table part to the full size of the table. - ``method`` – Compression method. Acceptable values ​: ``lz4`` or ``zstd``(experimental). -ClickHouse checks `min_part_size` and `min_part_size_ratio` and processes the `case` blocks that match these conditions. If none of the `` matches, ClickHouse applies the `lz4` compression algorithm. +ClickHouse checks `min_part_size` and `min_part_size_ratio` and processes the `case` blocks that match these conditions. If none of the `` matches, ClickHouse applies the `lz4` compression algorithm. **Example** @@ -456,7 +456,7 @@ Keys for server/client settings: - requireTLSv1 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips – Activates OpenSSL FIPS mode. Supported if the library's OpenSSL version supports FIPS. - privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ````, ``KeyFileHandler``, ``test``, ````. -- invalidCertificateHandler – Class (subclass of CertificateHandler) for verifying invalid certificates. For example: `` ConsoleCertificateHandler `` . +- invalidCertificateHandler – Class (subclass of CertificateHandler) for verifying invalid certificates. For example: `` ConsoleCertificateHandler `` . - disableProtocols – Protocols that are not allowed to use. - preferServerCiphers – Preferred server ciphers on the client. @@ -640,7 +640,7 @@ The uncompressed cache is advantageous for very short queries in individual case ## user_files_path {#server_settings-user_files_path} -The directory with user files. Used in the table function [file()](../../query_language/table_functions/file.md). +The directory with user files. Used in the table function [file()](../../query_language/table_functions/file.md). **Example** diff --git a/docs/zh/operations/settings/settings.md b/docs/zh/operations/settings/settings.md index 3a456749a78..4a40828babb 100644 --- a/docs/zh/operations/settings/settings.md +++ b/docs/zh/operations/settings/settings.md @@ -28,7 +28,7 @@ Forces a query to an out-of-date replica if updated data is not available. See " ClickHouse selects the most relevant from the outdated replicas of the table. -Used when performing `SELECT` from a distributed table that points to replicated tables. +Used when performing `SELECT` from a distributed table that points to replicated tables. By default, 1 (enabled). @@ -38,7 +38,7 @@ Disables query execution if the index can't be used by date. Works with tables in the MergeTree family. -If `force_index_by_date=1`, ClickHouse checks whether the query has a date key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For example, the condition `Date != ' 2000-01-01 '` is acceptable even when it matches all the data in the table (i.e., running the query requires a full scan). For more information about ranges of data in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". +If `force_index_by_date=1`, ClickHouse checks whether the query has a date key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For example, the condition `Date != ' 2000-01-01 '` is acceptable even when it matches all the data in the table (i.e., running the query requires a full scan). For more information about ranges of data in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". ## force_primary_key @@ -47,7 +47,7 @@ Disables query execution if indexing by the primary key is not possible. Works with tables in the MergeTree family. -If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For more information about data ranges in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". +If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For more information about data ranges in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". ## fsync_metadata @@ -126,7 +126,7 @@ Sets the time in seconds. If a replica lags more than the set value, this replic Default value: 0 (off). -Used when performing `SELECT` from a distributed table that points to replicated tables. +Used when performing `SELECT` from a distributed table that points to replicated tables. ## max_threads {#settings-max_threads} @@ -311,7 +311,7 @@ Replica lag is not controlled. Enable compilation of queries. By default, 0 (disabled). Compilation is only used for part of the query-processing pipeline: for the first stage of aggregation (GROUP BY). -If this portion of the pipeline was compiled, the query may run faster due to deployment of short cycles and inlining aggregate function calls. The maximum performance improvement (up to four times faster in rare cases) is seen for queries with multiple simple aggregate functions. Typically, the performance gain is insignificant. In very rare cases, it may slow down query execution. +If this portion of the pipeline was compiled, the query may run faster due to deployment of short cycles and inlining aggregate function calls. The maximum performance improvement (up to four times faster in rare cases) is seen for queries with multiple simple aggregate functions. Typically, the performance gain is insignificant. In very rare cases, it may slow down query execution. ## min_count_to_compile @@ -329,7 +329,7 @@ It works for JSONEachRow and TSKV formats. ## output_format_json_quote_64bit_integers -If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes. +If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes. ## format_csv_delimiter {#format_csv_delimiter} @@ -358,7 +358,7 @@ The default value is 0. All the replicas in the quorum are consistent, i.e., they contain data from all previous `INSERT` queries. The `INSERT` sequence is linearized. -When reading the data written from the `insert_quorum`, you can use the[select_sequential_consistency](#select-sequential-consistency) option. +When reading the data written from the `insert_quorum`, you can use the [select_sequential_consistency](#select-sequential-consistency) option. **ClickHouse generates an exception** diff --git a/docs/zh/operations/settings/settings_profiles.md b/docs/zh/operations/settings/settings_profiles.md index 3953ada8a83..c335e249212 100644 --- a/docs/zh/operations/settings/settings_profiles.md +++ b/docs/zh/operations/settings/settings_profiles.md @@ -2,7 +2,7 @@ # Settings profiles A settings profile is a collection of settings grouped under the same name. Each ClickHouse user has a profile. -To apply all the settings in a profile, set the `profile` setting. +To apply all the settings in a profile, set the `profile` setting. Example: @@ -58,9 +58,9 @@ Example: ``` -The example specifies two profiles: `default` and `web`. The `default` profile has a special purpose: it must always be present and is applied when starting the server. In other words, the `default` profile contains default settings. The `web` profile is a regular profile that can be set using the `SET` query or using a URL parameter in an HTTP query. +The example specifies two profiles: `default` and `web`. The `default` profile has a special purpose: it must always be present and is applied when starting the server. In other words, the `default` profile contains default settings. The `web` profile is a regular profile that can be set using the `SET` query or using a URL parameter in an HTTP query. -Settings profiles can inherit from each other. To use inheritance, indicate the `profile` setting before the other settings that are listed in the profile. +Settings profiles can inherit from each other. To use inheritance, indicate the `profile` setting before the other settings that are listed in the profile. [Original article](https://clickhouse.yandex/docs/en/operations/settings/settings_profiles/) diff --git a/docs/zh/operations/system_tables.md b/docs/zh/operations/system_tables.md index b41f7f02b95..d15d392d5f9 100644 --- a/docs/zh/operations/system_tables.md +++ b/docs/zh/operations/system_tables.md @@ -141,7 +141,7 @@ Formats: - active (UInt8) – Indicates whether the part is active. If a part is active, it is used in a table; otherwise, it will be deleted. Inactive data parts remain after merging. -- marks (UInt64) – The number of marks. To get the approximate number of rows in a data part, multiply ``marks`` by the index granularity (usually 8192). +- marks (UInt64) – The number of marks. To get the approximate number of rows in a data part, multiply ``marks`` by the index granularity (usually 8192). - marks_size (UInt64) – The size of the file with marks. @@ -373,7 +373,7 @@ If the path specified in 'path' doesn't exist, an exception will be thrown. Columns: -- `name String` — The name of the node. +- `name String` — The name of the node. - `path String` — The path to the node. - `value String` — Node value. - `dataLength Int32` — Size of the value. diff --git a/docs/zh/operations/table_engines/custom_partitioning_key.md b/docs/zh/operations/table_engines/custom_partitioning_key.md index 98df0bfc1c8..9e723efd42e 100644 --- a/docs/zh/operations/table_engines/custom_partitioning_key.md +++ b/docs/zh/operations/table_engines/custom_partitioning_key.md @@ -38,7 +38,7 @@ In the `system.parts` table, the `partition` column specifies the value of the p Old: `20140317_20140323_2_2_0` (minimum date - maximum date - minimum block number - maximum block number - level). -Now: `201403_2_2_0` (partition ID - minimum block number - maximum block number - level). +Now: `201403_2_2_0` (partition ID - minimum block number - maximum block number - level). The partition ID is its string identifier (human-readable, if possible) that is used for the names of data parts in the file system and in ZooKeeper. You can specify it in ALTER queries in place of the partition key. Example: Partition key `toYYYYMM(EventDate)`; ALTER can specify either `PARTITION 201710` or `PARTITION ID '201710'`. diff --git a/docs/zh/operations/table_engines/kafka.md b/docs/zh/operations/table_engines/kafka.md index d592c5a958e..e66b3130f4a 100644 --- a/docs/zh/operations/table_engines/kafka.md +++ b/docs/zh/operations/table_engines/kafka.md @@ -121,7 +121,7 @@ Kafka SETTINGS 与 `GraphiteMergeTree` 类似,Kafka 引擎支持使用ClickHouse配置文件进行扩展配置。可以使用两个配置键:全局 (`kafka`) 和 主题级别 (`kafka_*`)。首先应用全局配置,然后应用主题级配置(如果存在)。 ```xml - + cgrp smallest diff --git a/docs/zh/operations/table_engines/merge.md b/docs/zh/operations/table_engines/merge.md index e8de53bc286..366a5459bf8 100644 --- a/docs/zh/operations/table_engines/merge.md +++ b/docs/zh/operations/table_engines/merge.md @@ -1,8 +1,8 @@ # Merge -The `Merge` engine (not to be confused with `MergeTree`) does not store data itself, but allows reading from any number of other tables simultaneously. +The `Merge` engine (not to be confused with `MergeTree`) does not store data itself, but allows reading from any number of other tables simultaneously. Reading is automatically parallelized. Writing to a table is not supported. When reading, the indexes of tables that are actually being read are used, if they exist. -The `Merge` engine accepts parameters: the database name and a regular expression for tables. +The `Merge` engine accepts parameters: the database name and a regular expression for tables. Example: @@ -10,17 +10,17 @@ Example: Merge(hits, '^WatchLog') ``` -Data will be read from the tables in the `hits` database that have names that match the regular expression '`^WatchLog`'. +Data will be read from the tables in the `hits` database that have names that match the regular expression '`^WatchLog`'. Instead of the database name, you can use a constant expression that returns a string. For example, `currentDatabase()`. Regular expressions — [re2](https://github.com/google/re2) (supports a subset of PCRE), case-sensitive. See the notes about escaping symbols in regular expressions in the "match" section. -When selecting tables to read, the `Merge` table itself will not be selected, even if it matches the regex. This is to avoid loops. -It is possible to create two `Merge` tables that will endlessly try to read each others' data, but this is not a good idea. +When selecting tables to read, the `Merge` table itself will not be selected, even if it matches the regex. This is to avoid loops. +It is possible to create two `Merge` tables that will endlessly try to read each others' data, but this is not a good idea. -The typical way to use the `Merge` engine is for working with a large number of `TinyLog` tables as if with a single table. +The typical way to use the `Merge` engine is for working with a large number of `TinyLog` tables as if with a single table. Example 2: @@ -63,7 +63,7 @@ Virtual columns differ from normal columns in the following ways: The `Merge` type table contains a virtual `_table` column of the `String` type. (If the table already has a `_table` column, the virtual column is called `_table1`; if you already have `_table1`, it's called `_table2`, and so on.) It contains the name of the table that data was read from. -If the `WHERE/PREWHERE` clause contains conditions for the `_table` column that do not depend on other table columns (as one of the conjunction elements, or as an entire expression), these conditions are used as an index. The conditions are performed on a data set of table names to read data from, and the read operation will be performed from only those tables that the condition was triggered on. +If the `WHERE/PREWHERE` clause contains conditions for the `_table` column that do not depend on other table columns (as one of the conjunction elements, or as an entire expression), these conditions are used as an index. The conditions are performed on a data set of table names to read data from, and the read operation will be performed from only those tables that the condition was triggered on. [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/merge/) diff --git a/docs/zh/operations/table_engines/mergetree.md b/docs/zh/operations/table_engines/mergetree.md index 3481e9c54ef..0782e2b242d 100644 --- a/docs/zh/operations/table_engines/mergetree.md +++ b/docs/zh/operations/table_engines/mergetree.md @@ -61,7 +61,7 @@ For a description of request parameters, see [request description](../../query_l By default the primary key is the same as the sorting key (which is specified by the `ORDER BY` clause). Thus in most cases it is unnecessary to specify a separate `PRIMARY KEY` clause. -- `SAMPLE BY` — An expression for sampling. +- `SAMPLE BY` — An expression for sampling. If a sampling expression is used, the primary key must contain it. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. diff --git a/docs/zh/operations/table_engines/replacingmergetree.md b/docs/zh/operations/table_engines/replacingmergetree.md index 2d0e179ac30..568f94822dd 100644 --- a/docs/zh/operations/table_engines/replacingmergetree.md +++ b/docs/zh/operations/table_engines/replacingmergetree.md @@ -4,7 +4,7 @@ The engine differs from [MergeTree](mergetree.md) in that it removes duplicate e Data deduplication occurs only during a merge. Merging occurs in the background at an unknown time, so you can't plan for it. Some of the data may remain unprocessed. Although you can run an unscheduled merge using the `OPTIMIZE` query, don't count on using it, because the `OPTIMIZE` query will read and write a large amount of data. -Thus, `ReplacingMergeTree` is suitable for clearing out duplicate data in the background in order to save space, but it doesn't guarantee the absence of duplicates. +Thus, `ReplacingMergeTree` is suitable for clearing out duplicate data in the background in order to save space, but it doesn't guarantee the absence of duplicates. ## Creating a Table @@ -33,7 +33,7 @@ For a description of request parameters, see [request description](../../query_l **Query clauses** -When creating a `ReplacingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table. +When creating a `ReplacingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table.
Deprecated Method for Creating a Table diff --git a/docs/zh/operations/table_engines/summingmergetree.md b/docs/zh/operations/table_engines/summingmergetree.md index 2013c7142b9..d1e35124fe0 100644 --- a/docs/zh/operations/table_engines/summingmergetree.md +++ b/docs/zh/operations/table_engines/summingmergetree.md @@ -1,9 +1,9 @@ # SummingMergeTree -The engine inherits from [MergeTree](mergetree.md). The difference is that when merging data parts for `SummingMergeTree` tables ClickHouse replaces all the rows with the same primary key with one row which contains summarized values for the columns with the numeric data type. If the primary key is composed in a way that a single key value corresponds to large number of rows, this significantly reduces storage volume and speeds up data selection. +The engine inherits from [MergeTree](mergetree.md). The difference is that when merging data parts for `SummingMergeTree` tables ClickHouse replaces all the rows with the same primary key with one row which contains summarized values for the columns with the numeric data type. If the primary key is composed in a way that a single key value corresponds to large number of rows, this significantly reduces storage volume and speeds up data selection. -We recommend to use the engine together with `MergeTree`. Store complete data in `MergeTree` table, and use `SummingMergeTree` for aggregated data storing, for example, when preparing reports. Such an approach will prevent you from losing valuable data due to an incorrectly composed primary key. +We recommend to use the engine together with `MergeTree`. Store complete data in `MergeTree` table, and use `SummingMergeTree` for aggregated data storing, for example, when preparing reports. Such an approach will prevent you from losing valuable data due to an incorrectly composed primary key. ## Creating a Table @@ -31,7 +31,7 @@ The columns must be of a numeric type and must not be in the primary key. **Query clauses** -When creating a `SummingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table. +When creating a `SummingMergeTree` table the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table.
Deprecated Method for Creating a Table @@ -72,7 +72,7 @@ Insert data to it: :) INSERT INTO summtt Values(1,1),(1,2),(2,1) ``` -ClickHouse may sum all the rows not completely ([see below](#data-processing)), so we use an aggregate function `sum` and `GROUP BY` clause in the query. +ClickHouse may sum all the rows not completely ([see below](#data-processing)), so we use an aggregate function `sum` and `GROUP BY` clause in the query. ```sql SELECT key, sum(value) FROM summtt GROUP BY key diff --git a/docs/zh/operations/tips.md b/docs/zh/operations/tips.md index a34f79f869f..11b7a782deb 100644 --- a/docs/zh/operations/tips.md +++ b/docs/zh/operations/tips.md @@ -18,7 +18,7 @@ You can use `turbostat` to view the CPU's actual clock rate under a load. ## CPU Scaling Governor -Always use the `performance` scaling governor. The `on-demand` scaling governor works much worse with constantly high demand. +Always use the `performance` scaling governor. The `on-demand` scaling governor works much worse with constantly high demand. ```bash sudo echo 'performance' | tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor diff --git a/docs/zh/query_language/create.md b/docs/zh/query_language/create.md index 332b53f812a..78364e45afa 100644 --- a/docs/zh/query_language/create.md +++ b/docs/zh/query_language/create.md @@ -43,7 +43,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ... 以上所有情况,如果指定了`IF NOT EXISTS`,那么在该表已经存在的情况下,查询不会返回任何错误。在这种情况下,查询几乎不会做任何事情。 -在`ENGINE`子句后还可能存在一些其他的子句,更详细的信息可以参考[表引擎](../operations/table_engines/index.md)中关于建表的描述。 +在`ENGINE`子句后还可能存在一些其他的子句,更详细的信息可以参考 [表引擎](../operations/table_engines/index.md) 中关于建表的描述。 ### 默认值 diff --git a/docs/zh/query_language/insert_into.md b/docs/zh/query_language/insert_into.md index 58af549fc35..33caac11d42 100644 --- a/docs/zh/query_language/insert_into.md +++ b/docs/zh/query_language/insert_into.md @@ -14,9 +14,9 @@ INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), . - 如果存在`DEFAULT`表达式,根据`DEFAULT`表达式计算被填充的值。 - 如果没有定义`DEFAULT`表达式,则填充零或空字符串。 -如果[strict_insert_defaults=1](../operations/settings/settings.md),你必须在查询中列出所有没有定义`DEFAULT`表达式的列。 +如果 [strict_insert_defaults=1](../operations/settings/settings.md),你必须在查询中列出所有没有定义`DEFAULT`表达式的列。 -数据可以以ClickHouse支持的任何[输入输出格式](../interfaces/formats.md#formats)传递给INSERT。格式的名称必须显示的指定在查询中: +数据可以以ClickHouse支持的任何 [输入输出格式](../interfaces/formats.md#formats) 传递给INSERT。格式的名称必须显示的指定在查询中: ``` sql INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set diff --git a/docs/zh/query_language/select.md b/docs/zh/query_language/select.md index 607f26ded71..8786be6e208 100644 --- a/docs/zh/query_language/select.md +++ b/docs/zh/query_language/select.md @@ -339,7 +339,7 @@ ARRAY JOIN nest AS n, arrayEnumerate(`nest.x`) AS num JOIN子句用于连接数据,作用与[SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL))的定义相同。 !!! info "注意" - 与[ARRAY JOIN](#select-array-join-clause)没有关系. + 与 [ARRAY JOIN](#select-array-join-clause) 没有关系. ``` sql @@ -365,7 +365,7 @@ FROM 在使用`ALL`修饰符对JOIN进行修饰时,如果右表中存在多个与左表关联的数据,那么系统则将右表中所有可以与左表关联的数据全部返回在结果中。这与SQL标准的JOIN行为相同。 在使用`ANY`修饰符对JOIN进行修饰时,如果右表中存在多个与左表关联的数据,那么系统仅返回第一个与左表匹配的结果。如果左表与右表一一对应,不存在多余的行时,`ANY`与`ALL`的结果相同。 -你可以在会话中通过设置[join_default_strictness](../operations/settings/settings.md)来指定默认的JOIN修饰符。 +你可以在会话中通过设置 [join_default_strictness](../operations/settings/settings.md) 来指定默认的JOIN修饰符。 **`GLOBAL` distribution** @@ -373,7 +373,7 @@ FROM 当使用`GLOBAL ... JOIN`,首先会在请求服务器上计算右表并以临时表的方式将其发送到所有服务器。这时每台服务器将直接使用它进行计算。 -使用`GLOBAL`时需要小心。更多信息,参阅[Distributed subqueries](#select-distributed-subqueries)部分。 +使用`GLOBAL`时需要小心。更多信息,参阅 [Distributed subqueries](#select-distributed-subqueries) 部分。 **使用建议** @@ -435,13 +435,13 @@ LIMIT 10 在一些场景下,使用`IN`代替`JOIN`将会得到更高的效率。在各种类型的JOIN中,最高效的是`ANY LEFT JOIN`,然后是`ANY INNER JOIN`,效率最差的是`ALL LEFT JOIN`以及`ALL INNER JOIN`。 -如果你需要使用`JOIN`来关联一些纬度表(包含纬度属性的一些相对比较小的表,例如广告活动的名称),那么`JOIN`可能不是好的选择,因为语法负责,并且每次查询都将重新访问这些表。对于这种情况,您应该使用“外部字典”的功能来替换`JOIN`。更多信息,参见[外部字典](dicts/external_dicts.md)部分。 +如果你需要使用`JOIN`来关联一些纬度表(包含纬度属性的一些相对比较小的表,例如广告活动的名称),那么`JOIN`可能不是好的选择,因为语法负责,并且每次查询都将重新访问这些表。对于这种情况,您应该使用“外部字典”的功能来替换`JOIN`。更多信息,参见 [外部字典](dicts/external_dicts.md) 部分。 #### Null的处理 -JOIN的行为受[join_use_nulls](../operations/settings/settings.md)的影响。当`join_use_nulls=1`时,`JOIN`的工作与SQL标准相同。 +JOIN的行为受 [join_use_nulls](../operations/settings/settings.md) 的影响。当`join_use_nulls=1`时,`JOIN`的工作与SQL标准相同。 -如果JOIN的key是[Nullable](../data_types/nullable.md)类型的字段,则其中至少一个存在[NULL](syntax.md)值的key不会被关联。 +如果JOIN的key是 [Nullable](../data_types/nullable.md) 类型的字段,则其中至少一个存在 [NULL](syntax.md) 值的key不会被关联。 ### WHERE 子句 @@ -511,7 +511,7 @@ GROUP BY子句会为遇到的每一个不同的key计算一组聚合函数的值 #### NULL 处理 -对于GROUP BY子句,ClickHouse将[NULL](syntax.md)解释为一个值,并且支持`NULL=NULL`。 +对于GROUP BY子句,ClickHouse将 [NULL](syntax.md) 解释为一个值,并且支持`NULL=NULL`。 下面这个例子将说明这将意味着什么。 @@ -693,7 +693,7 @@ WHERE于HAVING不同之处在于WHERE在聚合前(GROUP BY)执行,HAVING在聚 在SELECT表达式中存在Array类型的列时,不能使用DISTINCT。 -`DISTINCT`可以与[NULL](syntax.md)一起工作,就好像`NULL`仅是一个特殊的值一样,并且`NULL=NULL`。换而言之,在`DISTINCT`的结果中,与`NULL`不同的组合仅能出现一次。 +`DISTINCT`可以与 [NULL](syntax.md)一起工作,就好像`NULL`仅是一个特殊的值一样,并且`NULL=NULL`。换而言之,在`DISTINCT`的结果中,与`NULL`不同的组合仅能出现一次。 ### LIMIT 子句 @@ -814,7 +814,7 @@ IN子句中的子查询仅在单个服务器上运行一次。不能够是相关 #### NULL 处理 -在处理中,IN操作符总是假定[NULL](syntax.md)值的操作结果总是等于`0`,而不管`NULL`位于左侧还是右侧。`NULL`值不应该包含在任何数据集中,它们彼此不能够对应,并且不能够比较。 +在处理中,IN操作符总是假定 [NULL](syntax.md) 值的操作结果总是等于`0`,而不管`NULL`位于左侧还是右侧。`NULL`值不应该包含在任何数据集中,它们彼此不能够对应,并且不能够比较。 下面的示例中有一个`t_null`表: @@ -851,11 +851,11 @@ FROM t_null 对于带有子查询的(类似与JOINs)IN中,有两种选择:普通的`IN`/`JOIN`与`GLOBAL IN` / `GLOBAL JOIN`。它们对于分布式查询的处理运行方式是不同的。 !!! 注意 - 请记住,下面描述的算法可能因为根据[settings](../operations/settings/settings.md)配置的不同而不同。 + 请记住,下面描述的算法可能因为根据 [settings](../operations/settings/settings.md) 配置的不同而不同。 当使用普通的IN时,查询总是被发送到远程的服务器,并且在每个服务器中运行“IN”或“JOIN”子句中的子查询。 -当使用`GLOBAL IN` / `GLOBAL JOIN`时,首先会为`GLOBAL IN` / `GLOBAL JOIN`运行所有子查询,并将结果收集到临时表中,并将临时表发送到每个远程服务器,并使用该临时表运行查询。 +当使用`GLOBAL IN` / `GLOBAL JOIN`时,首先会为`GLOBAL IN` / `GLOBAL JOIN`运行所有子查询,并将结果收集到临时表中,并将临时表发送到每个远程服务器,并使用该临时表运行查询。 对于非分布式查询,请使用普通的`IN` / `JOIN`。 From 3500d5a72eeb4a971f35d56fe98810c2f0c00e65 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Dec 2018 20:30:53 +0300 Subject: [PATCH 091/230] Removed the only test case that use "mouse_clicks" table [#CLICKHOUSE-2] --- dbms/tests/queries/1_stateful/00003_count_mouse_clicks.reference | 1 - dbms/tests/queries/1_stateful/00003_count_mouse_clicks.sql | 1 - 2 files changed, 2 deletions(-) delete mode 100644 dbms/tests/queries/1_stateful/00003_count_mouse_clicks.reference delete mode 100644 dbms/tests/queries/1_stateful/00003_count_mouse_clicks.sql diff --git a/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.reference b/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.reference deleted file mode 100644 index a0f1f03d797..00000000000 --- a/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.reference +++ /dev/null @@ -1 +0,0 @@ -5483925 diff --git a/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.sql b/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.sql deleted file mode 100644 index 3ed06b0facb..00000000000 --- a/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT count() FROM test.mouse_clicks From 6758c76da3858f009986bd4875e7cfd6ea501b00 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Dec 2018 20:30:53 +0300 Subject: [PATCH 092/230] Removed the only test case that use "mouse_clicks" table [#CLICKHOUSE-2] --- dbms/tests/queries/1_stateful/00003_count_mouse_clicks.reference | 1 - dbms/tests/queries/1_stateful/00003_count_mouse_clicks.sql | 1 - 2 files changed, 2 deletions(-) delete mode 100644 dbms/tests/queries/1_stateful/00003_count_mouse_clicks.reference delete mode 100644 dbms/tests/queries/1_stateful/00003_count_mouse_clicks.sql diff --git a/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.reference b/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.reference deleted file mode 100644 index a0f1f03d797..00000000000 --- a/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.reference +++ /dev/null @@ -1 +0,0 @@ -5483925 diff --git a/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.sql b/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.sql deleted file mode 100644 index 3ed06b0facb..00000000000 --- a/dbms/tests/queries/1_stateful/00003_count_mouse_clicks.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT count() FROM test.mouse_clicks From 3a8f3a05de238fec3e9726bd1bf40a1bf044cff6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Dec 2018 20:45:02 +0300 Subject: [PATCH 093/230] Fixed test #3914 --- dbms/src/Functions/FunctionsRound.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 4d8f5edccc3..0386cdab30d 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -139,7 +139,7 @@ struct IntegerRoundingComputation static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) { - if (scale > size_t(std::numeric_limits::max())) + if (scale > std::make_unsigned_t(std::numeric_limits::max())) *out = 0; else *out = compute(*in, scale); @@ -331,10 +331,10 @@ public: template static NO_INLINE void applyImpl(const PaddedPODArray & in, typename ColumnVector::Container & out) { - const T* end_in = in.data() + in.size(); + const T * end_in = in.data() + in.size(); - const T* __restrict p_in = in.data(); - T* __restrict p_out = out.data(); + const T * __restrict p_in = in.data(); + T * __restrict p_out = out.data(); while (p_in < end_in) { From a66ce65b22f18de820f484b312a825438cb345bb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Dec 2018 20:52:28 +0300 Subject: [PATCH 094/230] Fixed test, try 2 #3914 --- dbms/src/Functions/FunctionsRound.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 0386cdab30d..862e383c51c 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -139,7 +139,7 @@ struct IntegerRoundingComputation static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) { - if (scale > std::make_unsigned_t(std::numeric_limits::max())) + if (sizeof(T) <= sizeof(scale) && scale > size_t(std::numeric_limits::max())) *out = 0; else *out = compute(*in, scale); From 877750cd6a0d500e0a66ba095debcca8542ef3c9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Dec 2018 20:45:02 +0300 Subject: [PATCH 095/230] Fixed test #3914 --- dbms/src/Functions/FunctionsRound.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 4d8f5edccc3..0386cdab30d 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -139,7 +139,7 @@ struct IntegerRoundingComputation static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) { - if (scale > size_t(std::numeric_limits::max())) + if (scale > std::make_unsigned_t(std::numeric_limits::max())) *out = 0; else *out = compute(*in, scale); @@ -331,10 +331,10 @@ public: template static NO_INLINE void applyImpl(const PaddedPODArray & in, typename ColumnVector::Container & out) { - const T* end_in = in.data() + in.size(); + const T * end_in = in.data() + in.size(); - const T* __restrict p_in = in.data(); - T* __restrict p_out = out.data(); + const T * __restrict p_in = in.data(); + T * __restrict p_out = out.data(); while (p_in < end_in) { From a86818051625d743874ff0d82c6748104402e497 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Dec 2018 20:52:28 +0300 Subject: [PATCH 096/230] Fixed test, try 2 #3914 --- dbms/src/Functions/FunctionsRound.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 0386cdab30d..862e383c51c 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -139,7 +139,7 @@ struct IntegerRoundingComputation static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) { - if (scale > std::make_unsigned_t(std::numeric_limits::max())) + if (sizeof(T) <= sizeof(scale) && scale > size_t(std::numeric_limits::max())) *out = 0; else *out = compute(*in, scale); From fc4f7d2e77255119243340b3f75e36b3fd33bd90 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Dec 2018 21:06:38 +0300 Subject: [PATCH 097/230] Investigation #3921 --- dbms/src/Functions/FunctionsConversion.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 178df33853f..d0783a94a4b 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -40,6 +40,8 @@ #include #include +#include + namespace DB { @@ -1186,6 +1188,9 @@ struct ToIntMonotonicity ? !from_is_unsigned : right.get() >= 0; + DUMP(left, right); + DUMP(from_is_unsigned, to_is_unsigned, size_of_from, size_of_to, left_in_first_half, right_in_first_half) + /// Size of type is the same. if (size_of_from == size_of_to) { From 4ee7b80ab4c0c57e984dab6df326ff2445f3917e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Dec 2018 21:40:47 +0300 Subject: [PATCH 098/230] Fixed error #3870 --- dbms/src/Functions/FunctionsConversion.h | 23 ++++++++++------------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index d0783a94a4b..5df8547bf4a 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -40,8 +40,6 @@ #include #include -#include - namespace DB { @@ -1174,22 +1172,19 @@ struct ToIntMonotonicity /// Integer cases. - bool from_is_unsigned = type.isValueRepresentedByUnsignedInteger(); - bool to_is_unsigned = std::is_unsigned_v; + const bool from_is_unsigned = type.isValueRepresentedByUnsignedInteger(); + const bool to_is_unsigned = std::is_unsigned_v; - size_t size_of_from = type.getSizeOfValueInMemory(); - size_t size_of_to = sizeof(T); + const size_t size_of_from = type.getSizeOfValueInMemory(); + const size_t size_of_to = sizeof(T); - bool left_in_first_half = left.isNull() + const bool left_in_first_half = left.isNull() ? from_is_unsigned - : left.get() >= 0; + : (left.get() >= 0); - bool right_in_first_half = right.isNull() + const bool right_in_first_half = right.isNull() ? !from_is_unsigned - : right.get() >= 0; - - DUMP(left, right); - DUMP(from_is_unsigned, to_is_unsigned, size_of_from, size_of_to, left_in_first_half, right_in_first_half) + : (right.get() >= 0); /// Size of type is the same. if (size_of_from == size_of_to) @@ -1215,6 +1210,8 @@ struct ToIntMonotonicity /// signed -> unsigned. If arguments from the same half, then function is monotonic. if (left_in_first_half == right_in_first_half) return {true}; + + return {}; } /// Size of type is shrinked. From e1568fd9add79210beb9c3de5f0893cf1c41ead1 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 25 Dec 2018 21:49:09 +0300 Subject: [PATCH 099/230] Update PODArray.h --- dbms/src/Common/PODArray.h | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dbms/src/Common/PODArray.h b/dbms/src/Common/PODArray.h index 03771f4a8ec..0b6a0030b24 100644 --- a/dbms/src/Common/PODArray.h +++ b/dbms/src/Common/PODArray.h @@ -51,15 +51,19 @@ namespace DB */ static constexpr size_t EmptyPODArraySize = 1024; extern const char EmptyPODArray[EmptyPODArraySize]; + template , size_t pad_right_ = 0, size_t pad_left_ = 0> class PODArray : private boost::noncopyable, private TAllocator /// empty base optimization { protected: /// Round padding up to an whole number of elements to simplify arithmetic. static constexpr size_t pad_right = (pad_right_ + sizeof(T) - 1) / sizeof(T) * sizeof(T); + static constexpr size_t pad_left_unaligned = (pad_left_ + sizeof(T) - 1) / sizeof(T) * sizeof(T); - static constexpr size_t pad_left = pad_left_unaligned ? pad_left_unaligned + 15 - (pad_left_unaligned - 1) % 16 : 0; + static constexpr size_t pad_left = pad_left_unaligned ? (pad_left_unaligned + 15) / 16 * 16 : 0; + static constexpr char * null = pad_left ? const_cast(EmptyPODArray) + EmptyPODArraySize : nullptr; + static_assert(pad_left <= EmptyPODArraySize && "Left Padding exceeds EmptyPODArraySize. Element size too large?"); char * c_start = null; /// Does not include pad_left. From d99072ffb08c6c5722d0b90d0cc9b2ad2ecd7866 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 25 Dec 2018 21:51:37 +0300 Subject: [PATCH 100/230] Update PODArray.h --- dbms/src/Common/PODArray.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/PODArray.h b/dbms/src/Common/PODArray.h index 0b6a0030b24..408345b2592 100644 --- a/dbms/src/Common/PODArray.h +++ b/dbms/src/Common/PODArray.h @@ -95,7 +95,7 @@ protected: c_start = c_end = reinterpret_cast(TAllocator::alloc(bytes, std::forward(allocator_params)...)) + pad_left; c_end_of_storage = c_start + bytes - pad_right - pad_left; if (pad_left) - *(t_start() - 1) = {}; + t_start()[-1] = {}; } void dealloc() From 8120d5495ee467d564edfdda13bc0da8a4d19fec Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 25 Dec 2018 21:52:43 +0300 Subject: [PATCH 101/230] Update PODArray.h --- dbms/src/Common/PODArray.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/PODArray.h b/dbms/src/Common/PODArray.h index 408345b2592..fe2084c85c7 100644 --- a/dbms/src/Common/PODArray.h +++ b/dbms/src/Common/PODArray.h @@ -123,7 +123,7 @@ protected: c_end = c_start + end_diff; c_end_of_storage = c_start + bytes - pad_right - pad_left; if (pad_left) - *(t_start() - 1) = {}; + t_start()[-1] = {}; } bool isInitialized() const From 58690beb326cda2ea26e82ff34cd5e19a8af086c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Dec 2018 22:23:16 +0300 Subject: [PATCH 102/230] Updated MaridDB library: fixed error in UBSan [#CLICKHOUSE-2] --- .gitmodules | 2 +- contrib/mariadb-connector-c | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index a7061ececc6..923554a1532 100644 --- a/.gitmodules +++ b/.gitmodules @@ -36,7 +36,7 @@ url = https://github.com/ClickHouse-Extras/llvm [submodule "contrib/mariadb-connector-c"] path = contrib/mariadb-connector-c - url = https://github.com/MariaDB/mariadb-connector-c.git + url = https://github.com/ClickHouse-Extras/mariadb-connector-c.git [submodule "contrib/jemalloc"] path = contrib/jemalloc url = https://github.com/jemalloc/jemalloc.git diff --git a/contrib/mariadb-connector-c b/contrib/mariadb-connector-c index a0fd36cc5a5..d85d0e98999 160000 --- a/contrib/mariadb-connector-c +++ b/contrib/mariadb-connector-c @@ -1 +1 @@ -Subproject commit a0fd36cc5a5313414a5a2ebe9322577a29b4782a +Subproject commit d85d0e98999cd9e28ceb66645999b4a9ce85370e From 8f306e8b4593ee3910319e4814312c6996fd7cef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Dec 2018 22:31:18 +0300 Subject: [PATCH 103/230] Small improvements #3920 --- dbms/src/Common/PODArray.h | 28 +++++++++++++++------------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/dbms/src/Common/PODArray.h b/dbms/src/Common/PODArray.h index fe2084c85c7..280e0f9811b 100644 --- a/dbms/src/Common/PODArray.h +++ b/dbms/src/Common/PODArray.h @@ -20,6 +20,11 @@ namespace DB { +inline constexpr size_t integerRoundUp(size_t value, size_t dividend) +{ + return ((value + dividend - 1) / dividend) * dividend; +} + /** A dynamic array for POD types. * Designed for a small number of large arrays (rather than a lot of small ones). * To be more precise - for use in ColumnVector. @@ -37,6 +42,10 @@ namespace DB * The template parameter `pad_right` - always allocate at the end of the array as many unused bytes. * Can be used to make optimistic reading, writing, copying with unaligned SIMD instructions. * + * The template parameter `pad_left` - always allocate memory before 0th element of the array (rounded up to the whole number of elements) + * and zero initialize -1th element. It allows to use -1th element that will have value 0. + * This gives performance benefits when converting an array of offsets to array of sizes. + * * Some methods using allocator have TAllocatorParams variadic arguments. * These arguments will be passed to corresponding methods of TAllocator. * Example: pointer to Arena, that is used for allocations. @@ -57,14 +66,13 @@ class PODArray : private boost::noncopyable, private TAllocator /// empty bas { protected: /// Round padding up to an whole number of elements to simplify arithmetic. - static constexpr size_t pad_right = (pad_right_ + sizeof(T) - 1) / sizeof(T) * sizeof(T); - - static constexpr size_t pad_left_unaligned = (pad_left_ + sizeof(T) - 1) / sizeof(T) * sizeof(T); - static constexpr size_t pad_left = pad_left_unaligned ? (pad_left_unaligned + 15) / 16 * 16 : 0; - + static constexpr size_t pad_right = integerRoundUp(pad_right_, sizeof(T)); + /// pad_left is also rounded up to 16 bytes to maintain alignment of allocated memory. + static constexpr size_t pad_left = integerRoundUp(integerRoundUp(pad_left_, sizeof(T)), 16); + /// Empty array will point to this static memory as padding. static constexpr char * null = pad_left ? const_cast(EmptyPODArray) + EmptyPODArraySize : nullptr; - static_assert(pad_left <= EmptyPODArraySize && "Left Padding exceeds EmptyPODArraySize. Element size too large?"); + static_assert(pad_left <= EmptyPODArraySize && "Left Padding exceeds EmptyPODArraySize. Is the element size too large?"); char * c_start = null; /// Does not include pad_left. char * c_end = null; @@ -521,13 +529,7 @@ void swap(PODArray & lhs, PODArray> using PaddedPODArray = PODArray; - -inline constexpr size_t integerRound(size_t value, size_t dividend) -{ - return ((value + dividend - 1) / dividend) * dividend; -} - template -using PODArrayWithStackMemory = PODArray, integerRound(stack_size_in_bytes, sizeof(T))>>; +using PODArrayWithStackMemory = PODArray, integerRoundUp(stack_size_in_bytes, sizeof(T))>>; } From ce0cfbc02a055132b7b760112ec0f919b458e5ac Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Dec 2018 23:19:45 +0300 Subject: [PATCH 104/230] Removed optimization that confuses UBSan in gcc [#CLICKHOUSE-2] --- cmake/sanitize.cmake | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 1d8ed9461eb..644492d278b 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -31,6 +31,18 @@ if (SANITIZE) if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libubsan") endif () + if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + # Devirtualization with multiple virtual inheritance confuses UBSan. + # Example: + # + # ZooKeeperImpl.cpp:529:8: runtime error: member call on address 0x7f548d7ea490 which does not point to an object of type 'Request' + # 0x7f548d7ea490: note: object has invalid vptr + # 00 00 00 00 00 00 00 00 00 00 00 00 a8 a4 7e 8d 54 7f 00 00 00 00 00 00 00 00 00 00 00 00 00 00 + # + # http://lists.llvm.org/pipermail/cfe-commits/Week-of-Mon-20161017/174163.html + + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fno-devirtualize") + endif () else () message (FATAL_ERROR "Unknown sanitizer type: ${SANITIZE}") endif () From 06783113a771d925bdac30df8c1174d1d95ab55d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 00:18:40 +0300 Subject: [PATCH 105/230] Fixed UB [#CLICKHOUSE-2] --- dbms/src/Functions/arrayIndex.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Functions/arrayIndex.h b/dbms/src/Functions/arrayIndex.h index fc8a8a08e47..fdb5dcc8109 100644 --- a/dbms/src/Functions/arrayIndex.h +++ b/dbms/src/Functions/arrayIndex.h @@ -946,7 +946,9 @@ public: auto & data = source_block.getByPosition(0); data.column = ColumnArray::create(nested_col, col_array->getOffsetsPtr()); - data.type = static_cast(*block.getByPosition(arguments[0]).type).getNestedType(); + data.type = std::make_shared( + static_cast( + *static_cast(*block.getByPosition(arguments[0]).type).getNestedType()).getNestedType()); auto & null_map = source_block.getByPosition(2); null_map.column = nullable_col.getNullMapColumnPtr(); From 4cb431264a6d42161be93a4f8043b7425998f5a9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 02:06:39 +0300 Subject: [PATCH 106/230] Fixed UB #3926 --- dbms/src/Storages/AlterCommands.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index f067d3811cc..c8d46dd5764 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -44,10 +44,10 @@ struct AlterCommand String after_column; /// For DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN - bool if_exists; + bool if_exists = false; /// For ADD_COLUMN - bool if_not_exists; + bool if_not_exists = false; /// For MODIFY_ORDER_BY ASTPtr order_by; From 4c0b95ac0216ca11bcb2e8b35d0915cc75519135 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 02:11:36 +0300 Subject: [PATCH 107/230] Less dependencies [#CLICKHOUSE-2] --- dbms/src/Interpreters/InterpreterAlterQuery.cpp | 5 +++++ dbms/src/Interpreters/InterpreterAlterQuery.h | 10 ++++------ dbms/src/Storages/IStorage.cpp | 16 ++++++++++++++++ dbms/src/Storages/IStorage.h | 16 +--------------- dbms/src/Storages/MergeTree/MergeTreeData.h | 3 ++- 5 files changed, 28 insertions(+), 22 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index db9ec6a354c..c28c8aef32f 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -2,7 +2,12 @@ #include #include #include +#include #include +#include +#include +#include +#include #include #include diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.h b/dbms/src/Interpreters/InterpreterAlterQuery.h index 3e4453608ca..bd9f3a89d6a 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.h +++ b/dbms/src/Interpreters/InterpreterAlterQuery.h @@ -1,17 +1,15 @@ #pragma once -#include -#include -#include -#include -#include #include -#include namespace DB { +class Context; +class IAST; +using ASTPtr = std::shared_ptr; + /** Allows you add or remove a column in the table. * It also allows you to manipulate the partitions of the MergeTree family tables. */ diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index c0487b5ef02..07442ecbd15 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB @@ -13,4 +14,19 @@ TableStructureReadLock::TableStructureReadLock(StoragePtr storage_, bool lock_st structure_lock = storage->structure_lock->getLock(RWLockImpl::Read); } +void IStorage::alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) +{ + for (const auto & param : params) + { + if (param.is_mutable()) + throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + + auto lock = lockStructureForAlter(); + auto new_columns = getColumns(); + params.apply(new_columns); + context.getDatabase(database_name)->alterTable(context, table_name, new_columns, {}); + setColumns(std::move(new_columns)); +} + } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 9b8d76fb05b..23f56b47a8f 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -234,20 +233,7 @@ public: * This method must fully execute the ALTER query, taking care of the locks itself. * To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata. */ - virtual void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) - { - for (const auto & param : params) - { - if (param.is_mutable()) - throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); - } - - auto lock = lockStructureForAlter(); - auto new_columns = getColumns(); - params.apply(new_columns); - context.getDatabase(database_name)->alterTable(context, table_name, new_columns, {}); - setColumns(std::move(new_columns)); - } + virtual void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context); /** ALTER tables with regard to its partitions. * Should handle locks for each command on its own. diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index b8f01c40077..ed51a8cbd1a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include @@ -25,6 +24,8 @@ namespace DB { +class AlterCommands; + namespace ErrorCodes { extern const int LOGICAL_ERROR; From 3be55436bffde244fdb1d73a2072eb79330eb055 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 02:13:30 +0300 Subject: [PATCH 108/230] Less dependencies [#CLICKHOUSE-2] --- dbms/src/Storages/StorageMergeTree.cpp | 28 ++++++++++++++++---------- 1 file changed, 17 insertions(+), 11 deletions(-) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 6ee1e7ca9c9..911c60081e8 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -1,23 +1,29 @@ -#include +#include + +#include +#include #include #include + +#include +#include + +#include +#include +#include + +#include +#include +#include #include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include -#include + +#include namespace DB From 436b9b6eac048592b93801245198e529b2705ff2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 02:14:39 +0300 Subject: [PATCH 109/230] Less dependencies [#CLICKHOUSE-2] --- dbms/src/Storages/StorageBuffer.cpp | 1 + dbms/src/Storages/StorageDistributed.cpp | 1 + dbms/src/Storages/StorageMerge.cpp | 1 + 3 files changed, 3 insertions(+) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index b7707d37ec2..3749bb92d80 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 9086d1bf321..f681565d40e 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index e7acd420dec..4a6ad3c7c8a 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include From a204ed264fd6b444fdba59fc43aacc9647330fc5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 02:15:28 +0300 Subject: [PATCH 110/230] Less dependencies [#CLICKHOUSE-2] --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index afe8cbc02ab..111ba3e216e 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3,6 +3,7 @@ #include +#include #include #include #include From bbbefe2c038d98421c2b27829fc887263ffebf37 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 02:18:07 +0300 Subject: [PATCH 111/230] Less dependencies [#CLICKHOUSE-2] --- dbms/src/Storages/StorageMergeTree.cpp | 1 + dbms/src/Storages/StorageNull.cpp | 1 + dbms/src/Storages/StorageReplicatedMergeTree.cpp | 1 + 3 files changed, 3 insertions(+) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 911c60081e8..e35253ffcee 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Storages/StorageNull.cpp b/dbms/src/Storages/StorageNull.cpp index 0a1690d5d57..6aab8ae0ada 100644 --- a/dbms/src/Storages/StorageNull.cpp +++ b/dbms/src/Storages/StorageNull.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 111ba3e216e..aa50432ed2e 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include From cfc1e207abfeaa1d1a2cccb4ffee9b6cf31f8175 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 02:27:52 +0300 Subject: [PATCH 112/230] Fixed UB #3926 --- dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h | 4 ++-- dbms/src/Common/Arena.h | 6 ++++++ dbms/src/Interpreters/Join.cpp | 2 +- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h index 094cae05615..aabbf337711 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -171,7 +171,7 @@ struct GroupArrayListNodeBase UInt64 size; readVarUInt(size, buf); - Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + size, alignof(Node))); + Node * node = arena->alloc(); node->size = size; buf.read(node->data(), size); return node; @@ -187,7 +187,7 @@ struct GroupArrayListNodeString : public GroupArrayListNodeBase(column).getDataAt(row_num); - Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + string.size, alignof(Node))); + Node * node = arena->alloc(); node->next = nullptr; node->size = string.size; memcpy(node->data(), string.data, string.size); diff --git a/dbms/src/Common/Arena.h b/dbms/src/Common/Arena.h index a6c98324ea8..20297fd3195 100644 --- a/dbms/src/Common/Arena.h +++ b/dbms/src/Common/Arena.h @@ -149,6 +149,12 @@ public: } while (true); } + template + T * alloc() + { + return reinterpret_cast(alignedAlloc(sizeof(T), alignof(T))); + } + /** Rollback just performed allocation. * Must pass size not more that was just allocated. */ diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 9ab5f26f9e3..626e5a2530f 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -354,7 +354,7 @@ namespace * We will insert each time the element into the second place. * That is, the former second element, if it was, will be the third, and so on. */ - auto elem = reinterpret_cast(pool.alloc(sizeof(typename Map::mapped_type))); + auto elem = pool.alloc(); elem->next = it->second.next; it->second.next = elem; From 07bfa2f4602a90393a490be5215b8840be7d674a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 02:32:41 +0300 Subject: [PATCH 113/230] Added instruction for UBSan usage #3926 --- dbms/tests/instructions/sanitizers.md | 52 ++++++++++++++------------- 1 file changed, 27 insertions(+), 25 deletions(-) diff --git a/dbms/tests/instructions/sanitizers.md b/dbms/tests/instructions/sanitizers.md index ad7ce179927..65e35084a4a 100644 --- a/dbms/tests/instructions/sanitizers.md +++ b/dbms/tests/instructions/sanitizers.md @@ -3,7 +3,7 @@ Note: We use Address Sanitizer to run functional tests for every commit automatically. ``` -mkdir build && cd build +mkdir build_asan && cd build_asan ``` Note: using clang instead of gcc is strongly recommended. @@ -29,11 +29,9 @@ sudo -u clickhouse ./clickhouse-asan server --config /etc/clickhouse-server/conf # How to use Thread Sanitizer ``` -mkdir build && cd build +mkdir build_tsan && cd build_tsan ``` -## Note: All parameters are mandatory. - ``` CC=clang CXX=clang++ cmake -D SANITIZE=thread .. ninja @@ -52,26 +50,30 @@ sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1' ./clickhouse-tsan server --con ``` +# How to use Undefined Behaviour Sanitizer + +``` +mkdir build_ubsan && cd build_ubsan +``` + +``` +cmake -D SANITIZE=undefined .. +ninja +``` + +## Copy binary to your server + +``` +scp ./dbms/programs/clickhouse yourserver:~/clickhouse-ubsan +``` + +## Start ClickHouse and run tests + +``` +sudo -u clickhouse UBSAN_OPTIONS='print_stacktrace=1' ./clickhouse-ubsan server --config /etc/clickhouse-server/config.xml +``` + + # How to use Memory Sanitizer -First, build libc++ with MSan: - -``` -svn co http://llvm.org/svn/llvm-project/llvm/trunk llvm -(cd llvm/projects && svn co http://llvm.org/svn/llvm-project/libcxx/trunk libcxx) -(cd llvm/projects && svn co http://llvm.org/svn/llvm-project/libcxxabi/trunk libcxxabi) - -mkdir libcxx_msan && cd libcxx_msan -cmake ../llvm -DCMAKE_BUILD_TYPE=Release -DLLVM_USE_SANITIZER=Memory -DCMAKE_C_COMPILER=clang -DCMAKE_CXX_COMPILER=clang++ -make cxx -j24 -``` - -Then, build ClickHouse: - -``` -mkdir build && cd build -``` - -``` -CC=clang CXX=clang++ cmake -D SANITIZE=memory -D LIBCXX_PATH=/home/milovidov/libcxx_msan .. -``` +TODO From c858c144efe2de551941c3c2c9ea4734125a4979 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 02:39:40 +0300 Subject: [PATCH 114/230] Fixed UB #3926 --- dbms/src/Common/Arena.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/Arena.h b/dbms/src/Common/Arena.h index 20297fd3195..30a4f4fcb2e 100644 --- a/dbms/src/Common/Arena.h +++ b/dbms/src/Common/Arena.h @@ -36,7 +36,7 @@ private: static constexpr size_t pad_right = 15; /// Contiguous chunk of memory and pointer to free space inside it. Member of single-linked list. - struct Chunk : private Allocator /// empty base optimization + struct alignas(16) Chunk : private Allocator /// empty base optimization { char * begin; char * pos; From 1191f1575dd9dd8b6c6c008e442e00b4448ece89 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 02:44:44 +0300 Subject: [PATCH 115/230] Less dependencies [#CLICKHOUSE-2] --- dbms/src/DataStreams/CubeBlockInputStream.h | 1 - dbms/src/DataStreams/RollupBlockInputStream.h | 1 - dbms/src/Functions/FunctionsFindCluster.h | 10 +--------- 3 files changed, 1 insertion(+), 11 deletions(-) diff --git a/dbms/src/DataStreams/CubeBlockInputStream.h b/dbms/src/DataStreams/CubeBlockInputStream.h index 46f1a5bb432..e7f7bc91592 100644 --- a/dbms/src/DataStreams/CubeBlockInputStream.h +++ b/dbms/src/DataStreams/CubeBlockInputStream.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include diff --git a/dbms/src/DataStreams/RollupBlockInputStream.h b/dbms/src/DataStreams/RollupBlockInputStream.h index c3483db7f82..0e4b9b80900 100644 --- a/dbms/src/DataStreams/RollupBlockInputStream.h +++ b/dbms/src/DataStreams/RollupBlockInputStream.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include diff --git a/dbms/src/Functions/FunctionsFindCluster.h b/dbms/src/Functions/FunctionsFindCluster.h index 5b20767c8aa..cac34dfa88c 100644 --- a/dbms/src/Functions/FunctionsFindCluster.h +++ b/dbms/src/Functions/FunctionsFindCluster.h @@ -1,30 +1,22 @@ #pragma once -#include - -#include #include -#include #include #include #include -#include #include #include #include -#include #include -#include -#include + namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } From 5d1bc8540eeb6b8587d762a09f2d384a6713ea6f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 02:50:12 +0300 Subject: [PATCH 116/230] Less dependencies [#CLICKHOUSE-2] --- dbms/src/Functions/FunctionsFindCluster.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Functions/FunctionsFindCluster.h b/dbms/src/Functions/FunctionsFindCluster.h index cac34dfa88c..5b8db96d1ba 100644 --- a/dbms/src/Functions/FunctionsFindCluster.h +++ b/dbms/src/Functions/FunctionsFindCluster.h @@ -9,6 +9,8 @@ #include #include +#include + #include From 4104a0610ecdfe0b14e3a1db069b2aa56abcafdb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 03:18:11 +0300 Subject: [PATCH 117/230] Added a test #3569 --- dbms/tests/queries/0_stateless/00814_parsing_ub.reference | 4 ++++ dbms/tests/queries/0_stateless/00814_parsing_ub.sql | 4 ++++ 2 files changed, 8 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00814_parsing_ub.reference create mode 100644 dbms/tests/queries/0_stateless/00814_parsing_ub.sql diff --git a/dbms/tests/queries/0_stateless/00814_parsing_ub.reference b/dbms/tests/queries/0_stateless/00814_parsing_ub.reference new file mode 100644 index 00000000000..c60e197fcab --- /dev/null +++ b/dbms/tests/queries/0_stateless/00814_parsing_ub.reference @@ -0,0 +1,4 @@ +-128 +-32768 +-2147483648 +-9223372036854775808 diff --git a/dbms/tests/queries/0_stateless/00814_parsing_ub.sql b/dbms/tests/queries/0_stateless/00814_parsing_ub.sql new file mode 100644 index 00000000000..d58e6319721 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00814_parsing_ub.sql @@ -0,0 +1,4 @@ +SELECT toInt8('-128'); +SELECT toInt16('-32768'); +SELECT toInt32('-2147483648'); +SELECT toInt64('-9223372036854775808'); From e8dc222b610a5d5d5c58bea2949827e7457aa7f3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 03:28:13 +0300 Subject: [PATCH 118/230] Fixed UB #3926 --- dbms/src/Columns/ColumnVector.h | 31 ----------------------------- dbms/src/Interpreters/Aggregator.h | 7 +++---- dbms/src/Interpreters/Join.h | 7 +++---- dbms/src/Interpreters/SetVariants.h | 7 +++---- 4 files changed, 9 insertions(+), 43 deletions(-) diff --git a/dbms/src/Columns/ColumnVector.h b/dbms/src/Columns/ColumnVector.h index 1035c52ad8c..957eabff66b 100644 --- a/dbms/src/Columns/ColumnVector.h +++ b/dbms/src/Columns/ColumnVector.h @@ -86,37 +86,6 @@ template <> struct CompareHelper : public FloatCompareHelper { template <> struct CompareHelper : public FloatCompareHelper {}; -/** To implement `get64` function. - */ -template -inline UInt64 unionCastToUInt64(T x) { return x; } - -template <> inline UInt64 unionCastToUInt64(Float64 x) -{ - union - { - Float64 src; - UInt64 res; - }; - - src = x; - return res; -} - -template <> inline UInt64 unionCastToUInt64(Float32 x) -{ - union - { - Float32 src; - UInt64 res; - }; - - res = 0; - src = x; - return res; -} - - /** A template for columns that use a simple array to store. */ template diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index f3abe782134..1aaa1efba62 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -171,15 +171,14 @@ struct AggregationMethodOneNumber /// To use one `Method` in different threads, use different `State`. struct State { - const FieldType * vec; + const char * vec; /** Called at the start of each block processing. * Sets the variables needed for the other methods called in inner loops. */ void init(ColumnRawPtrs & key_columns) { - /// We may interpret ColumnInt32 as ColumnUInt32. This breaks strict aliasing but compiler doesn't see it. - vec = &reinterpret_cast *>(key_columns[0])->getData()[0]; + vec = key_columns[0]->getRawData().data; } /// Get the key from the key columns for insertion into the hash table. @@ -191,7 +190,7 @@ struct AggregationMethodOneNumber StringRefs & /*keys*/, /// Here references to key data in columns can be written. They can be used in the future. Arena & /*pool*/) const { - return unionCastToUInt64(vec[i]); + return unalignedLoad(vec + i * sizeof(Key)); } }; diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index e7bc3b1dfef..634adfd2972 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -28,15 +28,14 @@ struct JoinKeyGetterOneNumber { using Key = FieldType; - const FieldType * vec; + const char * vec; /** Created before processing of each block. * Initialize some members, used in another methods, called in inner loops. */ JoinKeyGetterOneNumber(const ColumnRawPtrs & key_columns) { - /// We may interpret ColumnInt32 as ColumnUInt32. This breaks strict aliasing but compiler doesn't see it. - vec = &reinterpret_cast *>(key_columns[0])->getData()[0]; + vec = key_columns[0]->getRawData().data; } Key getKey( @@ -45,7 +44,7 @@ struct JoinKeyGetterOneNumber size_t i, /// row number to get key from. const Sizes & /*key_sizes*/) const /// If keys are of fixed size - their sizes. Not used for methods with variable-length keys. { - return unionCastToUInt64(vec[i]); + return unalignedLoad(vec + i * sizeof(Key)); } /// Place additional data into memory pool, if needed, when new key was inserted into hash table. diff --git a/dbms/src/Interpreters/SetVariants.h b/dbms/src/Interpreters/SetVariants.h index 8023282a331..218943c031f 100644 --- a/dbms/src/Interpreters/SetVariants.h +++ b/dbms/src/Interpreters/SetVariants.h @@ -30,15 +30,14 @@ struct SetMethodOneNumber /// To use one `Method` in different threads, use different `State`. struct State { - const FieldType * vec; + const char * vec; /** Called at the start of each block processing. * Sets the variables required for the other methods called in inner loops. */ void init(const ColumnRawPtrs & key_columns) { - /// We may interpret ColumnInt32 as ColumnUInt32. This breaks strict aliasing but compiler doesn't see it. - vec = reinterpret_cast *>(key_columns[0])->getData().data(); + vec = key_columns[0]->getRawData().data; } /// Get key from key columns for insertion into hash table. @@ -48,7 +47,7 @@ struct SetMethodOneNumber size_t i, /// From what row of the block I get the key. const Sizes & /*key_sizes*/) const /// If keys of a fixed length - their lengths. Not used in methods for variable length keys. { - return unionCastToUInt64(vec[i]); + return unalignedLoad(vec + i * sizeof(Key)); } }; From b1b1c676c9284ea3c07464463e3e5c98bd71c3fe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 03:30:11 +0300 Subject: [PATCH 119/230] Fixed error #3926 --- dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h index aabbf337711..094cae05615 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -171,7 +171,7 @@ struct GroupArrayListNodeBase UInt64 size; readVarUInt(size, buf); - Node * node = arena->alloc(); + Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + size, alignof(Node))); node->size = size; buf.read(node->data(), size); return node; @@ -187,7 +187,7 @@ struct GroupArrayListNodeString : public GroupArrayListNodeBase(column).getDataAt(row_num); - Node * node = arena->alloc(); + Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + string.size, alignof(Node))); node->next = nullptr; node->size = string.size; memcpy(node->data(), string.data, string.size); From ee953b4653a09ed76aaba07e63e08a08d6218ca6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 03:31:51 +0300 Subject: [PATCH 120/230] Fixed UB (requires performance testing) #3569 --- dbms/src/IO/ReadHelpers.h | 24 +++++++++--------------- dbms/src/IO/WriteIntText.h | 2 -- libs/libcommon/src/JSON.cpp | 6 ++---- libs/libmysqlxx/src/Value.cpp | 6 ++---- 4 files changed, 13 insertions(+), 25 deletions(-) diff --git a/dbms/src/IO/ReadHelpers.h b/dbms/src/IO/ReadHelpers.h index dcf866848c3..8823fb3611e 100644 --- a/dbms/src/IO/ReadHelpers.h +++ b/dbms/src/IO/ReadHelpers.h @@ -254,7 +254,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf) static constexpr bool throw_exception = std::is_same_v; bool negative = false; - x = 0; + std::make_unsigned_t res = 0; if (buf.eof()) { if constexpr (throw_exception) @@ -290,22 +290,17 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf) case '7': [[fallthrough]]; case '8': [[fallthrough]]; case '9': - x *= 10; - x += *buf.position() - '0'; + res *= 10; + res += *buf.position() - '0'; break; default: - if (negative) - x = -x; + x = negative ? -res : res; return ReturnType(true); } ++buf.position(); } - /// NOTE Signed integer overflow is undefined behaviour. Consider we have '128' that is parsed as Int8 and overflowed. - /// We are happy if it is overflowed to -128 and then 'x = -x' does nothing. But UBSan will warn. - if (negative) - x = -x; - + x = negative ? -res : res; return ReturnType(true); } @@ -331,7 +326,7 @@ template void readIntTextUnsafe(T & x, ReadBuffer & buf) { bool negative = false; - x = 0; + std::make_unsigned_t res = 0; auto on_error = [] { @@ -365,8 +360,8 @@ void readIntTextUnsafe(T & x, ReadBuffer & buf) if ((*buf.position() & 0xF0) == 0x30) /// It makes sense to have this condition inside loop. { - x *= 10; - x += *buf.position() & 0x0F; + res *= 10; + res += *buf.position() & 0x0F; ++buf.position(); } else @@ -374,8 +369,7 @@ void readIntTextUnsafe(T & x, ReadBuffer & buf) } /// See note about undefined behaviour above. - if (std::is_signed_v && negative) - x = -x; + x = std::is_signed_v && negative ? -res : res; } template diff --git a/dbms/src/IO/WriteIntText.h b/dbms/src/IO/WriteIntText.h index 79e5a2f2644..80449091f50 100644 --- a/dbms/src/IO/WriteIntText.h +++ b/dbms/src/IO/WriteIntText.h @@ -185,7 +185,6 @@ namespace detail writeUIntText(static_cast>(x), buf); } -#if 1 inline void writeSIntText(__int128 x, WriteBuffer & buf) { static const __int128 min_int128 = __int128(0x8000000000000000ll) << 64; @@ -204,7 +203,6 @@ namespace detail writeUIntText(static_cast(x), buf); } -#endif } diff --git a/libs/libcommon/src/JSON.cpp b/libs/libcommon/src/JSON.cpp index fcc23bd9b7b..1d2c94e37ae 100644 --- a/libs/libcommon/src/JSON.cpp +++ b/libs/libcommon/src/JSON.cpp @@ -56,7 +56,7 @@ static UInt64 readUIntText(const char * buf, const char * end) static Int64 readIntText(const char * buf, const char * end) { bool negative = false; - Int64 x = 0; + UInt64 x = 0; if (buf == end) throw JSONException("JSON: cannot parse signed integer: unexpected end of data."); @@ -90,10 +90,8 @@ static Int64 readIntText(const char * buf, const char * end) } ++buf; } - if (negative) - x = -x; - return x; + return negative ? -x : x; } diff --git a/libs/libmysqlxx/src/Value.cpp b/libs/libmysqlxx/src/Value.cpp index 8e448604e5a..4fed33c5da9 100644 --- a/libs/libmysqlxx/src/Value.cpp +++ b/libs/libmysqlxx/src/Value.cpp @@ -44,7 +44,7 @@ UInt64 Value::readUIntText(const char * buf, size_t length) const Int64 Value::readIntText(const char * buf, size_t length) const { bool negative = false; - Int64 x = 0; + UInt64 x = 0; const char * end = buf + length; while (buf != end) @@ -74,10 +74,8 @@ Int64 Value::readIntText(const char * buf, size_t length) const } ++buf; } - if (negative) - x = -x; - return x; + return negative ? -x : x; } From 37bbc0ecd59f4338630c53ee63ce3c85b14b055f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 04:08:14 +0300 Subject: [PATCH 121/230] Fixed UB #3926 --- dbms/src/Interpreters/Aggregator.h | 2 +- dbms/src/Interpreters/Join.h | 2 +- dbms/src/Interpreters/SetVariants.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 1aaa1efba62..3fe1ff50db6 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -190,7 +190,7 @@ struct AggregationMethodOneNumber StringRefs & /*keys*/, /// Here references to key data in columns can be written. They can be used in the future. Arena & /*pool*/) const { - return unalignedLoad(vec + i * sizeof(Key)); + return unalignedLoad(vec + i * sizeof(FieldType)); } }; diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 634adfd2972..ff9e7ae1243 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -44,7 +44,7 @@ struct JoinKeyGetterOneNumber size_t i, /// row number to get key from. const Sizes & /*key_sizes*/) const /// If keys are of fixed size - their sizes. Not used for methods with variable-length keys. { - return unalignedLoad(vec + i * sizeof(Key)); + return unalignedLoad(vec + i * sizeof(FieldType)); } /// Place additional data into memory pool, if needed, when new key was inserted into hash table. diff --git a/dbms/src/Interpreters/SetVariants.h b/dbms/src/Interpreters/SetVariants.h index 218943c031f..731a7c41316 100644 --- a/dbms/src/Interpreters/SetVariants.h +++ b/dbms/src/Interpreters/SetVariants.h @@ -47,7 +47,7 @@ struct SetMethodOneNumber size_t i, /// From what row of the block I get the key. const Sizes & /*key_sizes*/) const /// If keys of a fixed length - their lengths. Not used in methods for variable length keys. { - return unalignedLoad(vec + i * sizeof(Key)); + return unalignedLoad(vec + i * sizeof(FieldType)); } }; From 985aa2c0347e34c4315817ec6ac09fdf98a4eab9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 04:12:13 +0300 Subject: [PATCH 122/230] Fixed build #3926 --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index b65d23f47e1..1a764f02986 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -174,7 +174,9 @@ MergeTreeData::MergeTreeData( else if (Poco::File(version_file_path).exists()) { ReadBufferFromFile buf(version_file_path); - readIntText(format_version, buf); + UInt32 read_format_version; + readIntText(read_format_version, buf); + format_version = read_format_version; if (!buf.eof()) throw Exception("Bad version file: " + version_file_path, ErrorCodes::CORRUPTED_DATA); } From 897199c22026a13be078831ed1a5acc4bf0eaf6e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 04:29:48 +0300 Subject: [PATCH 123/230] Fixed UB #3926 --- dbms/src/Common/Volnitsky.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/Volnitsky.h b/dbms/src/Common/Volnitsky.h index 8969b611f22..f1a8325d869 100644 --- a/dbms/src/Common/Volnitsky.h +++ b/dbms/src/Common/Volnitsky.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -121,9 +122,9 @@ protected: CRTP & self() { return static_cast(*this); } const CRTP & self() const { return const_cast(this)->self(); } - static const Ngram & toNGram(const UInt8 * const pos) + static Ngram toNGram(const UInt8 * const pos) { - return *reinterpret_cast(pos); + return unalignedLoad(pos); } void putNGramBase(const Ngram ngram, const int offset) From f2d789d384e205dd24da433f811d052fdea7aa97 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 05:06:54 +0300 Subject: [PATCH 124/230] Fixed UB #3926 --- dbms/src/Columns/ColumnDecimal.h | 4 ++-- dbms/src/Columns/ColumnFixedString.h | 4 ++-- dbms/src/Columns/ColumnVector.h | 4 ++-- dbms/src/Columns/IColumn.h | 13 +++++++++++++ dbms/src/Interpreters/AggregationCommon.h | 10 +++++----- 5 files changed, 24 insertions(+), 11 deletions(-) diff --git a/dbms/src/Columns/ColumnDecimal.h b/dbms/src/Columns/ColumnDecimal.h index f6dee8877bc..98d706ff546 100644 --- a/dbms/src/Columns/ColumnDecimal.h +++ b/dbms/src/Columns/ColumnDecimal.h @@ -53,13 +53,13 @@ private: /// A ColumnVector for Decimals template -class ColumnDecimal final : public COWPtrHelper> +class ColumnDecimal final : public COWPtrHelper> { static_assert(IsDecimalNumber); private: using Self = ColumnDecimal; - friend class COWPtrHelper; + friend class COWPtrHelper; public: using Container = DecimalPaddedPODArray; diff --git a/dbms/src/Columns/ColumnFixedString.h b/dbms/src/Columns/ColumnFixedString.h index fcfed7481e8..d03f0ea1804 100644 --- a/dbms/src/Columns/ColumnFixedString.h +++ b/dbms/src/Columns/ColumnFixedString.h @@ -12,10 +12,10 @@ namespace DB /** A column of values of "fixed-length string" type. * If you insert a smaller string, it will be padded with zero bytes. */ -class ColumnFixedString final : public COWPtrHelper +class ColumnFixedString final : public COWPtrHelper { public: - friend class COWPtrHelper; + friend class COWPtrHelper; using Chars = PaddedPODArray; diff --git a/dbms/src/Columns/ColumnVector.h b/dbms/src/Columns/ColumnVector.h index 957eabff66b..7693329567c 100644 --- a/dbms/src/Columns/ColumnVector.h +++ b/dbms/src/Columns/ColumnVector.h @@ -89,13 +89,13 @@ template <> struct CompareHelper : public FloatCompareHelper { /** A template for columns that use a simple array to store. */ template -class ColumnVector final : public COWPtrHelper> +class ColumnVector final : public COWPtrHelper> { static_assert(!IsDecimalNumber); private: using Self = ColumnVector; - friend class COWPtrHelper; + friend class COWPtrHelper; struct less; struct greater; diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 38df6ab3c38..55ba30b6fb3 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -389,4 +389,17 @@ struct IsMutableColumns template <> struct IsMutableColumns<> { static const bool value = true; }; + +/// Allows to access internal array of ColumnVector or ColumnFixedString without cast to concrete type. +/// Inherit ColumnVector and ColumnFixedString from this class instead of IColumn. +class ColumnVectorHelper : public IColumn +{ +public: + const char * getRawDataBegin() const + { + /// Assumes data layout of ColumnVector, ColumnFixedString and PODArray. + return reinterpret_cast(this); + } +}; + } diff --git a/dbms/src/Interpreters/AggregationCommon.h b/dbms/src/Interpreters/AggregationCommon.h index c101d5b8e27..a71c0b93738 100644 --- a/dbms/src/Interpreters/AggregationCommon.h +++ b/dbms/src/Interpreters/AggregationCommon.h @@ -102,23 +102,23 @@ static inline T ALWAYS_INLINE packFixed( switch (key_sizes[j]) { case 1: - memcpy(bytes + offset, &static_cast(column)->getData()[index], 1); + memcpy(bytes + offset, static_cast(column)->getRawDataBegin() + index, 1); offset += 1; break; case 2: - memcpy(bytes + offset, &static_cast(column)->getData()[index], 2); + memcpy(bytes + offset, static_cast(column)->getRawDataBegin() + index * 2, 2); offset += 2; break; case 4: - memcpy(bytes + offset, &static_cast(column)->getData()[index], 4); + memcpy(bytes + offset, static_cast(column)->getRawDataBegin() + index * 4, 4); offset += 4; break; case 8: - memcpy(bytes + offset, &static_cast(column)->getData()[index], 8); + memcpy(bytes + offset, static_cast(column)->getRawDataBegin() + index * 8, 8); offset += 8; break; default: - memcpy(bytes + offset, &static_cast(column)->getChars()[index * key_sizes[j]], key_sizes[j]); + memcpy(bytes + offset, static_cast(column)->getRawDataBegin() + index * key_sizes[j], key_sizes[j]); offset += key_sizes[j]; } } From a2fae24e0798f3f2c4ba4e6d642d39af5642c14d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 05:09:59 +0300 Subject: [PATCH 125/230] Fixed UB #3926 --- dbms/src/Columns/IColumn.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 55ba30b6fb3..527c14a94ae 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -398,7 +398,7 @@ public: const char * getRawDataBegin() const { /// Assumes data layout of ColumnVector, ColumnFixedString and PODArray. - return reinterpret_cast(this); + return reinterpret_cast(this) + sizeof(*this); } }; From b7f085c52555e6cda5915a9820158d29ea8e5957 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 05:47:16 +0300 Subject: [PATCH 126/230] Fixed UB #3926 --- dbms/src/Columns/IColumn.h | 8 +- dbms/src/Common/PODArray.h | 342 ++++++++++++++++------------- dbms/src/Interpreters/Aggregator.h | 2 +- 3 files changed, 192 insertions(+), 160 deletions(-) diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 527c14a94ae..6cb91e12498 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -392,14 +392,20 @@ struct IsMutableColumns<> { static const bool value = true; }; /// Allows to access internal array of ColumnVector or ColumnFixedString without cast to concrete type. /// Inherit ColumnVector and ColumnFixedString from this class instead of IColumn. +/// Assumes data layout of ColumnVector, ColumnFixedString and PODArray. class ColumnVectorHelper : public IColumn { public: const char * getRawDataBegin() const { - /// Assumes data layout of ColumnVector, ColumnFixedString and PODArray. return reinterpret_cast(this) + sizeof(*this); } + + template + void insertRawData(const char * ptr) + { + return reinterpret_cast, 15, 16> *>(reinterpret_cast(this) + sizeof(*this))->push_back_raw(ptr); + } }; } diff --git a/dbms/src/Common/PODArray.h b/dbms/src/Common/PODArray.h index 280e0f9811b..f8cbf3392a7 100644 --- a/dbms/src/Common/PODArray.h +++ b/dbms/src/Common/PODArray.h @@ -61,14 +61,17 @@ inline constexpr size_t integerRoundUp(size_t value, size_t dividend) static constexpr size_t EmptyPODArraySize = 1024; extern const char EmptyPODArray[EmptyPODArraySize]; -template , size_t pad_right_ = 0, size_t pad_left_ = 0> -class PODArray : private boost::noncopyable, private TAllocator /// empty base optimization +/** Base class that depend only on size of element, not on element itself. + * You can static_cast to this class if you want to insert some data regardless to the actual type T. + */ +template +class PODArrayBase : private boost::noncopyable, private TAllocator /// empty base optimization { protected: /// Round padding up to an whole number of elements to simplify arithmetic. - static constexpr size_t pad_right = integerRoundUp(pad_right_, sizeof(T)); + static constexpr size_t pad_right = integerRoundUp(pad_right_, ELEMENT_SIZE); /// pad_left is also rounded up to 16 bytes to maintain alignment of allocated memory. - static constexpr size_t pad_left = integerRoundUp(integerRoundUp(pad_left_, sizeof(T)), 16); + static constexpr size_t pad_left = integerRoundUp(integerRoundUp(pad_left_, ELEMENT_SIZE), 16); /// Empty array will point to this static memory as padding. static constexpr char * null = pad_left ? const_cast(EmptyPODArray) + EmptyPODArraySize : nullptr; @@ -78,16 +81,8 @@ protected: char * c_end = null; char * c_end_of_storage = null; /// Does not include pad_right. - T * t_start() { return reinterpret_cast(c_start); } - T * t_end() { return reinterpret_cast(c_end); } - T * t_end_of_storage() { return reinterpret_cast(c_end_of_storage); } - - const T * t_start() const { return reinterpret_cast(c_start); } - const T * t_end() const { return reinterpret_cast(c_end); } - const T * t_end_of_storage() const { return reinterpret_cast(c_end_of_storage); } - /// The amount of memory occupied by the num_elements of the elements. - static size_t byte_size(size_t num_elements) { return num_elements * sizeof(T); } + static size_t byte_size(size_t num_elements) { return num_elements * ELEMENT_SIZE; } /// Minimum amount of memory to allocate for num_elements, including padding. static size_t minimum_memory_for_elements(size_t num_elements) { return byte_size(num_elements) + pad_right + pad_left; } @@ -102,8 +97,9 @@ protected: { c_start = c_end = reinterpret_cast(TAllocator::alloc(bytes, std::forward(allocator_params)...)) + pad_left; c_end_of_storage = c_start + bytes - pad_right - pad_left; + if (pad_left) - t_start()[-1] = {}; + memset(c_start - ELEMENT_SIZE, 0, ELEMENT_SIZE); } void dealloc() @@ -126,12 +122,14 @@ protected: ptrdiff_t end_diff = c_end - c_start; c_start = reinterpret_cast( - TAllocator::realloc(c_start - pad_left, allocated_bytes(), bytes, std::forward(allocator_params)...)) + TAllocator::realloc(c_start - pad_left, allocated_bytes(), bytes, std::forward(allocator_params)...)) + pad_left; + c_end = c_start + end_diff; c_end_of_storage = c_start + bytes - pad_right - pad_left; + if (pad_left) - t_start()[-1] = {}; + memset(c_start - ELEMENT_SIZE, 0, ELEMENT_SIZE); } bool isInitialized() const @@ -150,9 +148,9 @@ protected: { if (size() == 0) { - // The allocated memory should be multiplication of sizeof(T) to hold the element, otherwise, + // The allocated memory should be multiplication of ELEMENT_SIZE to hold the element, otherwise, // memory issue such as corruption could appear in edge case. - realloc(std::max(((INITIAL_SIZE - 1) / sizeof(T) + 1) * sizeof(T), minimum_memory_for_elements(1)), + realloc(std::max(((INITIAL_SIZE - 1) / ELEMENT_SIZE + 1) * ELEMENT_SIZE, minimum_memory_for_elements(1)), std::forward(allocator_params)...); } else @@ -160,83 +158,13 @@ protected: } public: - using value_type = T; + bool empty() const { return c_end == c_start; } + size_t size() const { return (c_end - c_start) / ELEMENT_SIZE; } + size_t capacity() const { return (c_end_of_storage - c_start) / ELEMENT_SIZE; } size_t allocated_bytes() const { return c_end_of_storage - c_start + pad_right + pad_left; } - /// You can not just use `typedef`, because there is ambiguity for the constructors and `assign` functions. - struct iterator : public boost::iterator_adaptor - { - iterator() {} - iterator(T * ptr_) : iterator::iterator_adaptor_(ptr_) {} - }; - - struct const_iterator : public boost::iterator_adaptor - { - const_iterator() {} - const_iterator(const T * ptr_) : const_iterator::iterator_adaptor_(ptr_) {} - }; - - - PODArray() {} - - PODArray(size_t n) - { - alloc_for_num_elements(n); - c_end += byte_size(n); - } - - PODArray(size_t n, const T & x) - { - alloc_for_num_elements(n); - assign(n, x); - } - - PODArray(const_iterator from_begin, const_iterator from_end) - { - alloc_for_num_elements(from_end - from_begin); - insert(from_begin, from_end); - } - - PODArray(std::initializer_list il) : PODArray(std::begin(il), std::end(il)) {} - - ~PODArray() - { - dealloc(); - } - - PODArray(PODArray && other) - { - this->swap(other); - } - - PODArray & operator=(PODArray && other) - { - this->swap(other); - return *this; - } - - T * data() { return t_start(); } - const T * data() const { return t_start(); } - - size_t size() const { return t_end() - t_start(); } - bool empty() const { return t_end() == t_start(); } - size_t capacity() const { return t_end_of_storage() - t_start(); } - - T & operator[] (size_t n) { return t_start()[n]; } - const T & operator[] (size_t n) const { return t_start()[n]; } - - T & front() { return t_start()[0]; } - T & back() { return t_end()[-1]; } - const T & front() const { return t_start()[0]; } - const T & back() const { return t_end()[-1]; } - - iterator begin() { return t_start(); } - iterator end() { return t_end(); } - const_iterator begin() const { return t_start(); } - const_iterator end() const { return t_end(); } - const_iterator cbegin() const { return t_start(); } - const_iterator cend() const { return t_end(); } + void clear() { c_end = c_start; } template void reserve(size_t n, TAllocatorParams &&... allocator_params) @@ -257,42 +185,140 @@ public: c_end = c_start + byte_size(n); } + const char * raw_data() const + { + return c_start; + } + + template + void push_back_raw(const char * ptr, TAllocatorParams &&... allocator_params) + { + if (unlikely(c_end == c_end_of_storage)) + reserveForNextSize(std::forward(allocator_params)...); + + memcpy(c_end, ptr, ELEMENT_SIZE); + c_end += byte_size(1); + } + + ~PODArrayBase() + { + dealloc(); + } +}; + +template , size_t pad_right_ = 0, size_t pad_left_ = 0> +class PODArray : public PODArrayBase +{ +protected: + using Base = PODArrayBase; + + T * t_start() { return reinterpret_cast(this->c_start); } + T * t_end() { return reinterpret_cast(this->c_end); } + T * t_end_of_storage() { return reinterpret_cast(this->c_end_of_storage); } + + const T * t_start() const { return reinterpret_cast(this->c_start); } + const T * t_end() const { return reinterpret_cast(this->c_end); } + const T * t_end_of_storage() const { return reinterpret_cast(this->c_end_of_storage); } + +public: + using value_type = T; + + /// You can not just use `typedef`, because there is ambiguity for the constructors and `assign` functions. + struct iterator : public boost::iterator_adaptor + { + iterator() {} + iterator(T * ptr_) : iterator::iterator_adaptor_(ptr_) {} + }; + + struct const_iterator : public boost::iterator_adaptor + { + const_iterator() {} + const_iterator(const T * ptr_) : const_iterator::iterator_adaptor_(ptr_) {} + }; + + + PODArray() {} + + PODArray(size_t n) + { + this->alloc_for_num_elements(n); + this->c_end += this->byte_size(n); + } + + PODArray(size_t n, const T & x) + { + this->alloc_for_num_elements(n); + assign(n, x); + } + + PODArray(const_iterator from_begin, const_iterator from_end) + { + this->alloc_for_num_elements(from_end - from_begin); + insert(from_begin, from_end); + } + + PODArray(std::initializer_list il) : PODArray(std::begin(il), std::end(il)) {} + + PODArray(PODArray && other) + { + this->swap(other); + } + + PODArray & operator=(PODArray && other) + { + this->swap(other); + return *this; + } + + T * data() { return t_start(); } + const T * data() const { return t_start(); } + + T & operator[] (size_t n) { return t_start()[n]; } + const T & operator[] (size_t n) const { return t_start()[n]; } + + T & front() { return t_start()[0]; } + T & back() { return t_end()[-1]; } + const T & front() const { return t_start()[0]; } + const T & back() const { return t_end()[-1]; } + + iterator begin() { return t_start(); } + iterator end() { return t_end(); } + const_iterator begin() const { return t_start(); } + const_iterator end() const { return t_end(); } + const_iterator cbegin() const { return t_start(); } + const_iterator cend() const { return t_end(); } + /// Same as resize, but zeroes new elements. void resize_fill(size_t n) { - size_t old_size = size(); + size_t old_size = this->size(); if (n > old_size) { - reserve(n); - memset(c_end, 0, byte_size(n - old_size)); + this->reserve(n); + memset(this->c_end, 0, this->byte_size(n - old_size)); } - c_end = c_start + byte_size(n); + this->c_end = this->c_start + this->byte_size(n); } void resize_fill(size_t n, const T & value) { - size_t old_size = size(); + size_t old_size = this->size(); if (n > old_size) { - reserve(n); + this->reserve(n); std::fill(t_end(), t_end() + n - old_size, value); } - c_end = c_start + byte_size(n); - } - - void clear() - { - c_end = c_start; + this->c_end = this->c_start + this->byte_size(n); } template void push_back(const T & x, TAllocatorParams &&... allocator_params) { - if (unlikely(c_end == c_end_of_storage)) - reserveForNextSize(std::forward(allocator_params)...); + if (unlikely(this->c_end == this->c_end_of_storage)) + this->reserveForNextSize(std::forward(allocator_params)...); *t_end() = x; - c_end += byte_size(1); + this->c_end += this->byte_size(1); } /** This method doesn't allow to pass parameters for Allocator, @@ -301,25 +327,25 @@ public: template void emplace_back(Args &&... args) { - if (unlikely(c_end == c_end_of_storage)) - reserveForNextSize(); + if (unlikely(this->c_end == this->c_end_of_storage)) + this->reserveForNextSize(); new (t_end()) T(std::forward(args)...); - c_end += byte_size(1); + this->c_end += this->byte_size(1); } void pop_back() { - c_end -= byte_size(1); + this->c_end -= this->byte_size(1); } /// Do not insert into the array a piece of itself. Because with the resize, the iterators on themselves can be invalidated. template void insertPrepare(It1 from_begin, It2 from_end, TAllocatorParams &&... allocator_params) { - size_t required_capacity = size() + (from_end - from_begin); - if (required_capacity > capacity()) - reserve(roundUpToPowerOfTwoOrZero(required_capacity), std::forward(allocator_params)...); + size_t required_capacity = this->size() + (from_end - from_begin); + if (required_capacity > this->capacity()) + this->reserve(roundUpToPowerOfTwoOrZero(required_capacity), std::forward(allocator_params)...); } /// Do not insert into the array a piece of itself. Because with the resize, the iterators on themselves can be invalidated. @@ -336,9 +362,9 @@ public: { static_assert(pad_right_ >= 15); insertPrepare(from_begin, from_end, std::forward(allocator_params)...); - size_t bytes_to_copy = byte_size(from_end - from_begin); - memcpySmallAllowReadWriteOverflow15(c_end, reinterpret_cast(&*from_begin), bytes_to_copy); - c_end += bytes_to_copy; + size_t bytes_to_copy = this->byte_size(from_end - from_begin); + memcpySmallAllowReadWriteOverflow15(this->c_end, reinterpret_cast(&*from_begin), bytes_to_copy); + this->c_end += bytes_to_copy; } template @@ -346,22 +372,22 @@ public: { insertPrepare(from_begin, from_end); - size_t bytes_to_copy = byte_size(from_end - from_begin); + size_t bytes_to_copy = this->byte_size(from_end - from_begin); size_t bytes_to_move = (end() - it) * sizeof(T); if (unlikely(bytes_to_move)) - memcpy(c_end + bytes_to_copy - bytes_to_move, c_end - bytes_to_move, bytes_to_move); + memcpy(this->c_end + bytes_to_copy - bytes_to_move, this->c_end - bytes_to_move, bytes_to_move); - memcpy(c_end - bytes_to_move, reinterpret_cast(&*from_begin), bytes_to_copy); - c_end += bytes_to_copy; + memcpy(this->c_end - bytes_to_move, reinterpret_cast(&*from_begin), bytes_to_copy); + this->c_end += bytes_to_copy; } template void insert_assume_reserved(It1 from_begin, It2 from_end) { - size_t bytes_to_copy = byte_size(from_end - from_begin); - memcpy(c_end, reinterpret_cast(&*from_begin), bytes_to_copy); - c_end += bytes_to_copy; + size_t bytes_to_copy = this->byte_size(from_end - from_begin); + memcpy(this->c_end, reinterpret_cast(&*from_begin), bytes_to_copy); + this->c_end += bytes_to_copy; } void swap(PODArray & rhs) @@ -369,7 +395,7 @@ public: /// Swap two PODArray objects, arr1 and arr2, that satisfy the following conditions: /// - The elements of arr1 are stored on stack. /// - The elements of arr2 are stored on heap. - auto swap_stack_heap = [](PODArray & arr1, PODArray & arr2) + auto swap_stack_heap = [this](PODArray & arr1, PODArray & arr2) { size_t stack_size = arr1.size(); size_t stack_allocated = arr1.allocated_bytes(); @@ -383,27 +409,27 @@ public: /// arr1 takes ownership of the heap memory of arr2. arr1.c_start = arr2.c_start; arr1.c_end_of_storage = arr1.c_start + heap_allocated - arr1.pad_right; - arr1.c_end = arr1.c_start + byte_size(heap_size); + arr1.c_end = arr1.c_start + this->byte_size(heap_size); /// Allocate stack space for arr2. arr2.alloc(stack_allocated); /// Copy the stack content. - memcpy(arr2.c_start, stack_c_start, byte_size(stack_size)); - arr2.c_end = arr2.c_start + byte_size(stack_size); + memcpy(arr2.c_start, stack_c_start, this->byte_size(stack_size)); + arr2.c_end = arr2.c_start + this->byte_size(stack_size); }; - auto do_move = [](PODArray & src, PODArray & dest) + auto do_move = [this](PODArray & src, PODArray & dest) { if (src.isAllocatedFromStack()) { dest.dealloc(); dest.alloc(src.allocated_bytes()); - memcpy(dest.c_start, src.c_start, byte_size(src.size())); + memcpy(dest.c_start, src.c_start, this->byte_size(src.size())); dest.c_end = dest.c_start + (src.c_end - src.c_start); - src.c_start = null; - src.c_end = null; - src.c_end_of_storage = null; + src.c_start = Base::null; + src.c_end = Base::null; + src.c_end_of_storage = Base::null; } else { @@ -413,28 +439,28 @@ public: } }; - if (!isInitialized() && !rhs.isInitialized()) + if (!this->isInitialized() && !rhs.isInitialized()) return; - else if (!isInitialized() && rhs.isInitialized()) + else if (!this->isInitialized() && rhs.isInitialized()) { do_move(rhs, *this); return; } - else if (isInitialized() && !rhs.isInitialized()) + else if (this->isInitialized() && !rhs.isInitialized()) { do_move(*this, rhs); return; } - if (isAllocatedFromStack() && rhs.isAllocatedFromStack()) + if (this->isAllocatedFromStack() && rhs.isAllocatedFromStack()) { - size_t min_size = std::min(size(), rhs.size()); - size_t max_size = std::max(size(), rhs.size()); + size_t min_size = std::min(this->size(), rhs.size()); + size_t max_size = std::max(this->size(), rhs.size()); for (size_t i = 0; i < min_size; ++i) std::swap(this->operator[](i), rhs[i]); - if (size() == max_size) + if (this->size() == max_size) { for (size_t i = min_size; i < max_size; ++i) rhs[i] = this->operator[](i); @@ -445,33 +471,33 @@ public: this->operator[](i) = rhs[i]; } - size_t lhs_size = size(); - size_t lhs_allocated = allocated_bytes(); + size_t lhs_size = this->size(); + size_t lhs_allocated = this->allocated_bytes(); size_t rhs_size = rhs.size(); size_t rhs_allocated = rhs.allocated_bytes(); - c_end_of_storage = c_start + rhs_allocated - pad_right; - rhs.c_end_of_storage = rhs.c_start + lhs_allocated - pad_right; + this->c_end_of_storage = this->c_start + rhs_allocated - Base::pad_right; + rhs.c_end_of_storage = rhs.c_start + lhs_allocated - Base::pad_right; - c_end = c_start + byte_size(rhs_size); - rhs.c_end = rhs.c_start + byte_size(lhs_size); + this->c_end = this->c_start + this->byte_size(rhs_size); + rhs.c_end = rhs.c_start + this->byte_size(lhs_size); } - else if (isAllocatedFromStack() && !rhs.isAllocatedFromStack()) + else if (this->isAllocatedFromStack() && !rhs.isAllocatedFromStack()) swap_stack_heap(*this, rhs); - else if (!isAllocatedFromStack() && rhs.isAllocatedFromStack()) + else if (!this->isAllocatedFromStack() && rhs.isAllocatedFromStack()) swap_stack_heap(rhs, *this); else { - std::swap(c_start, rhs.c_start); - std::swap(c_end, rhs.c_end); - std::swap(c_end_of_storage, rhs.c_end_of_storage); + std::swap(this->c_start, rhs.c_start); + std::swap(this->c_end, rhs.c_end); + std::swap(this->c_end_of_storage, rhs.c_end_of_storage); } } void assign(size_t n, const T & x) { - resize(n); + this->resize(n); std::fill(begin(), end(), x); } @@ -479,12 +505,12 @@ public: void assign(It1 from_begin, It2 from_end) { size_t required_capacity = from_end - from_begin; - if (required_capacity > capacity()) - reserve(roundUpToPowerOfTwoOrZero(required_capacity)); + if (required_capacity > this->capacity()) + this->reserve(roundUpToPowerOfTwoOrZero(required_capacity)); - size_t bytes_to_copy = byte_size(required_capacity); - memcpy(c_start, reinterpret_cast(&*from_begin), bytes_to_copy); - c_end = c_start + bytes_to_copy; + size_t bytes_to_copy = this->byte_size(required_capacity); + memcpy(this->c_start, reinterpret_cast(&*from_begin), bytes_to_copy); + this->c_end = this->c_start + bytes_to_copy; } void assign(const PODArray & from) @@ -495,7 +521,7 @@ public: bool operator== (const PODArray & other) const { - if (size() != other.size()) + if (this->size() != other.size()) return false; const_iterator this_it = begin(); diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 3fe1ff50db6..319e8783c9e 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -218,7 +218,7 @@ struct AggregationMethodOneNumber */ static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes & /*key_sizes*/) { - static_cast *>(key_columns[0].get())->insertData(reinterpret_cast(&value.first), sizeof(value.first)); + static_cast(key_columns[0].get())->insertRawData(reinterpret_cast(&value.first)); } /// Get StringRef from value which can be inserted into column. From 56bd082105601f51a8fd89e22348759ce7826977 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 06:15:34 +0300 Subject: [PATCH 127/230] Fixed UB #3926 --- dbms/src/Columns/ColumnArray.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index 82830281373..a45004058e2 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -134,13 +134,13 @@ StringRef ColumnArray::getDataAt(size_t n) const * since it contains only the data laid in succession, but not the offsets. */ - size_t array_size = sizeAt(n); - if (array_size == 0) - return StringRef(); - size_t offset_of_first_elem = offsetAt(n); StringRef first = getData().getDataAtWithTerminatingZero(offset_of_first_elem); + size_t array_size = sizeAt(n); + if (array_size == 0) + return StringRef(first.data, 0); + size_t offset_of_last_elem = getOffsets()[n] - 1; StringRef last = getData().getDataAtWithTerminatingZero(offset_of_last_elem); From b5013194ec8e5735462687a94885a9027a2d8020 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 06:21:39 +0300 Subject: [PATCH 128/230] Fixed UB #3569 --- dbms/src/Parsers/ExpressionElementParsers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index d01dfd1b7a8..9df67f9b131 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -949,7 +949,7 @@ bool ParserNumber::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (pos_integer == pos_double && errno != ERANGE && (!negative || uint_value <= (1ULL << 63))) { if (negative) - res = -static_cast(uint_value); + res = static_cast(-uint_value); else res = uint_value; } From 665fc23f574931efe640a37c333aac1c282e574e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 06:27:26 +0300 Subject: [PATCH 129/230] Fixed UB #3569 --- dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h index 9a96425a240..9dc3882cca7 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h @@ -195,7 +195,7 @@ public: for (auto & rhs_elem : rhs_set) { cur_set.emplace(rhs_elem, it, inserted); - if (inserted) + if (inserted && it->size) it->data = arena->insert(it->data, it->size); } } From 0ba7c1dd3a9292cd6ba4b3ee0ef8aff28932e612 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 06:32:18 +0300 Subject: [PATCH 130/230] Fixed UB #3926 --- dbms/src/AggregateFunctions/Helpers.h | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/dbms/src/AggregateFunctions/Helpers.h b/dbms/src/AggregateFunctions/Helpers.h index e98da75cd53..99ea34ad2f9 100644 --- a/dbms/src/AggregateFunctions/Helpers.h +++ b/dbms/src/AggregateFunctions/Helpers.h @@ -28,8 +28,8 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate(std::forward(args)...); FOR_NUMERIC_TYPES(DISPATCH) #undef DISPATCH - if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate(std::forward(args)...); - if (which.idx == TypeIndex::Enum16) return new AggregateFunctionTemplate(std::forward(args)...); + if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate(std::forward(args)...); + if (which.idx == TypeIndex::Enum16) return new AggregateFunctionTemplate(std::forward(args)...); return nullptr; } @@ -41,8 +41,8 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate(std::forward(args)...); FOR_NUMERIC_TYPES(DISPATCH) #undef DISPATCH - if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate(std::forward(args)...); - if (which.idx == TypeIndex::Enum16) return new AggregateFunctionTemplate(std::forward(args)...); + if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate(std::forward(args)...); + if (which.idx == TypeIndex::Enum16) return new AggregateFunctionTemplate(std::forward(args)...); return nullptr; } @@ -54,8 +54,8 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate>(std::forward(args)...); FOR_NUMERIC_TYPES(DISPATCH) #undef DISPATCH - if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate>(std::forward(args)...); - if (which.idx == TypeIndex::Enum16) return new AggregateFunctionTemplate>(std::forward(args)...); + if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate>(std::forward(args)...); + if (which.idx == TypeIndex::Enum16) return new AggregateFunctionTemplate>(std::forward(args)...); return nullptr; } @@ -106,8 +106,8 @@ static IAggregateFunction * createWithTwoNumericTypesSecond(const IDataType & se if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate(std::forward(args)...); FOR_NUMERIC_TYPES(DISPATCH) #undef DISPATCH - if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate(std::forward(args)...); - if (which.idx == TypeIndex::Enum16) return new AggregateFunctionTemplate(std::forward(args)...); + if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate(std::forward(args)...); + if (which.idx == TypeIndex::Enum16) return new AggregateFunctionTemplate(std::forward(args)...); return nullptr; } @@ -121,9 +121,9 @@ static IAggregateFunction * createWithTwoNumericTypes(const IDataType & first_ty FOR_NUMERIC_TYPES(DISPATCH) #undef DISPATCH if (which.idx == TypeIndex::Enum8) - return createWithTwoNumericTypesSecond(second_type, std::forward(args)...); + return createWithTwoNumericTypesSecond(second_type, std::forward(args)...); if (which.idx == TypeIndex::Enum16) - return createWithTwoNumericTypesSecond(second_type, std::forward(args)...); + return createWithTwoNumericTypesSecond(second_type, std::forward(args)...); return nullptr; } From c86e861b6dd1bfcdad21ec833eae871161c5f533 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 06:38:57 +0300 Subject: [PATCH 131/230] Fixed error #3926 --- dbms/src/Columns/IColumn.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 6cb91e12498..80f408e173b 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -398,7 +398,7 @@ class ColumnVectorHelper : public IColumn public: const char * getRawDataBegin() const { - return reinterpret_cast(this) + sizeof(*this); + return *reinterpret_cast(reinterpret_cast(this) + sizeof(*this)); } template From 3d3385139b20949484b00229f12c25ca96607106 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 07:09:24 +0300 Subject: [PATCH 132/230] Fixed error #3926 --- dbms/src/IO/ReadHelpers.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/IO/ReadHelpers.h b/dbms/src/IO/ReadHelpers.h index 8823fb3611e..2a91cf69309 100644 --- a/dbms/src/IO/ReadHelpers.h +++ b/dbms/src/IO/ReadHelpers.h @@ -348,6 +348,7 @@ void readIntTextUnsafe(T & x, ReadBuffer & buf) if (*buf.position() == '0') /// There are many zeros in real datasets. { ++buf.position(); + x = 0; return; } From fc39ce8673b5af8c6bf6a1e6c2f0756d033e62c7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 07:12:14 +0300 Subject: [PATCH 133/230] Fixed UB #3926 --- dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h index 094cae05615..c6fb68c86dd 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -121,8 +121,11 @@ public: offsets_to.push_back(offsets_to.back() + size); - typename ColumnVector::Container & data_to = static_cast &>(arr_to.getData()).getData(); - data_to.insert(this->data(place).value.begin(), this->data(place).value.end()); + if (size) + { + typename ColumnVector::Container & data_to = static_cast &>(arr_to.getData()).getData(); + data_to.insert(this->data(place).value.begin(), this->data(place).value.end()); + } } bool allocatesMemoryInArena() const override From 5ca01486dd89112dc458ba47f4cd5e1627f45741 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 07:32:58 +0300 Subject: [PATCH 134/230] Fixed UB #3926 --- dbms/src/Interpreters/AggregationCommon.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Interpreters/AggregationCommon.h b/dbms/src/Interpreters/AggregationCommon.h index a71c0b93738..12c2d53819b 100644 --- a/dbms/src/Interpreters/AggregationCommon.h +++ b/dbms/src/Interpreters/AggregationCommon.h @@ -168,23 +168,23 @@ static inline T ALWAYS_INLINE packFixed( switch (key_sizes[j]) { case 1: - memcpy(bytes + offset, &static_cast(key_columns[j])->getData()[i], 1); + memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin() + i, 1); offset += 1; break; case 2: - memcpy(bytes + offset, &static_cast(key_columns[j])->getData()[i], 2); + memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin() + i * 2, 2); offset += 2; break; case 4: - memcpy(bytes + offset, &static_cast(key_columns[j])->getData()[i], 4); + memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin() + i * 4, 4); offset += 4; break; case 8: - memcpy(bytes + offset, &static_cast(key_columns[j])->getData()[i], 8); + memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin() + i * 8, 8); offset += 8; break; default: - memcpy(bytes + offset, &static_cast(key_columns[j])->getChars()[i * key_sizes[j]], key_sizes[j]); + memcpy(bytes + offset, static_cast(key_columns[j])->getRawDataBegin() + i * key_sizes[j], key_sizes[j]); offset += key_sizes[j]; } } From 4a45db2b9b6b640675d6a583d8668d70a672055c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 07:40:16 +0300 Subject: [PATCH 135/230] Fixed UB #3926 --- dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h index c6fb68c86dd..26708c87520 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -77,12 +77,14 @@ public: if (!limit_num_elems) { - cur_elems.value.insert(rhs_elems.value.begin(), rhs_elems.value.end(), arena); + if (rhs_elems.value.size()) + cur_elems.value.insert(rhs_elems.value.begin(), rhs_elems.value.end(), arena); } else { UInt64 elems_to_insert = std::min(static_cast(max_elems) - cur_elems.value.size(), rhs_elems.value.size()); - cur_elems.value.insert(rhs_elems.value.begin(), rhs_elems.value.begin() + elems_to_insert, arena); + if (elems_to_insert) + cur_elems.value.insert(rhs_elems.value.begin(), rhs_elems.value.begin() + elems_to_insert, arena); } } From ed50d90812713595385a974a1cc7d2151b4afc5a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 07:50:56 +0300 Subject: [PATCH 136/230] Allowed UB in certain cases #3926 --- dbms/src/Functions/bitShiftLeft.cpp | 2 +- dbms/src/Functions/bitShiftRight.cpp | 2 +- dbms/src/Functions/intExp2.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/bitShiftLeft.cpp b/dbms/src/Functions/bitShiftLeft.cpp index 5d0be9364cd..40cf7dd8ef2 100644 --- a/dbms/src/Functions/bitShiftLeft.cpp +++ b/dbms/src/Functions/bitShiftLeft.cpp @@ -10,7 +10,7 @@ struct BitShiftLeftImpl using ResultType = typename NumberTraits::ResultOfBit::Type; template - static inline Result apply(A a, B b) + static inline __attribute__((no_sanitize("shift-exponent"))) Result apply(A a, B b) { return static_cast(a) << static_cast(b); } diff --git a/dbms/src/Functions/bitShiftRight.cpp b/dbms/src/Functions/bitShiftRight.cpp index 503fb7192ab..706335b3789 100644 --- a/dbms/src/Functions/bitShiftRight.cpp +++ b/dbms/src/Functions/bitShiftRight.cpp @@ -10,7 +10,7 @@ struct BitShiftRightImpl using ResultType = typename NumberTraits::ResultOfBit::Type; template - static inline Result apply(A a, B b) + static inline __attribute__((no_sanitize("shift-exponent"))) Result apply(A a, B b) { return static_cast(a) >> static_cast(b); } diff --git a/dbms/src/Functions/intExp2.cpp b/dbms/src/Functions/intExp2.cpp index 86b751cb6f5..05b234294d9 100644 --- a/dbms/src/Functions/intExp2.cpp +++ b/dbms/src/Functions/intExp2.cpp @@ -11,7 +11,7 @@ struct IntExp2Impl { using ResultType = UInt64; - static inline ResultType apply(A a) + static inline __attribute__((no_sanitize("shift-exponent"))) ResultType apply(A a) { return intExp2(a); } From d54d2f755879ecc26c51afdda65940327d797787 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 07:56:38 +0300 Subject: [PATCH 137/230] Allowed UB in certain cases #3926 --- dbms/src/Functions/FunctionUnaryArithmetic.h | 4 ++-- dbms/src/Functions/bitShiftLeft.cpp | 2 +- dbms/src/Functions/bitShiftRight.cpp | 2 +- dbms/src/Functions/intExp2.cpp | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Functions/FunctionUnaryArithmetic.h b/dbms/src/Functions/FunctionUnaryArithmetic.h index 8cc02420a26..f193bac67cf 100644 --- a/dbms/src/Functions/FunctionUnaryArithmetic.h +++ b/dbms/src/Functions/FunctionUnaryArithmetic.h @@ -37,14 +37,14 @@ struct UnaryOperationImpl using ArrayA = typename ColVecA::Container; using ArrayC = typename ColVecC::Container; - static void NO_INLINE vector(const ArrayA & a, ArrayC & c) + static void NO_INLINE __attribute__((no_sanitize("shift-exponent"))) vector(const ArrayA & a, ArrayC & c) { size_t size = a.size(); for (size_t i = 0; i < size; ++i) c[i] = Op::apply(a[i]); } - static void constant(A a, ResultType & c) + static void __attribute__((no_sanitize("shift-exponent"))) constant(A a, ResultType & c) { c = Op::apply(a); } diff --git a/dbms/src/Functions/bitShiftLeft.cpp b/dbms/src/Functions/bitShiftLeft.cpp index 40cf7dd8ef2..5d0be9364cd 100644 --- a/dbms/src/Functions/bitShiftLeft.cpp +++ b/dbms/src/Functions/bitShiftLeft.cpp @@ -10,7 +10,7 @@ struct BitShiftLeftImpl using ResultType = typename NumberTraits::ResultOfBit::Type; template - static inline __attribute__((no_sanitize("shift-exponent"))) Result apply(A a, B b) + static inline Result apply(A a, B b) { return static_cast(a) << static_cast(b); } diff --git a/dbms/src/Functions/bitShiftRight.cpp b/dbms/src/Functions/bitShiftRight.cpp index 706335b3789..503fb7192ab 100644 --- a/dbms/src/Functions/bitShiftRight.cpp +++ b/dbms/src/Functions/bitShiftRight.cpp @@ -10,7 +10,7 @@ struct BitShiftRightImpl using ResultType = typename NumberTraits::ResultOfBit::Type; template - static inline __attribute__((no_sanitize("shift-exponent"))) Result apply(A a, B b) + static inline Result apply(A a, B b) { return static_cast(a) >> static_cast(b); } diff --git a/dbms/src/Functions/intExp2.cpp b/dbms/src/Functions/intExp2.cpp index 05b234294d9..86b751cb6f5 100644 --- a/dbms/src/Functions/intExp2.cpp +++ b/dbms/src/Functions/intExp2.cpp @@ -11,7 +11,7 @@ struct IntExp2Impl { using ResultType = UInt64; - static inline __attribute__((no_sanitize("shift-exponent"))) ResultType apply(A a) + static inline ResultType apply(A a) { return intExp2(a); } From ee4c7fe815ab4a113d42d771388f97430ef3d566 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 07:59:20 +0300 Subject: [PATCH 138/230] Allowed UB in certain cases #3926 --- dbms/src/Functions/FunctionUnaryArithmetic.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/FunctionUnaryArithmetic.h b/dbms/src/Functions/FunctionUnaryArithmetic.h index f193bac67cf..ceb5d88a923 100644 --- a/dbms/src/Functions/FunctionUnaryArithmetic.h +++ b/dbms/src/Functions/FunctionUnaryArithmetic.h @@ -37,14 +37,14 @@ struct UnaryOperationImpl using ArrayA = typename ColVecA::Container; using ArrayC = typename ColVecC::Container; - static void NO_INLINE __attribute__((no_sanitize("shift-exponent"))) vector(const ArrayA & a, ArrayC & c) + static void NO_INLINE __attribute__((no_sanitize("undefined"))) vector(const ArrayA & a, ArrayC & c) { size_t size = a.size(); for (size_t i = 0; i < size; ++i) c[i] = Op::apply(a[i]); } - static void __attribute__((no_sanitize("shift-exponent"))) constant(A a, ResultType & c) + static void __attribute__((no_sanitize("undefined"))) constant(A a, ResultType & c) { c = Op::apply(a); } From 14e1dfa0ea536d53d0672151913c43e92bcebeee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 08:11:53 +0300 Subject: [PATCH 139/230] Fixed UB #3926 --- libs/libcommon/include/common/arithmeticOverflow.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/libcommon/include/common/arithmeticOverflow.h b/libs/libcommon/include/common/arithmeticOverflow.h index 6d824208c0d..38e47ccda57 100644 --- a/libs/libcommon/include/common/arithmeticOverflow.h +++ b/libs/libcommon/include/common/arithmeticOverflow.h @@ -95,7 +95,7 @@ namespace common template <> inline bool mulOverflow(__int128 x, __int128 y, __int128 & res) { - res = x * y; + res = static_cast(x) * static_cast(y); /// Avoid signed integer overflow. if (!x || !y) return false; From 5abb2d02b9d35ee2238cf2611bbb6320e4eabc5d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 08:34:25 +0300 Subject: [PATCH 140/230] Fixed UB #3926 --- dbms/src/Columns/ColumnArray.h | 4 ++-- dbms/src/Columns/ColumnString.h | 4 ++-- dbms/src/Common/PODArray.h | 5 +++-- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/dbms/src/Columns/ColumnArray.h b/dbms/src/Columns/ColumnArray.h index 0caf6cbb87e..3e1b586e755 100644 --- a/dbms/src/Columns/ColumnArray.h +++ b/dbms/src/Columns/ColumnArray.h @@ -124,8 +124,8 @@ private: ColumnPtr data; ColumnPtr offsets; - size_t ALWAYS_INLINE offsetAt(size_t i) const { return getOffsets()[i - 1]; } - size_t ALWAYS_INLINE sizeAt(size_t i) const { return getOffsets()[i] - getOffsets()[i - 1]; } + size_t ALWAYS_INLINE offsetAt(ssize_t i) const { return getOffsets()[i - 1]; } + size_t ALWAYS_INLINE sizeAt(ssize_t i) const { return getOffsets()[i] - getOffsets()[i - 1]; } /// Multiply values if the nested column is ColumnVector. diff --git a/dbms/src/Columns/ColumnString.h b/dbms/src/Columns/ColumnString.h index 645f9dcac47..73c1ae35d37 100644 --- a/dbms/src/Columns/ColumnString.h +++ b/dbms/src/Columns/ColumnString.h @@ -31,10 +31,10 @@ private: /// For convenience, every string ends with terminating zero byte. Note that strings could contain zero bytes in the middle. Chars chars; - size_t ALWAYS_INLINE offsetAt(size_t i) const { return offsets[i - 1]; } + size_t ALWAYS_INLINE offsetAt(ssize_t i) const { return offsets[i - 1]; } /// Size of i-th element, including terminating zero. - size_t ALWAYS_INLINE sizeAt(size_t i) const { return offsets[i] - offsets[i - 1]; } + size_t ALWAYS_INLINE sizeAt(ssize_t i) const { return offsets[i] - offsets[i - 1]; } template struct less; diff --git a/dbms/src/Common/PODArray.h b/dbms/src/Common/PODArray.h index f8cbf3392a7..cda0e03c700 100644 --- a/dbms/src/Common/PODArray.h +++ b/dbms/src/Common/PODArray.h @@ -273,8 +273,9 @@ public: T * data() { return t_start(); } const T * data() const { return t_start(); } - T & operator[] (size_t n) { return t_start()[n]; } - const T & operator[] (size_t n) const { return t_start()[n]; } + /// The index is signed to access -1th element without pointer overflow. + T & operator[] (ssize_t n) { return t_start()[n]; } + const T & operator[] (ssize_t n) const { return t_start()[n]; } T & front() { return t_start()[0]; } T & back() { return t_end()[-1]; } From 15a218f951f746d5820b0cd059a4b9a1b01514bb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 09:03:18 +0300 Subject: [PATCH 141/230] Fixed UB #3926 --- dbms/src/IO/Operators.h | 58 +++++++++++++++++++++-------------------- 1 file changed, 30 insertions(+), 28 deletions(-) diff --git a/dbms/src/IO/Operators.h b/dbms/src/IO/Operators.h index 82c74fc0b16..7e276ba9248 100644 --- a/dbms/src/IO/Operators.h +++ b/dbms/src/IO/Operators.h @@ -5,6 +5,8 @@ #include #include +#include + namespace DB { @@ -29,15 +31,15 @@ enum QuoteManip { quote }; /// For strings, dates, datetimes enum DoubleQuoteManip { double_quote }; /// For strings, dates, datetimes - enclose in double quotes with escaping. In the rest, as usual. enum BinaryManip { binary }; /// Output in binary format. -struct EscapeManipWriteBuffer : WriteBuffer {}; -struct QuoteManipWriteBuffer : WriteBuffer {}; -struct DoubleQuoteManipWriteBuffer : WriteBuffer {}; -struct BinaryManipWriteBuffer : WriteBuffer {}; +struct EscapeManipWriteBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; +struct QuoteManipWriteBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; +struct DoubleQuoteManipWriteBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; +struct BinaryManipWriteBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; -struct EscapeManipReadBuffer : ReadBuffer {}; -struct QuoteManipReadBuffer : ReadBuffer {}; -struct DoubleQuoteManipReadBuffer : ReadBuffer {}; -struct BinaryManipReadBuffer : ReadBuffer {}; +struct EscapeManipReadBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; +struct QuoteManipReadBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; +struct DoubleQuoteManipReadBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; +struct BinaryManipReadBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; template WriteBuffer & operator<< (WriteBuffer & buf, const T & x) { writeText(x, buf); return buf; } @@ -47,20 +49,20 @@ template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const cha inline WriteBuffer & operator<< (WriteBuffer & buf, const char * x) { writeCString(x, buf); return buf; } -inline EscapeManipWriteBuffer & operator<< (WriteBuffer & buf, EscapeManip) { return static_cast(buf); } -inline QuoteManipWriteBuffer & operator<< (WriteBuffer & buf, QuoteManip) { return static_cast(buf); } -inline DoubleQuoteManipWriteBuffer & operator<< (WriteBuffer & buf, DoubleQuoteManip) { return static_cast(buf); } -inline BinaryManipWriteBuffer & operator<< (WriteBuffer & buf, BinaryManip) { return static_cast(buf); } +inline EscapeManipWriteBuffer operator<< (WriteBuffer & buf, EscapeManip) { return buf; } +inline QuoteManipWriteBuffer operator<< (WriteBuffer & buf, QuoteManip) { return buf; } +inline DoubleQuoteManipWriteBuffer operator<< (WriteBuffer & buf, DoubleQuoteManip) { return buf; } +inline BinaryManipWriteBuffer operator<< (WriteBuffer & buf, BinaryManip) { return buf; } -template WriteBuffer & operator<< (EscapeManipWriteBuffer & buf, const T & x) { writeText(x, buf); return buf; } -template WriteBuffer & operator<< (QuoteManipWriteBuffer & buf, const T & x) { writeQuoted(x, buf); return buf; } -template WriteBuffer & operator<< (DoubleQuoteManipWriteBuffer & buf, const T & x) { writeDoubleQuoted(x, buf); return buf; } -template WriteBuffer & operator<< (BinaryManipWriteBuffer & buf, const T & x) { writeBinary(x, buf); return buf; } +template WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const T & x) { writeText(x, buf.get()); return buf; } +template WriteBuffer & operator<< (QuoteManipWriteBuffer buf, const T & x) { writeQuoted(x, buf.get()); return buf; } +template WriteBuffer & operator<< (DoubleQuoteManipWriteBuffer buf, const T & x) { writeDoubleQuoted(x, buf.get()); return buf; } +template WriteBuffer & operator<< (BinaryManipWriteBuffer buf, const T & x) { writeBinary(x, buf.get()); return buf; } -inline WriteBuffer & operator<< (EscapeManipWriteBuffer & buf, const char * x) { writeAnyEscapedString<'\''>(x, x + strlen(x), buf); return buf; } -inline WriteBuffer & operator<< (QuoteManipWriteBuffer & buf, const char * x) { writeAnyQuotedString<'\''>(x, x + strlen(x), buf); return buf; } -inline WriteBuffer & operator<< (DoubleQuoteManipWriteBuffer & buf, const char * x) { writeAnyQuotedString<'"'>(x, x + strlen(x), buf); return buf; } -inline WriteBuffer & operator<< (BinaryManipWriteBuffer & buf, const char * x) { writeStringBinary(x, buf); return buf; } +inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const char * x) { writeAnyEscapedString<'\''>(x, x + strlen(x), buf.get()); return buf; } +inline WriteBuffer & operator<< (QuoteManipWriteBuffer buf, const char * x) { writeAnyQuotedString<'\''>(x, x + strlen(x), buf.get()); return buf; } +inline WriteBuffer & operator<< (DoubleQuoteManipWriteBuffer buf, const char * x) { writeAnyQuotedString<'"'>(x, x + strlen(x), buf.get()); return buf; } +inline WriteBuffer & operator<< (BinaryManipWriteBuffer buf, const char * x) { writeStringBinary(x, buf.get()); return buf; } /// The manipulator calls the WriteBuffer method `next` - this makes the buffer reset. For nested buffers, the reset is not recursive. enum FlushManip { flush }; @@ -75,14 +77,14 @@ template <> inline ReadBuffer & operator>> (ReadBuffer & buf, char & x) /// If you specify a string literal for reading, this will mean - make sure there is a sequence of bytes and skip it. inline ReadBuffer & operator>> (ReadBuffer & buf, const char * x) { assertString(x, buf); return buf; } -inline EscapeManipReadBuffer & operator>> (ReadBuffer & buf, EscapeManip) { return static_cast(buf); } -inline QuoteManipReadBuffer & operator>> (ReadBuffer & buf, QuoteManip) { return static_cast(buf); } -inline DoubleQuoteManipReadBuffer & operator>> (ReadBuffer & buf, DoubleQuoteManip) { return static_cast(buf); } -inline BinaryManipReadBuffer & operator>> (ReadBuffer & buf, BinaryManip) { return static_cast(buf); } +inline EscapeManipReadBuffer operator>> (ReadBuffer & buf, EscapeManip) { return buf; } +inline QuoteManipReadBuffer operator>> (ReadBuffer & buf, QuoteManip) { return buf; } +inline DoubleQuoteManipReadBuffer operator>> (ReadBuffer & buf, DoubleQuoteManip) { return buf; } +inline BinaryManipReadBuffer operator>> (ReadBuffer & buf, BinaryManip) { return buf; } -template ReadBuffer & operator>> (EscapeManipReadBuffer & buf, T & x) { readText(x, buf); return buf; } -template ReadBuffer & operator>> (QuoteManipReadBuffer & buf, T & x) { readQuoted(x, buf); return buf; } -template ReadBuffer & operator>> (DoubleQuoteManipReadBuffer & buf, T & x) { readDoubleQuoted(x, buf); return buf; } -template ReadBuffer & operator>> (BinaryManipReadBuffer & buf, T & x) { readBinary(x, buf); return buf; } +template ReadBuffer & operator>> (EscapeManipReadBuffer buf, T & x) { readText(x, buf.get()); return buf; } +template ReadBuffer & operator>> (QuoteManipReadBuffer buf, T & x) { readQuoted(x, buf.get()); return buf; } +template ReadBuffer & operator>> (DoubleQuoteManipReadBuffer buf, T & x) { readDoubleQuoted(x, buf.get()); return buf; } +template ReadBuffer & operator>> (BinaryManipReadBuffer buf, T & x) { readBinary(x, buf.get()); return buf; } } From 444eb21d280182bf85e2818ed7747f686385c4d1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 26 Dec 2018 12:56:52 +0300 Subject: [PATCH 142/230] Fix exponential backoff for replication queue --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index afe8cbc02ab..5e7dd32b8e6 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2077,7 +2077,16 @@ bool StorageReplicatedMergeTree::queueTask() LogEntryPtr & entry = selected.first; if (!entry) - return false; + { + /// Nothing to do, we can sleep for some time, just not to + /// abuse background pool scheduling policy + std::this_thread::sleep_for(std::chrono::seconds(5)); + + /// If we return false, than background pool for this task + /// will accumulate exponential backoff and after empty replication queue + /// we will sleep for a long time + return true; + } time_t prev_attempt_time = entry->last_attempt_time; From 98b372415d176dc37e125d4f1f90bc66d1f0a5f6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 26 Dec 2018 14:23:15 +0300 Subject: [PATCH 143/230] Decrease sleep time --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 5e7dd32b8e6..ae1e563d33f 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2080,7 +2080,7 @@ bool StorageReplicatedMergeTree::queueTask() { /// Nothing to do, we can sleep for some time, just not to /// abuse background pool scheduling policy - std::this_thread::sleep_for(std::chrono::seconds(5)); + std::this_thread::sleep_for(std::chrono::milliseconds(100)); /// If we return false, than background pool for this task /// will accumulate exponential backoff and after empty replication queue From 62905013d7ca160b0803997f03496b2b0b0f3fcd Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 26 Dec 2018 17:08:57 +0300 Subject: [PATCH 144/230] rewrite ExpressionAnalyzer.collectUsedColumns CLICKHOUSE-3996 --- dbms/src/Interpreters/ColumnNamesContext.cpp | 113 +++++++++ dbms/src/Interpreters/ColumnNamesContext.h | 74 ++++++ dbms/src/Interpreters/ExpressionAnalyzer.cpp | 203 +++++++++------ dbms/src/Interpreters/ExpressionAnalyzer.h | 3 +- .../RequiredSourceColumnsVisitor.cpp | 232 ++++++++++++++++++ .../RequiredSourceColumnsVisitor.h | 138 ++--------- 6 files changed, 576 insertions(+), 187 deletions(-) create mode 100644 dbms/src/Interpreters/ColumnNamesContext.cpp create mode 100644 dbms/src/Interpreters/ColumnNamesContext.h create mode 100644 dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp diff --git a/dbms/src/Interpreters/ColumnNamesContext.cpp b/dbms/src/Interpreters/ColumnNamesContext.cpp new file mode 100644 index 00000000000..1d17106d3e7 --- /dev/null +++ b/dbms/src/Interpreters/ColumnNamesContext.cpp @@ -0,0 +1,113 @@ +#include +#include + +namespace DB +{ + +bool ColumnNamesContext::addTableAliasIfAny(const IAST & ast) +{ + String alias = ast.tryGetAlias(); + if (alias.empty()) + return false; + + table_aliases.insert(alias); + return true; +} + +bool ColumnNamesContext::addColumnAliasIfAny(const IAST & ast, bool is_public) +{ + String alias = ast.tryGetAlias(); + if (alias.empty()) + return false; + + if (required_names.count(alias)) + masked_columns.insert(alias); + + if (is_public) + public_columns.insert(alias); + column_aliases.insert(alias); + return true; +} + +void ColumnNamesContext::addColumnIdentifier(const ASTIdentifier & node, bool is_public) +{ + if (!node.general()) + return; + + required_names.insert(node.name); + + if (!addColumnAliasIfAny(node, is_public) && is_public) + public_columns.insert(node.name); +} + +bool ColumnNamesContext::addArrayJoinAliasIfAny(const IAST & ast) +{ + String alias = ast.tryGetAlias(); + if (alias.empty()) + return false; + + array_join_columns.insert(alias); + return true; +} + +void ColumnNamesContext::addArrayJoinIdentifier(const ASTIdentifier & node) +{ + array_join_columns.insert(node.name); +} + +NameSet ColumnNamesContext::requiredColumns() const +{ + NameSet required; + for (const auto & name : required_names) + { + String table_name = Nested::extractTableName(name); + + /// Tech debt. There's its own logic for ARRAY JOIN columns. + if (array_join_columns.count(name) || array_join_columns.count(table_name)) + continue; + + if (!column_aliases.count(name) || masked_columns.count(name)) + required.insert(name); + } + return required; +} + +std::ostream & operator << (std::ostream & os, const ColumnNamesContext & cols) +{ + os << "required_names: "; + for (const auto & x : cols.required_names) + os << "'" << x << "' "; + os << "source_tables: "; + for (const auto & x : cols.tables) + { + auto alias = x.alias(); + auto name = x.name(); + if (alias && name) + os << "'" << *alias << "'/'" << *name << "' "; + else if (alias) + os << "'" << *alias << "' "; + else if (name) + os << "'" << *name << "' "; + } + os << "table_aliases: "; + for (const auto & x : cols.table_aliases) + os << "'" << x << "' "; + os << "private_aliases: "; + for (const auto & x : cols.private_aliases) + os << "'" << x << "' "; + os << "column_aliases: "; + for (const auto & x : cols.column_aliases) + os << "'" << x << "' "; + os << "public_columns: "; + for (const auto & x : cols.public_columns) + os << "'" << x << "' "; + os << "masked_columns: "; + for (const auto & x : cols.masked_columns) + os << "'" << x << "' "; + os << "array_join_columns: "; + for (const auto & x : cols.array_join_columns) + os << "'" << x << "' "; + return os; +} + +} diff --git a/dbms/src/Interpreters/ColumnNamesContext.h b/dbms/src/Interpreters/ColumnNamesContext.h new file mode 100644 index 00000000000..63ad5a4e2e4 --- /dev/null +++ b/dbms/src/Interpreters/ColumnNamesContext.h @@ -0,0 +1,74 @@ +#pragma once + +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +/// Information about table and column names extracted from ASTSelectQuery block. Do not include info from subselects. +struct ColumnNamesContext +{ + struct JoinedTable + { + const ASTTableExpression * expr; + const ASTTableJoin * join; + + std::optional alias() const + { + String alias; + if (expr->database_and_table_name) + alias = expr->database_and_table_name->tryGetAlias(); + else if (expr->table_function) + alias = expr->table_function->tryGetAlias(); + else if (expr->subquery) + alias = expr->subquery->tryGetAlias(); + if (!alias.empty()) + return alias; + return {}; + } + + std::optional name() const + { + if (auto * node = expr->database_and_table_name.get()) + if (auto * identifier = typeid_cast(node)) + return identifier->name; + return {}; + } + + std::optional joinKind() const + { + if (join) + return join->kind; + return {}; + } + }; + + NameSet required_names; + NameSet table_aliases; + NameSet private_aliases; + NameSet column_aliases; + NameSet masked_columns; + NameSet public_columns; + NameSet array_join_columns; + std::vector tables; /// ordered list of visited tables in FROM section with joins + bool has_table_join = false; + bool has_array_join = false; + + bool addTableAliasIfAny(const IAST & ast); + bool addColumnAliasIfAny(const IAST & ast, bool is_public = false); + void addColumnIdentifier(const ASTIdentifier & node, bool is_public = false); + bool addArrayJoinAliasIfAny(const IAST & ast); + void addArrayJoinIdentifier(const ASTIdentifier & node); + + NameSet requiredColumns() const; +}; + +std::ostream & operator << (std::ostream & os, const ColumnNamesContext & cols); + +} diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index e488aa5d81b..92e26786e95 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -106,10 +106,10 @@ ExpressionAnalyzer::ExpressionAnalyzer( removeDuplicateColumns(source_columns); } - /// Delete the unnecessary from `source_columns` list. Create `unknown_required_source_columns`. Form `columns_added_by_join`. + /// Delete the unnecessary from `source_columns` list. Form `columns_added_by_join`. collectUsedColumns(); - /// external_tables, subqueries_for_sets for global subqueries. + /// external_tables, subqueries_for_sets for global subqueries.f /// Replaces global subqueries with the generated names of temporary tables that will be sent to remote servers. initGlobalSubqueriesAndExternalTables(); @@ -1001,6 +1001,15 @@ void ExpressionAnalyzer::getAggregateInfo(Names & key_names, AggregateDescriptio aggregates = aggregate_descriptions; } +/// db.table.column -> table.column / table.column -> column +static String cropDatabaseOrTableName(const String & name) +{ + size_t pos = name.find('.', 0); + if (pos != std::string::npos) + return name.substr(pos + 1, name.size() - pos - 1); + return name; +} + void ExpressionAnalyzer::collectUsedColumns() { /** Calculate which columns are required to execute the expression. @@ -1008,83 +1017,131 @@ void ExpressionAnalyzer::collectUsedColumns() * After execution, columns will only contain the list of columns needed to read from the table. */ - NameSet required; - NameSet ignored; + RequiredSourceColumnsVisitor::Data columns_context; + RequiredSourceColumnsVisitor(columns_context).visit(query); - NameSet available_columns; - for (const auto & column : source_columns) - available_columns.insert(column.name); + NameSet required = columns_context.requiredColumns(); - if (select_query && select_query->array_join_expression_list()) +#if 0 + std::cerr << "Query: " << query << std::endl; + std::cerr << "CTX: " << columns_context << std::endl; + std::cerr << "source_columns: "; + for (const auto & name : source_columns) + std::cerr << "'" << name.name << "' "; + std::cerr << "required: "; + for (const auto & name : required) + std::cerr << "'" << name << "' "; + std::cerr << std::endl; +#endif + + if (columns_context.has_table_join) { - ASTs & expressions = select_query->array_join_expression_list()->children; - for (size_t i = 0; i < expressions.size(); ++i) + const AnalyzedJoin & analyzed_join = analyzedJoin(); +#if 0 + std::cerr << "key_names_left: "; + for (const auto & name : analyzed_join.key_names_left) + std::cerr << "'" << name << "' "; + std::cerr << "key_names_right: "; + for (const auto & name : analyzed_join.key_names_right) + std::cerr << "'" << name << "' "; + std::cerr << "columns_from_joined_table: "; + for (const auto & column : analyzed_join.columns_from_joined_table) + std::cerr << "'" << column.name_and_type.name << '/' << column.original_name << "' "; + std::cerr << "available_joined_columns: "; + for (const auto & column : analyzed_join.available_joined_columns) + std::cerr << "'" << column.name_and_type.name << '/' << column.original_name << "' "; + std::cerr << std::endl; +#endif + NameSet avaliable_columns; + for (const auto & name : source_columns) + avaliable_columns.insert(name.name); + + NameSet right_keys; + for (const auto & right_key_name : analyzed_join.key_names_right) + right_keys.insert(right_key_name); + + /** You also need to ignore the identifiers of the columns that are obtained by JOIN. + * (Do not assume that they are required for reading from the "left" table). + */ + columns_added_by_join.clear(); + for (const auto & joined_column : analyzed_join.available_joined_columns) { - /// Ignore the top-level identifiers from the ARRAY JOIN section. - /// Then add them separately. - if (typeid_cast(expressions[i].get())) + auto & name = joined_column.name_and_type.name; + if (required.count(name) && !avaliable_columns.count(name)) { - ignored.insert(expressions[i]->getColumnName()); - } - else - { - /// Nothing needs to be ignored for expressions in ARRAY JOIN. - NameSet empty; - RequiredSourceColumnsVisitor::Data visitor_data{available_columns, required, empty, empty, empty}; - RequiredSourceColumnsVisitor(visitor_data).visit(expressions[i]); - } + columns_added_by_join.push_back(joined_column); + required.erase(name); - ignored.insert(expressions[i]->getAliasOrColumnName()); + /// Some columns from right join key may be used in query. This columns will be appended to block during join. + if (right_keys.count(name)) + columns_added_by_join_from_right_keys.insert(name); + } } + + /// @fix filter required columns according to misqualified names in JOIN ON + if (columns_context.has_table_join && + columns_context.tables.size() >= 2 && + columns_context.tables[1].join && + columns_context.tables[1].join->on_expression) + { + NameSet fixed_required; + + for (const auto & req_name : required) + { + bool collated = false; + String cropped_name = req_name; + static const constexpr size_t max_column_prefix = 2; + + for (size_t i = 0; i < max_column_prefix && !collated; ++i) + { + cropped_name = cropDatabaseOrTableName(cropped_name); + + if (avaliable_columns.count(cropped_name)) + { + fixed_required.insert(cropped_name); + collated = true; + break; + } + + for (const auto & joined_column : analyzed_join.available_joined_columns) + { + auto & name = joined_column.name_and_type.name; + + if (cropped_name == name) + { + columns_added_by_join.push_back(joined_column); + if (right_keys.count(name)) + columns_added_by_join_from_right_keys.insert(name); + collated = true; + break; + } + } + } + + if (!collated) + fixed_required.insert(req_name); + } + + required.swap(fixed_required); + } + + /// @note required_columns_from_joined_table is output + joined_block_actions = analyzed_join.createJoinedBlockActions( + columns_added_by_join, select_query, context, required_columns_from_joined_table); } - /** You also need to ignore the identifiers of the columns that are obtained by JOIN. - * (Do not assume that they are required for reading from the "left" table). - */ - NameSet available_joined_columns; - for (const auto & joined_column : analyzedJoin().available_joined_columns) - available_joined_columns.insert(joined_column.name_and_type.name); - - NameSet required_joined_columns; - - for (const auto & left_key_ast : syntax->analyzed_join.key_asts_left) + if (columns_context.has_array_join) { - NameSet empty; - RequiredSourceColumnsVisitor::Data columns_data{available_columns, required, ignored, empty, required_joined_columns}; - ASTPtr tmp = left_key_ast; - RequiredSourceColumnsVisitor(columns_data).visit(tmp); + /// Insert the columns required for the ARRAY JOIN calculation into the required columns list. + NameSet array_join_sources; + for (const auto & result_source : syntax->array_join_result_to_source) + array_join_sources.insert(result_source.second); + + for (const auto & column_name_type : source_columns) + if (array_join_sources.count(column_name_type.name)) + required.insert(column_name_type.name); } - RequiredSourceColumnsVisitor::Data columns_visitor_data{available_columns, required, ignored, - available_joined_columns, required_joined_columns}; - RequiredSourceColumnsVisitor(columns_visitor_data).visit(query); - - columns_added_by_join = analyzedJoin().available_joined_columns; - for (auto it = columns_added_by_join.begin(); it != columns_added_by_join.end();) - { - if (required_joined_columns.count(it->name_and_type.name)) - ++it; - else - columns_added_by_join.erase(it++); - } - - joined_block_actions = analyzedJoin().createJoinedBlockActions( - columns_added_by_join, select_query, context, required_columns_from_joined_table); - - /// Some columns from right join key may be used in query. This columns will be appended to block during join. - for (const auto & right_key_name : analyzedJoin().key_names_right) - if (required_joined_columns.count(right_key_name)) - columns_added_by_join_from_right_keys.insert(right_key_name); - - /// Insert the columns required for the ARRAY JOIN calculation into the required columns list. - NameSet array_join_sources; - for (const auto & result_source : syntax->array_join_result_to_source) - array_join_sources.insert(result_source.second); - - for (const auto & column_name_type : source_columns) - if (array_join_sources.count(column_name_type.name)) - required.insert(column_name_type.name); - /// You need to read at least one column to find the number of rows. if (select_query && required.empty()) required.insert(ExpressionActions::getSmallestColumn(source_columns)); @@ -1118,9 +1175,17 @@ void ExpressionAnalyzer::collectUsedColumns() } if (!unknown_required_source_columns.empty()) + { + std::stringstream ss; + ss << columns_context; + ss << "source_columns: "; + for (const auto & name : source_columns) + ss << "'" << name.name << "' "; + throw Exception("Unknown identifier: " + *unknown_required_source_columns.begin() - + (select_query && !select_query->tables ? ". Note that there is no tables (FROM clause) in your query" : ""), - ErrorCodes::UNKNOWN_IDENTIFIER); + + (select_query && !select_query->tables ? ". Note that there is no tables (FROM clause) in your query" : "") + + ", context: " + ss.str(), ErrorCodes::UNKNOWN_IDENTIFIER); + } } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index ba1e3a252d0..5c89df50863 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -232,8 +232,7 @@ private: const AnalyzedJoin & analyzedJoin() const { return syntax->analyzed_join; } /** Remove all unnecessary columns from the list of all available columns of the table (`columns`). - * At the same time, form a set of unknown columns (`unknown_required_source_columns`), - * as well as the columns added by JOIN (`columns_added_by_join`). + * At the same time, form a set of columns added by JOIN (`columns_added_by_join`). */ void collectUsedColumns(); diff --git a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp new file mode 100644 index 00000000000..b4ce3281d09 --- /dev/null +++ b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -0,0 +1,232 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +static std::vector extractNamesFromLambda(const ASTFunction & node) +{ + if (node.arguments->children.size() != 2) + throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + ASTFunction * lambda_args_tuple = typeid_cast(node.arguments->children[0].get()); + + if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") + throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH); + + std::vector names; + for (auto & child : lambda_args_tuple->arguments->children) + { + ASTIdentifier * identifier = typeid_cast(child.get()); + if (!identifier) + throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); + + names.push_back(identifier->name); + } + + return names; +} + +bool RequiredSourceColumnsMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child) +{ + if (typeid_cast(child.get())) + return false; + + /// Processed. Do not need children. + if (typeid_cast(node.get()) || + typeid_cast(node.get()) || + typeid_cast(node.get()) || + typeid_cast(node.get())) + return false; + + if (auto * f = typeid_cast(node.get())) + { + /// "indexHint" is a special function for index analysis. Everything that is inside it is not calculated. @sa KeyCondition + /// "lambda" visit children itself. + if (f->name == "indexHint" || f->name == "lambda") + return false; + } + + return true; +} + +std::vector RequiredSourceColumnsMatcher::visit(ASTPtr & ast, Data & data) +{ + /// results are columns + + if (auto * t = typeid_cast(ast.get())) + { + data.addColumnAliasIfAny(*ast); + visit(*t, ast, data); + return {}; + } + if (auto * t = typeid_cast(ast.get())) + { + data.addColumnAliasIfAny(*ast); + visit(*t, ast, data); + return {}; + } + + /// results are tables + + if (auto * t = typeid_cast(ast.get())) + { + visit(*t, ast, data); + return {}; + } + + if (auto * t = typeid_cast(ast.get())) + { + //data.addTableAliasIfAny(*ast); alias is attached to child + visit(*t, ast, data); + return {}; + } + if (auto * t = typeid_cast(ast.get())) + { + data.addTableAliasIfAny(*ast); + return visit(*t, ast, data); + } + if (auto * t = typeid_cast(ast.get())) + { + data.addTableAliasIfAny(*ast); + return {}; + } + + /// other + + if (auto * t = typeid_cast(ast.get())) + { + data.has_array_join = true; + return visit(*t, ast, data); + } + + return {}; +} + +std::vector RequiredSourceColumnsMatcher::visit(ASTSelectQuery & select, const ASTPtr &, Data & data) +{ + /// special case for top-level SELECT items: they are publics + for (auto & node : select.select_expression_list->children) + { + if (auto * identifier = typeid_cast(node.get())) + data.addColumnIdentifier(*identifier, true); + else + data.addColumnAliasIfAny(*node, true); + } + + std::vector out; + for (auto & node : select.children) + if (node != select.select_expression_list) + out.push_back(&node); + + /// revisit select_expression_list (with children) when all the aliases are set + out.push_back(&select.select_expression_list); + return out; +} + +void RequiredSourceColumnsMatcher::visit(const ASTIdentifier & node, const ASTPtr &, Data & data) +{ + if (node.name.empty()) + throw Exception("Expected not empty name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!data.private_aliases.count(node.name)) + data.addColumnIdentifier(node); +} + +void RequiredSourceColumnsMatcher::visit(const ASTFunction & node, const ASTPtr &, Data & data) +{ + /// Do not add formal parameters of the lambda expression + if (node.name == "lambda") + { + Names local_aliases; + for (const auto & name : extractNamesFromLambda(node)) + if (data.private_aliases.insert(name).second) + local_aliases.push_back(name); + + /// visit child with masked local aliases + visit(node.arguments->children[1], data); + + for (const auto & name : local_aliases) + data.private_aliases.erase(name); + } +} + +void RequiredSourceColumnsMatcher::visit(ASTTablesInSelectQueryElement & node, const ASTPtr &, Data & data) +{ + ASTTableExpression * expr = nullptr; + ASTTableJoin * join = nullptr; + + for (auto & child : node.children) + { + if (auto * e = typeid_cast(child.get())) + expr = e; + if (auto * j = typeid_cast(child.get())) + join = j; + } + + if (join) + data.has_table_join = true; + data.tables.emplace_back(ColumnNamesContext::JoinedTable{expr, join}); +} + +std::vector RequiredSourceColumnsMatcher::visit(ASTTableExpression & node, const ASTPtr &, Data & data) +{ + /// ASTIdentifiers here are tables. Do not visit them as generic ones. + if (node.database_and_table_name) + data.addTableAliasIfAny(*node.database_and_table_name); + + std::vector out; + if (node.table_function) + { + data.addTableAliasIfAny(*node.table_function); + out.push_back(&node.table_function); + } + + if (node.subquery) + { + data.addTableAliasIfAny(*node.subquery); + out.push_back(&node.subquery); + } + + return out; +} + +std::vector RequiredSourceColumnsMatcher::visit(const ASTArrayJoin & node, const ASTPtr &, Data & data) +{ + ASTPtr expression_list = node.expression_list; + if (!expression_list || expression_list->children.empty()) + throw Exception("Expected not empty expression_list", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + std::vector out; + + /// Tech debt. Ignore ARRAY JOIN top-level identifiers and aliases. There's its own logic for them. + for (auto & expr : expression_list->children) + { + data.addArrayJoinAliasIfAny(*expr); + + if (auto * identifier = typeid_cast(expr.get())) + { + data.addArrayJoinIdentifier(*identifier); + continue; + } + + out.push_back(&expr); + } + + return out; +} + +} diff --git a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h index ebf948ffeb8..3e107111e96 100644 --- a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h +++ b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h @@ -1,139 +1,45 @@ #pragma once -#include -#include -#include -#include -#include -#include -#include -#include +#include +#include namespace DB { namespace ErrorCodes { - extern const int TYPE_MISMATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -/** Get a set of necessary columns to read from the table. - * In this case, the columns specified in ignored_names are considered unnecessary. And the ignored_names parameter can be modified. - * The set of columns available_joined_columns are the columns available from JOIN, they are not needed for reading from the main table. - * Put in required_joined_columns the set of columns available from JOIN and needed. - */ +class ASTIdentifier; +class ASTFunction; +class ASTSelectQuery; +struct ASTTablesInSelectQueryElement; +struct ASTArrayJoin; +struct ASTTableExpression; + class RequiredSourceColumnsMatcher { public: - struct Data - { - const NameSet & available_columns; - NameSet & required_source_columns; - NameSet & ignored_names; - const NameSet & available_joined_columns; - NameSet & required_joined_columns; - }; + using Data = ColumnNamesContext; static constexpr const char * label = "RequiredSourceColumns"; - static bool needChildVisit(ASTPtr & node, const ASTPtr & child) - { - /// We will not go to the ARRAY JOIN section, because we need to look at the names of non-ARRAY-JOIN columns. - /// There, `collectUsedColumns` will send us separately. - if (typeid_cast(child.get()) || - typeid_cast(child.get()) || - typeid_cast(child.get()) || - typeid_cast(child.get())) - return false; - - /// Processed. Do not need children. - if (typeid_cast(node.get())) - return false; - - if (auto * f = typeid_cast(node.get())) - { - /// "indexHint" is a special function for index analysis. Everything that is inside it is not calculated. @sa KeyCondition - /// "lambda" visit children itself. - if (f->name == "indexHint" || f->name == "lambda") - return false; - } - - return true; - } - - /** Find all the identifiers in the query. - * We will use depth first search in AST. - * In this case - * - for lambda functions we will not take formal parameters; - * - do not go into subqueries (they have their own identifiers); - * - there is some exception for the ARRAY JOIN clause (it has a slightly different identifiers); - * - we put identifiers available from JOIN in required_joined_columns. - */ - static std::vector visit(ASTPtr & ast, Data & data) - { - if (auto * t = typeid_cast(ast.get())) - visit(*t, ast, data); - if (auto * t = typeid_cast(ast.get())) - visit(*t, ast, data); - return {}; - } + static bool needChildVisit(ASTPtr & node, const ASTPtr & child); + static std::vector visit(ASTPtr & ast, Data & data); private: - static void visit(const ASTIdentifier & node, const ASTPtr &, Data & data) - { - if (node.general() - && !data.ignored_names.count(node.name) - && !data.ignored_names.count(Nested::extractTableName(node.name))) - { - /// Read column from left table if has. - if (!data.available_joined_columns.count(node.name) || data.available_columns.count(node.name)) - data.required_source_columns.insert(node.name); - else - data.required_joined_columns.insert(node.name); - } - } - - static void visit(const ASTFunction & node, const ASTPtr &, Data & data) - { - NameSet & ignored_names = data.ignored_names; - - if (node.name == "lambda") - { - if (node.arguments->children.size() != 2) - throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - ASTFunction * lambda_args_tuple = typeid_cast(node.arguments->children.at(0).get()); - - if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") - throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH); - - /// You do not need to add formal parameters of the lambda expression in required_source_columns. - Names added_ignored; - for (auto & child : lambda_args_tuple->arguments->children) - { - ASTIdentifier * identifier = typeid_cast(child.get()); - if (!identifier) - throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); - - String & name = identifier->name; - if (!ignored_names.count(name)) - { - ignored_names.insert(name); - added_ignored.push_back(name); - } - } - - /// @note It's a special case where we visit children inside the matcher, not in visitor. - visit(node.arguments->children[1], data); - - for (size_t i = 0; i < added_ignored.size(); ++i) - ignored_names.erase(added_ignored[i]); - } - } + static void visit(const ASTIdentifier & node, const ASTPtr &, Data & data); + static void visit(const ASTFunction & node, const ASTPtr &, Data & data); + static void visit(ASTTablesInSelectQueryElement & node, const ASTPtr &, Data & data); + static std::vector visit(ASTTableExpression & node, const ASTPtr &, Data & data); + static std::vector visit(const ASTArrayJoin & node, const ASTPtr &, Data & data); + static std::vector visit(ASTSelectQuery & select, const ASTPtr &, Data & data); }; -/// Get a set of necessary columns to read from the table. -using RequiredSourceColumnsVisitor = InDepthNodeVisitor; +/// Extracts all the information about columns and tables from ASTSelectQuery block into ColumnNamesContext object. +/// It doesn't use anithing but AST. It visits nodes from bottom to top except ASTFunction content to get aliases in right manner. +/// @note There's some ambiguousness with nested columns names that can't be solved without schema. +using RequiredSourceColumnsVisitor = InDepthNodeVisitor; } From 6ea13516f06abeccfd37df51378da04a7d578be2 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 26 Dec 2018 17:43:25 +0300 Subject: [PATCH 145/230] rewrite ExpressionAnalyzer.collectUsedColumns CLICKHOUSE-3996 --- dbms/src/Interpreters/ColumnNamesContext.cpp | 113 +++++++++ dbms/src/Interpreters/ColumnNamesContext.h | 74 ++++++ dbms/src/Interpreters/ExpressionAnalyzer.cpp | 203 +++++++++------ dbms/src/Interpreters/ExpressionAnalyzer.h | 3 +- .../RequiredSourceColumnsVisitor.cpp | 232 ++++++++++++++++++ .../RequiredSourceColumnsVisitor.h | 139 ++--------- 6 files changed, 576 insertions(+), 188 deletions(-) create mode 100644 dbms/src/Interpreters/ColumnNamesContext.cpp create mode 100644 dbms/src/Interpreters/ColumnNamesContext.h create mode 100644 dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp diff --git a/dbms/src/Interpreters/ColumnNamesContext.cpp b/dbms/src/Interpreters/ColumnNamesContext.cpp new file mode 100644 index 00000000000..1d17106d3e7 --- /dev/null +++ b/dbms/src/Interpreters/ColumnNamesContext.cpp @@ -0,0 +1,113 @@ +#include +#include + +namespace DB +{ + +bool ColumnNamesContext::addTableAliasIfAny(const IAST & ast) +{ + String alias = ast.tryGetAlias(); + if (alias.empty()) + return false; + + table_aliases.insert(alias); + return true; +} + +bool ColumnNamesContext::addColumnAliasIfAny(const IAST & ast, bool is_public) +{ + String alias = ast.tryGetAlias(); + if (alias.empty()) + return false; + + if (required_names.count(alias)) + masked_columns.insert(alias); + + if (is_public) + public_columns.insert(alias); + column_aliases.insert(alias); + return true; +} + +void ColumnNamesContext::addColumnIdentifier(const ASTIdentifier & node, bool is_public) +{ + if (!node.general()) + return; + + required_names.insert(node.name); + + if (!addColumnAliasIfAny(node, is_public) && is_public) + public_columns.insert(node.name); +} + +bool ColumnNamesContext::addArrayJoinAliasIfAny(const IAST & ast) +{ + String alias = ast.tryGetAlias(); + if (alias.empty()) + return false; + + array_join_columns.insert(alias); + return true; +} + +void ColumnNamesContext::addArrayJoinIdentifier(const ASTIdentifier & node) +{ + array_join_columns.insert(node.name); +} + +NameSet ColumnNamesContext::requiredColumns() const +{ + NameSet required; + for (const auto & name : required_names) + { + String table_name = Nested::extractTableName(name); + + /// Tech debt. There's its own logic for ARRAY JOIN columns. + if (array_join_columns.count(name) || array_join_columns.count(table_name)) + continue; + + if (!column_aliases.count(name) || masked_columns.count(name)) + required.insert(name); + } + return required; +} + +std::ostream & operator << (std::ostream & os, const ColumnNamesContext & cols) +{ + os << "required_names: "; + for (const auto & x : cols.required_names) + os << "'" << x << "' "; + os << "source_tables: "; + for (const auto & x : cols.tables) + { + auto alias = x.alias(); + auto name = x.name(); + if (alias && name) + os << "'" << *alias << "'/'" << *name << "' "; + else if (alias) + os << "'" << *alias << "' "; + else if (name) + os << "'" << *name << "' "; + } + os << "table_aliases: "; + for (const auto & x : cols.table_aliases) + os << "'" << x << "' "; + os << "private_aliases: "; + for (const auto & x : cols.private_aliases) + os << "'" << x << "' "; + os << "column_aliases: "; + for (const auto & x : cols.column_aliases) + os << "'" << x << "' "; + os << "public_columns: "; + for (const auto & x : cols.public_columns) + os << "'" << x << "' "; + os << "masked_columns: "; + for (const auto & x : cols.masked_columns) + os << "'" << x << "' "; + os << "array_join_columns: "; + for (const auto & x : cols.array_join_columns) + os << "'" << x << "' "; + return os; +} + +} diff --git a/dbms/src/Interpreters/ColumnNamesContext.h b/dbms/src/Interpreters/ColumnNamesContext.h new file mode 100644 index 00000000000..63ad5a4e2e4 --- /dev/null +++ b/dbms/src/Interpreters/ColumnNamesContext.h @@ -0,0 +1,74 @@ +#pragma once + +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +/// Information about table and column names extracted from ASTSelectQuery block. Do not include info from subselects. +struct ColumnNamesContext +{ + struct JoinedTable + { + const ASTTableExpression * expr; + const ASTTableJoin * join; + + std::optional alias() const + { + String alias; + if (expr->database_and_table_name) + alias = expr->database_and_table_name->tryGetAlias(); + else if (expr->table_function) + alias = expr->table_function->tryGetAlias(); + else if (expr->subquery) + alias = expr->subquery->tryGetAlias(); + if (!alias.empty()) + return alias; + return {}; + } + + std::optional name() const + { + if (auto * node = expr->database_and_table_name.get()) + if (auto * identifier = typeid_cast(node)) + return identifier->name; + return {}; + } + + std::optional joinKind() const + { + if (join) + return join->kind; + return {}; + } + }; + + NameSet required_names; + NameSet table_aliases; + NameSet private_aliases; + NameSet column_aliases; + NameSet masked_columns; + NameSet public_columns; + NameSet array_join_columns; + std::vector tables; /// ordered list of visited tables in FROM section with joins + bool has_table_join = false; + bool has_array_join = false; + + bool addTableAliasIfAny(const IAST & ast); + bool addColumnAliasIfAny(const IAST & ast, bool is_public = false); + void addColumnIdentifier(const ASTIdentifier & node, bool is_public = false); + bool addArrayJoinAliasIfAny(const IAST & ast); + void addArrayJoinIdentifier(const ASTIdentifier & node); + + NameSet requiredColumns() const; +}; + +std::ostream & operator << (std::ostream & os, const ColumnNamesContext & cols); + +} diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index e488aa5d81b..92e26786e95 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -106,10 +106,10 @@ ExpressionAnalyzer::ExpressionAnalyzer( removeDuplicateColumns(source_columns); } - /// Delete the unnecessary from `source_columns` list. Create `unknown_required_source_columns`. Form `columns_added_by_join`. + /// Delete the unnecessary from `source_columns` list. Form `columns_added_by_join`. collectUsedColumns(); - /// external_tables, subqueries_for_sets for global subqueries. + /// external_tables, subqueries_for_sets for global subqueries.f /// Replaces global subqueries with the generated names of temporary tables that will be sent to remote servers. initGlobalSubqueriesAndExternalTables(); @@ -1001,6 +1001,15 @@ void ExpressionAnalyzer::getAggregateInfo(Names & key_names, AggregateDescriptio aggregates = aggregate_descriptions; } +/// db.table.column -> table.column / table.column -> column +static String cropDatabaseOrTableName(const String & name) +{ + size_t pos = name.find('.', 0); + if (pos != std::string::npos) + return name.substr(pos + 1, name.size() - pos - 1); + return name; +} + void ExpressionAnalyzer::collectUsedColumns() { /** Calculate which columns are required to execute the expression. @@ -1008,83 +1017,131 @@ void ExpressionAnalyzer::collectUsedColumns() * After execution, columns will only contain the list of columns needed to read from the table. */ - NameSet required; - NameSet ignored; + RequiredSourceColumnsVisitor::Data columns_context; + RequiredSourceColumnsVisitor(columns_context).visit(query); - NameSet available_columns; - for (const auto & column : source_columns) - available_columns.insert(column.name); + NameSet required = columns_context.requiredColumns(); - if (select_query && select_query->array_join_expression_list()) +#if 0 + std::cerr << "Query: " << query << std::endl; + std::cerr << "CTX: " << columns_context << std::endl; + std::cerr << "source_columns: "; + for (const auto & name : source_columns) + std::cerr << "'" << name.name << "' "; + std::cerr << "required: "; + for (const auto & name : required) + std::cerr << "'" << name << "' "; + std::cerr << std::endl; +#endif + + if (columns_context.has_table_join) { - ASTs & expressions = select_query->array_join_expression_list()->children; - for (size_t i = 0; i < expressions.size(); ++i) + const AnalyzedJoin & analyzed_join = analyzedJoin(); +#if 0 + std::cerr << "key_names_left: "; + for (const auto & name : analyzed_join.key_names_left) + std::cerr << "'" << name << "' "; + std::cerr << "key_names_right: "; + for (const auto & name : analyzed_join.key_names_right) + std::cerr << "'" << name << "' "; + std::cerr << "columns_from_joined_table: "; + for (const auto & column : analyzed_join.columns_from_joined_table) + std::cerr << "'" << column.name_and_type.name << '/' << column.original_name << "' "; + std::cerr << "available_joined_columns: "; + for (const auto & column : analyzed_join.available_joined_columns) + std::cerr << "'" << column.name_and_type.name << '/' << column.original_name << "' "; + std::cerr << std::endl; +#endif + NameSet avaliable_columns; + for (const auto & name : source_columns) + avaliable_columns.insert(name.name); + + NameSet right_keys; + for (const auto & right_key_name : analyzed_join.key_names_right) + right_keys.insert(right_key_name); + + /** You also need to ignore the identifiers of the columns that are obtained by JOIN. + * (Do not assume that they are required for reading from the "left" table). + */ + columns_added_by_join.clear(); + for (const auto & joined_column : analyzed_join.available_joined_columns) { - /// Ignore the top-level identifiers from the ARRAY JOIN section. - /// Then add them separately. - if (typeid_cast(expressions[i].get())) + auto & name = joined_column.name_and_type.name; + if (required.count(name) && !avaliable_columns.count(name)) { - ignored.insert(expressions[i]->getColumnName()); - } - else - { - /// Nothing needs to be ignored for expressions in ARRAY JOIN. - NameSet empty; - RequiredSourceColumnsVisitor::Data visitor_data{available_columns, required, empty, empty, empty}; - RequiredSourceColumnsVisitor(visitor_data).visit(expressions[i]); - } + columns_added_by_join.push_back(joined_column); + required.erase(name); - ignored.insert(expressions[i]->getAliasOrColumnName()); + /// Some columns from right join key may be used in query. This columns will be appended to block during join. + if (right_keys.count(name)) + columns_added_by_join_from_right_keys.insert(name); + } } + + /// @fix filter required columns according to misqualified names in JOIN ON + if (columns_context.has_table_join && + columns_context.tables.size() >= 2 && + columns_context.tables[1].join && + columns_context.tables[1].join->on_expression) + { + NameSet fixed_required; + + for (const auto & req_name : required) + { + bool collated = false; + String cropped_name = req_name; + static const constexpr size_t max_column_prefix = 2; + + for (size_t i = 0; i < max_column_prefix && !collated; ++i) + { + cropped_name = cropDatabaseOrTableName(cropped_name); + + if (avaliable_columns.count(cropped_name)) + { + fixed_required.insert(cropped_name); + collated = true; + break; + } + + for (const auto & joined_column : analyzed_join.available_joined_columns) + { + auto & name = joined_column.name_and_type.name; + + if (cropped_name == name) + { + columns_added_by_join.push_back(joined_column); + if (right_keys.count(name)) + columns_added_by_join_from_right_keys.insert(name); + collated = true; + break; + } + } + } + + if (!collated) + fixed_required.insert(req_name); + } + + required.swap(fixed_required); + } + + /// @note required_columns_from_joined_table is output + joined_block_actions = analyzed_join.createJoinedBlockActions( + columns_added_by_join, select_query, context, required_columns_from_joined_table); } - /** You also need to ignore the identifiers of the columns that are obtained by JOIN. - * (Do not assume that they are required for reading from the "left" table). - */ - NameSet available_joined_columns; - for (const auto & joined_column : analyzedJoin().available_joined_columns) - available_joined_columns.insert(joined_column.name_and_type.name); - - NameSet required_joined_columns; - - for (const auto & left_key_ast : syntax->analyzed_join.key_asts_left) + if (columns_context.has_array_join) { - NameSet empty; - RequiredSourceColumnsVisitor::Data columns_data{available_columns, required, ignored, empty, required_joined_columns}; - ASTPtr tmp = left_key_ast; - RequiredSourceColumnsVisitor(columns_data).visit(tmp); + /// Insert the columns required for the ARRAY JOIN calculation into the required columns list. + NameSet array_join_sources; + for (const auto & result_source : syntax->array_join_result_to_source) + array_join_sources.insert(result_source.second); + + for (const auto & column_name_type : source_columns) + if (array_join_sources.count(column_name_type.name)) + required.insert(column_name_type.name); } - RequiredSourceColumnsVisitor::Data columns_visitor_data{available_columns, required, ignored, - available_joined_columns, required_joined_columns}; - RequiredSourceColumnsVisitor(columns_visitor_data).visit(query); - - columns_added_by_join = analyzedJoin().available_joined_columns; - for (auto it = columns_added_by_join.begin(); it != columns_added_by_join.end();) - { - if (required_joined_columns.count(it->name_and_type.name)) - ++it; - else - columns_added_by_join.erase(it++); - } - - joined_block_actions = analyzedJoin().createJoinedBlockActions( - columns_added_by_join, select_query, context, required_columns_from_joined_table); - - /// Some columns from right join key may be used in query. This columns will be appended to block during join. - for (const auto & right_key_name : analyzedJoin().key_names_right) - if (required_joined_columns.count(right_key_name)) - columns_added_by_join_from_right_keys.insert(right_key_name); - - /// Insert the columns required for the ARRAY JOIN calculation into the required columns list. - NameSet array_join_sources; - for (const auto & result_source : syntax->array_join_result_to_source) - array_join_sources.insert(result_source.second); - - for (const auto & column_name_type : source_columns) - if (array_join_sources.count(column_name_type.name)) - required.insert(column_name_type.name); - /// You need to read at least one column to find the number of rows. if (select_query && required.empty()) required.insert(ExpressionActions::getSmallestColumn(source_columns)); @@ -1118,9 +1175,17 @@ void ExpressionAnalyzer::collectUsedColumns() } if (!unknown_required_source_columns.empty()) + { + std::stringstream ss; + ss << columns_context; + ss << "source_columns: "; + for (const auto & name : source_columns) + ss << "'" << name.name << "' "; + throw Exception("Unknown identifier: " + *unknown_required_source_columns.begin() - + (select_query && !select_query->tables ? ". Note that there is no tables (FROM clause) in your query" : ""), - ErrorCodes::UNKNOWN_IDENTIFIER); + + (select_query && !select_query->tables ? ". Note that there is no tables (FROM clause) in your query" : "") + + ", context: " + ss.str(), ErrorCodes::UNKNOWN_IDENTIFIER); + } } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index ba1e3a252d0..5c89df50863 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -232,8 +232,7 @@ private: const AnalyzedJoin & analyzedJoin() const { return syntax->analyzed_join; } /** Remove all unnecessary columns from the list of all available columns of the table (`columns`). - * At the same time, form a set of unknown columns (`unknown_required_source_columns`), - * as well as the columns added by JOIN (`columns_added_by_join`). + * At the same time, form a set of columns added by JOIN (`columns_added_by_join`). */ void collectUsedColumns(); diff --git a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp new file mode 100644 index 00000000000..b4ce3281d09 --- /dev/null +++ b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -0,0 +1,232 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +static std::vector extractNamesFromLambda(const ASTFunction & node) +{ + if (node.arguments->children.size() != 2) + throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + ASTFunction * lambda_args_tuple = typeid_cast(node.arguments->children[0].get()); + + if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") + throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH); + + std::vector names; + for (auto & child : lambda_args_tuple->arguments->children) + { + ASTIdentifier * identifier = typeid_cast(child.get()); + if (!identifier) + throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); + + names.push_back(identifier->name); + } + + return names; +} + +bool RequiredSourceColumnsMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child) +{ + if (typeid_cast(child.get())) + return false; + + /// Processed. Do not need children. + if (typeid_cast(node.get()) || + typeid_cast(node.get()) || + typeid_cast(node.get()) || + typeid_cast(node.get())) + return false; + + if (auto * f = typeid_cast(node.get())) + { + /// "indexHint" is a special function for index analysis. Everything that is inside it is not calculated. @sa KeyCondition + /// "lambda" visit children itself. + if (f->name == "indexHint" || f->name == "lambda") + return false; + } + + return true; +} + +std::vector RequiredSourceColumnsMatcher::visit(ASTPtr & ast, Data & data) +{ + /// results are columns + + if (auto * t = typeid_cast(ast.get())) + { + data.addColumnAliasIfAny(*ast); + visit(*t, ast, data); + return {}; + } + if (auto * t = typeid_cast(ast.get())) + { + data.addColumnAliasIfAny(*ast); + visit(*t, ast, data); + return {}; + } + + /// results are tables + + if (auto * t = typeid_cast(ast.get())) + { + visit(*t, ast, data); + return {}; + } + + if (auto * t = typeid_cast(ast.get())) + { + //data.addTableAliasIfAny(*ast); alias is attached to child + visit(*t, ast, data); + return {}; + } + if (auto * t = typeid_cast(ast.get())) + { + data.addTableAliasIfAny(*ast); + return visit(*t, ast, data); + } + if (auto * t = typeid_cast(ast.get())) + { + data.addTableAliasIfAny(*ast); + return {}; + } + + /// other + + if (auto * t = typeid_cast(ast.get())) + { + data.has_array_join = true; + return visit(*t, ast, data); + } + + return {}; +} + +std::vector RequiredSourceColumnsMatcher::visit(ASTSelectQuery & select, const ASTPtr &, Data & data) +{ + /// special case for top-level SELECT items: they are publics + for (auto & node : select.select_expression_list->children) + { + if (auto * identifier = typeid_cast(node.get())) + data.addColumnIdentifier(*identifier, true); + else + data.addColumnAliasIfAny(*node, true); + } + + std::vector out; + for (auto & node : select.children) + if (node != select.select_expression_list) + out.push_back(&node); + + /// revisit select_expression_list (with children) when all the aliases are set + out.push_back(&select.select_expression_list); + return out; +} + +void RequiredSourceColumnsMatcher::visit(const ASTIdentifier & node, const ASTPtr &, Data & data) +{ + if (node.name.empty()) + throw Exception("Expected not empty name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!data.private_aliases.count(node.name)) + data.addColumnIdentifier(node); +} + +void RequiredSourceColumnsMatcher::visit(const ASTFunction & node, const ASTPtr &, Data & data) +{ + /// Do not add formal parameters of the lambda expression + if (node.name == "lambda") + { + Names local_aliases; + for (const auto & name : extractNamesFromLambda(node)) + if (data.private_aliases.insert(name).second) + local_aliases.push_back(name); + + /// visit child with masked local aliases + visit(node.arguments->children[1], data); + + for (const auto & name : local_aliases) + data.private_aliases.erase(name); + } +} + +void RequiredSourceColumnsMatcher::visit(ASTTablesInSelectQueryElement & node, const ASTPtr &, Data & data) +{ + ASTTableExpression * expr = nullptr; + ASTTableJoin * join = nullptr; + + for (auto & child : node.children) + { + if (auto * e = typeid_cast(child.get())) + expr = e; + if (auto * j = typeid_cast(child.get())) + join = j; + } + + if (join) + data.has_table_join = true; + data.tables.emplace_back(ColumnNamesContext::JoinedTable{expr, join}); +} + +std::vector RequiredSourceColumnsMatcher::visit(ASTTableExpression & node, const ASTPtr &, Data & data) +{ + /// ASTIdentifiers here are tables. Do not visit them as generic ones. + if (node.database_and_table_name) + data.addTableAliasIfAny(*node.database_and_table_name); + + std::vector out; + if (node.table_function) + { + data.addTableAliasIfAny(*node.table_function); + out.push_back(&node.table_function); + } + + if (node.subquery) + { + data.addTableAliasIfAny(*node.subquery); + out.push_back(&node.subquery); + } + + return out; +} + +std::vector RequiredSourceColumnsMatcher::visit(const ASTArrayJoin & node, const ASTPtr &, Data & data) +{ + ASTPtr expression_list = node.expression_list; + if (!expression_list || expression_list->children.empty()) + throw Exception("Expected not empty expression_list", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + std::vector out; + + /// Tech debt. Ignore ARRAY JOIN top-level identifiers and aliases. There's its own logic for them. + for (auto & expr : expression_list->children) + { + data.addArrayJoinAliasIfAny(*expr); + + if (auto * identifier = typeid_cast(expr.get())) + { + data.addArrayJoinIdentifier(*identifier); + continue; + } + + out.push_back(&expr); + } + + return out; +} + +} diff --git a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h index 859326acbe8..3e107111e96 100644 --- a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h +++ b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h @@ -1,140 +1,45 @@ #pragma once -#include -#include -#include -#include -#include -#include -#include -#include -#include "InDepthNodeVisitor.h" +#include +#include namespace DB { namespace ErrorCodes { - extern const int TYPE_MISMATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -/** Get a set of necessary columns to read from the table. - * In this case, the columns specified in ignored_names are considered unnecessary. And the ignored_names parameter can be modified. - * The set of columns available_joined_columns are the columns available from JOIN, they are not needed for reading from the main table. - * Put in required_joined_columns the set of columns available from JOIN and needed. - */ +class ASTIdentifier; +class ASTFunction; +class ASTSelectQuery; +struct ASTTablesInSelectQueryElement; +struct ASTArrayJoin; +struct ASTTableExpression; + class RequiredSourceColumnsMatcher { public: - struct Data - { - const NameSet & available_columns; - NameSet & required_source_columns; - NameSet & ignored_names; - const NameSet & available_joined_columns; - NameSet & required_joined_columns; - }; + using Data = ColumnNamesContext; static constexpr const char * label = "RequiredSourceColumns"; - static bool needChildVisit(ASTPtr & node, const ASTPtr & child) - { - /// We will not go to the ARRAY JOIN section, because we need to look at the names of non-ARRAY-JOIN columns. - /// There, `collectUsedColumns` will send us separately. - if (typeid_cast(child.get()) || - typeid_cast(child.get()) || - typeid_cast(child.get()) || - typeid_cast(child.get())) - return false; - - /// Processed. Do not need children. - if (typeid_cast(node.get())) - return false; - - if (auto * f = typeid_cast(node.get())) - { - /// "indexHint" is a special function for index analysis. Everything that is inside it is not calculated. @sa KeyCondition - /// "lambda" visit children itself. - if (f->name == "indexHint" || f->name == "lambda") - return false; - } - - return true; - } - - /** Find all the identifiers in the query. - * We will use depth first search in AST. - * In this case - * - for lambda functions we will not take formal parameters; - * - do not go into subqueries (they have their own identifiers); - * - there is some exception for the ARRAY JOIN clause (it has a slightly different identifiers); - * - we put identifiers available from JOIN in required_joined_columns. - */ - static std::vector visit(ASTPtr & ast, Data & data) - { - if (auto * t = typeid_cast(ast.get())) - visit(*t, ast, data); - if (auto * t = typeid_cast(ast.get())) - visit(*t, ast, data); - return {}; - } + static bool needChildVisit(ASTPtr & node, const ASTPtr & child); + static std::vector visit(ASTPtr & ast, Data & data); private: - static void visit(const ASTIdentifier & node, const ASTPtr &, Data & data) - { - if (node.general() - && !data.ignored_names.count(node.name) - && !data.ignored_names.count(Nested::extractTableName(node.name))) - { - /// Read column from left table if has. - if (!data.available_joined_columns.count(node.name) || data.available_columns.count(node.name)) - data.required_source_columns.insert(node.name); - else - data.required_joined_columns.insert(node.name); - } - } - - static void visit(const ASTFunction & node, const ASTPtr &, Data & data) - { - NameSet & ignored_names = data.ignored_names; - - if (node.name == "lambda") - { - if (node.arguments->children.size() != 2) - throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - ASTFunction * lambda_args_tuple = typeid_cast(node.arguments->children.at(0).get()); - - if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") - throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH); - - /// You do not need to add formal parameters of the lambda expression in required_source_columns. - Names added_ignored; - for (auto & child : lambda_args_tuple->arguments->children) - { - ASTIdentifier * identifier = typeid_cast(child.get()); - if (!identifier) - throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH); - - String & name = identifier->name; - if (!ignored_names.count(name)) - { - ignored_names.insert(name); - added_ignored.push_back(name); - } - } - - /// @note It's a special case where we visit children inside the matcher, not in visitor. - visit(node.arguments->children[1], data); - - for (size_t i = 0; i < added_ignored.size(); ++i) - ignored_names.erase(added_ignored[i]); - } - } + static void visit(const ASTIdentifier & node, const ASTPtr &, Data & data); + static void visit(const ASTFunction & node, const ASTPtr &, Data & data); + static void visit(ASTTablesInSelectQueryElement & node, const ASTPtr &, Data & data); + static std::vector visit(ASTTableExpression & node, const ASTPtr &, Data & data); + static std::vector visit(const ASTArrayJoin & node, const ASTPtr &, Data & data); + static std::vector visit(ASTSelectQuery & select, const ASTPtr &, Data & data); }; -/// Get a set of necessary columns to read from the table. -using RequiredSourceColumnsVisitor = InDepthNodeVisitor; +/// Extracts all the information about columns and tables from ASTSelectQuery block into ColumnNamesContext object. +/// It doesn't use anithing but AST. It visits nodes from bottom to top except ASTFunction content to get aliases in right manner. +/// @note There's some ambiguousness with nested columns names that can't be solved without schema. +using RequiredSourceColumnsVisitor = InDepthNodeVisitor; } From 89966db3baf87d551d23b55b8fd166f9fa820cd7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 26 Dec 2018 18:01:26 +0300 Subject: [PATCH 146/230] Add multiple codecs to compressor --- dbms/programs/compressor/Compressor.cpp | 34 ++++++++++++++++++--- dbms/programs/compressor/README.md | 27 ++++++++++++++++ dbms/src/Compression/CompressionFactory.cpp | 12 ++++++++ dbms/src/Compression/CompressionFactory.h | 3 ++ 4 files changed, 72 insertions(+), 4 deletions(-) create mode 100644 dbms/programs/compressor/README.md diff --git a/dbms/programs/compressor/Compressor.cpp b/dbms/programs/compressor/Compressor.cpp index e021893dfdf..98a4cc095c5 100644 --- a/dbms/programs/compressor/Compressor.cpp +++ b/dbms/programs/compressor/Compressor.cpp @@ -17,6 +17,7 @@ namespace DB namespace ErrorCodes { extern const int TOO_LARGE_SIZE_COMPRESSED; + extern const int BAD_ARGUMENTS; } } @@ -62,7 +63,8 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) ("block-size,b", boost::program_options::value()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size") ("hc", "use LZ4HC instead of LZ4") ("zstd", "use ZSTD instead of LZ4") - ("level", boost::program_options::value(), "compression level") + ("codec", boost::program_options::value>()->multitoken(), "use codecs combination instead of LZ4") + ("level", boost::program_options::value>()->multitoken(), "compression levels for codecs specified via --codec") ("none", "use no compression instead of LZ4") ("stat", "print block statistics of compressed data") ; @@ -85,6 +87,12 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) bool stat_mode = options.count("stat"); bool use_none = options.count("none"); unsigned block_size = options["block-size"].as(); + std::vector codecs; + if (options.count("codec")) + codecs = options["codec"].as>(); + + if ((use_lz4hc || use_zstd || use_none) && !codecs.empty()) + throw DB::Exception("Wrong options, codec flags like --zstd and --codec options are mutually exclusive", DB::ErrorCodes::BAD_ARGUMENTS); std::string method_family = "LZ4"; @@ -95,11 +103,29 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) else if (use_none) method_family = "NONE"; - std::optional level; + std::vector levels; if (options.count("level")) - level = options["level"].as(); + levels = options["level"].as>(); + + DB::CompressionCodecPtr codec; + if (!codecs.empty()) + { + if (levels.size() > codecs.size()) + throw DB::Exception("Specified more levels than codecs", DB::ErrorCodes::BAD_ARGUMENTS); + + std::vector codec_names; + for (size_t i = 0; i < codecs.size(); ++i) + { + if (i < levels.size()) + codec_names.emplace_back(codecs[i], levels[i]); + else + codec_names.emplace_back(codecs[i], std::nullopt); + } + codec = DB::CompressionCodecFactory::instance().get(codec_names); + } + else + codec = DB::CompressionCodecFactory::instance().get(method_family, levels.empty() ? std::nullopt : std::optional(levels.back())); - DB::CompressionCodecPtr codec = DB::CompressionCodecFactory::instance().get(method_family, level); DB::ReadBufferFromFileDescriptor rb(STDIN_FILENO); DB::WriteBufferFromFileDescriptor wb(STDOUT_FILENO); diff --git a/dbms/programs/compressor/README.md b/dbms/programs/compressor/README.md new file mode 100644 index 00000000000..92dfe50cbc1 --- /dev/null +++ b/dbms/programs/compressor/README.md @@ -0,0 +1,27 @@ +## ClickHouse compressor + +Simple program for data compression and decompression. + +### Examples + +Compress data with LZ4: +``` +$ ./clickhouse-compressor < input_file > output_file +``` + +Decompress data from LZ4 format: +``` +$ ./clickhouse-compressor --decompress < input_file > output_file +``` + +Compress data with ZSTD at level 5: + +``` +$ ./clickhouse-compressor --codec ZSTD --level 5 < input_file > output_file +``` + +Compress data with ZSTD level 10, LZ4HC level 7 and LZ4. + +``` +$ ./clickhouse-compressor --codec ZSTD --level 5 --codec LZ4HC --level 7 --codec LZ4 < input_file > output_file +``` diff --git a/dbms/src/Compression/CompressionFactory.cpp b/dbms/src/Compression/CompressionFactory.cpp index c70917e0cb1..05576f96f55 100644 --- a/dbms/src/Compression/CompressionFactory.cpp +++ b/dbms/src/Compression/CompressionFactory.cpp @@ -44,6 +44,18 @@ CompressionCodecPtr CompressionCodecFactory::get(const String & family_name, std } } +CompressionCodecPtr CompressionCodecFactory::get(const std::vector & codecs) const +{ + Codecs result; + for (const auto & [codec_name, level] : codecs) + result.push_back(get(codec_name, level)); + + if (result.size() == 1) + return result.back(); + + return std::make_shared(result); +} + CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast) const { if (const auto * func = typeid_cast(ast.get())) diff --git a/dbms/src/Compression/CompressionFactory.h b/dbms/src/Compression/CompressionFactory.h index e1019cb983a..9c18da2cecb 100644 --- a/dbms/src/Compression/CompressionFactory.h +++ b/dbms/src/Compression/CompressionFactory.h @@ -16,6 +16,8 @@ class ICompressionCodec; using CompressionCodecPtr = std::shared_ptr; +using CodecNameWithLevel = std::pair>; + class IAST; using ASTPtr = std::shared_ptr; @@ -43,6 +45,7 @@ public: /// For backward compatibility with config settings CompressionCodecPtr get(const String & family_name, std::optional level) const; + CompressionCodecPtr get(const std::vector & codecs) const; /// Register codec with parameters void registerCompressionCodec(const String & family_name, std::optional byte_code, Creator creator); From 371ec0d5766a568c971a48776e1d8ffdf7bd3c37 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 26 Dec 2018 18:40:47 +0300 Subject: [PATCH 147/230] fix gcc build --- dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp index b4ce3281d09..7d93da95e8d 100644 --- a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -99,7 +99,7 @@ std::vector RequiredSourceColumnsMatcher::visit(ASTPtr & ast, Data & d data.addTableAliasIfAny(*ast); return visit(*t, ast, data); } - if (auto * t = typeid_cast(ast.get())) + if (typeid_cast(ast.get())) { data.addTableAliasIfAny(*ast); return {}; From 31f82940e106f49ac0e7c950ce3c28094cdf8d9d Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 26 Dec 2018 19:18:21 +0300 Subject: [PATCH 148/230] Update Compressor.cpp --- dbms/programs/compressor/Compressor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/compressor/Compressor.cpp b/dbms/programs/compressor/Compressor.cpp index 98a4cc095c5..77fcdaaf5d8 100644 --- a/dbms/programs/compressor/Compressor.cpp +++ b/dbms/programs/compressor/Compressor.cpp @@ -1,5 +1,5 @@ #include - +#include #include #include From 9342700ce1f56f5521d2c216ec605a6585d35660 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 26 Dec 2018 20:03:29 +0300 Subject: [PATCH 149/230] Move processing pool to ternary logic --- dbms/src/Interpreters/DNSCacheUpdater.cpp | 8 ++++---- dbms/src/Interpreters/DNSCacheUpdater.h | 3 ++- .../MergeTree/BackgroundProcessingPool.cpp | 11 +++++++---- .../MergeTree/BackgroundProcessingPool.h | 10 +++++++++- dbms/src/Storages/StorageMergeTree.cpp | 15 +++++++++------ dbms/src/Storages/StorageMergeTree.h | 2 +- .../src/Storages/StorageReplicatedMergeTree.cpp | 17 ++++------------- dbms/src/Storages/StorageReplicatedMergeTree.h | 2 +- 8 files changed, 37 insertions(+), 31 deletions(-) diff --git a/dbms/src/Interpreters/DNSCacheUpdater.cpp b/dbms/src/Interpreters/DNSCacheUpdater.cpp index a4cc8a19dad..2a2d772ffb3 100644 --- a/dbms/src/Interpreters/DNSCacheUpdater.cpp +++ b/dbms/src/Interpreters/DNSCacheUpdater.cpp @@ -61,7 +61,7 @@ DNSCacheUpdater::DNSCacheUpdater(Context & context_) task_handle = pool.addTask([this] () { return run(); }); } -bool DNSCacheUpdater::run() +BackgroundProcessingPoolTaskResult DNSCacheUpdater::run() { /// TODO: Ensusre that we get global counter (not thread local) auto num_current_network_exceptions = ProfileEvents::global_counters[ProfileEvents::NetworkErrors].load(std::memory_order_relaxed); @@ -79,20 +79,20 @@ bool DNSCacheUpdater::run() last_num_network_erros = num_current_network_exceptions; last_update_time = time(nullptr); - return true; + return BackgroundProcessingPoolTaskResult::SUCCESS; } catch (...) { /// Do not increment ProfileEvents::NetworkErrors twice if (isNetworkError()) - return false; + return BackgroundProcessingPoolTaskResult::ERROR; throw; } } /// According to BackgroundProcessingPool logic, if task has done work, it could be executed again immediately. - return false; + return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; } DNSCacheUpdater::~DNSCacheUpdater() diff --git a/dbms/src/Interpreters/DNSCacheUpdater.h b/dbms/src/Interpreters/DNSCacheUpdater.h index ad57f37b5f6..885bcc143e3 100644 --- a/dbms/src/Interpreters/DNSCacheUpdater.h +++ b/dbms/src/Interpreters/DNSCacheUpdater.h @@ -11,6 +11,7 @@ namespace DB class Context; class BackgroundProcessingPool; class BackgroundProcessingPoolTaskInfo; +enum class BackgroundProcessingPoolTaskResult; /// Add a task to BackgroundProcessingPool that watch for ProfileEvents::NetworkErrors and updates DNS cache if it has increased @@ -25,7 +26,7 @@ public: static bool incrementNetworkErrorEventsIfNeeded(); private: - bool run(); + BackgroundProcessingPoolTaskResult run(); Context & context; BackgroundProcessingPool & pool; diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp index 6d81d714358..c0911ac4d5e 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -25,6 +25,7 @@ namespace DB static constexpr double thread_sleep_seconds = 10; static constexpr double thread_sleep_seconds_random_part = 1.0; +static constexpr double thread_sleep_seconds_if_nothing_to_do = 0.1; /// For exponential backoff. static constexpr double task_sleep_seconds_when_no_work_min = 10; @@ -146,7 +147,7 @@ void BackgroundProcessingPool::threadFunction() while (!shutdown) { - bool done_work = false; + TaskResult task_result = TaskResult::ERROR; TaskHandle task; try @@ -198,7 +199,7 @@ void BackgroundProcessingPool::threadFunction() { CurrentMetrics::Increment metric_increment{CurrentMetrics::BackgroundPoolTask}; - done_work = task->function(); + task_result = task->function(); } } catch (...) @@ -216,7 +217,7 @@ void BackgroundProcessingPool::threadFunction() if (task->removed) continue; - if (done_work) + if (task_result == TaskResult::SUCCESS) task->count_no_work_done = 0; else ++task->count_no_work_done; @@ -225,11 +226,13 @@ void BackgroundProcessingPool::threadFunction() /// If not, add delay before next run. Poco::Timestamp next_time_to_execute; /// current time - if (!done_work) + if (task_result == TaskResult::ERROR) next_time_to_execute += 1000000 * (std::min( task_sleep_seconds_when_no_work_max, task_sleep_seconds_when_no_work_min * std::pow(task_sleep_seconds_when_no_work_multiplier, task->count_no_work_done)) + std::uniform_real_distribution(0, task_sleep_seconds_when_no_work_random_part)(rng)); + else if (task_result == TaskResult::NOTHING_TO_DO) + next_time_to_execute += 1000000 * thread_sleep_seconds_if_nothing_to_do; tasks.erase(task->iterator); task->iterator = tasks.emplace(next_time_to_execute, task); diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h index 5e3c29e1af4..4eb5d4cce56 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h @@ -21,6 +21,12 @@ namespace DB class BackgroundProcessingPool; class BackgroundProcessingPoolTaskInfo; +enum class BackgroundProcessingPoolTaskResult +{ + SUCCESS, + ERROR, + NOTHING_TO_DO, +}; /** Using a fixed number of threads, perform an arbitrary number of tasks in an infinite loop. * In this case, one task can run simultaneously from different threads. * Designed for tasks that perform continuous background work (for example, merge). @@ -31,11 +37,13 @@ class BackgroundProcessingPool { public: /// Returns true, if some useful work was done. In that case, thread will not sleep before next run of this task. - using Task = std::function; + using TaskResult = BackgroundProcessingPoolTaskResult; + using Task = std::function; using TaskInfo = BackgroundProcessingPoolTaskInfo; using TaskHandle = std::shared_ptr; + BackgroundProcessingPool(int size_); size_t getNumberOfThreads() const diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 6ee1e7ca9c9..59e7d7b7d2c 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -581,13 +581,13 @@ bool StorageMergeTree::tryMutatePart() } -bool StorageMergeTree::backgroundTask() +BackgroundProcessingPoolTaskResult StorageMergeTree::backgroundTask() { if (shutdown_called) - return false; + return BackgroundProcessingPoolTaskResult::ERROR; if (merger_mutator.actions_blocker.isCancelled()) - return false; + return BackgroundProcessingPoolTaskResult::ERROR; try { @@ -601,16 +601,19 @@ bool StorageMergeTree::backgroundTask() ///TODO: read deduplicate option from table config if (merge(false /*aggressive*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/)) - return true; + return BackgroundProcessingPoolTaskResult::SUCCESS; - return tryMutatePart(); + if (tryMutatePart()) + return BackgroundProcessingPoolTaskResult::SUCCESS; + else + return BackgroundProcessingPoolTaskResult::ERROR; } catch (Exception & e) { if (e.code() == ErrorCodes::ABORTED) { LOG_INFO(log, e.message()); - return false; + return BackgroundProcessingPoolTaskResult::ERROR; } throw; diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index c80c06a9758..c0a2b8b5463 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -137,7 +137,7 @@ private: /// Try and find a single part to mutate and mutate it. If some part was successfully mutated, return true. bool tryMutatePart(); - bool backgroundTask(); + BackgroundProcessingPoolTaskResult backgroundTask(); Int64 getCurrentMutationVersion( const MergeTreeData::DataPartPtr & part, diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index ae1e563d33f..a31a4340205 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2053,13 +2053,13 @@ void StorageReplicatedMergeTree::mutationsUpdatingTask() } -bool StorageReplicatedMergeTree::queueTask() +BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() { /// If replication queue is stopped exit immediately as we successfully executed the task if (queue.actions_blocker.isCancelled()) { std::this_thread::sleep_for(std::chrono::milliseconds(5)); - return true; + return BackgroundProcessingPoolTaskResult::SUCCESS; } /// This object will mark the element of the queue as running. @@ -2077,16 +2077,7 @@ bool StorageReplicatedMergeTree::queueTask() LogEntryPtr & entry = selected.first; if (!entry) - { - /// Nothing to do, we can sleep for some time, just not to - /// abuse background pool scheduling policy - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - - /// If we return false, than background pool for this task - /// will accumulate exponential backoff and after empty replication queue - /// we will sleep for a long time - return true; - } + return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; time_t prev_attempt_time = entry->last_attempt_time; @@ -2134,7 +2125,7 @@ bool StorageReplicatedMergeTree::queueTask() bool need_sleep = !res && (entry->last_attempt_time - prev_attempt_time < 10); /// If there was no exception, you do not need to sleep. - return !need_sleep; + return need_sleep ? BackgroundProcessingPoolTaskResult::ERROR : BackgroundProcessingPoolTaskResult::SUCCESS; } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 753be7f088b..37566ddd1cf 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -427,7 +427,7 @@ private: /** Performs actions from the queue. */ - bool queueTask(); + BackgroundProcessingPoolTaskResult queueTask(); /// Postcondition: /// either leader_election is fully initialized (node in ZK is created and the watching thread is launched) From 601a6af0ec7dd7efcbcaf5a8daa842c65ad30039 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 26 Dec 2018 20:27:25 +0300 Subject: [PATCH 150/230] fix ZooKeeperNodeCache becoming unusable after SessionExpired event #2947 #3891 Previously after a SessionExpired event the context->zookeeper field was reinitialized with the old expired ZooKeeper instance. This led to inability to get new paths. Better not cache the ZooKeeper instance and get it at the start of each request. --- dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp | 10 ++++------ dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h | 2 +- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp index ff1e4fb5e53..71d45ca3cb7 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp @@ -20,23 +20,21 @@ ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, Even ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, Coordination::WatchCallback caller_watch_callback) { - zkutil::ZooKeeperPtr zookeeper; std::unordered_set invalidated_paths; { std::lock_guard lock(context->mutex); - if (!context->zookeeper) + if (context->all_paths_invalidated) { /// Possibly, there was a previous session and it has expired. Clear the cache. path_to_cached_znode.clear(); - - context->zookeeper = get_zookeeper(); + context->all_paths_invalidated = false; } - zookeeper = context->zookeeper; invalidated_paths.swap(context->invalidated_paths); } + zkutil::ZooKeeperPtr zookeeper = get_zookeeper(); if (!zookeeper) throw DB::Exception("Could not get znode: `" + path + "'. ZooKeeper not configured.", DB::ErrorCodes::NO_ZOOKEEPER); @@ -65,8 +63,8 @@ ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, Coor changed = owned_context->invalidated_paths.emplace(response.path).second; else if (response.state == Coordination::EXPIRED_SESSION) { - owned_context->zookeeper = nullptr; owned_context->invalidated_paths.clear(); + owned_context->all_paths_invalidated = true; changed = true; } } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h index d47cbcb36fe..73e7753cdcf 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.h @@ -53,8 +53,8 @@ private: struct Context { std::mutex mutex; - zkutil::ZooKeeperPtr zookeeper; std::unordered_set invalidated_paths; + bool all_paths_invalidated = false; }; std::shared_ptr context; From 0812f794216a96adf60dc1fea6debd183dc64107 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 26 Dec 2018 21:56:21 +0300 Subject: [PATCH 151/230] remove unused code --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 55 -------------------- dbms/src/Interpreters/ExpressionAnalyzer.h | 4 +- 2 files changed, 1 insertion(+), 58 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index e488aa5d81b..aa19a314634 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -337,56 +337,6 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & } } -bool ExpressionAnalyzer::isThereArrayJoin(const ASTPtr & ast) -{ - if (typeid_cast(ast.get())) - { - return false; - } - else if (ASTFunction * node = typeid_cast(ast.get())) - { - if (node->name == "arrayJoin") - { - return true; - } - if (functionIsInOrGlobalInOperator(node->name)) - { - return isThereArrayJoin(node->arguments->children.at(0)); - } - if (node->name == "indexHint") - { - return false; - } - if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name)) - { - return false; - } - for (auto & child : node->arguments->children) - { - if (isThereArrayJoin(child)) - { - return true; - } - } - return false; - } - else if (typeid_cast(ast.get())) - { - return false; - } - else - { - for (auto & child : ast->children) - { - if (isThereArrayJoin(child)) - { - return true; - } - } - return false; - } -} - void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts) { @@ -1124,9 +1074,4 @@ void ExpressionAnalyzer::collectUsedColumns() } -Names ExpressionAnalyzer::getRequiredSourceColumns() const -{ - return source_columns.getNames(); -} - } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index ba1e3a252d0..45944c48c85 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -150,7 +150,7 @@ public: /** Get a set of columns that are enough to read from the table to evaluate the expression. * Columns added from another table by JOIN are not counted. */ - Names getRequiredSourceColumns() const; + Names getRequiredSourceColumns() const { return source_columns.getNames(); } /** These methods allow you to build a chain of transformations over a block, that receives values in the desired sections of the query. * @@ -244,8 +244,6 @@ private: void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const; - bool isThereArrayJoin(const ASTPtr & ast); - /// If ast is ASTSelectQuery with JOIN, add actions for JOIN key columns. void getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions); From 380f27a12f3610bfc31dc66ddb975e0425c3a0cf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Dec 2018 23:32:11 +0300 Subject: [PATCH 152/230] Empty commit From 67dbfd8e81b72f74337af1fbbc61245833e68474 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 26 Dec 2018 23:33:29 +0300 Subject: [PATCH 153/230] Update ExpressionAnalyzer.cpp --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 92e26786e95..391fe1c8b84 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -109,7 +109,7 @@ ExpressionAnalyzer::ExpressionAnalyzer( /// Delete the unnecessary from `source_columns` list. Form `columns_added_by_join`. collectUsedColumns(); - /// external_tables, subqueries_for_sets for global subqueries.f + /// external_tables, subqueries_for_sets for global subqueries. /// Replaces global subqueries with the generated names of temporary tables that will be sent to remote servers. initGlobalSubqueriesAndExternalTables(); From 5c4ebfa2e175bc948815aeeb3d6fddf405647d58 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 26 Dec 2018 23:35:06 +0300 Subject: [PATCH 154/230] Update RequiredSourceColumnsVisitor.h --- dbms/src/Interpreters/RequiredSourceColumnsVisitor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h index 3e107111e96..4eb84b21366 100644 --- a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h +++ b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h @@ -38,7 +38,7 @@ private: }; /// Extracts all the information about columns and tables from ASTSelectQuery block into ColumnNamesContext object. -/// It doesn't use anithing but AST. It visits nodes from bottom to top except ASTFunction content to get aliases in right manner. +/// It doesn't use anything but AST. It visits nodes from bottom to top except ASTFunction content to get aliases in right manner. /// @note There's some ambiguousness with nested columns names that can't be solved without schema. using RequiredSourceColumnsVisitor = InDepthNodeVisitor; From 9e32de053d954e37113630b69d5387182a4683d6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 26 Dec 2018 23:53:39 +0300 Subject: [PATCH 155/230] Update Compressor.cpp --- dbms/programs/compressor/Compressor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/compressor/Compressor.cpp b/dbms/programs/compressor/Compressor.cpp index 77fcdaaf5d8..efa45b86a88 100644 --- a/dbms/programs/compressor/Compressor.cpp +++ b/dbms/programs/compressor/Compressor.cpp @@ -124,7 +124,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) codec = DB::CompressionCodecFactory::instance().get(codec_names); } else - codec = DB::CompressionCodecFactory::instance().get(method_family, levels.empty() ? std::nullopt : std::optional(levels.back())); + codec = DB::CompressionCodecFactory::instance().get(method_family, levels.empty() ? std::nullopt : std::optional(levels.back())); DB::ReadBufferFromFileDescriptor rb(STDIN_FILENO); From 5570ddada5cec2900e6dbf50af47a8af5b1575cd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 00:02:39 +0300 Subject: [PATCH 156/230] Fixed error in internal implementation of quantileTDigest [#CLICKHOUSE-2] --- dbms/src/AggregateFunctions/CMakeLists.txt | 4 ++++ dbms/src/AggregateFunctions/QuantileTDigest.h | 4 ++++ .../AggregateFunctions/tests/CMakeLists.txt | 2 ++ .../tests/quantile-t-digest.cpp | 22 +++++++++++++++++++ 4 files changed, 32 insertions(+) create mode 100644 dbms/src/AggregateFunctions/tests/CMakeLists.txt create mode 100644 dbms/src/AggregateFunctions/tests/quantile-t-digest.cpp diff --git a/dbms/src/AggregateFunctions/CMakeLists.txt b/dbms/src/AggregateFunctions/CMakeLists.txt index 56cc66d3913..77748500d5e 100644 --- a/dbms/src/AggregateFunctions/CMakeLists.txt +++ b/dbms/src/AggregateFunctions/CMakeLists.txt @@ -22,3 +22,7 @@ list(REMOVE_ITEM clickhouse_aggregate_functions_headers add_library(clickhouse_aggregate_functions ${LINK_MODE} ${clickhouse_aggregate_functions_sources}) target_link_libraries(clickhouse_aggregate_functions PRIVATE dbms) target_include_directories (clickhouse_aggregate_functions BEFORE PRIVATE ${COMMON_INCLUDE_DIR}) + +if (ENABLE_TESTS) + add_subdirectory (tests) +endif () diff --git a/dbms/src/AggregateFunctions/QuantileTDigest.h b/dbms/src/AggregateFunctions/QuantileTDigest.h index 0359679f442..ca7d4f2fb1a 100644 --- a/dbms/src/AggregateFunctions/QuantileTDigest.h +++ b/dbms/src/AggregateFunctions/QuantileTDigest.h @@ -225,6 +225,10 @@ public: summary.resize(size); buf.read(reinterpret_cast(summary.data()), size * sizeof(summary[0])); + + count = 0; + for (const auto & c : summary) + count += c.count; } /** Calculates the quantile q [0, 1] based on the digest. diff --git a/dbms/src/AggregateFunctions/tests/CMakeLists.txt b/dbms/src/AggregateFunctions/tests/CMakeLists.txt new file mode 100644 index 00000000000..c19a115981b --- /dev/null +++ b/dbms/src/AggregateFunctions/tests/CMakeLists.txt @@ -0,0 +1,2 @@ +add_executable (quantile-t-digest quantile-t-digest.cpp) +target_link_libraries (quantile-t-digest PRIVATE dbms clickhouse_aggregate_functions) diff --git a/dbms/src/AggregateFunctions/tests/quantile-t-digest.cpp b/dbms/src/AggregateFunctions/tests/quantile-t-digest.cpp new file mode 100644 index 00000000000..b4e58e6203c --- /dev/null +++ b/dbms/src/AggregateFunctions/tests/quantile-t-digest.cpp @@ -0,0 +1,22 @@ +#include +#include +#include + +int main(int, char **) +{ + using namespace DB; + + QuantileTDigest tdigest; + tdigest.add(1); + tdigest.add(2); + tdigest.add(3); + std::cout << tdigest.get(0.5) << "\n"; + WriteBufferFromOwnString wb; + tdigest.serialize(wb); + QuantileTDigest other; + ReadBufferFromString rb{wb.str()}; + other.deserialize(rb); + std::cout << other.get(0.5) << "\n"; + + return 0; +} From 1498ec11dc8d2eb694cfd35bc43b783bd86d5ee3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 00:12:56 +0300 Subject: [PATCH 157/230] Fixed error #3924 --- libs/libmysqlxx/cmake/find_mysqlclient.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/libmysqlxx/cmake/find_mysqlclient.cmake b/libs/libmysqlxx/cmake/find_mysqlclient.cmake index 85a6275f1ff..71cb2bfeb1d 100644 --- a/libs/libmysqlxx/cmake/find_mysqlclient.cmake +++ b/libs/libmysqlxx/cmake/find_mysqlclient.cmake @@ -5,7 +5,7 @@ else () option (USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" OFF) endif () -if (USE_INTERNAL_MYSQL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README") +if (USE_INTERNAL_MYSQL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README.md") message (WARNING "submodule contrib/mariadb-connector-c is missing. to fix try run: \n git submodule update --init --recursive") set (USE_INTERNAL_MYSQL_LIBRARY 0) endif () From 0d03fa8573dad1f550d3b766fb77351e02ef69b8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 00:12:56 +0300 Subject: [PATCH 158/230] Fixed error #3924 --- libs/libmysqlxx/cmake/find_mysqlclient.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/libmysqlxx/cmake/find_mysqlclient.cmake b/libs/libmysqlxx/cmake/find_mysqlclient.cmake index 85a6275f1ff..71cb2bfeb1d 100644 --- a/libs/libmysqlxx/cmake/find_mysqlclient.cmake +++ b/libs/libmysqlxx/cmake/find_mysqlclient.cmake @@ -5,7 +5,7 @@ else () option (USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" OFF) endif () -if (USE_INTERNAL_MYSQL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README") +if (USE_INTERNAL_MYSQL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README.md") message (WARNING "submodule contrib/mariadb-connector-c is missing. to fix try run: \n git submodule update --init --recursive") set (USE_INTERNAL_MYSQL_LIBRARY 0) endif () From 10362ab7e1d51d4ab31f75a9db95747656bdc271 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 01:56:54 +0300 Subject: [PATCH 159/230] Fixed build #3926 --- dbms/src/Functions/FunctionUnaryArithmetic.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/FunctionUnaryArithmetic.h b/dbms/src/Functions/FunctionUnaryArithmetic.h index ceb5d88a923..8cc02420a26 100644 --- a/dbms/src/Functions/FunctionUnaryArithmetic.h +++ b/dbms/src/Functions/FunctionUnaryArithmetic.h @@ -37,14 +37,14 @@ struct UnaryOperationImpl using ArrayA = typename ColVecA::Container; using ArrayC = typename ColVecC::Container; - static void NO_INLINE __attribute__((no_sanitize("undefined"))) vector(const ArrayA & a, ArrayC & c) + static void NO_INLINE vector(const ArrayA & a, ArrayC & c) { size_t size = a.size(); for (size_t i = 0; i < size; ++i) c[i] = Op::apply(a[i]); } - static void __attribute__((no_sanitize("undefined"))) constant(A a, ResultType & c) + static void constant(A a, ResultType & c) { c = Op::apply(a); } From 37a2d51e0ca7a53e50a733d763724a6012aa41fc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 02:17:04 +0300 Subject: [PATCH 160/230] Added attribute #3926 --- dbms/src/Core/Defines.h | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index 6fd41120299..9f0ae6b1369 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -104,3 +104,13 @@ #elif defined(__SANITIZE_THREAD__) #define THREAD_SANITIZER 1 #endif + +/// Explicitly allow undefined behaviour for certain functions. Use it as a function attribute. +/// It is useful in case when compiler cannot see (and exploit) it, but UBSan can. +/// Example: multiplication of signed integers with possibility of overflow when both sides are from user input. +#if defined(__clang__) + #define NO_UNDEFINED_SANITIZER __attribute__((__no_sanitize__("undefined"))) +#else + /// It does not work in GCC. GCC 7 cannot recognize this attribute and GCC 8 simply ignores it. + #define NO_UNDEFINED_SANITIZER +#endif From 2b5843873768dec0049baa31f10486dea50c350f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 02:19:29 +0300 Subject: [PATCH 161/230] Added warning suppression for 3rd party library #3926 --- contrib/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 989761bfb67..2622b74ceb7 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -5,7 +5,7 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-implicit-fallthrough -Wno-class-memaccess -Wno-sign-compare -std=c++1z") elseif (CMAKE_CXX_COMPILER_ID STREQUAL "Clang") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-function -Wno-unused-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-format -Wno-parentheses-equality") - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-format -std=c++1z") + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-format -Wno-inconsistent-missing-override -std=c++1z") endif () if (USE_INTERNAL_BOOST_LIBRARY) From 732f60c8be9c919e52e9daa82c8b9d1e8caad01f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 02:55:09 +0300 Subject: [PATCH 162/230] Allowed some UB under UBSan #3926 --- dbms/src/Core/Defines.h | 4 ++-- dbms/src/Functions/FunctionsRound.h | 2 -- dbms/src/Functions/abs.cpp | 2 +- dbms/src/Functions/bitRotateLeft.cpp | 2 +- dbms/src/Functions/bitRotateRight.cpp | 2 +- dbms/src/Functions/bitShiftLeft.cpp | 2 +- dbms/src/Functions/bitShiftRight.cpp | 2 +- dbms/src/Functions/divide.cpp | 2 +- dbms/src/Functions/minus.cpp | 2 +- dbms/src/Functions/multiply.cpp | 2 +- dbms/src/Functions/negate.cpp | 2 +- dbms/src/Functions/plus.cpp | 2 +- libs/libcommon/include/common/intExp.h | 2 +- 13 files changed, 13 insertions(+), 15 deletions(-) diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index 9f0ae6b1369..ee9ff1cbf79 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -109,8 +109,8 @@ /// It is useful in case when compiler cannot see (and exploit) it, but UBSan can. /// Example: multiplication of signed integers with possibility of overflow when both sides are from user input. #if defined(__clang__) - #define NO_UNDEFINED_SANITIZER __attribute__((__no_sanitize__("undefined"))) + #define NO_SANITIZE_UNDEFINED __attribute__((__no_sanitize__("undefined"))) #else /// It does not work in GCC. GCC 7 cannot recognize this attribute and GCC 8 simply ignores it. - #define NO_UNDEFINED_SANITIZER + #define NO_SANITIZE_UNDEFINED #endif diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 862e383c51c..fbdd821f669 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -1,7 +1,5 @@ #pragma once -#include -#include #include #include #include diff --git a/dbms/src/Functions/abs.cpp b/dbms/src/Functions/abs.cpp index 872c8404176..4a31e0eba5d 100644 --- a/dbms/src/Functions/abs.cpp +++ b/dbms/src/Functions/abs.cpp @@ -11,7 +11,7 @@ struct AbsImpl { using ResultType = std::conditional_t, A, typename NumberTraits::ResultOfAbs::Type>; - static inline ResultType apply(A a) + static inline NO_SANITIZE_UNDEFINED ResultType apply(A a) { if constexpr (IsDecimalNumber) return a < 0 ? A(-a) : a; diff --git a/dbms/src/Functions/bitRotateLeft.cpp b/dbms/src/Functions/bitRotateLeft.cpp index da8c7798544..65b7c90a68a 100644 --- a/dbms/src/Functions/bitRotateLeft.cpp +++ b/dbms/src/Functions/bitRotateLeft.cpp @@ -10,7 +10,7 @@ struct BitRotateLeftImpl using ResultType = typename NumberTraits::ResultOfBit::Type; template - static inline Result apply(A a, B b) + static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b) { return (static_cast(a) << static_cast(b)) | (static_cast(a) >> ((sizeof(Result) * 8) - static_cast(b))); diff --git a/dbms/src/Functions/bitRotateRight.cpp b/dbms/src/Functions/bitRotateRight.cpp index 7aa1b08f78e..8d39afc9b4f 100644 --- a/dbms/src/Functions/bitRotateRight.cpp +++ b/dbms/src/Functions/bitRotateRight.cpp @@ -10,7 +10,7 @@ struct BitRotateRightImpl using ResultType = typename NumberTraits::ResultOfBit::Type; template - static inline Result apply(A a, B b) + static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b) { return (static_cast(a) >> static_cast(b)) | (static_cast(a) << ((sizeof(Result) * 8) - static_cast(b))); diff --git a/dbms/src/Functions/bitShiftLeft.cpp b/dbms/src/Functions/bitShiftLeft.cpp index 5d0be9364cd..91cd7781fa5 100644 --- a/dbms/src/Functions/bitShiftLeft.cpp +++ b/dbms/src/Functions/bitShiftLeft.cpp @@ -10,7 +10,7 @@ struct BitShiftLeftImpl using ResultType = typename NumberTraits::ResultOfBit::Type; template - static inline Result apply(A a, B b) + static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b) { return static_cast(a) << static_cast(b); } diff --git a/dbms/src/Functions/bitShiftRight.cpp b/dbms/src/Functions/bitShiftRight.cpp index 503fb7192ab..18b1f1536ba 100644 --- a/dbms/src/Functions/bitShiftRight.cpp +++ b/dbms/src/Functions/bitShiftRight.cpp @@ -10,7 +10,7 @@ struct BitShiftRightImpl using ResultType = typename NumberTraits::ResultOfBit::Type; template - static inline Result apply(A a, B b) + static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b) { return static_cast(a) >> static_cast(b); } diff --git a/dbms/src/Functions/divide.cpp b/dbms/src/Functions/divide.cpp index 2e77c565066..3362ab8547d 100644 --- a/dbms/src/Functions/divide.cpp +++ b/dbms/src/Functions/divide.cpp @@ -11,7 +11,7 @@ struct DivideFloatingImpl static const constexpr bool allow_decimal = true; template - static inline Result apply(A a, B b) + static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b) { return static_cast(a) / b; } diff --git a/dbms/src/Functions/minus.cpp b/dbms/src/Functions/minus.cpp index d5d6ca0f917..dad7c14e4ae 100644 --- a/dbms/src/Functions/minus.cpp +++ b/dbms/src/Functions/minus.cpp @@ -12,7 +12,7 @@ struct MinusImpl static const constexpr bool allow_decimal = true; template - static inline Result apply(A a, B b) + static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b) { return static_cast(a) - b; } diff --git a/dbms/src/Functions/multiply.cpp b/dbms/src/Functions/multiply.cpp index 174cb81334b..1a39445f09f 100644 --- a/dbms/src/Functions/multiply.cpp +++ b/dbms/src/Functions/multiply.cpp @@ -12,7 +12,7 @@ struct MultiplyImpl static const constexpr bool allow_decimal = true; template - static inline Result apply(A a, B b) + static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b) { return static_cast(a) * b; } diff --git a/dbms/src/Functions/negate.cpp b/dbms/src/Functions/negate.cpp index d2804d8d9b6..6c6214cd69e 100644 --- a/dbms/src/Functions/negate.cpp +++ b/dbms/src/Functions/negate.cpp @@ -10,7 +10,7 @@ struct NegateImpl { using ResultType = std::conditional_t, A, typename NumberTraits::ResultOfNegate::Type>; - static inline ResultType apply(A a) + static inline NO_SANITIZE_UNDEFINED ResultType apply(A a) { return -static_cast(a); } diff --git a/dbms/src/Functions/plus.cpp b/dbms/src/Functions/plus.cpp index 85fe8a3684b..de3f6d0c789 100644 --- a/dbms/src/Functions/plus.cpp +++ b/dbms/src/Functions/plus.cpp @@ -12,7 +12,7 @@ struct PlusImpl static const constexpr bool allow_decimal = true; template - static inline Result apply(A a, B b) + static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b) { /// Next everywhere, static_cast - so that there is no wrong result in expressions of the form Int64 c = UInt32(a) * Int32(-1). return static_cast(a) + b; diff --git a/libs/libcommon/include/common/intExp.h b/libs/libcommon/include/common/intExp.h index 8c46d9f26c2..ff5d2978ffb 100644 --- a/libs/libcommon/include/common/intExp.h +++ b/libs/libcommon/include/common/intExp.h @@ -6,7 +6,7 @@ /// On overlow, the function returns unspecified value. -inline uint64_t intExp2(int x) +inline NO_SANITIZE_UNDEFINED uint64_t intExp2(int x) { return 1ULL << x; } From 2e323192a26378642c48ab4297ce024faf06a00c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 03:02:11 +0300 Subject: [PATCH 163/230] Addition to prev. revision #3926 --- dbms/src/Common/intExp.h | 129 +++++++++++++++++++++++++ dbms/src/Functions/FunctionsRound.h | 2 +- dbms/src/Functions/intExp10.cpp | 2 +- dbms/src/Functions/intExp2.cpp | 2 +- dbms/src/IO/WriteHelpers.h | 2 +- dbms/src/Parsers/ParserSampleRatio.cpp | 2 +- libs/libcommon/CMakeLists.txt | 1 - 7 files changed, 134 insertions(+), 6 deletions(-) create mode 100644 dbms/src/Common/intExp.h diff --git a/dbms/src/Common/intExp.h b/dbms/src/Common/intExp.h new file mode 100644 index 00000000000..3ee2d0a94b9 --- /dev/null +++ b/dbms/src/Common/intExp.h @@ -0,0 +1,129 @@ +#pragma once + +#include +#include + +#include + + +/// On overlow, the function returns unspecified value. + +inline NO_SANITIZE_UNDEFINED uint64_t intExp2(int x) +{ + return 1ULL << x; +} + +inline uint64_t intExp10(int x) +{ + if (x < 0) + return 0; + if (x > 19) + return std::numeric_limits::max(); + + static const uint64_t table[20] = + { + 1ULL, 10ULL, 100ULL, + 1000ULL, 10000ULL, 100000ULL, + 1000000ULL, 10000000ULL, 100000000ULL, + 1000000000ULL, 10000000000ULL, 100000000000ULL, + 1000000000000ULL, 10000000000000ULL, 100000000000000ULL, + 1000000000000000ULL, 10000000000000000ULL, 100000000000000000ULL, + 1000000000000000000ULL, 10000000000000000000ULL + }; + + return table[x]; +} + +namespace common +{ + +inline int exp10_i32(int x) +{ + static const int values[] = { + 1, + 10, + 100, + 1000, + 10000, + 100000, + 1000000, + 10000000, + 100000000, + 1000000000 + }; + return values[x]; +} + +inline int64_t exp10_i64(int x) +{ + static const int64_t values[] = { + 1ll, + 10ll, + 100ll, + 1000ll, + 10000ll, + 100000ll, + 1000000ll, + 10000000ll, + 100000000ll, + 1000000000ll, + 10000000000ll, + 100000000000ll, + 1000000000000ll, + 10000000000000ll, + 100000000000000ll, + 1000000000000000ll, + 10000000000000000ll, + 100000000000000000ll, + 1000000000000000000ll + }; + return values[x]; +} + +inline __int128 exp10_i128(int x) +{ + static const __int128 values[] = { + static_cast<__int128>(1ll), + static_cast<__int128>(10ll), + static_cast<__int128>(100ll), + static_cast<__int128>(1000ll), + static_cast<__int128>(10000ll), + static_cast<__int128>(100000ll), + static_cast<__int128>(1000000ll), + static_cast<__int128>(10000000ll), + static_cast<__int128>(100000000ll), + static_cast<__int128>(1000000000ll), + static_cast<__int128>(10000000000ll), + static_cast<__int128>(100000000000ll), + static_cast<__int128>(1000000000000ll), + static_cast<__int128>(10000000000000ll), + static_cast<__int128>(100000000000000ll), + static_cast<__int128>(1000000000000000ll), + static_cast<__int128>(10000000000000000ll), + static_cast<__int128>(100000000000000000ll), + static_cast<__int128>(1000000000000000000ll), + static_cast<__int128>(1000000000000000000ll) * 10ll, + static_cast<__int128>(1000000000000000000ll) * 100ll, + static_cast<__int128>(1000000000000000000ll) * 1000ll, + static_cast<__int128>(1000000000000000000ll) * 10000ll, + static_cast<__int128>(1000000000000000000ll) * 100000ll, + static_cast<__int128>(1000000000000000000ll) * 1000000ll, + static_cast<__int128>(1000000000000000000ll) * 10000000ll, + static_cast<__int128>(1000000000000000000ll) * 100000000ll, + static_cast<__int128>(1000000000000000000ll) * 1000000000ll, + static_cast<__int128>(1000000000000000000ll) * 10000000000ll, + static_cast<__int128>(1000000000000000000ll) * 100000000000ll, + static_cast<__int128>(1000000000000000000ll) * 1000000000000ll, + static_cast<__int128>(1000000000000000000ll) * 10000000000000ll, + static_cast<__int128>(1000000000000000000ll) * 100000000000000ll, + static_cast<__int128>(1000000000000000000ll) * 1000000000000000ll, + static_cast<__int128>(1000000000000000000ll) * 10000000000000000ll, + static_cast<__int128>(1000000000000000000ll) * 100000000000000000ll, + static_cast<__int128>(1000000000000000000ll) * 100000000000000000ll * 10ll, + static_cast<__int128>(1000000000000000000ll) * 100000000000000000ll * 100ll, + static_cast<__int128>(1000000000000000000ll) * 100000000000000000ll * 1000ll + }; + return values[x]; +} + +} diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index fbdd821f669..69716d9f121 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include #include diff --git a/dbms/src/Functions/intExp10.cpp b/dbms/src/Functions/intExp10.cpp index 057a4158b46..6f401f7f963 100644 --- a/dbms/src/Functions/intExp10.cpp +++ b/dbms/src/Functions/intExp10.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/dbms/src/Functions/intExp2.cpp b/dbms/src/Functions/intExp2.cpp index 86b751cb6f5..a9787c93fc9 100644 --- a/dbms/src/Functions/intExp2.cpp +++ b/dbms/src/Functions/intExp2.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/dbms/src/IO/WriteHelpers.h b/dbms/src/IO/WriteHelpers.h index 68a26791dca..4b59a754f0b 100644 --- a/dbms/src/IO/WriteHelpers.h +++ b/dbms/src/IO/WriteHelpers.h @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include diff --git a/dbms/src/Parsers/ParserSampleRatio.cpp b/dbms/src/Parsers/ParserSampleRatio.cpp index 3091ed91570..2f444bcf9e8 100644 --- a/dbms/src/Parsers/ParserSampleRatio.cpp +++ b/dbms/src/Parsers/ParserSampleRatio.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index b6187c52bf8..5c6c242407f 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -31,7 +31,6 @@ add_library (common ${LINK_MODE} include/common/ErrorHandlers.h include/common/preciseExp10.h include/common/shift10.h - include/common/intExp.h include/common/mremap.h include/common/likely.h include/common/logger_useful.h From 701dfd83bf3b15403616199a0ed4418e95ae5ae1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 03:02:38 +0300 Subject: [PATCH 164/230] Addition to prev. revision #3926 --- libs/libcommon/include/common/intExp.h | 127 ------------------------- 1 file changed, 127 deletions(-) delete mode 100644 libs/libcommon/include/common/intExp.h diff --git a/libs/libcommon/include/common/intExp.h b/libs/libcommon/include/common/intExp.h deleted file mode 100644 index ff5d2978ffb..00000000000 --- a/libs/libcommon/include/common/intExp.h +++ /dev/null @@ -1,127 +0,0 @@ -#pragma once - -#include -#include - - -/// On overlow, the function returns unspecified value. - -inline NO_SANITIZE_UNDEFINED uint64_t intExp2(int x) -{ - return 1ULL << x; -} - -inline uint64_t intExp10(int x) -{ - if (x < 0) - return 0; - if (x > 19) - return std::numeric_limits::max(); - - static const uint64_t table[20] = - { - 1ULL, 10ULL, 100ULL, - 1000ULL, 10000ULL, 100000ULL, - 1000000ULL, 10000000ULL, 100000000ULL, - 1000000000ULL, 10000000000ULL, 100000000000ULL, - 1000000000000ULL, 10000000000000ULL, 100000000000000ULL, - 1000000000000000ULL, 10000000000000000ULL, 100000000000000000ULL, - 1000000000000000000ULL, 10000000000000000000ULL - }; - - return table[x]; -} - -namespace common -{ - -inline int exp10_i32(int x) -{ - static const int values[] = { - 1, - 10, - 100, - 1000, - 10000, - 100000, - 1000000, - 10000000, - 100000000, - 1000000000 - }; - return values[x]; -} - -inline int64_t exp10_i64(int x) -{ - static const int64_t values[] = { - 1ll, - 10ll, - 100ll, - 1000ll, - 10000ll, - 100000ll, - 1000000ll, - 10000000ll, - 100000000ll, - 1000000000ll, - 10000000000ll, - 100000000000ll, - 1000000000000ll, - 10000000000000ll, - 100000000000000ll, - 1000000000000000ll, - 10000000000000000ll, - 100000000000000000ll, - 1000000000000000000ll - }; - return values[x]; -} - -inline __int128 exp10_i128(int x) -{ - static const __int128 values[] = { - static_cast<__int128>(1ll), - static_cast<__int128>(10ll), - static_cast<__int128>(100ll), - static_cast<__int128>(1000ll), - static_cast<__int128>(10000ll), - static_cast<__int128>(100000ll), - static_cast<__int128>(1000000ll), - static_cast<__int128>(10000000ll), - static_cast<__int128>(100000000ll), - static_cast<__int128>(1000000000ll), - static_cast<__int128>(10000000000ll), - static_cast<__int128>(100000000000ll), - static_cast<__int128>(1000000000000ll), - static_cast<__int128>(10000000000000ll), - static_cast<__int128>(100000000000000ll), - static_cast<__int128>(1000000000000000ll), - static_cast<__int128>(10000000000000000ll), - static_cast<__int128>(100000000000000000ll), - static_cast<__int128>(1000000000000000000ll), - static_cast<__int128>(1000000000000000000ll) * 10ll, - static_cast<__int128>(1000000000000000000ll) * 100ll, - static_cast<__int128>(1000000000000000000ll) * 1000ll, - static_cast<__int128>(1000000000000000000ll) * 10000ll, - static_cast<__int128>(1000000000000000000ll) * 100000ll, - static_cast<__int128>(1000000000000000000ll) * 1000000ll, - static_cast<__int128>(1000000000000000000ll) * 10000000ll, - static_cast<__int128>(1000000000000000000ll) * 100000000ll, - static_cast<__int128>(1000000000000000000ll) * 1000000000ll, - static_cast<__int128>(1000000000000000000ll) * 10000000000ll, - static_cast<__int128>(1000000000000000000ll) * 100000000000ll, - static_cast<__int128>(1000000000000000000ll) * 1000000000000ll, - static_cast<__int128>(1000000000000000000ll) * 10000000000000ll, - static_cast<__int128>(1000000000000000000ll) * 100000000000000ll, - static_cast<__int128>(1000000000000000000ll) * 1000000000000000ll, - static_cast<__int128>(1000000000000000000ll) * 10000000000000000ll, - static_cast<__int128>(1000000000000000000ll) * 100000000000000000ll, - static_cast<__int128>(1000000000000000000ll) * 100000000000000000ll * 10ll, - static_cast<__int128>(1000000000000000000ll) * 100000000000000000ll * 100ll, - static_cast<__int128>(1000000000000000000ll) * 100000000000000000ll * 1000ll - }; - return values[x]; -} - -} From 408f935754a075cc8a4339aede6e8cda7073702b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 03:03:18 +0300 Subject: [PATCH 165/230] Addition to prev. revision #3926 --- dbms/src/Common/intExp.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/intExp.h b/dbms/src/Common/intExp.h index 3ee2d0a94b9..163d835819f 100644 --- a/dbms/src/Common/intExp.h +++ b/dbms/src/Common/intExp.h @@ -3,7 +3,7 @@ #include #include -#include +#include /// On overlow, the function returns unspecified value. From 2e20bd4b4a7c519471ffb646765fbd19d55fa700 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 03:09:39 +0300 Subject: [PATCH 166/230] Addition to prev. revision #3926 --- dbms/src/Functions/FunctionsRound.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 69716d9f121..2c071b06096 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include From e308d2e069f8a7eef2cafba5c24c8a3763c4775b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 03:10:38 +0300 Subject: [PATCH 167/230] Addition to prev. revision #3926 --- dbms/src/Functions/FunctionsRound.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 2c071b06096..c744100da48 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include From d6c7825328a775ca32e0cd47a96300b15343cbae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 03:11:42 +0300 Subject: [PATCH 168/230] Addition to prev. revision #3926 --- dbms/src/Functions/FunctionsRound.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index c744100da48..3b026e970ff 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include From 39492139f088a0c6842ef4e532467da7f214a762 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 03:25:47 +0300 Subject: [PATCH 169/230] Allowed some UB #3926 --- dbms/src/Functions/FunctionsConversion.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 5df8547bf4a..0bc4b269527 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -99,7 +99,7 @@ struct ConvertImpl using ToFieldType = typename ToDataType::FieldType; template - static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/, + static void NO_SANITIZE_UNDEFINED execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/, Additions additions [[maybe_unused]] = Additions()) { const ColumnWithTypeAndName & named_from = block.getByPosition(arguments[0]); From 713a44c38e799d6adc29f77b05ec11228f7b4d33 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 03:33:32 +0300 Subject: [PATCH 170/230] Fixed UB #3926 --- .../DataStreams/MergingSortedBlockInputStream.cpp | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp b/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp index 8dd929759ca..7a62e6c5746 100644 --- a/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -1,5 +1,6 @@ #include #include +#include #include @@ -291,11 +292,18 @@ void MergingSortedBlockInputStream::readSuffixImpl() const BlockStreamProfileInfo & profile_info = getProfileInfo(); double seconds = profile_info.total_stopwatch.elapsedSeconds(); - LOG_DEBUG(log, std::fixed << std::setprecision(2) + + std::stringstream message; + message << std::fixed << std::setprecision(2) << "Merge sorted " << profile_info.blocks << " blocks, " << profile_info.rows << " rows" - << " in " << seconds << " sec., " + << " in " << seconds << " sec."; + + if (seconds) + message << ", " << profile_info.rows / seconds << " rows/sec., " - << profile_info.bytes / 1000000.0 / seconds << " MB/sec."); + << profile_info.bytes / 1000000.0 / seconds << " MB/sec." + + LOG_DEBUG(log, message.str()); } } From 3c84e474191ab6438353cc93e6fa036e44bf54f7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 03:33:54 +0300 Subject: [PATCH 171/230] Fixed UB #3926 --- dbms/src/DataStreams/MergingSortedBlockInputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp b/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp index 7a62e6c5746..f1a1a02084f 100644 --- a/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -301,7 +301,7 @@ void MergingSortedBlockInputStream::readSuffixImpl() if (seconds) message << ", " << profile_info.rows / seconds << " rows/sec., " - << profile_info.bytes / 1000000.0 / seconds << " MB/sec." + << profile_info.bytes / 1000000.0 / seconds << " MB/sec."; LOG_DEBUG(log, message.str()); } From c20f05875a5eb917c99e5029221bd1181e199f0b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 03:34:49 +0300 Subject: [PATCH 172/230] Moved code; added comment #3926 --- dbms/src/Columns/ColumnDecimal.h | 1 + dbms/src/Columns/ColumnFixedString.h | 3 ++- dbms/src/Columns/ColumnVector.h | 1 + dbms/src/Columns/ColumnVectorHelper.h | 39 +++++++++++++++++++++++++++ dbms/src/Columns/IColumn.h | 19 ------------- 5 files changed, 43 insertions(+), 20 deletions(-) create mode 100644 dbms/src/Columns/ColumnVectorHelper.h diff --git a/dbms/src/Columns/ColumnDecimal.h b/dbms/src/Columns/ColumnDecimal.h index 98d706ff546..50a6d9d67fb 100644 --- a/dbms/src/Columns/ColumnDecimal.h +++ b/dbms/src/Columns/ColumnDecimal.h @@ -3,6 +3,7 @@ #include #include +#include namespace DB diff --git a/dbms/src/Columns/ColumnFixedString.h b/dbms/src/Columns/ColumnFixedString.h index d03f0ea1804..b963e074df2 100644 --- a/dbms/src/Columns/ColumnFixedString.h +++ b/dbms/src/Columns/ColumnFixedString.h @@ -1,9 +1,10 @@ #pragma once -#include // memcpy +#include // memcmp #include #include +#include namespace DB diff --git a/dbms/src/Columns/ColumnVector.h b/dbms/src/Columns/ColumnVector.h index 7693329567c..1c5a45ef6ad 100644 --- a/dbms/src/Columns/ColumnVector.h +++ b/dbms/src/Columns/ColumnVector.h @@ -2,6 +2,7 @@ #include #include +#include #include diff --git a/dbms/src/Columns/ColumnVectorHelper.h b/dbms/src/Columns/ColumnVectorHelper.h new file mode 100644 index 00000000000..8a25812ffe7 --- /dev/null +++ b/dbms/src/Columns/ColumnVectorHelper.h @@ -0,0 +1,39 @@ +#pragma once + +#include + + +namespace DB +{ + +/** Allows to access internal array of ColumnVector or ColumnFixedString without cast to concrete type. + * We will inherit ColumnVector and ColumnFixedString from this class instead of IColumn. + * Assumes data layout of ColumnVector, ColumnFixedString and PODArray. + * + * Why it is needed? + * + * There are some algorithms that specialize on the size of data type but doesn't care about concrete type. + * The same specialization may work for UInt64, Int64, Float64, FixedString(8), if it only does byte moving and hashing. + * To avoid code bloat and compile time increase, we can use single template instantiation for these cases + * and just static_cast pointer to some single column type (e. g. ColumnUInt64) assuming that all types have identical memory layout. + * + * But this static_cast (downcast to unrelated type) is illegal according to the C++ standard and UBSan warns about it. + * To allow functional tests to work under UBSan we have to separate some base class that will present the memory layout in explicit way, + * and we will do static_cast to this class. + */ +class ColumnVectorHelper : public IColumn +{ +public: + const char * getRawDataBegin() const + { + return *reinterpret_cast(reinterpret_cast(this) + sizeof(*this)); + } + + template + void insertRawData(const char * ptr) + { + return reinterpret_cast, 15, 16> *>(reinterpret_cast(this) + sizeof(*this))->push_back_raw(ptr); + } +}; + +} diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 80f408e173b..38df6ab3c38 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -389,23 +389,4 @@ struct IsMutableColumns template <> struct IsMutableColumns<> { static const bool value = true; }; - -/// Allows to access internal array of ColumnVector or ColumnFixedString without cast to concrete type. -/// Inherit ColumnVector and ColumnFixedString from this class instead of IColumn. -/// Assumes data layout of ColumnVector, ColumnFixedString and PODArray. -class ColumnVectorHelper : public IColumn -{ -public: - const char * getRawDataBegin() const - { - return *reinterpret_cast(reinterpret_cast(this) + sizeof(*this)); - } - - template - void insertRawData(const char * ptr) - { - return reinterpret_cast, 15, 16> *>(reinterpret_cast(this) + sizeof(*this))->push_back_raw(ptr); - } -}; - } From 673fa80a9ccf891cd7381a71d4c103f7b1d7b62f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 03:40:10 +0300 Subject: [PATCH 173/230] Allowed some UB #3926 --- dbms/src/Functions/FunctionsConversion.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 0bc4b269527..75c86eb89b2 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -177,7 +177,7 @@ struct ToDateTransform32Or64 { static constexpr auto name = "toDate"; - static inline ToType execute(const FromType & from, const DateLUTImpl & time_zone) + static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { return (from < 0xFFFF) ? from : time_zone.toDayNum(from); } From fec30acf81abd1337ddc80278715547cea508382 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 03:51:14 +0300 Subject: [PATCH 174/230] Allowed some UB #3926 --- dbms/src/AggregateFunctions/AggregateFunctionAvg.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionAvg.h b/dbms/src/AggregateFunctions/AggregateFunctionAvg.h index 9cc2d8e26ed..53b42c42c9a 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionAvg.h @@ -25,7 +25,7 @@ struct AggregateFunctionAvgData UInt64 count = 0; template - ResultT result() const + ResultT NO_SANITIZE_UNDEFINED result() const { if constexpr (std::is_floating_point_v) if constexpr (std::numeric_limits::is_iec559) From 1c0659ac2f43136c19d698ff389351f0c0e6701a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 03:55:05 +0300 Subject: [PATCH 175/230] Fixed test #3926 --- dbms/tests/queries/0_stateless/00301_csv.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00301_csv.sh b/dbms/tests/queries/0_stateless/00301_csv.sh index c56ba27454e..648bdf4e575 100755 --- a/dbms/tests/queries/0_stateless/00301_csv.sh +++ b/dbms/tests/queries/0_stateless/00301_csv.sh @@ -15,7 +15,7 @@ Hello "world", 789 ,2016-01-03 $CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv ORDER BY d"; $CLICKHOUSE_CLIENT --query="DROP TABLE test.csv"; -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (t DateTime, s String) ENGINE = Memory"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (t DateTime('UTC'), s String) ENGINE = Memory"; echo '"2016-01-01 01:02:03","1" 2016-01-02 01:02:03, "2" From e8328c576b86ba6e48022d4212774d7c3217a4c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 03:59:52 +0300 Subject: [PATCH 176/230] Fixed UB #3926 --- dbms/src/Columns/ColumnAggregateFunction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Columns/ColumnAggregateFunction.cpp b/dbms/src/Columns/ColumnAggregateFunction.cpp index 8f8a44be868..7d3e001998a 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.cpp +++ b/dbms/src/Columns/ColumnAggregateFunction.cpp @@ -378,7 +378,7 @@ const char * ColumnAggregateFunction::deserializeAndInsertFromArena(const char * * as we cannot legally compare pointers after last element + 1 of some valid memory region. * Probably this will not work under UBSan. */ - ReadBufferFromMemory read_buffer(src_arena, std::numeric_limits::max() - src_arena); + ReadBufferFromMemory read_buffer(src_arena, std::numeric_limits::max() - src_arena - 1); func->deserialize(data.back(), read_buffer, &dst_arena); return read_buffer.position(); From 4c35a73a74c0e0766e1f269dffa798a0af4f5f0b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 04:03:16 +0300 Subject: [PATCH 177/230] Fixed UB #3926 --- .../AggregateFunctionStatisticsSimple.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h b/dbms/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h index e529238707d..0580a5131a2 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h @@ -68,12 +68,12 @@ struct VarMoments readPODBinary(*this, buf); } - T getPopulation() const + T NO_SANITIZE_UNDEFINED getPopulation() const { return (m2 - m1 * m1 / m0) / m0; } - T getSample() const + T NO_SANITIZE_UNDEFINED getSample() const { if (m0 == 0) return std::numeric_limits::quiet_NaN(); @@ -177,12 +177,12 @@ struct CovarMoments readPODBinary(*this, buf); } - T getPopulation() const + T NO_SANITIZE_UNDEFINED getPopulation() const { return (xy - x1 * y1 / m0) / m0; } - T getSample() const + T NO_SANITIZE_UNDEFINED getSample() const { if (m0 == 0) return std::numeric_limits::quiet_NaN(); @@ -232,7 +232,7 @@ struct CorrMoments readPODBinary(*this, buf); } - T get() const + T NO_SANITIZE_UNDEFINED get() const { return (m0 * xy - x1 * y1) / sqrt((m0 * x2 - x1 * x1) * (m0 * y2 - y1 * y1)); } From d2b7a847c78f11f1e6d4c325a2f20900236eae5e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 04:09:27 +0300 Subject: [PATCH 178/230] Allow tests to execute in random order #3926 --- .../queries/0_stateless/00098_1_union_all.sql | 3 +++ .../queries/0_stateless/00098_2_union_all.sql | 16 ++++++++++++++++ .../queries/0_stateless/00098_3_union_all.sql | 17 +++++++++++++++++ .../queries/0_stateless/00098_4_union_all.sql | 17 +++++++++++++++++ .../queries/0_stateless/00098_5_union_all.sql | 17 +++++++++++++++++ .../queries/0_stateless/00098_d_union_all.sql | 16 ++++++++++++++++ .../queries/0_stateless/00098_e_union_all.sql | 16 ++++++++++++++++ .../queries/0_stateless/00098_f_union_all.sql | 16 ++++++++++++++++ 8 files changed, 118 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00098_1_union_all.sql b/dbms/tests/queries/0_stateless/00098_1_union_all.sql index 6f96b710985..fe9a8939cdc 100644 --- a/dbms/tests/queries/0_stateless/00098_1_union_all.sql +++ b/dbms/tests/queries/0_stateless/00098_1_union_all.sql @@ -29,3 +29,6 @@ UNION ALL SELECT value AS val, value, name FROM data2014 WHERE name = 'Alice') ORDER BY val ASC; +DROP TABLE data2013; +DROP TABLE data2014; +DROP TABLE data2015; diff --git a/dbms/tests/queries/0_stateless/00098_2_union_all.sql b/dbms/tests/queries/0_stateless/00098_2_union_all.sql index 6666065efa0..d77eca70d25 100644 --- a/dbms/tests/queries/0_stateless/00098_2_union_all.sql +++ b/dbms/tests/queries/0_stateless/00098_2_union_all.sql @@ -1,6 +1,22 @@ +DROP TABLE IF EXISTS data2013; +DROP TABLE IF EXISTS data2014; + +CREATE TABLE data2013 (name String, value UInt32) ENGINE = Memory; +CREATE TABLE data2014 (name String, value UInt32) ENGINE = Memory; + +INSERT INTO data2013(name,value) VALUES('Alice', 1000); +INSERT INTO data2013(name,value) VALUES('Bob', 2000); +INSERT INTO data2013(name,value) VALUES('Carol', 5000); + +INSERT INTO data2014(name,value) VALUES('Alice', 2000); +INSERT INTO data2014(name,value) VALUES('Bob', 2000); +INSERT INTO data2014(name,value) VALUES('Dennis', 35000); + SELECT val FROM (SELECT value AS val FROM data2013 WHERE name = 'Alice' UNION ALL SELECT value AS val FROM data2014 WHERE name = 'Alice') ORDER BY val ASC; +DROP TABLE data2013; +DROP TABLE data2014; diff --git a/dbms/tests/queries/0_stateless/00098_3_union_all.sql b/dbms/tests/queries/0_stateless/00098_3_union_all.sql index 1494ffe5d47..555d1cea30f 100644 --- a/dbms/tests/queries/0_stateless/00098_3_union_all.sql +++ b/dbms/tests/queries/0_stateless/00098_3_union_all.sql @@ -1,5 +1,22 @@ +DROP TABLE IF EXISTS data2013; +DROP TABLE IF EXISTS data2014; + +CREATE TABLE data2013 (name String, value UInt32) ENGINE = Memory; +CREATE TABLE data2014 (name String, value UInt32) ENGINE = Memory; + +INSERT INTO data2013(name,value) VALUES('Alice', 1000); +INSERT INTO data2013(name,value) VALUES('Bob', 2000); +INSERT INTO data2013(name,value) VALUES('Carol', 5000); + +INSERT INTO data2014(name,value) VALUES('Alice', 2000); +INSERT INTO data2014(name,value) VALUES('Bob', 2000); +INSERT INTO data2014(name,value) VALUES('Dennis', 35000); + SELECT val FROM (SELECT value AS val FROM data2013 WHERE name = 'Alice' UNION /*comment*/ ALL SELECT value AS val FROM data2014 WHERE name = 'Alice') ORDER BY val ASC; + +DROP TABLE data2013; +DROP TABLE data2014; diff --git a/dbms/tests/queries/0_stateless/00098_4_union_all.sql b/dbms/tests/queries/0_stateless/00098_4_union_all.sql index 179a54c815b..09086ce7ac2 100644 --- a/dbms/tests/queries/0_stateless/00098_4_union_all.sql +++ b/dbms/tests/queries/0_stateless/00098_4_union_all.sql @@ -1,3 +1,17 @@ +DROP TABLE IF EXISTS data2013; +DROP TABLE IF EXISTS data2014; + +CREATE TABLE data2013 (name String, value UInt32) ENGINE = Memory; +CREATE TABLE data2014 (name String, value UInt32) ENGINE = Memory; + +INSERT INTO data2013(name,value) VALUES('Alice', 1000); +INSERT INTO data2013(name,value) VALUES('Bob', 2000); +INSERT INTO data2013(name,value) VALUES('Carol', 5000); + +INSERT INTO data2014(name,value) VALUES('Alice', 2000); +INSERT INTO data2014(name,value) VALUES('Bob', 2000); +INSERT INTO data2014(name,value) VALUES('Dennis', 35000); + SELECT val FROM (SELECT value AS val FROM data2013 WHERE name = 'Alice' UNION ALL @@ -5,3 +19,6 @@ SELECT value AS val FROM data2014 WHERE name = 'Alice' UNION ALL SELECT value AS val FROM data2014 WHERE name = 'Dennis') ORDER BY val ASC; + +DROP TABLE data2013; +DROP TABLE data2014; diff --git a/dbms/tests/queries/0_stateless/00098_5_union_all.sql b/dbms/tests/queries/0_stateless/00098_5_union_all.sql index 3bcf8624e61..c4d1a8dc329 100644 --- a/dbms/tests/queries/0_stateless/00098_5_union_all.sql +++ b/dbms/tests/queries/0_stateless/00098_5_union_all.sql @@ -1 +1,18 @@ +DROP TABLE IF EXISTS data2013; +DROP TABLE IF EXISTS data2014; + +CREATE TABLE data2013 (name String, value UInt32) ENGINE = Memory; +CREATE TABLE data2014 (name String, value UInt32) ENGINE = Memory; + +INSERT INTO data2013(name,value) VALUES('Alice', 1000); +INSERT INTO data2013(name,value) VALUES('Bob', 2000); +INSERT INTO data2013(name,value) VALUES('Carol', 5000); + +INSERT INTO data2014(name,value) VALUES('Alice', 2000); +INSERT INTO data2014(name,value) VALUES('Bob', 2000); +INSERT INTO data2014(name,value) VALUES('Dennis', 35000); + SELECT nn,vv FROM (SELECT name AS nn, value AS vv FROM data2013 UNION ALL SELECT name AS nn, value AS vv FROM data2014) ORDER BY nn,vv ASC; + +DROP TABLE data2013; +DROP TABLE data2014; diff --git a/dbms/tests/queries/0_stateless/00098_d_union_all.sql b/dbms/tests/queries/0_stateless/00098_d_union_all.sql index 282d46e81a6..1f6741888c5 100644 --- a/dbms/tests/queries/0_stateless/00098_d_union_all.sql +++ b/dbms/tests/queries/0_stateless/00098_d_union_all.sql @@ -1 +1,17 @@ +DROP TABLE IF EXISTS data2013; +DROP TABLE IF EXISTS data2015; + +CREATE TABLE data2013 (name String, value UInt32) ENGINE = Memory; +CREATE TABLE data2015 (data_name String, data_value UInt32) ENGINE = Memory; + +INSERT INTO data2013(name,value) VALUES('Alice', 1000); +INSERT INTO data2013(name,value) VALUES('Bob', 2000); +INSERT INTO data2013(name,value) VALUES('Carol', 5000); + +INSERT INTO data2015(data_name, data_value) VALUES('Foo', 42); +INSERT INTO data2015(data_name, data_value) VALUES('Bar', 1); + SELECT name FROM (SELECT name FROM data2013 UNION ALL SELECT data_name FROM data2015) ORDER BY name ASC; + +DROP TABLE data2013; +DROP TABLE data2015; diff --git a/dbms/tests/queries/0_stateless/00098_e_union_all.sql b/dbms/tests/queries/0_stateless/00098_e_union_all.sql index 68b26eac5d9..c0b74122edd 100644 --- a/dbms/tests/queries/0_stateless/00098_e_union_all.sql +++ b/dbms/tests/queries/0_stateless/00098_e_union_all.sql @@ -1 +1,17 @@ +DROP TABLE IF EXISTS data2013; +DROP TABLE IF EXISTS data2015; + +CREATE TABLE data2013 (name String, value UInt32) ENGINE = Memory; +CREATE TABLE data2015 (data_name String, data_value UInt32) ENGINE = Memory; + +INSERT INTO data2013(name,value) VALUES('Alice', 1000); +INSERT INTO data2013(name,value) VALUES('Bob', 2000); +INSERT INTO data2013(name,value) VALUES('Carol', 5000); + +INSERT INTO data2015(data_name, data_value) VALUES('Foo', 42); +INSERT INTO data2015(data_name, data_value) VALUES('Bar', 1); + SELECT X FROM (SELECT name AS X FROM data2013 UNION ALL SELECT data_name FROM data2015) ORDER BY X ASC; + +DROP TABLE data2013; +DROP TABLE data2015; diff --git a/dbms/tests/queries/0_stateless/00098_f_union_all.sql b/dbms/tests/queries/0_stateless/00098_f_union_all.sql index 561d2398869..849079742f9 100644 --- a/dbms/tests/queries/0_stateless/00098_f_union_all.sql +++ b/dbms/tests/queries/0_stateless/00098_f_union_all.sql @@ -1 +1,17 @@ +DROP TABLE IF EXISTS data2013; +DROP TABLE IF EXISTS data2015; + +CREATE TABLE data2013 (name String, value UInt32) ENGINE = Memory; +CREATE TABLE data2015 (data_name String, data_value UInt32) ENGINE = Memory; + +INSERT INTO data2013(name,value) VALUES('Alice', 1000); +INSERT INTO data2013(name,value) VALUES('Bob', 2000); +INSERT INTO data2013(name,value) VALUES('Carol', 5000); + +INSERT INTO data2015(data_name, data_value) VALUES('Foo', 42); +INSERT INTO data2015(data_name, data_value) VALUES('Bar', 1); + SELECT name FROM (SELECT name FROM data2013 UNION ALL SELECT data_name AS name FROM data2015) ORDER BY name ASC; + +DROP TABLE data2013; +DROP TABLE data2015; From 589d07860f77ce380c147eae901bd7315289c3b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 04:12:06 +0300 Subject: [PATCH 179/230] Fixed UB #3926 --- .../Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index a36741ba143..d7764244587 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -446,16 +446,19 @@ public: UInt64 & watch_prev_elapsed_, MergeTreeDataMergerMutator::MergeAlgorithm merge_alg_ = MergeAlgorithm::Vertical) : merge_entry(merge_entry_), watch_prev_elapsed(watch_prev_elapsed_), merge_alg(merge_alg_) { - average_elem_progress = (merge_alg == MergeAlgorithm::Horizontal) - ? 1.0 / num_total_rows - : column_sizes.keyColumnsProgress(1, num_total_rows); + if (num_total_rows) + { + average_elem_progress = (merge_alg == MergeAlgorithm::Horizontal) + ? 1.0 / num_total_rows + : column_sizes.keyColumnsProgress(1, num_total_rows); + } updateWatch(); } MergeList::Entry & merge_entry; UInt64 & watch_prev_elapsed; - Float64 average_elem_progress; + Float64 average_elem_progress = 0; const MergeAlgorithm merge_alg{MergeAlgorithm::Vertical}; void updateWatch() From da5b62ab6708b84ecd8c8f25803dce0160174a32 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 04:20:06 +0300 Subject: [PATCH 180/230] Fixed UB #3926 --- dbms/src/Core/AccurateComparison.h | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/dbms/src/Core/AccurateComparison.h b/dbms/src/Core/AccurateComparison.h index b7f44ff7ccf..1e3e236b1ea 100644 --- a/dbms/src/Core/AccurateComparison.h +++ b/dbms/src/Core/AccurateComparison.h @@ -267,55 +267,55 @@ inline bool_if_safe_conversion equalsOp(A a, B b) } template <> -inline bool equalsOp(DB::Float64 f, DB::UInt64 u) +inline bool NO_SANITIZE_UNDEFINED equalsOp(DB::Float64 f, DB::UInt64 u) { return static_cast(f) == u && f == static_cast(u); } template <> -inline bool equalsOp(DB::UInt64 u, DB::Float64 f) +inline bool NO_SANITIZE_UNDEFINED equalsOp(DB::UInt64 u, DB::Float64 f) { return u == static_cast(f) && static_cast(u) == f; } template <> -inline bool equalsOp(DB::Float64 f, DB::Int64 u) +inline bool NO_SANITIZE_UNDEFINED equalsOp(DB::Float64 f, DB::Int64 u) { return static_cast(f) == u && f == static_cast(u); } template <> -inline bool equalsOp(DB::Int64 u, DB::Float64 f) +inline bool NO_SANITIZE_UNDEFINED equalsOp(DB::Int64 u, DB::Float64 f) { return u == static_cast(f) && static_cast(u) == f; } template <> -inline bool equalsOp(DB::Float32 f, DB::UInt64 u) +inline bool NO_SANITIZE_UNDEFINED equalsOp(DB::Float32 f, DB::UInt64 u) { return static_cast(f) == u && f == static_cast(u); } template <> -inline bool equalsOp(DB::UInt64 u, DB::Float32 f) +inline bool NO_SANITIZE_UNDEFINED equalsOp(DB::UInt64 u, DB::Float32 f) { return u == static_cast(f) && static_cast(u) == f; } template <> -inline bool equalsOp(DB::Float32 f, DB::Int64 u) +inline bool NO_SANITIZE_UNDEFINED equalsOp(DB::Float32 f, DB::Int64 u) { return static_cast(f) == u && f == static_cast(u); } template <> -inline bool equalsOp(DB::Int64 u, DB::Float32 f) +inline bool NO_SANITIZE_UNDEFINED equalsOp(DB::Int64 u, DB::Float32 f) { return u == static_cast(f) && static_cast(u) == f; } template <> -inline bool equalsOp(DB::UInt128 u, DB::Float64 f) +inline bool NO_SANITIZE_UNDEFINED equalsOp(DB::UInt128 u, DB::Float64 f) { return u.low == 0 && equalsOp(static_cast(u.high), f); } @@ -338,7 +338,7 @@ inline bool equalsOp(DB::Float32 f, DB::UInt128 u) return equalsOp(static_cast(f), u); } -inline bool greaterOp(DB::Int128 i, DB::Float64 f) +inline bool NO_SANITIZE_UNDEFINED greaterOp(DB::Int128 i, DB::Float64 f) { static constexpr __int128 min_int128 = __int128(0x8000000000000000ll) << 64; static constexpr __int128 max_int128 = (__int128(0x7fffffffffffffffll) << 64) + 0xffffffffffffffffll; @@ -350,7 +350,7 @@ inline bool greaterOp(DB::Int128 i, DB::Float64 f) || (f < static_cast(max_int128) && i > static_cast(f)); } -inline bool greaterOp(DB::Float64 f, DB::Int128 i) +inline bool NO_SANITIZE_UNDEFINED greaterOp(DB::Float64 f, DB::Int128 i) { static constexpr __int128 min_int128 = __int128(0x8000000000000000ll) << 64; static constexpr __int128 max_int128 = (__int128(0x7fffffffffffffffll) << 64) + 0xffffffffffffffffll; @@ -365,8 +365,8 @@ inline bool greaterOp(DB::Float64 f, DB::Int128 i) inline bool greaterOp(DB::Int128 i, DB::Float32 f) { return greaterOp(i, static_cast(f)); } inline bool greaterOp(DB::Float32 f, DB::Int128 i) { return greaterOp(static_cast(f), i); } -inline bool equalsOp(DB::Int128 i, DB::Float64 f) { return i == static_cast(f) && static_cast(i) == f; } -inline bool equalsOp(DB::Int128 i, DB::Float32 f) { return i == static_cast(f) && static_cast(i) == f; } +inline bool NO_SANITIZE_UNDEFINED equalsOp(DB::Int128 i, DB::Float64 f) { return i == static_cast(f) && static_cast(i) == f; } +inline bool NO_SANITIZE_UNDEFINED equalsOp(DB::Int128 i, DB::Float32 f) { return i == static_cast(f) && static_cast(i) == f; } inline bool equalsOp(DB::Float64 f, DB::Int128 i) { return equalsOp(i, f); } inline bool equalsOp(DB::Float32 f, DB::Int128 i) { return equalsOp(i, f); } From 8b3371835a7bed6aa9dd97ab433c491425a87b83 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 04:22:17 +0300 Subject: [PATCH 181/230] Removed useless flag #3926 --- cmake/sanitize.cmake | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 644492d278b..1d8ed9461eb 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -31,18 +31,6 @@ if (SANITIZE) if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libubsan") endif () - if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") - # Devirtualization with multiple virtual inheritance confuses UBSan. - # Example: - # - # ZooKeeperImpl.cpp:529:8: runtime error: member call on address 0x7f548d7ea490 which does not point to an object of type 'Request' - # 0x7f548d7ea490: note: object has invalid vptr - # 00 00 00 00 00 00 00 00 00 00 00 00 a8 a4 7e 8d 54 7f 00 00 00 00 00 00 00 00 00 00 00 00 00 00 - # - # http://lists.llvm.org/pipermail/cfe-commits/Week-of-Mon-20161017/174163.html - - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fno-devirtualize") - endif () else () message (FATAL_ERROR "Unknown sanitizer type: ${SANITIZE}") endif () From 882531b9c0f1bb07b0f4a7ce0840d5b5d1f7b9d9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 04:40:00 +0300 Subject: [PATCH 182/230] Fixed UB #3926 --- dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index d7764244587..ff089fc98de 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -716,7 +716,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor merge_entry->bytes_written_uncompressed = merged_stream->getProfileInfo().bytes; /// Reservation updates is not performed yet, during the merge it may lead to higher free space requirements - if (disk_reservation) + if (disk_reservation && sum_input_rows_upper_bound) { /// The same progress from merge_entry could be used for both algorithms (it should be more accurate) /// But now we are using inaccurate row-based estimation in Horizontal case for backward compability From 76dafd4d95fff5cb76f14524a5ea3018fd6a5abd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 04:51:22 +0300 Subject: [PATCH 183/230] Fixed UB #3926 --- dbms/src/Functions/FunctionsStringArray.h | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/FunctionsStringArray.h b/dbms/src/Functions/FunctionsStringArray.h index 19838a56081..3422423434a 100644 --- a/dbms/src/Functions/FunctionsStringArray.h +++ b/dbms/src/Functions/FunctionsStringArray.h @@ -308,8 +308,17 @@ public: if (!re->match(pos, end - pos, matches) || !matches[0].length) return false; - token_begin = pos + matches[capture].offset; - token_end = token_begin + matches[capture].length; + if (matches[capture].offset == std::string::npos) + { + /// Empty match. + token_begin = pos; + token_end = pos; + } + else + { + token_begin = pos + matches[capture].offset; + token_end = token_begin + matches[capture].length; + } pos += matches[0].offset + matches[0].length; From 353c2268938cab06622cd2fc9f5f93433dfc3b6a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 04:56:16 +0300 Subject: [PATCH 184/230] Fixed UB #3926 --- dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h index f89943f1fc6..40b13acbbaa 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h @@ -96,7 +96,7 @@ private: /** Calculates the slope of a line between leftmost and rightmost data points. * (y2 - y1) / (x2 - x1) */ - Float64 getBoundingRatio(const AggregateFunctionBoundingRatioData & data) const + Float64 NO_SANITIZE_UNDEFINED getBoundingRatio(const AggregateFunctionBoundingRatioData & data) const { if (data.empty) return std::numeric_limits::quiet_NaN(); From 160ffe8c7f1bc94ff5c495c4b81ba38576307e37 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 05:00:31 +0300 Subject: [PATCH 185/230] Fixed UB #3926 --- dbms/src/Functions/FunctionBinaryArithmetic.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/FunctionBinaryArithmetic.h b/dbms/src/Functions/FunctionBinaryArithmetic.h index 12c306cdf34..5fe88441fd6 100644 --- a/dbms/src/Functions/FunctionBinaryArithmetic.h +++ b/dbms/src/Functions/FunctionBinaryArithmetic.h @@ -289,7 +289,7 @@ private: } template - static NativeResultType applyScaled(NativeResultType a, NativeResultType b, NativeResultType scale) + static NO_SANITIZE_UNDEFINED NativeResultType applyScaled(NativeResultType a, NativeResultType b, NativeResultType scale) { if constexpr (is_plus_minus_compare) { @@ -324,7 +324,7 @@ private: } } - static NativeResultType applyScaledDiv(NativeResultType a, NativeResultType b, NativeResultType scale) + static NO_SANITIZE_UNDEFINED NativeResultType applyScaledDiv(NativeResultType a, NativeResultType b, NativeResultType scale) { if constexpr (is_division) { From 05472db990a1b6465a138c65dacf8b4fef32a9a4 Mon Sep 17 00:00:00 2001 From: sdk2 <469656392@qq.com> Date: Thu, 27 Dec 2018 15:21:13 +0800 Subject: [PATCH 186/230] update docs kafka.md (#3936) --- docs/en/operations/table_engines/kafka.md | 2 +- docs/ru/operations/table_engines/kafka.md | 2 +- docs/zh/operations/table_engines/kafka.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/table_engines/kafka.md b/docs/en/operations/table_engines/kafka.md index 49c42fb0a16..3927e472e50 100644 --- a/docs/en/operations/table_engines/kafka.md +++ b/docs/en/operations/table_engines/kafka.md @@ -24,7 +24,7 @@ Kafka SETTINGS kafka_topic_list = 'topic1,topic2', kafka_group_name = 'group1', kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\n' + kafka_row_delimiter = '\n', kafka_schema = '', kafka_num_consumers = 2 ``` diff --git a/docs/ru/operations/table_engines/kafka.md b/docs/ru/operations/table_engines/kafka.md index d4f0c4bdfb0..4c426932596 100644 --- a/docs/ru/operations/table_engines/kafka.md +++ b/docs/ru/operations/table_engines/kafka.md @@ -23,7 +23,7 @@ Kafka SETTINGS kafka_topic_list = 'topic1,topic2', kafka_group_name = 'group1', kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\n' + kafka_row_delimiter = '\n', kafka_schema = '', kafka_num_consumers = 2 ``` diff --git a/docs/zh/operations/table_engines/kafka.md b/docs/zh/operations/table_engines/kafka.md index e66b3130f4a..22e001f2c1c 100644 --- a/docs/zh/operations/table_engines/kafka.md +++ b/docs/zh/operations/table_engines/kafka.md @@ -23,7 +23,7 @@ Kafka SETTINGS kafka_topic_list = 'topic1,topic2', kafka_group_name = 'group1', kafka_format = 'JSONEachRow', - kafka_row_delimiter = '\n' + kafka_row_delimiter = '\n', kafka_schema = '', kafka_num_consumers = 2 ``` From bf7401ced7945517934b1f5746b2681285f72fd2 Mon Sep 17 00:00:00 2001 From: Eric <2630001511@qq.com> Date: Thu, 27 Dec 2018 15:54:55 +0800 Subject: [PATCH 187/230] Update cli.md (#3938) add -- for config-file command line option --- docs/en/interfaces/cli.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index b3c3dd97b24..d665fbd3e17 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -91,7 +91,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--vertical, -E` – If specified, use the Vertical format by default to output the result. This is the same as '--format=Vertical'. In this format, each value is printed on a separate line, which is helpful when displaying wide tables. - `--time, -t` – If specified, print the query execution time to 'stderr' in non-interactive mode. - `--stacktrace` – If specified, also print the stack trace if an exception occurs. -- `-config-file` – The name of the configuration file. +- `--config-file` – The name of the configuration file. ### Configuration Files From 22bb17250f47ac2fae77327cfa040ad774fa53f8 Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 27 Dec 2018 16:09:11 +0300 Subject: [PATCH 188/230] autodisable logs in negative tests CLICKHOUSE-4228 --- dbms/programs/client/Client.cpp | 6 ++++ dbms/programs/client/TestHint.h | 32 ++++++++++--------- .../0_stateless/00700_decimal_aggregates.sql | 2 -- .../0_stateless/00700_decimal_arithm.sql | 2 -- .../0_stateless/00700_decimal_bounds.sql | 2 -- .../0_stateless/00700_decimal_casts.sql | 2 -- .../0_stateless/00700_decimal_compare.sql | 2 -- .../00700_decimal_complex_types.sql | 2 -- .../0_stateless/00700_decimal_defaults.sql | 2 -- .../0_stateless/00700_decimal_math.sql | 2 -- .../0_stateless/00700_decimal_null.sql | 2 -- 11 files changed, 23 insertions(+), 33 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 90153da9074..d5a354973e2 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -666,6 +666,12 @@ private: const bool test_mode = config().has("testmode"); if (config().has("multiquery")) { + { /// disable logs if expects errors + TestHint test_hint(test_mode, text); + if (test_hint.clientError() || test_hint.serverError()) + process("SET send_logs_level = 'none'"); + } + /// Several queries separated by ';'. /// INSERT data is ended by the end of line, not ';'. diff --git a/dbms/programs/client/TestHint.h b/dbms/programs/client/TestHint.h index 790e58ee7fe..cedd9e5b9be 100644 --- a/dbms/programs/client/TestHint.h +++ b/dbms/programs/client/TestHint.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -27,25 +28,26 @@ public: if (!enabled_) return; - /// TODO: This is absolutely wrong. Fragment may be contained inside string literal. - size_t pos = query.find("--"); + String full_comment; + Lexer lexer(query.data(), query.data() + query.size()); - if (pos != String::npos && query.find("--", pos + 2) != String::npos) - return; /// It's not last comment. Hint belongs to commented query. /// TODO Absolutely wrong: there maybe the following comment for the next query. - - if (pos != String::npos) + for (Token token = lexer.nextToken(); !token.isEnd(); token = lexer.nextToken()) { - /// TODO: This is also wrong. Comment may already have ended by line break. - pos = query.find('{', pos + 2); + if (token.type == TokenType::Comment) + full_comment += String(token.begin, token.begin + token.size()) + ' '; + } - if (pos != String::npos) + if (!full_comment.empty()) + { + size_t pos_start = full_comment.find('{', 0); + if (pos_start != String::npos) { - String hint = query.substr(pos + 1); - - /// TODO: And this is wrong for the same reason. - pos = hint.find('}'); - hint.resize(pos); - parse(hint); + size_t pos_end = full_comment.find('}', pos_start); + if (pos_end != String::npos) + { + String hint(full_comment.begin() + pos_start, full_comment.begin() + pos_end); + parse(hint); + } } } } diff --git a/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql b/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql index 951e1384e93..0e482c9b109 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql @@ -1,5 +1,3 @@ -SET send_logs_level = 'none'; - CREATE DATABASE IF NOT EXISTS test; DROP TABLE IF EXISTS test.decimal; diff --git a/dbms/tests/queries/0_stateless/00700_decimal_arithm.sql b/dbms/tests/queries/0_stateless/00700_decimal_arithm.sql index d2cdfc8d18a..fa0a3598c65 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_arithm.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_arithm.sql @@ -1,5 +1,3 @@ -SET send_logs_level = 'none'; - CREATE DATABASE IF NOT EXISTS test; DROP TABLE IF EXISTS test.decimal; diff --git a/dbms/tests/queries/0_stateless/00700_decimal_bounds.sql b/dbms/tests/queries/0_stateless/00700_decimal_bounds.sql index 4aa9c38b94a..13f62d2da18 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_bounds.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_bounds.sql @@ -1,5 +1,3 @@ -SET send_logs_level = 'none'; - CREATE DATABASE IF NOT EXISTS test; DROP TABLE IF EXISTS test.decimal; diff --git a/dbms/tests/queries/0_stateless/00700_decimal_casts.sql b/dbms/tests/queries/0_stateless/00700_decimal_casts.sql index 14593b50b5e..b1bb0c62d78 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_casts.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_casts.sql @@ -1,5 +1,3 @@ -SET send_logs_level = 'none'; - SELECT toDecimal32('1.1', 1), toDecimal32('1.1', 2), toDecimal32('1.1', 8); SELECT toDecimal32('1.1', 0); -- { serverError 69 } SELECT toDecimal32(1.1, 0), toDecimal32(1.1, 1), toDecimal32(1.1, 2), toDecimal32(1.1, 8); diff --git a/dbms/tests/queries/0_stateless/00700_decimal_compare.sql b/dbms/tests/queries/0_stateless/00700_decimal_compare.sql index a0d0c3407a4..767bd0e3232 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_compare.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_compare.sql @@ -1,5 +1,3 @@ -SET send_logs_level = 'none'; - CREATE DATABASE IF NOT EXISTS test; DROP TABLE IF EXISTS test.decimal; diff --git a/dbms/tests/queries/0_stateless/00700_decimal_complex_types.sql b/dbms/tests/queries/0_stateless/00700_decimal_complex_types.sql index 4afb09975ba..7cf763b43ea 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_complex_types.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_complex_types.sql @@ -1,5 +1,3 @@ -SET send_logs_level = 'none'; - CREATE DATABASE IF NOT EXISTS test; DROP TABLE IF EXISTS test.decimal; diff --git a/dbms/tests/queries/0_stateless/00700_decimal_defaults.sql b/dbms/tests/queries/0_stateless/00700_decimal_defaults.sql index 2c9cc29d708..e61d7ef9df9 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_defaults.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_defaults.sql @@ -1,5 +1,3 @@ -SET send_logs_level = 'none'; - CREATE DATABASE IF NOT EXISTS test; DROP TABLE IF EXISTS test.decimal; diff --git a/dbms/tests/queries/0_stateless/00700_decimal_math.sql b/dbms/tests/queries/0_stateless/00700_decimal_math.sql index fdc3eabfe71..a3a175777ed 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_math.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_math.sql @@ -1,5 +1,3 @@ -SET send_logs_level = 'none'; - SELECT toDecimal32('42.42', 4) AS x, toDecimal32(log(x), 4) AS y, exp(y); SELECT toDecimal32('42.42', 4) AS x, toDecimal32(log2(x), 4) AS y, exp2(y); SELECT toDecimal32('42.42', 4) AS x, toDecimal32(log10(x), 4) AS y, exp10(y); diff --git a/dbms/tests/queries/0_stateless/00700_decimal_null.sql b/dbms/tests/queries/0_stateless/00700_decimal_null.sql index 24571213241..1cc389b29c9 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_null.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_null.sql @@ -1,5 +1,3 @@ -SET send_logs_level = 'none'; - CREATE DATABASE IF NOT EXISTS test; DROP TABLE IF EXISTS test.decimal; From d9bc934b7d808878f06e37ed80c17d6efeca86f3 Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 27 Dec 2018 17:30:48 +0300 Subject: [PATCH 189/230] 3908 Fix build client only --- dbms/programs/config_tools.h.in | 1 + dbms/programs/main.cpp | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/programs/config_tools.h.in b/dbms/programs/config_tools.h.in index a7a538d9f1d..13aebfd3c83 100644 --- a/dbms/programs/config_tools.h.in +++ b/dbms/programs/config_tools.h.in @@ -12,3 +12,4 @@ #cmakedefine01 ENABLE_CLICKHOUSE_COMPRESSOR #cmakedefine01 ENABLE_CLICKHOUSE_FORMAT #cmakedefine01 ENABLE_CLICKHOUSE_OBFUSCATOR +#cmakedefine01 ENABLE_CLICKHOUSE_ODBC_BRIDGE diff --git a/dbms/programs/main.cpp b/dbms/programs/main.cpp index 29d64213d9c..e8b8cd365d6 100644 --- a/dbms/programs/main.cpp +++ b/dbms/programs/main.cpp @@ -53,7 +53,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv); #if ENABLE_CLICKHOUSE_COPIER || !defined(ENABLE_CLICKHOUSE_COPIER) int mainEntryClickHouseClusterCopier(int argc, char ** argv); #endif -#if ENABLE_CLICKHOUSE_OBFUSCATOR +#if ENABLE_CLICKHOUSE_OBFUSCATOR || !defined(ENABLE_CLICKHOUSE_OBFUSCATOR) int mainEntryClickHouseObfuscator(int argc, char ** argv); #endif #if ENABLE_CLICKHOUSE_ODBC_BRIDGE || !defined(ENABLE_CLICKHOUSE_ODBC_BRIDGE) @@ -102,7 +102,7 @@ std::pair clickhouse_applications[] = #if ENABLE_CLICKHOUSE_COPIER || !defined(ENABLE_CLICKHOUSE_COPIER) {"copier", mainEntryClickHouseClusterCopier}, #endif -#if ENABLE_CLICKHOUSE_OBFUSCATOR +#if ENABLE_CLICKHOUSE_OBFUSCATOR || !defined(ENABLE_CLICKHOUSE_OBFUSCATOR) {"obfuscator", mainEntryClickHouseObfuscator}, #endif #if ENABLE_CLICKHOUSE_ODBC_BRIDGE || !defined(ENABLE_CLICKHOUSE_ODBC_BRIDGE) From d791da03c775b462c9d34d9db70a388651550f65 Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 27 Dec 2018 17:48:47 +0300 Subject: [PATCH 190/230] fix test hint parsing --- dbms/programs/client/TestHint.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/client/TestHint.h b/dbms/programs/client/TestHint.h index cedd9e5b9be..7ddbfbb59f8 100644 --- a/dbms/programs/client/TestHint.h +++ b/dbms/programs/client/TestHint.h @@ -45,7 +45,7 @@ public: size_t pos_end = full_comment.find('}', pos_start); if (pos_end != String::npos) { - String hint(full_comment.begin() + pos_start, full_comment.begin() + pos_end); + String hint(full_comment.begin() + pos_start + 1, full_comment.begin() + pos_end); parse(hint); } } From aab8e5f4b24f0f7a3b2547cbe7a2692eb4c91cdc Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 27 Dec 2018 17:48:58 +0300 Subject: [PATCH 191/230] 2443 Do not write preprocessed configs to /etc/ - Microfixes --- dbms/tests/queries/shell_config.sh | 2 +- debian/clickhouse-server.postinst | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/tests/queries/shell_config.sh b/dbms/tests/queries/shell_config.sh index ca5e7981b3a..6a2e0e536f6 100644 --- a/dbms/tests/queries/shell_config.sh +++ b/dbms/tests/queries/shell_config.sh @@ -17,7 +17,7 @@ export CLICKHOUSE_CONFIG_CLIENT=${CLICKHOUSE_CONFIG_CLIENT:="/etc/clickhouse-cli [ -x "${CLICKHOUSE_BINARY}" ] && CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="$CLICKHOUSE_BINARY extract-from-config --config=$CLICKHOUSE_CONFIG"} export CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="$CLICKHOUSE_BINARY-extract-from-config --config=$CLICKHOUSE_CONFIG"} -export CLICKHOUSE_CONFIG_GREP=${CLICKHOUSE_CONFIG_GREP:="/etc/clickhouse-server/config-preprocessed.xml"} +export CLICKHOUSE_CONFIG_GREP=${CLICKHOUSE_CONFIG_GREP:="/etc/clickhouse-server/preprocessed/config.xml"} export CLICKHOUSE_HOST=${CLICKHOUSE_HOST:="localhost"} export CLICKHOUSE_PORT_TCP=${CLICKHOUSE_PORT_TCP:=`${CLICKHOUSE_EXTRACT_CONFIG} --try --key=tcp_port 2>/dev/null`} 2>/dev/null diff --git a/debian/clickhouse-server.postinst b/debian/clickhouse-server.postinst index 80b25d47f62..694507b9366 100644 --- a/debian/clickhouse-server.postinst +++ b/debian/clickhouse-server.postinst @@ -79,7 +79,7 @@ Please fix this and reinstall this package." >&2 rm -fv ${CLICKHOUSE_CONFDIR}/*-preprocessed.xml ||: fi - ln -s ${CLICKHOUSE_DATADIR}/preprocessed_configs ${CLICKHOUSE_CONFDIR}/preprocessed ||: + [ -e ${CLICKHOUSE_CONFDIR}/preprocessed ] || ln -s ${CLICKHOUSE_DATADIR}/preprocessed_configs ${CLICKHOUSE_CONFDIR}/preprocessed ||: if [ ! -d ${CLICKHOUSE_LOGDIR} ]; then mkdir -p ${CLICKHOUSE_LOGDIR} From 1a0a879a517b21c97d34abfaf961a70aa4b4cce3 Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 27 Dec 2018 18:23:37 +0300 Subject: [PATCH 192/230] Build fixes --- CMakeLists.txt | 5 ----- cmake/find_zlib.cmake | 13 +++++++------ cmake/limit_jobs.cmake | 20 +++++++++++--------- contrib/CMakeLists.txt | 1 + dbms/CMakeLists.txt | 16 ++++++++++++---- dbms/src/IO/ReadBufferFromHDFS.h | 5 +++-- utils/CMakeLists.txt | 7 +++++++ utils/check-style/check-include | 3 +++ 8 files changed, 44 insertions(+), 26 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 09227fead49..7b77687c645 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -25,11 +25,6 @@ endif () # Write compile_commands.json set(CMAKE_EXPORT_COMPILE_COMMANDS 1) - -set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") -set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") -include (cmake/limit_jobs.cmake) - include (cmake/find_ccache.cmake) if (NOT CMAKE_BUILD_TYPE OR CMAKE_BUILD_TYPE STREQUAL "None") diff --git a/cmake/find_zlib.cmake b/cmake/find_zlib.cmake index 90fe674ab36..fb6b8c7971d 100644 --- a/cmake/find_zlib.cmake +++ b/cmake/find_zlib.cmake @@ -8,23 +8,24 @@ endif () if (NOT ZLIB_FOUND) if (NOT MSVC) - set (INTERNAL_ZLIB_NAME "zlib-ng") + set (INTERNAL_ZLIB_NAME "zlib-ng" CACHE INTERNAL "") else () - set (INTERNAL_ZLIB_NAME "zlib") + set (INTERNAL_ZLIB_NAME "zlib" CACHE INTERNAL "") if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}") message (WARNING "Will use standard zlib, please clone manually:\n git clone https://github.com/madler/zlib.git ${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}") endif () endif () set (USE_INTERNAL_ZLIB_LIBRARY 1) - set (ZLIB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}" "${ClickHouse_BINARY_DIR}/contrib/${INTERNAL_ZLIB_NAME}") # generated zconf.h + set (ZLIB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}" "${ClickHouse_BINARY_DIR}/contrib/${INTERNAL_ZLIB_NAME}" CACHE INTERNAL "") # generated zconf.h set (ZLIB_INCLUDE_DIRS ${ZLIB_INCLUDE_DIR}) # for poco + set (ZLIB_INCLUDE_DIRECTORIES ${ZLIB_INCLUDE_DIR}) # for protobuf set (ZLIB_FOUND 1) # for poco if (USE_STATIC_LIBRARIES) - set (ZLIB_LIBRARIES zlibstatic) + set (ZLIB_LIBRARIES zlibstatic CACHE INTERNAL "") else () - set (ZLIB_LIBRARIES zlib) + set (ZLIB_LIBRARIES zlib CACHE INTERNAL "") endif () endif () -message (STATUS "Using zlib: ${ZLIB_INCLUDE_DIR} : ${ZLIB_LIBRARIES}") +message (STATUS "Using ${INTERNAL_ZLIB_NAME}: ${ZLIB_INCLUDE_DIR} : ${ZLIB_LIBRARIES}") diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index d71c5260240..dfc1ce82bb7 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -6,30 +6,32 @@ cmake_host_system_information(RESULT AVAILABLE_PHYSICAL_MEMORY QUERY AVAILABLE_PHYSICAL_MEMORY) # Not available under freebsd option(PARALLEL_COMPILE_JOBS "Define the maximum number of concurrent compilation jobs" "") -if (NOT PARALLEL_COMPILE_JOBS AND AVAILABLE_PHYSICAL_MEMORY) - math(EXPR PARALLEL_COMPILE_JOBS ${AVAILABLE_PHYSICAL_MEMORY}/2500) # ~2.5gb max per one compiler +if (NOT PARALLEL_COMPILE_JOBS AND AVAILABLE_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY) + math(EXPR PARALLEL_COMPILE_JOBS ${AVAILABLE_PHYSICAL_MEMORY}/${MAX_COMPILER_MEMORY}) if (NOT PARALLEL_COMPILE_JOBS) set (PARALLEL_COMPILE_JOBS 1) endif () endif () if (PARALLEL_COMPILE_JOBS) - set_property(GLOBAL APPEND PROPERTY JOB_POOLS compile_job_pool=${PARALLEL_COMPILE_JOBS}) - set(CMAKE_JOB_POOL_COMPILE compile_job_pool) + set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) + string (REPLACE "/" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) + set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS}) endif () option(PARALLEL_LINK_JOBS "Define the maximum number of concurrent link jobs" "") -if (NOT PARALLEL_LINK_JOBS AND AVAILABLE_PHYSICAL_MEMORY) - math(EXPR PARALLEL_LINK_JOBS ${AVAILABLE_PHYSICAL_MEMORY}/4000) # ~4gb max per one linker +if (NOT PARALLEL_LINK_JOBS AND AVAILABLE_PHYSICAL_MEMORY AND MAX_LINKER_MEMORY) + math(EXPR PARALLEL_LINK_JOBS ${AVAILABLE_PHYSICAL_MEMORY}/${MAX_LINKER_MEMORY}) if (NOT PARALLEL_LINK_JOBS) set (PARALLEL_LINK_JOBS 1) endif () endif () if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) - message(STATUS "Have ${AVAILABLE_PHYSICAL_MEMORY} megabytes of memory. Limiting concurrent linkers jobs to ${PARALLEL_LINK_JOBS} and compiler jobs to ${PARALLEL_COMPILE_JOBS}") + message(STATUS "${CMAKE_CURRENT_SOURCE_DIR}: Have ${AVAILABLE_PHYSICAL_MEMORY} megabytes of memory. Limiting concurrent linkers jobs to ${PARALLEL_LINK_JOBS} and compiler jobs to ${PARALLEL_COMPILE_JOBS}") endif () if (LLVM_PARALLEL_LINK_JOBS) - set_property(GLOBAL APPEND PROPERTY JOB_POOLS link_job_pool=${PARALLEL_LINK_JOBS}) - set(CMAKE_JOB_POOL_LINK link_job_pool) + set(CMAKE_JOB_POOL_LINK link_job_pool${CMAKE_CURRENT_SOURCE_DIR}) + string (REPLACE "/" "_" CMAKE_JOB_POOL_LINK ${CMAKE_JOB_POOL_LINK}) + set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_LINK}=${PARALLEL_LINK_JOBS}) endif () diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 989761bfb67..ec824c3567c 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -206,6 +206,7 @@ if (USE_INTERNAL_HDFS3_LIBRARY) if (USE_INTERNAL_PROTOBUF_LIBRARY) set(protobuf_BUILD_TESTS OFF CACHE INTERNAL "" FORCE) set(protobuf_BUILD_SHARED_LIBS OFF CACHE INTERNAL "" FORCE) + set(protobuf_WITH_ZLIB 0 CACHE INTERNAL "" FORCE) # actually will use zlib, but skip find add_subdirectory(protobuf/cmake) endif () add_subdirectory(libhdfs3-cmake) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index b7f11731662..d5ac533b0cd 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -2,15 +2,23 @@ if (USE_INCLUDE_WHAT_YOU_USE) set (CMAKE_CXX_INCLUDE_WHAT_YOU_USE ${IWYU_PATH}) endif () -include(${CMAKE_CURRENT_SOURCE_DIR}/cmake/find_vectorclass.cmake) +set (MAX_COMPILER_MEMORY 2500 CACHE INTERNAL "") +if (MAKE_STATIC_LIBRARIES) + set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") +else() + set (MAX_LINKER_MEMORY 2500 CACHE INTERNAL "") +endif () +include (../cmake/limit_jobs.cmake) + +include(cmake/find_vectorclass.cmake) set (CONFIG_VERSION ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config_version.h) set (CONFIG_COMMON ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config.h) include (cmake/version.cmake) message (STATUS "Will build ${VERSION_FULL}") -configure_file (${CMAKE_CURRENT_SOURCE_DIR}/src/Common/config.h.in ${CONFIG_COMMON}) -configure_file (${CMAKE_CURRENT_SOURCE_DIR}/src/Common/config_version.h.in ${CONFIG_VERSION}) +configure_file (src/Common/config.h.in ${CONFIG_COMMON}) +configure_file (src/Common/config_version.h.in ${CONFIG_VERSION}) if (NOT MSVC) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra") @@ -53,7 +61,7 @@ add_subdirectory (src) set(dbms_headers) set(dbms_sources) -include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) +include(../cmake/dbms_glob_sources.cmake) add_headers_and_sources(clickhouse_common_io src/Common) add_headers_and_sources(clickhouse_common_io src/Common/HashTable) diff --git a/dbms/src/IO/ReadBufferFromHDFS.h b/dbms/src/IO/ReadBufferFromHDFS.h index 21705c1ef30..a97ad5ece2f 100644 --- a/dbms/src/IO/ReadBufferFromHDFS.h +++ b/dbms/src/IO/ReadBufferFromHDFS.h @@ -7,6 +7,7 @@ #include #include #include +#include #ifndef O_DIRECT #define O_DIRECT 00040000 @@ -52,7 +53,7 @@ namespace DB if (fs == nullptr) { - throw Exception("Unable to connect to HDFS: " + String(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); + throw Exception("Unable to connect to HDFS: " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); } fin = hdfsOpenFile(fs, path.c_str(), O_RDONLY, 0, 0, 0); @@ -77,7 +78,7 @@ namespace DB int bytes_read = hdfsRead(fs, fin, internal_buffer.begin(), internal_buffer.size()); if (bytes_read < 0) { - throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + String(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); + throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); } if (bytes_read) diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index ba202d6bddc..08a60cdf19c 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -3,6 +3,13 @@ if (NOT NO_WERROR) set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Werror") endif () +if (MAKE_STATIC_LIBRARIES) + set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") +else() + set (MAX_LINKER_MEMORY 2500 CACHE INTERNAL "") +endif () +include (../cmake/limit_jobs.cmake) + # Utils used in package add_subdirectory (config-processor) add_subdirectory (report) diff --git a/utils/check-style/check-include b/utils/check-style/check-include index eede9940dcd..4a02f488278 100755 --- a/utils/check-style/check-include +++ b/utils/check-style/check-include @@ -52,6 +52,9 @@ inc="-I. \ -I./libs/libdaemon/include \ -I./libs/consistent-hashing \ -I./libs/consistent-hashing-sumbur \ +-I./contrib/libhdfs3/include \ +-I./contrib/base64/include \ +-I./contrib/lz4/lib \ -I./dbms/src \ -I${BUILD_DIR}/dbms/src" From b94bfa5785d4ed69ab2bcbb2e18828cce2dff562 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 18:39:41 +0300 Subject: [PATCH 193/230] Fixed test #3926 --- dbms/tests/queries/0_stateless/00301_csv.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00301_csv.sh b/dbms/tests/queries/0_stateless/00301_csv.sh index 648bdf4e575..857850000b9 100755 --- a/dbms/tests/queries/0_stateless/00301_csv.sh +++ b/dbms/tests/queries/0_stateless/00301_csv.sh @@ -15,7 +15,7 @@ Hello "world", 789 ,2016-01-03 $CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv ORDER BY d"; $CLICKHOUSE_CLIENT --query="DROP TABLE test.csv"; -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (t DateTime('UTC'), s String) ENGINE = Memory"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (t DateTime('Europe/Moscow'), s String) ENGINE = Memory"; echo '"2016-01-01 01:02:03","1" 2016-01-02 01:02:03, "2" From 3d0c26f573b425299bd57f412443196034389e02 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 27 Dec 2018 18:55:51 +0300 Subject: [PATCH 194/230] Fix ipv4 mask restrictions for users and add integration tests --- dbms/src/Interpreters/Users.cpp | 10 ++- dbms/tests/integration/helpers/cluster.py | 44 ++++++++++- .../helpers/docker_compose_hdfs.yml | 2 +- .../helpers/docker_compose_kafka.yml | 2 +- .../helpers/docker_compose_mysql.yml | 2 +- .../helpers/docker_compose_postgres.yml | 2 +- .../helpers/docker_compose_zookeeper.yml | 2 +- .../test_user_ip_restrictions/__init__.py | 0 .../configs/config_ipv6.xml | 1 + .../configs/users_ipv4.xml | 27 +++++++ .../configs/users_ipv6.xml | 26 +++++++ .../test_user_ip_restrictions/test.py | 74 +++++++++++++++++++ 12 files changed, 179 insertions(+), 13 deletions(-) create mode 100644 dbms/tests/integration/test_user_ip_restrictions/__init__.py create mode 100644 dbms/tests/integration/test_user_ip_restrictions/configs/config_ipv6.xml create mode 100644 dbms/tests/integration/test_user_ip_restrictions/configs/users_ipv4.xml create mode 100644 dbms/tests/integration/test_user_ip_restrictions/configs/users_ipv6.xml create mode 100644 dbms/tests/integration/test_user_ip_restrictions/test.py diff --git a/dbms/src/Interpreters/Users.cpp b/dbms/src/Interpreters/Users.cpp index f3165e26497..0ac4e4aff16 100644 --- a/dbms/src/Interpreters/Users.cpp +++ b/dbms/src/Interpreters/Users.cpp @@ -70,18 +70,20 @@ public: else { String addr(str, 0, pos - str.c_str()); - mask_address = toIPv6(Poco::Net::IPAddress(addr)); + auto real_address = Poco::Net::IPAddress(addr); String str_mask(str, addr.length() + 1, str.length() - addr.length() - 1); if (isDigits(str_mask)) { UInt8 prefix_bits = parse(pos + 1); - construct(prefix_bits); + construct(prefix_bits, real_address.family() == Poco::Net::AddressFamily::IPv4); } else { subnet_mask = netmaskToIPv6(Poco::Net::IPAddress(str_mask)); } + + mask_address = toIPv6(real_address); } } @@ -97,9 +99,9 @@ private: subnet_mask = Poco::Net::IPAddress(128, Poco::Net::IPAddress::IPv6); } - void construct(UInt8 prefix_bits) + void construct(UInt8 prefix_bits, bool is_ipv4) { - prefix_bits = mask_address.family() == Poco::Net::IPAddress::IPv4 ? prefix_bits + 96 : prefix_bits; + prefix_bits = is_ipv4 ? prefix_bits + 96 : prefix_bits; subnet_mask = Poco::Net::IPAddress(prefix_bits, Poco::Net::IPAddress::IPv6); } diff --git a/dbms/tests/integration/helpers/cluster.py b/dbms/tests/integration/helpers/cluster.py index 22d34d05844..59309d2bb99 100644 --- a/dbms/tests/integration/helpers/cluster.py +++ b/dbms/tests/integration/helpers/cluster.py @@ -97,7 +97,7 @@ class ClickHouseCluster: cmd += " client" return cmd - def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False): + def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False, ipv4_address=None, ipv6_address=None): """Add an instance to the cluster. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. @@ -116,7 +116,8 @@ class ClickHouseCluster: instance = ClickHouseInstance( self, self.base_dir, name, config_dir, main_configs, user_configs, macros, with_zookeeper, self.zookeeper_config_path, with_mysql, with_kafka, self.base_configs_dir, self.server_bin_path, - clickhouse_path_dir, with_odbc_drivers, hostname=hostname, env_variables=env_variables, image=image, stay_alive=stay_alive) + clickhouse_path_dir, with_odbc_drivers, hostname=hostname, env_variables=env_variables, image=image, + stay_alive=stay_alive, ipv4_address=ipv4_address, ipv6_address=ipv6_address) self.instances[name] = instance self.base_cmd.extend(['--file', instance.docker_compose_path]) @@ -332,7 +333,7 @@ CLICKHOUSE_START_COMMAND = "clickhouse server --config-file=/etc/clickhouse-serv CLICKHOUSE_STAY_ALIVE_COMMAND = 'bash -c "{} --daemon; tail -f /dev/null"'.format(CLICKHOUSE_START_COMMAND) DOCKER_COMPOSE_TEMPLATE = ''' -version: '2' +version: '2.2' services: {name}: image: {image} @@ -347,6 +348,22 @@ services: depends_on: {depends_on} env_file: - {env_file} + {networks} + {app_net} + {ipv4_address} + {ipv6_address} + +networks: + app_net: + driver: bridge + enable_ipv6: true + ipam: + driver: default + config: + - subnet: 10.5.0.0/12 + gateway: 10.5.1.1 + - subnet: 2001:3984:3989::/64 + gateway: 2001:3984:3989::1 ''' class ClickHouseInstance: @@ -354,7 +371,8 @@ class ClickHouseInstance: def __init__( self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, base_configs_dir, server_bin_path, - clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False): + clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", + stay_alive=False, ipv4_address=None, ipv6_address=None): self.name = name self.base_cmd = cluster.base_cmd[:] @@ -391,6 +409,8 @@ class ClickHouseInstance: self.default_timeout = 20.0 # 20 sec self.image = image self.stay_alive = stay_alive + self.ipv4_address = ipv4_address + self.ipv6_address = ipv6_address # Connects to the instance via clickhouse-client, sends a query (1st argument) and returns the answer def query(self, sql, stdin=None, timeout=None, settings=None, user=None, ignore_error=False): @@ -609,6 +629,18 @@ class ClickHouseInstance: if self.stay_alive: entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND + ipv4_address = ipv6_address = "" + if self.ipv4_address is None and self.ipv6_address is None: + networks = "" + app_net = "" + else: + networks = "networks:" + app_net = "app_net:" + if self.ipv4_address is not None: + ipv4_address = "ipv4_address: " + self.ipv4_address + if self.ipv6_address is not None: + ipv6_address = "ipv6_address: " + self.ipv6_address + with open(self.docker_compose_path, 'w') as docker_compose: docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format( image=self.image, @@ -623,6 +655,10 @@ class ClickHouseInstance: env_file=env_file, odbc_ini_path=odbc_ini_path, entrypoint_cmd=entrypoint_cmd, + networks=networks, + app_net=app_net, + ipv4_address=ipv4_address, + ipv6_address=ipv6_address, )) diff --git a/dbms/tests/integration/helpers/docker_compose_hdfs.yml b/dbms/tests/integration/helpers/docker_compose_hdfs.yml index ecfb0c329b3..010a6cd37c1 100644 --- a/dbms/tests/integration/helpers/docker_compose_hdfs.yml +++ b/dbms/tests/integration/helpers/docker_compose_hdfs.yml @@ -1,4 +1,4 @@ -version: '2' +version: '2.2' services: hdfs1: image: sequenceiq/hadoop-docker:2.7.0 diff --git a/dbms/tests/integration/helpers/docker_compose_kafka.yml b/dbms/tests/integration/helpers/docker_compose_kafka.yml index 42dd154b1e8..461d4626a81 100644 --- a/dbms/tests/integration/helpers/docker_compose_kafka.yml +++ b/dbms/tests/integration/helpers/docker_compose_kafka.yml @@ -1,4 +1,4 @@ -version: '2' +version: '2.2' services: kafka_zookeeper: diff --git a/dbms/tests/integration/helpers/docker_compose_mysql.yml b/dbms/tests/integration/helpers/docker_compose_mysql.yml index 6106b588f76..30c76ef1981 100644 --- a/dbms/tests/integration/helpers/docker_compose_mysql.yml +++ b/dbms/tests/integration/helpers/docker_compose_mysql.yml @@ -1,4 +1,4 @@ -version: '2' +version: '2.2' services: mysql1: image: mysql:5.7 diff --git a/dbms/tests/integration/helpers/docker_compose_postgres.yml b/dbms/tests/integration/helpers/docker_compose_postgres.yml index 74049d9f0b3..0745fe827c0 100644 --- a/dbms/tests/integration/helpers/docker_compose_postgres.yml +++ b/dbms/tests/integration/helpers/docker_compose_postgres.yml @@ -1,4 +1,4 @@ -version: '2' +version: '2.2' services: postgres1: image: postgres diff --git a/dbms/tests/integration/helpers/docker_compose_zookeeper.yml b/dbms/tests/integration/helpers/docker_compose_zookeeper.yml index 365744f106d..22c45cc5960 100644 --- a/dbms/tests/integration/helpers/docker_compose_zookeeper.yml +++ b/dbms/tests/integration/helpers/docker_compose_zookeeper.yml @@ -1,4 +1,4 @@ -version: '2' +version: '2.2' services: zoo1: image: zookeeper diff --git a/dbms/tests/integration/test_user_ip_restrictions/__init__.py b/dbms/tests/integration/test_user_ip_restrictions/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_user_ip_restrictions/configs/config_ipv6.xml b/dbms/tests/integration/test_user_ip_restrictions/configs/config_ipv6.xml new file mode 100644 index 00000000000..675225f9575 --- /dev/null +++ b/dbms/tests/integration/test_user_ip_restrictions/configs/config_ipv6.xml @@ -0,0 +1 @@ +:: diff --git a/dbms/tests/integration/test_user_ip_restrictions/configs/users_ipv4.xml b/dbms/tests/integration/test_user_ip_restrictions/configs/users_ipv4.xml new file mode 100644 index 00000000000..f505af55a31 --- /dev/null +++ b/dbms/tests/integration/test_user_ip_restrictions/configs/users_ipv4.xml @@ -0,0 +1,27 @@ + + + + 10000000000 + 0 + random + + + + + + + + 10.5.173.1 + 10.5.172.0/24 + 10.5.175.0/255.255.255.0 + + default + default + + + + + + + + diff --git a/dbms/tests/integration/test_user_ip_restrictions/configs/users_ipv6.xml b/dbms/tests/integration/test_user_ip_restrictions/configs/users_ipv6.xml new file mode 100644 index 00000000000..eeacb72fde5 --- /dev/null +++ b/dbms/tests/integration/test_user_ip_restrictions/configs/users_ipv6.xml @@ -0,0 +1,26 @@ + + + + 10000000000 + 0 + random + + + + + + + + 2001:3984:3989:0:0:0:1:1111 + 2001:3984:3989:0:0:0:0:0/112 + + default + default + + + + + + + + diff --git a/dbms/tests/integration/test_user_ip_restrictions/test.py b/dbms/tests/integration/test_user_ip_restrictions/test.py new file mode 100644 index 00000000000..731f2bd7fa8 --- /dev/null +++ b/dbms/tests/integration/test_user_ip_restrictions/test.py @@ -0,0 +1,74 @@ +import time +import pytest + +from helpers.cluster import ClickHouseCluster + +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) + +node_ipv4 = cluster.add_instance('node_ipv4', config_dir="configs", user_configs=['configs/users_ipv4.xml'], ipv4_address='10.5.172.77') +client_ipv4_ok = cluster.add_instance('client_ipv4_ok', config_dir="configs", ipv4_address='10.5.172.10') +client_ipv4_ok_direct = cluster.add_instance('client_ipv4_ok_direct', config_dir="configs", ipv4_address='10.5.173.1') +client_ipv4_ok_full_mask = cluster.add_instance('client_ipv4_ok_full_mask', config_dir="configs", ipv4_address='10.5.175.77') +client_ipv4_bad = cluster.add_instance('client_ipv4_bad', config_dir="configs", ipv4_address='10.5.173.10') + +node_ipv6 = cluster.add_instance('node_ipv6', config_dir="configs", main_configs=["configs/config_ipv6.xml"], user_configs=['configs/users_ipv6.xml'], ipv6_address='2001:3984:3989::1:1000') +client_ipv6_ok = cluster.add_instance('client_ipv6_ok', config_dir="configs", ipv6_address='2001:3984:3989::5555') +client_ipv6_ok_direct = cluster.add_instance('client_ipv6_ok_direct', config_dir="configs", ipv6_address='2001:3984:3989::1:1111') +client_ipv6_bad = cluster.add_instance('client_ipv6_bad', config_dir="configs", ipv6_address='2001:3984:3989::1:1112') + + +@pytest.fixture(scope="module") +def setup_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_ipv4(setup_cluster): + try: + client_ipv4_ok.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host 10.5.172.77 --query 'select 1'"], privileged=True, user='root') + except Exception as ex: + assert False, "allowed client with 10.5.172.10 cannot connect to server with allowed mask '10.5.172.0/24'" + + try: + client_ipv4_ok_direct.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host 10.5.172.77 --query 'select 1'"], privileged=True, user='root') + except Exception as ex: + assert False, "allowed client with 10.5.173.1 cannot connect to server with allowed ip '10.5.173.1'" + + try: + client_ipv4_ok_full_mask.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host 10.5.172.77 --query 'select 1'"], privileged=True, user='root') + except Exception as ex: + assert False, "allowed client with 10.5.175.77 cannot connect to server with allowed ip '10.5.175.0/255.255.255.0'" + + try: + client_ipv4_bad.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host 10.5.172.77 --query 'select 1'"], privileged=True, user='root') + assert False, "restricted client with 10.5.173.10 can connect to server with allowed mask '10.5.172.0/24'" + except AssertionError: + raise + except Exception as ex: + print ex + +def test_ipv6(setup_cluster): + try: + client_ipv6_ok.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host 2001:3984:3989::1:1000 --query 'select 1'"], privileged=True, user='root') + except Exception as ex: + print ex + assert False, "allowed client with 2001:3984:3989:0:0:0:1:1111 cannot connect to server with allowed mask '2001:3984:3989:0:0:0:0:0/112'" + + try: + client_ipv6_ok_direct.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host 2001:3984:3989:0:0:0:1:1000 --query 'select 1'"], privileged=True, user='root') + except Exception as ex: + assert False, "allowed client with 2001:3984:3989:0:0:0:1:1111 cannot connect to server with allowed ip '2001:3984:3989:0:0:0:1:1111'" + + try: + client_ipv6_bad.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host 2001:3984:3989:0:0:0:1:1000 --query 'select 1'"], privileged=True, user='root') + assert False, "restricted client with 2001:3984:3989:0:0:0:1:1112 can connect to server with allowed mask '2001:3984:3989:0:0:0:0:0/112'" + except AssertionError: + raise + except Exception as ex: + print ex From e8245f593527049c242de407262c950be6ed0c33 Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 27 Dec 2018 19:12:42 +0300 Subject: [PATCH 195/230] basic support for multiple_joins_emulation CLICKHOUSE-3996 --- dbms/src/Interpreters/Settings.h | 1 + dbms/src/Interpreters/executeQuery.cpp | 13 ++++---- .../00820_multiple_joins.reference | 7 ++++ .../0_stateless/00820_multiple_joins.sql | 32 +++++++++++++++++++ 4 files changed, 47 insertions(+), 6 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00820_multiple_joins.reference create mode 100644 dbms/tests/queries/0_stateless/00820_multiple_joins.sql diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 5b55ebba908..d4eb2a6d8dd 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -296,6 +296,7 @@ struct Settings M(SettingBool, enable_debug_queries, false, "Enables debug queries such as AST.") \ M(SettingBool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.") \ M(SettingBool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.") \ + M(SettingBool, allow_experimental_multiple_joins_emulation, false, "Emulate multiple joins using subselects") \ #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ TYPE NAME {DEFAULT}; diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 2fd865ef0cb..75fd09f5676 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -188,12 +188,13 @@ static std::tuple executeQueryImpl( if (!internal) logQuery(query.substr(0, settings.log_queries_cut_to_length), context); -#if 1 - JoinToSubqueryTransformVisitor::Data join_to_subs_data; - JoinToSubqueryTransformVisitor(join_to_subs_data).visit(ast); - if (join_to_subs_data.done) - logQuery(queryToString(*ast), context); -#endif + if (settings.allow_experimental_multiple_joins_emulation) + { + JoinToSubqueryTransformVisitor::Data join_to_subs_data; + JoinToSubqueryTransformVisitor(join_to_subs_data).visit(ast); + if (join_to_subs_data.done) + logQuery(queryToString(*ast), context); + } /// Check the limits. checkASTSizeLimits(*ast, settings); diff --git a/dbms/tests/queries/0_stateless/00820_multiple_joins.reference b/dbms/tests/queries/0_stateless/00820_multiple_joins.reference new file mode 100644 index 00000000000..8acab23ce21 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00820_multiple_joins.reference @@ -0,0 +1,7 @@ +0 0 0 +6 60 600 +12 120 1200 +18 180 1800 +0 0 0 +10 100 1000 +20 200 2000 diff --git a/dbms/tests/queries/0_stateless/00820_multiple_joins.sql b/dbms/tests/queries/0_stateless/00820_multiple_joins.sql new file mode 100644 index 00000000000..3885c7d3535 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00820_multiple_joins.sql @@ -0,0 +1,32 @@ +USE test; + +DROP TABLE IF EXISTS table1; +DROP TABLE IF EXISTS table2; +DROP TABLE IF EXISTS table3; +DROP TABLE IF EXISTS table5; + +CREATE TABLE table1 (a UInt32) ENGINE = Memory; +CREATE TABLE table2 (a UInt32, b UInt32) ENGINE = Memory; +CREATE TABLE table3 (b UInt32, c UInt32) ENGINE = Memory; +CREATE TABLE table5 (a UInt32, b UInt32, c UInt32) ENGINE = Memory; + +INSERT INTO table1 SELECT number FROM numbers(21); +INSERT INTO table2 SELECT number * 2, number * 20 FROM numbers(11); +INSERT INTO table3 SELECT number * 30, number * 300 FROM numbers(10); +INSERT INTO table5 SELECT number * 5, number * 50, number * 500 FROM numbers(10); + +SET allow_experimental_multiple_joins_emulation = 1; + +-- FIXME: wrong names qualification +select a, b, c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on b = t3.b; +select a, b, c from table1 as t1 join table2 as t2 on t1.a = t2.a join table5 as t5 on a = t5.a AND b = t5.b; +--select a, b, c, d from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on b = t3.b join table5 as t5 on c = t5.c; + +--select t1.x, t2.y, t3.z from table1 as t1 join table2 as t2 on t1.x = t2.x join table3 as t3 on t2.y = t3.y; +--select t1.x, t2.y, t3.z from table1 as t1 join (select * from table2 as t2 join table3 as t3 on t2.y = t3.y) on t1.x = t2.x; +--select t1.x, j1.y, j1.z from table1 as t1 join (select * from table2 as t2 join table3 as t3 on t2.y = t3.y) as j1 on t1.x = j1.x; + +DROP TABLE table1; +DROP TABLE table2; +DROP TABLE table3; +DROP TABLE table5; From da00c6e918551091775328e6036357fa86bbdbeb Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 27 Dec 2018 20:23:49 +0300 Subject: [PATCH 196/230] retry loading ZK substitutions after ZK errors #2947 --- dbms/src/Common/Config/ConfigProcessor.cpp | 10 ++++++++++ dbms/src/Common/Config/ConfigReloader.cpp | 16 +++++++++++++++- dbms/src/Common/Config/ConfigReloader.h | 1 + 3 files changed, 26 insertions(+), 1 deletion(-) diff --git a/dbms/src/Common/Config/ConfigProcessor.cpp b/dbms/src/Common/Config/ConfigProcessor.cpp index 7e9579366b1..36ff7a99529 100644 --- a/dbms/src/Common/Config/ConfigProcessor.cpp +++ b/dbms/src/Common/Config/ConfigProcessor.cpp @@ -447,6 +447,11 @@ XMLDocumentPtr ConfigProcessor::processConfig( merge(config, with); contributing_files.push_back(merge_file); } + catch (Exception & e) + { + e.addMessage("while merging config '" + path + "' with '" + merge_file + "'"); + throw; + } catch (Poco::Exception & e) { throw Poco::Exception("Failed to merge config with '" + merge_file + "': " + e.displayText()); @@ -479,6 +484,11 @@ XMLDocumentPtr ConfigProcessor::processConfig( doIncludesRecursive(config, include_from, getRootNode(config.get()), zk_node_cache, zk_changed_event, contributing_zk_paths); } + catch (Exception & e) + { + e.addMessage("while preprocessing config '" + path + "'"); + throw; + } catch (Poco::Exception & e) { throw Poco::Exception("Failed to preprocess config '" + path + "': " + e.displayText(), e); diff --git a/dbms/src/Common/Config/ConfigReloader.cpp b/dbms/src/Common/Config/ConfigReloader.cpp index 60447638f46..f6edd3f16fb 100644 --- a/dbms/src/Common/Config/ConfigReloader.cpp +++ b/dbms/src/Common/Config/ConfigReloader.cpp @@ -81,7 +81,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac std::lock_guard lock(reload_mutex); FilesChangesTracker new_files = getNewFileList(); - if (force || new_files.isDifferOrNewerThan(files)) + if (force || need_reload_from_zk || new_files.isDifferOrNewerThan(files)) { ConfigProcessor config_processor(path); ConfigProcessor::LoadedConfig loaded_config; @@ -94,6 +94,17 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac loaded_config = config_processor.loadConfigWithZooKeeperIncludes( zk_node_cache, zk_changed_event, fallback_to_preprocessed); } + catch (const Coordination::Exception & e) + { + if (Coordination::isHardwareError(e.code)) + need_reload_from_zk = true; + + if (throw_on_error) + throw; + + tryLogCurrentException(log, "ZooKeeper error when loading config from `" + path + "'"); + return; + } catch (...) { if (throw_on_error) @@ -110,7 +121,10 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac * When file has been written (and contain valid data), we don't load new data since modification time remains the same. */ if (!loaded_config.loaded_from_preprocessed) + { files = std::move(new_files); + need_reload_from_zk = false; + } try { diff --git a/dbms/src/Common/Config/ConfigReloader.h b/dbms/src/Common/Config/ConfigReloader.h index 9ceaa981109..ca4c97c5aee 100644 --- a/dbms/src/Common/Config/ConfigReloader.h +++ b/dbms/src/Common/Config/ConfigReloader.h @@ -75,6 +75,7 @@ private: std::string preprocessed_dir; FilesChangesTracker files; zkutil::ZooKeeperNodeCache zk_node_cache; + bool need_reload_from_zk = false; zkutil::EventPtr zk_changed_event = std::make_shared(); Updater updater; From 5016bb97da485e2ea0f4ed5c7d83e8e41493651a Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 27 Dec 2018 21:21:06 +0300 Subject: [PATCH 197/230] Fix build --- cmake/limit_jobs.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index dfc1ce82bb7..1f05dad912b 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -14,7 +14,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND AVAILABLE_PHYSICAL_MEMORY AND MAX_COMPILER_MEM endif () if (PARALLEL_COMPILE_JOBS) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) - string (REPLACE "/" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) + string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS}) endif () @@ -31,7 +31,7 @@ endif () if (LLVM_PARALLEL_LINK_JOBS) set(CMAKE_JOB_POOL_LINK link_job_pool${CMAKE_CURRENT_SOURCE_DIR}) - string (REPLACE "/" "_" CMAKE_JOB_POOL_LINK ${CMAKE_JOB_POOL_LINK}) + string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_LINK ${CMAKE_JOB_POOL_LINK}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_LINK}=${PARALLEL_LINK_JOBS}) endif () From 0d4e166db9b00ec109069cb1661277e1bbc1fccd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 22:33:27 +0300 Subject: [PATCH 198/230] Fixed error: read one byte after buffer (this byte was unused) [#CLICKHOUSE-2] --- dbms/src/Common/CompactArray.h | 41 +++++++++++++--------------------- 1 file changed, 15 insertions(+), 26 deletions(-) diff --git a/dbms/src/Common/CompactArray.h b/dbms/src/Common/CompactArray.h index ee4d74b3ed2..55ae4239760 100644 --- a/dbms/src/Common/CompactArray.h +++ b/dbms/src/Common/CompactArray.h @@ -55,26 +55,6 @@ public: return locus; } - void readText(ReadBuffer & in) - { - for (size_t i = 0; i < BITSET_SIZE; ++i) - { - if (i != 0) - assertChar(',', in); - readIntText(bitset[i], in); - } - } - - void writeText(WriteBuffer & out) const - { - for (size_t i = 0; i < BITSET_SIZE; ++i) - { - if (i != 0) - writeCString(",", out); - writeIntText(bitset[i], out); - } - } - private: /// number of bytes in bitset static constexpr size_t BITSET_SIZE = (static_cast(bucket_count) * content_width + 7) / 8; @@ -165,7 +145,9 @@ private: bool fits_in_byte; }; -/** The `Locus` structure contains the necessary information to find for each cell +/** TODO This code looks very suboptimal. + * + * The `Locus` structure contains the necessary information to find for each cell * the corresponding byte and offset, in bits, from the beginning of the cell. Since in general * case the size of one byte is not divisible by the size of one cell, cases possible * when one cell overlaps two bytes. Therefore, the `Locus` structure contains two @@ -219,13 +201,20 @@ private: void ALWAYS_INLINE init(BucketIndex bucket_index) { + /// offset in bits to the leftmost bit size_t l = static_cast(bucket_index) * content_width; - index_l = l >> 3; - offset_l = l & 7; - size_t r = static_cast(bucket_index + 1) * content_width; - index_r = r >> 3; - offset_r = r & 7; + /// offset of byte that contains the leftmost bit + index_l = l / 8; + + /// offset in bits to the leftmost bit at that byte + offset_l = l % 8; + + /// offset of byte that contains the rightmost bit + index_r = (l + content_width - 1) / 8; + + /// offset in bits to the next to the rightmost bit at that byte; or zero if the rightmost bit is the rightmost bit in that byte. + offset_r = (l + content_width) % 8; } UInt8 ALWAYS_INLINE read(UInt8 value_l) const From 54e2b6ea91b5fa8c0647d67a416b311094684a09 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 22:38:17 +0300 Subject: [PATCH 199/230] Suppress warnings in 3rd party library [#CLICKHOUSE-2] --- contrib/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 2622b74ceb7..514ed752529 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -4,7 +4,7 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-stringop-overflow") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-implicit-fallthrough -Wno-class-memaccess -Wno-sign-compare -std=c++1z") elseif (CMAKE_CXX_COMPILER_ID STREQUAL "Clang") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-function -Wno-unused-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-format -Wno-parentheses-equality") + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-function -Wno-unused-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-format -Wno-parentheses-equality -Wno-tautological-constant-compare -Wno-tautological-constant-out-of-range-compare -Wno-implicit-function-declaration -Wno-return-type -Wno-pointer-bool-conversion -Wno-enum-conversion -Wno-int-conversion -Wno-switch") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-format -Wno-inconsistent-missing-override -std=c++1z") endif () From 01db559a7035465accde0b1159d5dbdd90284277 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 22:39:14 +0300 Subject: [PATCH 200/230] Fixed test in non-local timezone [#CLICKHOUSE-2] --- .../0_stateless/00374_json_each_row_input_with_noisy_fields.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00374_json_each_row_input_with_noisy_fields.sh b/dbms/tests/queries/0_stateless/00374_json_each_row_input_with_noisy_fields.sh index dfe508ad124..538a46e0fda 100755 --- a/dbms/tests/queries/0_stateless/00374_json_each_row_input_with_noisy_fields.sh +++ b/dbms/tests/queries/0_stateless/00374_json_each_row_input_with_noisy_fields.sh @@ -24,7 +24,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.json_noisy" echo $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test.json_each_row" -$CLICKHOUSE_CLIENT -q "CREATE TABLE test.json_each_row (d DateTime) ENGINE = Memory" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test.json_each_row (d DateTime('Europe/Moscow')) ENGINE = Memory" echo '{"d" : "2017-08-31 18:36:48", "t" : ""} {"d" : "1504193808", "t" : -1} {"d" : 1504193808, "t" : []} From c3c053e7b6ce8deefab47313f72a877cba1d2a97 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 27 Dec 2018 22:42:25 +0300 Subject: [PATCH 201/230] Add cap SYS_PTRACE for LeakSanitizer --- dbms/tests/integration/helpers/cluster.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/tests/integration/helpers/cluster.py b/dbms/tests/integration/helpers/cluster.py index 59309d2bb99..329ea631bfc 100644 --- a/dbms/tests/integration/helpers/cluster.py +++ b/dbms/tests/integration/helpers/cluster.py @@ -345,6 +345,8 @@ services: - {logs_dir}:/var/log/clickhouse-server/ {odbc_ini_path} entrypoint: {entrypoint_cmd} + cap_add: + - SYS_PTRACE depends_on: {depends_on} env_file: - {env_file} From 88bdbc4dc775fb59b7ae5df4589046f05b673674 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 22:58:30 +0300 Subject: [PATCH 202/230] Some additions to Amos Bird's optimization #3920 --- dbms/src/Functions/length.cpp | 4 ++-- dbms/src/Interpreters/Aggregator.h | 2 +- dbms/src/Interpreters/Join.h | 2 +- dbms/src/Interpreters/SetVariants.h | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Functions/length.cpp b/dbms/src/Functions/length.cpp index 24a2984abe2..67267434794 100644 --- a/dbms/src/Functions/length.cpp +++ b/dbms/src/Functions/length.cpp @@ -17,7 +17,7 @@ struct LengthImpl { size_t size = offsets.size(); for (size_t i = 0; i < size; ++i) - res[i] = i == 0 ? (offsets[i] - 1) : (offsets[i] - 1 - offsets[i - 1]); + res[i] = offsets[i] - 1 - offsets[i - 1]; } static void vector_fixed_to_constant(const ColumnString::Chars & /*data*/, size_t n, UInt64 & res) @@ -33,7 +33,7 @@ struct LengthImpl { size_t size = offsets.size(); for (size_t i = 0; i < size; ++i) - res[i] = i == 0 ? (offsets[i]) : (offsets[i] - offsets[i - 1]); + res[i] = offsets[i] - offsets[i - 1]; } }; diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 319e8783c9e..7a7414e5658 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -271,7 +271,7 @@ struct AggregationMethodString { return StringRef( &(*chars)[(*offsets)[i - 1]], - (i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1); + ((*offsets)[i] - (*offsets)[i - 1]) - 1); } }; diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index ff9e7ae1243..ec1950ce641 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -75,7 +75,7 @@ struct JoinKeyGetterString { return StringRef( &(*chars)[(*offsets)[i - 1]], - (i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1); + ((*offsets)[i] - (*offsets)[i - 1]) - 1); } static void onNewKey(Key & key, Arena & pool) diff --git a/dbms/src/Interpreters/SetVariants.h b/dbms/src/Interpreters/SetVariants.h index 731a7c41316..9d4cc74d1a0 100644 --- a/dbms/src/Interpreters/SetVariants.h +++ b/dbms/src/Interpreters/SetVariants.h @@ -86,7 +86,7 @@ struct SetMethodString { return StringRef( &(*chars)[(*offsets)[i - 1]], - (i == 0 ? (*offsets)[i] : ((*offsets)[i] - (*offsets)[i - 1])) - 1); + ((*offsets)[i] - (*offsets)[i - 1]) - 1); } }; From 29ffa46838db5c7aaa2ddbbf885dc59f82b6d126 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 23:02:51 +0300 Subject: [PATCH 203/230] Less indirection #3920 --- dbms/src/Interpreters/Aggregator.h | 12 ++++++------ dbms/src/Interpreters/Join.h | 12 ++++++------ dbms/src/Interpreters/SetVariants.h | 12 ++++++------ 3 files changed, 18 insertions(+), 18 deletions(-) diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 7a7414e5658..6323f53f6fa 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -250,15 +250,15 @@ struct AggregationMethodString struct State { - const ColumnString::Offsets * offsets; - const ColumnString::Chars * chars; + const IColumn::Offset * offsets; + const UInt8 * chars; void init(ColumnRawPtrs & key_columns) { const IColumn & column = *key_columns[0]; const ColumnString & column_string = static_cast(column); - offsets = &column_string.getOffsets(); - chars = &column_string.getChars(); + offsets = column_string.getOffsets().data(); + chars = column_string.getChars().data(); } ALWAYS_INLINE Key getKey( @@ -270,8 +270,8 @@ struct AggregationMethodString Arena & /*pool*/) const { return StringRef( - &(*chars)[(*offsets)[i - 1]], - ((*offsets)[i] - (*offsets)[i - 1]) - 1); + chars + offsets[i - 1], + offsets[i] - offsets[i - 1] - 1); } }; diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index ec1950ce641..7bd68d593e1 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -56,15 +56,15 @@ struct JoinKeyGetterString { using Key = StringRef; - const ColumnString::Offsets * offsets; - const ColumnString::Chars * chars; + const IColumn::Offset * offsets; + const UInt8 * chars; JoinKeyGetterString(const ColumnRawPtrs & key_columns) { const IColumn & column = *key_columns[0]; const ColumnString & column_string = static_cast(column); - offsets = &column_string.getOffsets(); - chars = &column_string.getChars(); + offsets = column_string.getOffsets().data(); + chars = column_string.getChars().data(); } Key getKey( @@ -74,8 +74,8 @@ struct JoinKeyGetterString const Sizes &) const { return StringRef( - &(*chars)[(*offsets)[i - 1]], - ((*offsets)[i] - (*offsets)[i - 1]) - 1); + chars + offsets[i - 1], + offsets[i] - offsets[i - 1] - 1); } static void onNewKey(Key & key, Arena & pool) diff --git a/dbms/src/Interpreters/SetVariants.h b/dbms/src/Interpreters/SetVariants.h index 9d4cc74d1a0..da1dc2140ba 100644 --- a/dbms/src/Interpreters/SetVariants.h +++ b/dbms/src/Interpreters/SetVariants.h @@ -67,15 +67,15 @@ struct SetMethodString struct State { - const ColumnString::Offsets * offsets; - const ColumnString::Chars * chars; + const IColumn::Offset * offsets; + const UInt8 * chars; void init(const ColumnRawPtrs & key_columns) { const IColumn & column = *key_columns[0]; const ColumnString & column_string = static_cast(column); - offsets = &column_string.getOffsets(); - chars = &column_string.getChars(); + offsets = column_string.getOffsets().data(); + chars = column_string.getChars().data(); } Key getKey( @@ -85,8 +85,8 @@ struct SetMethodString const Sizes &) const { return StringRef( - &(*chars)[(*offsets)[i - 1]], - ((*offsets)[i] - (*offsets)[i - 1]) - 1); + chars + offsets[i - 1], + offsets[i] - offsets[i - 1] - 1); } }; From 220e568894c960e31f2e3180a056bbb4aa1d9ce8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 23:10:37 +0300 Subject: [PATCH 204/230] Fixed UB #3920 --- dbms/src/Interpreters/Aggregator.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 6323f53f6fa..882ac6fbd6d 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -280,7 +280,8 @@ struct AggregationMethodString static ALWAYS_INLINE void onNewKey(typename Data::value_type & value, size_t /*keys_size*/, StringRefs & /*keys*/, Arena & pool) { - value.first.data = pool.insert(value.first.data, value.first.size); + if (value.first.size) + value.first.data = pool.insert(value.first.data, value.first.size); } static ALWAYS_INLINE void onExistingKey(const Key & /*key*/, StringRefs & /*keys*/, Arena & /*pool*/) {} From b109ec8c7718377580d9884b798d800599a4dae4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 23:24:37 +0300 Subject: [PATCH 205/230] Fixed UB #3920 --- dbms/src/Interpreters/Aggregator.h | 2 +- dbms/src/Interpreters/Join.h | 2 +- dbms/src/Interpreters/SetVariants.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 882ac6fbd6d..b8c31c9f346 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -264,7 +264,7 @@ struct AggregationMethodString ALWAYS_INLINE Key getKey( const ColumnRawPtrs & /*key_columns*/, size_t /*keys_size*/, - size_t i, + ssize_t i, const Sizes & /*key_sizes*/, StringRefs & /*keys*/, Arena & /*pool*/) const diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 7bd68d593e1..59181876e6d 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -70,7 +70,7 @@ struct JoinKeyGetterString Key getKey( const ColumnRawPtrs &, size_t, - size_t i, + ssize_t i, const Sizes &) const { return StringRef( diff --git a/dbms/src/Interpreters/SetVariants.h b/dbms/src/Interpreters/SetVariants.h index da1dc2140ba..eaf9896b01c 100644 --- a/dbms/src/Interpreters/SetVariants.h +++ b/dbms/src/Interpreters/SetVariants.h @@ -81,7 +81,7 @@ struct SetMethodString Key getKey( const ColumnRawPtrs &, size_t, - size_t i, + ssize_t i, const Sizes &) const { return StringRef( From dae73f4d32a61d8eda4c58771546b6de9db4084c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 23:44:07 +0300 Subject: [PATCH 206/230] Fixed tests that was depend on timezone #3948 --- .../queries/0_stateless/00502_sum_map.sql | 2 +- .../00506_shard_global_in_union.sql | 2 +- .../0_stateless/00735_conditional.reference | 12 +- .../queries/0_stateless/00735_conditional.sql | 104 +++++++++--------- 4 files changed, 60 insertions(+), 60 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00502_sum_map.sql b/dbms/tests/queries/0_stateless/00502_sum_map.sql index 39fb1f04333..e6377155dac 100644 --- a/dbms/tests/queries/0_stateless/00502_sum_map.sql +++ b/dbms/tests/queries/0_stateless/00502_sum_map.sql @@ -18,7 +18,7 @@ select sumMap(val, cnt) from ( SELECT [ CAST(1, 'UInt64') ] as val, [1] as cnt ) select sumMap(val, cnt) from ( SELECT [ CAST(1, 'Float64') ] as val, [1] as cnt ); select sumMap(val, cnt) from ( SELECT [ CAST('a', 'Enum16(\'a\'=1)') ] as val, [1] as cnt ); -select sumMap(val, cnt) from ( SELECT [ CAST(1, 'DateTime') ] as val, [1] as cnt ); +select sumMap(val, cnt) from ( SELECT [ CAST(1, 'DateTime(\'Europe/Moscow\')') ] as val, [1] as cnt ); select sumMap(val, cnt) from ( SELECT [ CAST(1, 'Date') ] as val, [1] as cnt ); select sumMap(val, cnt) from ( SELECT [ CAST('01234567-89ab-cdef-0123-456789abcdef', 'UUID') ] as val, [1] as cnt ); select sumMap(val, cnt) from ( SELECT [ CAST(1.01, 'Decimal(10,2)') ] as val, [1] as cnt ); diff --git a/dbms/tests/queries/0_stateless/00506_shard_global_in_union.sql b/dbms/tests/queries/0_stateless/00506_shard_global_in_union.sql index 241cde60356..8bc65ee9e64 100644 --- a/dbms/tests/queries/0_stateless/00506_shard_global_in_union.sql +++ b/dbms/tests/queries/0_stateless/00506_shard_global_in_union.sql @@ -21,7 +21,7 @@ DROP TABLE IF EXISTS test.union_bug; CREATE TABLE test.union_bug ( Event String, - Datetime DateTime + Datetime DateTime('Europe/Moscow') ) Engine = Memory; INSERT INTO test.union_bug VALUES ('A', 1), ('B', 2); diff --git a/dbms/tests/queries/0_stateless/00735_conditional.reference b/dbms/tests/queries/0_stateless/00735_conditional.reference index bdea753dc4f..5601ae3784e 100644 --- a/dbms/tests/queries/0_stateless/00735_conditional.reference +++ b/dbms/tests/queries/0_stateless/00735_conditional.reference @@ -68,9 +68,9 @@ value vs value 0 1 1 UInt64 UInt32 UInt64 0 1 1 UInt64 UInt64 UInt64 0000-00-00 1970-01-02 1970-01-02 Date Date Date -0000-00-00 1970-01-01 03:00:01 1970-01-01 03:00:01 Date DateTime DateTime -0000-00-00 00:00:00 1970-01-02 1970-01-01 03:00:01 DateTime Date DateTime -0000-00-00 00:00:00 1970-01-01 03:00:01 1970-01-01 03:00:01 DateTime DateTime DateTime +0000-00-00 1970-01-01 03:00:01 1970-01-01 03:00:01 Date DateTime(\'Europe/Moscow\') DateTime +0000-00-00 00:00:00 1970-01-02 1970-01-01 03:00:01 DateTime(\'Europe/Moscow\') Date DateTime +0000-00-00 00:00:00 1970-01-01 03:00:01 1970-01-01 03:00:01 DateTime(\'Europe/Moscow\') DateTime(\'Europe/Moscow\') DateTime(\'Europe/Moscow\') 00000000-0000-0000-0000-000000000000 00000000-0000-0001-0000-000000000000 00000000-0000-0001-0000-000000000000 UUID UUID UUID column vs value 0 1 1 Int8 Int8 Int8 @@ -142,7 +142,7 @@ column vs value 0 1 1 UInt64 UInt32 UInt64 0 1 1 UInt64 UInt64 UInt64 0000-00-00 1970-01-02 1970-01-02 Date Date Date -0000-00-00 1970-01-01 03:00:01 1970-01-01 03:00:01 Date DateTime DateTime -0000-00-00 00:00:00 1970-01-02 1970-01-01 03:00:01 DateTime Date DateTime -0000-00-00 00:00:00 1970-01-01 03:00:01 1970-01-01 03:00:01 DateTime DateTime DateTime +0000-00-00 1970-01-01 03:00:01 1970-01-01 03:00:01 Date DateTime(\'Europe/Moscow\') DateTime +0000-00-00 00:00:00 1970-01-02 1970-01-01 03:00:01 DateTime(\'Europe/Moscow\') Date DateTime +0000-00-00 00:00:00 1970-01-01 03:00:01 1970-01-01 03:00:01 DateTime(\'Europe/Moscow\') DateTime(\'Europe/Moscow\') DateTime(\'Europe/Moscow\') 00000000-0000-0000-0000-000000000000 00000000-0000-0001-0000-000000000000 00000000-0000-0001-0000-000000000000 UUID UUID UUID diff --git a/dbms/tests/queries/0_stateless/00735_conditional.sql b/dbms/tests/queries/0_stateless/00735_conditional.sql index 3ab2f8f9e75..c8cae5a36aa 100644 --- a/dbms/tests/queries/0_stateless/00735_conditional.sql +++ b/dbms/tests/queries/0_stateless/00735_conditional.sql @@ -13,7 +13,7 @@ SELECT toInt8(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toInt8(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt8(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt8(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT toInt8(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt8(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt8(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt8(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT toInt8(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -30,7 +30,7 @@ SELECT toInt16(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toInt16(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt16(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt16(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT toInt16(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt16(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt16(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt16(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT toInt16(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -47,7 +47,7 @@ SELECT toInt32(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toInt32(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt32(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toInt32(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT toInt32(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt32(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt32(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt32(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT toInt32(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -64,7 +64,7 @@ SELECT toInt64(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toInt64(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt64(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt64(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT toInt64(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toInt64(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt64(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toInt64(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT toInt64(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -81,7 +81,7 @@ SELECT toUInt8(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toUInt8(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt8(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt8(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT toUInt8(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt8(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt8(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt8(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT toUInt8(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -98,7 +98,7 @@ SELECT toUInt16(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x) SELECT toUInt16(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt16(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt16(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT toUInt16(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt16(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt16(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt16(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT toUInt16(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -115,7 +115,7 @@ SELECT toUInt32(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x) SELECT toUInt32(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt32(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUInt32(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT toUInt32(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt32(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt32(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt32(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT toUInt32(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -132,7 +132,7 @@ SELECT toUInt64(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x) SELECT toUInt64(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt64(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt64(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT toUInt64(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUInt64(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt64(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUInt64(0) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT toUInt64(0) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -149,28 +149,28 @@ SELECT toDate(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toDate(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toDate(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toDate(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT toDate(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toDate(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toDate(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toDate(0) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toDate(0) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT toDate(0) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT toDateTime(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDateTime(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDateTime(0) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDateTime(0) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDateTime(0) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDateTime(0) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDateTime(0) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDateTime(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDateTime(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDateTime(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDateTime(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT toDateTime(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT toDateTime(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDateTime(0) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDateTime(0) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT toDateTime(0) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0, 'Europe/Moscow') AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0, 'Europe/Moscow') AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0, 'Europe/Moscow') AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0, 'Europe/Moscow') AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0, 'Europe/Moscow') AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0, 'Europe/Moscow') AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0, 'Europe/Moscow') AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0, 'Europe/Moscow') AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0, 'Europe/Moscow') AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0, 'Europe/Moscow') AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0, 'Europe/Moscow') AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toDateTime(0, 'Europe/Moscow') AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT toDateTime(0, 'Europe/Moscow') AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0, 'Europe/Moscow') AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0, 'Europe/Moscow') AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toDateTime(0, 'Europe/Moscow') AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUUID(0) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUUID(0) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } @@ -183,7 +183,7 @@ SELECT toUUID(0) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), SELECT toUUID(0) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUUID(0) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUUID(0) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT toUUID(0) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT toUUID(0) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUUID(0) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT toUUID(0) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT toUUID(0) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } @@ -202,7 +202,7 @@ SELECT materialize(toInt8(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, to SELECT materialize(toInt8(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toInt8(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toInt8(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT materialize(toInt8(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt8(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt8(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt8(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT materialize(toInt8(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -219,7 +219,7 @@ SELECT materialize(toInt16(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, t SELECT materialize(toInt16(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toInt16(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toInt16(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT materialize(toInt16(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt16(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt16(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt16(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT materialize(toInt16(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -236,7 +236,7 @@ SELECT materialize(toInt32(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, t SELECT materialize(toInt32(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toInt32(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toInt32(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT materialize(toInt32(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt32(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt32(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt32(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT materialize(toInt32(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -253,7 +253,7 @@ SELECT materialize(toInt64(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, t SELECT materialize(toInt64(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt64(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt64(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT materialize(toInt64(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toInt64(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt64(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toInt64(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT materialize(toInt64(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -270,7 +270,7 @@ SELECT materialize(toUInt8(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, t SELECT materialize(toUInt8(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt8(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt8(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT materialize(toUInt8(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt8(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt8(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt8(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT materialize(toUInt8(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -287,7 +287,7 @@ SELECT materialize(toUInt16(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, SELECT materialize(toUInt16(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt16(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt16(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT materialize(toUInt16(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt16(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt16(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt16(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT materialize(toUInt16(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -304,7 +304,7 @@ SELECT materialize(toUInt32(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, SELECT materialize(toUInt32(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt32(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUInt32(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT materialize(toUInt32(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt32(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt32(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt32(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT materialize(toUInt32(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -321,7 +321,7 @@ SELECT materialize(toUInt64(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, SELECT materialize(toUInt64(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt64(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt64(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT materialize(toUInt64(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUInt64(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt64(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUInt64(0)) AS x, toDecimal32(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } SELECT materialize(toUInt64(0)) AS x, toDecimal64(1, 0) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 48 } @@ -338,28 +338,28 @@ SELECT materialize(toDate(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, to SELECT materialize(toDate(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toDate(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toDate(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT materialize(toDate(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toDate(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toDate(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toDate(0)) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toDate(0)) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } SELECT materialize(toDate(0)) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT materialize(toDateTime(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDateTime(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDateTime(0)) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDateTime(0)) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDateTime(0)) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDateTime(0)) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDateTime(0)) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDateTime(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDateTime(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDateTime(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDateTime(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT materialize(toDateTime(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -SELECT materialize(toDateTime(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDateTime(0)) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDateTime(0)) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } -SELECT materialize(toDateTime(0)) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toUInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toUInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toUInt32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toDateTime(0, 'Europe/Moscow')) AS x, toDecimal128(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUUID(0)) AS x, toInt8(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUUID(0)) AS x, toInt16(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } @@ -372,7 +372,7 @@ SELECT materialize(toUUID(0)) AS x, toUInt64(1) AS y, ((x > y) ? x : y) AS z, to SELECT materialize(toUUID(0)) AS x, toFloat32(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUUID(0)) AS x, toFloat64(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUUID(0)) AS x, toDate(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 43 } -SELECT materialize(toUUID(0)) AS x, toDateTime(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } +SELECT materialize(toUUID(0)) AS x, toDateTime(1, 'Europe/Moscow') AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUUID(0)) AS x, toUUID(1) AS y, ((x > y) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); SELECT materialize(toUUID(0)) AS x, toDecimal32(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } SELECT materialize(toUUID(0)) AS x, toDecimal64(1, 0) AS y, ((x = 0) ? x : y) AS z, toTypeName(x), toTypeName(y), toTypeName(z); -- { serverError 386 } From 4e2c1132d9ca1f55da342e858801d24560f7423d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Dec 2018 23:48:21 +0300 Subject: [PATCH 207/230] Fixed UB #3920 --- dbms/src/Columns/ColumnString.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Columns/ColumnString.h b/dbms/src/Columns/ColumnString.h index 73c1ae35d37..6b2e58d54d1 100644 --- a/dbms/src/Columns/ColumnString.h +++ b/dbms/src/Columns/ColumnString.h @@ -153,7 +153,8 @@ public: const size_t new_size = old_size + length + 1; chars.resize(new_size); - memcpy(&chars[old_size], pos, length); + if (length) + memcpy(&chars[old_size], pos, length); chars[old_size + length] = 0; offsets.push_back(new_size); } From 237a9247cbd81201331f7ee12885150cb0560103 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Dec 2018 00:14:32 +0300 Subject: [PATCH 208/230] Fixed UB #3920 --- dbms/src/Interpreters/Join.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 59181876e6d..65371239851 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -80,7 +80,8 @@ struct JoinKeyGetterString static void onNewKey(Key & key, Arena & pool) { - key.data = pool.insert(key.data, key.size); + if (key.size) + key.data = pool.insert(key.data, key.size); } }; From 928649004ccfef7c6db1d2fe30b7c92610c4b11a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Dec 2018 00:15:47 +0300 Subject: [PATCH 209/230] Fixed UB #3920 --- dbms/src/Interpreters/SetVariants.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/SetVariants.h b/dbms/src/Interpreters/SetVariants.h index eaf9896b01c..f241ddf4a00 100644 --- a/dbms/src/Interpreters/SetVariants.h +++ b/dbms/src/Interpreters/SetVariants.h @@ -92,7 +92,8 @@ struct SetMethodString static void onNewKey(typename Data::value_type & value, size_t, Arena & pool) { - value.data = pool.insert(value.data, value.size); + if (value.size) + value.data = pool.insert(value.data, value.size); } }; From f17095b146dd994c2314b8b384e5ef8f372e950e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Dec 2018 02:53:09 +0300 Subject: [PATCH 210/230] Updated instruction #3948 --- dbms/tests/instructions/sanitizers.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/tests/instructions/sanitizers.md b/dbms/tests/instructions/sanitizers.md index 65e35084a4a..e934ece7f10 100644 --- a/dbms/tests/instructions/sanitizers.md +++ b/dbms/tests/instructions/sanitizers.md @@ -53,9 +53,11 @@ sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1' ./clickhouse-tsan server --con # How to use Undefined Behaviour Sanitizer ``` -mkdir build_ubsan && cd build_ubsan +CC=clang CXX=clang++ mkdir build_ubsan && cd build_ubsan ``` +Note: clang is mandatory, because gcc (in version 8) has false positives due to devirtualization and it has less amount of checks. + ``` cmake -D SANITIZE=undefined .. ninja From eb457f2bd2de002a57dbae08468989dfde23c936 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 Dec 2018 18:22:38 +0300 Subject: [PATCH 211/230] Do not copy dictionary in ColumnLowCardinality::cloneEmpty --- dbms/src/Columns/ColumnLowCardinality.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/src/Columns/ColumnLowCardinality.cpp b/dbms/src/Columns/ColumnLowCardinality.cpp index ca34d0b3bb4..5ac16763b76 100644 --- a/dbms/src/Columns/ColumnLowCardinality.cpp +++ b/dbms/src/Columns/ColumnLowCardinality.cpp @@ -236,6 +236,9 @@ void ColumnLowCardinality::gather(ColumnGathererStream & gatherer) MutableColumnPtr ColumnLowCardinality::cloneResized(size_t size) const { auto unique_ptr = dictionary.getColumnUniquePtr(); + if (size == 0) + unique_ptr = unique_ptr->cloneEmpty(); + return ColumnLowCardinality::create((*std::move(unique_ptr)).mutate(), getIndexes().cloneResized(size)); } From bae0370954840ac5006ad1b99e8bd410af4f982f Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 28 Dec 2018 18:38:58 +0300 Subject: [PATCH 212/230] fix crash in JoinToSubqueryTransformVisitor (wrong casts) --- .../JoinToSubqueryTransformVisitor.cpp | 21 ++++++++++--------- .../0_stateless/00820_multiple_joins.sql | 6 +----- 2 files changed, 12 insertions(+), 15 deletions(-) diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 1520711aa68..80c670c8794 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -57,7 +58,7 @@ struct RewriteTablesVisitorData static bool needRewrite(ASTSelectQuery & select) { - auto tables = static_cast(select.tables.get()); + auto tables = typeid_cast(select.tables.get()); if (!tables) return false; @@ -70,11 +71,11 @@ static bool needRewrite(ASTSelectQuery & select) static void appendTableNameAndAlias(std::vector & hidden, const ASTPtr & table_element) { - auto element = static_cast(table_element.get()); + auto element = typeid_cast(table_element.get()); if (!element || element->children.empty()) throw Exception("Expected TablesInSelectQueryElement with at least one child", ErrorCodes::LOGICAL_ERROR); - auto table_expression = static_cast(element->children[0].get()); + auto table_expression = typeid_cast(element->children[0].get()); if (!table_expression || table_expression->children.empty()) throw Exception("Expected TableExpression with at least one child", ErrorCodes::LOGICAL_ERROR); @@ -82,10 +83,10 @@ static void appendTableNameAndAlias(std::vector & hidden, const ASTPtr & if (!alias.empty()) hidden.push_back(alias); - auto identifier = static_cast(table_expression->children[0].get()); - if (!identifier && alias.empty()) + if (auto * identifier = typeid_cast(table_expression->children[0].get())) + hidden.push_back(identifier->name); + else if (alias.empty()) throw Exception("Expected Identifier or subquery with alias", ErrorCodes::LOGICAL_ERROR); - hidden.push_back(identifier->name); } @@ -103,7 +104,7 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast if (!needRewrite(select)) return; - auto tables = static_cast(select.tables.get()); + auto tables = typeid_cast(select.tables.get()); if (!tables) throw Exception("TablesInSelectQuery expected", ErrorCodes::LOGICAL_ERROR); @@ -141,15 +142,15 @@ ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTSelectQuery & select, ASTP OneTypeMatcher>; using RewriteVisitor = InDepthNodeVisitor; - auto left = static_cast(ast_left.get()); - auto right = static_cast(ast_right.get()); + auto left = typeid_cast(ast_left.get()); + auto right = typeid_cast(ast_right.get()); if (!left || !right) throw Exception("Two TablesInSelectQueryElements expected", ErrorCodes::LOGICAL_ERROR); if (!right->table_join || right->array_join) return {}; - auto table_join = static_cast(right->table_join.get()); + auto table_join = typeid_cast(right->table_join.get()); if (table_join->kind != ASTTableJoin::Kind::Inner) return {}; diff --git a/dbms/tests/queries/0_stateless/00820_multiple_joins.sql b/dbms/tests/queries/0_stateless/00820_multiple_joins.sql index 3885c7d3535..66b594f917f 100644 --- a/dbms/tests/queries/0_stateless/00820_multiple_joins.sql +++ b/dbms/tests/queries/0_stateless/00820_multiple_joins.sql @@ -20,11 +20,7 @@ SET allow_experimental_multiple_joins_emulation = 1; -- FIXME: wrong names qualification select a, b, c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on b = t3.b; select a, b, c from table1 as t1 join table2 as t2 on t1.a = t2.a join table5 as t5 on a = t5.a AND b = t5.b; ---select a, b, c, d from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on b = t3.b join table5 as t5 on c = t5.c; - ---select t1.x, t2.y, t3.z from table1 as t1 join table2 as t2 on t1.x = t2.x join table3 as t3 on t2.y = t3.y; ---select t1.x, t2.y, t3.z from table1 as t1 join (select * from table2 as t2 join table3 as t3 on t2.y = t3.y) on t1.x = t2.x; ---select t1.x, j1.y, j1.z from table1 as t1 join (select * from table2 as t2 join table3 as t3 on t2.y = t3.y) as j1 on t1.x = j1.x; +--select a, b, c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on b = t3.b join table5 as t5 on c = t5.c; DROP TABLE table1; DROP TABLE table2; From e21f414a34d6f49bf81a775acb28b8123b51ea70 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Fri, 28 Dec 2018 19:05:39 +0300 Subject: [PATCH 213/230] Describe how to get build log in chrome-trace format --- dbms/tests/instructions/ninja_trace.txt | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 dbms/tests/instructions/ninja_trace.txt diff --git a/dbms/tests/instructions/ninja_trace.txt b/dbms/tests/instructions/ninja_trace.txt new file mode 100644 index 00000000000..d36573e4a9c --- /dev/null +++ b/dbms/tests/instructions/ninja_trace.txt @@ -0,0 +1,5 @@ +To obtain the build trace in Chrome-trace format you need to do the following: +- ninja -C ${BUILD_DIR} +- ninjatrace ${BUILD_DIR}/.ninja_log > ninja_trace.json + +You can get `ninjatrace` from https://github.com/nico/ninjatracing.git - also there is a more detailed instruction. From 8bc00f4632d943e55dcb60a1b85c64b79635b538 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 28 Dec 2018 16:39:44 +0300 Subject: [PATCH 214/230] CLICKHOUSE-4229: Fix user forwarding for replicated queries which actually send queries to master node --- .../src/Interpreters/InterpreterDropQuery.cpp | 4 +- dbms/src/Storages/IStorage.h | 2 +- dbms/src/Storages/StorageDistributed.cpp | 2 +- dbms/src/Storages/StorageDistributed.h | 2 +- dbms/src/Storages/StorageJoin.cpp | 2 +- dbms/src/Storages/StorageJoin.h | 2 +- dbms/src/Storages/StorageLog.cpp | 2 +- dbms/src/Storages/StorageLog.h | 2 +- dbms/src/Storages/StorageMaterializedView.cpp | 2 +- dbms/src/Storages/StorageMaterializedView.h | 2 +- dbms/src/Storages/StorageMemory.cpp | 2 +- dbms/src/Storages/StorageMemory.h | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 2 +- dbms/src/Storages/StorageMergeTree.h | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 78 ++++++++++--------- .../src/Storages/StorageReplicatedMergeTree.h | 20 ++--- dbms/src/Storages/StorageSet.cpp | 2 +- dbms/src/Storages/StorageSet.h | 2 +- dbms/src/Storages/StorageStripeLog.cpp | 2 +- dbms/src/Storages/StorageStripeLog.h | 2 +- dbms/src/Storages/StorageTinyLog.cpp | 2 +- dbms/src/Storages/StorageTinyLog.h | 2 +- dbms/tests/integration/helpers/test_tools.py | 4 +- .../__init__.py | 0 .../configs/remote_servers.xml | 19 +++++ .../configs/users_local_default.xml | 39 ++++++++++ .../test.py | 63 +++++++++++++++ 27 files changed, 195 insertions(+), 70 deletions(-) create mode 100644 dbms/tests/integration/test_send_request_to_leader_replica/__init__.py create mode 100644 dbms/tests/integration/test_send_request_to_leader_replica/configs/remote_servers.xml create mode 100644 dbms/tests/integration/test_send_request_to_leader_replica/configs/users_local_default.xml create mode 100644 dbms/tests/integration/test_send_request_to_leader_replica/test.py diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 2c386718f98..c3af6d7fa0a 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -80,7 +80,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t /// If table was already dropped by anyone, an exception will be thrown auto table_lock = database_and_table.second->lockForAlter(); /// Drop table data, don't touch metadata - database_and_table.second->truncate(query_ptr); + database_and_table.second->truncate(query_ptr, context); } else if (kind == ASTDropQuery::Kind::Drop) { @@ -128,7 +128,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(String & table_name, ASTDr /// If table was already dropped by anyone, an exception will be thrown auto table_lock = table->lockForAlter(); /// Drop table data, don't touch metadata - table->truncate(query_ptr); + table->truncate(query_ptr, context); } else if (kind == ASTDropQuery::Kind::Drop) { diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 23f56b47a8f..07637acee87 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -214,7 +214,7 @@ public: /** Clear the table data and leave it empty. * Must be called under lockForAlter. */ - virtual void truncate(const ASTPtr & /*query*/) + virtual void truncate(const ASTPtr & /*query*/, const Context & /* context */) { throw Exception("Truncate is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index f681565d40e..61f9534d8ac 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -360,7 +360,7 @@ void StorageDistributed::shutdown() } -void StorageDistributed::truncate(const ASTPtr &) +void StorageDistributed::truncate(const ASTPtr &, const Context &) { std::lock_guard lock(cluster_nodes_mutex); diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index e14d9f7081f..91aa82ff4bf 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -76,7 +76,7 @@ public: void drop() override {} /// Removes temporary data in local filesystem. - void truncate(const ASTPtr &) override; + void truncate(const ASTPtr &, const Context &) override; void rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & new_table_name) override { table_name = new_table_name; } /// in the sub-tables, you need to manually add and delete columns diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index e05146ad431..97cf689f8db 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -51,7 +51,7 @@ StorageJoin::StorageJoin( } -void StorageJoin::truncate(const ASTPtr &) +void StorageJoin::truncate(const ASTPtr &, const Context &) { Poco::File(path).remove(true); Poco::File(path).createDirectories(); diff --git a/dbms/src/Storages/StorageJoin.h b/dbms/src/Storages/StorageJoin.h index 25c5128a349..ebe9dc68ce4 100644 --- a/dbms/src/Storages/StorageJoin.h +++ b/dbms/src/Storages/StorageJoin.h @@ -25,7 +25,7 @@ class StorageJoin : public ext::shared_ptr_helper, public StorageSe public: String getName() const override { return "Join"; } - void truncate(const ASTPtr &) override; + void truncate(const ASTPtr &, const Context &) override; /// Access the innards. JoinPtr & getJoin() { return join; } diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 462f20ad582..19ff9831f29 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -520,7 +520,7 @@ void StorageLog::rename(const String & new_path_to_db, const String & /*new_data marks_file = Poco::File(path + escapeForFileName(name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME); } -void StorageLog::truncate(const ASTPtr &) +void StorageLog::truncate(const ASTPtr &, const Context &) { std::shared_lock lock(rwlock); diff --git a/dbms/src/Storages/StorageLog.h b/dbms/src/Storages/StorageLog.h index 91a03670e59..a808bdec1f0 100644 --- a/dbms/src/Storages/StorageLog.h +++ b/dbms/src/Storages/StorageLog.h @@ -40,7 +40,7 @@ public: bool checkData() const override; - void truncate(const ASTPtr &) override; + void truncate(const ASTPtr &, const Context &) override; std::string full_path() const { return path + escapeForFileName(name) + '/';} diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index b73ee12b473..5df31fbab4d 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -229,7 +229,7 @@ void StorageMaterializedView::drop() executeDropQuery(ASTDropQuery::Kind::Drop, global_context, target_database_name, target_table_name); } -void StorageMaterializedView::truncate(const ASTPtr &) +void StorageMaterializedView::truncate(const ASTPtr &, const Context &) { if (has_inner_table) executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, target_database_name, target_table_name); diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index 176a95c2d03..d308bd3550a 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -31,7 +31,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override; void drop() override; - void truncate(const ASTPtr &) override; + void truncate(const ASTPtr &, const Context &) override; bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override; diff --git a/dbms/src/Storages/StorageMemory.cpp b/dbms/src/Storages/StorageMemory.cpp index ac8f6b79f74..631071d2316 100644 --- a/dbms/src/Storages/StorageMemory.cpp +++ b/dbms/src/Storages/StorageMemory.cpp @@ -127,7 +127,7 @@ void StorageMemory::drop() data.clear(); } -void StorageMemory::truncate(const ASTPtr &) +void StorageMemory::truncate(const ASTPtr &, const Context &) { std::lock_guard lock(mutex); data.clear(); diff --git a/dbms/src/Storages/StorageMemory.h b/dbms/src/Storages/StorageMemory.h index 27c948bf87f..39bcce76894 100644 --- a/dbms/src/Storages/StorageMemory.h +++ b/dbms/src/Storages/StorageMemory.h @@ -40,7 +40,7 @@ public: void drop() override; - void truncate(const ASTPtr &) override; + void truncate(const ASTPtr &, const Context &) override; void rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & new_table_name) override { table_name = new_table_name; } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index ccdb8dd6c51..afdd3a449f0 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -157,7 +157,7 @@ void StorageMergeTree::drop() data.dropAllData(); } -void StorageMergeTree::truncate(const ASTPtr &) +void StorageMergeTree::truncate(const ASTPtr &, const Context &) { { /// Asks to complete merges and does not allow them to start. diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index c0a2b8b5463..0a458eb761b 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -67,7 +67,7 @@ public: std::vector getMutationsStatus() const; void drop() override; - void truncate(const ASTPtr &) override; + void truncate(const ASTPtr &, const Context &) override; void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 3785d6b7a72..05076521787 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2934,13 +2934,13 @@ BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, } -bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) +bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & query_context) { assertNotReadonly(); if (!is_leader) { - sendRequestToLeaderReplica(query, context.getSettingsRef()); + sendRequestToLeaderReplica(query, query_context); return true; } @@ -2955,7 +2955,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p auto handle_noop = [&] (const String & message) { - if (context.getSettingsRef().optimize_throw_if_noop) + if (query_context.getSettingsRef().optimize_throw_if_noop) throw Exception(message, ErrorCodes::CANNOT_ASSIGN_OPTIMIZE); return false; }; @@ -2993,7 +2993,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p else { UInt64 disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path); - String partition_id = data.getPartitionIDFromQuery(partition, context); + String partition_id = data.getPartitionIDFromQuery(partition, query_context); selected = merger_mutator.selectAllPartsToMergeWithinPartition( future_merged_part, disk_space, can_merge, partition_id, final, &disable_reason); } @@ -3010,7 +3010,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p } /// TODO: Bad setting name for such purpose - if (context.getSettingsRef().replication_alter_partitions_sync != 0) + if (query_context.getSettingsRef().replication_alter_partitions_sync != 0) waitForAllReplicasToProcessLogEntry(merge_entry); return true; @@ -3018,7 +3018,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p void StorageReplicatedMergeTree::alter(const AlterCommands & params, - const String & /*database_name*/, const String & /*table_name*/, const Context & context) + const String & /*database_name*/, const String & /*table_name*/, const Context & query_context) { assertNotReadonly(); @@ -3113,7 +3113,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, std::set inactive_replicas; std::set timed_out_replicas; - time_t replication_alter_columns_timeout = context.getSettingsRef().replication_alter_columns_timeout; + time_t replication_alter_columns_timeout = query_context.getSettingsRef().replication_alter_columns_timeout; for (const String & replica : replicas) { @@ -3277,7 +3277,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, LOG_DEBUG(log, "ALTER finished"); } -void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) +void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & query_context) { for (const PartitionCommand & command : commands) { @@ -3285,46 +3285,46 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part { case PartitionCommand::DROP_PARTITION: checkPartitionCanBeDropped(command.partition); - dropPartition(query, command.partition, command.detach, context); + dropPartition(query, command.partition, command.detach, query_context); break; case PartitionCommand::ATTACH_PARTITION: - attachPartition(command.partition, command.part, context); + attachPartition(command.partition, command.part, query_context); break; case PartitionCommand::REPLACE_PARTITION: { checkPartitionCanBeDropped(command.partition); - String from_database = command.from_database.empty() ? context.getCurrentDatabase() : command.from_database; - auto from_storage = context.getTable(from_database, command.from_table); - replacePartitionFrom(from_storage, command.partition, command.replace, context); + String from_database = command.from_database.empty() ? query_context.getCurrentDatabase() : command.from_database; + auto from_storage = query_context.getTable(from_database, command.from_table); + replacePartitionFrom(from_storage, command.partition, command.replace, query_context); } break; case PartitionCommand::FETCH_PARTITION: - fetchPartition(command.partition, command.from_zookeeper_path, context); + fetchPartition(command.partition, command.from_zookeeper_path, query_context); break; case PartitionCommand::FREEZE_PARTITION: { auto lock = lockStructure(false); - data.freezePartition(command.partition, command.with_name, context); + data.freezePartition(command.partition, command.with_name, query_context); } break; case PartitionCommand::CLEAR_COLUMN: - clearColumnInPartition(command.partition, command.column_name, context); + clearColumnInPartition(command.partition, command.column_name, query_context); break; case PartitionCommand::FREEZE_ALL_PARTITIONS: { auto lock = lockStructure(false); - data.freezeAll(command.with_name, context); + data.freezeAll(command.with_name, query_context); } break; default: - IStorage::alterPartition(query, commands, context); // should throw an exception. + IStorage::alterPartition(query, commands, query_context); // should throw an exception. } } } @@ -3382,13 +3382,13 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St void StorageReplicatedMergeTree::clearColumnInPartition( - const ASTPtr & partition, const Field & column_name, const Context & context) + const ASTPtr & partition, const Field & column_name, const Context & query_context) { assertNotReadonly(); /// We don't block merges, so anyone can manage this task (not only leader) - String partition_id = data.getPartitionIDFromQuery(partition, context); + String partition_id = data.getPartitionIDFromQuery(partition, query_context); MergeTreePartInfo drop_range_info; if (!getFakePartCoveringAllPartsInPartition(partition_id, drop_range_info)) @@ -3409,9 +3409,9 @@ void StorageReplicatedMergeTree::clearColumnInPartition( entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); /// If necessary, wait until the operation is performed on itself or on all replicas. - if (context.getSettingsRef().replication_alter_partitions_sync != 0) + if (query_context.getSettingsRef().replication_alter_partitions_sync != 0) { - if (context.getSettingsRef().replication_alter_partitions_sync == 1) + if (query_context.getSettingsRef().replication_alter_partitions_sync == 1) waitForReplicaToProcessLogEntry(replica_name, entry); else waitForAllReplicasToProcessLogEntry(entry); @@ -3419,7 +3419,7 @@ void StorageReplicatedMergeTree::clearColumnInPartition( } -void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context) +void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & query_context) { assertNotReadonly(); @@ -3429,19 +3429,19 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPt { // TODO: we can manually reconstruct the query from outside the |dropPartition()| and remove the |query| argument from interface. // It's the only place where we need this argument. - sendRequestToLeaderReplica(query, context.getSettingsRef()); + sendRequestToLeaderReplica(query, query_context); return; } - String partition_id = data.getPartitionIDFromQuery(partition, context); + String partition_id = data.getPartitionIDFromQuery(partition, query_context); LogEntry entry; if (dropPartsInPartition(*zookeeper, partition_id, entry, detach)) { /// If necessary, wait until the operation is performed on itself or on all replicas. - if (context.getSettingsRef().replication_alter_partitions_sync != 0) + if (query_context.getSettingsRef().replication_alter_partitions_sync != 0) { - if (context.getSettingsRef().replication_alter_partitions_sync == 1) + if (query_context.getSettingsRef().replication_alter_partitions_sync == 1) waitForReplicaToProcessLogEntry(replica_name, entry); else waitForAllReplicasToProcessLogEntry(entry); @@ -3450,7 +3450,7 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPt } -void StorageReplicatedMergeTree::truncate(const ASTPtr & query) +void StorageReplicatedMergeTree::truncate(const ASTPtr & query, const Context & query_context) { assertNotReadonly(); @@ -3458,7 +3458,7 @@ void StorageReplicatedMergeTree::truncate(const ASTPtr & query) if (!is_leader) { - sendRequestToLeaderReplica(query, context.getSettingsRef()); + sendRequestToLeaderReplica(query, query_context); return; } @@ -3474,7 +3474,7 @@ void StorageReplicatedMergeTree::truncate(const ASTPtr & query) } -void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & context) +void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & query_context) { // TODO: should get some locks to prevent race with 'alter … modify column' @@ -3485,7 +3485,7 @@ void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool if (attach_part) partition_id = typeid_cast(*partition).value.safeGet(); else - partition_id = data.getPartitionIDFromQuery(partition, context); + partition_id = data.getPartitionIDFromQuery(partition, query_context); String source_dir = "detached/"; @@ -3886,7 +3886,7 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) /// TODO: Probably it is better to have queue in ZK with tasks for leader (like DDL) -void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query, const Settings & settings) +void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query, const Context & query_context) { auto live_replicas = getZooKeeper()->getChildren(zookeeper_path + "/leader_election"); if (live_replicas.empty()) @@ -3922,15 +3922,19 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query const Cluster::Address & address = findClusterAddress(leader_address); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context.getSettingsRef()); + + const auto & query_settings = query_context.getSettingsRef(); + const auto & query_client_info = query_context.getClientInfo(); + Connection connection( leader_address.host, leader_address.queries_port, leader_address.database, - context.getClientInfo().current_user, context.getClientInfo().current_password, timeouts, "ClickHouse replica"); + query_client_info.current_user, query_client_info.current_password, timeouts, "ClickHouse replica"); std::stringstream new_query_ss; formatAST(*new_query, new_query_ss, false, true); - RemoteBlockInputStream stream(connection, new_query_ss.str(), {}, context, &settings); + RemoteBlockInputStream stream(connection, new_query_ss.str(), {}, context, &query_settings); NullBlockOutputStream output({}); copyData(stream, output); @@ -4073,9 +4077,9 @@ void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, t } -void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const String & from_, const Context & context) +void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const String & from_, const Context & query_context) { - String partition_id = data.getPartitionIDFromQuery(partition, context); + String partition_id = data.getPartitionIDFromQuery(partition, query_context); String from = from_; if (from.back() == '/') @@ -4167,7 +4171,7 @@ void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const if (try_no) LOG_INFO(log, "Some of parts (" << missing_parts.size() << ") are missing. Will try to fetch covering parts."); - if (try_no >= context.getSettings().max_fetch_partition_retries_count) + if (try_no >= query_context.getSettings().max_fetch_partition_retries_count) throw Exception("Too many retries to fetch parts from " + best_replica_path, ErrorCodes::TOO_MANY_RETRIES_TO_FETCH_PARTS); Strings parts = getZooKeeper()->getChildren(best_replica_path + "/parts"); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index aeb378f7d8b..30a97667c28 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -113,11 +113,11 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override; - bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override; + bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & query_context) override; - void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) override; + void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & query_context) override; - void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) override; + void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & query_context) override; void mutate(const MutationCommands & commands, const Context & context) override; @@ -127,7 +127,7 @@ public: */ void drop() override; - void truncate(const ASTPtr &) override; + void truncate(const ASTPtr &, const Context &) override; void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; @@ -502,7 +502,7 @@ private: void waitForReplicaToProcessLogEntry(const String & replica_name, const ReplicatedMergeTreeLogEntryData & entry); /// Choose leader replica, send requst to it and wait. - void sendRequestToLeaderReplica(const ASTPtr & query, const Settings & settings); + void sendRequestToLeaderReplica(const ASTPtr & query, const Context & query_context); /// Throw an exception if the table is readonly. void assertNotReadonly() const; @@ -529,11 +529,11 @@ private: const Cluster::Address & findClusterAddress(const ReplicatedMergeTreeAddress & leader_address) const; // Partition helpers - void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context); - void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context); - void attachPartition(const ASTPtr & partition, bool part, const Context & context); - void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context); - void fetchPartition(const ASTPtr & partition, const String & from, const Context & context); + void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & query_context); + void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & query_context); + void attachPartition(const ASTPtr & partition, bool part, const Context & query_context); + void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & query_context); + void fetchPartition(const ASTPtr & partition, const String & from, const Context & query_context); protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. diff --git a/dbms/src/Storages/StorageSet.cpp b/dbms/src/Storages/StorageSet.cpp index 2348520674d..8a937893a35 100644 --- a/dbms/src/Storages/StorageSet.cpp +++ b/dbms/src/Storages/StorageSet.cpp @@ -119,7 +119,7 @@ void StorageSet::insertBlock(const Block & block) { set->insertFromBlock(block); size_t StorageSet::getSize() const { return set->getTotalRowCount(); } -void StorageSet::truncate(const ASTPtr &) +void StorageSet::truncate(const ASTPtr &, const Context &) { Poco::File(path).remove(true); Poco::File(path).createDirectories(); diff --git a/dbms/src/Storages/StorageSet.h b/dbms/src/Storages/StorageSet.h index 400e9670349..c5c7560e0f2 100644 --- a/dbms/src/Storages/StorageSet.h +++ b/dbms/src/Storages/StorageSet.h @@ -65,7 +65,7 @@ public: /// Access the insides. SetPtr & getSet() { return set; } - void truncate(const ASTPtr &) override; + void truncate(const ASTPtr &, const Context &) override; private: SetPtr set; diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index 9cf512880fe..1886f783dcb 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -288,7 +288,7 @@ bool StorageStripeLog::checkData() const return file_checker.check(); } -void StorageStripeLog::truncate(const ASTPtr &) +void StorageStripeLog::truncate(const ASTPtr &, const Context &) { if (name.empty()) throw Exception("Logical error: table name is empty", ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Storages/StorageStripeLog.h b/dbms/src/Storages/StorageStripeLog.h index 22010b39b63..0282235de51 100644 --- a/dbms/src/Storages/StorageStripeLog.h +++ b/dbms/src/Storages/StorageStripeLog.h @@ -53,7 +53,7 @@ public: String getDataPath() const override { return full_path(); } - void truncate(const ASTPtr &) override; + void truncate(const ASTPtr &, const Context &) override; private: String path; diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 5f66f0f5049..8c46e9a1ff1 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -407,7 +407,7 @@ bool StorageTinyLog::checkData() const return file_checker.check(); } -void StorageTinyLog::truncate(const ASTPtr &) +void StorageTinyLog::truncate(const ASTPtr &, const Context &) { if (name.empty()) throw Exception("Logical error: table name is empty", ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Storages/StorageTinyLog.h b/dbms/src/Storages/StorageTinyLog.h index c33c2c87d9a..b96570b4428 100644 --- a/dbms/src/Storages/StorageTinyLog.h +++ b/dbms/src/Storages/StorageTinyLog.h @@ -52,7 +52,7 @@ public: String getDataPath() const override { return full_path(); } - void truncate(const ASTPtr &) override; + void truncate(const ASTPtr &, const Context &) override; private: String path; diff --git a/dbms/tests/integration/helpers/test_tools.py b/dbms/tests/integration/helpers/test_tools.py index c4fbf075eb6..e90580de44c 100644 --- a/dbms/tests/integration/helpers/test_tools.py +++ b/dbms/tests/integration/helpers/test_tools.py @@ -28,13 +28,13 @@ def assert_eq_with_retry(instance, query, expectation, retry_count=20, sleep_tim expectation_tsv = TSV(expectation) for i in xrange(retry_count): try: - if TSV(instance.query(query)) == expectation_tsv: + if TSV(instance.query(query, user=user, stdin=stdin, timeout=timeout, settings=settings, ignore_error=ignore_error)) == expectation_tsv: break time.sleep(sleep_time) except Exception as ex: print "assert_eq_with_retry retry {} exception {}".format(i + 1, ex) time.sleep(sleep_time) else: - val = TSV(instance.query(query)) + val = TSV(instance.query(query, user=user, stdin=stdin, timeout=timeout, settings=settings, ignore_error=ignore_error)) if expectation_tsv != val: raise AssertionError("'{}' != '{}'\n{}".format(expectation_tsv, val, '\n'.join(expectation_tsv.diff(val, n1="expectation", n2="query")))) diff --git a/dbms/tests/integration/test_send_request_to_leader_replica/__init__.py b/dbms/tests/integration/test_send_request_to_leader_replica/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_send_request_to_leader_replica/configs/remote_servers.xml b/dbms/tests/integration/test_send_request_to_leader_replica/configs/remote_servers.xml new file mode 100644 index 00000000000..03e31203d8b --- /dev/null +++ b/dbms/tests/integration/test_send_request_to_leader_replica/configs/remote_servers.xml @@ -0,0 +1,19 @@ + + + + + true + + node1 + 9000 + awesome + + + node2 + 9000 + awesome + + + + + diff --git a/dbms/tests/integration/test_send_request_to_leader_replica/configs/users_local_default.xml b/dbms/tests/integration/test_send_request_to_leader_replica/configs/users_local_default.xml new file mode 100644 index 00000000000..ed27c17e5e6 --- /dev/null +++ b/dbms/tests/integration/test_send_request_to_leader_replica/configs/users_local_default.xml @@ -0,0 +1,39 @@ + + + + 10000000000 + 0 + random + + + 10000000000 + 0 + random + + + + + + + + 127.0.0.1 + ::1 + + default + default + + + + + ::/0 + + awesome + default + + + + + + + + diff --git a/dbms/tests/integration/test_send_request_to_leader_replica/test.py b/dbms/tests/integration/test_send_request_to_leader_replica/test.py new file mode 100644 index 00000000000..2cbfd8bbc7b --- /dev/null +++ b/dbms/tests/integration/test_send_request_to_leader_replica/test.py @@ -0,0 +1,63 @@ +import time + +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], user_configs=['configs/users_local_default.xml'], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], user_configs=['configs/users_local_default.xml'], with_zookeeper=True) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + + for node in [node1, node2]: + node.query(''' + CREATE TABLE sometable(date Date, id UInt32, value Int32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '{replica}', date, id, 8192); + '''.format(replica=node.name), user='awesome') + + + yield cluster + + finally: + cluster.shutdown() + +@pytest.mark.parametrize("query,expected", [ + ("ALTER TABLE sometable DROP PARTITION 201706", '1'), + ("TRUNCATE TABLE sometable", '0'), + ("OPTIMIZE TABLE sometable", '4'), +]) +def test_alter_table_drop_partition(started_cluster, query, expected): + to_insert = '''\ +2017-06-16 111 0 +2017-06-16 222 1 +2017-06-16 333 2 +2017-07-16 444 3 +''' + node1.query("INSERT INTO sometable FORMAT TSV", stdin=to_insert, user='awesome') + + assert_eq_with_retry(node1, "SELECT COUNT(*) from sometable", '4', user='awesome') + assert_eq_with_retry(node2, "SELECT COUNT(*) from sometable", '4', user='awesome') + + ### It maybe leader and everything will be ok + node1.query(query, user='awesome') + + assert_eq_with_retry(node1, "SELECT COUNT(*) from sometable", expected, user='awesome') + assert_eq_with_retry(node2, "SELECT COUNT(*) from sometable", expected, user='awesome') + + node1.query("INSERT INTO sometable FORMAT TSV", stdin=to_insert, user='awesome') + + assert_eq_with_retry(node1, "SELECT COUNT(*) from sometable", '4', user='awesome') + assert_eq_with_retry(node2, "SELECT COUNT(*) from sometable", '4', user='awesome') + + ### If node1 is leader than node2 will be slave + node2.query(query, user='awesome') + + assert_eq_with_retry(node1, "SELECT COUNT(*) from sometable", expected, user='awesome') + assert_eq_with_retry(node2, "SELECT COUNT(*) from sometable", expected, user='awesome') From ab32a06c85df3ebbc3db75ab8ac63a5dbe5c48d1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 28 Dec 2018 19:28:24 +0300 Subject: [PATCH 215/230] Fix code remove redundat files --- .../Storages/StorageReplicatedMergeTree.cpp | 17 +-- .../src/Storages/StorageReplicatedMergeTree.h | 3 +- .../__init__.py | 0 .../configs/node1.xml | 109 ---------------- .../configs/node2.xml | 117 ------------------ .../configs/node3.xml | 109 ---------------- .../configs/remote_servers.xml | 49 -------- .../test_replicated_drop_partition/test.py | 77 ------------ 8 files changed, 12 insertions(+), 469 deletions(-) delete mode 100644 dbms/tests/integration/test_replicated_drop_partition/__init__.py delete mode 100644 dbms/tests/integration/test_replicated_drop_partition/configs/node1.xml delete mode 100644 dbms/tests/integration/test_replicated_drop_partition/configs/node2.xml delete mode 100644 dbms/tests/integration/test_replicated_drop_partition/configs/node3.xml delete mode 100644 dbms/tests/integration/test_replicated_drop_partition/configs/remote_servers.xml delete mode 100644 dbms/tests/integration/test_replicated_drop_partition/test.py diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 05076521787..1b13a69c3e9 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3920,17 +3920,23 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query else throw Exception("Can't proxy this query. Unsupported query type", ErrorCodes::NOT_IMPLEMENTED); - const Cluster::Address & address = findClusterAddress(leader_address); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context.getSettingsRef()); const auto & query_settings = query_context.getSettingsRef(); const auto & query_client_info = query_context.getClientInfo(); + String user = query_client_info.current_user; + String password = query_client_info.current_password; + if (auto address = findClusterAddress(leader_address); address) + { + user = address->user; + password = address->password; + } Connection connection( leader_address.host, leader_address.queries_port, leader_address.database, - query_client_info.current_user, query_client_info.current_password, timeouts, "ClickHouse replica"); + user, password, timeouts, "ClickHouse replica"); std::stringstream new_query_ss; formatAST(*new_query, new_query_ss, false, true); @@ -3942,7 +3948,7 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query } -const Cluster::Address & StorageReplicatedMergeTree::findClusterAddress(const ReplicatedMergeTreeAddress & leader_address) const +std::optional StorageReplicatedMergeTree::findClusterAddress(const ReplicatedMergeTreeAddress & leader_address) const { for (auto & iter : context.getClusters().getContainer()) { @@ -3953,15 +3959,12 @@ const Cluster::Address & StorageReplicatedMergeTree::findClusterAddress(const Re for (size_t replica_num = 0; replica_num < shards[shard_num].size(); ++replica_num) { const Cluster::Address & address = shards[shard_num][replica_num]; - if (address.host_name == leader_address.host && address.port == leader_address.queries_port) - { return address; - } } } } - throw Exception("Not found replicate leader host " + leader_address.host + ":" + DB::toString(leader_address.queries_port) + ". Please check the 'remote_servers' section in your configuration file (config.xml, config.d, metrika.xml).", ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION); + return {}; } void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 30a97667c28..2ceb1eb2408 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -526,7 +526,8 @@ private: bool dropPartsInPartition(zkutil::ZooKeeper & zookeeper, String & partition_id, StorageReplicatedMergeTree::LogEntry & entry, bool detach); - const Cluster::Address & findClusterAddress(const ReplicatedMergeTreeAddress & leader_address) const; + /// Find cluster address for host + std::optional findClusterAddress(const ReplicatedMergeTreeAddress & leader_address) const; // Partition helpers void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & query_context); diff --git a/dbms/tests/integration/test_replicated_drop_partition/__init__.py b/dbms/tests/integration/test_replicated_drop_partition/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/integration/test_replicated_drop_partition/configs/node1.xml b/dbms/tests/integration/test_replicated_drop_partition/configs/node1.xml deleted file mode 100644 index a8edd566a09..00000000000 --- a/dbms/tests/integration/test_replicated_drop_partition/configs/node1.xml +++ /dev/null @@ -1,109 +0,0 @@ - - - - - - - - 10000000000 - - - 0 - - - random - - - - - 1 - - - - - - - - - pTe5Tb0s - - - - ::/0 - - - - default - - - default - - - - - - - ::1 - 127.0.0.1 - - readonly - default - - - - - - - - - - - 3600 - - - 0 - 0 - 0 - 0 - 0 - - - - diff --git a/dbms/tests/integration/test_replicated_drop_partition/configs/node2.xml b/dbms/tests/integration/test_replicated_drop_partition/configs/node2.xml deleted file mode 100644 index 2186d210a29..00000000000 --- a/dbms/tests/integration/test_replicated_drop_partition/configs/node2.xml +++ /dev/null @@ -1,117 +0,0 @@ - - - - - - - - 10000000000 - - - 0 - - - random - - - - - 1 - - - - - - - - - dtnDvTr9 - - - - ::/0 - - - - default - - - default - - - - azAUGBFl - - ::/0 - - default - default - - - - - - ::1 - 127.0.0.1 - - readonly - default - - - - - - - - - - - 3600 - - - 0 - 0 - 0 - 0 - 0 - - - - diff --git a/dbms/tests/integration/test_replicated_drop_partition/configs/node3.xml b/dbms/tests/integration/test_replicated_drop_partition/configs/node3.xml deleted file mode 100644 index d972d03b1c4..00000000000 --- a/dbms/tests/integration/test_replicated_drop_partition/configs/node3.xml +++ /dev/null @@ -1,109 +0,0 @@ - - - - - - - - 10000000000 - - - 0 - - - random - - - - - 1 - - - - - - - - - ROgXGTDq - - - - ::/0 - - - - default - - - default - - - - - - - ::1 - 127.0.0.1 - - readonly - default - - - - - - - - - - - 3600 - - - 0 - 0 - 0 - 0 - 0 - - - - diff --git a/dbms/tests/integration/test_replicated_drop_partition/configs/remote_servers.xml b/dbms/tests/integration/test_replicated_drop_partition/configs/remote_servers.xml deleted file mode 100644 index 866058d831a..00000000000 --- a/dbms/tests/integration/test_replicated_drop_partition/configs/remote_servers.xml +++ /dev/null @@ -1,49 +0,0 @@ - - - - - true - - node1 - 9000 - default - pTe5Tb0s - - - node2 - 9000 - default - dtnDvTr9 - - - - - - true - - node2 - 9000 - zhang - azAUGBFl - - - node3 - 9000 - default - ROgXGTDq - - - - - - ::/0 - - - - 10000000000 - 0.01 - lz4 - - - - diff --git a/dbms/tests/integration/test_replicated_drop_partition/test.py b/dbms/tests/integration/test_replicated_drop_partition/test.py deleted file mode 100644 index 9aacb55d369..00000000000 --- a/dbms/tests/integration/test_replicated_drop_partition/test.py +++ /dev/null @@ -1,77 +0,0 @@ -import pytest -import time -import sys - -from helpers.cluster import ClickHouseCluster -from helpers.network import PartitionManager - -from helpers.test_tools import assert_eq_with_retry - - -cluster = ClickHouseCluster(__file__) - -def _fill_nodes(nodes, users, passwords, cluster): - for i in range(0,len(nodes)): - node[i].query( - ''' - CREATE DATABASE IF NOT EXISTS test; - - CREATE TABLE test_table{cluster} (date Date, id UInt32, dummy UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{cluster}/replicated', '{replica}', toYYYYMMDD(date), id, 8192); - '''.format(cluster=cluster, replica=node.name), settings={"password": passwords[i]}, user=users[i]) - - -node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], user_configs=['configs/node1.xml'], with_zookeeper=True) -node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], user_configs=['configs/node2.xml'], with_zookeeper=True) -node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml'], user_configs=['configs/node3.xml'], with_zookeeper=True) - -@pytest.fixture(scope="module") -def normal_work(): - try: - cluster.start() - - _fill_nodes([node1, node2],[default,default],[pTe5Tb0s,dtnDvTr9],1) - _fill_nodes([node2, node3],[zhang,default],[azAUGBFl,ROgXGTDq],2) - - yield cluster - - finally: - cluster.shutdown() - -def test_normal_work(normal_work): - node1.query("insert into test_table1 values ('2017-06-16', 111, 0)",settings={"password": pTe5Tb0s}, user=default) - node1.query("insert into test_table1 values ('2017-06-16', 222, 0)",settings={"password": pTe5Tb0s}, user=default) - node1.query("insert into test_table1 values ('2017-06-17', 333, 0)",settings={"password": pTe5Tb0s}, user=default) - node1.query("insert into test_table1 values ('2017-06-17', 444, 0)",settings={"password": pTe5Tb0s}, user=default) - node1.query("insert into test_table1 values ('2017-06-18', 555, 0)",settings={"password": pTe5Tb0s}, user=default) - node1.query("insert into test_table1 values ('2017-06-18', 666, 0)",settings={"password": pTe5Tb0s}, user=default) - - - assert_eq_with_retry(node1, "SELECT id FROM test_table1 order by id limit 1", '111',settings={"password": pTe5Tb0s}, user=default) - assert_eq_with_retry(node2, "SELECT id FROM test_table1 order by id limit 1", '111',settings={"password": dtnDvTr9}, user=default) - - - node2.query("insert into test_table2 values ('2017-06-17', 333, 0)",settings={"password": azAUGBFl}, user=zhang) - node2.query("insert into test_table2 values ('2017-06-17', 444, 0)",settings={"password": azAUGBFl}, user=zhang) - node2.query("insert into test_table2 values ('2017-06-18', 555, 0)",settings={"password": azAUGBFl}, user=zhang) - node2.query("insert into test_table2 values ('2017-06-18', 666, 0)",settings={"password": azAUGBFl}, user=zhang) - node2.query("insert into test_table2 values ('2017-06-19', 777, 0)",settings={"password": azAUGBFl}, user=zhang) - node2.query("insert into test_table2 values ('2017-06-19', 888, 0)",settings={"password": azAUGBFl}, user=zhang) - - - assert_eq_with_retry(node2, "SELECT id FROM test_table2 order by id limit 1", '333',settings={"password": azAUGBFl}, user=zhang) - assert_eq_with_retry(node3, "SELECT id FROM test_table2 order by id limit 1", '333',settings={"password": ROgXGTDq}, user=default) - - node1.query("ALTER TABLE test_table1 DROP PARTITION 20170616") - assert_eq_with_retry(node1, "SELECT id FROM test_table1 order by id limit 1", '333',settings={"password": pTe5Tb0s}, user=default) - assert_eq_with_retry(node2, "SELECT id FROM test_table1 order by id limit 1", '333',settings={"password": dtnDvTr9}, user=default) - - node2.query("ALTER TABLE test_table1 DROP PARTITION 20170617") - assert_eq_with_retry(node1, "SELECT id FROM test_table1 order by id limit 1", '555',settings={"password": pTe5Tb0s}, user=default) - assert_eq_with_retry(node2, "SELECT id FROM test_table1 order by id limit 1", '555',settings={"password": dtnDvTr9}, user=default) - - - node3.query("ALTER TABLE test_table2 DROP PARTITION 20170617") - assert_eq_with_retry(node2, "SELECT id FROM test_table2 order by id limit 1", '555',settings={"password": azAUGBFl}, user=zhang) - assert_eq_with_retry(node3, "SELECT id FROM test_table2 order by id limit 1", '555',settings={"password": ROgXGTDq}, user=default) - From 06095340c7ea89e87cc2cb5b2ef8fc25173814c8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Dec 2018 19:45:45 +0300 Subject: [PATCH 216/230] Slightly better code #3940 --- dbms/programs/client/TestHint.h | 25 +++++++++++++------------ 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/dbms/programs/client/TestHint.h b/dbms/programs/client/TestHint.h index 7ddbfbb59f8..c1ac913e7a8 100644 --- a/dbms/programs/client/TestHint.h +++ b/dbms/programs/client/TestHint.h @@ -28,25 +28,26 @@ public: if (!enabled_) return; - String full_comment; Lexer lexer(query.data(), query.data() + query.size()); for (Token token = lexer.nextToken(); !token.isEnd(); token = lexer.nextToken()) { if (token.type == TokenType::Comment) - full_comment += String(token.begin, token.begin + token.size()) + ' '; - } - - if (!full_comment.empty()) - { - size_t pos_start = full_comment.find('{', 0); - if (pos_start != String::npos) { - size_t pos_end = full_comment.find('}', pos_start); - if (pos_end != String::npos) + String comment(token.begin, token.begin + token.size()); + + if (!comment.empty()) { - String hint(full_comment.begin() + pos_start + 1, full_comment.begin() + pos_end); - parse(hint); + size_t pos_start = comment.find('{', 0); + if (pos_start != String::npos) + { + size_t pos_end = comment.find('}', pos_start); + if (pos_end != String::npos) + { + String hint(comment.begin() + pos_start + 1, comment.begin() + pos_end); + parse(hint); + } + } } } } From 43da59d0fc222ba8321cc1ce020946c41180fc82 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 28 Dec 2018 20:11:52 +0300 Subject: [PATCH 217/230] Improve test --- dbms/src/Interpreters/Cluster.cpp | 3 ++ dbms/src/Interpreters/Cluster.h | 2 + .../Storages/StorageReplicatedMergeTree.cpp | 3 +- .../src/Storages/StorageReplicatedMergeTree.h | 1 + .../configs/remote_servers.xml | 11 ++++ ...ocal_default.xml => user_good_allowed.xml} | 11 ++-- .../configs/user_good_restricted.xml | 52 +++++++++++++++++++ .../test.py | 49 ++++++++++------- 8 files changed, 106 insertions(+), 26 deletions(-) rename dbms/tests/integration/test_send_request_to_leader_replica/configs/{users_local_default.xml => user_good_allowed.xml} (89%) create mode 100644 dbms/tests/integration/test_send_request_to_leader_replica/configs/user_good_restricted.xml diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index db81bc58061..834a27f26a2 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -54,6 +54,9 @@ Cluster::Address::Address(const Poco::Util::AbstractConfiguration & config, cons host_name = config.getString(config_prefix + ".host"); port = static_cast(config.getInt(config_prefix + ".port")); + if (config.has(config_prefix + ".user")) + user_specified = true; + user = config.getString(config_prefix + ".user", "default"); password = config.getString(config_prefix + ".password", ""); default_database = config.getString(config_prefix + ".default_database", ""); diff --git a/dbms/src/Interpreters/Cluster.h b/dbms/src/Interpreters/Cluster.h index f998ad8f912..aa78f77b6a7 100644 --- a/dbms/src/Interpreters/Cluster.h +++ b/dbms/src/Interpreters/Cluster.h @@ -62,6 +62,8 @@ public: UInt32 replica_num; /// The locality is determined at the initialization, and is not changed even if DNS is changed bool is_local; + bool user_specified = false; + Protocol::Compression compression = Protocol::Compression::Enable; Protocol::Secure secure = Protocol::Secure::Disable; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 1b13a69c3e9..17514cc2031 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3959,7 +3959,8 @@ std::optional StorageReplicatedMergeTree::findClusterAddress(c for (size_t replica_num = 0; replica_num < shards[shard_num].size(); ++replica_num) { const Cluster::Address & address = shards[shard_num][replica_num]; - if (address.host_name == leader_address.host && address.port == leader_address.queries_port) + /// user is actually specified, not default + if (address.host_name == leader_address.host && address.port == leader_address.queries_port && address.user_specified) return address; } } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 2ceb1eb2408..1094c9dd218 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/tests/integration/test_send_request_to_leader_replica/configs/remote_servers.xml b/dbms/tests/integration/test_send_request_to_leader_replica/configs/remote_servers.xml index 03e31203d8b..149569e4c28 100644 --- a/dbms/tests/integration/test_send_request_to_leader_replica/configs/remote_servers.xml +++ b/dbms/tests/integration/test_send_request_to_leader_replica/configs/remote_servers.xml @@ -14,6 +14,17 @@ awesome + + true + + node3 + 9000 + + + node4 + 9000 + + diff --git a/dbms/tests/integration/test_send_request_to_leader_replica/configs/users_local_default.xml b/dbms/tests/integration/test_send_request_to_leader_replica/configs/user_good_allowed.xml similarity index 89% rename from dbms/tests/integration/test_send_request_to_leader_replica/configs/users_local_default.xml rename to dbms/tests/integration/test_send_request_to_leader_replica/configs/user_good_allowed.xml index ed27c17e5e6..e72fb61d2af 100644 --- a/dbms/tests/integration/test_send_request_to_leader_replica/configs/users_local_default.xml +++ b/dbms/tests/integration/test_send_request_to_leader_replica/configs/user_good_allowed.xml @@ -5,13 +5,12 @@ 0 random - + 10000000000 0 random - + - @@ -22,14 +21,14 @@ default default - + ::/0 - awesome + good default - + diff --git a/dbms/tests/integration/test_send_request_to_leader_replica/configs/user_good_restricted.xml b/dbms/tests/integration/test_send_request_to_leader_replica/configs/user_good_restricted.xml new file mode 100644 index 00000000000..ea0714aec11 --- /dev/null +++ b/dbms/tests/integration/test_send_request_to_leader_replica/configs/user_good_restricted.xml @@ -0,0 +1,52 @@ + + + + 10000000000 + 0 + random + + + 10000000000 + 0 + random + + + + 10000000000 + 0 + random + + + + + + + 127.0.0.1 + ::1 + + default + default + + + + + ::/0 + + good + default + + + + + ::/0 + + awesome + default + + + + + + + + diff --git a/dbms/tests/integration/test_send_request_to_leader_replica/test.py b/dbms/tests/integration/test_send_request_to_leader_replica/test.py index 2cbfd8bbc7b..913a5e6ff7a 100644 --- a/dbms/tests/integration/test_send_request_to_leader_replica/test.py +++ b/dbms/tests/integration/test_send_request_to_leader_replica/test.py @@ -7,8 +7,10 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], user_configs=['configs/users_local_default.xml'], with_zookeeper=True) -node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], user_configs=['configs/users_local_default.xml'], with_zookeeper=True) +node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], user_configs=['configs/user_good_restricted.xml'], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], user_configs=['configs/user_good_restricted.xml'], with_zookeeper=True) +node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml'], user_configs=['configs/user_good_allowed.xml'], with_zookeeper=True) +node4 = cluster.add_instance('node4', main_configs=['configs/remote_servers.xml'], user_configs=['configs/user_good_allowed.xml'], with_zookeeper=True) @pytest.fixture(scope="module") def started_cluster(): @@ -22,42 +24,51 @@ def started_cluster(): ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '{replica}', date, id, 8192); '''.format(replica=node.name), user='awesome') + for node in [node3, node4]: + node.query(''' + CREATE TABLE someothertable(date Date, id UInt32, value Int32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/someothertable', '{replica}', date, id, 8192); + '''.format(replica=node.name), user='good') + yield cluster finally: cluster.shutdown() -@pytest.mark.parametrize("query,expected", [ - ("ALTER TABLE sometable DROP PARTITION 201706", '1'), - ("TRUNCATE TABLE sometable", '0'), - ("OPTIMIZE TABLE sometable", '4'), +@pytest.mark.parametrize("table,query,expected,n1,n2", [ + ("sometable","ALTER TABLE sometable DROP PARTITION 201706", '1', node1, node2), + ("sometable","TRUNCATE TABLE sometable", '0', node1, node2), + ("sometable", "OPTIMIZE TABLE sometable", '4', node1, node2), + ("someothertable","ALTER TABLE someothertable DROP PARTITION 201706", '1', node3, node4), + ("someothertable","TRUNCATE TABLE someothertable", '0', node3, node4), + ("someothertable", "OPTIMIZE TABLE someothertable", '4', node3, node4), ]) -def test_alter_table_drop_partition(started_cluster, query, expected): +def test_alter_table_drop_partition(started_cluster, table, query, expected, n1, n2): to_insert = '''\ 2017-06-16 111 0 2017-06-16 222 1 2017-06-16 333 2 2017-07-16 444 3 ''' - node1.query("INSERT INTO sometable FORMAT TSV", stdin=to_insert, user='awesome') + n1.query("INSERT INTO {} FORMAT TSV".format(table), stdin=to_insert, user='good') - assert_eq_with_retry(node1, "SELECT COUNT(*) from sometable", '4', user='awesome') - assert_eq_with_retry(node2, "SELECT COUNT(*) from sometable", '4', user='awesome') + assert_eq_with_retry(n1, "SELECT COUNT(*) from {}".format(table), '4', user='good') + assert_eq_with_retry(n2, "SELECT COUNT(*) from {}".format(table), '4', user='good') ### It maybe leader and everything will be ok - node1.query(query, user='awesome') + n1.query(query, user='good') - assert_eq_with_retry(node1, "SELECT COUNT(*) from sometable", expected, user='awesome') - assert_eq_with_retry(node2, "SELECT COUNT(*) from sometable", expected, user='awesome') + assert_eq_with_retry(n1, "SELECT COUNT(*) from {}".format(table), expected, user='good') + assert_eq_with_retry(n2, "SELECT COUNT(*) from {}".format(table), expected, user='good') - node1.query("INSERT INTO sometable FORMAT TSV", stdin=to_insert, user='awesome') + n1.query("INSERT INTO {} FORMAT TSV".format(table), stdin=to_insert, user='good') - assert_eq_with_retry(node1, "SELECT COUNT(*) from sometable", '4', user='awesome') - assert_eq_with_retry(node2, "SELECT COUNT(*) from sometable", '4', user='awesome') + assert_eq_with_retry(n1, "SELECT COUNT(*) from {}".format(table), '4', user='good') + assert_eq_with_retry(n2, "SELECT COUNT(*) from {}".format(table), '4', user='good') ### If node1 is leader than node2 will be slave - node2.query(query, user='awesome') + n2.query(query, user='good') - assert_eq_with_retry(node1, "SELECT COUNT(*) from sometable", expected, user='awesome') - assert_eq_with_retry(node2, "SELECT COUNT(*) from sometable", expected, user='awesome') + assert_eq_with_retry(n1, "SELECT COUNT(*) from {}".format(table), expected, user='good') + assert_eq_with_retry(n2, "SELECT COUNT(*) from {}".format(table), expected, user='good') From 3c82dd8809cb950ce62b74614e40318e4fcd266c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Dec 2018 20:15:42 +0300 Subject: [PATCH 218/230] Fixed exception safety of ThreadPool constructor [#CLICKHOUSE-4113] --- libs/libcommon/include/common/ThreadPool.h | 2 ++ libs/libcommon/src/ThreadPool.cpp | 20 ++++++++++++++++++-- 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/libs/libcommon/include/common/ThreadPool.h b/libs/libcommon/include/common/ThreadPool.h index 980fdcba355..dd82c0c0399 100644 --- a/libs/libcommon/include/common/ThreadPool.h +++ b/libs/libcommon/include/common/ThreadPool.h @@ -56,6 +56,8 @@ private: void worker(); + + void finalize(); }; diff --git a/libs/libcommon/src/ThreadPool.cpp b/libs/libcommon/src/ThreadPool.cpp index 4da7c9689b8..e45e64853dc 100644 --- a/libs/libcommon/src/ThreadPool.cpp +++ b/libs/libcommon/src/ThreadPool.cpp @@ -6,8 +6,17 @@ ThreadPool::ThreadPool(size_t m_size) : m_size(m_size) { threads.reserve(m_size); - for (size_t i = 0; i < m_size; ++i) - threads.emplace_back([this] { worker(); }); + + try + { + for (size_t i = 0; i < m_size; ++i) + threads.emplace_back([this] { worker(); }); + } + catch (...) + { + finalize(); + throw; + } } void ThreadPool::schedule(Job job) @@ -40,6 +49,11 @@ void ThreadPool::wait() } ThreadPool::~ThreadPool() +{ + finalize(); +} + +void ThreadPool::finalize() { { std::unique_lock lock(mutex); @@ -50,6 +64,8 @@ ThreadPool::~ThreadPool() for (auto & thread : threads) thread.join(); + + threads.clear(); } size_t ThreadPool::active() const From 0c78ef85733080e3650e4271d233a5850463ae6e Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 28 Dec 2018 20:26:10 +0300 Subject: [PATCH 219/230] Fix cyclic lib depend (make clickhouse_compression lib) --- dbms/CMakeLists.txt | 4 +- dbms/programs/compressor/CMakeLists.txt | 2 +- dbms/src/Common/tests/CMakeLists.txt | 8 +-- dbms/src/Compression/CMakeLists.txt | 9 +++ .../CachedCompressedReadBuffer.cpp | 5 +- .../Compression/CachedCompressedReadBuffer.h | 58 ++++++++++++++++++ .../CompressedReadBuffer.cpp | 2 +- dbms/src/Compression/CompressedReadBuffer.h | 33 +++++++++++ .../CompressedReadBufferBase.cpp | 2 +- .../CompressedReadBufferBase.h | 0 .../CompressedReadBufferFromFile.cpp | 3 +- .../CompressedReadBufferFromFile.h | 45 ++++++++++++++ .../CompressedWriteBuffer.cpp | 2 +- dbms/src/Compression/CompressedWriteBuffer.h | 55 +++++++++++++++++ .../CompressionCodecWriteBuffer.h | 0 dbms/src/IO/CachedCompressedReadBuffer.h | 59 +------------------ dbms/src/IO/CompressedReadBuffer.h | 34 +---------- dbms/src/IO/CompressedReadBufferFromFile.h | 46 +-------------- dbms/src/IO/CompressedWriteBuffer.h | 56 +----------------- dbms/src/IO/tests/CMakeLists.txt | 5 +- utils/check-marks/CMakeLists.txt | 2 +- 21 files changed, 223 insertions(+), 207 deletions(-) rename dbms/src/{IO => Compression}/CachedCompressedReadBuffer.cpp (98%) create mode 100644 dbms/src/Compression/CachedCompressedReadBuffer.h rename dbms/src/{IO => Compression}/CompressedReadBuffer.cpp (98%) create mode 100644 dbms/src/Compression/CompressedReadBuffer.h rename dbms/src/{IO => Compression}/CompressedReadBufferBase.cpp (99%) rename dbms/src/{IO => Compression}/CompressedReadBufferBase.h (100%) rename dbms/src/{IO => Compression}/CompressedReadBufferFromFile.cpp (98%) create mode 100644 dbms/src/Compression/CompressedReadBufferFromFile.h rename dbms/src/{IO => Compression}/CompressedWriteBuffer.cpp (97%) create mode 100644 dbms/src/Compression/CompressedWriteBuffer.h rename dbms/src/{IO => Compression}/CompressionCodecWriteBuffer.h (100%) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index e4c7887f700..a89e7c4df41 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -64,7 +64,6 @@ set(dbms_sources) include(../cmake/dbms_glob_sources.cmake) add_headers_and_sources(clickhouse_common_io src/Common) -add_headers_and_sources(clickhouse_common_io src/Compression) add_headers_and_sources(clickhouse_common_io src/Common/HashTable) add_headers_and_sources(clickhouse_common_io src/IO) @@ -159,7 +158,6 @@ target_link_libraries (clickhouse_common_io PUBLIC common PRIVATE - clickhouse_parsers string_utils widechar_width ${LINK_LIBRARIES_ONLY_ON_X86_64} @@ -186,6 +184,8 @@ target_link_libraries (clickhouse_common_io ) target_link_libraries (dbms + PUBLIC + clickhouse_compression PRIVATE clickhouse_parsers clickhouse_common_config diff --git a/dbms/programs/compressor/CMakeLists.txt b/dbms/programs/compressor/CMakeLists.txt index 7aa2cad5708..bf3accfb8af 100644 --- a/dbms/programs/compressor/CMakeLists.txt +++ b/dbms/programs/compressor/CMakeLists.txt @@ -1,5 +1,5 @@ add_library (clickhouse-compressor-lib ${LINK_MODE} Compressor.cpp) -target_link_libraries (clickhouse-compressor-lib PRIVATE clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries (clickhouse-compressor-lib PRIVATE clickhouse_compression clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) if (CLICKHOUSE_SPLIT_BINARY) # Also in utils diff --git a/dbms/src/Common/tests/CMakeLists.txt b/dbms/src/Common/tests/CMakeLists.txt index 05984c9d42e..5e22d7e37cd 100644 --- a/dbms/src/Common/tests/CMakeLists.txt +++ b/dbms/src/Common/tests/CMakeLists.txt @@ -20,10 +20,10 @@ add_executable (small_table small_table.cpp) target_link_libraries (small_table PRIVATE clickhouse_common_io) add_executable (parallel_aggregation parallel_aggregation.cpp) -target_link_libraries (parallel_aggregation PRIVATE clickhouse_common_io) +target_link_libraries (parallel_aggregation PRIVATE clickhouse_compression clickhouse_common_io) add_executable (parallel_aggregation2 parallel_aggregation2.cpp) -target_link_libraries (parallel_aggregation2 PRIVATE clickhouse_common_io) +target_link_libraries (parallel_aggregation2 PRIVATE clickhouse_compression clickhouse_common_io) add_executable (int_hashes_perf int_hashes_perf.cpp AvalancheTest.cpp Random.cpp) target_link_libraries (int_hashes_perf PRIVATE clickhouse_common_io) @@ -42,7 +42,7 @@ add_executable (shell_command_test shell_command_test.cpp) target_link_libraries (shell_command_test PRIVATE clickhouse_common_io) add_executable (arena_with_free_lists arena_with_free_lists.cpp) -target_link_libraries (arena_with_free_lists PRIVATE clickhouse_common_io) +target_link_libraries (arena_with_free_lists PRIVATE clickhouse_compression clickhouse_common_io) add_executable (pod_array pod_array.cpp) target_link_libraries (pod_array PRIVATE clickhouse_common_io) @@ -61,7 +61,7 @@ target_link_libraries (space_saving PRIVATE clickhouse_common_io) add_executable (integer_hash_tables_and_hashes integer_hash_tables_and_hashes.cpp) target_include_directories (integer_hash_tables_and_hashes SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) -target_link_libraries (integer_hash_tables_and_hashes PRIVATE clickhouse_common_io) +target_link_libraries (integer_hash_tables_and_hashes PRIVATE clickhouse_compression clickhouse_common_io) add_executable (allocator allocator.cpp) target_link_libraries (allocator PRIVATE clickhouse_common_io) diff --git a/dbms/src/Compression/CMakeLists.txt b/dbms/src/Compression/CMakeLists.txt index e69de29bb2d..6a0d6c90622 100644 --- a/dbms/src/Compression/CMakeLists.txt +++ b/dbms/src/Compression/CMakeLists.txt @@ -0,0 +1,9 @@ +include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) +add_headers_and_sources(clickhouse_compression .) +add_library(clickhouse_compression ${LINK_MODE} ${clickhouse_compression_headers} ${clickhouse_compression_sources}) +target_link_libraries(clickhouse_compression PRIVATE clickhouse_parsers clickhouse_common_io ${ZSTD_LIBRARY}) +target_include_directories(clickhouse_compression PUBLIC ${DBMS_INCLUDE_DIR}) + +#if(ENABLE_TESTS) +# add_subdirectory(tests) +#endif() diff --git a/dbms/src/IO/CachedCompressedReadBuffer.cpp b/dbms/src/Compression/CachedCompressedReadBuffer.cpp similarity index 98% rename from dbms/src/IO/CachedCompressedReadBuffer.cpp rename to dbms/src/Compression/CachedCompressedReadBuffer.cpp index 50c97edf1a3..bfaf9695f9b 100644 --- a/dbms/src/IO/CachedCompressedReadBuffer.cpp +++ b/dbms/src/Compression/CachedCompressedReadBuffer.cpp @@ -1,9 +1,10 @@ +#include "CachedCompressedReadBuffer.h" + #include -#include +#include "CachedCompressedReadBuffer.h" #include #include #include -#include "CachedCompressedReadBuffer.h" namespace DB diff --git a/dbms/src/Compression/CachedCompressedReadBuffer.h b/dbms/src/Compression/CachedCompressedReadBuffer.h new file mode 100644 index 00000000000..174ddb98587 --- /dev/null +++ b/dbms/src/Compression/CachedCompressedReadBuffer.h @@ -0,0 +1,58 @@ +#pragma once + +#include +#include +#include +#include "CompressedReadBufferBase.h" +#include +#include + + +namespace DB +{ + + +/** A buffer for reading from a compressed file using the cache of decompressed blocks. + * The external cache is passed as an argument to the constructor. + * Allows you to increase performance in cases where the same blocks are often read. + * Disadvantages: + * - in case you need to read a lot of data in a row, but of them only a part is cached, you have to do seek-and. + */ +class CachedCompressedReadBuffer : public CompressedReadBufferBase, public ReadBuffer +{ +private: + const std::string path; + UncompressedCache * cache; + size_t buf_size; + size_t estimated_size; + size_t aio_threshold; + + std::unique_ptr file_in; + size_t file_pos; + + /// A piece of data from the cache, or a piece of read data that we put into the cache. + UncompressedCache::MappedPtr owned_cell; + + void initInput(); + bool nextImpl() override; + + /// Passed into file_in. + ReadBufferFromFileBase::ProfileCallback profile_callback; + clockid_t clock_type {}; + +public: + CachedCompressedReadBuffer( + const std::string & path_, UncompressedCache * cache_, size_t estimated_size_, size_t aio_threshold_, + size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); + + + void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); + + void setProfileCallback(const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_ = CLOCK_MONOTONIC_COARSE) + { + profile_callback = profile_callback_; + clock_type = clock_type_; + } +}; + +} diff --git a/dbms/src/IO/CompressedReadBuffer.cpp b/dbms/src/Compression/CompressedReadBuffer.cpp similarity index 98% rename from dbms/src/IO/CompressedReadBuffer.cpp rename to dbms/src/Compression/CompressedReadBuffer.cpp index cc540161c92..699a44455fc 100644 --- a/dbms/src/IO/CompressedReadBuffer.cpp +++ b/dbms/src/Compression/CompressedReadBuffer.cpp @@ -1,4 +1,4 @@ -#include +#include "CompressedReadBuffer.h" #include #include diff --git a/dbms/src/Compression/CompressedReadBuffer.h b/dbms/src/Compression/CompressedReadBuffer.h new file mode 100644 index 00000000000..1e8ea4784c7 --- /dev/null +++ b/dbms/src/Compression/CompressedReadBuffer.h @@ -0,0 +1,33 @@ +#pragma once + +#include "CompressedReadBufferBase.h" +#include +#include + + +namespace DB +{ + +class CompressedReadBuffer : public CompressedReadBufferBase, public BufferWithOwnMemory +{ +private: + size_t size_compressed = 0; + + bool nextImpl() override; + +public: + CompressedReadBuffer(ReadBuffer & in_) + : CompressedReadBufferBase(&in_), BufferWithOwnMemory(0) + { + } + + size_t readBig(char * to, size_t n) override; + + /// The compressed size of the current block. + size_t getSizeCompressed() const + { + return size_compressed; + } +}; + +} diff --git a/dbms/src/IO/CompressedReadBufferBase.cpp b/dbms/src/Compression/CompressedReadBufferBase.cpp similarity index 99% rename from dbms/src/IO/CompressedReadBufferBase.cpp rename to dbms/src/Compression/CompressedReadBufferBase.cpp index 5ac795a82e1..1ce83134f03 100644 --- a/dbms/src/IO/CompressedReadBufferBase.cpp +++ b/dbms/src/Compression/CompressedReadBufferBase.cpp @@ -1,4 +1,4 @@ -#include +#include "CompressedReadBufferBase.h" #include diff --git a/dbms/src/IO/CompressedReadBufferBase.h b/dbms/src/Compression/CompressedReadBufferBase.h similarity index 100% rename from dbms/src/IO/CompressedReadBufferBase.h rename to dbms/src/Compression/CompressedReadBufferBase.h diff --git a/dbms/src/IO/CompressedReadBufferFromFile.cpp b/dbms/src/Compression/CompressedReadBufferFromFile.cpp similarity index 98% rename from dbms/src/IO/CompressedReadBufferFromFile.cpp rename to dbms/src/Compression/CompressedReadBufferFromFile.cpp index 25008c205b5..75103435c13 100644 --- a/dbms/src/IO/CompressedReadBufferFromFile.cpp +++ b/dbms/src/Compression/CompressedReadBufferFromFile.cpp @@ -1,4 +1,5 @@ -#include +#include "CompressedReadBufferFromFile.h" + #include #include #include diff --git a/dbms/src/Compression/CompressedReadBufferFromFile.h b/dbms/src/Compression/CompressedReadBufferFromFile.h new file mode 100644 index 00000000000..288a66e321a --- /dev/null +++ b/dbms/src/Compression/CompressedReadBufferFromFile.h @@ -0,0 +1,45 @@ +#pragma once + +#include "CompressedReadBufferBase.h" +#include +#include +#include +#include + + +namespace DB +{ + + +/// Unlike CompressedReadBuffer, it can do seek. +class CompressedReadBufferFromFile : public CompressedReadBufferBase, public BufferWithOwnMemory +{ +private: + /** At any time, one of two things is true: + * a) size_compressed = 0 + * b) + * - `working_buffer` contains the entire block. + * - `file_in` points to the end of this block. + * - `size_compressed` contains the compressed size of this block. + */ + std::unique_ptr p_file_in; + ReadBufferFromFileBase & file_in; + size_t size_compressed = 0; + + bool nextImpl() override; + +public: + CompressedReadBufferFromFile( + const std::string & path, size_t estimated_size, size_t aio_threshold, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + + void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); + + size_t readBig(char * to, size_t n) override; + + void setProfileCallback(const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_ = CLOCK_MONOTONIC_COARSE) + { + file_in.setProfileCallback(profile_callback_, clock_type_); + } +}; + +} diff --git a/dbms/src/IO/CompressedWriteBuffer.cpp b/dbms/src/Compression/CompressedWriteBuffer.cpp similarity index 97% rename from dbms/src/IO/CompressedWriteBuffer.cpp rename to dbms/src/Compression/CompressedWriteBuffer.cpp index 24a2021555f..7fc8d5ab5f9 100644 --- a/dbms/src/IO/CompressedWriteBuffer.cpp +++ b/dbms/src/Compression/CompressedWriteBuffer.cpp @@ -8,7 +8,7 @@ #include #include -#include +#include "CompressedWriteBuffer.h" #include diff --git a/dbms/src/Compression/CompressedWriteBuffer.h b/dbms/src/Compression/CompressedWriteBuffer.h new file mode 100644 index 00000000000..a9612b463a5 --- /dev/null +++ b/dbms/src/Compression/CompressedWriteBuffer.h @@ -0,0 +1,55 @@ +#pragma once + +#include + +#include + +#include +#include +#include +#include + + +namespace DB +{ + +class CompressedWriteBuffer : public BufferWithOwnMemory +{ +private: + WriteBuffer & out; + CompressionCodecPtr codec; + + PODArray compressed_buffer; + + void nextImpl() override; + +public: + CompressedWriteBuffer( + WriteBuffer & out_, + CompressionCodecPtr codec_ = CompressionCodecFactory::instance().getDefaultCodec(), + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + + /// The amount of compressed data + size_t getCompressedBytes() + { + nextIfAtEnd(); + return out.count(); + } + + /// How many uncompressed bytes were written to the buffer + size_t getUncompressedBytes() + { + return count(); + } + + /// How many bytes are in the buffer (not yet compressed) + size_t getRemainingBytes() + { + nextIfAtEnd(); + return offset(); + } + + ~CompressedWriteBuffer() override; +}; + +} diff --git a/dbms/src/IO/CompressionCodecWriteBuffer.h b/dbms/src/Compression/CompressionCodecWriteBuffer.h similarity index 100% rename from dbms/src/IO/CompressionCodecWriteBuffer.h rename to dbms/src/Compression/CompressionCodecWriteBuffer.h diff --git a/dbms/src/IO/CachedCompressedReadBuffer.h b/dbms/src/IO/CachedCompressedReadBuffer.h index 1b5e41972f3..936c1520407 100644 --- a/dbms/src/IO/CachedCompressedReadBuffer.h +++ b/dbms/src/IO/CachedCompressedReadBuffer.h @@ -1,58 +1 @@ -#pragma once - -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - - -/** A buffer for reading from a compressed file using the cache of decompressed blocks. - * The external cache is passed as an argument to the constructor. - * Allows you to increase performance in cases where the same blocks are often read. - * Disadvantages: - * - in case you need to read a lot of data in a row, but of them only a part is cached, you have to do seek-and. - */ -class CachedCompressedReadBuffer : public CompressedReadBufferBase, public ReadBuffer -{ -private: - const std::string path; - UncompressedCache * cache; - size_t buf_size; - size_t estimated_size; - size_t aio_threshold; - - std::unique_ptr file_in; - size_t file_pos; - - /// A piece of data from the cache, or a piece of read data that we put into the cache. - UncompressedCache::MappedPtr owned_cell; - - void initInput(); - bool nextImpl() override; - - /// Passed into file_in. - ReadBufferFromFileBase::ProfileCallback profile_callback; - clockid_t clock_type {}; - -public: - CachedCompressedReadBuffer( - const std::string & path_, UncompressedCache * cache_, size_t estimated_size_, size_t aio_threshold_, - size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); - - - void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); - - void setProfileCallback(const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_ = CLOCK_MONOTONIC_COARSE) - { - profile_callback = profile_callback_; - clock_type = clock_type_; - } -}; - -} +#include diff --git a/dbms/src/IO/CompressedReadBuffer.h b/dbms/src/IO/CompressedReadBuffer.h index 60ba29012b8..88ad9a2fad3 100644 --- a/dbms/src/IO/CompressedReadBuffer.h +++ b/dbms/src/IO/CompressedReadBuffer.h @@ -1,33 +1 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ - -class CompressedReadBuffer : public CompressedReadBufferBase, public BufferWithOwnMemory -{ -private: - size_t size_compressed = 0; - - bool nextImpl() override; - -public: - CompressedReadBuffer(ReadBuffer & in_) - : CompressedReadBufferBase(&in_), BufferWithOwnMemory(0) - { - } - - size_t readBig(char * to, size_t n) override; - - /// The compressed size of the current block. - size_t getSizeCompressed() const - { - return size_compressed; - } -}; - -} +#include diff --git a/dbms/src/IO/CompressedReadBufferFromFile.h b/dbms/src/IO/CompressedReadBufferFromFile.h index f1332ea4187..4a659366aaa 100644 --- a/dbms/src/IO/CompressedReadBufferFromFile.h +++ b/dbms/src/IO/CompressedReadBufferFromFile.h @@ -1,45 +1 @@ -#pragma once - -#include -#include -#include -#include -#include - - -namespace DB -{ - - -/// Unlike CompressedReadBuffer, it can do seek. -class CompressedReadBufferFromFile : public CompressedReadBufferBase, public BufferWithOwnMemory -{ -private: - /** At any time, one of two things is true: - * a) size_compressed = 0 - * b) - * - `working_buffer` contains the entire block. - * - `file_in` points to the end of this block. - * - `size_compressed` contains the compressed size of this block. - */ - std::unique_ptr p_file_in; - ReadBufferFromFileBase & file_in; - size_t size_compressed = 0; - - bool nextImpl() override; - -public: - CompressedReadBufferFromFile( - const std::string & path, size_t estimated_size, size_t aio_threshold, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); - - void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block); - - size_t readBig(char * to, size_t n) override; - - void setProfileCallback(const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_ = CLOCK_MONOTONIC_COARSE) - { - file_in.setProfileCallback(profile_callback_, clock_type_); - } -}; - -} +#include diff --git a/dbms/src/IO/CompressedWriteBuffer.h b/dbms/src/IO/CompressedWriteBuffer.h index a9612b463a5..c1dfa3f54b3 100644 --- a/dbms/src/IO/CompressedWriteBuffer.h +++ b/dbms/src/IO/CompressedWriteBuffer.h @@ -1,55 +1 @@ -#pragma once - -#include - -#include - -#include -#include -#include -#include - - -namespace DB -{ - -class CompressedWriteBuffer : public BufferWithOwnMemory -{ -private: - WriteBuffer & out; - CompressionCodecPtr codec; - - PODArray compressed_buffer; - - void nextImpl() override; - -public: - CompressedWriteBuffer( - WriteBuffer & out_, - CompressionCodecPtr codec_ = CompressionCodecFactory::instance().getDefaultCodec(), - size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); - - /// The amount of compressed data - size_t getCompressedBytes() - { - nextIfAtEnd(); - return out.count(); - } - - /// How many uncompressed bytes were written to the buffer - size_t getUncompressedBytes() - { - return count(); - } - - /// How many bytes are in the buffer (not yet compressed) - size_t getRemainingBytes() - { - nextIfAtEnd(); - return offset(); - } - - ~CompressedWriteBuffer() override; -}; - -} +#include diff --git a/dbms/src/IO/tests/CMakeLists.txt b/dbms/src/IO/tests/CMakeLists.txt index 59a1e0088c2..79b30266281 100644 --- a/dbms/src/IO/tests/CMakeLists.txt +++ b/dbms/src/IO/tests/CMakeLists.txt @@ -19,8 +19,9 @@ target_link_libraries (valid_utf8_perf PRIVATE clickhouse_common_io) add_executable (valid_utf8 valid_utf8.cpp) target_link_libraries (valid_utf8 PRIVATE clickhouse_common_io) +# TODO move to Compression add_executable (compressed_buffer compressed_buffer.cpp) -target_link_libraries (compressed_buffer PRIVATE clickhouse_common_io) +target_link_libraries (compressed_buffer PRIVATE clickhouse_compression clickhouse_common_io) add_executable (var_uint var_uint.cpp) target_link_libraries (var_uint PRIVATE clickhouse_common_io) @@ -29,7 +30,7 @@ add_executable (read_escaped_string read_escaped_string.cpp) target_link_libraries (read_escaped_string PRIVATE clickhouse_common_io) add_executable (async_write async_write.cpp) -target_link_libraries (async_write PRIVATE clickhouse_common_io) +target_link_libraries (async_write PRIVATE clickhouse_compression clickhouse_common_io) add_executable (parse_int_perf parse_int_perf.cpp) target_link_libraries (parse_int_perf PRIVATE clickhouse_common_io) diff --git a/utils/check-marks/CMakeLists.txt b/utils/check-marks/CMakeLists.txt index 9c534364691..86cff8fb233 100644 --- a/utils/check-marks/CMakeLists.txt +++ b/utils/check-marks/CMakeLists.txt @@ -1,2 +1,2 @@ add_executable (check-marks main.cpp) -target_link_libraries(check-marks PRIVATE clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries(check-marks PRIVATE clickhouse_compression clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) From bdaf1ac10916d10d009f4f3a207d7227a350e375 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 26 Dec 2018 19:44:57 +0300 Subject: [PATCH 220/230] Support multiclass models for CatBoost. --- .../src/Functions/FunctionsExternalModels.cpp | 93 +++++++++++++++++-- dbms/src/Functions/FunctionsExternalModels.h | 4 +- dbms/src/Interpreters/CatBoostModel.cpp | 88 ++++++++++++++++-- dbms/src/Interpreters/CatBoostModel.h | 8 ++ 4 files changed, 175 insertions(+), 18 deletions(-) diff --git a/dbms/src/Functions/FunctionsExternalModels.cpp b/dbms/src/Functions/FunctionsExternalModels.cpp index 6afbad31857..6c11cb78bb5 100644 --- a/dbms/src/Functions/FunctionsExternalModels.cpp +++ b/dbms/src/Functions/FunctionsExternalModels.cpp @@ -10,6 +10,10 @@ #include #include #include +#include +#include +#include +#include namespace DB { @@ -26,17 +30,43 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -DataTypePtr FunctionModelEvaluate::getReturnTypeImpl(const DataTypes & arguments) const +DataTypePtr FunctionModelEvaluate::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const { if (arguments.size() < 2) throw Exception("Function " + getName() + " expects at least 2 arguments", ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION); - if (!isString(arguments[0])) - throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + if (!isString(arguments[0].type)) + throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return std::make_shared(); + const auto name_col = checkAndGetColumnConst(arguments[0].column.get()); + if (!name_col) + throw Exception("First argument of function " + getName() + " must be a constant string", + ErrorCodes::ILLEGAL_COLUMN); + + bool has_nullable = false; + for (size_t i = 1; i < arguments.size(); ++i) + has_nullable = has_nullable || arguments[i].type->isNullable(); + + auto model = models.getModel(name_col->getValue()); + auto type = model->getReturnType(); + + if (has_nullable) + { + if (auto * tuple = typeid_cast(type.get())) + { + auto elements = tuple->getElements(); + for (auto & element : elements) + element = makeNullable(element); + + type = std::make_shared(elements); + } + else + type = makeNullable(type); + } + + return type; } void FunctionModelEvaluate::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) @@ -49,11 +79,58 @@ void FunctionModelEvaluate::executeImpl(Block & block, const ColumnNumbers & arg auto model = models.getModel(name_col->getValue()); ColumnRawPtrs columns; - columns.reserve(arguments.size()); - for (auto i : ext::range(1, arguments.size())) - columns.push_back(block.getByPosition(arguments[i]).column.get()); + Columns materialized_columns; + ColumnPtr null_map; - block.getByPosition(result).column = model->evaluate(columns); + columns.reserve(arguments.size()); + for (auto arg : ext::range(1, arguments.size())) + { + auto & column = block.getByPosition(arguments[arg]).column; + columns.push_back(column.get()); + if (auto full_column = column->convertToFullColumnIfConst()) + { + materialized_columns.push_back(full_column); + columns.back() = full_column.get(); + } + if (auto * col_nullable = typeid_cast(columns.back())) + { + if (!null_map) + null_map = col_nullable->getNullMapColumnPtr(); + else + { + auto mut_null_map = (*std::move(null_map)).mutate(); + + NullMap & result_null_map = static_cast(*mut_null_map).getData(); + const NullMap & src_null_map = col_nullable->getNullMapColumn().getData(); + + for (size_t i = 0, size = result_null_map.size(); i < size; ++i) + if (src_null_map[i]) + result_null_map[i] = 1; + + null_map = std::move(mut_null_map); + } + + columns.back() = &col_nullable->getNestedColumn(); + } + } + + auto res = model->evaluate(columns); + + if (null_map) + { + if (auto * tuple = typeid_cast(res.get())) + { + auto nested = tuple->getColumns(); + for (auto & col : nested) + col = ColumnNullable::create(col, null_map); + + res = ColumnTuple::create(nested); + } + else + res = ColumnNullable::create(res, null_map); + } + + block.getByPosition(result).column = res; } void registerFunctionsExternalModels(FunctionFactory & factory) diff --git a/dbms/src/Functions/FunctionsExternalModels.h b/dbms/src/Functions/FunctionsExternalModels.h index e32fe7f066f..ab193e0a2bc 100644 --- a/dbms/src/Functions/FunctionsExternalModels.h +++ b/dbms/src/Functions/FunctionsExternalModels.h @@ -25,9 +25,11 @@ public: bool isDeterministic() const override { return false; } + bool useDefaultImplementationForNulls() const override { return false; } + size_t getNumberOfArguments() const override { return 0; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; diff --git a/dbms/src/Interpreters/CatBoostModel.cpp b/dbms/src/Interpreters/CatBoostModel.cpp index bd7c07813d0..7f75a14e325 100644 --- a/dbms/src/Interpreters/CatBoostModel.cpp +++ b/dbms/src/Interpreters/CatBoostModel.cpp @@ -5,11 +5,14 @@ #include #include #include +#include #include #include #include #include #include +#include +#include namespace DB { @@ -51,12 +54,16 @@ struct CatBoostWrapperAPI double * result, size_t resultSize); int (* GetStringCatFeatureHash)(const char * data, size_t size); - int (* GetIntegerCatFeatureHash)(long long val); size_t (* GetFloatFeaturesCount)(ModelCalcerHandle* calcer); - size_t (* GetCatFeaturesCount)(ModelCalcerHandle* calcer); + size_t (* GetTreeCount)(ModelCalcerHandle* modelHandle); + size_t (* GetDimensionsCount)(ModelCalcerHandle* modelHandle); + + bool (* CheckModelMetadataHasKey)(ModelCalcerHandle* modelHandle, const char* keyPtr, size_t keySize); + size_t (*GetModelInfoValueSize)(ModelCalcerHandle* modelHandle, const char* keyPtr, size_t keySize); + const char* (*GetModelInfoValue)(ModelCalcerHandle* modelHandle, const char* keyPtr, size_t keySize); }; @@ -95,6 +102,9 @@ public: float_features_count = api->GetFloatFeaturesCount(handle_->get()); cat_features_count = api->GetCatFeaturesCount(handle_->get()); + tree_count = 1; + if (api->GetDimensionsCount) + tree_count = api->GetDimensionsCount(handle_->get()); handle = std::move(handle_); } @@ -146,17 +156,48 @@ public: } } - return evalImpl(columns, float_features_count, cat_features_count, cat_features_are_strings); + auto result = evalImpl(columns, float_features_count, cat_features_count, tree_count, cat_features_are_strings); + + if (tree_count == 1) + return result; + + size_t column_size = columns.front()->size(); + auto result_buf = result->getData().data(); + + /// Multiple trees case. Copy data to several columns. + MutableColumns mutable_columns(tree_count); + std::vector column_ptrs(tree_count); + for (size_t i = 0; i < tree_count; ++i) + { + auto col = ColumnFloat64::create(column_size); + column_ptrs[i] = col->getData().data(); + mutable_columns[i] = std::move(col); + } + + Float64 * data = result_buf; + for (size_t row = 0; row < column_size; ++row) + { + for (size_t i = 0; i < tree_count; ++i) + { + *column_ptrs[i] = *data; + ++column_ptrs[i]; + ++data; + } + } + + return ColumnTuple::create(std::move(mutable_columns)); } size_t getFloatFeaturesCount() const override { return float_features_count; } size_t getCatFeaturesCount() const override { return cat_features_count; } + size_t getTreeCount() const override { return tree_count; } private: std::unique_ptr handle; const CatBoostWrapperAPI * api; size_t float_features_count; size_t cat_features_count; + size_t tree_count; /// Buffer should be allocated with features_count * column->size() elements. /// Place column elements in positions buffer[0], buffer[features_count], ... , buffer[size * features_count] @@ -322,13 +363,17 @@ private: /// * CalcModelPredictionFlat if no cat features /// * CalcModelPrediction if all cat features are strings /// * CalcModelPredictionWithHashedCatFeatures if has int cat features. - ColumnPtr evalImpl(const ColumnRawPtrs & columns, size_t float_features_count_current, size_t cat_features_count_current, - bool cat_features_are_strings) const + ColumnFloat64::MutablePtr evalImpl( + const ColumnRawPtrs & columns, + size_t float_features_count_current, + size_t cat_features_count_current, + size_t tree_count, + bool cat_features_are_strings) const { std::string error_msg = "Error occurred while applying CatBoost model: "; size_t column_size = columns.front()->size(); - auto result = ColumnFloat64::create(column_size); + auto result = ColumnFloat64::create(column_size * tree_count); auto result_buf = result->getData().data(); if (!column_size) @@ -344,7 +389,7 @@ private: { if (!api->CalcModelPredictionFlat(handle->get(), column_size, float_features_buf, float_features_count_current, - result_buf, column_size)) + result_buf, column_size * tree_count)) { throw Exception(error_msg + api->GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL); @@ -368,7 +413,7 @@ private: if (!api->CalcModelPrediction(handle->get(), column_size, float_features_buf, float_features_count_current, cat_features_buf, cat_features_count_current, - result_buf, column_size)) + result_buf, column_size * tree_count)) { throw Exception(error_msg + api->GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL); } @@ -384,7 +429,7 @@ private: handle->get(), column_size, float_features_buf, float_features_count_current, cat_features_buf, cat_features_count_current, - result_buf, column_size)) + result_buf, column_size * tree_count)) { throw Exception(error_msg + api->GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL); } @@ -413,6 +458,9 @@ private: template void load(T& func, const std::string & name) { func = lib.get(name); } + + template + void tryLoad(T& func, const std::string & name) { func = lib.tryGet(name); } }; void CatBoostLibHolder::initAPI() @@ -428,6 +476,11 @@ void CatBoostLibHolder::initAPI() load(api.GetIntegerCatFeatureHash, "GetIntegerCatFeatureHash"); load(api.GetFloatFeaturesCount, "GetFloatFeaturesCount"); load(api.GetCatFeaturesCount, "GetCatFeaturesCount"); + tryLoad(api.CheckModelMetadataHasKey, "CheckModelMetadataHasKey"); + tryLoad(api.GetModelInfoValueSize, "GetModelInfoValueSize"); + tryLoad(api.GetModelInfoValue, "GetModelInfoValue"); + tryLoad(api.GetTreeCount, "GetTreeCount"); + tryLoad(api.GetDimensionsCount, "GetDimensionsCount"); } std::shared_ptr getCatBoostWrapperHolder(const std::string & lib_path) @@ -474,6 +527,7 @@ void CatBoostModel::init() model = std::make_unique(api, model_path); float_features_count = model->getFloatFeaturesCount(); cat_features_count = model->getCatFeaturesCount(); + tree_count = model->getTreeCount(); } const ExternalLoadableLifetime & CatBoostModel::getLifetime() const @@ -501,6 +555,22 @@ size_t CatBoostModel::getCatFeaturesCount() const return cat_features_count; } +size_t CatBoostModel::getTreeCount() const +{ + return tree_count; +} + +DataTypePtr CatBoostModel::getReturnType() const +{ + auto type = std::make_shared(); + if (tree_count == 1) + return type; + + DataTypes types(tree_count, type); + + return std::make_shared(types); +} + ColumnPtr CatBoostModel::evaluate(const ColumnRawPtrs & columns) const { if (!model) diff --git a/dbms/src/Interpreters/CatBoostModel.h b/dbms/src/Interpreters/CatBoostModel.h index b6a937fe048..0fc379d8d00 100644 --- a/dbms/src/Interpreters/CatBoostModel.h +++ b/dbms/src/Interpreters/CatBoostModel.h @@ -27,14 +27,19 @@ public: virtual size_t getFloatFeaturesCount() const = 0; virtual size_t getCatFeaturesCount() const = 0; + virtual size_t getTreeCount() const = 0; }; +class IDataType; +using DataTypePtr = std::shared_ptr; + /// General ML model evaluator interface. class IModel : public IExternalLoadable { public: virtual ColumnPtr evaluate(const ColumnRawPtrs & columns) const = 0; virtual std::string getTypeName() const = 0; + virtual DataTypePtr getReturnType() const = 0; }; class CatBoostModel : public IModel @@ -48,6 +53,8 @@ public: size_t getFloatFeaturesCount() const; size_t getCatFeaturesCount() const; + size_t getTreeCount() const; + DataTypePtr getReturnType() const override; /// IExternalLoadable interface. @@ -76,6 +83,7 @@ private: size_t float_features_count; size_t cat_features_count; + size_t tree_count; std::chrono::time_point creation_time; std::exception_ptr creation_exception; From 300bbdfb43f351a9979f538c531d15951fe2581f Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 28 Dec 2018 21:15:26 +0300 Subject: [PATCH 221/230] Fix cyclic lib depend ppart 2: clean --- dbms/CMakeLists.txt | 15 +++++++-------- dbms/programs/compressor/Compressor.cpp | 4 ++-- dbms/programs/server/HTTPHandler.cpp | 4 ++-- dbms/programs/server/InterserverIOHTTPHandler.cpp | 2 +- dbms/programs/server/TCPHandler.cpp | 4 ++-- dbms/src/Client/Connection.cpp | 4 ++-- dbms/src/Common/tests/CMakeLists.txt | 2 +- dbms/src/Common/tests/arena_with_free_lists.cpp | 2 +- .../tests/integer_hash_tables_and_hashes.cpp | 2 +- dbms/src/Common/tests/parallel_aggregation.cpp | 2 +- dbms/src/Common/tests/parallel_aggregation2.cpp | 2 +- dbms/src/Compression/CMakeLists.txt | 2 +- .../Compression/CachedCompressedReadBuffer.cpp | 3 +-- dbms/src/Compression/CompressedReadBuffer.cpp | 2 +- dbms/src/Compression/CompressedReadBufferBase.h | 2 +- .../Compression/CompressedReadBufferFromFile.cpp | 2 +- dbms/src/Compression/CompressionCodecLZ4.cpp | 5 +++-- dbms/src/Compression/CompressionCodecLZ4.h | 2 +- dbms/src/Compression/ICompressionCodec.cpp | 5 +++-- dbms/src/Compression/ICompressionCodec.h | 3 +-- .../{IO => Compression}/LZ4_decompress_faster.cpp | 5 ++--- .../{IO => Compression}/LZ4_decompress_faster.h | 0 dbms/src/Compression/tests/CMakeLists.txt | 5 +++++ .../tests/cached_compressed_read_buffer.cpp | 2 +- .../tests/compressed_buffer.cpp | 4 ++-- .../src/DataStreams/AggregatingBlockInputStream.h | 2 +- .../DataStreams/MergeSortingBlockInputStream.cpp | 2 +- .../DataStreams/MergeSortingBlockInputStream.h | 2 +- dbms/src/DataStreams/NativeBlockInputStream.cpp | 2 +- dbms/src/DataStreams/NativeBlockOutputStream.cpp | 2 +- .../ParallelAggregatingBlockInputStream.h | 2 +- dbms/src/IO/CachedCompressedReadBuffer.h | 1 - dbms/src/IO/CompressedReadBuffer.h | 1 - dbms/src/IO/CompressedReadBufferFromFile.h | 1 - dbms/src/IO/CompressedWriteBuffer.h | 1 - dbms/src/IO/tests/CMakeLists.txt | 7 ------- dbms/src/IO/tests/async_write.cpp | 2 +- dbms/src/IO/tests/parse_int_perf.cpp | 4 ++-- dbms/src/IO/tests/read_float_perf.cpp | 2 +- dbms/src/Interpreters/Aggregator.cpp | 2 +- dbms/src/Interpreters/tests/CMakeLists.txt | 12 ++++++------ dbms/src/Interpreters/tests/hash_map.cpp | 2 +- dbms/src/Interpreters/tests/hash_map_string.cpp | 2 +- dbms/src/Interpreters/tests/hash_map_string_2.cpp | 2 +- dbms/src/Interpreters/tests/hash_map_string_3.cpp | 2 +- .../Interpreters/tests/hash_map_string_small.cpp | 2 +- .../src/Interpreters/tests/two_level_hash_map.cpp | 2 +- .../src/Storages/Distributed/DirectoryMonitor.cpp | 2 +- .../Distributed/DistributedBlockOutputStream.cpp | 2 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp | 4 ++-- .../MergeTree/MergeTreeDataPartChecksum.cpp | 4 ++-- dbms/src/Storages/MergeTree/MergeTreeDataWriter.h | 2 +- dbms/src/Storages/MergeTree/MergeTreeReader.cpp | 2 +- dbms/src/Storages/MergeTree/MergeTreeReader.h | 2 +- .../Storages/MergeTree/MergedBlockOutputStream.h | 2 +- dbms/src/Storages/MergeTree/checkDataPart.cpp | 2 +- dbms/src/Storages/StorageLog.cpp | 4 ++-- dbms/src/Storages/StorageSet.cpp | 4 ++-- dbms/src/Storages/StorageStripeLog.cpp | 4 ++-- dbms/src/Storages/StorageTinyLog.cpp | 4 ++-- utils/check-marks/main.cpp | 6 +++--- utils/compressor/CMakeLists.txt | 2 +- utils/compressor/decompress_perf.cpp | 2 +- 64 files changed, 92 insertions(+), 100 deletions(-) rename dbms/src/{IO => Compression}/LZ4_decompress_faster.cpp (99%) rename dbms/src/{IO => Compression}/LZ4_decompress_faster.h (100%) create mode 100644 dbms/src/Compression/tests/CMakeLists.txt rename dbms/src/{IO => Compression}/tests/cached_compressed_read_buffer.cpp (96%) rename dbms/src/{IO => Compression}/tests/compressed_buffer.cpp (95%) delete mode 100644 dbms/src/IO/CachedCompressedReadBuffer.h delete mode 100644 dbms/src/IO/CompressedReadBuffer.h delete mode 100644 dbms/src/IO/CompressedReadBufferFromFile.h delete mode 100644 dbms/src/IO/CompressedWriteBuffer.h diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index a89e7c4df41..f8f2afd0626 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -150,10 +150,6 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELW PROPERTIES COMPILE_FLAGS -g0) endif () -if (NOT ARCH_ARM AND CPUID_LIBRARY) - set (LINK_LIBRARIES_ONLY_ON_X86_64 ${CPUID_LIBRARY}) -endif() - target_link_libraries (clickhouse_common_io PUBLIC common @@ -161,8 +157,8 @@ target_link_libraries (clickhouse_common_io string_utils widechar_width ${LINK_LIBRARIES_ONLY_ON_X86_64} - ${LZ4_LIBRARY} - ${ZSTD_LIBRARY} + #${LZ4_LIBRARY} + #${ZSTD_LIBRARY} ${DOUBLE_CONVERSION_LIBRARIES} pocoext PUBLIC @@ -183,10 +179,13 @@ target_link_libraries (clickhouse_common_io ${CMAKE_DL_LIBS} ) +if (NOT ARCH_ARM AND CPUID_LIBRARY) + target_link_libraries (clickhouse_common_io PRIVATE ${CPUID_LIBRARY}) +endif() + target_link_libraries (dbms - PUBLIC - clickhouse_compression PRIVATE + clickhouse_compression clickhouse_parsers clickhouse_common_config PUBLIC diff --git a/dbms/programs/compressor/Compressor.cpp b/dbms/programs/compressor/Compressor.cpp index efa45b86a88..de51f16833e 100644 --- a/dbms/programs/compressor/Compressor.cpp +++ b/dbms/programs/compressor/Compressor.cpp @@ -5,8 +5,8 @@ #include #include #include -#include -#include +#include +#include #include #include diff --git a/dbms/programs/server/HTTPHandler.cpp b/dbms/programs/server/HTTPHandler.cpp index 34037a7c7c0..d86c526784b 100644 --- a/dbms/programs/server/HTTPHandler.cpp +++ b/dbms/programs/server/HTTPHandler.cpp @@ -19,8 +19,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/dbms/programs/server/InterserverIOHTTPHandler.cpp b/dbms/programs/server/InterserverIOHTTPHandler.cpp index 3c93ee1989a..94095365b6a 100644 --- a/dbms/programs/server/InterserverIOHTTPHandler.cpp +++ b/dbms/programs/server/InterserverIOHTTPHandler.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include #include diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 3bfdedaeb58..c3dff11146e 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -12,8 +12,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index f72eadcaec4..b91391a9894 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -2,8 +2,8 @@ #include #include -#include -#include +#include +#include #include #include #include diff --git a/dbms/src/Common/tests/CMakeLists.txt b/dbms/src/Common/tests/CMakeLists.txt index 5e22d7e37cd..ec9636ce664 100644 --- a/dbms/src/Common/tests/CMakeLists.txt +++ b/dbms/src/Common/tests/CMakeLists.txt @@ -1,5 +1,5 @@ add_executable (hashes_test hashes_test.cpp) -target_link_libraries (hashes_test PRIVATE dbms ${OPENSSL_CRYPTO_LIBRARY}) +target_link_libraries (hashes_test PRIVATE clickhouse_common_io ${OPENSSL_CRYPTO_LIBRARY} ${CITYHASH_LIBRARIES}) add_executable (sip_hash sip_hash.cpp) target_link_libraries (sip_hash PRIVATE clickhouse_common_io) diff --git a/dbms/src/Common/tests/arena_with_free_lists.cpp b/dbms/src/Common/tests/arena_with_free_lists.cpp index 5091551b550..4d4915f5dcc 100644 --- a/dbms/src/Common/tests/arena_with_free_lists.cpp +++ b/dbms/src/Common/tests/arena_with_free_lists.cpp @@ -20,7 +20,7 @@ #include #include #include -#include +#include #include using namespace DB; diff --git a/dbms/src/Common/tests/integer_hash_tables_and_hashes.cpp b/dbms/src/Common/tests/integer_hash_tables_and_hashes.cpp index ca8c6140d93..20b557a5de6 100644 --- a/dbms/src/Common/tests/integer_hash_tables_and_hashes.cpp +++ b/dbms/src/Common/tests/integer_hash_tables_and_hashes.cpp @@ -14,7 +14,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/Common/tests/parallel_aggregation.cpp b/dbms/src/Common/tests/parallel_aggregation.cpp index f6ee3b01659..15a193de3d7 100644 --- a/dbms/src/Common/tests/parallel_aggregation.cpp +++ b/dbms/src/Common/tests/parallel_aggregation.cpp @@ -13,7 +13,7 @@ //#include #include -#include +#include #include #include diff --git a/dbms/src/Common/tests/parallel_aggregation2.cpp b/dbms/src/Common/tests/parallel_aggregation2.cpp index dac95746c02..603390ec736 100644 --- a/dbms/src/Common/tests/parallel_aggregation2.cpp +++ b/dbms/src/Common/tests/parallel_aggregation2.cpp @@ -13,7 +13,7 @@ //#include #include -#include +#include #include #include diff --git a/dbms/src/Compression/CMakeLists.txt b/dbms/src/Compression/CMakeLists.txt index 6a0d6c90622..3c2aa580ced 100644 --- a/dbms/src/Compression/CMakeLists.txt +++ b/dbms/src/Compression/CMakeLists.txt @@ -1,7 +1,7 @@ include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) add_headers_and_sources(clickhouse_compression .) add_library(clickhouse_compression ${LINK_MODE} ${clickhouse_compression_headers} ${clickhouse_compression_sources}) -target_link_libraries(clickhouse_compression PRIVATE clickhouse_parsers clickhouse_common_io ${ZSTD_LIBRARY}) +target_link_libraries(clickhouse_compression PRIVATE clickhouse_parsers clickhouse_common_io ${ZSTD_LIBRARY} ${LZ4_LIBRARY}) target_include_directories(clickhouse_compression PUBLIC ${DBMS_INCLUDE_DIR}) #if(ENABLE_TESTS) diff --git a/dbms/src/Compression/CachedCompressedReadBuffer.cpp b/dbms/src/Compression/CachedCompressedReadBuffer.cpp index bfaf9695f9b..e87a9a45019 100644 --- a/dbms/src/Compression/CachedCompressedReadBuffer.cpp +++ b/dbms/src/Compression/CachedCompressedReadBuffer.cpp @@ -1,10 +1,9 @@ #include "CachedCompressedReadBuffer.h" #include -#include "CachedCompressedReadBuffer.h" #include #include -#include +#include namespace DB diff --git a/dbms/src/Compression/CompressedReadBuffer.cpp b/dbms/src/Compression/CompressedReadBuffer.cpp index 699a44455fc..bae56e2a283 100644 --- a/dbms/src/Compression/CompressedReadBuffer.cpp +++ b/dbms/src/Compression/CompressedReadBuffer.cpp @@ -1,6 +1,6 @@ #include "CompressedReadBuffer.h" #include -#include +#include namespace DB diff --git a/dbms/src/Compression/CompressedReadBufferBase.h b/dbms/src/Compression/CompressedReadBufferBase.h index 34ff798a8f1..f44140dcd04 100644 --- a/dbms/src/Compression/CompressedReadBufferBase.h +++ b/dbms/src/Compression/CompressedReadBufferBase.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include diff --git a/dbms/src/Compression/CompressedReadBufferFromFile.cpp b/dbms/src/Compression/CompressedReadBufferFromFile.cpp index 75103435c13..759acf0b2a5 100644 --- a/dbms/src/Compression/CompressedReadBufferFromFile.cpp +++ b/dbms/src/Compression/CompressedReadBufferFromFile.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB diff --git a/dbms/src/Compression/CompressionCodecLZ4.cpp b/dbms/src/Compression/CompressionCodecLZ4.cpp index a2f9262ae06..68318a484f2 100644 --- a/dbms/src/Compression/CompressionCodecLZ4.cpp +++ b/dbms/src/Compression/CompressionCodecLZ4.cpp @@ -1,9 +1,10 @@ -#include +#include "CompressionCodecLZ4.h" + #include #include #include #include -#include +#include #include "CompressionCodecLZ4.h" #include #include diff --git a/dbms/src/Compression/CompressionCodecLZ4.h b/dbms/src/Compression/CompressionCodecLZ4.h index fe3339b40a1..5ce137dba54 100644 --- a/dbms/src/Compression/CompressionCodecLZ4.h +++ b/dbms/src/Compression/CompressionCodecLZ4.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/dbms/src/Compression/ICompressionCodec.cpp b/dbms/src/Compression/ICompressionCodec.cpp index 45ed8250cd7..f9707d7c9da 100644 --- a/dbms/src/Compression/ICompressionCodec.cpp +++ b/dbms/src/Compression/ICompressionCodec.cpp @@ -1,5 +1,6 @@ -#include -#include +#include "ICompressionCodec.h" + +#include #include #include #include diff --git a/dbms/src/Compression/ICompressionCodec.h b/dbms/src/Compression/ICompressionCodec.h index eaf877b479f..2602e15c600 100644 --- a/dbms/src/Compression/ICompressionCodec.h +++ b/dbms/src/Compression/ICompressionCodec.h @@ -9,8 +9,7 @@ #include #include #include -#include - +#include #include namespace DB diff --git a/dbms/src/IO/LZ4_decompress_faster.cpp b/dbms/src/Compression/LZ4_decompress_faster.cpp similarity index 99% rename from dbms/src/IO/LZ4_decompress_faster.cpp rename to dbms/src/Compression/LZ4_decompress_faster.cpp index 243bdd997ce..53f72a76245 100644 --- a/dbms/src/IO/LZ4_decompress_faster.cpp +++ b/dbms/src/Compression/LZ4_decompress_faster.cpp @@ -1,12 +1,11 @@ +#include "LZ4_decompress_faster.h" + #include #include #include #include - -#include #include #include - #include #include #include diff --git a/dbms/src/IO/LZ4_decompress_faster.h b/dbms/src/Compression/LZ4_decompress_faster.h similarity index 100% rename from dbms/src/IO/LZ4_decompress_faster.h rename to dbms/src/Compression/LZ4_decompress_faster.h diff --git a/dbms/src/Compression/tests/CMakeLists.txt b/dbms/src/Compression/tests/CMakeLists.txt new file mode 100644 index 00000000000..50f212f18c3 --- /dev/null +++ b/dbms/src/Compression/tests/CMakeLists.txt @@ -0,0 +1,5 @@ +add_executable (compressed_buffer compressed_buffer.cpp) +target_link_libraries (compressed_buffer PRIVATE clickhouse_compression clickhouse_common_io) + +add_executable (cached_compressed_read_buffer cached_compressed_read_buffer.cpp) +target_link_libraries (cached_compressed_read_buffer PRIVATE clickhouse_compression clickhouse_common_io) diff --git a/dbms/src/IO/tests/cached_compressed_read_buffer.cpp b/dbms/src/Compression/tests/cached_compressed_read_buffer.cpp similarity index 96% rename from dbms/src/IO/tests/cached_compressed_read_buffer.cpp rename to dbms/src/Compression/tests/cached_compressed_read_buffer.cpp index df30a3bc064..fb30d691745 100644 --- a/dbms/src/IO/tests/cached_compressed_read_buffer.cpp +++ b/dbms/src/Compression/tests/cached_compressed_read_buffer.cpp @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/IO/tests/compressed_buffer.cpp b/dbms/src/Compression/tests/compressed_buffer.cpp similarity index 95% rename from dbms/src/IO/tests/compressed_buffer.cpp rename to dbms/src/Compression/tests/compressed_buffer.cpp index bcb8d7ae9ce..346a4068a6c 100644 --- a/dbms/src/IO/tests/compressed_buffer.cpp +++ b/dbms/src/Compression/tests/compressed_buffer.cpp @@ -9,8 +9,8 @@ #include #include #include -#include -#include +#include +#include #include #include diff --git a/dbms/src/DataStreams/AggregatingBlockInputStream.h b/dbms/src/DataStreams/AggregatingBlockInputStream.h index a28814dcab8..19483455742 100644 --- a/dbms/src/DataStreams/AggregatingBlockInputStream.h +++ b/dbms/src/DataStreams/AggregatingBlockInputStream.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp index 527adc4ec64..0dfd07fc6b6 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.h b/dbms/src/DataStreams/MergeSortingBlockInputStream.h index 033a695ac27..7ff6ea6cfc2 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.h +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.h @@ -12,7 +12,7 @@ #include #include -#include +#include namespace DB diff --git a/dbms/src/DataStreams/NativeBlockInputStream.cpp b/dbms/src/DataStreams/NativeBlockInputStream.cpp index 7eeba3b9e50..2dbedd01b38 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockInputStream.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.cpp b/dbms/src/DataStreams/NativeBlockOutputStream.cpp index fb5aadb2fb3..11c3944afbb 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockOutputStream.cpp @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h index 0a74557d449..5719dc68e84 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/IO/CachedCompressedReadBuffer.h b/dbms/src/IO/CachedCompressedReadBuffer.h deleted file mode 100644 index 936c1520407..00000000000 --- a/dbms/src/IO/CachedCompressedReadBuffer.h +++ /dev/null @@ -1 +0,0 @@ -#include diff --git a/dbms/src/IO/CompressedReadBuffer.h b/dbms/src/IO/CompressedReadBuffer.h deleted file mode 100644 index 88ad9a2fad3..00000000000 --- a/dbms/src/IO/CompressedReadBuffer.h +++ /dev/null @@ -1 +0,0 @@ -#include diff --git a/dbms/src/IO/CompressedReadBufferFromFile.h b/dbms/src/IO/CompressedReadBufferFromFile.h deleted file mode 100644 index 4a659366aaa..00000000000 --- a/dbms/src/IO/CompressedReadBufferFromFile.h +++ /dev/null @@ -1 +0,0 @@ -#include diff --git a/dbms/src/IO/CompressedWriteBuffer.h b/dbms/src/IO/CompressedWriteBuffer.h deleted file mode 100644 index c1dfa3f54b3..00000000000 --- a/dbms/src/IO/CompressedWriteBuffer.h +++ /dev/null @@ -1 +0,0 @@ -#include diff --git a/dbms/src/IO/tests/CMakeLists.txt b/dbms/src/IO/tests/CMakeLists.txt index 79b30266281..73497def442 100644 --- a/dbms/src/IO/tests/CMakeLists.txt +++ b/dbms/src/IO/tests/CMakeLists.txt @@ -19,10 +19,6 @@ target_link_libraries (valid_utf8_perf PRIVATE clickhouse_common_io) add_executable (valid_utf8 valid_utf8.cpp) target_link_libraries (valid_utf8 PRIVATE clickhouse_common_io) -# TODO move to Compression -add_executable (compressed_buffer compressed_buffer.cpp) -target_link_libraries (compressed_buffer PRIVATE clickhouse_compression clickhouse_common_io) - add_executable (var_uint var_uint.cpp) target_link_libraries (var_uint PRIVATE clickhouse_common_io) @@ -44,9 +40,6 @@ target_link_libraries (read_write_int PRIVATE clickhouse_common_io) add_executable (mempbrk mempbrk.cpp) target_link_libraries (mempbrk PRIVATE clickhouse_common_io) -add_executable (cached_compressed_read_buffer cached_compressed_read_buffer.cpp) -target_link_libraries (cached_compressed_read_buffer PRIVATE clickhouse_common_io dbms ) - add_executable (o_direct_and_dirty_pages o_direct_and_dirty_pages.cpp) target_link_libraries (o_direct_and_dirty_pages PRIVATE clickhouse_common_io) diff --git a/dbms/src/IO/tests/async_write.cpp b/dbms/src/IO/tests/async_write.cpp index 9cf1b2721cf..29361f2a8ab 100644 --- a/dbms/src/IO/tests/async_write.cpp +++ b/dbms/src/IO/tests/async_write.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include diff --git a/dbms/src/IO/tests/parse_int_perf.cpp b/dbms/src/IO/tests/parse_int_perf.cpp index e3b40927a85..8b3c516d6aa 100644 --- a/dbms/src/IO/tests/parse_int_perf.cpp +++ b/dbms/src/IO/tests/parse_int_perf.cpp @@ -8,8 +8,8 @@ #include #include #include -#include -#include +#include +#include #include #include diff --git a/dbms/src/IO/tests/read_float_perf.cpp b/dbms/src/IO/tests/read_float_perf.cpp index cb4c5e90a06..8a870d4a960 100644 --- a/dbms/src/IO/tests/read_float_perf.cpp +++ b/dbms/src/IO/tests/read_float_perf.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include /** How to test: diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index d90da905c2e..176791bddd7 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/tests/CMakeLists.txt b/dbms/src/Interpreters/tests/CMakeLists.txt index 7660527cd87..2f814c5a6a0 100644 --- a/dbms/src/Interpreters/tests/CMakeLists.txt +++ b/dbms/src/Interpreters/tests/CMakeLists.txt @@ -12,28 +12,28 @@ target_link_libraries (aggregate PRIVATE dbms) add_executable (hash_map hash_map.cpp) target_include_directories (hash_map SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) -target_link_libraries (hash_map PRIVATE dbms) +target_link_libraries (hash_map PRIVATE dbms clickhouse_compression) add_executable (hash_map3 hash_map3.cpp) target_link_libraries (hash_map3 PRIVATE dbms ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES}) add_executable (hash_map_string hash_map_string.cpp) target_include_directories (hash_map_string SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) -target_link_libraries (hash_map_string PRIVATE dbms) +target_link_libraries (hash_map_string PRIVATE dbms clickhouse_compression) add_executable (hash_map_string_2 hash_map_string_2.cpp) -target_link_libraries (hash_map_string_2 PRIVATE dbms) +target_link_libraries (hash_map_string_2 PRIVATE dbms clickhouse_compression) add_executable (hash_map_string_3 hash_map_string_3.cpp) -target_link_libraries (hash_map_string_3 PRIVATE dbms ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES}) +target_link_libraries (hash_map_string_3 PRIVATE dbms clickhouse_compression ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES}) add_executable (hash_map_string_small hash_map_string_small.cpp) target_include_directories (hash_map_string_small SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) -target_link_libraries (hash_map_string_small PRIVATE dbms) +target_link_libraries (hash_map_string_small PRIVATE dbms clickhouse_compression) add_executable (two_level_hash_map two_level_hash_map.cpp) target_include_directories (two_level_hash_map SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) -target_link_libraries (two_level_hash_map PRIVATE dbms) +target_link_libraries (two_level_hash_map PRIVATE dbms clickhouse_compression) add_executable (compiler_test compiler_test.cpp) target_link_libraries (compiler_test PRIVATE dbms) diff --git a/dbms/src/Interpreters/tests/hash_map.cpp b/dbms/src/Interpreters/tests/hash_map.cpp index d7e90e5e49b..4b60fb1cd77 100644 --- a/dbms/src/Interpreters/tests/hash_map.cpp +++ b/dbms/src/Interpreters/tests/hash_map.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/tests/hash_map_string.cpp b/dbms/src/Interpreters/tests/hash_map_string.cpp index 5d2a8c305da..9d22130a7a7 100644 --- a/dbms/src/Interpreters/tests/hash_map_string.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/tests/hash_map_string_2.cpp b/dbms/src/Interpreters/tests/hash_map_string_2.cpp index aba8d502270..43d25b71c47 100644 --- a/dbms/src/Interpreters/tests/hash_map_string_2.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string_2.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/tests/hash_map_string_3.cpp b/dbms/src/Interpreters/tests/hash_map_string_3.cpp index 24c923db8ad..cc862bfda63 100644 --- a/dbms/src/Interpreters/tests/hash_map_string_3.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string_3.cpp @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/tests/hash_map_string_small.cpp b/dbms/src/Interpreters/tests/hash_map_string_small.cpp index f3b48207cc1..1a8ad75540a 100644 --- a/dbms/src/Interpreters/tests/hash_map_string_small.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string_small.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/tests/two_level_hash_map.cpp b/dbms/src/Interpreters/tests/two_level_hash_map.cpp index d7fe0d8b4b3..7b793d4f33a 100644 --- a/dbms/src/Interpreters/tests/two_level_hash_map.cpp +++ b/dbms/src/Interpreters/tests/two_level_hash_map.cpp @@ -14,7 +14,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp index 74821f432e7..a5a4fca28a3 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 528c83167af..669fe2b63e1 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 729d4b321b0..789dc08bfa4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp index aca955fad7c..e7c9350a660 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp @@ -2,8 +2,8 @@ #include #include -#include -#include +#include +#include #include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 2b34413b6b9..21411c157f1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -5,8 +5,8 @@ #include #include #include -#include -#include +#include +#include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.h b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.h index 93d8ecb98ae..95371021939 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -4,7 +4,7 @@ #include #include -#include +#include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index 537a4d82722..89e19732c45 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index 988ca570ddd..879ed05da23 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h index b09abfc88a9..0ae8e83aafb 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include diff --git a/dbms/src/Storages/MergeTree/checkDataPart.cpp b/dbms/src/Storages/MergeTree/checkDataPart.cpp index eac9145692b..e88f027ad75 100644 --- a/dbms/src/Storages/MergeTree/checkDataPart.cpp +++ b/dbms/src/Storages/MergeTree/checkDataPart.cpp @@ -5,7 +5,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 462f20ad582..cb694756a39 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -6,8 +6,8 @@ #include #include -#include -#include +#include +#include #include #include diff --git a/dbms/src/Storages/StorageSet.cpp b/dbms/src/Storages/StorageSet.cpp index 2348520674d..01dc7c23bbf 100644 --- a/dbms/src/Storages/StorageSet.cpp +++ b/dbms/src/Storages/StorageSet.cpp @@ -1,9 +1,9 @@ #include #include #include -#include +#include #include -#include +#include #include #include #include diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index 9cf512880fe..f6f3cc6706f 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -11,8 +11,8 @@ #include #include -#include -#include +#include +#include #include #include diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 5f66f0f5049..202905094c2 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -13,8 +13,8 @@ #include #include -#include -#include +#include +#include #include #include diff --git a/utils/check-marks/main.cpp b/utils/check-marks/main.cpp index 63e785b8e9d..52020dfff25 100644 --- a/utils/check-marks/main.cpp +++ b/utils/check-marks/main.cpp @@ -2,14 +2,14 @@ #include -#include -#include +#include +#include #include #include #include #include #include -#include +#include /** This program checks correctness of .mrk (marks) file for corresponding compressed .bin file. diff --git a/utils/compressor/CMakeLists.txt b/utils/compressor/CMakeLists.txt index cd140051203..2dec2117943 100644 --- a/utils/compressor/CMakeLists.txt +++ b/utils/compressor/CMakeLists.txt @@ -13,7 +13,7 @@ add_executable (mutator mutator.cpp) target_link_libraries(mutator PRIVATE clickhouse_common_io) add_executable (decompress_perf decompress_perf.cpp) -target_link_libraries(decompress_perf PRIVATE clickhouse_common_io ${LZ4_LIBRARY}) +target_link_libraries(decompress_perf PRIVATE clickhouse_common_io clickhouse_compression ${LZ4_LIBRARY}) if (NOT USE_INTERNAL_ZSTD_LIBRARY) target_include_directories (zstd_test BEFORE PRIVATE ${ZSTD_INCLUDE_DIR}) diff --git a/utils/compressor/decompress_perf.cpp b/utils/compressor/decompress_perf.cpp index 91c2128cef1..3e598b10876 100644 --- a/utils/compressor/decompress_perf.cpp +++ b/utils/compressor/decompress_perf.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include #include From 2632c080d7f304649ee614eb56e000a8f75f2262 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 Dec 2018 21:22:39 +0300 Subject: [PATCH 222/230] Added multiclass model test. --- .../catboost/helpers/server_with_models.py | 7 +++ .../external_models/catboost/helpers/table.py | 7 ++- .../test_apply_catboost_model/test.py | 58 +++++++++++++++++++ 3 files changed, 71 insertions(+), 1 deletion(-) diff --git a/dbms/tests/external_models/catboost/helpers/server_with_models.py b/dbms/tests/external_models/catboost/helpers/server_with_models.py index e0ed81980e1..ad9feea99fe 100644 --- a/dbms/tests/external_models/catboost/helpers/server_with_models.py +++ b/dbms/tests/external_models/catboost/helpers/server_with_models.py @@ -20,6 +20,13 @@ CLICKHOUSE_CONFIG = \ users.xml {tcp_port} {catboost_dynamic_library_path} + + trace + {path}/clickhouse-server.log + {path}/clickhouse-server.err.log + never + 50 + ''' diff --git a/dbms/tests/external_models/catboost/helpers/table.py b/dbms/tests/external_models/catboost/helpers/table.py index 2e9c454ab10..e6b05ac7b7b 100644 --- a/dbms/tests/external_models/catboost/helpers/table.py +++ b/dbms/tests/external_models/catboost/helpers/table.py @@ -56,7 +56,12 @@ class ClickHouseTable: columns = ', '.join(list(float_columns) + list(cat_columns)) query = "select modelEvaluate('{}', {}) from test.{} format TSV" result = self.client.query(query.format(model_name, columns, self.table_name)) - return tuple(map(float, filter(len, map(str.strip, result.split())))) + + def parse_row(row): + values = tuple(map(float, filter(len, map(str.strip, row.replace('(', '').replace(')', '').split(','))))) + return values if len(values) != 1 else values[0] + + return tuple(map(parse_row, filter(len, map(str.strip, result.split('\n'))))) def _drop_table(self): self.client.query('drop table test.{}'.format(self.table_name)) diff --git a/dbms/tests/external_models/catboost/test_apply_catboost_model/test.py b/dbms/tests/external_models/catboost/test_apply_catboost_model/test.py index 792ba9a13c8..00b9fe0dce1 100644 --- a/dbms/tests/external_models/catboost/test_apply_catboost_model/test.py +++ b/dbms/tests/external_models/catboost/test_apply_catboost_model/test.py @@ -234,3 +234,61 @@ def test_apply_float_features_with_mixed_cat_features(): print 'clickhouse predictions', pred_ch check_predictions(name, test_target, pred_python, pred_ch, 0.9) + + +def test_apply_multiclass(): + + name = 'test_apply_float_features_with_mixed_cat_features' + + train_size = 10000 + test_size = 10000 + + def gen_data(size, seed): + data = { + 'a': generate_uniform_float_column(size, 0., 1., seed + 1), + 'b': generate_uniform_float_column(size, 0., 1., seed + 2), + 'c': generate_uniform_string_column(size, ['a', 'b', 'c'], seed + 3), + 'd': generate_uniform_int_column(size, 1, 4, seed + 4) + } + return DataFrame.from_dict(data) + + def get_target(df): + def target_filter(row): + if row['a'] > .3 and row['b'] > .3 and row['c'] != 'a': + return 0 + elif row['a'] * row['b'] > 0.1 and row['c'] != 'b' and row['d'] != 2: + return 1 + else: + return 2 + + return df.apply(target_filter, axis=1).as_matrix() + + train_df = gen_data(train_size, 42) + test_df = gen_data(test_size, 43) + + train_target = get_target(train_df) + test_target = get_target(test_df) + + print + print 'train target', train_target + print 'test target', test_target + + params = { + 'iterations': 10, + 'depth': 4, + 'learning_rate': 1, + 'loss_function': 'MultiClass' + } + + model = train_catboost_model(train_df, train_target, ['c', 'd'], params) + pred_python = model.predict(test_df)[:,0].astype(int) + + server = ClickHouseServerWithCatboostModels(name, CLICKHOUSE_TESTS_SERVER_BIN_PATH, PORT) + server.add_model(name, model) + with server: + pred_ch = np.argmax(np.array(server.apply_model(name, test_df, [])), axis=1) + + print 'python predictions', pred_python + print 'clickhouse predictions', pred_ch + + check_predictions(name, test_target, pred_python, pred_ch, 0.9) From 7dd6c4947ed998a7c1ee4530ebba86585d2aad09 Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 28 Dec 2018 21:31:28 +0300 Subject: [PATCH 223/230] Fix cyclic lib depend: part 3: clean --- dbms/CMakeLists.txt | 9 --------- dbms/src/Compression/CMakeLists.txt | 13 ++++++++++--- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index f8f2afd0626..ffe72b0c018 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -157,8 +157,6 @@ target_link_libraries (clickhouse_common_io string_utils widechar_width ${LINK_LIBRARIES_ONLY_ON_X86_64} - #${LZ4_LIBRARY} - #${ZSTD_LIBRARY} ${DOUBLE_CONVERSION_LIBRARIES} pocoext PUBLIC @@ -278,13 +276,6 @@ if (USE_HDFS) target_include_directories (dbms SYSTEM BEFORE PRIVATE ${HDFS3_INCLUDE_DIR}) endif() -if (NOT USE_INTERNAL_LZ4_LIBRARY) - target_include_directories (dbms SYSTEM BEFORE PRIVATE ${LZ4_INCLUDE_DIR}) -endif () -if (NOT USE_INTERNAL_ZSTD_LIBRARY) - target_include_directories (dbms SYSTEM BEFORE PRIVATE ${ZSTD_INCLUDE_DIR}) -endif () - if (USE_JEMALLOC) target_include_directories (dbms SYSTEM BEFORE PRIVATE ${JEMALLOC_INCLUDE_DIR}) # used in Interpreters/AsynchronousMetrics.cpp endif () diff --git a/dbms/src/Compression/CMakeLists.txt b/dbms/src/Compression/CMakeLists.txt index 3c2aa580ced..288d452bebf 100644 --- a/dbms/src/Compression/CMakeLists.txt +++ b/dbms/src/Compression/CMakeLists.txt @@ -4,6 +4,13 @@ add_library(clickhouse_compression ${LINK_MODE} ${clickhouse_compression_headers target_link_libraries(clickhouse_compression PRIVATE clickhouse_parsers clickhouse_common_io ${ZSTD_LIBRARY} ${LZ4_LIBRARY}) target_include_directories(clickhouse_compression PUBLIC ${DBMS_INCLUDE_DIR}) -#if(ENABLE_TESTS) -# add_subdirectory(tests) -#endif() +if (NOT USE_INTERNAL_LZ4_LIBRARY) + target_include_directories(clickhouse_compression SYSTEM BEFORE PRIVATE ${LZ4_INCLUDE_DIR}) +endif () +if (NOT USE_INTERNAL_ZSTD_LIBRARY) + target_include_directories(clickhouse_compression SYSTEM BEFORE PRIVATE ${ZSTD_INCLUDE_DIR}) +endif () + +if(ENABLE_TESTS) + add_subdirectory(tests) +endif() From de10df4b1278d2c6adebd1a4db2929e5233987b0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Dec 2018 21:37:23 +0300 Subject: [PATCH 224/230] Added a test just in case [#CLICKHOUSE-3995] --- .../00815_left_join_on_stepanel.reference | 2 ++ .../00815_left_join_on_stepanel.sql | 19 +++++++++++++++++++ 2 files changed, 21 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00815_left_join_on_stepanel.reference create mode 100644 dbms/tests/queries/0_stateless/00815_left_join_on_stepanel.sql diff --git a/dbms/tests/queries/0_stateless/00815_left_join_on_stepanel.reference b/dbms/tests/queries/0_stateless/00815_left_join_on_stepanel.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00815_left_join_on_stepanel.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/dbms/tests/queries/0_stateless/00815_left_join_on_stepanel.sql b/dbms/tests/queries/0_stateless/00815_left_join_on_stepanel.sql new file mode 100644 index 00000000000..695760c8633 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00815_left_join_on_stepanel.sql @@ -0,0 +1,19 @@ +USE test; + +DROP TABLE IF EXISTS fact_cpc_clicks; +DROP TABLE IF EXISTS dim_model; + +CREATE TABLE fact_cpc_clicks (model_id UInt8) ENGINE = Memory; +CREATE TABLE dim_model (model_id UInt8) ENGINE = Memory; + +INSERT INTO fact_cpc_clicks VALUES (1); +INSERT INTO dim_model VALUES (1); + +select f.model_id from fact_cpc_clicks as f left join dim_model as d on f.model_id=d.model_id limit 10; + +USE default; + +select f.model_id from test.fact_cpc_clicks as f left join test.dim_model as d on f.model_id=d.model_id limit 10; + +DROP TABLE test.fact_cpc_clicks; +DROP TABLE test.dim_model; From cb6317995120cf82a38a5df84b6cc1525cf32d04 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Dec 2018 23:33:36 +0300 Subject: [PATCH 225/230] Added a test by Denis Zhuravlev #2170 --- .../0_stateless/00816_key_condition_den_crane.reference | 1 + .../0_stateless/00816_key_condition_den_crane.sql | 9 +++++++++ 2 files changed, 10 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00816_key_condition_den_crane.reference create mode 100644 dbms/tests/queries/0_stateless/00816_key_condition_den_crane.sql diff --git a/dbms/tests/queries/0_stateless/00816_key_condition_den_crane.reference b/dbms/tests/queries/0_stateless/00816_key_condition_den_crane.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00816_key_condition_den_crane.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00816_key_condition_den_crane.sql b/dbms/tests/queries/0_stateless/00816_key_condition_den_crane.sql new file mode 100644 index 00000000000..abc30e73a66 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00816_key_condition_den_crane.sql @@ -0,0 +1,9 @@ +USE test; + +drop table if exists testSegmtFault; +CREATE TABLE testSegmtFault (key1 Int32, id1 Int64, c1 Int64) ENGINE = MergeTree PARTITION BY id1 ORDER BY (key1); +insert into testSegmtFault values ( -1, 1, 0 ); + +SELECT count(*) FROM testSegmtFault PREWHERE id1 IN (1); + +drop table testSegmtFault; From d5b00298ebe7b94884674fe09b6497d3097c5537 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Dec 2018 23:35:05 +0300 Subject: [PATCH 226/230] Revert "Added a test by Denis Zhuravlev #2170" This reverts commit cb6317995120cf82a38a5df84b6cc1525cf32d04. --- .../0_stateless/00816_key_condition_den_crane.reference | 1 - .../0_stateless/00816_key_condition_den_crane.sql | 9 --------- 2 files changed, 10 deletions(-) delete mode 100644 dbms/tests/queries/0_stateless/00816_key_condition_den_crane.reference delete mode 100644 dbms/tests/queries/0_stateless/00816_key_condition_den_crane.sql diff --git a/dbms/tests/queries/0_stateless/00816_key_condition_den_crane.reference b/dbms/tests/queries/0_stateless/00816_key_condition_den_crane.reference deleted file mode 100644 index d00491fd7e5..00000000000 --- a/dbms/tests/queries/0_stateless/00816_key_condition_den_crane.reference +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/dbms/tests/queries/0_stateless/00816_key_condition_den_crane.sql b/dbms/tests/queries/0_stateless/00816_key_condition_den_crane.sql deleted file mode 100644 index abc30e73a66..00000000000 --- a/dbms/tests/queries/0_stateless/00816_key_condition_den_crane.sql +++ /dev/null @@ -1,9 +0,0 @@ -USE test; - -drop table if exists testSegmtFault; -CREATE TABLE testSegmtFault (key1 Int32, id1 Int64, c1 Int64) ENGINE = MergeTree PARTITION BY id1 ORDER BY (key1); -insert into testSegmtFault values ( -1, 1, 0 ); - -SELECT count(*) FROM testSegmtFault PREWHERE id1 IN (1); - -drop table testSegmtFault; From 37251167930235272fb7a8df673a761e257a9c2a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 28 Dec 2018 23:58:52 +0300 Subject: [PATCH 227/230] Added a test from Boris Granveaud #3421 --- .../bugs/00816_concurrent_alter_column.sh | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) create mode 100755 dbms/tests/queries/bugs/00816_concurrent_alter_column.sh diff --git a/dbms/tests/queries/bugs/00816_concurrent_alter_column.sh b/dbms/tests/queries/bugs/00816_concurrent_alter_column.sh new file mode 100755 index 00000000000..63e8e48e0bb --- /dev/null +++ b/dbms/tests/queries/bugs/00816_concurrent_alter_column.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +echo "DROP TABLE IF EXISTS test.test" | ${CLICKHOUSE_CLIENT} +echo "CREATE TABLE test.test (ts DATETIME) ENGINE = MergeTree PARTITION BY toStartOfDay(ts) ORDER BY tuple()" | ${CLICKHOUSE_CLIENT} + +for i in {1..500}; do echo "ALTER TABLE test.test ADD COLUMN c$i DOUBLE;"; done | ${CLICKHOUSE_CLIENT} -n + +for i in {1..500}; do echo "ALTER TABLE test.test ADD COLUMN d DOUBLE" | ${CLICKHOUSE_CLIENT}; echo "ALTER TABLE test.test DROP COLUMN d" | ${CLICKHOUSE_CLIENT} -n; done & +for i in {1..500}; do echo "ALTER TABLE test.test ADD COLUMN e DOUBLE" | ${CLICKHOUSE_CLIENT}; echo "ALTER TABLE test.test DROP COLUMN e" | ${CLICKHOUSE_CLIENT} -n; done & + +wait + +echo "DROP TABLE test.test" | ${CLICKHOUSE_CLIENT} From 9f3b5a11e7cd59c2fe1ad41fd5e20f0a9e0a430a Mon Sep 17 00:00:00 2001 From: vicdashkov Date: Fri, 28 Dec 2018 13:58:02 -0800 Subject: [PATCH 228/230] added description for secure argument to cli docs (#3961) * added --secure option to docs * added lost dot in flag description --- docs/en/interfaces/cli.md | 2 ++ docs/ru/interfaces/cli.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index d665fbd3e17..632049b5901 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -92,6 +92,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--time, -t` – If specified, print the query execution time to 'stderr' in non-interactive mode. - `--stacktrace` – If specified, also print the stack trace if an exception occurs. - `--config-file` – The name of the configuration file. +- `--secure` – If specified, will connect to server over secure connection. ### Configuration Files @@ -108,6 +109,7 @@ Example of a config file: username password + False ``` diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index eaf2cf132af..4cb4742078e 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -95,6 +95,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA - `--time, -t` - если указано, в неинтерактивном режиме вывести время выполнения запроса в stderr. - `--stacktrace` - если указано, в случае исключения, выводить также его стек трейс. - `--config-file` - имя конфигурационного файла. +- `--secure` - если указано, будет использован безопасный канал. ### Конфигурационные файлы @@ -111,6 +112,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA username password + False ``` [Оригинальная статья](https://clickhouse.yandex/docs/ru/interfaces/cli/) From e5729c58b1bb72cdbcee3595e3132822491011dc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Dec 2018 03:31:49 +0300 Subject: [PATCH 229/230] Added bug [#CLICKHOUSE-4235] --- dbms/tests/queries/bugs/view_bad_types.sql | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 dbms/tests/queries/bugs/view_bad_types.sql diff --git a/dbms/tests/queries/bugs/view_bad_types.sql b/dbms/tests/queries/bugs/view_bad_types.sql new file mode 100644 index 00000000000..38daabfd6b8 --- /dev/null +++ b/dbms/tests/queries/bugs/view_bad_types.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS test.table; +CREATE TABLE test.table (x UInt16) ENGINE = TinyLog; +INSERT INTO test.table SELECT * FROM system.numbers LIMIT 10; + +DROP TABLE IF EXISTS test.view; +CREATE VIEW test.view (x UInt64) AS SELECT * FROM test.table; + +SELECT x, any(x) FROM test.view GROUP BY x; + +DROP TABLE test.view; +DROP TABLE test.table; From cd6a1c9bc9c81a3c19f1018abe4a89c0c29890d3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Dec 2018 03:53:11 +0300 Subject: [PATCH 230/230] Fixed uninitialized memory in "runningDifference" function [#CLICKHOUSE-2] --- dbms/src/Functions/runningDifference.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/src/Functions/runningDifference.h b/dbms/src/Functions/runningDifference.h index 4059d6f3ada..9f525b24526 100644 --- a/dbms/src/Functions/runningDifference.h +++ b/dbms/src/Functions/runningDifference.h @@ -59,7 +59,10 @@ private: for (size_t i = 0; i < size; ++i) { if (null_map && (*null_map)[i]) + { + dst[i] = Dst{}; continue; + } if (!has_prev_value) {