mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 09:02:00 +00:00
Better
This commit is contained in:
parent
41914588fd
commit
327d17ac6a
@ -444,63 +444,21 @@ void IStorage::setPartitionKey(const StorageMetadataKeyField & partition_key_)
|
||||
partition_key = partition_key_;
|
||||
}
|
||||
|
||||
bool IStorage::hasPartitionKey() const
|
||||
{
|
||||
return partition_key.expression != nullptr;
|
||||
}
|
||||
|
||||
Names IStorage::getColumnsRequiredForPartitionKey() const
|
||||
{
|
||||
if (hasPartitionKey())
|
||||
return partition_key.expressions->getRequiredColumns();
|
||||
return partition_key.expression->getRequiredColumns();
|
||||
return {};
|
||||
}
|
||||
|
||||
Names IStorage::getColumnsRequiredForSampling() const
|
||||
const StorageMetadataKeyField & IStorage::getSortingKey() const
|
||||
{
|
||||
if (hasSamplingKey())
|
||||
return sampling_key.expressions->getRequiredColumns();
|
||||
return {};
|
||||
}
|
||||
|
||||
bool IStorage::hasPartitionKey() const
|
||||
{
|
||||
return partition_key.expressions != nullptr;
|
||||
}
|
||||
|
||||
|
||||
bool IStorage::supportsSampling() const
|
||||
{
|
||||
return hasSamplingKey();
|
||||
}
|
||||
|
||||
Names IStorage::getColumnsRequiredForSortingKey() const
|
||||
{
|
||||
if (hasSortingKey())
|
||||
return sorting_key.expressions->getRequiredColumns();
|
||||
return {};
|
||||
}
|
||||
|
||||
|
||||
Names IStorage::getSortingKeyColumns() const
|
||||
{
|
||||
if (hasSortingKey())
|
||||
return sorting_key.expression_column_names;
|
||||
return {};
|
||||
}
|
||||
|
||||
const StorageMetadataKeyField & IStorage::getSamplingKey() const
|
||||
{
|
||||
return sampling_key;
|
||||
}
|
||||
void IStorage::setSamplingKey(const StorageMetadataKeyField & sampling_key_)
|
||||
{
|
||||
sampling_key = sampling_key_;
|
||||
}
|
||||
|
||||
bool IStorage::hasSamplingKey() const
|
||||
{
|
||||
return sampling_key.expressions != nullptr;
|
||||
}
|
||||
|
||||
bool IStorage::hasSortingKey() const
|
||||
{
|
||||
return sorting_key.expressions != nullptr;
|
||||
return sorting_key;
|
||||
}
|
||||
|
||||
void IStorage::setSortingKey(const StorageMetadataKeyField & sorting_key_)
|
||||
@ -508,9 +466,23 @@ void IStorage::setSortingKey(const StorageMetadataKeyField & sorting_key_)
|
||||
sorting_key = sorting_key_;
|
||||
}
|
||||
|
||||
const StorageMetadataKeyField & IStorage::getSortingKey() const
|
||||
bool IStorage::hasSortingKey() const
|
||||
{
|
||||
return sorting_key;
|
||||
return sorting_key.expression != nullptr;
|
||||
}
|
||||
|
||||
Names IStorage::getColumnsRequiredForSortingKey() const
|
||||
{
|
||||
if (hasSortingKey())
|
||||
return sorting_key.expression->getRequiredColumns();
|
||||
return {};
|
||||
}
|
||||
|
||||
Names IStorage::getSortingKeyColumns() const
|
||||
{
|
||||
if (hasSortingKey())
|
||||
return sorting_key.column_names;
|
||||
return {};
|
||||
}
|
||||
|
||||
const StorageMetadataKeyField & IStorage::getPrimaryKey() const
|
||||
@ -523,21 +495,50 @@ void IStorage::setPrimaryKey(const StorageMetadataKeyField & primary_key_)
|
||||
primary_key = primary_key_;
|
||||
}
|
||||
|
||||
bool IStorage::hasPrimaryKey() const
|
||||
{
|
||||
return primary_key.expressions != nullptr;
|
||||
}
|
||||
|
||||
Names IStorage::getColumnsRequiredForPrimaryKey() const
|
||||
{
|
||||
if (primary_key.expressions != nullptr)
|
||||
return primary_key.expressions->getRequiredColumns();
|
||||
return {};
|
||||
}
|
||||
|
||||
bool IStorage::isPrimaryKeyDefined() const
|
||||
{
|
||||
return primary_key.definition_ast != nullptr;
|
||||
}
|
||||
|
||||
bool IStorage::hasPrimaryKey() const
|
||||
{
|
||||
return primary_key.expression != nullptr;
|
||||
}
|
||||
|
||||
Names IStorage::getColumnsRequiredForPrimaryKey() const
|
||||
{
|
||||
if (hasPrimaryKey())
|
||||
return primary_key.expression->getRequiredColumns();
|
||||
return {};
|
||||
}
|
||||
|
||||
Names IStorage::getPrimaryKeyColumns() const
|
||||
{
|
||||
if (hasSortingKey())
|
||||
return primary_key.column_names;
|
||||
return {};
|
||||
}
|
||||
|
||||
const StorageMetadataKeyField & IStorage::getSamplingKey() const
|
||||
{
|
||||
return sampling_key;
|
||||
}
|
||||
|
||||
void IStorage::setSamplingKey(const StorageMetadataKeyField & sampling_key_)
|
||||
{
|
||||
sampling_key = sampling_key_;
|
||||
}
|
||||
|
||||
bool IStorage::hasSamplingKey() const
|
||||
{
|
||||
return sampling_key.expression != nullptr;
|
||||
}
|
||||
|
||||
Names IStorage::getColumnsRequiredForSampling() const
|
||||
{
|
||||
if (hasSamplingKey())
|
||||
return sampling_key.expression->getRequiredColumns();
|
||||
return {};
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -101,7 +101,7 @@ public:
|
||||
virtual bool isView() const { return false; }
|
||||
|
||||
/// Returns true if the storage supports queries with the SAMPLE section.
|
||||
virtual bool supportsSampling() const;
|
||||
virtual bool supportsSampling() const { return hasSamplingKey(); }
|
||||
|
||||
/// Returns true if the storage supports queries with the FINAL section.
|
||||
virtual bool supportsFinal() const { return false; }
|
||||
@ -184,23 +184,6 @@ public: /// thread-unsafe part. lockStructure must be acquired
|
||||
/// By default return empty list of columns.
|
||||
virtual NamesAndTypesList getVirtuals() const;
|
||||
|
||||
const StorageMetadataKeyField & getPartitionKey() const;
|
||||
void setPartitionKey(const StorageMetadataKeyField & partition_key_);
|
||||
bool hasPartitionKey() const;
|
||||
|
||||
const StorageMetadataKeyField & getSamplingKey() const;
|
||||
void setSamplingKey(const StorageMetadataKeyField & sampling_key_);
|
||||
bool hasSamplingKey() const;
|
||||
|
||||
const StorageMetadataKeyField & getSortingKey() const;
|
||||
void setSortingKey(const StorageMetadataKeyField & sorting_key_);
|
||||
bool hasSortingKey() const;
|
||||
|
||||
const StorageMetadataKeyField & getPrimaryKey() const;
|
||||
void setPrimaryKey(const StorageMetadataKeyField & primary_key_);
|
||||
bool hasPrimaryKey() const;
|
||||
|
||||
|
||||
protected: /// still thread-unsafe part.
|
||||
void setIndices(IndicesDescription indices_);
|
||||
|
||||
@ -222,11 +205,6 @@ private:
|
||||
StorageMetadataKeyField sorting_key;
|
||||
StorageMetadataKeyField sampling_key;
|
||||
|
||||
//StorageMetadataField rows_ttl_entry;
|
||||
|
||||
//std::vector<StorageMetadataField> column_ttl_entries;
|
||||
//std::vector<StorageMetadataField> move_ttl_entries;
|
||||
|
||||
private:
|
||||
RWLockImpl::LockHolder tryLockTimed(
|
||||
const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const SettingSeconds & acquire_timeout) const;
|
||||
@ -468,36 +446,66 @@ public:
|
||||
/// Returns data paths if storage supports it, empty vector otherwise.
|
||||
virtual Strings getDataPaths() const { return {}; }
|
||||
|
||||
/// Returns structure with partition key
|
||||
const StorageMetadataKeyField & getPartitionKey() const;
|
||||
/// Set partition key for storage (methods bellow, are just wrappers for this
|
||||
/// struct)
|
||||
void setPartitionKey(const StorageMetadataKeyField & partition_key_);
|
||||
/// Returns ASTExpressionList of partition key expression for storage or nullptr if there is none.
|
||||
virtual ASTPtr getPartitionKeyAST() const { return partition_key.definition_ast; }
|
||||
|
||||
/// Returns ASTExpressionList of sorting key expression for storage or nullptr if there is none.
|
||||
virtual ASTPtr getSortingKeyAST() const { return sorting_key.definition_ast; }
|
||||
|
||||
/// Returns ASTExpressionList of primary key expression for storage or nullptr if there is none.
|
||||
virtual ASTPtr getPrimaryKeyAST() const { return primary_key.definition_ast; }
|
||||
|
||||
/// Returns sampling expression AST for storage or nullptr if there is none.
|
||||
virtual ASTPtr getSamplingKeyAST() const { return sampling_key.definition_ast; }
|
||||
|
||||
ASTPtr getPartitionKeyAST() const { return partition_key.definition_ast; }
|
||||
/// Storage has partition key
|
||||
bool hasPartitionKey() const;
|
||||
/// Returns column names that need to be read to calculate partition key.
|
||||
virtual Names getColumnsRequiredForPartitionKey() const;
|
||||
Names getColumnsRequiredForPartitionKey() const;
|
||||
|
||||
|
||||
/// Returns structure with sorting key
|
||||
const StorageMetadataKeyField & getSortingKey() const;
|
||||
/// Set sorting key for storage (methods bellow, are just wrappers for this
|
||||
/// struct)
|
||||
void setSortingKey(const StorageMetadataKeyField & sorting_key_);
|
||||
/// Returns ASTExpressionList of sorting key expression for storage or nullptr if there is none.
|
||||
ASTPtr getSortingKeyAST() const { return sorting_key.definition_ast; }
|
||||
/// Storage has sorting key
|
||||
bool hasSortingKey() const;
|
||||
/// Returns column names that need to be read to calculate sorting key.
|
||||
virtual Names getColumnsRequiredForSortingKey() const;
|
||||
|
||||
/// Returns column names that need to be read to calculate primary key.
|
||||
virtual Names getColumnsRequiredForPrimaryKey() const;
|
||||
|
||||
/// Returns column names that need to be read to calculate sampling key.
|
||||
virtual Names getColumnsRequiredForSampling() const;
|
||||
|
||||
/// Returns column names that need to be read for FINAL to work.
|
||||
virtual Names getColumnsRequiredForFinal() const { return getColumnsRequiredForSortingKey(); }
|
||||
|
||||
Names getColumnsRequiredForSortingKey() const;
|
||||
/// Returns columns names in sorting key specified by user in ORDER BY
|
||||
/// expression. For example: 'a', 'x * y', 'toStartOfMonth(date)', etc.
|
||||
virtual Names getSortingKeyColumns() const;
|
||||
Names getSortingKeyColumns() const;
|
||||
|
||||
/// Returns structure with primary key
|
||||
const StorageMetadataKeyField & getPrimaryKey() const;
|
||||
/// Set primary key for storage (methods bellow, are just wrappers for this
|
||||
/// struct)
|
||||
void setPrimaryKey(const StorageMetadataKeyField & primary_key_);
|
||||
/// Returns ASTExpressionList of primary key expression for storage or nullptr if there is none.
|
||||
ASTPtr getPrimaryKeyAST() const { return primary_key.definition_ast; }
|
||||
/// Storage has user-defined (in CREATE query) sorting key
|
||||
bool isPrimaryKeyDefined() const;
|
||||
/// Storage has primary key (maybe part of some other key)
|
||||
bool hasPrimaryKey() const;
|
||||
/// Returns column names that need to be read to calculate primary key.
|
||||
Names getColumnsRequiredForPrimaryKey() const;
|
||||
/// Returns columns names in sorting key specified by. For example: 'a', 'x
|
||||
/// * y', 'toStartOfMonth(date)', etc.
|
||||
Names getPrimaryKeyColumns() const;
|
||||
|
||||
/// Returns structure with sampling key
|
||||
const StorageMetadataKeyField & getSamplingKey() const;
|
||||
/// Set sampling key for storage (methods bellow, are just wrappers for this
|
||||
/// struct)
|
||||
void setSamplingKey(const StorageMetadataKeyField & sampling_key_);
|
||||
/// Returns sampling expression AST for storage or nullptr if there is none.
|
||||
ASTPtr getSamplingKeyAST() const { return sampling_key.definition_ast; }
|
||||
/// Storage has sampling key
|
||||
bool hasSamplingKey() const;
|
||||
/// Returns column names that need to be read to calculate sampling key.
|
||||
Names getColumnsRequiredForSampling() const;
|
||||
|
||||
/// Returns column names that need to be read for FINAL to work.
|
||||
Names getColumnsRequiredForFinal() const { return getColumnsRequiredForSortingKey(); }
|
||||
|
||||
|
||||
/// Returns columns, which will be needed to calculate dependencies
|
||||
/// (skip indices, TTL expressions) if we update @updated_columns set of columns.
|
||||
|
@ -439,7 +439,7 @@ void IMergeTreeDataPart::loadIndex()
|
||||
throw Exception("Index granularity is not loaded before index loading", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
const auto & primary_key = storage.getPrimaryKey();
|
||||
size_t key_size = primary_key.expression_column_names.size();
|
||||
size_t key_size = primary_key.column_names.size();
|
||||
|
||||
if (key_size)
|
||||
{
|
||||
@ -845,7 +845,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const
|
||||
|
||||
if (!checksums.empty())
|
||||
{
|
||||
if (!storage.getPrimaryKey().expression_column_names.empty() && !checksums.files.count("primary.idx"))
|
||||
if (storage.hasPrimaryKey() && !checksums.files.count("primary.idx"))
|
||||
throw Exception("No checksum for primary.idx", ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
@ -879,7 +879,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const
|
||||
};
|
||||
|
||||
/// Check that the primary key index is not empty.
|
||||
if (!storage.getPrimaryKey().expression_column_names.empty())
|
||||
if (storage.hasPrimaryKey())
|
||||
check_file_not_empty(volume->getDisk(), path + "primary.idx");
|
||||
|
||||
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
|
@ -155,7 +155,7 @@ MergeTreeData::MergeTreeData(
|
||||
StorageMetadataKeyField sampling_key = StorageMetadataKeyField::getKeyFromAST(metadata.sample_by_ast, getColumns(), global_context);
|
||||
|
||||
const auto & primary_key = getPrimaryKey();
|
||||
if (!primary_key.sample_block.has(sampling_key.expression_column_names[0])
|
||||
if (!primary_key.sample_block.has(sampling_key.column_names[0])
|
||||
&& !attach && !settings->compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility.
|
||||
throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
@ -473,18 +473,18 @@ void MergeTreeData::setProperties(const StorageInMemoryMetadata & metadata, bool
|
||||
|
||||
StorageMetadataKeyField new_sorting_key;
|
||||
new_sorting_key.definition_ast = metadata.order_by_ast;
|
||||
new_sorting_key.expression_column_names = std::move(new_sorting_key_columns);
|
||||
new_sorting_key.expression_ast = std::move(new_sorting_key_expr_list);
|
||||
new_sorting_key.expressions = std::move(new_sorting_key_expr);
|
||||
new_sorting_key.column_names = std::move(new_sorting_key_columns);
|
||||
new_sorting_key.expression_list_ast = std::move(new_sorting_key_expr_list);
|
||||
new_sorting_key.expression = std::move(new_sorting_key_expr);
|
||||
new_sorting_key.sample_block = std::move(new_sorting_key_sample);
|
||||
new_sorting_key.data_types = std::move(new_sorting_key_data_types);
|
||||
setSortingKey(new_sorting_key);
|
||||
|
||||
StorageMetadataKeyField new_primary_key;
|
||||
new_primary_key.definition_ast = metadata.primary_key_ast;
|
||||
new_primary_key.expression_column_names = std::move(new_primary_key_columns);
|
||||
new_primary_key.expression_ast = std::move(new_primary_key_expr_list);
|
||||
new_primary_key.expressions = std::move(new_primary_key_expr);
|
||||
new_primary_key.column_names = std::move(new_primary_key_columns);
|
||||
new_primary_key.expression_list_ast = std::move(new_primary_key_expr_list);
|
||||
new_primary_key.expression = std::move(new_primary_key_expr);
|
||||
new_primary_key.sample_block = std::move(new_primary_key_sample);
|
||||
new_primary_key.data_types = std::move(new_primary_key_data_types);
|
||||
setPrimaryKey(new_primary_key);
|
||||
@ -526,13 +526,13 @@ void MergeTreeData::initPartitionKey(ASTPtr partition_by_ast)
|
||||
{
|
||||
StorageMetadataKeyField new_partition_key = StorageMetadataKeyField::getKeyFromAST(partition_by_ast, getColumns(), global_context);
|
||||
|
||||
if (new_partition_key.expression_ast->children.empty())
|
||||
if (new_partition_key.expression_list_ast->children.empty())
|
||||
return;
|
||||
|
||||
checkKeyExpression(*new_partition_key.expressions, new_partition_key.sample_block, "Partition");
|
||||
checkKeyExpression(*new_partition_key.expression, new_partition_key.sample_block, "Partition");
|
||||
|
||||
/// Add all columns used in the partition key to the min-max index.
|
||||
const NamesAndTypesList & minmax_idx_columns_with_types = new_partition_key.expressions->getRequiredColumnsWithTypes();
|
||||
const NamesAndTypesList & minmax_idx_columns_with_types = new_partition_key.expression->getRequiredColumnsWithTypes();
|
||||
minmax_idx_expr = std::make_shared<ExpressionActions>(minmax_idx_columns_with_types, global_context);
|
||||
for (const NameAndTypePair & column : minmax_idx_columns_with_types)
|
||||
{
|
||||
@ -633,7 +633,7 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns,
|
||||
NameSet columns_ttl_forbidden;
|
||||
|
||||
if (hasPartitionKey())
|
||||
for (const auto & col : getPartitionKey().expressions->getRequiredColumns())
|
||||
for (const auto & col : getColumnsRequiredForPartitionKey())
|
||||
columns_ttl_forbidden.insert(col);
|
||||
|
||||
if (hasSortingKey())
|
||||
@ -1437,7 +1437,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
|
||||
if (hasSortingKey())
|
||||
{
|
||||
auto sorting_key_expr = getSortingKey().expressions;
|
||||
auto sorting_key_expr = getSortingKey().expression;
|
||||
for (const ExpressionAction & action : sorting_key_expr->getActions())
|
||||
{
|
||||
auto action_columns = action.getNeededColumns();
|
||||
@ -3100,7 +3100,7 @@ bool MergeTreeData::isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const A
|
||||
{
|
||||
const String column_name = node->getColumnName();
|
||||
|
||||
for (const auto & name : getPrimaryKey().expression_column_names)
|
||||
for (const auto & name : getPrimaryKeyColumns())
|
||||
if (column_name == name)
|
||||
return true;
|
||||
|
||||
|
@ -607,7 +607,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
NamesAndTypesList merging_columns;
|
||||
Names gathering_column_names, merging_column_names;
|
||||
extractMergingAndGatheringColumns(
|
||||
storage_columns, data.getSortingKey().expressions, data.skip_indices,
|
||||
storage_columns, data.getSortingKey().expression, data.skip_indices,
|
||||
data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names);
|
||||
|
||||
auto single_disk_volume = std::make_shared<SingleDiskVolume>("volume_" + future_part.name, disk);
|
||||
@ -726,7 +726,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
pipes.emplace_back(std::move(pipe));
|
||||
}
|
||||
|
||||
Names sort_columns = data.getSortingKey().expression_column_names;
|
||||
Names sort_columns = data.getSortingKeyColumns();
|
||||
SortDescription sort_description;
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
|
@ -223,9 +223,9 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const auto & primary_key = data.getPrimaryKey();
|
||||
Names primary_key_columns = primary_key.expression_column_names;
|
||||
Names primary_key_columns = primary_key.column_names;
|
||||
|
||||
KeyCondition key_condition(query_info, context, primary_key_columns, primary_key.expressions);
|
||||
KeyCondition key_condition(query_info, context, primary_key_columns, primary_key.expression);
|
||||
|
||||
if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue())
|
||||
{
|
||||
@ -461,14 +461,14 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
if (select.final())
|
||||
{
|
||||
sampling_key_ast = std::make_shared<ASTIdentifier>(sampling_key.expression_column_names[0]);
|
||||
sampling_key_ast = std::make_shared<ASTIdentifier>(sampling_key.column_names[0]);
|
||||
/// We do spoil available_real_columns here, but it is not used later.
|
||||
available_real_columns.emplace_back(sampling_key.expression_column_names[0], std::move(sampling_column_type));
|
||||
available_real_columns.emplace_back(sampling_key.column_names[0], std::move(sampling_column_type));
|
||||
}
|
||||
|
||||
if (has_lower_limit)
|
||||
{
|
||||
if (!key_condition.addCondition(sampling_key.expression_column_names[0], Range::createLeftBounded(lower, true)))
|
||||
if (!key_condition.addCondition(sampling_key.column_names[0], Range::createLeftBounded(lower, true)))
|
||||
throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
ASTPtr args = std::make_shared<ASTExpressionList>();
|
||||
@ -485,7 +485,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
if (has_upper_limit)
|
||||
{
|
||||
if (!key_condition.addCondition(sampling_key.expression_column_names[0], Range::createRightBounded(upper, false)))
|
||||
if (!key_condition.addCondition(sampling_key.column_names[0], Range::createRightBounded(upper, false)))
|
||||
throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
ASTPtr args = std::make_shared<ASTExpressionList>();
|
||||
@ -640,7 +640,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
else if (settings.optimize_read_in_order && query_info.input_sorting_info)
|
||||
{
|
||||
size_t prefix_size = query_info.input_sorting_info->order_key_prefix_descr.size();
|
||||
auto order_key_prefix_ast = data.getSortingKey().expression_ast->clone();
|
||||
auto order_key_prefix_ast = data.getSortingKey().expression_list_ast->clone();
|
||||
order_key_prefix_ast->children.resize(prefix_size);
|
||||
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(order_key_prefix_ast, data.getColumns().getAllPhysical());
|
||||
@ -1025,7 +1025,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder(
|
||||
{
|
||||
SortDescription sort_description;
|
||||
for (size_t j = 0; j < input_sorting_info->order_key_prefix_descr.size(); ++j)
|
||||
sort_description.emplace_back(data.getSortingKey().expression_column_names[j],
|
||||
sort_description.emplace_back(data.getSortingKey().column_names[j],
|
||||
input_sorting_info->direction, 1);
|
||||
|
||||
/// Drop temporary columns, added by 'sorting_key_prefix_expr'
|
||||
@ -1098,11 +1098,11 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
|
||||
if (!out_projection)
|
||||
out_projection = createProjection(pipe, data);
|
||||
|
||||
pipe.addSimpleTransform(std::make_shared<ExpressionTransform>(pipe.getHeader(), data.getSortingKey().expressions));
|
||||
pipe.addSimpleTransform(std::make_shared<ExpressionTransform>(pipe.getHeader(), data.getSortingKey().expression));
|
||||
pipes.emplace_back(std::move(pipe));
|
||||
}
|
||||
|
||||
Names sort_columns = data.getSortingKey().expression_column_names;
|
||||
Names sort_columns = data.getSortingKeyColumns();
|
||||
SortDescription sort_description;
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
@ -1300,7 +1300,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
|
||||
{
|
||||
auto index_block = std::make_shared<Block>();
|
||||
for (size_t i = 0; i < used_key_size; ++i)
|
||||
index_block->insert({index[i], primary_key.data_types[i], primary_key.expression_column_names[i]});
|
||||
index_block->insert({index[i], primary_key.data_types[i], primary_key.column_names[i]});
|
||||
|
||||
create_field_ref = [index_block](size_t row, size_t column, FieldRef & field)
|
||||
{
|
||||
|
@ -146,7 +146,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block
|
||||
|
||||
Block block_copy = block;
|
||||
const auto & partition_key = data.getPartitionKey();
|
||||
partition_key.expressions->execute(block_copy);
|
||||
partition_key.expression->execute(block_copy);
|
||||
|
||||
ColumnRawPtrs partition_columns;
|
||||
partition_columns.reserve(partition_key.sample_block.columns());
|
||||
@ -262,7 +262,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
if (data.hasSortingKey() || data.hasSkipIndices())
|
||||
data.sorting_key_and_skip_indices_expr->execute(block);
|
||||
|
||||
Names sort_columns = data.getSortingKey().expression_column_names;
|
||||
Names sort_columns = data.getSortingKeyColumns();
|
||||
SortDescription sort_description;
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
|
@ -40,8 +40,8 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer(
|
||||
log{log_}
|
||||
{
|
||||
const auto & primary_key = data.getPrimaryKey();
|
||||
if (!primary_key.expression_column_names.empty())
|
||||
first_primary_key_column = primary_key.expression_column_names[0];
|
||||
if (!primary_key.column_names.empty())
|
||||
first_primary_key_column = primary_key.column_names[0];
|
||||
|
||||
calculateColumnSizes(data, queried_columns);
|
||||
determineArrayJoinedNames(query_info.query->as<ASTSelectQuery &>());
|
||||
|
@ -161,7 +161,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
|
||||
std::inserter(skip_indexes_column_names_set, skip_indexes_column_names_set.end()));
|
||||
Names skip_indexes_column_names(skip_indexes_column_names_set.begin(), skip_indexes_column_names_set.end());
|
||||
|
||||
Block primary_key_block = getBlockAndPermute(block, storage.getPrimaryKey().expression_column_names, permutation);
|
||||
Block primary_key_block = getBlockAndPermute(block, storage.getPrimaryKeyColumns(), permutation);
|
||||
Block skip_indexes_block = getBlockAndPermute(block, skip_indexes_column_names, permutation);
|
||||
|
||||
writer->write(block, permutation, primary_key_block, skip_indexes_block);
|
||||
|
@ -41,17 +41,17 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr
|
||||
/// - When we have only ORDER BY, than store it in "primary key:" row of /metadata
|
||||
/// - When we have both, than store PRIMARY KEY in "primary key:" row and ORDER BY in "sorting key:" row of /metadata
|
||||
if (!data.isPrimaryKeyDefined())
|
||||
primary_key = formattedAST(data.getSortingKey().expression_ast);
|
||||
primary_key = formattedAST(data.getSortingKey().expression_list_ast);
|
||||
else
|
||||
{
|
||||
primary_key = formattedAST(data.getPrimaryKey().expression_ast);
|
||||
sorting_key = formattedAST(data.getSortingKey().expression_ast);
|
||||
primary_key = formattedAST(data.getPrimaryKey().expression_list_ast);
|
||||
sorting_key = formattedAST(data.getSortingKey().expression_list_ast);
|
||||
}
|
||||
|
||||
data_format_version = data.format_version;
|
||||
|
||||
if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
partition_key = formattedAST(data.getPartitionKey().expression_ast);
|
||||
partition_key = formattedAST(data.getPartitionKey().expression_list_ast);
|
||||
|
||||
ttl_table = formattedAST(data.ttl_table_ast);
|
||||
|
||||
|
@ -116,19 +116,20 @@ StorageMetadataKeyField StorageMetadataKeyField::getKeyFromAST(const ASTPtr & de
|
||||
{
|
||||
StorageMetadataKeyField result;
|
||||
result.definition_ast = definition_ast;
|
||||
result.expression_ast = extractKeyExpressionList(definition_ast);
|
||||
result.expression_list_ast = extractKeyExpressionList(definition_ast);
|
||||
|
||||
if (result.expression_ast->children.empty())
|
||||
if (result.expression_list_ast->children.empty())
|
||||
return result;
|
||||
|
||||
const auto & children = result.expression_ast->children;
|
||||
const auto & children = result.expression_list_ast->children;
|
||||
for (const auto & child : children)
|
||||
result.expression_column_names.emplace_back(child->getColumnName());
|
||||
result.column_names.emplace_back(child->getColumnName());
|
||||
|
||||
{
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(result.expression_ast, columns.getAllPhysical());
|
||||
result.expressions = ExpressionAnalyzer(result.expression_ast->clone(), syntax_result, context).getActions(true);
|
||||
result.sample_block = result.expressions->getSampleBlock();
|
||||
auto expr = result.expression_list_ast->clone();
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(expr, columns.getAllPhysical());
|
||||
result.expression = ExpressionAnalyzer(expr, syntax_result, context).getActions(true);
|
||||
result.sample_block = result.expression->getSampleBlock();
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < result.sample_block.columns(); ++i)
|
||||
|
@ -52,22 +52,22 @@ struct StorageMetadataKeyField
|
||||
ASTPtr definition_ast;
|
||||
|
||||
/// ASTExpressionList with key fields, example: (x, toStartOfMonth(date))).
|
||||
ASTPtr expression_ast;
|
||||
ASTPtr expression_list_ast;
|
||||
|
||||
/// Expressions from expression_list_ast from expression_analyzer. Useful,
|
||||
/// Expression from expression_list_ast created by ExpressionAnalyzer. Useful,
|
||||
/// when you need to get required columns for key, example: a, date, b.
|
||||
ExpressionActionsPtr expressions;
|
||||
|
||||
/// Column names in key definition, example: x, toStartOfMonth(date), a * b.
|
||||
Names expression_column_names;
|
||||
ExpressionActionsPtr expression;
|
||||
|
||||
/// Sample block with key columns (names, types, empty column)
|
||||
Block sample_block;
|
||||
|
||||
/// Column names in key definition, example: x, toStartOfMonth(date), a * b.
|
||||
Names column_names;
|
||||
|
||||
/// Types from sample block ordered in columns order.
|
||||
DataTypes data_types;
|
||||
|
||||
/// Parse key structure from key definition. Requires all columns available
|
||||
/// Parse key structure from key definition. Requires all columns, available
|
||||
/// in storage.
|
||||
static StorageMetadataKeyField getKeyFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context);
|
||||
};
|
||||
|
Loading…
Reference in New Issue
Block a user