Merge pull request #68733 from ClickHouse/write-metadata-to-disk-and-zk-in-same-format

Write metadata to disk and keeper in the same format
This commit is contained in:
Miсhael Stetsyuk 2024-08-23 10:51:15 +00:00 committed by GitHub
commit a4ee666ec5
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
6 changed files with 128 additions and 13 deletions

View File

@ -113,7 +113,15 @@ bool ColumnDescription::operator==(const ColumnDescription & other) const
&& ast_to_str(ttl) == ast_to_str(other.ttl);
}
void ColumnDescription::writeText(WriteBuffer & buf) const
String formatASTStateAware(IAST & ast, IAST::FormatState & state)
{
WriteBufferFromOwnString buf;
IAST::FormatSettings settings(buf, true, false);
ast.formatImpl(settings, state, IAST::FormatStateStacked());
return buf.str();
}
void ColumnDescription::writeText(WriteBuffer & buf, IAST::FormatState & state, bool include_comment) const
{
/// NOTE: Serialization format is insane.
@ -126,20 +134,21 @@ void ColumnDescription::writeText(WriteBuffer & buf) const
writeChar('\t', buf);
DB::writeText(DB::toString(default_desc.kind), buf);
writeChar('\t', buf);
writeEscapedString(queryToString(default_desc.expression), buf);
writeEscapedString(formatASTStateAware(*default_desc.expression, state), buf);
}
if (!comment.empty())
if (!comment.empty() && include_comment)
{
writeChar('\t', buf);
DB::writeText("COMMENT ", buf);
writeEscapedString(queryToString(ASTLiteral(Field(comment))), buf);
auto ast = ASTLiteral(Field(comment));
writeEscapedString(formatASTStateAware(ast, state), buf);
}
if (codec)
{
writeChar('\t', buf);
writeEscapedString(queryToString(codec), buf);
writeEscapedString(formatASTStateAware(*codec, state), buf);
}
if (!settings.empty())
@ -150,21 +159,21 @@ void ColumnDescription::writeText(WriteBuffer & buf) const
ASTSetQuery ast;
ast.is_standalone = false;
ast.changes = settings;
writeEscapedString(queryToString(ast), buf);
writeEscapedString(formatASTStateAware(ast, state), buf);
DB::writeText(")", buf);
}
if (!statistics.empty())
{
writeChar('\t', buf);
writeEscapedString(queryToString(statistics.getAST()), buf);
writeEscapedString(formatASTStateAware(*statistics.getAST(), state), buf);
}
if (ttl)
{
writeChar('\t', buf);
DB::writeText("TTL ", buf);
writeEscapedString(queryToString(ttl), buf);
writeEscapedString(formatASTStateAware(*ttl, state), buf);
}
writeChar('\n', buf);
@ -895,16 +904,17 @@ void ColumnsDescription::resetColumnTTLs()
}
String ColumnsDescription::toString() const
String ColumnsDescription::toString(bool include_comments) const
{
WriteBufferFromOwnString buf;
IAST::FormatState ast_format_state;
writeCString("columns format version: 1\n", buf);
DB::writeText(columns.size(), buf);
writeCString(" columns:\n", buf);
for (const ColumnDescription & column : columns)
column.writeText(buf);
column.writeText(buf, ast_format_state, include_comments);
return buf.str();
}

View File

@ -104,7 +104,7 @@ struct ColumnDescription
bool operator==(const ColumnDescription & other) const;
bool operator!=(const ColumnDescription & other) const { return !(*this == other); }
void writeText(WriteBuffer & buf) const;
void writeText(WriteBuffer & buf, IAST::FormatState & state, bool include_comment) const;
void readText(ReadBuffer & buf);
};
@ -137,7 +137,7 @@ public:
/// NOTE Must correspond with Nested::flatten function.
void flattenNested(); /// TODO: remove, insert already flattened Nested columns.
bool operator==(const ColumnsDescription & other) const { return columns == other.columns; }
bool operator==(const ColumnsDescription & other) const { return toString(false) == other.toString(false); }
bool operator!=(const ColumnsDescription & other) const { return !(*this == other); }
auto begin() const { return columns.begin(); }
@ -221,7 +221,7 @@ public:
/// Does column has non default specified compression codec
bool hasCompressionCodec(const String & column_name) const;
String toString() const;
String toString(bool include_comments = true) const;
static ColumnsDescription parse(const String & str);
size_t size() const

View File

@ -0,0 +1,26 @@
<clickhouse>
<keeper_server>
<tcp_port>2181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<session_timeout_ms>20000</session_timeout_ms>
</coordination_settings>
<raft_configuration>
<server>
<id>1</id>
<hostname>localhost</hostname>
<port>9444</port>
</server>
</raft_configuration>
</keeper_server>
<zookeeper>
<node index="1">
<host>localhost</host>
<port>2181</port>
</node>
<session_timeout_ms>20000</session_timeout_ms>
</zookeeper>
</clickhouse>

View File

@ -0,0 +1,8 @@
<clickhouse>
<users>
<default>
<profile>default</profile>
<no_password></no_password>
</default>
</users>
</clickhouse>

View File

@ -0,0 +1,71 @@
import pytest
import random
import string
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node",
main_configs=[
"config/enable_keeper.xml",
"config/users.xml",
],
stay_alive=True,
with_minio=True,
macros={"shard": 1, "replica": 1},
)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def randomize_table_name(table_name, random_suffix_length=10):
letters = string.ascii_letters + string.digits
return f"{table_name}_{''.join(random.choice(letters) for _ in range(random_suffix_length))}"
@pytest.mark.parametrize("engine", ["ReplicatedMergeTree"])
def test_aliases_in_default_expr_not_break_table_structure(start_cluster, engine):
"""
Making sure that using aliases in columns' default expressions does not lead to having different columns metadata in ZooKeeper and on disk.
Issue: https://github.com/ClickHouse/clickhouse-private/issues/5150
"""
data = '{"event": {"col1-key": "col1-val", "col2-key": "col2-val"}}'
table_name = randomize_table_name("t")
node.query(
f"""
DROP TABLE IF EXISTS {table_name};
CREATE TABLE {table_name}
(
`data` String,
`col1` String DEFAULT JSONExtractString(JSONExtractString(data, 'event') AS event, 'col1-key'),
`col2` String MATERIALIZED JSONExtractString(JSONExtractString(data, 'event') AS event, 'col2-key')
)
ENGINE = {engine}('/test/{table_name}', '{{replica}}')
ORDER BY col1
"""
)
node.restart_clickhouse()
node.query(
f"""
INSERT INTO {table_name} (data) VALUES ('{data}');
"""
)
assert node.query(f"SELECT data FROM {table_name}").strip() == data
assert node.query(f"SELECT col1 FROM {table_name}").strip() == "col1-val"
assert node.query(f"SELECT col2 FROM {table_name}").strip() == "col2-val"
node.query(f"DROP TABLE {table_name}")