Fixed tests

This commit is contained in:
Maksim Kita 2024-02-14 17:47:51 +03:00
parent 0cce386207
commit d4778d858d
7 changed files with 70 additions and 9 deletions

View File

@ -27,7 +27,7 @@ Columns:
- `metadata_modification_time` ([DateTime](../../sql-reference/data-types/datetime.md)) - Time of latest modification of the table metadata.
- `metadata_version` ([Int32](../../sql-reference/data-types/int-uint.md)) - Metadata version for ReplicatedMergeTree table, -1 for non replicated table.
- `metadata_version` ([Int32](../../sql-reference/data-types/int-uint.md)) - Metadata version for ReplicatedMergeTree table, 0 for non ReplicatedMergeTree table.
- `dependencies_database` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) - Database dependencies.

View File

@ -43,7 +43,7 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_)
{"data_paths", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "Paths to the table data in the file systems."},
{"metadata_path", std::make_shared<DataTypeString>(), "Path to the table metadata in the file system."},
{"metadata_modification_time", std::make_shared<DataTypeDateTime>(), "Time of latest modification of the table metadata."},
{"metadata_version", std::make_shared<DataTypeInt32>(), "Metadata version for ReplicatedMergeTree table."},
{"metadata_version", std::make_shared<DataTypeInt32>(), "Metadata version for ReplicatedMergeTree table, 0 for non ReplicatedMergeTree table."},
{"dependencies_database", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "Database dependencies."},
{"dependencies_table", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "Table dependencies (materialized views the current table)."},
{"create_table_query", std::make_shared<DataTypeString>(), "The query that was used to create the table."},
@ -291,7 +291,7 @@ protected:
// metadata_version
// Temporary tables does not support replication
if (columns_mask[src_index++])
res_columns[res_index++]->insert(-1);
res_columns[res_index++]->insertDefault();
// dependencies_database
if (columns_mask[src_index++])
@ -317,7 +317,7 @@ protected:
while (src_index < columns_mask.size())
{
// total_rows
if (src_index == 18 && columns_mask[src_index])
if (src_index == 19 && columns_mask[src_index])
{
if (auto total_rows = table.second->totalRows(settings))
res_columns[res_index++]->insert(*total_rows);
@ -325,7 +325,7 @@ protected:
res_columns[res_index++]->insertDefault();
}
// total_bytes
else if (src_index == 19 && columns_mask[src_index])
else if (src_index == 20 && columns_mask[src_index])
{
if (auto total_bytes = table.second->totalBytes(settings))
res_columns[res_index++]->insert(*total_bytes);
@ -433,7 +433,7 @@ protected:
if (metadata_snapshot && table->supportsReplication())
res_columns[res_index++]->insert(metadata_snapshot->metadata_version);
else
res_columns[res_index++]->insert(-1);
res_columns[res_index++]->insertDefault();
}
{

View File

@ -11,10 +11,11 @@ std::pair<std::vector<UInt8>, Block> getQueriedColumnsMaskAndHeader(const Block
NameSet names_set(column_names.begin(), column_names.end());
for (size_t i = 0; i < columns_mask.size(); ++i)
{
if (names_set.contains(sample_block.getByPosition(i).name))
const auto & column_with_type_and_name = sample_block.getByPosition(i);
if (names_set.contains(column_with_type_and_name.name))
{
columns_mask[i] = 1;
header.insert(sample_block.getByPosition(i));
header.insert(column_with_type_and_name);
}
}

View File

@ -1083,6 +1083,7 @@ CREATE TABLE system.tables
`data_paths` Array(String),
`metadata_path` String,
`metadata_modification_time` DateTime,
`metadata_version` Int32,
`dependencies_database` Array(String),
`dependencies_table` Array(String),
`create_table_query` String,

View File

@ -1,6 +1,6 @@
/* we will `use system` to bypass style check,
because `show create table` statement
cannot fit the requirement in check-sytle, which is as
cannot fit the requirement in check-style, which is as
"# Queries to:
tables_with_database_column=(

View File

@ -0,0 +1,9 @@
test_temporary_table_02989 0
--
test_table 0
--
test_table_replicated 0
--
test_table_replicated 1
--
test_table_replicated 2

View File

@ -0,0 +1,50 @@
-- Tags: zookeeper, no-parallel
DROP TABLE IF EXISTS test_temporary_table_02989;
CREATE TEMPORARY TABLE test_temporary_table_02989
(
id UInt64,
value String
) ENGINE=MergeTree ORDER BY id;
SELECT name, metadata_version FROM system.tables WHERE name = 'test_temporary_table_02989' AND is_temporary;
DROP TABLE test_temporary_table_02989;
DROP TABLE IF EXISTS test_table;
CREATE TABLE test_table
(
id UInt64,
value String
) ENGINE=MergeTree ORDER BY id;
SELECT '--';
SELECT name, metadata_version FROM system.tables WHERE database = currentDatabase() AND name = 'test_table';
DROP TABLE test_table;
DROP TABLE IF EXISTS test_table_replicated;
CREATE TABLE test_table_replicated
(
id UInt64,
value String
) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_table_replicated', '1_replica') ORDER BY id;
SELECT '--';
SELECT name, metadata_version FROM system.tables WHERE database = currentDatabase() AND name = 'test_table_replicated';
ALTER TABLE test_table_replicated ADD COLUMN insert_time DateTime;
SELECT '--';
SELECT name, metadata_version FROM system.tables WHERE database = currentDatabase() AND name = 'test_table_replicated';
ALTER TABLE test_table_replicated ADD COLUMN insert_time_updated DateTime;
SELECT '--';
SELECT name, metadata_version FROM system.tables WHERE database = currentDatabase() AND name = 'test_table_replicated';
DROP TABLE test_table_replicated;