未验证 提交 11b4bc71 编写于 作者: A Anton Popov 提交者: GitHub

Merge pull request #10889 from CurtizJ/tuple-literal

Fix backward compatibility with tuples and distributed.
......@@ -2,6 +2,8 @@
#include <Common/FieldVisitors.h>
#include <Parsers/ASTLiteral.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
namespace DB
......@@ -14,30 +16,59 @@ void ASTLiteral::updateTreeHashImpl(SipHash & hash_state) const
applyVisitor(FieldVisitorHash(hash_state), value);
}
/// Writes 'tuple' word before tuple literals for backward compatibility reasons.
/// TODO: remove, when versions lower than 20.3 will be rearely used.
class FieldVisitorToColumnName : public StaticVisitor<String>
{
public:
template<typename T>
String operator() (const T & x) const { return visitor(x); }
private:
FieldVisitorToString visitor;
};
template<>
String FieldVisitorToColumnName::operator() (const Tuple & x) const
{
WriteBufferFromOwnString wb;
wb << "tuple(";
for (auto it = x.begin(); it != x.end(); ++it)
{
if (it != x.begin())
wb << ", ";
wb << applyVisitor(*this, *it);
}
wb << ')';
return wb.str();
}
void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const
{
/// 100 - just arbitrary value.
constexpr auto min_elements_for_hashing = 100;
/// Special case for very large arrays and tuples. Instead of listing all elements, will use hash of them.
/// Special case for very large arrays. Instead of listing all elements, will use hash of them.
/// (Otherwise column name will be too long, that will lead to significant slowdown of expression analysis.)
/// TODO: Also do hashing for large tuples, when versions lower than 20.3 will be rarely used, because it breaks backward compatibility.
auto type = value.getType();
if ((type == Field::Types::Array && value.get<const Array &>().size() > min_elements_for_hashing)
|| (type == Field::Types::Tuple && value.get<const Tuple &>().size() > min_elements_for_hashing))
if ((type == Field::Types::Array && value.get<const Array &>().size() > min_elements_for_hashing))
{
SipHash hash;
applyVisitor(FieldVisitorHash(hash), value);
UInt64 low, high;
hash.get128(low, high);
writeCString(type == Field::Types::Array ? "__array_" : "__tuple_", ostr);
writeCString("__array_", ostr);
writeText(low, ostr);
ostr.write('_');
writeText(high, ostr);
}
else
{
String column_name = applyVisitor(FieldVisitorToString(), value);
String column_name = applyVisitor(FieldVisitorToColumnName(), value);
writeString(column_name, ostr);
}
}
......
<yandex>
<remote_servers>
<test_cluster>
<shard>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>node2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</yandex>
import pytest
import time
from helpers.cluster import ClickHouseCluster
from helpers.network import PartitionManager
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
node_old = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], image='yandex/clickhouse-server:19.17.8.54', stay_alive=True, with_installed_binary=True)
node_new = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'])
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
for node in (node_old, node_new):
node.query("CREATE TABLE local_table(id UInt32, val String) ENGINE = MergeTree ORDER BY id")
node_old.query("INSERT INTO local_table VALUES (1, 'node1')")
node_new.query("INSERT INTO local_table VALUES (2, 'node2')")
node_old.query("CREATE TABLE distributed(id UInt32, val String) ENGINE = Distributed(test_cluster, default, local_table)")
node_new.query("CREATE TABLE distributed(id UInt32, val String) ENGINE = Distributed(test_cluster, default, local_table)")
yield cluster
finally:
cluster.shutdown()
def test_distributed_in_tuple(started_cluster):
query1 = "SELECT count() FROM distributed WHERE (id, val) IN ((1, 'node1'), (2, 'a'), (3, 'b'))"
query2 = "SELECT sum((id, val) IN ((1, 'node1'), (2, 'a'), (3, 'b'))) FROM distributed"
assert node_old.query(query1) == "1\n"
assert node_old.query(query2) == "1\n"
assert node_new.query(query1) == "1\n"
assert node_new.query(query2) == "1\n"
large_set = '(' + ','.join([str(i) for i in range(1000)]) + ')'
query3 = "SELECT count() FROM distributed WHERE id IN " + large_set
query4 = "SELECT sum(id IN {}) FROM distributed".format(large_set)
assert node_old.query(query3) == "2\n"
assert node_old.query(query4) == "2\n"
assert node_new.query(query3) == "2\n"
assert node_new.query(query4) == "2\n"
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册