mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #21334 from ClickHouse/fix_alter_partition_key
Fix alter modify query for partition key and other metadata fields
This commit is contained in:
commit
6b1005aea5
@ -363,7 +363,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
|
|||||||
new_data_part->uuid = part_uuid;
|
new_data_part->uuid = part_uuid;
|
||||||
new_data_part->is_temp = true;
|
new_data_part->is_temp = true;
|
||||||
new_data_part->setColumns(block.getNamesAndTypesList());
|
new_data_part->setColumns(block.getNamesAndTypesList());
|
||||||
new_data_part->minmax_idx.update(block, data.minmax_idx_columns);
|
new_data_part->minmax_idx.update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey()));
|
||||||
new_data_part->partition.create(metadata_snapshot, block, 0);
|
new_data_part->partition.create(metadata_snapshot, block, 0);
|
||||||
|
|
||||||
MergedBlockOutputStream part_out(new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {}));
|
MergedBlockOutputStream part_out(new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {}));
|
||||||
|
@ -57,13 +57,18 @@ static std::unique_ptr<ReadBufferFromFileBase> openForReading(const DiskPtr & di
|
|||||||
|
|
||||||
void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path)
|
void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path)
|
||||||
{
|
{
|
||||||
size_t minmax_idx_size = data.minmax_idx_column_types.size();
|
auto metadata_snapshot = data.getInMemoryMetadataPtr();
|
||||||
|
const auto & partition_key = metadata_snapshot->getPartitionKey();
|
||||||
|
|
||||||
|
auto minmax_column_names = data.getMinMaxColumnsNames(partition_key);
|
||||||
|
auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key);
|
||||||
|
size_t minmax_idx_size = minmax_column_types.size();
|
||||||
hyperrectangle.reserve(minmax_idx_size);
|
hyperrectangle.reserve(minmax_idx_size);
|
||||||
for (size_t i = 0; i < minmax_idx_size; ++i)
|
for (size_t i = 0; i < minmax_idx_size; ++i)
|
||||||
{
|
{
|
||||||
String file_name = part_path + "minmax_" + escapeForFileName(data.minmax_idx_columns[i]) + ".idx";
|
String file_name = part_path + "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx";
|
||||||
auto file = openForReading(disk_, file_name);
|
auto file = openForReading(disk_, file_name);
|
||||||
const DataTypePtr & data_type = data.minmax_idx_column_types[i];
|
const DataTypePtr & data_type = minmax_column_types[i];
|
||||||
|
|
||||||
Field min_val;
|
Field min_val;
|
||||||
data_type->deserializeBinary(min_val, *file);
|
data_type->deserializeBinary(min_val, *file);
|
||||||
@ -78,7 +83,13 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Dis
|
|||||||
void IMergeTreeDataPart::MinMaxIndex::store(
|
void IMergeTreeDataPart::MinMaxIndex::store(
|
||||||
const MergeTreeData & data, const DiskPtr & disk_, const String & part_path, Checksums & out_checksums) const
|
const MergeTreeData & data, const DiskPtr & disk_, const String & part_path, Checksums & out_checksums) const
|
||||||
{
|
{
|
||||||
store(data.minmax_idx_columns, data.minmax_idx_column_types, disk_, part_path, out_checksums);
|
auto metadata_snapshot = data.getInMemoryMetadataPtr();
|
||||||
|
const auto & partition_key = metadata_snapshot->getPartitionKey();
|
||||||
|
|
||||||
|
auto minmax_column_names = data.getMinMaxColumnsNames(partition_key);
|
||||||
|
auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key);
|
||||||
|
|
||||||
|
store(minmax_column_names, minmax_column_types, disk_, part_path, out_checksums);
|
||||||
}
|
}
|
||||||
|
|
||||||
void IMergeTreeDataPart::MinMaxIndex::store(
|
void IMergeTreeDataPart::MinMaxIndex::store(
|
||||||
@ -1168,6 +1179,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const
|
|||||||
|
|
||||||
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
|
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
|
||||||
const auto & pk = metadata_snapshot->getPrimaryKey();
|
const auto & pk = metadata_snapshot->getPrimaryKey();
|
||||||
|
const auto & partition_key = metadata_snapshot->getPartitionKey();
|
||||||
if (!checksums.empty())
|
if (!checksums.empty())
|
||||||
{
|
{
|
||||||
if (!pk.column_names.empty() && !checksums.files.count("primary.idx"))
|
if (!pk.column_names.empty() && !checksums.files.count("primary.idx"))
|
||||||
@ -1183,7 +1195,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const
|
|||||||
|
|
||||||
if (!isEmpty())
|
if (!isEmpty())
|
||||||
{
|
{
|
||||||
for (const String & col_name : storage.minmax_idx_columns)
|
for (const String & col_name : storage.getMinMaxColumnsNames(partition_key))
|
||||||
{
|
{
|
||||||
if (!checksums.files.count("minmax_" + escapeForFileName(col_name) + ".idx"))
|
if (!checksums.files.count("minmax_" + escapeForFileName(col_name) + ".idx"))
|
||||||
throw Exception("No minmax idx file checksum for column " + col_name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
throw Exception("No minmax idx file checksum for column " + col_name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||||
@ -1214,7 +1226,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const
|
|||||||
if (metadata_snapshot->hasPartitionKey())
|
if (metadata_snapshot->hasPartitionKey())
|
||||||
check_file_not_empty(volume->getDisk(), path + "partition.dat");
|
check_file_not_empty(volume->getDisk(), path + "partition.dat");
|
||||||
|
|
||||||
for (const String & col_name : storage.minmax_idx_columns)
|
for (const String & col_name : storage.getMinMaxColumnsNames(partition_key))
|
||||||
check_file_not_empty(volume->getDisk(), path + "minmax_" + escapeForFileName(col_name) + ".idx");
|
check_file_not_empty(volume->getDisk(), path + "minmax_" + escapeForFileName(col_name) + ".idx");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -166,7 +166,9 @@ MergeTreeData::MergeTreeData(
|
|||||||
{
|
{
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
|
|
||||||
checkPartitionKeyAndInitMinMax(metadata_.partition_key);
|
checkPartitionKeyAndInitMinMax(metadata_.partition_key);
|
||||||
|
setProperties(metadata_, metadata_, attach);
|
||||||
if (minmax_idx_date_column_pos == -1)
|
if (minmax_idx_date_column_pos == -1)
|
||||||
throw Exception("Could not find Date column", ErrorCodes::BAD_TYPE_OF_FIELD);
|
throw Exception("Could not find Date column", ErrorCodes::BAD_TYPE_OF_FIELD);
|
||||||
}
|
}
|
||||||
@ -183,7 +185,6 @@ MergeTreeData::MergeTreeData(
|
|||||||
checkPartitionKeyAndInitMinMax(metadata_.partition_key);
|
checkPartitionKeyAndInitMinMax(metadata_.partition_key);
|
||||||
min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING;
|
min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING;
|
||||||
}
|
}
|
||||||
|
|
||||||
setProperties(metadata_, metadata_, attach);
|
setProperties(metadata_, metadata_, attach);
|
||||||
|
|
||||||
/// NOTE: using the same columns list as is read when performing actual merges.
|
/// NOTE: using the same columns list as is read when performing actual merges.
|
||||||
@ -423,6 +424,29 @@ ExpressionActionsPtr getCombinedIndicesExpression(
|
|||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ExpressionActionsPtr MergeTreeData::getMinMaxExpr(const KeyDescription & partition_key)
|
||||||
|
{
|
||||||
|
NamesAndTypesList partition_key_columns;
|
||||||
|
if (!partition_key.column_names.empty())
|
||||||
|
partition_key_columns = partition_key.expression->getRequiredColumnsWithTypes();
|
||||||
|
|
||||||
|
return std::make_shared<ExpressionActions>(std::make_shared<ActionsDAG>(partition_key_columns));
|
||||||
|
}
|
||||||
|
|
||||||
|
Names MergeTreeData::getMinMaxColumnsNames(const KeyDescription & partition_key)
|
||||||
|
{
|
||||||
|
if (!partition_key.column_names.empty())
|
||||||
|
return partition_key.expression->getRequiredColumns();
|
||||||
|
return {};
|
||||||
|
}
|
||||||
|
|
||||||
|
DataTypes MergeTreeData::getMinMaxColumnsTypes(const KeyDescription & partition_key)
|
||||||
|
{
|
||||||
|
if (!partition_key.column_names.empty())
|
||||||
|
return partition_key.expression->getRequiredColumnsWithTypes().getTypes();
|
||||||
|
return {};
|
||||||
|
}
|
||||||
|
|
||||||
ExpressionActionsPtr MergeTreeData::getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const
|
ExpressionActionsPtr MergeTreeData::getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const
|
||||||
{
|
{
|
||||||
return getCombinedIndicesExpression(metadata_snapshot->getPrimaryKey(), metadata_snapshot->getSecondaryIndices(), metadata_snapshot->getColumns(), global_context);
|
return getCombinedIndicesExpression(metadata_snapshot->getPrimaryKey(), metadata_snapshot->getSecondaryIndices(), metadata_snapshot->getColumns(), global_context);
|
||||||
@ -442,19 +466,13 @@ void MergeTreeData::checkPartitionKeyAndInitMinMax(const KeyDescription & new_pa
|
|||||||
checkKeyExpression(*new_partition_key.expression, new_partition_key.sample_block, "Partition", allow_nullable_key);
|
checkKeyExpression(*new_partition_key.expression, new_partition_key.sample_block, "Partition", allow_nullable_key);
|
||||||
|
|
||||||
/// Add all columns used in the partition key to the min-max index.
|
/// Add all columns used in the partition key to the min-max index.
|
||||||
const NamesAndTypesList & minmax_idx_columns_with_types = new_partition_key.expression->getRequiredColumnsWithTypes();
|
DataTypes minmax_idx_columns_types = getMinMaxColumnsTypes(new_partition_key);
|
||||||
minmax_idx_expr = std::make_shared<ExpressionActions>(std::make_shared<ActionsDAG>(minmax_idx_columns_with_types));
|
|
||||||
for (const NameAndTypePair & column : minmax_idx_columns_with_types)
|
|
||||||
{
|
|
||||||
minmax_idx_columns.emplace_back(column.name);
|
|
||||||
minmax_idx_column_types.emplace_back(column.type);
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Try to find the date column in columns used by the partition key (a common case).
|
/// Try to find the date column in columns used by the partition key (a common case).
|
||||||
bool encountered_date_column = false;
|
bool encountered_date_column = false;
|
||||||
for (size_t i = 0; i < minmax_idx_column_types.size(); ++i)
|
for (size_t i = 0; i < minmax_idx_columns_types.size(); ++i)
|
||||||
{
|
{
|
||||||
if (typeid_cast<const DataTypeDate *>(minmax_idx_column_types[i].get()))
|
if (typeid_cast<const DataTypeDate *>(minmax_idx_columns_types[i].get()))
|
||||||
{
|
{
|
||||||
if (!encountered_date_column)
|
if (!encountered_date_column)
|
||||||
{
|
{
|
||||||
@ -470,9 +488,9 @@ void MergeTreeData::checkPartitionKeyAndInitMinMax(const KeyDescription & new_pa
|
|||||||
}
|
}
|
||||||
if (!encountered_date_column)
|
if (!encountered_date_column)
|
||||||
{
|
{
|
||||||
for (size_t i = 0; i < minmax_idx_column_types.size(); ++i)
|
for (size_t i = 0; i < minmax_idx_columns_types.size(); ++i)
|
||||||
{
|
{
|
||||||
if (typeid_cast<const DataTypeDateTime *>(minmax_idx_column_types[i].get()))
|
if (typeid_cast<const DataTypeDateTime *>(minmax_idx_columns_types[i].get()))
|
||||||
{
|
{
|
||||||
if (!encountered_date_column)
|
if (!encountered_date_column)
|
||||||
{
|
{
|
||||||
@ -3517,7 +3535,7 @@ bool MergeTreeData::isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(
|
|||||||
if (column_name == name)
|
if (column_name == name)
|
||||||
return true;
|
return true;
|
||||||
|
|
||||||
for (const auto & name : minmax_idx_columns)
|
for (const auto & name : getMinMaxColumnsNames(metadata_snapshot->getPartitionKey()))
|
||||||
if (column_name == name)
|
if (column_name == name)
|
||||||
return true;
|
return true;
|
||||||
|
|
||||||
|
@ -692,12 +692,17 @@ public:
|
|||||||
|
|
||||||
bool is_custom_partitioned = false;
|
bool is_custom_partitioned = false;
|
||||||
|
|
||||||
ExpressionActionsPtr minmax_idx_expr;
|
/// Used only for old syntax tables. Never changes after init.
|
||||||
Names minmax_idx_columns;
|
|
||||||
DataTypes minmax_idx_column_types;
|
|
||||||
Int64 minmax_idx_date_column_pos = -1; /// In a common case minmax index includes a date column.
|
Int64 minmax_idx_date_column_pos = -1; /// In a common case minmax index includes a date column.
|
||||||
Int64 minmax_idx_time_column_pos = -1; /// In other cases, minmax index often includes a dateTime column.
|
Int64 minmax_idx_time_column_pos = -1; /// In other cases, minmax index often includes a dateTime column.
|
||||||
|
|
||||||
|
/// Get partition key expression on required columns
|
||||||
|
static ExpressionActionsPtr getMinMaxExpr(const KeyDescription & partition_key);
|
||||||
|
/// Get column names required for partition key
|
||||||
|
static Names getMinMaxColumnsNames(const KeyDescription & partition_key);
|
||||||
|
/// Get column types required for partition key
|
||||||
|
static DataTypes getMinMaxColumnsTypes(const KeyDescription & partition_key);
|
||||||
|
|
||||||
ExpressionActionsPtr getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const;
|
ExpressionActionsPtr getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const;
|
||||||
ExpressionActionsPtr getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const;
|
ExpressionActionsPtr getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const;
|
||||||
|
|
||||||
|
@ -1779,7 +1779,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns(
|
|||||||
Block block;
|
Block block;
|
||||||
while (checkOperationIsNotCanceled(merge_entry) && (block = mutating_stream->read()))
|
while (checkOperationIsNotCanceled(merge_entry) && (block = mutating_stream->read()))
|
||||||
{
|
{
|
||||||
minmax_idx.update(block, data.minmax_idx_columns);
|
minmax_idx.update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey()));
|
||||||
out.write(block);
|
out.write(block);
|
||||||
|
|
||||||
merge_entry->rows_written += block.rows();
|
merge_entry->rows_written += block.rows();
|
||||||
|
@ -242,16 +242,21 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
|
|||||||
|
|
||||||
std::optional<KeyCondition> minmax_idx_condition;
|
std::optional<KeyCondition> minmax_idx_condition;
|
||||||
std::optional<PartitionPruner> partition_pruner;
|
std::optional<PartitionPruner> partition_pruner;
|
||||||
if (data.minmax_idx_expr)
|
DataTypes minmax_columns_types;
|
||||||
|
if (metadata_snapshot->hasPartitionKey())
|
||||||
{
|
{
|
||||||
minmax_idx_condition.emplace(query_info, context, data.minmax_idx_columns, data.minmax_idx_expr);
|
const auto & partition_key = metadata_snapshot->getPartitionKey();
|
||||||
|
auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key);
|
||||||
|
minmax_columns_types = data.getMinMaxColumnsTypes(partition_key);
|
||||||
|
|
||||||
|
minmax_idx_condition.emplace(query_info, context, minmax_columns_names, data.getMinMaxExpr(partition_key));
|
||||||
partition_pruner.emplace(metadata_snapshot->getPartitionKey(), query_info, context, false /* strict */);
|
partition_pruner.emplace(metadata_snapshot->getPartitionKey(), query_info, context, false /* strict */);
|
||||||
|
|
||||||
if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless()))
|
if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless()))
|
||||||
{
|
{
|
||||||
String msg = "Neither MinMax index by columns (";
|
String msg = "Neither MinMax index by columns (";
|
||||||
bool first = true;
|
bool first = true;
|
||||||
for (const String & col : data.minmax_idx_columns)
|
for (const String & col : minmax_columns_names)
|
||||||
{
|
{
|
||||||
if (first)
|
if (first)
|
||||||
first = false;
|
first = false;
|
||||||
@ -268,9 +273,9 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
|
|||||||
const Context & query_context = context.hasQueryContext() ? context.getQueryContext() : context;
|
const Context & query_context = context.hasQueryContext() ? context.getQueryContext() : context;
|
||||||
|
|
||||||
if (query_context.getSettingsRef().allow_experimental_query_deduplication)
|
if (query_context.getSettingsRef().allow_experimental_query_deduplication)
|
||||||
selectPartsToReadWithUUIDFilter(parts, part_values, minmax_idx_condition, partition_pruner, max_block_numbers_to_read, query_context);
|
selectPartsToReadWithUUIDFilter(parts, part_values, minmax_idx_condition, minmax_columns_types, partition_pruner, max_block_numbers_to_read, query_context);
|
||||||
else
|
else
|
||||||
selectPartsToRead(parts, part_values, minmax_idx_condition, partition_pruner, max_block_numbers_to_read);
|
selectPartsToRead(parts, part_values, minmax_idx_condition, minmax_columns_types, partition_pruner, max_block_numbers_to_read);
|
||||||
|
|
||||||
|
|
||||||
/// Sampling.
|
/// Sampling.
|
||||||
@ -1885,8 +1890,9 @@ void MergeTreeDataSelectExecutor::selectPartsToRead(
|
|||||||
MergeTreeData::DataPartsVector & parts,
|
MergeTreeData::DataPartsVector & parts,
|
||||||
const std::unordered_set<String> & part_values,
|
const std::unordered_set<String> & part_values,
|
||||||
const std::optional<KeyCondition> & minmax_idx_condition,
|
const std::optional<KeyCondition> & minmax_idx_condition,
|
||||||
|
const DataTypes & minmax_columns_types,
|
||||||
std::optional<PartitionPruner> & partition_pruner,
|
std::optional<PartitionPruner> & partition_pruner,
|
||||||
const PartitionIdToMaxBlock * max_block_numbers_to_read) const
|
const PartitionIdToMaxBlock * max_block_numbers_to_read)
|
||||||
{
|
{
|
||||||
auto prev_parts = parts;
|
auto prev_parts = parts;
|
||||||
parts.clear();
|
parts.clear();
|
||||||
@ -1900,7 +1906,7 @@ void MergeTreeDataSelectExecutor::selectPartsToRead(
|
|||||||
continue;
|
continue;
|
||||||
|
|
||||||
if (minmax_idx_condition && !minmax_idx_condition->checkInHyperrectangle(
|
if (minmax_idx_condition && !minmax_idx_condition->checkInHyperrectangle(
|
||||||
part->minmax_idx.hyperrectangle, data.minmax_idx_column_types).can_be_true)
|
part->minmax_idx.hyperrectangle, minmax_columns_types).can_be_true)
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
if (partition_pruner)
|
if (partition_pruner)
|
||||||
@ -1924,6 +1930,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter(
|
|||||||
MergeTreeData::DataPartsVector & parts,
|
MergeTreeData::DataPartsVector & parts,
|
||||||
const std::unordered_set<String> & part_values,
|
const std::unordered_set<String> & part_values,
|
||||||
const std::optional<KeyCondition> & minmax_idx_condition,
|
const std::optional<KeyCondition> & minmax_idx_condition,
|
||||||
|
const DataTypes & minmax_columns_types,
|
||||||
std::optional<PartitionPruner> & partition_pruner,
|
std::optional<PartitionPruner> & partition_pruner,
|
||||||
const PartitionIdToMaxBlock * max_block_numbers_to_read,
|
const PartitionIdToMaxBlock * max_block_numbers_to_read,
|
||||||
const Context & query_context) const
|
const Context & query_context) const
|
||||||
@ -1950,7 +1957,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter(
|
|||||||
continue;
|
continue;
|
||||||
|
|
||||||
if (minmax_idx_condition
|
if (minmax_idx_condition
|
||||||
&& !minmax_idx_condition->checkInHyperrectangle(part->minmax_idx.hyperrectangle, data.minmax_idx_column_types)
|
&& !minmax_idx_condition->checkInHyperrectangle(part->minmax_idx.hyperrectangle, minmax_columns_types)
|
||||||
.can_be_true)
|
.can_be_true)
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
|
@ -119,18 +119,20 @@ private:
|
|||||||
|
|
||||||
/// Select the parts in which there can be data that satisfy `minmax_idx_condition` and that match the condition on `_part`,
|
/// Select the parts in which there can be data that satisfy `minmax_idx_condition` and that match the condition on `_part`,
|
||||||
/// as well as `max_block_number_to_read`.
|
/// as well as `max_block_number_to_read`.
|
||||||
void selectPartsToRead(
|
static void selectPartsToRead(
|
||||||
MergeTreeData::DataPartsVector & parts,
|
MergeTreeData::DataPartsVector & parts,
|
||||||
const std::unordered_set<String> & part_values,
|
const std::unordered_set<String> & part_values,
|
||||||
const std::optional<KeyCondition> & minmax_idx_condition,
|
const std::optional<KeyCondition> & minmax_idx_condition,
|
||||||
|
const DataTypes & minmax_columns_types,
|
||||||
std::optional<PartitionPruner> & partition_pruner,
|
std::optional<PartitionPruner> & partition_pruner,
|
||||||
const PartitionIdToMaxBlock * max_block_numbers_to_read) const;
|
const PartitionIdToMaxBlock * max_block_numbers_to_read);
|
||||||
|
|
||||||
/// Same as previous but also skip parts uuids if any to the query context, or skip parts which uuids marked as excluded.
|
/// Same as previous but also skip parts uuids if any to the query context, or skip parts which uuids marked as excluded.
|
||||||
void selectPartsToReadWithUUIDFilter(
|
void selectPartsToReadWithUUIDFilter(
|
||||||
MergeTreeData::DataPartsVector & parts,
|
MergeTreeData::DataPartsVector & parts,
|
||||||
const std::unordered_set<String> & part_values,
|
const std::unordered_set<String> & part_values,
|
||||||
const std::optional<KeyCondition> & minmax_idx_condition,
|
const std::optional<KeyCondition> & minmax_idx_condition,
|
||||||
|
const DataTypes & minmax_columns_types,
|
||||||
std::optional<PartitionPruner> & partition_pruner,
|
std::optional<PartitionPruner> & partition_pruner,
|
||||||
const PartitionIdToMaxBlock * max_block_numbers_to_read,
|
const PartitionIdToMaxBlock * max_block_numbers_to_read,
|
||||||
const Context & query_context) const;
|
const Context & query_context) const;
|
||||||
|
@ -268,7 +268,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
|||||||
Int64 temp_index = data.insert_increment.get();
|
Int64 temp_index = data.insert_increment.get();
|
||||||
|
|
||||||
IMergeTreeDataPart::MinMaxIndex minmax_idx;
|
IMergeTreeDataPart::MinMaxIndex minmax_idx;
|
||||||
minmax_idx.update(block, data.minmax_idx_columns);
|
minmax_idx.update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey()));
|
||||||
|
|
||||||
MergeTreePartition partition(std::move(block_with_partition.partition));
|
MergeTreePartition partition(std::move(block_with_partition.partition));
|
||||||
|
|
||||||
|
@ -191,7 +191,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor
|
|||||||
{
|
{
|
||||||
MergedBlockOutputStream part_out(part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {}));
|
MergedBlockOutputStream part_out(part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {}));
|
||||||
|
|
||||||
part->minmax_idx.update(block, storage.minmax_idx_columns);
|
part->minmax_idx.update(block, storage.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey()));
|
||||||
part->partition.create(metadata_snapshot, block, 0);
|
part->partition.create(metadata_snapshot, block, 0);
|
||||||
if (metadata_snapshot->hasSortingKey())
|
if (metadata_snapshot->hasSortingKey())
|
||||||
metadata_snapshot->getSortingKey().expression->execute(block);
|
metadata_snapshot->getSortingKey().expression->execute(block);
|
||||||
|
@ -26,7 +26,10 @@ static String formattedAST(const ASTPtr & ast)
|
|||||||
ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTreeData & data, const StorageMetadataPtr & metadata_snapshot)
|
ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTreeData & data, const StorageMetadataPtr & metadata_snapshot)
|
||||||
{
|
{
|
||||||
if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||||
date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos];
|
{
|
||||||
|
auto minmax_idx_column_names = data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey());
|
||||||
|
date_column = minmax_idx_column_names[data.minmax_idx_date_column_pos];
|
||||||
|
}
|
||||||
|
|
||||||
const auto data_settings = data.getSettings();
|
const auto data_settings = data.getSettings();
|
||||||
sampling_expression = formattedAST(metadata_snapshot->getSamplingKeyAST());
|
sampling_expression = formattedAST(metadata_snapshot->getSamplingKeyAST());
|
||||||
|
@ -897,21 +897,8 @@ void StorageReplicatedMergeTree::setTableStructure(
|
|||||||
StorageInMemoryMetadata old_metadata = getInMemoryMetadata();
|
StorageInMemoryMetadata old_metadata = getInMemoryMetadata();
|
||||||
|
|
||||||
if (new_columns != new_metadata.columns)
|
if (new_columns != new_metadata.columns)
|
||||||
{
|
|
||||||
new_metadata.columns = new_columns;
|
new_metadata.columns = new_columns;
|
||||||
|
|
||||||
new_metadata.column_ttls_by_name.clear();
|
|
||||||
for (const auto & [name, ast] : new_metadata.columns.getColumnTTLs())
|
|
||||||
{
|
|
||||||
auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, new_metadata.columns, global_context, new_metadata.primary_key);
|
|
||||||
new_metadata.column_ttls_by_name[name] = new_ttl_entry;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// The type of partition key expression may change
|
|
||||||
if (new_metadata.partition_key.definition_ast != nullptr)
|
|
||||||
new_metadata.partition_key.recalculateWithNewColumns(new_metadata.columns, global_context);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!metadata_diff.empty())
|
if (!metadata_diff.empty())
|
||||||
{
|
{
|
||||||
auto parse_key_expr = [] (const String & key_expr)
|
auto parse_key_expr = [] (const String & key_expr)
|
||||||
@ -977,6 +964,47 @@ void StorageReplicatedMergeTree::setTableStructure(
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Changes in columns may affect following metadata fields
|
||||||
|
if (new_metadata.columns != old_metadata.columns)
|
||||||
|
{
|
||||||
|
new_metadata.column_ttls_by_name.clear();
|
||||||
|
for (const auto & [name, ast] : new_metadata.columns.getColumnTTLs())
|
||||||
|
{
|
||||||
|
auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, new_metadata.columns, global_context, new_metadata.primary_key);
|
||||||
|
new_metadata.column_ttls_by_name[name] = new_ttl_entry;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (new_metadata.partition_key.definition_ast != nullptr)
|
||||||
|
new_metadata.partition_key.recalculateWithNewColumns(new_metadata.columns, global_context);
|
||||||
|
|
||||||
|
if (!metadata_diff.sorting_key_changed) /// otherwise already updated
|
||||||
|
new_metadata.sorting_key.recalculateWithNewColumns(new_metadata.columns, global_context);
|
||||||
|
|
||||||
|
/// Primary key is special, it exists even if not defined
|
||||||
|
if (new_metadata.primary_key.definition_ast != nullptr)
|
||||||
|
{
|
||||||
|
new_metadata.primary_key.recalculateWithNewColumns(new_metadata.columns, global_context);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
new_metadata.primary_key = KeyDescription::getKeyFromAST(new_metadata.sorting_key.definition_ast, new_metadata.columns, global_context);
|
||||||
|
new_metadata.primary_key.definition_ast = nullptr;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!metadata_diff.sampling_expression_changed && new_metadata.sampling_key.definition_ast != nullptr)
|
||||||
|
new_metadata.sampling_key.recalculateWithNewColumns(new_metadata.columns, global_context);
|
||||||
|
|
||||||
|
if (!metadata_diff.skip_indices_changed) /// otherwise already updated
|
||||||
|
{
|
||||||
|
for (auto & index : new_metadata.secondary_indices)
|
||||||
|
index.recalculateWithNewColumns(new_metadata.columns, global_context);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!metadata_diff.ttl_table_changed && new_metadata.table_ttl.definition_ast != nullptr)
|
||||||
|
new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST(
|
||||||
|
new_metadata.table_ttl.definition_ast, new_metadata.columns, global_context, new_metadata.primary_key);
|
||||||
|
}
|
||||||
|
|
||||||
/// Even if the primary/sorting/partition keys didn't change we must reinitialize it
|
/// Even if the primary/sorting/partition keys didn't change we must reinitialize it
|
||||||
/// because primary/partition key column types might have changed.
|
/// because primary/partition key column types might have changed.
|
||||||
checkTTLExpressions(new_metadata, old_metadata);
|
checkTTLExpressions(new_metadata, old_metadata);
|
||||||
|
@ -0,0 +1,6 @@
|
|||||||
|
IU lada 2101 1970-04-19 15:00:00
|
||||||
|
PS jeep Grand Cherokee 2005-10-03 15:00:00
|
||||||
|
PS jeep Grand Cherokee 2005-10-03 15:00:00
|
||||||
|
IU lada 2101 1970-04-19 15:00:00
|
||||||
|
PS jeep Grand Cherokee 2005-10-03 15:00:00
|
||||||
|
PS jeep Grand Cherokee 2005-10-03 15:00:00
|
@ -0,0 +1,63 @@
|
|||||||
|
DROP TABLE IF EXISTS report;
|
||||||
|
|
||||||
|
CREATE TABLE report
|
||||||
|
(
|
||||||
|
`product` Enum8('IU' = 1, 'WS' = 2),
|
||||||
|
`machine` String,
|
||||||
|
`branch` String,
|
||||||
|
`generated_time` DateTime
|
||||||
|
)
|
||||||
|
ENGINE = MergeTree
|
||||||
|
PARTITION BY (product, toYYYYMM(generated_time))
|
||||||
|
ORDER BY (product, machine, branch, generated_time);
|
||||||
|
|
||||||
|
INSERT INTO report VALUES ('IU', 'lada', '2101', toDateTime('1970-04-19 15:00:00'));
|
||||||
|
|
||||||
|
SELECT * FROM report WHERE product = 'IU';
|
||||||
|
|
||||||
|
ALTER TABLE report MODIFY COLUMN product Enum8('IU' = 1, 'WS' = 2, 'PS' = 3);
|
||||||
|
|
||||||
|
SELECT * FROM report WHERE product = 'PS';
|
||||||
|
|
||||||
|
INSERT INTO report VALUES ('PS', 'jeep', 'Grand Cherokee', toDateTime('2005-10-03 15:00:00'));
|
||||||
|
|
||||||
|
SELECT * FROM report WHERE product = 'PS';
|
||||||
|
|
||||||
|
DETACH TABLE report;
|
||||||
|
ATTACH TABLE report;
|
||||||
|
|
||||||
|
SELECT * FROM report WHERE product = 'PS';
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS report;
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS replicated_report;
|
||||||
|
|
||||||
|
CREATE TABLE replicated_report
|
||||||
|
(
|
||||||
|
`product` Enum8('IU' = 1, 'WS' = 2),
|
||||||
|
`machine` String,
|
||||||
|
`branch` String,
|
||||||
|
`generated_time` DateTime
|
||||||
|
)
|
||||||
|
ENGINE = ReplicatedMergeTree('/clickhouse/01747_alter_partition_key/t', '1')
|
||||||
|
PARTITION BY (product, toYYYYMM(generated_time))
|
||||||
|
ORDER BY (product, machine, branch, generated_time);
|
||||||
|
|
||||||
|
INSERT INTO replicated_report VALUES ('IU', 'lada', '2101', toDateTime('1970-04-19 15:00:00'));
|
||||||
|
|
||||||
|
SELECT * FROM replicated_report WHERE product = 'IU';
|
||||||
|
|
||||||
|
ALTER TABLE replicated_report MODIFY COLUMN product Enum8('IU' = 1, 'WS' = 2, 'PS' = 3) SETTINGS replication_alter_partitions_sync=2;
|
||||||
|
|
||||||
|
SELECT * FROM replicated_report WHERE product = 'PS';
|
||||||
|
|
||||||
|
INSERT INTO replicated_report VALUES ('PS', 'jeep', 'Grand Cherokee', toDateTime('2005-10-03 15:00:00'));
|
||||||
|
|
||||||
|
SELECT * FROM replicated_report WHERE product = 'PS';
|
||||||
|
|
||||||
|
DETACH TABLE replicated_report;
|
||||||
|
ATTACH TABLE replicated_report;
|
||||||
|
|
||||||
|
SELECT * FROM replicated_report WHERE product = 'PS';
|
||||||
|
|
||||||
|
DROP TABLE IF EXISTS replicated_report;
|
@ -261,7 +261,8 @@
|
|||||||
"00116_storage_set",
|
"00116_storage_set",
|
||||||
"00083_create_merge_tree_zookeeper",
|
"00083_create_merge_tree_zookeeper",
|
||||||
"00062_replicated_merge_tree_alter_zookeeper",
|
"00062_replicated_merge_tree_alter_zookeeper",
|
||||||
"01720_constraints_complex_types"
|
"01720_constraints_complex_types",
|
||||||
|
"01747_alter_partition_key_enum_zookeeper"
|
||||||
],
|
],
|
||||||
"polymorphic-parts": [
|
"polymorphic-parts": [
|
||||||
"01508_partition_pruning_long", /// bug, shoud be fixed
|
"01508_partition_pruning_long", /// bug, shoud be fixed
|
||||||
@ -748,6 +749,7 @@
|
|||||||
"01676_dictget_in_default_expression",
|
"01676_dictget_in_default_expression",
|
||||||
"01700_system_zookeeper_path_in",
|
"01700_system_zookeeper_path_in",
|
||||||
"01715_background_checker_blather_zookeeper",
|
"01715_background_checker_blather_zookeeper",
|
||||||
|
"01747_alter_partition_key_enum_zookeeper",
|
||||||
"attach",
|
"attach",
|
||||||
"ddl_dictionaries",
|
"ddl_dictionaries",
|
||||||
"dictionary",
|
"dictionary",
|
||||||
|
Loading…
Reference in New Issue
Block a user