mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge branch 'master' into metric-async-insert-bytes
This commit is contained in:
commit
3924609ab2
@ -100,20 +100,9 @@ bool checkPositionalArguments(ASTPtr & argument, const ASTSelectQuery * select_q
|
||||
{
|
||||
auto columns = select_query->select()->children;
|
||||
|
||||
const auto * group_by_expr_with_alias = dynamic_cast<const ASTWithAlias *>(argument.get());
|
||||
if (group_by_expr_with_alias && !group_by_expr_with_alias->alias.empty())
|
||||
{
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
const auto * col_with_alias = dynamic_cast<const ASTWithAlias *>(column.get());
|
||||
if (col_with_alias)
|
||||
{
|
||||
const auto & alias = col_with_alias->alias;
|
||||
if (!alias.empty() && alias == group_by_expr_with_alias->alias)
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
const auto * expr_with_alias = dynamic_cast<const ASTWithAlias *>(argument.get());
|
||||
if (expr_with_alias && !expr_with_alias->alias.empty())
|
||||
return false;
|
||||
|
||||
const auto * ast_literal = typeid_cast<const ASTLiteral *>(argument.get());
|
||||
if (!ast_literal)
|
||||
@ -130,7 +119,7 @@ bool checkPositionalArguments(ASTPtr & argument, const ASTSelectQuery * select_q
|
||||
pos, columns.size());
|
||||
|
||||
const auto & column = columns[--pos];
|
||||
if (typeid_cast<const ASTIdentifier *>(column.get()))
|
||||
if (typeid_cast<const ASTIdentifier *>(column.get()) || typeid_cast<const ASTLiteral *>(column.get()))
|
||||
{
|
||||
argument = column->clone();
|
||||
}
|
||||
@ -1324,7 +1313,9 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai
|
||||
throw Exception("Bad ORDER BY expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE);
|
||||
|
||||
if (getContext()->getSettingsRef().enable_positional_arguments)
|
||||
{
|
||||
replaceForPositionalArguments(ast->children.at(0), select_query, ASTSelectQuery::Expression::ORDER_BY);
|
||||
}
|
||||
}
|
||||
|
||||
getRootActions(select_query->orderBy(), only_types, step.actions());
|
||||
|
@ -962,18 +962,29 @@ public:
|
||||
/// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin.
|
||||
for (size_t j = 0, size = right_indexes.size(); j < size; ++j)
|
||||
{
|
||||
const auto & column = *block.getByPosition(right_indexes[j]).column;
|
||||
if (auto * nullable_col = typeid_cast<ColumnNullable *>(columns[j].get()); nullable_col && !column.isNullable())
|
||||
nullable_col->insertFromNotNullable(column, row_num);
|
||||
auto column_from_block = block.getByPosition(right_indexes[j]);
|
||||
if (type_name[j].type->lowCardinality() != column_from_block.type->lowCardinality())
|
||||
{
|
||||
JoinCommon::changeLowCardinalityInplace(column_from_block);
|
||||
}
|
||||
|
||||
if (auto * nullable_col = typeid_cast<ColumnNullable *>(columns[j].get());
|
||||
nullable_col && !column_from_block.column->isNullable())
|
||||
nullable_col->insertFromNotNullable(*column_from_block.column, row_num);
|
||||
else
|
||||
columns[j]->insertFrom(column, row_num);
|
||||
columns[j]->insertFrom(*column_from_block.column, row_num);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t j = 0, size = right_indexes.size(); j < size; ++j)
|
||||
{
|
||||
columns[j]->insertFrom(*block.getByPosition(right_indexes[j]).column, row_num);
|
||||
auto column_from_block = block.getByPosition(right_indexes[j]);
|
||||
if (type_name[j].type->lowCardinality() != column_from_block.type->lowCardinality())
|
||||
{
|
||||
JoinCommon::changeLowCardinalityInplace(column_from_block);
|
||||
}
|
||||
columns[j]->insertFrom(*column_from_block.column, row_num);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1013,6 +1024,7 @@ private:
|
||||
|
||||
void addColumn(const ColumnWithTypeAndName & src_column, const std::string & qualified_name)
|
||||
{
|
||||
|
||||
columns.push_back(src_column.column->cloneEmpty());
|
||||
columns.back()->reserve(src_column.column->size());
|
||||
type_name.emplace_back(src_column.type, src_column.name, qualified_name);
|
||||
|
@ -326,9 +326,10 @@ ColumnRawPtrMap materializeColumnsInplaceMap(Block & block, const Names & names)
|
||||
|
||||
for (const auto & column_name : names)
|
||||
{
|
||||
auto & column = block.getByName(column_name).column;
|
||||
column = recursiveRemoveLowCardinality(column->convertToFullColumnIfConst());
|
||||
ptrs[column_name] = column.get();
|
||||
auto & column = block.getByName(column_name);
|
||||
column.column = recursiveRemoveLowCardinality(column.column->convertToFullColumnIfConst());
|
||||
column.type = recursiveRemoveLowCardinality(column.type);
|
||||
ptrs[column_name] = column.column.get();
|
||||
}
|
||||
|
||||
return ptrs;
|
||||
|
@ -279,14 +279,17 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite
|
||||
ProfileEvents::increment(ProfileEvents::DataAfterMergeDiffersFromReplica);
|
||||
|
||||
LOG_ERROR(log,
|
||||
"{}. Data after merge is not byte-identical to data on another replicas. There could be several"
|
||||
" reasons: 1. Using newer version of compression library after server update. 2. Using another"
|
||||
" compression method. 3. Non-deterministic compression algorithm (highly unlikely). 4."
|
||||
" Non-deterministic merge algorithm due to logical error in code. 5. Data corruption in memory due"
|
||||
" to bug in code. 6. Data corruption in memory due to hardware issue. 7. Manual modification of"
|
||||
" source data after server startup. 8. Manual modification of checksums stored in ZooKeeper. 9."
|
||||
" Part format related settings like 'enable_mixed_granularity_parts' are different on different"
|
||||
" replicas. We will download merged part from replica to force byte-identical result.",
|
||||
"{}. Data after merge is not byte-identical to data on another replicas. There could be several reasons:"
|
||||
" 1. Using newer version of compression library after server update."
|
||||
" 2. Using another compression method."
|
||||
" 3. Non-deterministic compression algorithm (highly unlikely)."
|
||||
" 4. Non-deterministic merge algorithm due to logical error in code."
|
||||
" 5. Data corruption in memory due to bug in code."
|
||||
" 6. Data corruption in memory due to hardware issue."
|
||||
" 7. Manual modification of source data after server startup."
|
||||
" 8. Manual modification of checksums stored in ZooKeeper."
|
||||
" 9. Part format related settings like 'enable_mixed_granularity_parts' are different on different replicas."
|
||||
" We will download merged part from replica to force byte-identical result.",
|
||||
getCurrentExceptionMessage(false));
|
||||
|
||||
write_part_log(ExecutionStatus::fromCurrentException());
|
||||
|
@ -185,7 +185,8 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::DataAfterMutationDiffersFromReplica);
|
||||
|
||||
LOG_ERROR(log, "{}. Data after mutation is not byte-identical to data on another replicas. We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false));
|
||||
LOG_ERROR(log, "{}. Data after mutation is not byte-identical to data on another replicas. "
|
||||
"We will download merged part from replica to force byte-identical result.", getCurrentExceptionMessage(false));
|
||||
|
||||
write_part_log(ExecutionStatus::fromCurrentException());
|
||||
|
||||
|
@ -1312,10 +1312,14 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil:
|
||||
|
||||
if (replica_part_header.getColumnsHash() != local_part_header.getColumnsHash())
|
||||
{
|
||||
/// Either it's a bug or ZooKeeper contains broken data.
|
||||
/// TODO Fix KILL MUTATION and replace CHECKSUM_DOESNT_MATCH with LOGICAL_ERROR
|
||||
/// (some replicas may skip killed mutation even if it was executed on other replicas)
|
||||
throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, "Part {} from {} has different columns hash", part_name, replica);
|
||||
/// Currently there are two (known) cases when it may happen:
|
||||
/// - KILL MUTATION query had removed mutation before all replicas have executed assigned MUTATE_PART entries.
|
||||
/// Some replicas may skip this mutation and update part version without actually applying any changes.
|
||||
/// It leads to mismatching checksum if changes were applied on other replicas.
|
||||
/// - ALTER_METADATA and MERGE_PARTS were reordered on some replicas.
|
||||
/// It may lead to different number of columns in merged parts on these replicas.
|
||||
throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, "Part {} from {} has different columns hash "
|
||||
"(it may rarely happen on race condition with KILL MUTATION or ALTER COLUMN).", part_name, replica);
|
||||
}
|
||||
|
||||
replica_part_header.getChecksums().checkEqual(local_part_header.getChecksums(), true);
|
||||
|
@ -111,3 +111,9 @@ select substr('aaaaaaaaaaaaaa', 8) as a group by a;
|
||||
aaaaaaa
|
||||
select substr('aaaaaaaaaaaaaa', 8) as a group by substr('aaaaaaaaaaaaaa', 8);
|
||||
aaaaaaa
|
||||
select b from (select 5 as a, 'Hello' as b order by a);
|
||||
Hello
|
||||
select b from (select 5 as a, 'Hello' as b group by a);
|
||||
Hello
|
||||
select b from (select 5 as a, 'Hello' as b order by 1);
|
||||
Hello
|
||||
|
@ -45,3 +45,6 @@ explain syntax select plus(1, 1) as a group by a;
|
||||
select substr('aaaaaaaaaaaaaa', 8) as a group by a;
|
||||
select substr('aaaaaaaaaaaaaa', 8) as a group by substr('aaaaaaaaaaaaaa', 8);
|
||||
|
||||
select b from (select 5 as a, 'Hello' as b order by a);
|
||||
select b from (select 5 as a, 'Hello' as b group by a);
|
||||
select b from (select 5 as a, 'Hello' as b order by 1);
|
||||
|
@ -0,0 +1,4 @@
|
||||
x x
|
||||
x x
|
||||
x x
|
||||
x x
|
27
tests/queries/0_stateless/02244_lowcardinality_hash_join.sql
Normal file
27
tests/queries/0_stateless/02244_lowcardinality_hash_join.sql
Normal file
@ -0,0 +1,27 @@
|
||||
-- Tags: no-parallel
|
||||
DROP TABLE IF EXISTS lc_table;
|
||||
|
||||
CREATE TABLE lc_table
|
||||
(
|
||||
col LowCardinality(String)
|
||||
) ENGINE=TinyLog;
|
||||
|
||||
INSERT INTO lc_table VALUES('x');
|
||||
|
||||
SELECT *
|
||||
FROM lc_table
|
||||
INNER JOIN lc_table AS lc_table2 ON lc_table.col = lc_table2.col;
|
||||
|
||||
SELECT *
|
||||
FROM lc_table
|
||||
INNER JOIN lc_table AS lc_table2 ON CAST(lc_table.col AS String) = CAST(lc_table2.col AS String);
|
||||
|
||||
SELECT *
|
||||
FROM lc_table
|
||||
INNER JOIN lc_table AS lc_table2 ON (lc_table.col = lc_table2.col) OR (lc_table.col = lc_table2.col);
|
||||
|
||||
SELECT *
|
||||
FROM lc_table
|
||||
INNER JOIN lc_table AS lc_table2 ON (CAST(lc_table.col AS String) = CAST(lc_table2.col AS String)) OR (CAST(lc_table.col AS String) = CAST(lc_table2.col AS String));
|
||||
|
||||
DROP TABLE IF EXISTS lc_table;
|
@ -0,0 +1,14 @@
|
||||
202112-0 (202112,0)
|
||||
202201-0 (202201,0)
|
||||
202301-0 (202301,0)
|
||||
202112-0 2021-12-31 22:30:00 2021-12-31 22:30:00 2021-12-31 14:30:00 2021-12-31 14:30:00 1000
|
||||
202201-0 2022-01-01 00:30:00 2022-01-31 22:30:00 2021-12-31 16:30:00 2022-01-31 14:30:00 2000
|
||||
202301-0 2023-01-31 22:30:00 2023-01-31 22:30:00 2023-01-31 14:30:00 2023-01-31 14:30:00 1000
|
||||
202112-0
|
||||
default weird_partitions_02245 1 1000 1
|
||||
202201-0
|
||||
default weird_partitions_02245 1 2000 1
|
||||
202112-0
|
||||
202201-0
|
||||
default weird_partitions_02245 2 3000 2
|
||||
default weird_partitions_02245 0 0 0
|
61
tests/queries/0_stateless/02245_weird_partitions_pruning.sql
Normal file
61
tests/queries/0_stateless/02245_weird_partitions_pruning.sql
Normal file
@ -0,0 +1,61 @@
|
||||
-- We use a hack - partition by ignore(d1). In some cases there are two columns
|
||||
-- not fully correlated (<1) (date_begin - date_end or datetime - datetime_in_TZ_with_DST)
|
||||
-- If we partition by these columns instead of one it will be twice more partitions.
|
||||
-- Partition by (.., ignore(d1)) allows to partition by the first column but build
|
||||
-- min_max indexes for both column, so partition pruning works for both columns.
|
||||
-- It's very similar to min_max skip index but gives bigger performance boost,
|
||||
-- because partition pruning happens on very early query stage.
|
||||
|
||||
|
||||
DROP TABLE IF EXISTS weird_partitions_02245;
|
||||
|
||||
CREATE TABLE weird_partitions_02245(d DateTime, d1 DateTime default d - toIntervalHour(8), id Int64)
|
||||
Engine=MergeTree
|
||||
PARTITION BY (toYYYYMM(toDateTime(d)), ignore(d1))
|
||||
ORDER BY id;
|
||||
|
||||
INSERT INTO weird_partitions_02245(d, id)
|
||||
SELECT
|
||||
toDateTime('2021-12-31 22:30:00') AS d,
|
||||
number
|
||||
FROM numbers(1000);
|
||||
|
||||
INSERT INTO weird_partitions_02245(d, id)
|
||||
SELECT
|
||||
toDateTime('2022-01-01 00:30:00') AS d,
|
||||
number
|
||||
FROM numbers(1000);
|
||||
|
||||
INSERT INTO weird_partitions_02245(d, id)
|
||||
SELECT
|
||||
toDateTime('2022-01-31 22:30:00') AS d,
|
||||
number
|
||||
FROM numbers(1000);
|
||||
|
||||
INSERT INTO weird_partitions_02245(d, id)
|
||||
SELECT
|
||||
toDateTime('2023-01-31 22:30:00') AS d,
|
||||
number
|
||||
FROM numbers(1000);
|
||||
|
||||
OPTIMIZE TABLE weird_partitions_02245;
|
||||
OPTIMIZE TABLE weird_partitions_02245;
|
||||
|
||||
SELECT DISTINCT _partition_id, _partition_value FROM weird_partitions_02245 ORDER BY _partition_id ASC;
|
||||
|
||||
SELECT _partition_id, min(d), max(d), min(d1), max(d1), count() FROM weird_partitions_02245 GROUP BY _partition_id ORDER BY _partition_id ASC;
|
||||
|
||||
select DISTINCT _partition_id from weird_partitions_02245 where d >= '2021-12-31 00:00:00' and d < '2022-01-01 00:00:00' ORDER BY _partition_id;
|
||||
explain estimate select DISTINCT _partition_id from weird_partitions_02245 where d >= '2021-12-31 00:00:00' and d < '2022-01-01 00:00:00';
|
||||
|
||||
select DISTINCT _partition_id from weird_partitions_02245 where d >= '2022-01-01 00:00:00' and d1 >= '2021-12-31 00:00:00' and d1 < '2022-01-01 00:00:00' ORDER BY _partition_id;;
|
||||
explain estimate select DISTINCT _partition_id from weird_partitions_02245 where d >= '2022-01-01 00:00:00' and d1 >= '2021-12-31 00:00:00' and d1 < '2022-01-01 00:00:00';
|
||||
|
||||
select DISTINCT _partition_id from weird_partitions_02245 where d1 >= '2021-12-31 00:00:00' and d1 < '2022-01-01 00:00:00' ORDER BY _partition_id;;
|
||||
explain estimate select DISTINCT _partition_id from weird_partitions_02245 where d1 >= '2021-12-31 00:00:00' and d1 < '2022-01-01 00:00:00';
|
||||
|
||||
select DISTINCT _partition_id from weird_partitions_02245 where d >= '2022-01-01 00:00:00' and d1 >= '2021-12-31 00:00:00' and d1 < '2020-01-01 00:00:00' ORDER BY _partition_id;;
|
||||
explain estimate select DISTINCT _partition_id from weird_partitions_02245 where d >= '2022-01-01 00:00:00' and d1 >= '2021-12-31 00:00:00' and d1 < '2020-01-01 00:00:00';
|
||||
|
||||
DROP TABLE weird_partitions_02245;
|
||||
|
Loading…
Reference in New Issue
Block a user