mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Merge pull request #29266 from CurtizJ/normalize-asts
Normalize ASTs in ColumnsDescription
This commit is contained in:
commit
7628273df4
@ -29,6 +29,7 @@
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/FunctionNameNormalizer.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -201,6 +202,12 @@ void ColumnsDescription::add(ColumnDescription column, const String & after_colu
|
||||
throw Exception("Cannot add column " + column.name + ": column with this name already exists",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
/// Normalize ASTs to be compatible with InterpreterCreateQuery.
|
||||
if (column.default_desc.expression)
|
||||
FunctionNameNormalizer::visit(column.default_desc.expression.get());
|
||||
if (column.ttl)
|
||||
FunctionNameNormalizer::visit(column.ttl.get());
|
||||
|
||||
auto insert_it = columns.cend();
|
||||
|
||||
if (first)
|
||||
|
25
src/Storages/tests/gtest_columns_description_normalize.cpp
Normal file
25
src/Storages/tests/gtest_columns_description_normalize.cpp
Normal file
@ -0,0 +1,25 @@
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Common/tests/gtest_global_register.h>
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
TEST(ColumnsDescription, Normalize)
|
||||
{
|
||||
constexpr auto columns = "columns format version: 1\n"
|
||||
"3 columns:\n"
|
||||
"`a` UInt32\n"
|
||||
"`b` String\tDEFAULT\tIf(a = 0, 'true', 'false')\n"
|
||||
"`c` String\tDEFAULT\tcAsT(a, 'String')\n";
|
||||
|
||||
constexpr auto columns_normalized = "columns format version: 1\n"
|
||||
"3 columns:\n"
|
||||
"`a` UInt32\n"
|
||||
"`b` String\tDEFAULT\tif(a = 0, 'true', 'false')\n"
|
||||
"`c` String\tDEFAULT\tcast(a, 'String')\n";
|
||||
|
||||
tryRegisterFunctions();
|
||||
|
||||
ASSERT_EQ(ColumnsDescription::parse(columns), ColumnsDescription::parse(columns_normalized));
|
||||
}
|
@ -0,0 +1,55 @@
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance('node1', with_zookeeper=True, image='yandex/clickhouse-server', tag='20.12.4.5', stay_alive=True, with_installed_binary=True)
|
||||
node2 = cluster.add_instance('node2', with_zookeeper=True, image='yandex/clickhouse-server', tag='20.12.4.5', stay_alive=True, with_installed_binary=True)
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
yield cluster
|
||||
|
||||
except Exception as ex:
|
||||
print(ex)
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
def test_replicated_merge_tree_defaults_compatibility(started_cluster):
|
||||
# This test checks, that result of parsing list of columns with defaults
|
||||
# from 'CREATE/ATTACH' is compatible with parsing from zookeeper metadata on different versions.
|
||||
# We create table and write 'columns' node in zookeeper with old version, than restart with new version
|
||||
# drop and try recreate one replica. During startup of table structure is checked between 'CREATE' query and zookeeper.
|
||||
|
||||
create_query = '''
|
||||
CREATE TABLE test.table
|
||||
(
|
||||
a UInt32,
|
||||
b String DEFAULT If(a = 0, 'true', 'false'),
|
||||
c String DEFAULT Cast(a, 'String')
|
||||
)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/table', '{replica}')
|
||||
ORDER BY a
|
||||
'''
|
||||
|
||||
for node in (node1, node2):
|
||||
node.query("CREATE DATABASE test ENGINE = Ordinary")
|
||||
node.query(create_query.format(replica=node.name))
|
||||
|
||||
node1.query("DETACH TABLE test.table")
|
||||
node2.query("SYSTEM DROP REPLICA 'node1' FROM TABLE test.table")
|
||||
node1.exec_in_container(["bash", "-c", "rm /var/lib/clickhouse/metadata/test/table.sql"])
|
||||
node1.exec_in_container(["bash", "-c", "rm -r /var/lib/clickhouse/data/test/table"])
|
||||
|
||||
zk = cluster.get_kazoo_client('zoo1')
|
||||
exists_replica_1 = zk.exists("/clickhouse/tables/test/table/replicas/node1")
|
||||
assert exists_replica_1 == None
|
||||
|
||||
node1.restart_with_latest_version()
|
||||
node2.restart_with_latest_version()
|
||||
|
||||
node1.query(create_query.format(replica=1))
|
||||
node1.query("EXISTS TABLE test.table") == "1\n"
|
Loading…
Reference in New Issue
Block a user