Merge pull request #61734 from vitlibar/fix-columns-after-alter-table-modify-query

Fix columns after executing MODIFY QUERY for a materialized view with internal table
This commit is contained in:
Vitaly Baranov 2024-03-26 15:35:22 +01:00 committed by GitHub
commit 83e006e372
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 249 additions and 6 deletions

View File

@ -70,6 +70,16 @@ static void removeNonCommonColumns(const Block & src_header, Block & target_head
target_header.erase(target_only_positions);
}
namespace
{
void checkTargetTableHasQueryOutputColumns(const ColumnsDescription & target_table_columns, const ColumnsDescription & select_query_output_columns)
{
for (const auto & column : select_query_output_columns)
if (!target_table_columns.has(column.name))
throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "Column {} does not exist in the materialized view's inner table", column.name);
}
}
StorageMaterializedView::StorageMaterializedView(
const StorageID & table_id_,
ContextPtr local_context,
@ -402,11 +412,13 @@ void StorageMaterializedView::alter(
/// Check the materialized view's inner table structure.
if (has_inner_table)
{
const Block & block = InterpreterSelectWithUnionQuery::getSampleBlock(new_select.select_query, local_context);
const auto & inner_table_metadata = tryGetTargetTable()->getInMemoryMetadata().columns;
for (const auto & name : block.getNames())
if (!inner_table_metadata.has(name))
throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "Column {} does not exist in the materialized view's inner table", name);
/// If this materialized view has an inner table it should always have the same columns as this materialized view.
/// Try to find mistakes in the select query (it shouldn't have columns which are not in the inner table).
auto target_table_metadata = getTargetTable()->getInMemoryMetadataPtr();
const auto & select_query_output_columns = new_metadata.columns; /// AlterCommands::alter() analyzed the query and assigned `new_metadata.columns` before.
checkTargetTableHasQueryOutputColumns(target_table_metadata->columns, select_query_output_columns);
/// We need to copy the target table's columns (after checkTargetTableHasQueryOutputColumns() they can be still different - e.g. the data types of those columns can differ).
new_metadata.columns = target_table_metadata->columns;
}
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata);

View File

@ -71,6 +71,7 @@ namespace ErrorCodes
extern const int QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW;
extern const int SUPPORT_IS_DISABLED;
extern const int TABLE_WAS_NOT_DROPPED;
extern const int NO_SUCH_COLUMN_IN_TABLE;
extern const int NOT_IMPLEMENTED;
extern const int UNSUPPORTED_METHOD;
}
@ -339,6 +340,13 @@ namespace
table_expr->children.push_back(table_expr->database_and_table_name);
return fetch_query;
}
void checkTargetTableHasQueryOutputColumns(const ColumnsDescription & target_table_columns, const ColumnsDescription & select_query_output_columns)
{
for (const auto & column : select_query_output_columns)
if (!target_table_columns.has(column.name))
throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "Column {} does not exist in the window view's inner table", column.name);
}
}
static void extractDependentTable(ContextPtr context, ASTPtr & query, String & select_database_name, String & select_table_name)
@ -482,6 +490,18 @@ void StorageWindowView::alter(
new_metadata.setSelectQuery(new_select);
/// Check the window view's inner target table structure.
if (has_inner_target_table)
{
/// If this window view has an inner target table it should always have the same columns as this window view.
/// Try to find mistakes in the select query (it shouldn't have columns which are not in the inner target table).
auto target_table_metadata = getTargetTable()->getInMemoryMetadataPtr();
const auto & select_query_output_columns = new_metadata.columns; /// AlterCommands::alter() analyzed the query and assigned `new_metadata.columns` before.
checkTargetTableHasQueryOutputColumns(target_table_metadata->columns, select_query_output_columns);
/// We need to copy the target table's columns (after checkTargetTableHasQueryOutputColumns() they can be still different - e.g. in data types).
new_metadata.columns = target_table_metadata->columns;
}
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata);
setInMemoryMetadata(new_metadata);

View File

@ -4,7 +4,7 @@ CREATE MATERIALIZED VIEW default.a\nREFRESH AFTER 2 SECOND\n(\n `x` UInt64\n)
<3: time difference at least> 1000
<4: next refresh in> 2
<4.5: altered> Scheduled Finished 2052-01-01 00:00:00
CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` Int16\n)\nENGINE = Memory\nAS SELECT x * 2 AS x\nFROM default.src
CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT x * 2 AS x\nFROM default.src
<5: no refresh> 3
<6: refreshed> 2
<7: refreshed> Scheduled Finished 2054-01-01 00:00:00

View File

@ -0,0 +1,16 @@
mv before:
timestamp c12
DateTime Nullable(String)
2024-02-22 00:00:00 00
2024-02-22 00:00:01 11
2024-02-22 00:00:02 22
BACKUP_CREATED
RESTORED
mv after:
timestamp c12
DateTime Nullable(String)
2024-02-22 00:00:00 00
2024-02-22 00:00:01 11
2024-02-22 00:00:02 22

View File

@ -0,0 +1,35 @@
#!/usr/bin/env bash
# Tags: no-ordinary-database, no-replicated-database
# Tag no-ordinary-database: TO DO
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
db="$CLICKHOUSE_DATABASE"
db_2="${db}_2"
backup_name="${db}_backup"
${CLICKHOUSE_CLIENT} --multiquery "
DROP TABLE IF EXISTS src;
DROP TABLE IF EXISTS mv;
CREATE TABLE src(Timestamp DateTime64(9), c1 String, c2 String) ENGINE=MergeTree ORDER BY Timestamp;
"
${CLICKHOUSE_CLIENT} -q "CREATE MATERIALIZED VIEW mv(timestamp DateTime, c12 Nullable(String)) ENGINE=MergeTree ORDER BY timestamp AS SELECT Timestamp as timestamp, c1 || c2 as c12 FROM src"
${CLICKHOUSE_CLIENT} -q "INSERT INTO src SELECT '2024-02-22'::DateTime + number, number, number FROM numbers(3)"
echo 'mv before:'
${CLICKHOUSE_CLIENT} -q "SELECT * FROM ${db}.mv ORDER BY timestamp FORMAT TSVWithNamesAndTypes"
${CLICKHOUSE_CLIENT} -q "ALTER TABLE mv MODIFY QUERY SELECT Timestamp as timestamp, c1 || c2 as c12 FROM src"
echo
${CLICKHOUSE_CLIENT} -q "BACKUP DATABASE $db TO Disk('backups', '${backup_name}')" | grep -o "BACKUP_CREATED"
${CLICKHOUSE_CLIENT} -q "RESTORE DATABASE $db AS ${db_2} FROM Disk('backups', '${backup_name}')" | grep -o "RESTORED"
echo $'\nmv after:'
${CLICKHOUSE_CLIENT} -q "SELECT * FROM ${db_2}.mv ORDER BY timestamp FORMAT TSVWithNamesAndTypes"
${CLICKHOUSE_CLIENT} -q "DROP DATABASE ${db_2}"

View File

@ -0,0 +1,86 @@
src:
Timestamp c1 c2
DateTime64(9) String String
2024-02-22 00:00:00.000000000 0 0
2024-02-22 00:00:01.000000000 1 1
2024-02-22 00:00:02.000000000 2 2
mv:
timestamp c12
DateTime Nullable(String)
2024-02-22 00:00:00 00
2024-02-22 00:00:01 11
2024-02-22 00:00:02 22
inner:
timestamp c12
DateTime Nullable(String)
2024-02-22 00:00:00 00
2024-02-22 00:00:01 11
2024-02-22 00:00:02 22
Test 1. MODIFY QUERY doesn't change columns.
mv:
timestamp c12
DateTime Nullable(String)
2024-02-22 00:00:00 00
2024-02-22 00:00:01 11
2024-02-22 00:00:02 22
inner:
timestamp c12
DateTime Nullable(String)
2024-02-22 00:00:00 00
2024-02-22 00:00:01 11
2024-02-22 00:00:02 22
Test 2. MODIFY QUERY with explicit data types doesn't change columns.
mv:
timestamp c12
DateTime Nullable(String)
2024-02-22 00:00:00 00
2024-02-22 00:00:01 11
2024-02-22 00:00:02 22
inner:
timestamp c12
DateTime Nullable(String)
2024-02-22 00:00:00 00
2024-02-22 00:00:01 11
2024-02-22 00:00:02 22
Test 3. MODIFY QUERY can even fix wrong columns.
Before MODIFY QUERY:
mv:
timestamp c12
DateTime64(9) String
2024-02-22 00:00:00.000000000 00
2024-02-22 00:00:01.000000000 11
2024-02-22 00:00:02.000000000 22
inner:
timestamp c12
DateTime Nullable(String)
2024-02-22 00:00:00 00
2024-02-22 00:00:01 11
2024-02-22 00:00:02 22
After MODIFY QUERY:
mv:
timestamp c12
DateTime Nullable(String)
2024-02-22 00:00:00 00
2024-02-22 00:00:01 11
2024-02-22 00:00:02 22
inner:
timestamp c12
DateTime Nullable(String)
2024-02-22 00:00:00 00
2024-02-22 00:00:01 11
2024-02-22 00:00:02 22

View File

@ -0,0 +1,74 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --multiquery "
DROP TABLE IF EXISTS src;
DROP TABLE IF EXISTS mv;
CREATE TABLE src(Timestamp DateTime64(9), c1 String, c2 String) ENGINE=MergeTree ORDER BY Timestamp;
"
${CLICKHOUSE_CLIENT} -q "CREATE MATERIALIZED VIEW mv(timestamp DateTime, c12 Nullable(String)) ENGINE=MergeTree ORDER BY timestamp AS SELECT Timestamp as timestamp, c1 || c2 as c12 FROM src"
mv_uuid=$(${CLICKHOUSE_CLIENT} -q "SELECT uuid FROM system.tables WHERE table='mv' AND database=currentDatabase()")
if [ "${mv_uuid}" != "00000000-0000-0000-0000-000000000000" ]; then
inner_table_name=".inner_id.${mv_uuid}"
else
inner_table_name=".inner.mv"
fi
#echo "inner_table_name=$inner_table_name"
${CLICKHOUSE_CLIENT} -q "INSERT INTO src SELECT '2024-02-22'::DateTime + number, number, number FROM numbers(3)"
echo $'src:'
${CLICKHOUSE_CLIENT} -q "SELECT * FROM src ORDER BY Timestamp FORMAT TSVWithNamesAndTypes"
function show_mv_and_inner()
{
echo $'\nmv:'
${CLICKHOUSE_CLIENT} -q "SELECT * FROM mv ORDER BY timestamp FORMAT TSVWithNamesAndTypes"
echo $'\ninner:'
${CLICKHOUSE_CLIENT} -q "SELECT * FROM \`$inner_table_name\` ORDER BY timestamp FORMAT TSVWithNamesAndTypes"
}
show_mv_and_inner
#################
echo $'\nTest 1. MODIFY QUERY doesn\'t change columns.'
${CLICKHOUSE_CLIENT} -q "ALTER TABLE mv MODIFY QUERY SELECT Timestamp as timestamp, c1 || c2 as c12 FROM src"
show_mv_and_inner
#################
echo $'\nTest 2. MODIFY QUERY with explicit data types doesn\'t change columns.'
${CLICKHOUSE_CLIENT} -q "ALTER TABLE mv MODIFY QUERY SELECT Timestamp::DateTime64(9) as timestamp, (c1 || c2)::String as c12 FROM src"
show_mv_and_inner
#################
echo $'\nTest 3. MODIFY QUERY can even fix wrong columns.' # We need that because of https://github.com/ClickHouse/ClickHouse/issues/60369
mv_metadata_path=$(${CLICKHOUSE_CLIENT} -q "SELECT metadata_path FROM system.tables WHERE table='mv' AND database=currentDatabase()")
${CLICKHOUSE_CLIENT} -q "DETACH TABLE mv"
#cat $mv_metadata_path
sed -i -e 's/`timestamp` DateTime,/`timestamp` DateTime64(9),/g' -e 's/`c12` Nullable(String)/`c12` String/g' "$mv_metadata_path"
#cat $mv_metadata_path
${CLICKHOUSE_CLIENT} -q "ATTACH TABLE mv"
echo $'\nBefore MODIFY QUERY:'
show_mv_and_inner
${CLICKHOUSE_CLIENT} -q "ALTER TABLE mv MODIFY QUERY SELECT Timestamp as timestamp, c1 || c2 as c12 FROM src"
echo $'\nAfter MODIFY QUERY:'
show_mv_and_inner