mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
commit
a1ba67e029
@ -714,6 +714,8 @@ ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast
|
||||
new_columns_list->set(new_columns_list->columns, new_columns);
|
||||
if (const auto * indices = query_ast->as<ASTCreateQuery>()->columns_list->indices)
|
||||
new_columns_list->set(new_columns_list->indices, indices->clone());
|
||||
if (const auto * projections = query_ast->as<ASTCreateQuery>()->columns_list->projections)
|
||||
new_columns_list->set(new_columns_list->projections, projections->clone());
|
||||
|
||||
new_query.replace(new_query.columns_list, new_columns_list);
|
||||
|
||||
|
@ -1033,6 +1033,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
auto & database_catalog = DatabaseCatalog::instance();
|
||||
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
|
||||
attachSystemTablesServer(*database_catalog.getSystemDatabase(), has_zookeeper);
|
||||
/// We load temporary database first, because projections need it.
|
||||
database_catalog.loadTemporaryDatabase();
|
||||
/// Then, load remaining databases
|
||||
loadMetadata(global_context, default_database);
|
||||
database_catalog.loadDatabases();
|
||||
|
@ -53,6 +53,12 @@ enum class AccessType
|
||||
M(ALTER_CLEAR_INDEX, "CLEAR INDEX", TABLE, ALTER_INDEX) \
|
||||
M(ALTER_INDEX, "INDEX", GROUP, ALTER_TABLE) /* allows to execute ALTER ORDER BY or ALTER {ADD|DROP...} INDEX */\
|
||||
\
|
||||
M(ALTER_ADD_PROJECTION, "ADD PROJECTION", TABLE, ALTER_PROJECTION) \
|
||||
M(ALTER_DROP_PROJECTION, "DROP PROJECTION", TABLE, ALTER_PROJECTION) \
|
||||
M(ALTER_MATERIALIZE_PROJECTION, "MATERIALIZE PROJECTION", TABLE, ALTER_PROJECTION) \
|
||||
M(ALTER_CLEAR_PROJECTION, "CLEAR PROJECTION", TABLE, ALTER_PROJECTION) \
|
||||
M(ALTER_PROJECTION, "PROJECTION", GROUP, ALTER_TABLE) /* allows to execute ALTER ORDER BY or ALTER {ADD|DROP...} PROJECTION */\
|
||||
\
|
||||
M(ALTER_ADD_CONSTRAINT, "ADD CONSTRAINT", TABLE, ALTER_CONSTRAINT) \
|
||||
M(ALTER_DROP_CONSTRAINT, "DROP CONSTRAINT", TABLE, ALTER_CONSTRAINT) \
|
||||
M(ALTER_CONSTRAINT, "CONSTRAINT", GROUP, ALTER_TABLE) /* allows to execute ALTER {ADD|DROP} CONSTRAINT */\
|
||||
|
@ -549,6 +549,9 @@
|
||||
M(579, INCORRECT_PART_TYPE) \
|
||||
M(580, CANNOT_SET_ROUNDING_MODE) \
|
||||
M(581, TOO_LARGE_DISTRIBUTED_DEPTH) \
|
||||
M(582, NO_SUCH_PROJECTION_IN_TABLE) \
|
||||
M(583, ILLEGAL_PROJECTION) \
|
||||
M(584, PROJECTION_NOT_USED) \
|
||||
\
|
||||
M(998, POSTGRESQL_CONNECTION_FAILURE) \
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
|
@ -135,6 +135,12 @@
|
||||
M(MergeTreeDataWriterBlocks, "Number of blocks INSERTed to MergeTree tables. Each block forms a data part of level zero.") \
|
||||
M(MergeTreeDataWriterBlocksAlreadySorted, "Number of blocks INSERTed to MergeTree tables that appeared to be already sorted.") \
|
||||
\
|
||||
M(MergeTreeDataProjectionWriterRows, "Number of rows INSERTed to MergeTree tables projection.") \
|
||||
M(MergeTreeDataProjectionWriterUncompressedBytes, "Uncompressed bytes (for columns as they stored in memory) INSERTed to MergeTree tables projection.") \
|
||||
M(MergeTreeDataProjectionWriterCompressedBytes, "Bytes written to filesystem for data INSERTed to MergeTree tables projection.") \
|
||||
M(MergeTreeDataProjectionWriterBlocks, "Number of blocks INSERTed to MergeTree tables projection. Each block forms a data part of level zero.") \
|
||||
M(MergeTreeDataProjectionWriterBlocksAlreadySorted, "Number of blocks INSERTed to MergeTree tables projection that appeared to be already sorted.") \
|
||||
\
|
||||
M(CannotRemoveEphemeralNode, "Number of times an error happened while trying to remove ephemeral node. This is not an issue, because our implementation of ZooKeeper library guarantee that the session will expire and the node will be removed.") \
|
||||
\
|
||||
M(RegexpCreated, "Compiled regular expressions. Identical regular expressions compiled just once and cached forever.") \
|
||||
|
@ -434,6 +434,8 @@ class IColumn;
|
||||
M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \
|
||||
M(Bool, allow_experimental_map_type, false, "Allow data type Map", 0) \
|
||||
M(Bool, allow_experimental_window_functions, false, "Allow experimental window functions", 0) \
|
||||
M(Bool, allow_experimental_projection_optimization, false, "Enable projection optimization when processing SELECT queries", 0) \
|
||||
M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \
|
||||
M(Bool, use_antlr_parser, false, "Parse incoming queries using ANTLR-generated experimental parser", 0) \
|
||||
M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \
|
||||
M(Bool, insert_null_as_default, true, "Insert DEFAULT values instead of NULL in INSERT SELECT (UNION ALL)", 0) \
|
||||
|
@ -480,7 +480,7 @@ void RemoteQueryExecutor::sendExternalTables()
|
||||
SelectQueryInfo query_info;
|
||||
auto metadata_snapshot = cur->getInMemoryMetadataPtr();
|
||||
QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage(
|
||||
context, QueryProcessingStage::Complete, query_info);
|
||||
context, QueryProcessingStage::Complete, metadata_snapshot, query_info);
|
||||
|
||||
Pipe pipe = cur->read(
|
||||
metadata_snapshot->getColumns().getNamesOfPhysical(),
|
||||
|
@ -148,10 +148,12 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo
|
||||
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(metadata.columns);
|
||||
ASTPtr new_indices = InterpreterCreateQuery::formatIndices(metadata.secondary_indices);
|
||||
ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(metadata.constraints);
|
||||
ASTPtr new_projections = InterpreterCreateQuery::formatProjections(metadata.projections);
|
||||
|
||||
ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns);
|
||||
ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->indices, new_indices);
|
||||
ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->constraints, new_constraints);
|
||||
ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->projections, new_projections);
|
||||
|
||||
if (metadata.select.select_query)
|
||||
{
|
||||
|
@ -439,6 +439,99 @@ void ActionsDAG::removeUnusedActions(bool allow_remove_inputs)
|
||||
inputs.erase(it, inputs.end());
|
||||
}
|
||||
|
||||
NameSet ActionsDAG::foldActionsByProjection(
|
||||
const NameSet & required_columns, const Block & projection_block_for_keys, const String & predicate_column_name, bool add_missing_keys)
|
||||
{
|
||||
std::unordered_set<const Node *> visited_nodes;
|
||||
std::unordered_set<std::string_view> visited_index_names;
|
||||
std::stack<Node *> stack;
|
||||
std::vector<const ColumnWithTypeAndName *> missing_input_from_projection_keys;
|
||||
|
||||
for (const auto & node : index)
|
||||
{
|
||||
if (required_columns.find(node->result_name) != required_columns.end() || node->result_name == predicate_column_name)
|
||||
{
|
||||
visited_nodes.insert(node);
|
||||
visited_index_names.insert(node->result_name);
|
||||
stack.push(const_cast<Node *>(node));
|
||||
}
|
||||
}
|
||||
|
||||
if (add_missing_keys)
|
||||
{
|
||||
for (const auto & column : required_columns)
|
||||
{
|
||||
if (visited_index_names.find(column) == visited_index_names.end())
|
||||
{
|
||||
if (const ColumnWithTypeAndName * column_with_type_name = projection_block_for_keys.findByName(column))
|
||||
{
|
||||
const auto * node = &addInput(*column_with_type_name);
|
||||
visited_nodes.insert(node);
|
||||
index.push_back(node);
|
||||
visited_index_names.insert(column);
|
||||
}
|
||||
else
|
||||
{
|
||||
// Missing column
|
||||
return {};
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
while (!stack.empty())
|
||||
{
|
||||
auto * node = stack.top();
|
||||
stack.pop();
|
||||
|
||||
if (const ColumnWithTypeAndName * column_with_type_name = projection_block_for_keys.findByName(node->result_name))
|
||||
{
|
||||
if (node->type != ActionsDAG::ActionType::INPUT)
|
||||
{
|
||||
/// Projection folding.
|
||||
node->type = ActionsDAG::ActionType::INPUT;
|
||||
node->result_type = std::move(column_with_type_name->type);
|
||||
node->result_name = std::move(column_with_type_name->name);
|
||||
node->children.clear();
|
||||
inputs.push_back(node);
|
||||
}
|
||||
}
|
||||
|
||||
for (const auto * child : node->children)
|
||||
{
|
||||
if (visited_nodes.count(child) == 0)
|
||||
{
|
||||
stack.push(const_cast<Node *>(child));
|
||||
visited_nodes.insert(child);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
std::erase_if(inputs, [&](const Node * node) { return visited_nodes.count(node) == 0; });
|
||||
std::erase_if(index, [&](const Node * node) { return visited_index_names.count(node->result_name) == 0; });
|
||||
nodes.remove_if([&](const Node & node) { return visited_nodes.count(&node) == 0; });
|
||||
|
||||
NameSet next_required_columns;
|
||||
for (const auto & input : inputs)
|
||||
next_required_columns.insert(input->result_name);
|
||||
|
||||
return next_required_columns;
|
||||
}
|
||||
|
||||
void ActionsDAG::reorderAggregationKeysForProjection(const std::unordered_map<std::string_view, size_t> & key_names_pos_map)
|
||||
{
|
||||
std::sort(index.begin(), index.end(), [&key_names_pos_map](const Node * lhs, const Node * rhs)
|
||||
{
|
||||
return key_names_pos_map.find(lhs->result_name)->second < key_names_pos_map.find(rhs->result_name)->second;
|
||||
});
|
||||
}
|
||||
|
||||
void ActionsDAG::addAggregatesViaProjection(const Block & aggregates)
|
||||
{
|
||||
for (const auto & aggregate : aggregates)
|
||||
index.push_back(&addInput(aggregate));
|
||||
}
|
||||
|
||||
void ActionsDAG::addAliases(const NamesWithAliases & aliases)
|
||||
{
|
||||
std::unordered_map<std::string_view, size_t> names_map;
|
||||
|
@ -168,6 +168,14 @@ public:
|
||||
void removeUnusedActions(const Names & required_names);
|
||||
void removeUnusedActions(const NameSet & required_names);
|
||||
|
||||
NameSet foldActionsByProjection(
|
||||
const NameSet & required_columns,
|
||||
const Block & projection_block_for_keys,
|
||||
const String & predicate_column_name = {},
|
||||
bool add_missing_keys = true);
|
||||
void reorderAggregationKeysForProjection(const std::unordered_map<std::string_view, size_t> & key_names_pos_map);
|
||||
void addAggregatesViaProjection(const Block & aggregates);
|
||||
|
||||
bool hasArrayJoin() const;
|
||||
bool hasStatefulFunctions() const;
|
||||
bool trivial() const; /// If actions has no functions or array join.
|
||||
|
@ -1902,6 +1902,85 @@ void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl(
|
||||
block.clear();
|
||||
}
|
||||
|
||||
bool Aggregator::mergeBlock(Block block, AggregatedDataVariants & result, bool & no_more_keys)
|
||||
{
|
||||
/// `result` will destroy the states of aggregate functions in the destructor
|
||||
result.aggregator = this;
|
||||
|
||||
/// How to perform the aggregation?
|
||||
if (result.empty())
|
||||
{
|
||||
result.init(method_chosen);
|
||||
result.keys_size = params.keys_size;
|
||||
result.key_sizes = key_sizes;
|
||||
LOG_TRACE(log, "Aggregation method: {}", result.getMethodName());
|
||||
}
|
||||
|
||||
if (result.type == AggregatedDataVariants::Type::without_key || block.info.is_overflows)
|
||||
mergeWithoutKeyStreamsImpl(block, result);
|
||||
|
||||
#define M(NAME, IS_TWO_LEVEL) \
|
||||
else if (result.type == AggregatedDataVariants::Type::NAME) \
|
||||
mergeStreamsImpl(block, result.aggregates_pool, *result.NAME, result.NAME->data, result.without_key, no_more_keys);
|
||||
|
||||
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
||||
#undef M
|
||||
else if (result.type != AggregatedDataVariants::Type::without_key)
|
||||
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
||||
|
||||
size_t result_size = result.sizeWithoutOverflowRow();
|
||||
Int64 current_memory_usage = 0;
|
||||
if (auto * memory_tracker_child = CurrentThread::getMemoryTracker())
|
||||
if (auto * memory_tracker = memory_tracker_child->getParent())
|
||||
current_memory_usage = memory_tracker->get();
|
||||
|
||||
/// Here all the results in the sum are taken into account, from different threads.
|
||||
auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation;
|
||||
|
||||
bool worth_convert_to_two_level
|
||||
= (params.group_by_two_level_threshold && result_size >= params.group_by_two_level_threshold)
|
||||
|| (params.group_by_two_level_threshold_bytes && result_size_bytes >= static_cast<Int64>(params.group_by_two_level_threshold_bytes));
|
||||
|
||||
/** Converting to a two-level data structure.
|
||||
* It allows you to make, in the subsequent, an effective merge - either economical from memory or parallel.
|
||||
*/
|
||||
if (result.isConvertibleToTwoLevel() && worth_convert_to_two_level)
|
||||
result.convertToTwoLevel();
|
||||
|
||||
/// Checking the constraints.
|
||||
if (!checkLimits(result_size, no_more_keys))
|
||||
return false;
|
||||
|
||||
/** Flush data to disk if too much RAM is consumed.
|
||||
* Data can only be flushed to disk if a two-level aggregation structure is used.
|
||||
*/
|
||||
if (params.max_bytes_before_external_group_by
|
||||
&& result.isTwoLevel()
|
||||
&& current_memory_usage > static_cast<Int64>(params.max_bytes_before_external_group_by)
|
||||
&& worth_convert_to_two_level)
|
||||
{
|
||||
size_t size = current_memory_usage + params.min_free_disk_space;
|
||||
|
||||
std::string tmp_path = params.tmp_volume->getDisk()->getPath();
|
||||
|
||||
// enoughSpaceInDirectory() is not enough to make it right, since
|
||||
// another process (or another thread of aggregator) can consume all
|
||||
// space.
|
||||
//
|
||||
// But true reservation (IVolume::reserve()) cannot be used here since
|
||||
// current_memory_usage does not takes compression into account and
|
||||
// will reserve way more that actually will be used.
|
||||
//
|
||||
// Hence let's do a simple check.
|
||||
if (!enoughSpaceInDirectory(tmp_path, size))
|
||||
throw Exception("Not enough space for external aggregation in " + tmp_path, ErrorCodes::NOT_ENOUGH_SPACE);
|
||||
|
||||
writeToTemporaryFile(result, tmp_path);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVariants & result, size_t max_threads)
|
||||
{
|
||||
|
@ -983,6 +983,8 @@ public:
|
||||
/// Merge partially aggregated blocks separated to buckets into one data structure.
|
||||
void mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVariants & result, size_t max_threads);
|
||||
|
||||
bool mergeBlock(Block block, AggregatedDataVariants & result, bool & no_more_keys);
|
||||
|
||||
/// Merge several partially aggregated blocks into one.
|
||||
/// Precondition: for all blocks block.info.is_overflows flag must be the same.
|
||||
/// (either all blocks are from overflow data or none blocks are).
|
||||
|
@ -992,7 +992,8 @@ bool Context::hasScalar(const String & name) const
|
||||
}
|
||||
|
||||
|
||||
void Context::addQueryAccessInfo(const String & quoted_database_name, const String & full_quoted_table_name, const Names & column_names)
|
||||
void Context::addQueryAccessInfo(
|
||||
const String & quoted_database_name, const String & full_quoted_table_name, const Names & column_names, const String & projection_name)
|
||||
{
|
||||
assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL);
|
||||
std::lock_guard<std::mutex> lock(query_access_info.mutex);
|
||||
@ -1000,6 +1001,8 @@ void Context::addQueryAccessInfo(const String & quoted_database_name, const Stri
|
||||
query_access_info.tables.emplace(full_quoted_table_name);
|
||||
for (const auto & column_name : column_names)
|
||||
query_access_info.columns.emplace(full_quoted_table_name + "." + backQuoteIfNeed(column_name));
|
||||
if (!projection_name.empty())
|
||||
query_access_info.projections.emplace(full_quoted_table_name + "." + backQuoteIfNeed(projection_name));
|
||||
}
|
||||
|
||||
|
||||
|
@ -202,6 +202,7 @@ private:
|
||||
databases = rhs.databases;
|
||||
tables = rhs.tables;
|
||||
columns = rhs.columns;
|
||||
projections = rhs.projections;
|
||||
}
|
||||
|
||||
QueryAccessInfo(QueryAccessInfo && rhs) = delete;
|
||||
@ -217,6 +218,7 @@ private:
|
||||
std::swap(databases, rhs.databases);
|
||||
std::swap(tables, rhs.tables);
|
||||
std::swap(columns, rhs.columns);
|
||||
std::swap(projections, rhs.projections);
|
||||
}
|
||||
|
||||
/// To prevent a race between copy-constructor and other uses of this structure.
|
||||
@ -224,6 +226,7 @@ private:
|
||||
std::set<std::string> databases{};
|
||||
std::set<std::string> tables{};
|
||||
std::set<std::string> columns{};
|
||||
std::set<std::string> projections;
|
||||
};
|
||||
|
||||
QueryAccessInfo query_access_info;
|
||||
@ -430,7 +433,11 @@ public:
|
||||
bool hasScalar(const String & name) const;
|
||||
|
||||
const QueryAccessInfo & getQueryAccessInfo() const { return query_access_info; }
|
||||
void addQueryAccessInfo(const String & quoted_database_name, const String & full_quoted_table_name, const Names & column_names);
|
||||
void addQueryAccessInfo(
|
||||
const String & quoted_database_name,
|
||||
const String & full_quoted_table_name,
|
||||
const Names & column_names,
|
||||
const String & projection_name = {});
|
||||
|
||||
/// Supported factories for records in query_log
|
||||
enum class QueryLogFactories
|
||||
|
@ -135,13 +135,16 @@ StoragePtr TemporaryTableHolder::getTable() const
|
||||
}
|
||||
|
||||
|
||||
void DatabaseCatalog::loadDatabases()
|
||||
void DatabaseCatalog::loadTemporaryDatabase()
|
||||
{
|
||||
drop_delay_sec = getContext()->getConfigRef().getInt("database_atomic_delay_before_drop_table_sec", default_drop_delay_sec);
|
||||
|
||||
auto db_for_temporary_and_external_tables = std::make_shared<DatabaseMemory>(TEMPORARY_DATABASE, getContext());
|
||||
attachDatabase(TEMPORARY_DATABASE, db_for_temporary_and_external_tables);
|
||||
}
|
||||
|
||||
void DatabaseCatalog::loadDatabases()
|
||||
{
|
||||
loadMarkedAsDroppedTables();
|
||||
auto task_holder = getContext()->getSchedulePool().createTask("DatabaseCatalog", [this](){ this->dropTableDataTask(); });
|
||||
drop_task = std::make_unique<BackgroundSchedulePoolTaskHolder>(std::move(task_holder));
|
||||
|
@ -127,6 +127,7 @@ public:
|
||||
static DatabaseCatalog & instance();
|
||||
static void shutdown();
|
||||
|
||||
void loadTemporaryDatabase();
|
||||
void loadDatabases();
|
||||
|
||||
/// Get an object that protects the table from concurrently executing multiple DDL operations.
|
||||
|
@ -1638,7 +1638,7 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si
|
||||
|
||||
if (hasWhere())
|
||||
{
|
||||
auto where_column_name = query.where()->getColumnName();
|
||||
where_column_name = query.where()->getColumnName();
|
||||
remove_where_filter = chain.steps.at(where_step_num)->required_output.find(where_column_name)->second;
|
||||
}
|
||||
}
|
||||
|
@ -209,6 +209,7 @@ struct ExpressionAnalysisResult
|
||||
bool has_order_by = false;
|
||||
bool has_window = false;
|
||||
|
||||
String where_column_name;
|
||||
bool remove_where_filter = false;
|
||||
bool optimize_read_in_order = false;
|
||||
bool optimize_aggregation_in_order = false;
|
||||
@ -231,6 +232,9 @@ struct ExpressionAnalysisResult
|
||||
/// perform SELECT DISTINCT.
|
||||
Names selected_columns;
|
||||
|
||||
/// Columns to read from storage if any.
|
||||
Names required_columns;
|
||||
|
||||
/// Columns will be removed after prewhere actions execution.
|
||||
NameSet columns_to_remove_after_prewhere;
|
||||
|
||||
|
@ -241,6 +241,24 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS
|
||||
required_access.emplace_back(AccessType::ALTER_DROP_CONSTRAINT, database, table);
|
||||
break;
|
||||
}
|
||||
case ASTAlterCommand::ADD_PROJECTION:
|
||||
{
|
||||
required_access.emplace_back(AccessType::ALTER_ADD_PROJECTION, database, table);
|
||||
break;
|
||||
}
|
||||
case ASTAlterCommand::DROP_PROJECTION:
|
||||
{
|
||||
if (command.clear_projection)
|
||||
required_access.emplace_back(AccessType::ALTER_CLEAR_PROJECTION, database, table);
|
||||
else
|
||||
required_access.emplace_back(AccessType::ALTER_DROP_PROJECTION, database, table);
|
||||
break;
|
||||
}
|
||||
case ASTAlterCommand::MATERIALIZE_PROJECTION:
|
||||
{
|
||||
required_access.emplace_back(AccessType::ALTER_MATERIALIZE_PROJECTION, database, table);
|
||||
break;
|
||||
}
|
||||
case ASTAlterCommand::MODIFY_TTL:
|
||||
case ASTAlterCommand::REMOVE_TTL:
|
||||
{
|
||||
|
@ -143,7 +143,9 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
storage->set(storage->engine, engine);
|
||||
create.set(create.storage, storage);
|
||||
}
|
||||
else if ((create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty()))
|
||||
else if ((create.columns_list
|
||||
&& ((create.columns_list->indices && !create.columns_list->indices->children.empty())
|
||||
|| (create.columns_list->projections && !create.columns_list->projections->children.empty()))))
|
||||
{
|
||||
/// Currently, there are no database engines, that support any arguments.
|
||||
throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", serializeAST(*create.storage));
|
||||
@ -360,6 +362,16 @@ ASTPtr InterpreterCreateQuery::formatConstraints(const ConstraintsDescription &
|
||||
return res;
|
||||
}
|
||||
|
||||
ASTPtr InterpreterCreateQuery::formatProjections(const ProjectionsDescription & projections)
|
||||
{
|
||||
auto res = std::make_shared<ASTExpressionList>();
|
||||
|
||||
for (const auto & projection : projections)
|
||||
res->children.push_back(projection.definition_ast->clone());
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
|
||||
const ASTExpressionList & columns_ast, ContextPtr context_, bool attach)
|
||||
{
|
||||
@ -518,6 +530,13 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS
|
||||
properties.indices.push_back(
|
||||
IndexDescription::getIndexFromAST(index->clone(), properties.columns, getContext()));
|
||||
|
||||
if (create.columns_list->projections)
|
||||
for (const auto & projection_ast : create.columns_list->projections->children)
|
||||
{
|
||||
auto projection = ProjectionDescription::getProjectionFromAST(projection_ast, properties.columns, getContext());
|
||||
properties.projections.add(std::move(projection));
|
||||
}
|
||||
|
||||
properties.constraints = getConstraintsDescription(create.columns_list->constraints);
|
||||
}
|
||||
else if (!create.as_table.empty())
|
||||
@ -564,10 +583,12 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS
|
||||
ASTPtr new_columns = formatColumns(properties.columns);
|
||||
ASTPtr new_indices = formatIndices(properties.indices);
|
||||
ASTPtr new_constraints = formatConstraints(properties.constraints);
|
||||
ASTPtr new_projections = formatProjections(properties.projections);
|
||||
|
||||
create.columns_list->setOrReplace(create.columns_list->columns, new_columns);
|
||||
create.columns_list->setOrReplace(create.columns_list->indices, new_indices);
|
||||
create.columns_list->setOrReplace(create.columns_list->constraints, new_constraints);
|
||||
create.columns_list->setOrReplace(create.columns_list->projections, new_projections);
|
||||
|
||||
validateTableStructure(create, properties);
|
||||
/// Set the table engine if it was not specified explicitly.
|
||||
|
@ -35,6 +35,7 @@ public:
|
||||
|
||||
static ASTPtr formatIndices(const IndicesDescription & indices);
|
||||
static ASTPtr formatConstraints(const ConstraintsDescription & constraints);
|
||||
static ASTPtr formatProjections(const ProjectionsDescription & projections);
|
||||
|
||||
void setForceRestoreData(bool has_force_restore_data_flag_)
|
||||
{
|
||||
@ -66,6 +67,7 @@ private:
|
||||
ColumnsDescription columns;
|
||||
IndicesDescription indices;
|
||||
ConstraintsDescription constraints;
|
||||
ProjectionsDescription projections;
|
||||
};
|
||||
|
||||
BlockIO createDatabase(ASTCreateQuery & create);
|
||||
|
@ -282,6 +282,8 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
{
|
||||
checkStackSize();
|
||||
|
||||
query_info.ignore_projections = options.ignore_projections;
|
||||
|
||||
initSettings();
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
@ -381,6 +383,8 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
TreeRewriterResult(source_header.getNamesAndTypesList(), storage, metadata_snapshot),
|
||||
options, joined_tables.tablesWithColumns(), required_result_column_names, table_join);
|
||||
|
||||
query_info.syntax_analyzer_result = syntax_analyzer_result;
|
||||
|
||||
/// Save scalar sub queries's results in the query context
|
||||
if (!options.only_analyze && context->hasQueryContext())
|
||||
for (const auto & it : syntax_analyzer_result->getScalars())
|
||||
@ -417,6 +421,13 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
}
|
||||
}
|
||||
|
||||
if (query.prewhere() && query.where())
|
||||
{
|
||||
/// Filter block in WHERE instead to get better performance
|
||||
query.setExpression(
|
||||
ASTSelectQuery::Expression::WHERE, makeASTFunction("and", query.prewhere()->clone(), query.where()->clone()));
|
||||
}
|
||||
|
||||
query_analyzer = std::make_unique<SelectQueryExpressionAnalyzer>(
|
||||
query_ptr, syntax_analyzer_result, context, metadata_snapshot,
|
||||
NameSet(required_result_column_names.begin(), required_result_column_names.end()),
|
||||
@ -500,12 +511,6 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
query.setExpression(ASTSelectQuery::Expression::WHERE, std::make_shared<ASTLiteral>(0u));
|
||||
need_analyze_again = true;
|
||||
}
|
||||
if (query.prewhere() && query.where())
|
||||
{
|
||||
/// Filter block in WHERE instead to get better performance
|
||||
query.setExpression(ASTSelectQuery::Expression::WHERE, makeASTFunction("and", query.prewhere()->clone(), query.where()->clone()));
|
||||
need_analyze_again = true;
|
||||
}
|
||||
|
||||
if (need_analyze_again)
|
||||
{
|
||||
@ -536,6 +541,13 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
}
|
||||
}
|
||||
|
||||
/// Add prewhere actions with alias columns and record needed columns from storage.
|
||||
if (storage)
|
||||
{
|
||||
addPrewhereAliasActions();
|
||||
analysis_result.required_columns = required_columns;
|
||||
}
|
||||
|
||||
/// Blocks used in expression analysis contains size 1 const columns for constant folding and
|
||||
/// null non-const columns to avoid useless memory allocations. However, a valid block sample
|
||||
/// requires all columns to be of size 0, thus we need to sanitize the block here.
|
||||
@ -547,7 +559,10 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan)
|
||||
executeImpl(query_plan, input, std::move(input_pipe));
|
||||
|
||||
/// We must guarantee that result structure is the same as in getSampleBlock()
|
||||
if (!blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, result_header))
|
||||
///
|
||||
/// But if we ignore aggregation, plan header does not match result_header.
|
||||
/// TODO: add special stage for InterpreterSelectQuery?
|
||||
if (!options.ignore_aggregation && !blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, result_header))
|
||||
{
|
||||
auto convert_actions_dag = ActionsDAG::makeConvertingActions(
|
||||
query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(),
|
||||
@ -580,7 +595,14 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
|
||||
query_info.query = query_ptr;
|
||||
|
||||
if (storage && !options.only_analyze)
|
||||
from_stage = storage->getQueryProcessingStage(context, options.to_stage, query_info);
|
||||
{
|
||||
from_stage = storage->getQueryProcessingStage(context, options.to_stage, metadata_snapshot, query_info);
|
||||
|
||||
/// TODO how can we make IN index work if we cache parts before selecting a projection?
|
||||
/// XXX Used for IN set index analysis. Is this a proper way?
|
||||
if (query_info.projection)
|
||||
metadata_snapshot->selected_projection = query_info.projection->desc;
|
||||
}
|
||||
|
||||
/// Do I need to perform the first part of the pipeline?
|
||||
/// Running on remote servers during distributed processing or if query is not distributed.
|
||||
@ -889,31 +911,24 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
|
||||
bool to_aggregation_stage = false;
|
||||
bool from_aggregation_stage = false;
|
||||
|
||||
if (expressions.filter_info)
|
||||
/// Do I need to aggregate in a separate row rows that have not passed max_rows_to_group_by.
|
||||
bool aggregate_overflow_row =
|
||||
expressions.need_aggregate &&
|
||||
query.group_by_with_totals &&
|
||||
settings.max_rows_to_group_by &&
|
||||
settings.group_by_overflow_mode == OverflowMode::ANY &&
|
||||
settings.totals_mode != TotalsMode::AFTER_HAVING_EXCLUSIVE;
|
||||
|
||||
/// Do I need to immediately finalize the aggregate functions after the aggregation?
|
||||
bool aggregate_final =
|
||||
expressions.need_aggregate &&
|
||||
options.to_stage > QueryProcessingStage::WithMergeableState &&
|
||||
!query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube;
|
||||
|
||||
if (query_info.projection && query_info.projection->desc->type == ProjectionDescription::Type::Aggregate)
|
||||
{
|
||||
if (!expressions.prewhere_info)
|
||||
{
|
||||
const bool does_storage_support_prewhere = !input && !input_pipe && storage && storage->supportsPrewhere();
|
||||
if (does_storage_support_prewhere && settings.optimize_move_to_prewhere)
|
||||
{
|
||||
/// Execute row level filter in prewhere as a part of "move to prewhere" optimization.
|
||||
expressions.prewhere_info = std::make_shared<PrewhereDAGInfo>(
|
||||
std::move(expressions.filter_info->actions),
|
||||
std::move(expressions.filter_info->column_name));
|
||||
expressions.prewhere_info->prewhere_actions->projectInput(false);
|
||||
expressions.prewhere_info->remove_prewhere_column = expressions.filter_info->do_remove_column;
|
||||
expressions.prewhere_info->need_filter = true;
|
||||
expressions.filter_info = nullptr;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Add row level security actions to prewhere.
|
||||
expressions.prewhere_info->row_level_filter_actions = std::move(expressions.filter_info->actions);
|
||||
expressions.prewhere_info->row_level_column_name = std::move(expressions.filter_info->column_name);
|
||||
expressions.prewhere_info->row_level_filter_actions->projectInput(false);
|
||||
expressions.filter_info = nullptr;
|
||||
}
|
||||
query_info.projection->aggregate_overflow_row = aggregate_overflow_row;
|
||||
query_info.projection->aggregate_final = aggregate_final;
|
||||
}
|
||||
|
||||
if (options.only_analyze)
|
||||
@ -958,6 +973,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
|
||||
|
||||
// To remove additional columns in dry run
|
||||
// For example, sample column which can be removed in this stage
|
||||
// TODO There seems to be no place initializing remove_columns_actions
|
||||
if (expressions.prewhere_info->remove_columns_actions)
|
||||
{
|
||||
auto remove_columns = std::make_unique<ExpressionStep>(
|
||||
@ -1003,20 +1019,6 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
|
||||
|
||||
if (options.to_stage > QueryProcessingStage::FetchColumns)
|
||||
{
|
||||
/// Do I need to aggregate in a separate row rows that have not passed max_rows_to_group_by.
|
||||
bool aggregate_overflow_row =
|
||||
expressions.need_aggregate &&
|
||||
query.group_by_with_totals &&
|
||||
settings.max_rows_to_group_by &&
|
||||
settings.group_by_overflow_mode == OverflowMode::ANY &&
|
||||
settings.totals_mode != TotalsMode::AFTER_HAVING_EXCLUSIVE;
|
||||
|
||||
/// Do I need to immediately finalize the aggregate functions after the aggregation?
|
||||
bool aggregate_final =
|
||||
expressions.need_aggregate &&
|
||||
options.to_stage > QueryProcessingStage::WithMergeableState &&
|
||||
!query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube;
|
||||
|
||||
auto preliminary_sort = [&]()
|
||||
{
|
||||
/** For distributed query processing,
|
||||
@ -1030,7 +1032,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
|
||||
&& !expressions.has_window)
|
||||
{
|
||||
if (expressions.has_order_by)
|
||||
executeOrder(query_plan, query_info.input_order_info);
|
||||
executeOrder(
|
||||
query_plan,
|
||||
query_info.input_order_info ? query_info.input_order_info
|
||||
: (query_info.projection ? query_info.projection->input_order_info : nullptr));
|
||||
|
||||
if (expressions.has_order_by && query.limitLength())
|
||||
executeDistinct(query_plan, false, expressions.selected_columns, true);
|
||||
@ -1064,7 +1069,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
|
||||
|
||||
if (expressions.first_stage)
|
||||
{
|
||||
if (expressions.filter_info)
|
||||
// If there is a storage that supports prewhere, this will always be nullptr
|
||||
// Thus, we don't actually need to check if projection is active.
|
||||
if (!query_info.projection && expressions.filter_info)
|
||||
{
|
||||
auto row_level_security_step = std::make_unique<FilterStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
@ -1151,12 +1158,13 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
|
||||
}
|
||||
}
|
||||
|
||||
if (expressions.hasWhere())
|
||||
if (!query_info.projection && expressions.hasWhere())
|
||||
executeWhere(query_plan, expressions.before_where, expressions.remove_where_filter);
|
||||
|
||||
if (expressions.need_aggregate)
|
||||
{
|
||||
executeAggregation(query_plan, expressions.before_aggregation, aggregate_overflow_row, aggregate_final, query_info.input_order_info);
|
||||
executeAggregation(
|
||||
query_plan, expressions.before_aggregation, aggregate_overflow_row, aggregate_final, query_info.input_order_info);
|
||||
/// We need to reset input order info, so that executeOrder can't use it
|
||||
query_info.input_order_info.reset();
|
||||
}
|
||||
@ -1288,7 +1296,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
|
||||
else if (!expressions.first_stage && !expressions.need_aggregate && !(query.group_by_with_totals && !aggregate_final))
|
||||
executeMergeSorted(query_plan, "for ORDER BY");
|
||||
else /// Otherwise, just sort.
|
||||
executeOrder(query_plan, query_info.input_order_info);
|
||||
executeOrder(
|
||||
query_plan,
|
||||
query_info.input_order_info ? query_info.input_order_info
|
||||
: (query_info.projection ? query_info.projection->input_order_info : nullptr));
|
||||
}
|
||||
|
||||
/** Optimization - if there are several sources and there is LIMIT, then first apply the preliminary LIMIT,
|
||||
@ -1393,13 +1404,59 @@ static StreamLocalLimits getLimitsForStorage(const Settings & settings, const Se
|
||||
return limits;
|
||||
}
|
||||
|
||||
void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info)
|
||||
static void executeMergeAggregatedImpl(
|
||||
QueryPlan & query_plan,
|
||||
bool overflow_row,
|
||||
bool final,
|
||||
bool is_remote_storage,
|
||||
const Settings & settings,
|
||||
const NamesAndTypesList & aggregation_keys,
|
||||
const AggregateDescriptions & aggregates)
|
||||
{
|
||||
const auto & header_before_merge = query_plan.getCurrentDataStream().header;
|
||||
|
||||
ColumnNumbers keys;
|
||||
for (const auto & key : aggregation_keys)
|
||||
keys.push_back(header_before_merge.getPositionByName(key.name));
|
||||
|
||||
/** There are two modes of distributed aggregation.
|
||||
*
|
||||
* 1. In different threads read from the remote servers blocks.
|
||||
* Save all the blocks in the RAM. Merge blocks.
|
||||
* If the aggregation is two-level - parallelize to the number of buckets.
|
||||
*
|
||||
* 2. In one thread, read blocks from different servers in order.
|
||||
* RAM stores only one block from each server.
|
||||
* If the aggregation is a two-level aggregation, we consistently merge the blocks of each next level.
|
||||
*
|
||||
* The second option consumes less memory (up to 256 times less)
|
||||
* in the case of two-level aggregation, which is used for large results after GROUP BY,
|
||||
* but it can work more slowly.
|
||||
*/
|
||||
|
||||
Aggregator::Params params(header_before_merge, keys, aggregates, overflow_row, settings.max_threads);
|
||||
|
||||
auto transform_params = std::make_shared<AggregatingTransformParams>(params, final);
|
||||
|
||||
auto merging_aggregated = std::make_unique<MergingAggregatedStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
std::move(transform_params),
|
||||
settings.distributed_aggregation_memory_efficient && is_remote_storage,
|
||||
settings.max_threads,
|
||||
settings.aggregation_memory_efficient_merge_threads);
|
||||
|
||||
query_plan.addStep(std::move(merging_aggregated));
|
||||
}
|
||||
|
||||
void InterpreterSelectQuery::addEmptySourceToQueryPlan(
|
||||
QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, ContextPtr context_)
|
||||
{
|
||||
Pipe pipe(std::make_shared<NullSource>(source_header));
|
||||
|
||||
if (query_info.prewhere_info)
|
||||
PrewhereInfoPtr prewhere_info_ptr = query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info;
|
||||
if (prewhere_info_ptr)
|
||||
{
|
||||
auto & prewhere_info = *query_info.prewhere_info;
|
||||
auto & prewhere_info = *prewhere_info_ptr;
|
||||
|
||||
if (prewhere_info.alias_actions)
|
||||
{
|
||||
@ -1450,15 +1507,234 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c
|
||||
auto read_from_pipe = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
|
||||
read_from_pipe->setStepDescription("Read from NullSource");
|
||||
query_plan.addStep(std::move(read_from_pipe));
|
||||
|
||||
if (query_info.projection)
|
||||
{
|
||||
if (query_info.projection->before_where)
|
||||
{
|
||||
auto where_step = std::make_unique<FilterStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
query_info.projection->before_where,
|
||||
query_info.projection->where_column_name,
|
||||
query_info.projection->remove_where_filter);
|
||||
|
||||
where_step->setStepDescription("WHERE");
|
||||
query_plan.addStep(std::move(where_step));
|
||||
}
|
||||
|
||||
if (query_info.projection->desc->type == ProjectionDescription::Type::Aggregate)
|
||||
{
|
||||
if (query_info.projection->before_aggregation)
|
||||
{
|
||||
auto expression_before_aggregation
|
||||
= std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), query_info.projection->before_aggregation);
|
||||
expression_before_aggregation->setStepDescription("Before GROUP BY");
|
||||
query_plan.addStep(std::move(expression_before_aggregation));
|
||||
}
|
||||
|
||||
executeMergeAggregatedImpl(
|
||||
query_plan,
|
||||
query_info.projection->aggregate_overflow_row,
|
||||
query_info.projection->aggregate_final,
|
||||
false,
|
||||
context_->getSettingsRef(),
|
||||
query_info.projection->aggregation_keys,
|
||||
query_info.projection->aggregate_descriptions);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void InterpreterSelectQuery::addPrewhereAliasActions()
|
||||
{
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
auto & expressions = analysis_result;
|
||||
if (expressions.filter_info)
|
||||
{
|
||||
if (!expressions.prewhere_info)
|
||||
{
|
||||
const bool does_storage_support_prewhere = !input && !input_pipe && storage && storage->supportsPrewhere();
|
||||
if (does_storage_support_prewhere && settings.optimize_move_to_prewhere)
|
||||
{
|
||||
/// Execute row level filter in prewhere as a part of "move to prewhere" optimization.
|
||||
expressions.prewhere_info = std::make_shared<PrewhereDAGInfo>(
|
||||
std::move(expressions.filter_info->actions),
|
||||
std::move(expressions.filter_info->column_name));
|
||||
expressions.prewhere_info->prewhere_actions->projectInput(false);
|
||||
expressions.prewhere_info->remove_prewhere_column = expressions.filter_info->do_remove_column;
|
||||
expressions.prewhere_info->need_filter = true;
|
||||
expressions.filter_info = nullptr;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Add row level security actions to prewhere.
|
||||
expressions.prewhere_info->row_level_filter_actions = std::move(expressions.filter_info->actions);
|
||||
expressions.prewhere_info->row_level_column_name = std::move(expressions.filter_info->column_name);
|
||||
expressions.prewhere_info->row_level_filter_actions->projectInput(false);
|
||||
expressions.filter_info = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
auto & prewhere_info = analysis_result.prewhere_info;
|
||||
auto & columns_to_remove_after_prewhere = analysis_result.columns_to_remove_after_prewhere;
|
||||
|
||||
/// Detect, if ALIAS columns are required for query execution
|
||||
auto alias_columns_required = false;
|
||||
const ColumnsDescription & storage_columns = metadata_snapshot->getColumns();
|
||||
for (const auto & column_name : required_columns)
|
||||
{
|
||||
auto column_default = storage_columns.getDefault(column_name);
|
||||
if (column_default && column_default->kind == ColumnDefaultKind::Alias)
|
||||
{
|
||||
alias_columns_required = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
/// There are multiple sources of required columns:
|
||||
/// - raw required columns,
|
||||
/// - columns deduced from ALIAS columns,
|
||||
/// - raw required columns from PREWHERE,
|
||||
/// - columns deduced from ALIAS columns from PREWHERE.
|
||||
/// PREWHERE is a special case, since we need to resolve it and pass directly to `IStorage::read()`
|
||||
/// before any other executions.
|
||||
if (alias_columns_required)
|
||||
{
|
||||
NameSet required_columns_from_prewhere; /// Set of all (including ALIAS) required columns for PREWHERE
|
||||
NameSet required_aliases_from_prewhere; /// Set of ALIAS required columns for PREWHERE
|
||||
|
||||
if (prewhere_info)
|
||||
{
|
||||
/// Get some columns directly from PREWHERE expression actions
|
||||
auto prewhere_required_columns = prewhere_info->prewhere_actions->getRequiredColumns().getNames();
|
||||
required_columns_from_prewhere.insert(prewhere_required_columns.begin(), prewhere_required_columns.end());
|
||||
|
||||
if (prewhere_info->row_level_filter_actions)
|
||||
{
|
||||
auto row_level_required_columns = prewhere_info->row_level_filter_actions->getRequiredColumns().getNames();
|
||||
required_columns_from_prewhere.insert(row_level_required_columns.begin(), row_level_required_columns.end());
|
||||
}
|
||||
}
|
||||
|
||||
/// Expression, that contains all raw required columns
|
||||
ASTPtr required_columns_all_expr = std::make_shared<ASTExpressionList>();
|
||||
|
||||
/// Expression, that contains raw required columns for PREWHERE
|
||||
ASTPtr required_columns_from_prewhere_expr = std::make_shared<ASTExpressionList>();
|
||||
|
||||
/// Sort out already known required columns between expressions,
|
||||
/// also populate `required_aliases_from_prewhere`.
|
||||
for (const auto & column : required_columns)
|
||||
{
|
||||
ASTPtr column_expr;
|
||||
const auto column_default = storage_columns.getDefault(column);
|
||||
bool is_alias = column_default && column_default->kind == ColumnDefaultKind::Alias;
|
||||
if (is_alias)
|
||||
{
|
||||
auto column_decl = storage_columns.get(column);
|
||||
column_expr = column_default->expression->clone();
|
||||
// recursive visit for alias to alias
|
||||
replaceAliasColumnsInQuery(
|
||||
column_expr, metadata_snapshot->getColumns(), syntax_analyzer_result->getArrayJoinSourceNameSet(), context);
|
||||
|
||||
column_expr = addTypeConversionToAST(
|
||||
std::move(column_expr), column_decl.type->getName(), metadata_snapshot->getColumns().getAll(), context);
|
||||
column_expr = setAlias(column_expr, column);
|
||||
}
|
||||
else
|
||||
column_expr = std::make_shared<ASTIdentifier>(column);
|
||||
|
||||
if (required_columns_from_prewhere.count(column))
|
||||
{
|
||||
required_columns_from_prewhere_expr->children.emplace_back(std::move(column_expr));
|
||||
|
||||
if (is_alias)
|
||||
required_aliases_from_prewhere.insert(column);
|
||||
}
|
||||
else
|
||||
required_columns_all_expr->children.emplace_back(std::move(column_expr));
|
||||
}
|
||||
|
||||
/// Columns, which we will get after prewhere and filter executions.
|
||||
NamesAndTypesList required_columns_after_prewhere;
|
||||
NameSet required_columns_after_prewhere_set;
|
||||
|
||||
/// Collect required columns from prewhere expression actions.
|
||||
if (prewhere_info)
|
||||
{
|
||||
NameSet columns_to_remove(columns_to_remove_after_prewhere.begin(), columns_to_remove_after_prewhere.end());
|
||||
Block prewhere_actions_result = prewhere_info->prewhere_actions->getResultColumns();
|
||||
|
||||
/// Populate required columns with the columns, added by PREWHERE actions and not removed afterwards.
|
||||
/// XXX: looks hacky that we already know which columns after PREWHERE we won't need for sure.
|
||||
for (const auto & column : prewhere_actions_result)
|
||||
{
|
||||
if (prewhere_info->remove_prewhere_column && column.name == prewhere_info->prewhere_column_name)
|
||||
continue;
|
||||
|
||||
if (columns_to_remove.count(column.name))
|
||||
continue;
|
||||
|
||||
required_columns_all_expr->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
|
||||
required_columns_after_prewhere.emplace_back(column.name, column.type);
|
||||
}
|
||||
|
||||
required_columns_after_prewhere_set
|
||||
= ext::map<NameSet>(required_columns_after_prewhere, [](const auto & it) { return it.name; });
|
||||
}
|
||||
|
||||
auto syntax_result
|
||||
= TreeRewriter(context).analyze(required_columns_all_expr, required_columns_after_prewhere, storage, metadata_snapshot);
|
||||
alias_actions = ExpressionAnalyzer(required_columns_all_expr, syntax_result, context).getActionsDAG(true);
|
||||
|
||||
/// The set of required columns could be added as a result of adding an action to calculate ALIAS.
|
||||
required_columns = alias_actions->getRequiredColumns().getNames();
|
||||
|
||||
/// Do not remove prewhere filter if it is a column which is used as alias.
|
||||
if (prewhere_info && prewhere_info->remove_prewhere_column)
|
||||
if (required_columns.end() != std::find(required_columns.begin(), required_columns.end(), prewhere_info->prewhere_column_name))
|
||||
prewhere_info->remove_prewhere_column = false;
|
||||
|
||||
/// Remove columns which will be added by prewhere.
|
||||
required_columns.erase(
|
||||
std::remove_if(
|
||||
required_columns.begin(),
|
||||
required_columns.end(),
|
||||
[&](const String & name) { return required_columns_after_prewhere_set.count(name) != 0; }),
|
||||
required_columns.end());
|
||||
|
||||
if (prewhere_info)
|
||||
{
|
||||
/// Don't remove columns which are needed to be aliased.
|
||||
for (const auto & name : required_columns)
|
||||
prewhere_info->prewhere_actions->tryRestoreColumn(name);
|
||||
|
||||
auto analyzed_result
|
||||
= TreeRewriter(context).analyze(required_columns_from_prewhere_expr, metadata_snapshot->getColumns().getAllPhysical());
|
||||
prewhere_info->alias_actions
|
||||
= ExpressionAnalyzer(required_columns_from_prewhere_expr, analyzed_result, context).getActionsDAG(true, false);
|
||||
|
||||
/// Add (physical?) columns required by alias actions.
|
||||
auto required_columns_from_alias = prewhere_info->alias_actions->getRequiredColumns();
|
||||
Block prewhere_actions_result = prewhere_info->prewhere_actions->getResultColumns();
|
||||
for (auto & column : required_columns_from_alias)
|
||||
if (!prewhere_actions_result.has(column.name))
|
||||
if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column.name))
|
||||
required_columns.push_back(column.name);
|
||||
|
||||
/// Add physical columns required by prewhere actions.
|
||||
for (const auto & column : required_columns_from_prewhere)
|
||||
if (required_aliases_from_prewhere.count(column) == 0)
|
||||
if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column))
|
||||
required_columns.push_back(column);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan)
|
||||
{
|
||||
auto & query = getSelectQuery();
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
auto & expressions = analysis_result;
|
||||
auto & prewhere_info = expressions.prewhere_info;
|
||||
auto & columns_to_remove_after_prewhere = expressions.columns_to_remove_after_prewhere;
|
||||
|
||||
/// Optimization for trivial query like SELECT count() FROM table.
|
||||
bool optimize_trivial_count =
|
||||
@ -1527,160 +1803,6 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
|
||||
}
|
||||
}
|
||||
|
||||
/// Actions to calculate ALIAS if required.
|
||||
ActionsDAGPtr alias_actions;
|
||||
|
||||
if (storage)
|
||||
{
|
||||
/// Detect, if ALIAS columns are required for query execution
|
||||
auto alias_columns_required = false;
|
||||
const ColumnsDescription & storage_columns = metadata_snapshot->getColumns();
|
||||
for (const auto & column_name : required_columns)
|
||||
{
|
||||
auto column_default = storage_columns.getDefault(column_name);
|
||||
if (column_default && column_default->kind == ColumnDefaultKind::Alias)
|
||||
{
|
||||
alias_columns_required = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
/// There are multiple sources of required columns:
|
||||
/// - raw required columns,
|
||||
/// - columns deduced from ALIAS columns,
|
||||
/// - raw required columns from PREWHERE,
|
||||
/// - columns deduced from ALIAS columns from PREWHERE.
|
||||
/// PREWHERE is a special case, since we need to resolve it and pass directly to `IStorage::read()`
|
||||
/// before any other executions.
|
||||
if (alias_columns_required)
|
||||
{
|
||||
NameSet required_columns_from_prewhere; /// Set of all (including ALIAS) required columns for PREWHERE
|
||||
NameSet required_aliases_from_prewhere; /// Set of ALIAS required columns for PREWHERE
|
||||
|
||||
if (prewhere_info)
|
||||
{
|
||||
/// Get some columns directly from PREWHERE expression actions
|
||||
auto prewhere_required_columns = prewhere_info->prewhere_actions->getRequiredColumns().getNames();
|
||||
required_columns_from_prewhere.insert(prewhere_required_columns.begin(), prewhere_required_columns.end());
|
||||
|
||||
if (prewhere_info->row_level_filter_actions)
|
||||
{
|
||||
auto row_level_required_columns = prewhere_info->row_level_filter_actions->getRequiredColumns().getNames();
|
||||
required_columns_from_prewhere.insert(row_level_required_columns.begin(), row_level_required_columns.end());
|
||||
}
|
||||
}
|
||||
|
||||
/// Expression, that contains all raw required columns
|
||||
ASTPtr required_columns_all_expr = std::make_shared<ASTExpressionList>();
|
||||
|
||||
/// Expression, that contains raw required columns for PREWHERE
|
||||
ASTPtr required_columns_from_prewhere_expr = std::make_shared<ASTExpressionList>();
|
||||
|
||||
/// Sort out already known required columns between expressions,
|
||||
/// also populate `required_aliases_from_prewhere`.
|
||||
for (const auto & column : required_columns)
|
||||
{
|
||||
ASTPtr column_expr;
|
||||
const auto column_default = storage_columns.getDefault(column);
|
||||
bool is_alias = column_default && column_default->kind == ColumnDefaultKind::Alias;
|
||||
if (is_alias)
|
||||
{
|
||||
auto column_decl = storage_columns.get(column);
|
||||
column_expr = column_default->expression->clone();
|
||||
// recursive visit for alias to alias
|
||||
replaceAliasColumnsInQuery(column_expr, metadata_snapshot->getColumns(), syntax_analyzer_result->getArrayJoinSourceNameSet(), context);
|
||||
|
||||
column_expr = addTypeConversionToAST(std::move(column_expr), column_decl.type->getName(), metadata_snapshot->getColumns().getAll(), context);
|
||||
column_expr = setAlias(column_expr, column);
|
||||
}
|
||||
else
|
||||
column_expr = std::make_shared<ASTIdentifier>(column);
|
||||
|
||||
if (required_columns_from_prewhere.count(column))
|
||||
{
|
||||
required_columns_from_prewhere_expr->children.emplace_back(std::move(column_expr));
|
||||
|
||||
if (is_alias)
|
||||
required_aliases_from_prewhere.insert(column);
|
||||
}
|
||||
else
|
||||
required_columns_all_expr->children.emplace_back(std::move(column_expr));
|
||||
}
|
||||
|
||||
/// Columns, which we will get after prewhere and filter executions.
|
||||
NamesAndTypesList required_columns_after_prewhere;
|
||||
NameSet required_columns_after_prewhere_set;
|
||||
|
||||
/// Collect required columns from prewhere expression actions.
|
||||
if (prewhere_info)
|
||||
{
|
||||
NameSet columns_to_remove(columns_to_remove_after_prewhere.begin(), columns_to_remove_after_prewhere.end());
|
||||
Block prewhere_actions_result = prewhere_info->prewhere_actions->getResultColumns();
|
||||
|
||||
/// Populate required columns with the columns, added by PREWHERE actions and not removed afterwards.
|
||||
/// XXX: looks hacky that we already know which columns after PREWHERE we won't need for sure.
|
||||
for (const auto & column : prewhere_actions_result)
|
||||
{
|
||||
if (prewhere_info->remove_prewhere_column && column.name == prewhere_info->prewhere_column_name)
|
||||
continue;
|
||||
|
||||
if (columns_to_remove.count(column.name))
|
||||
continue;
|
||||
|
||||
required_columns_all_expr->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
|
||||
required_columns_after_prewhere.emplace_back(column.name, column.type);
|
||||
}
|
||||
|
||||
required_columns_after_prewhere_set
|
||||
= ext::map<NameSet>(required_columns_after_prewhere, [](const auto & it) { return it.name; });
|
||||
}
|
||||
|
||||
auto syntax_result = TreeRewriter(context).analyze(required_columns_all_expr, required_columns_after_prewhere, storage, metadata_snapshot);
|
||||
alias_actions = ExpressionAnalyzer(required_columns_all_expr, syntax_result, context).getActionsDAG(true);
|
||||
|
||||
/// The set of required columns could be added as a result of adding an action to calculate ALIAS.
|
||||
required_columns = alias_actions->getRequiredColumns().getNames();
|
||||
|
||||
/// Do not remove prewhere filter if it is a column which is used as alias.
|
||||
if (prewhere_info && prewhere_info->remove_prewhere_column)
|
||||
if (required_columns.end()
|
||||
!= std::find(required_columns.begin(), required_columns.end(), prewhere_info->prewhere_column_name))
|
||||
prewhere_info->remove_prewhere_column = false;
|
||||
|
||||
/// Remove columns which will be added by prewhere.
|
||||
required_columns.erase(std::remove_if(required_columns.begin(), required_columns.end(), [&](const String & name)
|
||||
{
|
||||
return required_columns_after_prewhere_set.count(name) != 0;
|
||||
}), required_columns.end());
|
||||
|
||||
if (prewhere_info)
|
||||
{
|
||||
/// Don't remove columns which are needed to be aliased.
|
||||
for (const auto & name : required_columns)
|
||||
prewhere_info->prewhere_actions->tryRestoreColumn(name);
|
||||
|
||||
auto analyzed_result
|
||||
= TreeRewriter(context).analyze(required_columns_from_prewhere_expr, metadata_snapshot->getColumns().getAllPhysical());
|
||||
prewhere_info->alias_actions
|
||||
= ExpressionAnalyzer(required_columns_from_prewhere_expr, analyzed_result, context).getActionsDAG(true, false);
|
||||
|
||||
/// Add (physical?) columns required by alias actions.
|
||||
auto required_columns_from_alias = prewhere_info->alias_actions->getRequiredColumns();
|
||||
Block prewhere_actions_result = prewhere_info->prewhere_actions->getResultColumns();
|
||||
for (auto & column : required_columns_from_alias)
|
||||
if (!prewhere_actions_result.has(column.name))
|
||||
if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column.name))
|
||||
required_columns.push_back(column.name);
|
||||
|
||||
/// Add physical columns required by prewhere actions.
|
||||
for (const auto & column : required_columns_from_prewhere)
|
||||
if (required_aliases_from_prewhere.count(column) == 0)
|
||||
if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column))
|
||||
required_columns.push_back(column);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Limitation on the number of columns to read.
|
||||
/// It's not applied in 'only_analyze' mode, because the query could be analyzed without removal of unnecessary columns.
|
||||
if (!options.only_analyze && settings.max_columns_to_read && required_columns.size() > settings.max_columns_to_read)
|
||||
@ -1771,9 +1893,10 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
|
||||
if (max_streams > 1 && !is_remote)
|
||||
max_streams *= settings.max_streams_to_max_threads_ratio;
|
||||
|
||||
query_info.syntax_analyzer_result = syntax_analyzer_result;
|
||||
// TODO figure out how to make set for projections
|
||||
query_info.sets = query_analyzer->getPreparedSets();
|
||||
auto actions_settings = ExpressionActionsSettings::fromContext(context);
|
||||
auto & prewhere_info = analysis_result.prewhere_info;
|
||||
|
||||
if (prewhere_info)
|
||||
{
|
||||
@ -1795,27 +1918,52 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
|
||||
|
||||
/// Create optimizer with prepared actions.
|
||||
/// Maybe we will need to calc input_order_info later, e.g. while reading from StorageMerge.
|
||||
if (analysis_result.optimize_read_in_order || analysis_result.optimize_aggregation_in_order)
|
||||
if ((analysis_result.optimize_read_in_order || analysis_result.optimize_aggregation_in_order)
|
||||
&& (!query_info.projection || query_info.projection->complete))
|
||||
{
|
||||
if (analysis_result.optimize_read_in_order)
|
||||
query_info.order_optimizer = std::make_shared<ReadInOrderOptimizer>(
|
||||
analysis_result.order_by_elements_actions,
|
||||
getSortDescription(query, context),
|
||||
query_info.syntax_analyzer_result);
|
||||
{
|
||||
if (query_info.projection)
|
||||
{
|
||||
query_info.projection->order_optimizer = std::make_shared<ReadInOrderOptimizer>(
|
||||
// TODO Do we need a projection variant for this field?
|
||||
analysis_result.order_by_elements_actions,
|
||||
getSortDescription(query, context),
|
||||
query_info.syntax_analyzer_result);
|
||||
}
|
||||
else
|
||||
{
|
||||
query_info.order_optimizer = std::make_shared<ReadInOrderOptimizer>(
|
||||
analysis_result.order_by_elements_actions, getSortDescription(query, context), query_info.syntax_analyzer_result);
|
||||
}
|
||||
}
|
||||
else
|
||||
query_info.order_optimizer = std::make_shared<ReadInOrderOptimizer>(
|
||||
analysis_result.group_by_elements_actions,
|
||||
getSortDescriptionFromGroupBy(query),
|
||||
query_info.syntax_analyzer_result);
|
||||
{
|
||||
if (query_info.projection)
|
||||
{
|
||||
query_info.projection->order_optimizer = std::make_shared<ReadInOrderOptimizer>(
|
||||
query_info.projection->group_by_elements_actions,
|
||||
getSortDescriptionFromGroupBy(query),
|
||||
query_info.syntax_analyzer_result);
|
||||
}
|
||||
else
|
||||
{
|
||||
query_info.order_optimizer = std::make_shared<ReadInOrderOptimizer>(
|
||||
analysis_result.group_by_elements_actions, getSortDescriptionFromGroupBy(query), query_info.syntax_analyzer_result);
|
||||
}
|
||||
}
|
||||
|
||||
query_info.input_order_info = query_info.order_optimizer->getInputOrder(metadata_snapshot, context);
|
||||
if (query_info.projection)
|
||||
query_info.projection->input_order_info
|
||||
= query_info.projection->order_optimizer->getInputOrder(query_info.projection->desc->metadata, context);
|
||||
else
|
||||
query_info.input_order_info = query_info.order_optimizer->getInputOrder(metadata_snapshot, context);
|
||||
}
|
||||
|
||||
StreamLocalLimits limits;
|
||||
SizeLimits leaf_limits;
|
||||
std::shared_ptr<const EnabledQuota> quota;
|
||||
|
||||
|
||||
/// Set the limits and quota for reading data, the speed and time of the query.
|
||||
if (!options.ignore_limits)
|
||||
{
|
||||
@ -1834,15 +1982,21 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
|
||||
{
|
||||
auto local_storage_id = storage->getStorageID();
|
||||
context->getQueryContext()->addQueryAccessInfo(
|
||||
backQuoteIfNeed(local_storage_id.getDatabaseName()), local_storage_id.getFullTableName(), required_columns);
|
||||
backQuoteIfNeed(local_storage_id.getDatabaseName()),
|
||||
local_storage_id.getFullTableName(),
|
||||
required_columns,
|
||||
query_info.projection ? query_info.projection->desc->name : "");
|
||||
}
|
||||
|
||||
/// Create step which reads from empty source if storage has no data.
|
||||
if (!query_plan.isInitialized())
|
||||
{
|
||||
auto header = metadata_snapshot->getSampleBlockForColumns(
|
||||
required_columns, storage->getVirtuals(), storage->getStorageID());
|
||||
addEmptySourceToQueryPlan(query_plan, header, query_info);
|
||||
auto header = query_info.projection
|
||||
? query_info.projection->desc->metadata->getSampleBlockForColumns(
|
||||
query_info.projection->required_columns, storage->getVirtuals(), storage->getStorageID())
|
||||
: metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID());
|
||||
|
||||
addEmptySourceToQueryPlan(query_plan, header, query_info, context);
|
||||
}
|
||||
|
||||
/// Extend lifetime of context, table lock, storage. Set limits and quota.
|
||||
@ -1898,6 +2052,9 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
|
||||
expression_before_aggregation->setStepDescription("Before GROUP BY");
|
||||
query_plan.addStep(std::move(expression_before_aggregation));
|
||||
|
||||
if (options.ignore_aggregation)
|
||||
return;
|
||||
|
||||
const auto & header_before_aggregation = query_plan.getCurrentDataStream().header;
|
||||
ColumnNumbers keys;
|
||||
for (const auto & key : query_analyzer->aggregationKeys())
|
||||
@ -1948,44 +2105,23 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
|
||||
query_plan.addStep(std::move(aggregating_step));
|
||||
}
|
||||
|
||||
|
||||
void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final)
|
||||
{
|
||||
const auto & header_before_merge = query_plan.getCurrentDataStream().header;
|
||||
/// If aggregate projection was chosen for table, avoid adding MergeAggregated.
|
||||
/// It is already added by storage (because of performance issues).
|
||||
/// TODO: We should probably add another one processing stage for storage?
|
||||
/// WithMergeableStateAfterAggregation is not ok because, e.g., it skips sorting after aggregation.
|
||||
if (query_info.projection && query_info.projection->desc->type == ProjectionDescription::Type::Aggregate)
|
||||
return;
|
||||
|
||||
ColumnNumbers keys;
|
||||
for (const auto & key : query_analyzer->aggregationKeys())
|
||||
keys.push_back(header_before_merge.getPositionByName(key.name));
|
||||
|
||||
/** There are two modes of distributed aggregation.
|
||||
*
|
||||
* 1. In different threads read from the remote servers blocks.
|
||||
* Save all the blocks in the RAM. Merge blocks.
|
||||
* If the aggregation is two-level - parallelize to the number of buckets.
|
||||
*
|
||||
* 2. In one thread, read blocks from different servers in order.
|
||||
* RAM stores only one block from each server.
|
||||
* If the aggregation is a two-level aggregation, we consistently merge the blocks of each next level.
|
||||
*
|
||||
* The second option consumes less memory (up to 256 times less)
|
||||
* in the case of two-level aggregation, which is used for large results after GROUP BY,
|
||||
* but it can work more slowly.
|
||||
*/
|
||||
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
Aggregator::Params params(header_before_merge, keys, query_analyzer->aggregates(), overflow_row, settings.max_threads);
|
||||
|
||||
auto transform_params = std::make_shared<AggregatingTransformParams>(params, final);
|
||||
|
||||
auto merging_aggregated = std::make_unique<MergingAggregatedStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
std::move(transform_params),
|
||||
settings.distributed_aggregation_memory_efficient && storage && storage->isRemote(),
|
||||
settings.max_threads,
|
||||
settings.aggregation_memory_efficient_merge_threads);
|
||||
|
||||
query_plan.addStep(std::move(merging_aggregated));
|
||||
executeMergeAggregatedImpl(
|
||||
query_plan,
|
||||
overflow_row,
|
||||
final,
|
||||
storage && storage->isRemote(),
|
||||
context->getSettingsRef(),
|
||||
query_analyzer->aggregationKeys(),
|
||||
query_analyzer->aggregates());
|
||||
}
|
||||
|
||||
|
||||
@ -2458,8 +2594,11 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan)
|
||||
|
||||
void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan, SubqueriesForSets & subqueries_for_sets)
|
||||
{
|
||||
if (query_info.input_order_info)
|
||||
executeMergeSorted(query_plan, query_info.input_order_info->order_key_prefix_descr, 0, "before creating sets for subqueries and joins");
|
||||
const auto & input_order_info = query_info.input_order_info
|
||||
? query_info.input_order_info
|
||||
: (query_info.projection ? query_info.projection->input_order_info : nullptr);
|
||||
if (input_order_info)
|
||||
executeMergeSorted(query_plan, input_order_info->order_key_prefix_descr, 0, "before creating sets for subqueries and joins");
|
||||
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
|
@ -15,11 +15,13 @@
|
||||
|
||||
#include <Columns/FilterDescription.h>
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
namespace Poco
|
||||
{
|
||||
class Logger;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct SubqueryForSet;
|
||||
class InterpreterSelectWithUnionQuery;
|
||||
class Context;
|
||||
@ -85,9 +87,20 @@ public:
|
||||
|
||||
virtual void ignoreWithTotals() override;
|
||||
|
||||
ASTPtr getQuery() const { return query_ptr; }
|
||||
|
||||
const SelectQueryInfo & getQueryInfo() const { return query_info; }
|
||||
|
||||
static void addEmptySourceToQueryPlan(QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info);
|
||||
const SelectQueryExpressionAnalyzer * getQueryAnalyzer() const { return query_analyzer.get(); }
|
||||
|
||||
const ExpressionAnalysisResult & getAnalysisResult() const { return analysis_result; }
|
||||
|
||||
const Names & getRequiredColumns() const { return required_columns; }
|
||||
|
||||
bool hasAggregation() const { return query_analyzer->hasAggregation(); }
|
||||
|
||||
static void addEmptySourceToQueryPlan(
|
||||
QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, ContextPtr context_);
|
||||
|
||||
Names getRequiredColumns() { return required_columns; }
|
||||
|
||||
@ -100,10 +113,12 @@ private:
|
||||
const StoragePtr & storage_,
|
||||
const SelectQueryOptions &,
|
||||
const Names & required_result_column_names = {},
|
||||
const StorageMetadataPtr & metadata_snapshot_= nullptr);
|
||||
const StorageMetadataPtr & metadata_snapshot_ = nullptr);
|
||||
|
||||
ASTSelectQuery & getSelectQuery() { return query_ptr->as<ASTSelectQuery &>(); }
|
||||
|
||||
void addPrewhereAliasActions();
|
||||
|
||||
Block getSampleBlockImpl();
|
||||
|
||||
void executeImpl(QueryPlan & query_plan, const BlockInputStreamPtr & prepared_input, std::optional<Pipe> prepared_pipe);
|
||||
@ -112,7 +127,8 @@ private:
|
||||
|
||||
void executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan);
|
||||
void executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter);
|
||||
void executeAggregation(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info);
|
||||
void executeAggregation(
|
||||
QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info);
|
||||
void executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final);
|
||||
void executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool overflow_row, bool final);
|
||||
void executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression);
|
||||
@ -131,7 +147,8 @@ private:
|
||||
void executeDistinct(QueryPlan & query_plan, bool before_order, Names columns, bool pre_distinct);
|
||||
void executeExtremes(QueryPlan & query_plan);
|
||||
void executeSubqueriesInSetsAndJoins(QueryPlan & query_plan, std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
|
||||
void executeMergeSorted(QueryPlan & query_plan, const SortDescription & sort_description, UInt64 limit, const std::string & description);
|
||||
void
|
||||
executeMergeSorted(QueryPlan & query_plan, const SortDescription & sort_description, UInt64 limit, const std::string & description);
|
||||
|
||||
String generateFilterActions(ActionsDAGPtr & actions, const Names & prerequisite_columns = {}) const;
|
||||
|
||||
@ -168,12 +185,15 @@ private:
|
||||
/// Structure of query source (table, subquery, etc).
|
||||
Block source_header;
|
||||
|
||||
/// Actions to calculate ALIAS if required.
|
||||
ActionsDAGPtr alias_actions;
|
||||
|
||||
/// The subquery interpreter, if the subquery
|
||||
std::unique_ptr<InterpreterSelectWithUnionQuery> interpreter_subquery;
|
||||
|
||||
/// Table from where to read data, if not subquery.
|
||||
StoragePtr storage;
|
||||
StorageID table_id = StorageID::createEmpty(); /// Will be initialized if storage is not nullptr
|
||||
StorageID table_id = StorageID::createEmpty(); /// Will be initialized if storage is not nullptr
|
||||
TableLockHolder table_lock;
|
||||
|
||||
/// Used when we read from prepared input, not table or subquery.
|
||||
|
@ -391,6 +391,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
|
||||
const ColumnsDescription & columns_desc = metadata_snapshot->getColumns();
|
||||
const IndicesDescription & indices_desc = metadata_snapshot->getSecondaryIndices();
|
||||
const ProjectionsDescription & projections_desc = metadata_snapshot->getProjections();
|
||||
NamesAndTypesList all_columns = columns_desc.getAllPhysical();
|
||||
|
||||
NameSet updated_columns;
|
||||
@ -402,8 +403,8 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
}
|
||||
}
|
||||
|
||||
/// We need to know which columns affect which MATERIALIZED columns and data skipping indices
|
||||
/// to recalculate them if dependencies are updated.
|
||||
/// We need to know which columns affect which MATERIALIZED columns, data skipping indices
|
||||
/// and projections to recalculate them if dependencies are updated.
|
||||
std::unordered_map<String, Names> column_to_affected_materialized;
|
||||
if (!updated_columns.empty())
|
||||
{
|
||||
@ -424,7 +425,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
validateUpdateColumns(storage, metadata_snapshot, updated_columns, column_to_affected_materialized);
|
||||
}
|
||||
|
||||
/// Columns, that we need to read for calculation of skip indices or TTL expressions.
|
||||
/// Columns, that we need to read for calculation of skip indices, projections or TTL expressions.
|
||||
auto dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns);
|
||||
|
||||
/// First, break a sequence of commands into stages.
|
||||
@ -432,6 +433,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
{
|
||||
if (command.type == MutationCommand::DELETE)
|
||||
{
|
||||
mutation_kind.set(MutationKind::MUTATE_OTHER);
|
||||
if (stages.empty() || !stages.back().column_to_updated.empty())
|
||||
stages.emplace_back(context);
|
||||
|
||||
@ -440,6 +442,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
}
|
||||
else if (command.type == MutationCommand::UPDATE)
|
||||
{
|
||||
mutation_kind.set(MutationKind::MUTATE_OTHER);
|
||||
if (stages.empty() || !stages.back().column_to_updated.empty())
|
||||
stages.emplace_back(context);
|
||||
if (stages.size() == 1) /// First stage only supports filtering and can't update columns.
|
||||
@ -530,6 +533,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
}
|
||||
else if (command.type == MutationCommand::MATERIALIZE_INDEX)
|
||||
{
|
||||
mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION);
|
||||
auto it = std::find_if(
|
||||
std::cbegin(indices_desc), std::end(indices_desc),
|
||||
[&](const IndexDescription & index)
|
||||
@ -544,9 +548,29 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
const auto required_columns = syntax_result->requiredSourceColumns();
|
||||
for (const auto & column : required_columns)
|
||||
dependencies.emplace(column, ColumnDependency::SKIP_INDEX);
|
||||
materialized_indices.emplace(command.index_name);
|
||||
}
|
||||
else if (command.type == MutationCommand::MATERIALIZE_PROJECTION)
|
||||
{
|
||||
mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION);
|
||||
const auto & projection = projections_desc.get(command.projection_name);
|
||||
for (const auto & column : projection.required_columns)
|
||||
dependencies.emplace(column, ColumnDependency::PROJECTION);
|
||||
materialized_projections.emplace(command.projection_name);
|
||||
}
|
||||
else if (command.type == MutationCommand::DROP_INDEX)
|
||||
{
|
||||
mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION);
|
||||
materialized_indices.erase(command.index_name);
|
||||
}
|
||||
else if (command.type == MutationCommand::DROP_PROJECTION)
|
||||
{
|
||||
mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION);
|
||||
materialized_projections.erase(command.projection_name);
|
||||
}
|
||||
else if (command.type == MutationCommand::MATERIALIZE_TTL)
|
||||
{
|
||||
mutation_kind.set(MutationKind::MUTATE_OTHER);
|
||||
if (metadata_snapshot->hasRowsTTL())
|
||||
{
|
||||
for (const auto & column : all_columns)
|
||||
@ -571,11 +595,11 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
dependencies.insert(dependency);
|
||||
}
|
||||
|
||||
/// Recalc only skip indices of columns, that could be updated by TTL.
|
||||
/// Recalc only skip indices and projections of columns which could be updated by TTL.
|
||||
auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns);
|
||||
for (const auto & dependency : new_dependencies)
|
||||
{
|
||||
if (dependency.kind == ColumnDependency::SKIP_INDEX)
|
||||
if (dependency.kind == ColumnDependency::SKIP_INDEX || dependency.kind == ColumnDependency::PROJECTION)
|
||||
dependencies.insert(dependency);
|
||||
}
|
||||
|
||||
@ -589,6 +613,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
}
|
||||
else if (command.type == MutationCommand::READ_COLUMN)
|
||||
{
|
||||
mutation_kind.set(MutationKind::MUTATE_OTHER);
|
||||
if (stages.empty() || !stages.back().column_to_updated.empty())
|
||||
stages.emplace_back(context);
|
||||
if (stages.size() == 1) /// First stage only supports filtering and can't update columns.
|
||||
@ -600,7 +625,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
throw Exception("Unknown mutation command type: " + DB::toString<int>(command.type), ErrorCodes::UNKNOWN_MUTATION_COMMAND);
|
||||
}
|
||||
|
||||
/// We care about affected indices because we also need to rewrite them
|
||||
/// We care about affected indices and projections because we also need to rewrite them
|
||||
/// when one of index columns updated or filtered with delete.
|
||||
/// The same about columns, that are needed for calculation of TTL expressions.
|
||||
if (!dependencies.empty())
|
||||
@ -654,7 +679,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
updated_header = std::make_unique<Block>(pipeline->getHeader());
|
||||
}
|
||||
|
||||
/// Special step to recalculate affected indices and TTL expressions.
|
||||
/// Special step to recalculate affected indices, projections and TTL expressions.
|
||||
stages.emplace_back(context);
|
||||
for (const auto & column : unchanged_columns)
|
||||
stages.back().column_to_updated.emplace(
|
||||
@ -945,4 +970,10 @@ bool MutationsInterpreter::isAffectingAllColumns() const
|
||||
return stages.back().isAffectingAllColumns(storage_columns);
|
||||
}
|
||||
|
||||
void MutationsInterpreter::MutationKind::set(const MutationKindEnum & kind)
|
||||
{
|
||||
if (mutation_kind < kind)
|
||||
mutation_kind = kind;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -59,6 +59,24 @@ public:
|
||||
/// Latest mutation stage affects all columns in storage
|
||||
bool isAffectingAllColumns() const;
|
||||
|
||||
NameSet grabMaterializedIndices() { return std::move(materialized_indices); }
|
||||
|
||||
NameSet grabMaterializedProjections() { return std::move(materialized_projections); }
|
||||
|
||||
struct MutationKind
|
||||
{
|
||||
enum MutationKindEnum
|
||||
{
|
||||
MUTATE_UNKNOWN,
|
||||
MUTATE_INDEX_PROJECTION,
|
||||
MUTATE_OTHER,
|
||||
} mutation_kind = MUTATE_UNKNOWN;
|
||||
|
||||
void set(const MutationKindEnum & kind);
|
||||
};
|
||||
|
||||
MutationKind::MutationKindEnum getMutationKind() const { return mutation_kind.mutation_kind; }
|
||||
|
||||
private:
|
||||
ASTPtr prepare(bool dry_run);
|
||||
|
||||
@ -125,6 +143,11 @@ private:
|
||||
std::unique_ptr<Block> updated_header;
|
||||
std::vector<Stage> stages;
|
||||
bool is_prepared = false; /// Has the sequence of stages been prepared.
|
||||
|
||||
NameSet materialized_indices;
|
||||
NameSet materialized_projections;
|
||||
|
||||
MutationKind mutation_kind; /// Do we meet any index or projection mutation.
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -63,6 +63,8 @@ Block QueryLogElement::createBlock()
|
||||
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())), "tables"},
|
||||
{std::make_shared<DataTypeArray>(
|
||||
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())), "columns"},
|
||||
{std::make_shared<DataTypeArray>(
|
||||
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())), "projections"},
|
||||
{std::make_shared<DataTypeInt32>(), "exception_code"},
|
||||
{std::make_shared<DataTypeString>(), "exception"},
|
||||
{std::make_shared<DataTypeString>(), "stack_trace"},
|
||||
@ -144,6 +146,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const
|
||||
auto & column_databases = typeid_cast<ColumnArray &>(*columns[i++]);
|
||||
auto & column_tables = typeid_cast<ColumnArray &>(*columns[i++]);
|
||||
auto & column_columns = typeid_cast<ColumnArray &>(*columns[i++]);
|
||||
auto & column_projections = typeid_cast<ColumnArray &>(*columns[i++]);
|
||||
|
||||
auto fill_column = [](const std::set<String> & data, ColumnArray & column)
|
||||
{
|
||||
@ -160,6 +163,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const
|
||||
fill_column(query_databases, column_databases);
|
||||
fill_column(query_tables, column_tables);
|
||||
fill_column(query_columns, column_columns);
|
||||
fill_column(query_projections, column_projections);
|
||||
}
|
||||
|
||||
columns[i++]->insert(exception_code);
|
||||
|
@ -57,6 +57,7 @@ struct QueryLogElement
|
||||
std::set<String> query_databases;
|
||||
std::set<String> query_tables;
|
||||
std::set<String> query_columns;
|
||||
std::set<String> query_projections;
|
||||
|
||||
std::unordered_set<String> used_aggregate_functions;
|
||||
std::unordered_set<String> used_aggregate_function_combinators;
|
||||
|
@ -248,6 +248,8 @@ void QueryNormalizer::visit(ASTPtr & ast, Data & data)
|
||||
|
||||
current_asts.erase(initial_ast.get());
|
||||
current_asts.erase(ast.get());
|
||||
if (data.ignore_alias && !ast->tryGetAlias().empty())
|
||||
ast->setAlias("");
|
||||
finished_asts[initial_ast] = ast;
|
||||
|
||||
/// @note can not place it in CheckASTDepth dtor cause of exception.
|
||||
|
@ -48,12 +48,14 @@ public:
|
||||
MapOfASTs finished_asts; /// already processed vertices (and by what they replaced)
|
||||
SetOfASTs current_asts; /// vertices in the current call stack of this method
|
||||
std::string current_alias; /// the alias referencing to the ancestor of ast (the deepest ancestor with aliases)
|
||||
bool ignore_alias; /// normalize query without any aliases
|
||||
|
||||
Data(const Aliases & aliases_, const NameSet & source_columns_set_, ExtractedSettings && settings_)
|
||||
Data(const Aliases & aliases_, const NameSet & source_columns_set_, bool ignore_alias_, ExtractedSettings && settings_)
|
||||
: aliases(aliases_)
|
||||
, source_columns_set(source_columns_set_)
|
||||
, settings(settings_)
|
||||
, level(0)
|
||||
, ignore_alias(ignore_alias_)
|
||||
{}
|
||||
};
|
||||
|
||||
|
@ -32,6 +32,14 @@ struct SelectQueryOptions
|
||||
bool remove_duplicates = false;
|
||||
bool ignore_quota = false;
|
||||
bool ignore_limits = false;
|
||||
/// This is a temporary flag to avoid adding aggregating step. Used for projections.
|
||||
/// TODO: we need more stages for InterpreterSelectQuery
|
||||
bool ignore_aggregation = false;
|
||||
/// This flag is needed to analyze query ignoring table projections.
|
||||
/// It is needed because we build another one InterpreterSelectQuery while analyzing projections.
|
||||
/// It helps to avoid infinite recursion.
|
||||
bool ignore_projections = false;
|
||||
bool ignore_alias = false;
|
||||
bool is_internal = false;
|
||||
bool is_subquery = false; // non-subquery can also have subquery_depth > 0, e.g. insert select
|
||||
|
||||
@ -83,6 +91,24 @@ struct SelectQueryOptions
|
||||
return *this;
|
||||
}
|
||||
|
||||
SelectQueryOptions & ignoreProjections(bool value = true)
|
||||
{
|
||||
ignore_projections = value;
|
||||
return *this;
|
||||
}
|
||||
|
||||
SelectQueryOptions & ignoreAggregation(bool value = true)
|
||||
{
|
||||
ignore_aggregation = value;
|
||||
return *this;
|
||||
}
|
||||
|
||||
SelectQueryOptions & ignoreAlias(bool value = true)
|
||||
{
|
||||
ignore_alias = value;
|
||||
return *this;
|
||||
}
|
||||
|
||||
SelectQueryOptions & setInternal(bool value = false)
|
||||
{
|
||||
is_internal = value;
|
||||
|
@ -913,7 +913,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
|
||||
all_source_columns_set.insert(name);
|
||||
}
|
||||
|
||||
normalize(query, result.aliases, all_source_columns_set, settings);
|
||||
normalize(query, result.aliases, all_source_columns_set, select_options.ignore_alias, settings);
|
||||
|
||||
/// Remove unneeded columns according to 'required_result_columns'.
|
||||
/// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside.
|
||||
@ -968,7 +968,7 @@ TreeRewriterResultPtr TreeRewriter::analyze(
|
||||
|
||||
TreeRewriterResult result(source_columns, storage, metadata_snapshot, false);
|
||||
|
||||
normalize(query, result.aliases, result.source_columns_set, settings);
|
||||
normalize(query, result.aliases, result.source_columns_set, false, settings);
|
||||
|
||||
/// Executing scalar subqueries. Column defaults could be a scalar subquery.
|
||||
executeScalarSubqueries(query, getContext(), 0, result.scalars, false);
|
||||
@ -993,7 +993,8 @@ TreeRewriterResultPtr TreeRewriter::analyze(
|
||||
return std::make_shared<const TreeRewriterResult>(result);
|
||||
}
|
||||
|
||||
void TreeRewriter::normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, const Settings & settings)
|
||||
void TreeRewriter::normalize(
|
||||
ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings)
|
||||
{
|
||||
CustomizeCountDistinctVisitor::Data data_count_distinct{settings.count_distinct_implementation};
|
||||
CustomizeCountDistinctVisitor(data_count_distinct).visit(query);
|
||||
@ -1053,7 +1054,7 @@ void TreeRewriter::normalize(ASTPtr & query, Aliases & aliases, const NameSet &
|
||||
FunctionNameNormalizer().visit(query.get());
|
||||
|
||||
/// Common subexpression elimination. Rewrite rules.
|
||||
QueryNormalizer::Data normalizer_data(aliases, source_columns_set, settings);
|
||||
QueryNormalizer::Data normalizer_data(aliases, source_columns_set, ignore_alias, settings);
|
||||
QueryNormalizer(normalizer_data).visit(query);
|
||||
}
|
||||
|
||||
|
@ -115,7 +115,7 @@ public:
|
||||
std::shared_ptr<TableJoin> table_join = {}) const;
|
||||
|
||||
private:
|
||||
static void normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, const Settings & settings);
|
||||
static void normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -656,6 +656,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
elem.query_databases = info.databases;
|
||||
elem.query_tables = info.tables;
|
||||
elem.query_columns = info.columns;
|
||||
elem.query_projections = info.projections;
|
||||
}
|
||||
|
||||
interpreter->extendQueryLogElem(elem, ast, context, query_database, query_table);
|
||||
|
@ -20,6 +20,6 @@ TEST(QueryNormalizer, SimpleCycleAlias)
|
||||
aliases["b"] = parseQuery(parser, "a as b", 0, 0)->children[0];
|
||||
|
||||
Settings settings;
|
||||
QueryNormalizer::Data normalizer_data(aliases, {}, settings);
|
||||
QueryNormalizer::Data normalizer_data(aliases, {}, false, settings);
|
||||
EXPECT_THROW(QueryNormalizer(normalizer_data).visit(ast), Exception);
|
||||
}
|
||||
|
@ -177,6 +177,41 @@ void ASTAlterCommand::formatImpl(
|
||||
<< "DROP CONSTRAINT " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
constraint->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::ADD_PROJECTION)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD PROJECTION " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
projection_decl->formatImpl(settings, state, frame);
|
||||
|
||||
if (first)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " FIRST " << (settings.hilite ? hilite_none : "");
|
||||
else if (projection)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " AFTER " << (settings.hilite ? hilite_none : "");
|
||||
projection->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
else if (type == ASTAlterCommand::DROP_PROJECTION)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str
|
||||
<< (clear_projection ? "CLEAR " : "DROP ") << "PROJECTION " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
projection->formatImpl(settings, state, frame);
|
||||
if (partition)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str<< " IN PARTITION " << (settings.hilite ? hilite_none : "");
|
||||
partition->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
else if (type == ASTAlterCommand::MATERIALIZE_PROJECTION)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str
|
||||
<< "MATERIALIZE PROJECTION " << (settings.hilite ? hilite_none : "");
|
||||
projection->formatImpl(settings, state, frame);
|
||||
if (partition)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str<< " IN PARTITION " << (settings.hilite ? hilite_none : "");
|
||||
partition->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
else if (type == ASTAlterCommand::DROP_PARTITION)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str
|
||||
|
@ -46,6 +46,10 @@ public:
|
||||
ADD_CONSTRAINT,
|
||||
DROP_CONSTRAINT,
|
||||
|
||||
ADD_PROJECTION,
|
||||
DROP_PROJECTION,
|
||||
MATERIALIZE_PROJECTION,
|
||||
|
||||
DROP_PARTITION,
|
||||
DROP_DETACHED_PARTITION,
|
||||
ATTACH_PARTITION,
|
||||
@ -106,6 +110,17 @@ public:
|
||||
*/
|
||||
ASTPtr constraint;
|
||||
|
||||
/** The ADD PROJECTION query stores the ProjectionDeclaration there.
|
||||
*/
|
||||
ASTPtr projection_decl;
|
||||
|
||||
/** The ADD PROJECTION query stores the name of the projection following AFTER.
|
||||
* The DROP PROJECTION query stores the name for deletion.
|
||||
* The MATERIALIZE PROJECTION query stores the name of the projection to materialize.
|
||||
* The CLEAR PROJECTION query stores the name of the projection to clear.
|
||||
*/
|
||||
ASTPtr projection;
|
||||
|
||||
/** Used in DROP PARTITION, ATTACH PARTITION FROM, UPDATE, DELETE queries.
|
||||
* The value or ID of the partition is stored here.
|
||||
*/
|
||||
@ -141,6 +156,8 @@ public:
|
||||
|
||||
bool clear_index = false; /// for CLEAR INDEX (do not drop index from metadata)
|
||||
|
||||
bool clear_projection = false; /// for CLEAR PROJECTION (do not drop projection from metadata)
|
||||
|
||||
bool if_not_exists = false; /// option for ADD_COLUMN
|
||||
|
||||
bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN
|
||||
|
@ -125,6 +125,8 @@ ASTPtr ASTColumns::clone() const
|
||||
res->set(res->indices, indices->clone());
|
||||
if (constraints)
|
||||
res->set(res->constraints, constraints->clone());
|
||||
if (projections)
|
||||
res->set(res->projections, projections->clone());
|
||||
if (primary_key)
|
||||
res->set(res->primary_key, primary_key->clone());
|
||||
|
||||
@ -165,6 +167,16 @@ void ASTColumns::formatImpl(const FormatSettings & s, FormatState & state, Forma
|
||||
list.children.push_back(elem);
|
||||
}
|
||||
}
|
||||
if (projections)
|
||||
{
|
||||
for (const auto & projection : projections->children)
|
||||
{
|
||||
auto elem = std::make_shared<ASTColumnsElement>();
|
||||
elem->prefix = "PROJECTION";
|
||||
elem->set(elem->elem, projection->clone());
|
||||
list.children.push_back(elem);
|
||||
}
|
||||
}
|
||||
|
||||
if (!list.children.empty())
|
||||
{
|
||||
|
@ -41,6 +41,7 @@ public:
|
||||
ASTExpressionList * columns = nullptr;
|
||||
ASTExpressionList * indices = nullptr;
|
||||
ASTExpressionList * constraints = nullptr;
|
||||
ASTExpressionList * projections = nullptr;
|
||||
IAST * primary_key = nullptr;
|
||||
|
||||
String getID(char) const override { return "Columns definition"; }
|
||||
|
28
src/Parsers/ASTProjectionDeclaration.cpp
Normal file
28
src/Parsers/ASTProjectionDeclaration.cpp
Normal file
@ -0,0 +1,28 @@
|
||||
#include <Parsers/ASTProjectionDeclaration.h>
|
||||
#include <Common/quoteString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
ASTPtr ASTProjectionDeclaration::clone() const
|
||||
{
|
||||
auto clone = std::make_shared<ASTProjectionDeclaration>(*this);
|
||||
clone->cloneChildren();
|
||||
return clone;
|
||||
}
|
||||
|
||||
|
||||
void ASTProjectionDeclaration::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
settings.ostr << backQuoteIfNeed(name);
|
||||
std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' ');
|
||||
std::string nl_or_nothing = settings.one_line ? "" : "\n";
|
||||
settings.ostr << nl_or_nothing << indent_str << "(" << nl_or_nothing;
|
||||
FormatStateStacked frame_nested = frame;
|
||||
frame_nested.need_parens = false;
|
||||
++frame_nested.indent;
|
||||
query->formatImpl(settings, state, frame_nested);
|
||||
settings.ostr << nl_or_nothing << indent_str << ")";
|
||||
}
|
||||
|
||||
}
|
24
src/Parsers/ASTProjectionDeclaration.h
Normal file
24
src/Parsers/ASTProjectionDeclaration.h
Normal file
@ -0,0 +1,24 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/** name (subquery)
|
||||
*/
|
||||
class ASTProjectionDeclaration : public IAST
|
||||
{
|
||||
public:
|
||||
String name;
|
||||
ASTPtr query;
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String getID(char) const override { return "Projection"; }
|
||||
|
||||
ASTPtr clone() const override;
|
||||
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
|
||||
};
|
||||
|
||||
}
|
142
src/Parsers/ASTProjectionSelectQuery.cpp
Normal file
142
src/Parsers/ASTProjectionSelectQuery.cpp
Normal file
@ -0,0 +1,142 @@
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTOrderByElement.h>
|
||||
#include <Parsers/ASTProjectionSelectQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
ASTPtr ASTProjectionSelectQuery::clone() const
|
||||
{
|
||||
auto res = std::make_shared<ASTProjectionSelectQuery>(*this);
|
||||
res->children.clear();
|
||||
res->positions.clear();
|
||||
|
||||
#define CLONE(expr) res->setExpression(expr, getExpression(expr, true))
|
||||
|
||||
/** NOTE Members must clone exactly in the same order,
|
||||
* in which they were inserted into `children` in ParserSelectQuery.
|
||||
* This is important because of the children's names the identifier (getTreeHash) is compiled,
|
||||
* which can be used for column identifiers in the case of subqueries in the IN statement.
|
||||
* For distributed query processing, in case one of the servers is localhost and the other one is not,
|
||||
* localhost query is executed within the process and is cloned,
|
||||
* and the request is sent to the remote server in text form via TCP.
|
||||
* And if the cloning order does not match the parsing order,
|
||||
* then different servers will get different identifiers.
|
||||
*/
|
||||
CLONE(Expression::WITH);
|
||||
CLONE(Expression::SELECT);
|
||||
CLONE(Expression::WHERE);
|
||||
CLONE(Expression::GROUP_BY);
|
||||
CLONE(Expression::ORDER_BY);
|
||||
|
||||
#undef CLONE
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void ASTProjectionSelectQuery::updateTreeHashImpl(SipHash & hash_state) const
|
||||
{
|
||||
hash_state.update(distinct);
|
||||
IAST::updateTreeHashImpl(hash_state);
|
||||
}
|
||||
|
||||
|
||||
void ASTProjectionSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
frame.current_select = this;
|
||||
frame.need_parens = false;
|
||||
std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' ');
|
||||
|
||||
if (with())
|
||||
{
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "WITH " << (s.hilite ? hilite_none : "");
|
||||
s.one_line ? with()->formatImpl(s, state, frame) : with()->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
|
||||
s.ostr << s.nl_or_ws;
|
||||
}
|
||||
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << indent_str << "SELECT " << (distinct ? "DISTINCT " : "") << (s.hilite ? hilite_none : "");
|
||||
|
||||
s.one_line ? select()->formatImpl(s, state, frame) : select()->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
|
||||
|
||||
if (where())
|
||||
{
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "WHERE " << (s.hilite ? hilite_none : "");
|
||||
where()->formatImpl(s, state, frame);
|
||||
}
|
||||
|
||||
if (groupBy())
|
||||
{
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "GROUP BY " << (s.hilite ? hilite_none : "");
|
||||
s.one_line ? groupBy()->formatImpl(s, state, frame) : groupBy()->as<ASTExpressionList &>().formatImplMultiline(s, state, frame);
|
||||
}
|
||||
|
||||
if (orderBy())
|
||||
{
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "ORDER BY " << (s.hilite ? hilite_none : "");
|
||||
orderBy()->formatImpl(s, state, frame);
|
||||
}
|
||||
}
|
||||
|
||||
void ASTProjectionSelectQuery::setExpression(Expression expr, ASTPtr && ast)
|
||||
{
|
||||
if (ast)
|
||||
{
|
||||
auto it = positions.find(expr);
|
||||
if (it == positions.end())
|
||||
{
|
||||
positions[expr] = children.size();
|
||||
children.emplace_back(ast);
|
||||
}
|
||||
else
|
||||
children[it->second] = ast;
|
||||
}
|
||||
else if (positions.count(expr))
|
||||
{
|
||||
size_t pos = positions[expr];
|
||||
children.erase(children.begin() + pos);
|
||||
positions.erase(expr);
|
||||
for (auto & pr : positions)
|
||||
if (pr.second > pos)
|
||||
--pr.second;
|
||||
}
|
||||
}
|
||||
|
||||
ASTPtr & ASTProjectionSelectQuery::getExpression(Expression expr)
|
||||
{
|
||||
if (!positions.count(expr))
|
||||
throw Exception("Get expression before set", ErrorCodes::LOGICAL_ERROR);
|
||||
return children[positions[expr]];
|
||||
}
|
||||
|
||||
ASTPtr ASTProjectionSelectQuery::cloneToASTSelect() const
|
||||
{
|
||||
auto select_query = std::make_shared<ASTSelectQuery>();
|
||||
ASTPtr node = select_query;
|
||||
if (with())
|
||||
select_query->setExpression(ASTSelectQuery::Expression::WITH, with()->clone());
|
||||
if (select())
|
||||
select_query->setExpression(ASTSelectQuery::Expression::SELECT, select()->clone());
|
||||
if (where())
|
||||
select_query->setExpression(ASTSelectQuery::Expression::WHERE, where()->clone());
|
||||
if (groupBy())
|
||||
select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, groupBy()->clone());
|
||||
// Get rid of orderBy. It's used for projection definition only
|
||||
if (orderBy())
|
||||
select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, orderBy()->clone());
|
||||
return node;
|
||||
}
|
||||
|
||||
}
|
63
src/Parsers/ASTProjectionSelectQuery.h
Normal file
63
src/Parsers/ASTProjectionSelectQuery.h
Normal file
@ -0,0 +1,63 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Names.h>
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/** PROJECTION SELECT query
|
||||
*/
|
||||
class ASTProjectionSelectQuery : public IAST
|
||||
{
|
||||
public:
|
||||
enum class Expression : uint8_t
|
||||
{
|
||||
WITH,
|
||||
SELECT,
|
||||
WHERE,
|
||||
GROUP_BY,
|
||||
ORDER_BY,
|
||||
};
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String getID(char) const override { return "ProjectionSelectQuery"; }
|
||||
|
||||
ASTPtr clone() const override;
|
||||
|
||||
bool distinct = false;
|
||||
|
||||
ASTPtr & refSelect() { return getExpression(Expression::SELECT); }
|
||||
ASTPtr & refWhere() { return getExpression(Expression::WHERE); }
|
||||
|
||||
const ASTPtr with() const { return getExpression(Expression::WITH); }
|
||||
const ASTPtr select() const { return getExpression(Expression::SELECT); }
|
||||
const ASTPtr where() const { return getExpression(Expression::WHERE); }
|
||||
const ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); }
|
||||
const ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); }
|
||||
|
||||
/// Set/Reset/Remove expression.
|
||||
void setExpression(Expression expr, ASTPtr && ast);
|
||||
|
||||
ASTPtr getExpression(Expression expr, bool clone = false) const
|
||||
{
|
||||
auto it = positions.find(expr);
|
||||
if (it != positions.end())
|
||||
return clone ? children[it->second]->clone() : children[it->second];
|
||||
return {};
|
||||
}
|
||||
|
||||
void updateTreeHashImpl(SipHash & hash_state) const override;
|
||||
|
||||
ASTPtr cloneToASTSelect() const;
|
||||
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
|
||||
private:
|
||||
std::unordered_map<Expression, size_t> positions;
|
||||
|
||||
ASTPtr & getExpression(Expression expr);
|
||||
};
|
||||
|
||||
}
|
@ -43,6 +43,11 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKeyword s_add_constraint("ADD CONSTRAINT");
|
||||
ParserKeyword s_drop_constraint("DROP CONSTRAINT");
|
||||
|
||||
ParserKeyword s_add_projection("ADD PROJECTION");
|
||||
ParserKeyword s_drop_projection("DROP PROJECTION");
|
||||
ParserKeyword s_clear_projection("CLEAR PROJECTION");
|
||||
ParserKeyword s_materialize_projection("MATERIALIZE PROJECTION");
|
||||
|
||||
ParserKeyword s_add("ADD");
|
||||
ParserKeyword s_drop("DROP");
|
||||
ParserKeyword s_suspend("SUSPEND");
|
||||
@ -101,6 +106,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserCompoundColumnDeclaration parser_col_decl;
|
||||
ParserIndexDeclaration parser_idx_decl;
|
||||
ParserConstraintDeclaration parser_constraint_decl;
|
||||
ParserProjectionDeclaration parser_projection_decl;
|
||||
ParserCompoundColumnDeclaration parser_modify_col_decl(false, false, true);
|
||||
ParserPartition parser_partition;
|
||||
ParserExpression parser_exp_elem;
|
||||
@ -255,10 +261,11 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
command->clear_index = true;
|
||||
command->detach = false;
|
||||
|
||||
if (!s_in_partition.ignore(pos, expected))
|
||||
return false;
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
return false;
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else if (s_materialize_index.ignore(pos, expected))
|
||||
{
|
||||
@ -277,6 +284,70 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else if (s_add_projection.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
command->if_not_exists = true;
|
||||
|
||||
if (!parser_projection_decl.parse(pos, command->projection_decl, expected))
|
||||
return false;
|
||||
|
||||
if (s_first.ignore(pos, expected))
|
||||
command->first = true;
|
||||
else if (s_after.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_name.parse(pos, command->projection, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
command->type = ASTAlterCommand::ADD_PROJECTION;
|
||||
}
|
||||
else if (s_drop_projection.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->projection, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_PROJECTION;
|
||||
command->detach = false;
|
||||
}
|
||||
else if (s_clear_projection.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->projection, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_PROJECTION;
|
||||
command->clear_projection = true;
|
||||
command->detach = false;
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else if (s_materialize_projection.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->projection, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::MATERIALIZE_PROJECTION;
|
||||
command->detach = false;
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else if (s_move_part.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_string_literal.parse(pos, command->partition, expected))
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
#include <Parsers/ASTProjectionDeclaration.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
@ -13,6 +14,7 @@
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
#include <Parsers/ParserDictionary.h>
|
||||
#include <Parsers/ParserDictionaryAttributeDeclaration.h>
|
||||
#include <Parsers/ParserProjectionSelectQuery.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
|
||||
@ -152,14 +154,47 @@ bool ParserConstraintDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
}
|
||||
|
||||
|
||||
bool ParserProjectionDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserIdentifier name_p;
|
||||
ParserProjectionSelectQuery query_p;
|
||||
ParserToken s_lparen(TokenType::OpeningRoundBracket);
|
||||
ParserToken s_rparen(TokenType::ClosingRoundBracket);
|
||||
ASTPtr name;
|
||||
ASTPtr query;
|
||||
|
||||
if (!name_p.parse(pos, name, expected))
|
||||
return false;
|
||||
|
||||
if (!s_lparen.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!query_p.parse(pos, query, expected))
|
||||
return false;
|
||||
|
||||
if (!s_rparen.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
auto projection = std::make_shared<ASTProjectionDeclaration>();
|
||||
projection->name = name->as<ASTIdentifier &>().name();
|
||||
projection->query = query;
|
||||
projection->children.emplace_back(projection->query);
|
||||
node = projection;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool ParserTablePropertyDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKeyword s_index("INDEX");
|
||||
ParserKeyword s_constraint("CONSTRAINT");
|
||||
ParserKeyword s_projection("PROJECTION");
|
||||
ParserKeyword s_primary_key("PRIMARY KEY");
|
||||
|
||||
ParserIndexDeclaration index_p;
|
||||
ParserConstraintDeclaration constraint_p;
|
||||
ParserProjectionDeclaration projection_p;
|
||||
ParserColumnDeclaration column_p{true, true};
|
||||
ParserExpression primary_key_p;
|
||||
|
||||
@ -175,6 +210,11 @@ bool ParserTablePropertyDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expecte
|
||||
if (!constraint_p.parse(pos, new_node, expected))
|
||||
return false;
|
||||
}
|
||||
else if (s_projection.ignore(pos, expected))
|
||||
{
|
||||
if (!projection_p.parse(pos, new_node, expected))
|
||||
return false;
|
||||
}
|
||||
else if (s_primary_key.ignore(pos, expected))
|
||||
{
|
||||
if (!primary_key_p.parse(pos, new_node, expected))
|
||||
@ -202,6 +242,12 @@ bool ParserConstraintDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expect
|
||||
.parse(pos, node, expected);
|
||||
}
|
||||
|
||||
bool ParserProjectionDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
return ParserList(std::make_unique<ParserProjectionDeclaration>(), std::make_unique<ParserToken>(TokenType::Comma), false)
|
||||
.parse(pos, node, expected);
|
||||
}
|
||||
|
||||
bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ASTPtr list;
|
||||
@ -214,6 +260,7 @@ bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
ASTPtr columns = std::make_shared<ASTExpressionList>();
|
||||
ASTPtr indices = std::make_shared<ASTExpressionList>();
|
||||
ASTPtr constraints = std::make_shared<ASTExpressionList>();
|
||||
ASTPtr projections = std::make_shared<ASTExpressionList>();
|
||||
ASTPtr primary_key;
|
||||
|
||||
for (const auto & elem : list->children)
|
||||
@ -224,6 +271,8 @@ bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
indices->children.push_back(elem);
|
||||
else if (elem->as<ASTConstraintDeclaration>())
|
||||
constraints->children.push_back(elem);
|
||||
else if (elem->as<ASTProjectionDeclaration>())
|
||||
projections->children.push_back(elem);
|
||||
else if (elem->as<ASTIdentifier>() || elem->as<ASTFunction>())
|
||||
{
|
||||
if (primary_key)
|
||||
@ -245,6 +294,8 @@ bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
res->set(res->indices, indices);
|
||||
if (!constraints->children.empty())
|
||||
res->set(res->constraints, constraints);
|
||||
if (!projections->children.empty())
|
||||
res->set(res->projections, projections);
|
||||
if (primary_key)
|
||||
res->set(res->primary_key, primary_key);
|
||||
|
||||
|
@ -284,6 +284,13 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
class ParserProjectionDeclaration : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "projection declaration"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
class ParserTablePropertyDeclaration : public IParserBase
|
||||
{
|
||||
protected:
|
||||
@ -306,6 +313,13 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
class ParserProjectionDeclarationList : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "projection declaration list"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
class ParserTablePropertiesDeclarationList : public IParserBase
|
||||
{
|
||||
|
101
src/Parsers/ParserProjectionSelectQuery.cpp
Normal file
101
src/Parsers/ParserProjectionSelectQuery.cpp
Normal file
@ -0,0 +1,101 @@
|
||||
#include <memory>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTProjectionSelectQuery.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/ParserProjectionSelectQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
bool ParserProjectionSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
auto select_query = std::make_shared<ASTProjectionSelectQuery>();
|
||||
node = select_query;
|
||||
|
||||
ParserKeyword s_with("WITH");
|
||||
ParserKeyword s_select("SELECT");
|
||||
ParserKeyword s_distinct("DISTINCT");
|
||||
ParserKeyword s_where("WHERE");
|
||||
ParserKeyword s_group_by("GROUP BY");
|
||||
ParserKeyword s_order_by("ORDER BY");
|
||||
|
||||
ParserNotEmptyExpressionList exp_list(false);
|
||||
ParserNotEmptyExpressionList exp_list_for_with_clause(false);
|
||||
ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword.
|
||||
ParserExpressionWithOptionalAlias exp_elem(false);
|
||||
ParserExpression order_expression_p;
|
||||
|
||||
ASTPtr with_expression_list;
|
||||
ASTPtr select_expression_list;
|
||||
ASTPtr where_expression;
|
||||
ASTPtr group_expression_list;
|
||||
ASTPtr order_expression;
|
||||
|
||||
/// WITH expr list
|
||||
{
|
||||
if (s_with.ignore(pos, expected))
|
||||
{
|
||||
if (!exp_list_for_with_clause.parse(pos, with_expression_list, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/// SELECT [DISTINCT] [TOP N [WITH TIES]] expr list
|
||||
{
|
||||
if (!s_select.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (s_distinct.ignore(pos, expected))
|
||||
select_query->distinct = true;
|
||||
|
||||
if (!exp_list_for_select_clause.parse(pos, select_expression_list, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
// TODO: wait for condition normalizer to land
|
||||
/// WHERE expr
|
||||
// if (s_where.ignore(pos, expected))
|
||||
// {
|
||||
// if (!exp_elem.parse(pos, where_expression, expected))
|
||||
// return false;
|
||||
// }
|
||||
|
||||
// If group by is specified, AggregatingMergeTree engine is used, and the group by keys are implied to be order by keys
|
||||
if (s_group_by.ignore(pos, expected))
|
||||
{
|
||||
if (!ParserList(std::make_unique<ParserExpression>(), std::make_unique<ParserToken>(TokenType::Comma))
|
||||
.parse(pos, group_expression_list, expected))
|
||||
return false;
|
||||
}
|
||||
if (s_order_by.ignore(pos, expected))
|
||||
{
|
||||
ASTPtr expr_list;
|
||||
if (!ParserList(std::make_unique<ParserExpression>(), std::make_unique<ParserToken>(TokenType::Comma)).parse(pos, expr_list, expected))
|
||||
return false;
|
||||
|
||||
if (expr_list->children.size() == 1)
|
||||
{
|
||||
order_expression = expr_list->children.front();
|
||||
}
|
||||
else
|
||||
{
|
||||
auto function_node = std::make_shared<ASTFunction>();
|
||||
function_node->name = "tuple";
|
||||
function_node->arguments = expr_list;
|
||||
function_node->children.push_back(expr_list);
|
||||
order_expression = function_node;
|
||||
}
|
||||
}
|
||||
|
||||
select_query->setExpression(ASTProjectionSelectQuery::Expression::WITH, std::move(with_expression_list));
|
||||
select_query->setExpression(ASTProjectionSelectQuery::Expression::SELECT, std::move(select_expression_list));
|
||||
// select_query->setExpression(ASTProjectionSelectQuery::Expression::WHERE, std::move(where_expression));
|
||||
select_query->setExpression(ASTProjectionSelectQuery::Expression::GROUP_BY, std::move(group_expression_list));
|
||||
select_query->setExpression(ASTProjectionSelectQuery::Expression::ORDER_BY, std::move(order_expression));
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
17
src/Parsers/ParserProjectionSelectQuery.h
Normal file
17
src/Parsers/ParserProjectionSelectQuery.h
Normal file
@ -0,0 +1,17 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
class ParserProjectionSelectQuery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "PROJECTION SELECT query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
@ -38,6 +38,8 @@ SRCS(
|
||||
ASTOptimizeQuery.cpp
|
||||
ASTOrderByElement.cpp
|
||||
ASTPartition.cpp
|
||||
ASTProjectionDeclaration.cpp
|
||||
ASTProjectionSelectQuery.cpp
|
||||
ASTQualifiedAsterisk.cpp
|
||||
ASTQueryParameter.cpp
|
||||
ASTQueryWithOnCluster.cpp
|
||||
@ -105,6 +107,7 @@ SRCS(
|
||||
ParserKillQueryQuery.cpp
|
||||
ParserOptimizeQuery.cpp
|
||||
ParserPartition.cpp
|
||||
ParserProjectionSelectQuery.cpp
|
||||
ParserQuery.cpp
|
||||
ParserQueryWithOutput.cpp
|
||||
ParserRenameQuery.cpp
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <Processors/ISource.h>
|
||||
#include <Processors/Pipe.h>
|
||||
#include <Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h>
|
||||
|
||||
#include <DataStreams/materializeBlock.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
@ -522,8 +522,18 @@ void AggregatingTransform::consume(Chunk chunk)
|
||||
src_rows += num_rows;
|
||||
src_bytes += chunk.bytes();
|
||||
|
||||
if (!params->aggregator.executeOnBlock(chunk.detachColumns(), num_rows, variants, key_columns, aggregate_columns, no_more_keys))
|
||||
is_consume_finished = true;
|
||||
if (params->only_merge)
|
||||
{
|
||||
auto block = getInputs().front().getHeader().cloneWithColumns(chunk.detachColumns());
|
||||
block = materializeBlock(block);
|
||||
if (!params->aggregator.mergeBlock(block, variants, no_more_keys))
|
||||
is_consume_finished = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!params->aggregator.executeOnBlock(chunk.detachColumns(), num_rows, variants, key_columns, aggregate_columns, no_more_keys))
|
||||
is_consume_finished = true;
|
||||
}
|
||||
}
|
||||
|
||||
void AggregatingTransform::initGenerate()
|
||||
|
@ -32,6 +32,7 @@ struct AggregatingTransformParams
|
||||
Aggregator::Params params;
|
||||
Aggregator aggregator;
|
||||
bool final;
|
||||
bool only_merge = false;
|
||||
|
||||
AggregatingTransformParams(const Aggregator::Params & params_, bool final_)
|
||||
: params(params_), aggregator(params), final(final_) {}
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
#include <Parsers/ASTProjectionDeclaration.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
@ -228,6 +229,25 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::ADD_PROJECTION)
|
||||
{
|
||||
AlterCommand command;
|
||||
command.ast = command_ast->clone();
|
||||
command.projection_decl = command_ast->projection_decl;
|
||||
command.type = AlterCommand::ADD_PROJECTION;
|
||||
|
||||
const auto & ast_projection_decl = command_ast->projection_decl->as<ASTProjectionDeclaration &>();
|
||||
|
||||
command.projection_name = ast_projection_decl.name;
|
||||
|
||||
if (command_ast->projection)
|
||||
command.after_projection_name = command_ast->projection->as<ASTIdentifier &>().name();
|
||||
|
||||
command.first = command_ast->first;
|
||||
command.if_not_exists = command_ast->if_not_exists;
|
||||
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::DROP_CONSTRAINT)
|
||||
{
|
||||
AlterCommand command;
|
||||
@ -253,6 +273,21 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::DROP_PROJECTION)
|
||||
{
|
||||
AlterCommand command;
|
||||
command.ast = command_ast->clone();
|
||||
command.type = AlterCommand::DROP_PROJECTION;
|
||||
command.projection_name = command_ast->projection->as<ASTIdentifier &>().name();
|
||||
command.if_exists = command_ast->if_exists;
|
||||
if (command_ast->clear_projection)
|
||||
command.clear = true;
|
||||
|
||||
if (command_ast->partition)
|
||||
command.partition = command_ast->partition;
|
||||
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::MODIFY_TTL)
|
||||
{
|
||||
AlterCommand command;
|
||||
@ -499,6 +534,16 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
}
|
||||
metadata.constraints.constraints.erase(erase_it);
|
||||
}
|
||||
else if (type == ADD_PROJECTION)
|
||||
{
|
||||
auto projection = ProjectionDescription::getProjectionFromAST(projection_decl, metadata.columns, context);
|
||||
metadata.projections.add(std::move(projection), after_projection_name, first, if_not_exists);
|
||||
}
|
||||
else if (type == DROP_PROJECTION)
|
||||
{
|
||||
if (!partition && !clear)
|
||||
metadata.projections.remove(projection_name);
|
||||
}
|
||||
else if (type == MODIFY_TTL)
|
||||
{
|
||||
metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST(ttl, metadata.columns, context, metadata.primary_key);
|
||||
@ -645,7 +690,7 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada
|
||||
if (isRemovingProperty() || type == REMOVE_TTL)
|
||||
return false;
|
||||
|
||||
if (type == DROP_COLUMN || type == DROP_INDEX || type == RENAME_COLUMN)
|
||||
if (type == DROP_COLUMN || type == DROP_INDEX || type == DROP_PROJECTION || type == RENAME_COLUMN)
|
||||
return true;
|
||||
|
||||
if (type != MODIFY_COLUMN || data_type == nullptr)
|
||||
@ -737,6 +782,17 @@ std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(Storage
|
||||
|
||||
result.predicate = nullptr;
|
||||
}
|
||||
else if (type == DROP_PROJECTION)
|
||||
{
|
||||
result.type = MutationCommand::Type::DROP_PROJECTION;
|
||||
result.column_name = projection_name;
|
||||
if (clear)
|
||||
result.clear = true;
|
||||
if (partition)
|
||||
result.partition = partition;
|
||||
|
||||
result.predicate = nullptr;
|
||||
}
|
||||
else if (type == RENAME_COLUMN)
|
||||
{
|
||||
result.type = MutationCommand::Type::RENAME_COLUMN;
|
||||
@ -760,6 +816,8 @@ String alterTypeToString(const AlterCommand::Type type)
|
||||
return "ADD CONSTRAINT";
|
||||
case AlterCommand::Type::ADD_INDEX:
|
||||
return "ADD INDEX";
|
||||
case AlterCommand::Type::ADD_PROJECTION:
|
||||
return "ADD PROJECTION";
|
||||
case AlterCommand::Type::COMMENT_COLUMN:
|
||||
return "COMMENT COLUMN";
|
||||
case AlterCommand::Type::DROP_COLUMN:
|
||||
@ -768,6 +826,8 @@ String alterTypeToString(const AlterCommand::Type type)
|
||||
return "DROP CONSTRAINT";
|
||||
case AlterCommand::Type::DROP_INDEX:
|
||||
return "DROP INDEX";
|
||||
case AlterCommand::Type::DROP_PROJECTION:
|
||||
return "DROP PROJECTION";
|
||||
case AlterCommand::Type::MODIFY_COLUMN:
|
||||
return "MODIFY COLUMN";
|
||||
case AlterCommand::Type::MODIFY_ORDER_BY:
|
||||
@ -823,7 +883,33 @@ void AlterCommands::apply(StorageInMemoryMetadata & metadata, ContextPtr context
|
||||
|
||||
/// Changes in columns may lead to changes in secondary indices
|
||||
for (auto & index : metadata_copy.secondary_indices)
|
||||
index = IndexDescription::getIndexFromAST(index.definition_ast, metadata_copy.columns, context);
|
||||
{
|
||||
try
|
||||
{
|
||||
index = IndexDescription::getIndexFromAST(index.definition_ast, metadata_copy.columns, context);
|
||||
}
|
||||
catch (Exception & exception)
|
||||
{
|
||||
exception.addMessage("Cannot apply mutation because it breaks skip index " + index.name);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
/// Changes in columns may lead to changes in projections
|
||||
ProjectionsDescription new_projections;
|
||||
for (const auto & projection : metadata_copy.projections)
|
||||
{
|
||||
try
|
||||
{
|
||||
new_projections.add(ProjectionDescription::getProjectionFromAST(projection.definition_ast, metadata_copy.columns, context));
|
||||
}
|
||||
catch (Exception & exception)
|
||||
{
|
||||
exception.addMessage("Cannot apply mutation because it breaks projection " + projection.name);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
metadata_copy.projections = std::move(new_projections);
|
||||
|
||||
/// Changes in columns may lead to changes in TTL expressions.
|
||||
auto column_ttl_asts = metadata_copy.columns.getColumnTTLs();
|
||||
|
@ -34,6 +34,8 @@ struct AlterCommand
|
||||
DROP_INDEX,
|
||||
ADD_CONSTRAINT,
|
||||
DROP_CONSTRAINT,
|
||||
ADD_PROJECTION,
|
||||
DROP_PROJECTION,
|
||||
MODIFY_TTL,
|
||||
MODIFY_SETTING,
|
||||
MODIFY_QUERY,
|
||||
@ -103,6 +105,13 @@ struct AlterCommand
|
||||
// For ADD/DROP CONSTRAINT
|
||||
String constraint_name;
|
||||
|
||||
/// For ADD PROJECTION
|
||||
ASTPtr projection_decl = nullptr;
|
||||
String after_projection_name;
|
||||
|
||||
/// For ADD/DROP PROJECTION
|
||||
String projection_name;
|
||||
|
||||
/// For MODIFY TTL
|
||||
ASTPtr ttl = nullptr;
|
||||
|
||||
|
@ -17,6 +17,9 @@ struct ColumnDependency
|
||||
/// Exists any skip index, that requires @column_name
|
||||
SKIP_INDEX,
|
||||
|
||||
/// Exists any projection, that requires @column_name
|
||||
PROJECTION,
|
||||
|
||||
/// Exists any TTL expression, that requires @column_name
|
||||
TTL_EXPRESSION,
|
||||
|
||||
@ -32,7 +35,7 @@ struct ColumnDependency
|
||||
|
||||
bool isReadOnly() const
|
||||
{
|
||||
return kind == SKIP_INDEX || kind == TTL_EXPRESSION;
|
||||
return kind == SKIP_INDEX || kind == PROJECTION || kind == TTL_EXPRESSION;
|
||||
}
|
||||
|
||||
bool operator==(const ColumnDependency & other) const
|
||||
|
@ -105,8 +105,9 @@ void IStorage::read(
|
||||
auto pipe = read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
|
||||
if (pipe.empty())
|
||||
{
|
||||
auto header = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID());
|
||||
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info);
|
||||
auto header = (query_info.projection ? query_info.projection->desc->metadata : metadata_snapshot)
|
||||
->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID());
|
||||
InterpreterSelectQuery::addEmptySourceToQueryPlan(query_plan, header, query_info, context);
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -224,16 +224,19 @@ public:
|
||||
|
||||
/** Returns stage to which query is going to be processed in read() function.
|
||||
* (Normally, the function only reads the columns from the list, but in other cases,
|
||||
* for example, the request can be partially processed on a remote server.)
|
||||
* for example, the request can be partially processed on a remote server, or an aggregate projection.)
|
||||
*
|
||||
* SelectQueryInfo is required since the stage can depends on the query
|
||||
* (see Distributed() engine and optimize_skip_unused_shards).
|
||||
* (see Distributed() engine and optimize_skip_unused_shards,
|
||||
* see also MergeTree engine and allow_experimental_projection_optimization).
|
||||
* And to store optimized cluster (after optimize_skip_unused_shards).
|
||||
* It will also store needed stuff for projection query pipeline.
|
||||
*
|
||||
* QueryProcessingStage::Enum required for Distributed over Distributed,
|
||||
* since it cannot return Complete for intermediate queries never.
|
||||
*/
|
||||
virtual QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum /*to_stage*/, SelectQueryInfo &) const
|
||||
virtual QueryProcessingStage::Enum
|
||||
getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageMetadataPtr &, SelectQueryInfo &) const
|
||||
{
|
||||
return QueryProcessingStage::FetchColumns;
|
||||
}
|
||||
|
@ -33,7 +33,11 @@ public:
|
||||
bool supportsSampling() const override { return true; }
|
||||
bool supportsFinal() const override { return true; }
|
||||
|
||||
QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum /*to_stage*/, SelectQueryInfo &) const override { return to_stage; }
|
||||
QueryProcessingStage::Enum
|
||||
getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageMetadataPtr &, SelectQueryInfo &) const override
|
||||
{
|
||||
return to_stage;
|
||||
}
|
||||
|
||||
Pipe read(
|
||||
const Names & /*column_names*/,
|
||||
|
@ -52,6 +52,7 @@ constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_TYPE = 3;
|
||||
constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_DEFAULT_COMPRESSION = 4;
|
||||
constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_UUID = 5;
|
||||
constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_S3_COPY = 6;
|
||||
constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PROJECTION = 7;
|
||||
|
||||
|
||||
std::string getEndpointId(const std::string & node_id)
|
||||
@ -116,7 +117,7 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write
|
||||
}
|
||||
|
||||
/// We pretend to work as older server version, to be sure that client will correctly process our version
|
||||
response.addCookie({"server_protocol_version", toString(std::min(client_protocol_version, REPLICATION_PROTOCOL_VERSION_WITH_PARTS_S3_COPY))});
|
||||
response.addCookie({"server_protocol_version", toString(std::min(client_protocol_version, REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PROJECTION))});
|
||||
|
||||
++total_sends;
|
||||
SCOPE_EXIT({--total_sends;});
|
||||
@ -126,9 +127,23 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write
|
||||
|
||||
LOG_TRACE(log, "Sending part {}", part_name);
|
||||
|
||||
MergeTreeData::DataPartPtr part;
|
||||
|
||||
auto report_broken_part = [&]()
|
||||
{
|
||||
if (part && part->isProjectionPart())
|
||||
{
|
||||
data.reportBrokenPart(part->getParentPart()->name);
|
||||
}
|
||||
else
|
||||
{
|
||||
data.reportBrokenPart(part_name);
|
||||
}
|
||||
};
|
||||
|
||||
try
|
||||
{
|
||||
MergeTreeData::DataPartPtr part = findPart(part_name);
|
||||
part = findPart(part_name);
|
||||
|
||||
CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedSend};
|
||||
|
||||
@ -148,34 +163,41 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write
|
||||
if (client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_UUID)
|
||||
writeUUIDText(part->uuid, out);
|
||||
|
||||
if (isInMemoryPart(part))
|
||||
sendPartFromMemory(part, out);
|
||||
else
|
||||
{
|
||||
bool try_use_s3_copy = false;
|
||||
bool try_use_s3_copy = false;
|
||||
|
||||
if (data_settings->allow_s3_zero_copy_replication
|
||||
&& client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_S3_COPY)
|
||||
{ /// if source and destination are in the same S3 storage we try to use S3 CopyObject request first
|
||||
int send_s3_metadata = parse<int>(params.get("send_s3_metadata", "0"));
|
||||
if (send_s3_metadata == 1)
|
||||
if (data_settings->allow_s3_zero_copy_replication
|
||||
&& client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_S3_COPY)
|
||||
{ /// if source and destination are in the same S3 storage we try to use S3 CopyObject request first
|
||||
int send_s3_metadata = parse<int>(params.get("send_s3_metadata", "0"));
|
||||
if (send_s3_metadata == 1)
|
||||
{
|
||||
auto disk = part->volume->getDisk();
|
||||
if (disk->getType() == DB::DiskType::Type::S3)
|
||||
{
|
||||
auto disk = part->volume->getDisk();
|
||||
if (disk->getType() == DB::DiskType::Type::S3)
|
||||
{
|
||||
try_use_s3_copy = true;
|
||||
}
|
||||
try_use_s3_copy = true;
|
||||
}
|
||||
}
|
||||
if (try_use_s3_copy)
|
||||
{
|
||||
response.addCookie({"send_s3_metadata", "1"});
|
||||
sendPartS3Metadata(part, out);
|
||||
}
|
||||
}
|
||||
if (try_use_s3_copy)
|
||||
{
|
||||
response.addCookie({"send_s3_metadata", "1"});
|
||||
sendPartS3Metadata(part, out);
|
||||
}
|
||||
else if (client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PROJECTION)
|
||||
{
|
||||
const auto & projections = part->getProjectionParts();
|
||||
writeBinary(projections.size(), out);
|
||||
if (isInMemoryPart(part))
|
||||
sendPartFromMemory(part, out, projections);
|
||||
else
|
||||
sendPartFromDisk(part, out, client_protocol_version, projections);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (isInMemoryPart(part))
|
||||
sendPartFromMemory(part, out);
|
||||
else
|
||||
{
|
||||
sendPartFromDisk(part, out, client_protocol_version);
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (const NetException &)
|
||||
@ -186,19 +208,34 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (e.code() != ErrorCodes::ABORTED && e.code() != ErrorCodes::CANNOT_WRITE_TO_OSTREAM)
|
||||
data.reportBrokenPart(part_name);
|
||||
report_broken_part();
|
||||
|
||||
throw;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
data.reportBrokenPart(part_name);
|
||||
report_broken_part();
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void Service::sendPartFromMemory(const MergeTreeData::DataPartPtr & part, WriteBuffer & out)
|
||||
void Service::sendPartFromMemory(
|
||||
const MergeTreeData::DataPartPtr & part, WriteBuffer & out, const std::map<String, std::shared_ptr<IMergeTreeDataPart>> & projections)
|
||||
{
|
||||
auto metadata_snapshot = data.getInMemoryMetadataPtr();
|
||||
for (const auto & [name, projection] : projections)
|
||||
{
|
||||
auto projection_sample_block = metadata_snapshot->projections.get(name).sample_block;
|
||||
auto part_in_memory = asInMemoryPart(projection);
|
||||
if (!part_in_memory)
|
||||
throw Exception("Projection " + name + " of part " + part->name + " is not stored in memory", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
writeStringBinary(name, out);
|
||||
projection->checksums.write(out);
|
||||
NativeBlockOutputStream block_out(out, 0, projection_sample_block);
|
||||
block_out.write(part_in_memory->block);
|
||||
}
|
||||
|
||||
auto part_in_memory = asInMemoryPart(part);
|
||||
if (!part_in_memory)
|
||||
throw Exception("Part " + part->name + " is not stored in memory", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -208,7 +245,11 @@ void Service::sendPartFromMemory(const MergeTreeData::DataPartPtr & part, WriteB
|
||||
block_out.write(part_in_memory->block);
|
||||
}
|
||||
|
||||
void Service::sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuffer & out, int client_protocol_version)
|
||||
MergeTreeData::DataPart::Checksums Service::sendPartFromDisk(
|
||||
const MergeTreeData::DataPartPtr & part,
|
||||
WriteBuffer & out,
|
||||
int client_protocol_version,
|
||||
const std::map<String, std::shared_ptr<IMergeTreeDataPart>> & projections)
|
||||
{
|
||||
/// We'll take a list of files from the list of checksums.
|
||||
MergeTreeData::DataPart::Checksums checksums = part->checksums;
|
||||
@ -224,6 +265,24 @@ void Service::sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuf
|
||||
|
||||
auto disk = part->volume->getDisk();
|
||||
MergeTreeData::DataPart::Checksums data_checksums;
|
||||
for (const auto & [name, projection] : part->getProjectionParts())
|
||||
{
|
||||
// Get rid of projection files
|
||||
checksums.files.erase(name + ".proj");
|
||||
auto it = projections.find(name);
|
||||
if (it != projections.end())
|
||||
{
|
||||
writeStringBinary(name, out);
|
||||
MergeTreeData::DataPart::Checksums projection_checksum = sendPartFromDisk(it->second, out, client_protocol_version);
|
||||
data_checksums.addFile(name + ".proj", projection_checksum.getTotalSizeOnDisk(), projection_checksum.getTotalChecksumUInt128());
|
||||
}
|
||||
else if (part->checksums.has(name + ".proj"))
|
||||
{
|
||||
// We don't send this projection, just add out checksum to bypass the following check
|
||||
const auto & our_checksum = part->checksums.files.find(name + ".proj")->second;
|
||||
data_checksums.addFile(name + ".proj", our_checksum.file_size, our_checksum.file_hash);
|
||||
}
|
||||
}
|
||||
|
||||
writeBinary(checksums.files.size(), out);
|
||||
for (const auto & it : checksums.files)
|
||||
@ -254,6 +313,7 @@ void Service::sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuf
|
||||
}
|
||||
|
||||
part->checksums.checkEqual(data_checksums, false);
|
||||
return data_checksums;
|
||||
}
|
||||
|
||||
void Service::sendPartS3Metadata(const MergeTreeData::DataPartPtr & part, WriteBuffer & out)
|
||||
@ -348,7 +408,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
|
||||
{
|
||||
{"endpoint", getEndpointId(replica_path)},
|
||||
{"part", part_name},
|
||||
{"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_PARTS_S3_COPY)},
|
||||
{"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PROJECTION)},
|
||||
{"compress", "false"}
|
||||
});
|
||||
|
||||
@ -491,8 +551,14 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
|
||||
in.setNextCallback(ReplicatedFetchReadCallback(*entry));
|
||||
|
||||
|
||||
return part_type == "InMemory" ? downloadPartToMemory(part_name, part_uuid, metadata_snapshot, std::move(reservation), in)
|
||||
: downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix_, sync, std::move(reservation), in);
|
||||
size_t projections = 0;
|
||||
if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PROJECTION)
|
||||
readBinary(projections, in);
|
||||
|
||||
MergeTreeData::DataPart::Checksums checksums;
|
||||
return part_type == "InMemory"
|
||||
? downloadPartToMemory(part_name, part_uuid, metadata_snapshot, std::move(reservation), in, projections)
|
||||
: downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix_, sync, reservation->getDisk(), in, projections, checksums);
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
|
||||
@ -500,8 +566,48 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
|
||||
const UUID & part_uuid,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
ReservationPtr reservation,
|
||||
PooledReadWriteBufferFromHTTP & in)
|
||||
PooledReadWriteBufferFromHTTP & in,
|
||||
size_t projections)
|
||||
{
|
||||
auto volume = std::make_shared<SingleDiskVolume>("volume_" + part_name, reservation->getDisk(), 0);
|
||||
MergeTreeData::MutableDataPartPtr new_data_part =
|
||||
std::make_shared<MergeTreeDataPartInMemory>(data, part_name, volume);
|
||||
|
||||
for (auto i = 0ul; i < projections; ++i)
|
||||
{
|
||||
String projection_name;
|
||||
readStringBinary(projection_name, in);
|
||||
MergeTreeData::DataPart::Checksums checksums;
|
||||
if (!checksums.read(in))
|
||||
throw Exception("Cannot deserialize checksums", ErrorCodes::CORRUPTED_DATA);
|
||||
|
||||
NativeBlockInputStream block_in(in, 0);
|
||||
auto block = block_in.read();
|
||||
|
||||
MergeTreePartInfo new_part_info("all", 0, 0, 0);
|
||||
MergeTreeData::MutableDataPartPtr new_projection_part =
|
||||
std::make_shared<MergeTreeDataPartInMemory>(data, projection_name, new_part_info, volume, projection_name, new_data_part.get());
|
||||
|
||||
new_projection_part->is_temp = false;
|
||||
new_projection_part->setColumns(block.getNamesAndTypesList());
|
||||
MergeTreePartition partition{};
|
||||
IMergeTreeDataPart::MinMaxIndex minmax_idx{};
|
||||
new_projection_part->partition = std::move(partition);
|
||||
new_projection_part->minmax_idx = std::move(minmax_idx);
|
||||
|
||||
MergedBlockOutputStream part_out(
|
||||
new_projection_part,
|
||||
metadata_snapshot->projections.get(projection_name).metadata,
|
||||
block.getNamesAndTypesList(),
|
||||
{},
|
||||
CompressionCodecFactory::instance().get("NONE", {}));
|
||||
part_out.writePrefix();
|
||||
part_out.write(block);
|
||||
part_out.writeSuffixAndFinalizePart(new_projection_part);
|
||||
new_projection_part->checksums.checkEqual(checksums, /* have_uncompressed = */ true);
|
||||
new_data_part->addProjectionPart(projection_name, std::move(new_projection_part));
|
||||
}
|
||||
|
||||
MergeTreeData::DataPart::Checksums checksums;
|
||||
if (!checksums.read(in))
|
||||
throw Exception("Cannot deserialize checksums", ErrorCodes::CORRUPTED_DATA);
|
||||
@ -509,17 +615,14 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
|
||||
NativeBlockInputStream block_in(in, 0);
|
||||
auto block = block_in.read();
|
||||
|
||||
auto volume = std::make_shared<SingleDiskVolume>("volume_" + part_name, reservation->getDisk(), 0);
|
||||
MergeTreeData::MutableDataPartPtr new_data_part =
|
||||
std::make_shared<MergeTreeDataPartInMemory>(data, part_name, volume);
|
||||
|
||||
new_data_part->uuid = part_uuid;
|
||||
new_data_part->is_temp = true;
|
||||
new_data_part->setColumns(block.getNamesAndTypesList());
|
||||
new_data_part->minmax_idx.update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey()));
|
||||
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", {}));
|
||||
part_out.writePrefix();
|
||||
part_out.write(block);
|
||||
part_out.writeSuffixAndFinalizePart(new_data_part);
|
||||
@ -528,47 +631,17 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
|
||||
return new_data_part;
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk(
|
||||
const String & part_name,
|
||||
void Fetcher::downloadBaseOrProjectionPartToDisk(
|
||||
const String & replica_path,
|
||||
bool to_detached,
|
||||
const String & tmp_prefix_,
|
||||
const String & part_download_path,
|
||||
bool sync,
|
||||
const ReservationPtr reservation,
|
||||
PooledReadWriteBufferFromHTTP & in)
|
||||
DiskPtr disk,
|
||||
PooledReadWriteBufferFromHTTP & in,
|
||||
MergeTreeData::DataPart::Checksums & checksums) const
|
||||
{
|
||||
size_t files;
|
||||
readBinary(files, in);
|
||||
|
||||
auto disk = reservation->getDisk();
|
||||
|
||||
static const String TMP_PREFIX = "tmp_fetch_";
|
||||
String tmp_prefix = tmp_prefix_.empty() ? TMP_PREFIX : tmp_prefix_;
|
||||
|
||||
/// We will remove directory if it's already exists. Make precautions.
|
||||
if (tmp_prefix.empty() //-V560
|
||||
|| part_name.empty()
|
||||
|| std::string::npos != tmp_prefix.find_first_of("/.")
|
||||
|| std::string::npos != part_name.find_first_of("/."))
|
||||
throw Exception("Logical error: tmp_prefix and part_name cannot be empty or contain '.' or '/' characters.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
String part_relative_path = String(to_detached ? "detached/" : "") + tmp_prefix + part_name;
|
||||
String part_download_path = data.getRelativeDataPath() + part_relative_path + "/";
|
||||
|
||||
if (disk->exists(part_download_path))
|
||||
{
|
||||
LOG_WARNING(log, "Directory {} already exists, probably result of a failed fetch. Will remove it before fetching part.",
|
||||
fullPath(disk, part_download_path));
|
||||
disk->removeRecursive(part_download_path);
|
||||
}
|
||||
|
||||
disk->createDirectories(part_download_path);
|
||||
|
||||
SyncGuardPtr sync_guard;
|
||||
if (data.getSettings()->fsync_part_directory)
|
||||
sync_guard = disk->getDirectorySyncGuard(part_download_path);
|
||||
|
||||
MergeTreeData::DataPart::Checksums checksums;
|
||||
for (size_t i = 0; i < files; ++i)
|
||||
{
|
||||
String file_name;
|
||||
@ -613,16 +686,69 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk(
|
||||
if (sync)
|
||||
hashing_out.sync();
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk(
|
||||
const String & part_name,
|
||||
const String & replica_path,
|
||||
bool to_detached,
|
||||
const String & tmp_prefix_,
|
||||
bool sync,
|
||||
DiskPtr disk,
|
||||
PooledReadWriteBufferFromHTTP & in,
|
||||
size_t projections,
|
||||
MergeTreeData::DataPart::Checksums & checksums)
|
||||
{
|
||||
static const String TMP_PREFIX = "tmp_fetch_";
|
||||
String tmp_prefix = tmp_prefix_.empty() ? TMP_PREFIX : tmp_prefix_;
|
||||
|
||||
/// We will remove directory if it's already exists. Make precautions.
|
||||
if (tmp_prefix.empty() //-V560
|
||||
|| part_name.empty()
|
||||
|| std::string::npos != tmp_prefix.find_first_of("/.")
|
||||
|| std::string::npos != part_name.find_first_of("/."))
|
||||
throw Exception("Logical error: tmp_prefix and part_name cannot be empty or contain '.' or '/' characters.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
String part_relative_path = String(to_detached ? "detached/" : "") + tmp_prefix + part_name;
|
||||
String part_download_path = data.getRelativeDataPath() + part_relative_path + "/";
|
||||
|
||||
if (disk->exists(part_download_path))
|
||||
{
|
||||
LOG_WARNING(log, "Directory {} already exists, probably result of a failed fetch. Will remove it before fetching part.",
|
||||
fullPath(disk, part_download_path));
|
||||
disk->removeRecursive(part_download_path);
|
||||
}
|
||||
|
||||
disk->createDirectories(part_download_path);
|
||||
|
||||
SyncGuardPtr sync_guard;
|
||||
if (data.getSettings()->fsync_part_directory)
|
||||
sync_guard = disk->getDirectorySyncGuard(part_download_path);
|
||||
|
||||
CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedFetch};
|
||||
|
||||
for (auto i = 0ul; i < projections; ++i)
|
||||
{
|
||||
String projection_name;
|
||||
readStringBinary(projection_name, in);
|
||||
MergeTreeData::DataPart::Checksums projection_checksum;
|
||||
disk->createDirectories(part_download_path + projection_name + ".proj/");
|
||||
downloadBaseOrProjectionPartToDisk(
|
||||
replica_path, part_download_path + projection_name + ".proj/", sync, disk, in, projection_checksum);
|
||||
checksums.addFile(
|
||||
projection_name + ".proj", projection_checksum.getTotalSizeOnDisk(), projection_checksum.getTotalChecksumUInt128());
|
||||
}
|
||||
|
||||
// Download the base part
|
||||
downloadBaseOrProjectionPartToDisk(replica_path, part_download_path, sync, disk, in, checksums);
|
||||
|
||||
assertEOF(in);
|
||||
|
||||
auto volume = std::make_shared<SingleDiskVolume>("volume_" + part_name, disk, 0);
|
||||
MergeTreeData::MutableDataPartPtr new_data_part = data.createPart(part_name, volume, part_relative_path);
|
||||
new_data_part->is_temp = true;
|
||||
new_data_part->is_temp = false;
|
||||
new_data_part->modification_time = time(nullptr);
|
||||
new_data_part->loadColumnsChecksumsIndexes(true, false);
|
||||
new_data_part->checksums.checkEqual(checksums, false);
|
||||
|
||||
return new_data_part;
|
||||
}
|
||||
|
||||
|
@ -36,8 +36,17 @@ public:
|
||||
|
||||
private:
|
||||
MergeTreeData::DataPartPtr findPart(const String & name);
|
||||
void sendPartFromMemory(const MergeTreeData::DataPartPtr & part, WriteBuffer & out);
|
||||
void sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuffer & out, int client_protocol_version);
|
||||
void sendPartFromMemory(
|
||||
const MergeTreeData::DataPartPtr & part,
|
||||
WriteBuffer & out,
|
||||
const std::map<String, std::shared_ptr<IMergeTreeDataPart>> & projections = {});
|
||||
|
||||
MergeTreeData::DataPart::Checksums sendPartFromDisk(
|
||||
const MergeTreeData::DataPartPtr & part,
|
||||
WriteBuffer & out,
|
||||
int client_protocol_version,
|
||||
const std::map<String, std::shared_ptr<IMergeTreeDataPart>> & projections = {});
|
||||
|
||||
void sendPartS3Metadata(const MergeTreeData::DataPartPtr & part, WriteBuffer & out);
|
||||
|
||||
/// StorageReplicatedMergeTree::shutdown() waits for all parts exchange handlers to finish,
|
||||
@ -74,21 +83,32 @@ public:
|
||||
ActionBlocker blocker;
|
||||
|
||||
private:
|
||||
void downloadBaseOrProjectionPartToDisk(
|
||||
const String & replica_path,
|
||||
const String & part_download_path,
|
||||
bool sync,
|
||||
DiskPtr disk,
|
||||
PooledReadWriteBufferFromHTTP & in,
|
||||
MergeTreeData::DataPart::Checksums & checksums) const;
|
||||
|
||||
MergeTreeData::MutableDataPartPtr downloadPartToDisk(
|
||||
const String & part_name,
|
||||
const String & replica_path,
|
||||
bool to_detached,
|
||||
const String & tmp_prefix_,
|
||||
bool sync,
|
||||
ReservationPtr reservation,
|
||||
PooledReadWriteBufferFromHTTP & in);
|
||||
DiskPtr disk,
|
||||
PooledReadWriteBufferFromHTTP & in,
|
||||
size_t projections,
|
||||
MergeTreeData::DataPart::Checksums & checksums);
|
||||
|
||||
MergeTreeData::MutableDataPartPtr downloadPartToMemory(
|
||||
const String & part_name,
|
||||
const UUID & part_uuid,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
ReservationPtr reservation,
|
||||
PooledReadWriteBufferFromHTTP & in);
|
||||
PooledReadWriteBufferFromHTTP & in,
|
||||
size_t projections);
|
||||
|
||||
MergeTreeData::MutableDataPartPtr downloadPartToS3(
|
||||
const String & part_name,
|
||||
|
@ -255,15 +255,23 @@ static void decrementTypeMetric(MergeTreeDataPartType type)
|
||||
|
||||
|
||||
IMergeTreeDataPart::IMergeTreeDataPart(
|
||||
MergeTreeData & storage_, const String & name_, const VolumePtr & volume_, const std::optional<String> & relative_path_, Type part_type_)
|
||||
MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const VolumePtr & volume_,
|
||||
const std::optional<String> & relative_path_,
|
||||
Type part_type_,
|
||||
const IMergeTreeDataPart * parent_part_)
|
||||
: storage(storage_)
|
||||
, name(name_)
|
||||
, info(MergeTreePartInfo::fromPartName(name_, storage.format_version))
|
||||
, volume(volume_)
|
||||
, volume(parent_part_ ? parent_part_->volume : volume_)
|
||||
, relative_path(relative_path_.value_or(name_))
|
||||
, index_granularity_info(storage_, part_type_)
|
||||
, part_type(part_type_)
|
||||
, parent_part(parent_part_)
|
||||
{
|
||||
if (parent_part)
|
||||
state = State::Committed;
|
||||
incrementStateMetric(state);
|
||||
incrementTypeMetric(part_type);
|
||||
}
|
||||
@ -274,15 +282,19 @@ IMergeTreeDataPart::IMergeTreeDataPart(
|
||||
const MergeTreePartInfo & info_,
|
||||
const VolumePtr & volume_,
|
||||
const std::optional<String> & relative_path_,
|
||||
Type part_type_)
|
||||
Type part_type_,
|
||||
const IMergeTreeDataPart * parent_part_)
|
||||
: storage(storage_)
|
||||
, name(name_)
|
||||
, info(info_)
|
||||
, volume(volume_)
|
||||
, volume(parent_part_ ? parent_part_->volume : volume_)
|
||||
, relative_path(relative_path_.value_or(name_))
|
||||
, index_granularity_info(storage_, part_type_)
|
||||
, part_type(part_type_)
|
||||
, parent_part(parent_part_)
|
||||
{
|
||||
if (parent_part)
|
||||
state = State::Committed;
|
||||
incrementStateMetric(state);
|
||||
incrementTypeMetric(part_type);
|
||||
}
|
||||
@ -416,7 +428,10 @@ void IMergeTreeDataPart::removeIfNeeded()
|
||||
}
|
||||
}
|
||||
|
||||
remove(false);
|
||||
if (parent_part)
|
||||
projectionRemove(parent_part->getFullRelativePath());
|
||||
else
|
||||
remove(false);
|
||||
|
||||
if (state == State::DeleteOnDestroy)
|
||||
{
|
||||
@ -509,7 +524,9 @@ size_t IMergeTreeDataPart::getFileSizeOrZero(const String & file_name) const
|
||||
String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize(const StorageMetadataPtr & metadata_snapshot) const
|
||||
{
|
||||
const auto & storage_columns = metadata_snapshot->getColumns().getAllPhysical();
|
||||
auto alter_conversions = storage.getAlterConversionsForPart(shared_from_this());
|
||||
MergeTreeData::AlterConversions alter_conversions;
|
||||
if (!parent_part)
|
||||
alter_conversions = storage.getAlterConversionsForPart(shared_from_this());
|
||||
|
||||
std::optional<std::string> minimum_size_column;
|
||||
UInt64 minimum_size = std::numeric_limits<UInt64>::max();
|
||||
@ -543,7 +560,7 @@ String IMergeTreeDataPart::getFullPath() const
|
||||
if (relative_path.empty())
|
||||
throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return storage.getFullPathOnDisk(volume->getDisk()) + relative_path + "/";
|
||||
return storage.getFullPathOnDisk(volume->getDisk()) + (parent_part ? parent_part->relative_path + "/" : "") + relative_path + "/";
|
||||
}
|
||||
|
||||
String IMergeTreeDataPart::getFullRelativePath() const
|
||||
@ -551,7 +568,7 @@ String IMergeTreeDataPart::getFullRelativePath() const
|
||||
if (relative_path.empty())
|
||||
throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return storage.relative_data_path + relative_path + "/";
|
||||
return storage.relative_data_path + (parent_part ? parent_part->relative_path + "/" : "") + relative_path + "/";
|
||||
}
|
||||
|
||||
void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency)
|
||||
@ -571,13 +588,33 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks
|
||||
loadIndex(); /// Must be called after loadIndexGranularity as it uses the value of `index_granularity`
|
||||
loadRowsCount(); /// Must be called after loadIndexGranularity() as it uses the value of `index_granularity`.
|
||||
loadPartitionAndMinMaxIndex();
|
||||
loadTTLInfos();
|
||||
if (!parent_part)
|
||||
{
|
||||
loadTTLInfos();
|
||||
loadProjections(require_columns_checksums, check_consistency);
|
||||
}
|
||||
|
||||
if (check_consistency)
|
||||
checkConsistency(require_columns_checksums);
|
||||
loadDefaultCompressionCodec();
|
||||
|
||||
}
|
||||
|
||||
void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool check_consistency)
|
||||
{
|
||||
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
|
||||
for (const auto & projection : metadata_snapshot->projections)
|
||||
{
|
||||
String path = getFullRelativePath() + projection.name + ".proj";
|
||||
if (volume->getDisk()->exists(path))
|
||||
{
|
||||
auto part = storage.createPart(projection.name, {"all", 0, 0, 0}, volume, projection.name + ".proj", this);
|
||||
part->loadColumnsChecksumsIndexes(require_columns_checksums, check_consistency);
|
||||
projection_parts.emplace(projection.name, std::move(part));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void IMergeTreeDataPart::loadIndexGranularity()
|
||||
{
|
||||
throw Exception("Method 'loadIndexGranularity' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
@ -590,6 +627,8 @@ void IMergeTreeDataPart::loadIndex()
|
||||
throw Exception("Index granularity is not loaded before index loading", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
|
||||
if (parent_part)
|
||||
metadata_snapshot = metadata_snapshot->projections.get(name).metadata;
|
||||
const auto & primary_key = metadata_snapshot->getPrimaryKey();
|
||||
size_t key_size = primary_key.column_names.size();
|
||||
|
||||
@ -744,7 +783,7 @@ CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const
|
||||
|
||||
void IMergeTreeDataPart::loadPartitionAndMinMaxIndex()
|
||||
{
|
||||
if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
if (storage.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING && !parent_part)
|
||||
{
|
||||
DayNum min_date;
|
||||
DayNum max_date;
|
||||
@ -757,9 +796,19 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex()
|
||||
else
|
||||
{
|
||||
String path = getFullRelativePath();
|
||||
partition.load(storage, volume->getDisk(), path);
|
||||
if (!parent_part)
|
||||
partition.load(storage, volume->getDisk(), path);
|
||||
|
||||
if (!isEmpty())
|
||||
minmax_idx.load(storage, volume->getDisk(), path);
|
||||
{
|
||||
if (parent_part)
|
||||
// projection parts don't have minmax_idx, and it's always initialized
|
||||
minmax_idx.initialized = true;
|
||||
else
|
||||
minmax_idx.load(storage, volume->getDisk(), path);
|
||||
}
|
||||
if (parent_part)
|
||||
return;
|
||||
}
|
||||
|
||||
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
|
||||
@ -815,7 +864,7 @@ void IMergeTreeDataPart::loadRowsCount()
|
||||
{
|
||||
rows_count = 0;
|
||||
}
|
||||
else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT)
|
||||
else if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || part_type == Type::COMPACT || parent_part)
|
||||
{
|
||||
if (!volume->getDisk()->exists(path))
|
||||
throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
@ -953,6 +1002,8 @@ void IMergeTreeDataPart::loadColumns(bool require)
|
||||
{
|
||||
String path = getFullRelativePath() + "columns.txt";
|
||||
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
|
||||
if (parent_part)
|
||||
metadata_snapshot = metadata_snapshot->projections.get(name).metadata;
|
||||
NamesAndTypesList loaded_columns;
|
||||
|
||||
if (!volume->getDisk()->exists(path))
|
||||
@ -1002,7 +1053,7 @@ UInt64 IMergeTreeDataPart::calculateTotalSizeOnDisk(const DiskPtr & disk_, const
|
||||
disk_->listFiles(from, files);
|
||||
UInt64 res = 0;
|
||||
for (const auto & file : files)
|
||||
res += calculateTotalSizeOnDisk(disk_, from + file);
|
||||
res += calculateTotalSizeOnDisk(disk_, from + "/" + file);
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -1012,7 +1063,7 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_
|
||||
assertOnDisk();
|
||||
|
||||
String from = getFullRelativePath();
|
||||
String to = storage.relative_data_path + new_relative_path + "/";
|
||||
String to = storage.relative_data_path + (parent_part ? parent_part->relative_path + "/" : "") + new_relative_path + "/";
|
||||
|
||||
if (!volume->getDisk()->exists(from))
|
||||
throw Exception("Part directory " + fullPath(volume->getDisk(), from) + " doesn't exist. Most likely it is a logical error.", ErrorCodes::FILE_DOESNT_EXIST);
|
||||
@ -1054,6 +1105,13 @@ void IMergeTreeDataPart::remove(bool keep_s3) const
|
||||
if (relative_path.empty())
|
||||
throw Exception("Part relative_path cannot be empty. This is bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (isProjectionPart())
|
||||
{
|
||||
LOG_WARNING(storage.log, "Projection part {} should be removed by its parent {}.", name, parent_part->name);
|
||||
projectionRemove(parent_part->getFullRelativePath());
|
||||
return;
|
||||
}
|
||||
|
||||
/** Atomic directory removal:
|
||||
* - rename directory to temporary name;
|
||||
* - remove it recursive.
|
||||
@ -1070,36 +1128,45 @@ void IMergeTreeDataPart::remove(bool keep_s3) const
|
||||
String to = storage.relative_data_path + "delete_tmp_" + name;
|
||||
// TODO directory delete_tmp_<name> is never removed if server crashes before returning from this function
|
||||
|
||||
if (volume->getDisk()->exists(to))
|
||||
auto disk = volume->getDisk();
|
||||
if (disk->exists(to))
|
||||
{
|
||||
LOG_WARNING(storage.log, "Directory {} (to which part must be renamed before removing) already exists. Most likely this is due to unclean restart. Removing it.", fullPath(volume->getDisk(), to));
|
||||
|
||||
LOG_WARNING(storage.log, "Directory {} (to which part must be renamed before removing) already exists. Most likely this is due to unclean restart. Removing it.", fullPath(disk, to));
|
||||
try
|
||||
{
|
||||
volume->getDisk()->removeSharedRecursive(to + "/", keep_s3);
|
||||
disk->removeSharedRecursive(to + "/", keep_s3);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_ERROR(storage.log, "Cannot recursively remove directory {}. Exception: {}", fullPath(volume->getDisk(), to), getCurrentExceptionMessage(false));
|
||||
LOG_ERROR(storage.log, "Cannot recursively remove directory {}. Exception: {}", fullPath(disk, to), getCurrentExceptionMessage(false));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
volume->getDisk()->moveDirectory(from, to);
|
||||
disk->moveDirectory(from, to);
|
||||
}
|
||||
catch (const Poco::FileNotFoundException &)
|
||||
{
|
||||
LOG_ERROR(storage.log, "Directory {} (part to remove) doesn't exist or one of nested files has gone. Most likely this is due to manual removing. This should be discouraged. Ignoring.", fullPath(volume->getDisk(), to));
|
||||
LOG_ERROR(storage.log, "Directory {} (part to remove) doesn't exist or one of nested files has gone. Most likely this is due to manual removing. This should be discouraged. Ignoring.", fullPath(disk, to));
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
// Record existing projection directories so we don't remove them twice
|
||||
std::unordered_set<String> projection_directories;
|
||||
for (const auto & [p_name, projection_part] : projection_parts)
|
||||
{
|
||||
projection_part->projectionRemove(to);
|
||||
projection_directories.emplace(p_name + ".proj");
|
||||
}
|
||||
|
||||
|
||||
if (checksums.empty())
|
||||
{
|
||||
/// If the part is not completely written, we cannot use fast path by listing files.
|
||||
volume->getDisk()->removeSharedRecursive(to + "/", keep_s3);
|
||||
disk->removeSharedRecursive(to + "/", keep_s3);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1112,30 +1179,82 @@ void IMergeTreeDataPart::remove(bool keep_s3) const
|
||||
# pragma GCC diagnostic ignored "-Wunused-variable"
|
||||
#endif
|
||||
for (const auto & [file, _] : checksums.files)
|
||||
volume->getDisk()->removeSharedFile(to + "/" + file, keep_s3);
|
||||
{
|
||||
if (projection_directories.find(file) == projection_directories.end())
|
||||
disk->removeSharedFile(to + "/" + file, keep_s3);
|
||||
}
|
||||
#if !defined(__clang__)
|
||||
# pragma GCC diagnostic pop
|
||||
#endif
|
||||
|
||||
for (const auto & file : {"checksums.txt", "columns.txt"})
|
||||
volume->getDisk()->removeSharedFile(to + "/" + file, keep_s3);
|
||||
disk->removeSharedFile(to + "/" + file, keep_s3);
|
||||
|
||||
volume->getDisk()->removeSharedFileIfExists(to + "/" + DEFAULT_COMPRESSION_CODEC_FILE_NAME, keep_s3);
|
||||
volume->getDisk()->removeSharedFileIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME, keep_s3);
|
||||
disk->removeSharedFileIfExists(to + "/" + DEFAULT_COMPRESSION_CODEC_FILE_NAME, keep_s3);
|
||||
disk->removeSharedFileIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME, keep_s3);
|
||||
|
||||
volume->getDisk()->removeDirectory(to);
|
||||
disk->removeDirectory(to);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Recursive directory removal does many excessive "stat" syscalls under the hood.
|
||||
|
||||
LOG_ERROR(storage.log, "Cannot quickly remove directory {} by removing files; fallback to recursive removal. Reason: {}", fullPath(volume->getDisk(), to), getCurrentExceptionMessage(false));
|
||||
LOG_ERROR(storage.log, "Cannot quickly remove directory {} by removing files; fallback to recursive removal. Reason: {}", fullPath(disk, to), getCurrentExceptionMessage(false));
|
||||
|
||||
volume->getDisk()->removeSharedRecursive(to + "/", keep_s3);
|
||||
disk->removeSharedRecursive(to + "/", keep_s3);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void IMergeTreeDataPart::projectionRemove(const String & parent_to) const
|
||||
{
|
||||
String to = parent_to + "/" + relative_path;
|
||||
auto disk = volume->getDisk();
|
||||
if (checksums.empty())
|
||||
{
|
||||
|
||||
LOG_ERROR(
|
||||
storage.log,
|
||||
"Cannot quickly remove directory {} by removing files; fallback to recursive removal. Reason: checksums.txt is missing",
|
||||
fullPath(disk, to));
|
||||
/// If the part is not completely written, we cannot use fast path by listing files.
|
||||
disk->removeRecursive(to + "/");
|
||||
}
|
||||
else
|
||||
{
|
||||
try
|
||||
{
|
||||
/// Remove each expected file in directory, then remove directory itself.
|
||||
|
||||
#if !defined(__clang__)
|
||||
# pragma GCC diagnostic push
|
||||
# pragma GCC diagnostic ignored "-Wunused-variable"
|
||||
#endif
|
||||
for (const auto & [file, _] : checksums.files)
|
||||
disk->removeFile(to + "/" + file);
|
||||
#if !defined(__clang__)
|
||||
# pragma GCC diagnostic pop
|
||||
#endif
|
||||
|
||||
for (const auto & file : {"checksums.txt", "columns.txt"})
|
||||
disk->removeFile(to + "/" + file);
|
||||
disk->removeFileIfExists(to + "/" + DEFAULT_COMPRESSION_CODEC_FILE_NAME);
|
||||
disk->removeFileIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME);
|
||||
|
||||
disk->removeDirectory(to);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Recursive directory removal does many excessive "stat" syscalls under the hood.
|
||||
|
||||
LOG_ERROR(storage.log, "Cannot quickly remove directory {} by removing files; fallback to recursive removal. Reason: {}", fullPath(disk, to), getCurrentExceptionMessage(false));
|
||||
|
||||
disk->removeRecursive(to + "/");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
String IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix) const
|
||||
{
|
||||
String res;
|
||||
@ -1205,6 +1324,13 @@ void IMergeTreeDataPart::checkConsistencyBase() const
|
||||
String path = getFullRelativePath();
|
||||
|
||||
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
|
||||
if (parent_part)
|
||||
metadata_snapshot = metadata_snapshot->projections.get(name).metadata;
|
||||
else
|
||||
{
|
||||
// No need to check projections here because we already did consistent checking when loading projections if necessary.
|
||||
}
|
||||
|
||||
const auto & pk = metadata_snapshot->getPrimaryKey();
|
||||
const auto & partition_key = metadata_snapshot->getPartitionKey();
|
||||
if (!checksums.empty())
|
||||
@ -1220,7 +1346,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const
|
||||
if (metadata_snapshot->hasPartitionKey() && !checksums.files.count("partition.dat"))
|
||||
throw Exception("No checksum for partition.dat", ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
if (!isEmpty())
|
||||
if (!isEmpty() && !parent_part)
|
||||
{
|
||||
for (const String & col_name : storage.getMinMaxColumnsNames(partition_key))
|
||||
{
|
||||
@ -1253,8 +1379,11 @@ void IMergeTreeDataPart::checkConsistencyBase() const
|
||||
if (metadata_snapshot->hasPartitionKey())
|
||||
check_file_not_empty(volume->getDisk(), path + "partition.dat");
|
||||
|
||||
for (const String & col_name : storage.getMinMaxColumnsNames(partition_key))
|
||||
check_file_not_empty(volume->getDisk(), path + "minmax_" + escapeForFileName(col_name) + ".idx");
|
||||
if (!parent_part)
|
||||
{
|
||||
for (const String & col_name : storage.getMinMaxColumnsNames(partition_key))
|
||||
check_file_not_empty(volume->getDisk(), path + "minmax_" + escapeForFileName(col_name) + ".idx");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeIndexGranularity.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndexGranularityInfo.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndices.h>
|
||||
#include <Storages/MergeTree/MergeTreeProjections.h>
|
||||
#include <Storages/MergeTree/MergeTreePartInfo.h>
|
||||
#include <Storages/MergeTree/MergeTreePartition.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
|
||||
@ -74,14 +75,16 @@ public:
|
||||
const MergeTreePartInfo & info_,
|
||||
const VolumePtr & volume,
|
||||
const std::optional<String> & relative_path,
|
||||
Type part_type_);
|
||||
Type part_type_,
|
||||
const IMergeTreeDataPart * parent_part_);
|
||||
|
||||
IMergeTreeDataPart(
|
||||
MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const VolumePtr & volume,
|
||||
const std::optional<String> & relative_path,
|
||||
Type part_type_);
|
||||
Type part_type_,
|
||||
const IMergeTreeDataPart * parent_part_);
|
||||
|
||||
virtual MergeTreeReaderPtr getReader(
|
||||
const NamesAndTypesList & columns_,
|
||||
@ -133,6 +136,8 @@ public:
|
||||
|
||||
void remove(bool keep_s3 = false) const;
|
||||
|
||||
void projectionRemove(const String & parent_to) const;
|
||||
|
||||
/// Initialize columns (from columns.txt if exists, or create from column files if not).
|
||||
/// Load checksums from checksums.txt if exists. Load index if required.
|
||||
void loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency);
|
||||
@ -350,6 +355,23 @@ public:
|
||||
|
||||
String getRelativePathForPrefix(const String & prefix) const;
|
||||
|
||||
bool isProjectionPart() const { return parent_part != nullptr; }
|
||||
|
||||
const IMergeTreeDataPart * getParentPart() const { return parent_part; }
|
||||
|
||||
const std::map<String, std::shared_ptr<IMergeTreeDataPart>> & getProjectionParts() const { return projection_parts; }
|
||||
|
||||
void addProjectionPart(const String & projection_name, std::shared_ptr<IMergeTreeDataPart> && projection_part)
|
||||
{
|
||||
projection_parts.emplace(projection_name, std::move(projection_part));
|
||||
}
|
||||
|
||||
bool hasProjection(const String & projection_name) const
|
||||
{
|
||||
return projection_parts.find(projection_name) != projection_parts.end();
|
||||
}
|
||||
|
||||
void loadProjections(bool require_columns_checksums, bool check_consistency);
|
||||
|
||||
/// Return set of metadat file names without checksums. For example,
|
||||
/// columns.txt or checksums.txt itself.
|
||||
@ -392,6 +414,11 @@ protected:
|
||||
NamesAndTypesList columns;
|
||||
const Type part_type;
|
||||
|
||||
/// Not null when it's a projection part.
|
||||
const IMergeTreeDataPart * parent_part;
|
||||
|
||||
std::map<String, std::shared_ptr<IMergeTreeDataPart>> projection_parts;
|
||||
|
||||
void removeIfNeeded();
|
||||
|
||||
virtual void checkConsistency(bool require_part_metadata) const;
|
||||
|
@ -8,6 +8,8 @@
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Processors/Transforms/AggregatingTransform.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -205,6 +207,7 @@ namespace
|
||||
{
|
||||
virtual ~VirtualColumnsInserter() = default;
|
||||
|
||||
virtual void insertArrayOfStringsColumn(const ColumnPtr & column, const String & name) = 0;
|
||||
virtual void insertStringColumn(const ColumnPtr & column, const String & name) = 0;
|
||||
virtual void insertUInt64Column(const ColumnPtr & column, const String & name) = 0;
|
||||
virtual void insertUUIDColumn(const ColumnPtr & column, const String & name) = 0;
|
||||
@ -229,13 +232,20 @@ static void injectVirtualColumnsImpl(
|
||||
throw Exception("Cannot insert virtual columns to non-empty chunk without specified task.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
const IMergeTreeDataPart * part = nullptr;
|
||||
if (rows)
|
||||
{
|
||||
part = task->data_part.get();
|
||||
if (part->isProjectionPart())
|
||||
part = part->getParentPart();
|
||||
}
|
||||
for (const auto & virtual_column_name : virtual_columns)
|
||||
{
|
||||
if (virtual_column_name == "_part")
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = DataTypeString().createColumnConst(rows, task->data_part->name)->convertToFullColumnIfConst();
|
||||
column = DataTypeString().createColumnConst(rows, part->name)->convertToFullColumnIfConst();
|
||||
else
|
||||
column = DataTypeString().createColumn();
|
||||
|
||||
@ -265,7 +275,7 @@ static void injectVirtualColumnsImpl(
|
||||
{
|
||||
ColumnPtr column;
|
||||
if (rows)
|
||||
column = DataTypeString().createColumnConst(rows, task->data_part->info.partition_id)->convertToFullColumnIfConst();
|
||||
column = DataTypeString().createColumnConst(rows, part->info.partition_id)->convertToFullColumnIfConst();
|
||||
else
|
||||
column = DataTypeString().createColumn();
|
||||
|
||||
@ -288,6 +298,11 @@ namespace
|
||||
{
|
||||
explicit VirtualColumnsInserterIntoBlock(Block & block_) : block(block_) {}
|
||||
|
||||
void insertArrayOfStringsColumn(const ColumnPtr & column, const String & name) final
|
||||
{
|
||||
block.insert({column, std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), name});
|
||||
}
|
||||
|
||||
void insertStringColumn(const ColumnPtr & column, const String & name) final
|
||||
{
|
||||
block.insert({column, std::make_shared<DataTypeString>(), name});
|
||||
@ -323,6 +338,11 @@ namespace
|
||||
{
|
||||
explicit VirtualColumnsInserterIntoColumns(Columns & columns_) : columns(columns_) {}
|
||||
|
||||
void insertArrayOfStringsColumn(const ColumnPtr & column, const String &) final
|
||||
{
|
||||
columns.push_back(column);
|
||||
}
|
||||
|
||||
void insertStringColumn(const ColumnPtr & column, const String &) final
|
||||
{
|
||||
columns.push_back(column);
|
||||
|
@ -28,7 +28,7 @@ void MergeTreeBlockOutputStream::write(const Block & block)
|
||||
{
|
||||
Stopwatch watch;
|
||||
|
||||
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, metadata_snapshot, optimize_on_insert);
|
||||
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, metadata_snapshot, context);
|
||||
|
||||
/// If optimize_on_insert setting is true, current_block could become empty after merge
|
||||
/// and we didn't create part.
|
||||
|
@ -14,11 +14,15 @@ class StorageMergeTree;
|
||||
class MergeTreeBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
MergeTreeBlockOutputStream(StorageMergeTree & storage_, const StorageMetadataPtr metadata_snapshot_, size_t max_parts_per_block_, bool optimize_on_insert_)
|
||||
MergeTreeBlockOutputStream(
|
||||
StorageMergeTree & storage_,
|
||||
const StorageMetadataPtr metadata_snapshot_,
|
||||
size_t max_parts_per_block_,
|
||||
ContextPtr context_)
|
||||
: storage(storage_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, max_parts_per_block(max_parts_per_block_)
|
||||
, optimize_on_insert(optimize_on_insert_)
|
||||
, context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -30,7 +34,7 @@ private:
|
||||
StorageMergeTree & storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
size_t max_parts_per_block;
|
||||
bool optimize_on_insert;
|
||||
ContextPtr context;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -87,7 +87,9 @@ NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageMetada
|
||||
bool have_at_least_one_physical_column = false;
|
||||
|
||||
const auto & storage_columns = metadata_snapshot->getColumns();
|
||||
auto alter_conversions = storage.getAlterConversionsForPart(part);
|
||||
MergeTreeData::AlterConversions alter_conversions;
|
||||
if (!part->isProjectionPart())
|
||||
alter_conversions = storage.getAlterConversionsForPart(part);
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
/// We are going to fetch only physical columns
|
||||
|
@ -16,12 +16,12 @@
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/inplaceBlockConversions.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTNameTypePair.h>
|
||||
@ -398,6 +398,23 @@ void MergeTreeData::checkProperties(
|
||||
}
|
||||
}
|
||||
|
||||
if (!new_metadata.projections.empty())
|
||||
{
|
||||
std::unordered_set<String> projections_names;
|
||||
|
||||
for (const auto & projection : new_metadata.projections)
|
||||
{
|
||||
MergeTreeProjectionFactory::instance().validate(projection);
|
||||
|
||||
if (projections_names.find(projection.name) != projections_names.end())
|
||||
throw Exception(
|
||||
"Projection with name " + backQuote(projection.name) + " already exists",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
projections_names.insert(projection.name);
|
||||
}
|
||||
}
|
||||
|
||||
checkKeyExpression(*new_sorting_key.expression, new_sorting_key.sample_block, "Sorting", allow_nullable_key);
|
||||
|
||||
}
|
||||
@ -715,8 +732,9 @@ Block MergeTreeData::getBlockWithVirtualPartColumns(const MergeTreeData::DataPar
|
||||
auto & part_uuid_column = columns[2];
|
||||
auto & partition_value_column = columns[3];
|
||||
|
||||
for (const auto & part : parts)
|
||||
for (const auto & part_or_projection : parts)
|
||||
{
|
||||
const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get();
|
||||
part_column->insert(part->name);
|
||||
partition_id_column->insert(part->info.partition_id);
|
||||
part_uuid_column->insert(part->uuid);
|
||||
@ -772,7 +790,7 @@ std::optional<UInt64> MergeTreeData::totalRowsByPartitionPredicateImpl(
|
||||
size_t res = 0;
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
if ((part_values.empty() || part_values.find(part->name) != part_values.end()) && !partition_pruner.canBePruned(part))
|
||||
if ((part_values.empty() || part_values.find(part->name) != part_values.end()) && !partition_pruner.canBePruned(*part))
|
||||
res += part->rows_count;
|
||||
}
|
||||
return res;
|
||||
@ -1692,6 +1710,12 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
|
||||
"ALTER ADD INDEX is not supported for tables with the old syntax",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
if (command.type == AlterCommand::ADD_PROJECTION && !is_custom_partitioned)
|
||||
{
|
||||
throw Exception(
|
||||
"ALTER ADD PROJECTION is not supported for tables with the old syntax",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
if (command.type == AlterCommand::RENAME_COLUMN)
|
||||
{
|
||||
if (columns_in_keys.count(command.column_name))
|
||||
@ -1865,14 +1889,14 @@ MergeTreeDataPartType MergeTreeData::choosePartTypeOnDisk(size_t bytes_uncompres
|
||||
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeData::createPart(const String & name,
|
||||
MergeTreeDataPartType type, const MergeTreePartInfo & part_info,
|
||||
const VolumePtr & volume, const String & relative_path) const
|
||||
const VolumePtr & volume, const String & relative_path, const IMergeTreeDataPart * parent_part) const
|
||||
{
|
||||
if (type == MergeTreeDataPartType::COMPACT)
|
||||
return std::make_shared<MergeTreeDataPartCompact>(*this, name, part_info, volume, relative_path);
|
||||
return std::make_shared<MergeTreeDataPartCompact>(*this, name, part_info, volume, relative_path, parent_part);
|
||||
else if (type == MergeTreeDataPartType::WIDE)
|
||||
return std::make_shared<MergeTreeDataPartWide>(*this, name, part_info, volume, relative_path);
|
||||
return std::make_shared<MergeTreeDataPartWide>(*this, name, part_info, volume, relative_path, parent_part);
|
||||
else if (type == MergeTreeDataPartType::IN_MEMORY)
|
||||
return std::make_shared<MergeTreeDataPartInMemory>(*this, name, part_info, volume, relative_path);
|
||||
return std::make_shared<MergeTreeDataPartInMemory>(*this, name, part_info, volume, relative_path, parent_part);
|
||||
else
|
||||
throw Exception("Unknown type of part " + relative_path, ErrorCodes::UNKNOWN_PART_TYPE);
|
||||
}
|
||||
@ -1890,17 +1914,17 @@ static MergeTreeDataPartType getPartTypeFromMarkExtension(const String & mrk_ext
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeData::createPart(
|
||||
const String & name, const VolumePtr & volume, const String & relative_path) const
|
||||
const String & name, const VolumePtr & volume, const String & relative_path, const IMergeTreeDataPart * parent_part) const
|
||||
{
|
||||
return createPart(name, MergeTreePartInfo::fromPartName(name, format_version), volume, relative_path);
|
||||
return createPart(name, MergeTreePartInfo::fromPartName(name, format_version), volume, relative_path, parent_part);
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeData::createPart(
|
||||
const String & name, const MergeTreePartInfo & part_info,
|
||||
const VolumePtr & volume, const String & relative_path) const
|
||||
const VolumePtr & volume, const String & relative_path, const IMergeTreeDataPart * parent_part) const
|
||||
{
|
||||
MergeTreeDataPartType type;
|
||||
auto full_path = relative_data_path + relative_path + "/";
|
||||
auto full_path = relative_data_path + (parent_part ? parent_part->relative_path + "/" : "") + relative_path + "/";
|
||||
auto mrk_ext = MergeTreeIndexGranularityInfo::getMarksExtensionFromFilesystem(volume->getDisk(), full_path);
|
||||
|
||||
if (mrk_ext)
|
||||
@ -1911,7 +1935,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createPart(
|
||||
type = choosePartTypeOnDisk(0, 0);
|
||||
}
|
||||
|
||||
return createPart(name, type, part_info, volume, relative_path);
|
||||
return createPart(name, type, part_info, volume, relative_path, parent_part);
|
||||
}
|
||||
|
||||
void MergeTreeData::changeSettings(
|
||||
@ -3174,7 +3198,8 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc
|
||||
return partition_id;
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector(const DataPartStates & affordable_states, DataPartStateVector * out_states) const
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector(
|
||||
const DataPartStates & affordable_states, DataPartStateVector * out_states, bool require_projection_parts) const
|
||||
{
|
||||
DataPartsVector res;
|
||||
DataPartsVector buf;
|
||||
@ -3183,27 +3208,64 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector(const DataPartS
|
||||
|
||||
for (auto state : affordable_states)
|
||||
{
|
||||
std::swap(buf, res);
|
||||
res.clear();
|
||||
|
||||
auto range = getDataPartsStateRange(state);
|
||||
std::merge(range.begin(), range.end(), buf.begin(), buf.end(), std::back_inserter(res), LessDataPart());
|
||||
|
||||
if (require_projection_parts)
|
||||
{
|
||||
for (const auto & part : range)
|
||||
{
|
||||
for (const auto & [p_name, projection_part] : part->getProjectionParts())
|
||||
res.push_back(projection_part);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
std::swap(buf, res);
|
||||
res.clear();
|
||||
std::merge(range.begin(), range.end(), buf.begin(), buf.end(), std::back_inserter(res), LessDataPart());
|
||||
}
|
||||
}
|
||||
|
||||
if (out_states != nullptr)
|
||||
{
|
||||
out_states->resize(res.size());
|
||||
for (size_t i = 0; i < res.size(); ++i)
|
||||
(*out_states)[i] = res[i]->getState();
|
||||
if (require_projection_parts)
|
||||
{
|
||||
for (size_t i = 0; i < res.size(); ++i)
|
||||
(*out_states)[i] = res[i]->getParentPart()->getState();
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < res.size(); ++i)
|
||||
(*out_states)[i] = res[i]->getState();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
MergeTreeData::DataPartsVector MergeTreeData::getAllDataPartsVector(MergeTreeData::DataPartStateVector * out_states) const
|
||||
MergeTreeData::DataPartsVector
|
||||
MergeTreeData::getAllDataPartsVector(MergeTreeData::DataPartStateVector * out_states, bool require_projection_parts) const
|
||||
{
|
||||
DataPartsVector res;
|
||||
if (require_projection_parts)
|
||||
{
|
||||
auto lock = lockParts();
|
||||
for (const auto & part : data_parts_by_info)
|
||||
{
|
||||
for (const auto & [p_name, projection_part] : part->getProjectionParts())
|
||||
res.push_back(projection_part);
|
||||
}
|
||||
|
||||
if (out_states != nullptr)
|
||||
{
|
||||
out_states->resize(res.size());
|
||||
for (size_t i = 0; i < res.size(); ++i)
|
||||
(*out_states)[i] = res[i]->getParentPart()->getState();
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
auto lock = lockParts();
|
||||
res.assign(data_parts_by_info.begin(), data_parts_by_info.end());
|
||||
@ -3710,9 +3772,17 @@ bool MergeTreeData::mayBenefitFromIndexForIn(
|
||||
for (const auto & index : metadata_snapshot->getSecondaryIndices())
|
||||
if (index_wrapper_factory.get(index)->mayBenefitFromIndexForIn(item))
|
||||
return true;
|
||||
if (metadata_snapshot->selected_projection
|
||||
&& metadata_snapshot->selected_projection->isPrimaryKeyColumnPossiblyWrappedInFunctions(item))
|
||||
return true;
|
||||
}
|
||||
/// The tuple itself may be part of the primary key, so check that as a last resort.
|
||||
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand, metadata_snapshot);
|
||||
if (isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand, metadata_snapshot))
|
||||
return true;
|
||||
if (metadata_snapshot->selected_projection
|
||||
&& metadata_snapshot->selected_projection->isPrimaryKeyColumnPossiblyWrappedInFunctions(left_in_operand))
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -3720,10 +3790,411 @@ bool MergeTreeData::mayBenefitFromIndexForIn(
|
||||
if (index_wrapper_factory.get(index)->mayBenefitFromIndexForIn(left_in_operand))
|
||||
return true;
|
||||
|
||||
if (metadata_snapshot->selected_projection
|
||||
&& metadata_snapshot->selected_projection->isPrimaryKeyColumnPossiblyWrappedInFunctions(left_in_operand))
|
||||
return true;
|
||||
|
||||
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand, metadata_snapshot);
|
||||
}
|
||||
}
|
||||
|
||||
using PartitionIdToMaxBlock = std::unordered_map<String, Int64>;
|
||||
|
||||
static void selectBestProjection(
|
||||
const MergeTreeDataSelectExecutor & reader,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Names & required_columns,
|
||||
ProjectionCandidate & candidate,
|
||||
ContextPtr query_context,
|
||||
const PartitionIdToMaxBlock * max_added_blocks,
|
||||
const Settings & settings,
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
ProjectionCandidate *& selected_candidate,
|
||||
size_t & min_sum_marks)
|
||||
{
|
||||
MergeTreeData::DataPartsVector projection_parts;
|
||||
MergeTreeData::DataPartsVector normal_parts;
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
const auto & projections = part->getProjectionParts();
|
||||
auto it = projections.find(candidate.desc->name);
|
||||
if (it != projections.end())
|
||||
projection_parts.push_back(it->second);
|
||||
else
|
||||
normal_parts.push_back(part);
|
||||
}
|
||||
|
||||
if (projection_parts.empty())
|
||||
return;
|
||||
|
||||
candidate.merge_tree_data_select_base_cache = std::make_unique<MergeTreeDataSelectCache>();
|
||||
candidate.merge_tree_data_select_projection_cache = std::make_unique<MergeTreeDataSelectCache>();
|
||||
reader.readFromParts(
|
||||
projection_parts,
|
||||
candidate.required_columns,
|
||||
metadata_snapshot,
|
||||
candidate.desc->metadata,
|
||||
query_info, // TODO syntax_analysis_result set in index
|
||||
query_context,
|
||||
0, // max_block_size is unused when getting cache
|
||||
settings.max_threads,
|
||||
max_added_blocks,
|
||||
candidate.merge_tree_data_select_projection_cache.get());
|
||||
|
||||
size_t sum_marks = candidate.merge_tree_data_select_projection_cache->sum_marks;
|
||||
if (normal_parts.empty())
|
||||
{
|
||||
// All parts are projection parts which allows us to use in_order_optimization.
|
||||
// TODO It might be better to use a complete projection even with more marks to read.
|
||||
candidate.complete = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
reader.readFromParts(
|
||||
normal_parts,
|
||||
required_columns,
|
||||
metadata_snapshot,
|
||||
metadata_snapshot,
|
||||
query_info, // TODO syntax_analysis_result set in index
|
||||
query_context,
|
||||
0, // max_block_size is unused when getting cache
|
||||
settings.max_threads,
|
||||
max_added_blocks,
|
||||
candidate.merge_tree_data_select_base_cache.get());
|
||||
sum_marks += candidate.merge_tree_data_select_base_cache->sum_marks;
|
||||
}
|
||||
|
||||
// We choose the projection with least sum_marks to read.
|
||||
if (sum_marks < min_sum_marks)
|
||||
{
|
||||
selected_candidate = &candidate;
|
||||
min_sum_marks = sum_marks;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
bool MergeTreeData::getQueryProcessingStageWithAggregateProjection(
|
||||
ContextPtr query_context, const StorageMetadataPtr & metadata_snapshot, SelectQueryInfo & query_info) const
|
||||
{
|
||||
const auto & settings = query_context->getSettingsRef();
|
||||
if (!settings.allow_experimental_projection_optimization || query_info.ignore_projections)
|
||||
return false;
|
||||
|
||||
const auto & query_ptr = query_info.query;
|
||||
|
||||
InterpreterSelectQuery select(
|
||||
query_ptr, query_context, SelectQueryOptions{QueryProcessingStage::WithMergeableState}.ignoreProjections().ignoreAlias());
|
||||
const auto & analysis_result = select.getAnalysisResult();
|
||||
|
||||
bool can_use_aggregate_projection = true;
|
||||
/// If the first stage of the query pipeline is more complex than Aggregating - Expression - Filter - ReadFromStorage,
|
||||
/// we cannot use aggregate projection.
|
||||
if (analysis_result.join != nullptr || analysis_result.array_join != nullptr)
|
||||
can_use_aggregate_projection = false;
|
||||
|
||||
/// Check if all needed columns can be provided by some aggregate projection. Here we also try
|
||||
/// to find expression matches. For example, suppose an aggregate projection contains a column
|
||||
/// named sum(x) and the given query also has an expression called sum(x), it's a match. This is
|
||||
/// why we need to ignore all aliases during projection creation and the above query planning.
|
||||
/// It's also worth noting that, sqrt(sum(x)) will also work because we can treat sum(x) as a
|
||||
/// required column.
|
||||
|
||||
/// The ownership of ProjectionDescription is hold in metadata_snapshot which lives along with
|
||||
/// InterpreterSelect, thus we can store the raw pointer here.
|
||||
std::vector<ProjectionCandidate> candidates;
|
||||
NameSet keys;
|
||||
std::unordered_map<std::string_view, size_t> key_name_pos_map;
|
||||
size_t pos = 0;
|
||||
for (const auto & desc : select.getQueryAnalyzer()->aggregationKeys())
|
||||
{
|
||||
keys.insert(desc.name);
|
||||
key_name_pos_map.insert({desc.name, pos++});
|
||||
}
|
||||
auto actions_settings = ExpressionActionsSettings::fromSettings(settings);
|
||||
|
||||
// All required columns should be provided by either current projection or previous actions
|
||||
// Let's traverse backward to finish the check.
|
||||
// TODO what if there is a column with name sum(x) and an aggregate sum(x)?
|
||||
auto rewrite_before_where =
|
||||
[&](ProjectionCandidate & candidate, const ProjectionDescription & projection,
|
||||
NameSet & required_columns, const Block & source_block, const Block & aggregates)
|
||||
{
|
||||
if (analysis_result.before_where)
|
||||
{
|
||||
candidate.where_column_name = analysis_result.where_column_name;
|
||||
candidate.remove_where_filter = analysis_result.remove_where_filter;
|
||||
candidate.before_where = analysis_result.before_where->clone();
|
||||
// std::cerr << fmt::format("before_where_actions = \n{}", candidate.before_where->dumpDAG()) << std::endl;
|
||||
required_columns = candidate.before_where->foldActionsByProjection(
|
||||
required_columns,
|
||||
projection.sample_block_for_keys,
|
||||
candidate.where_column_name);
|
||||
// std::cerr << fmt::format("before_where_actions = \n{}", candidate.before_where->dumpDAG()) << std::endl;
|
||||
// std::cerr << fmt::format("where_required_columns = \n{}", fmt::join(required_columns, ", ")) << std::endl;
|
||||
|
||||
if (required_columns.empty())
|
||||
return false;
|
||||
candidate.before_where->addAggregatesViaProjection(aggregates);
|
||||
}
|
||||
|
||||
if (analysis_result.prewhere_info)
|
||||
{
|
||||
const auto & prewhere_info = analysis_result.prewhere_info;
|
||||
candidate.prewhere_info = std::make_shared<PrewhereInfo>();
|
||||
candidate.prewhere_info->prewhere_column_name = prewhere_info->prewhere_column_name;
|
||||
candidate.prewhere_info->remove_prewhere_column = prewhere_info->remove_prewhere_column;
|
||||
// std::cerr << fmt::format("remove prewhere column : {}", candidate.prewhere_info->remove_prewhere_column) << std::endl;
|
||||
candidate.prewhere_info->row_level_column_name = prewhere_info->row_level_column_name;
|
||||
candidate.prewhere_info->need_filter = prewhere_info->need_filter;
|
||||
|
||||
auto prewhere_actions = prewhere_info->prewhere_actions->clone();
|
||||
auto prewhere_required_columns = required_columns;
|
||||
// required_columns should not contain columns generated by prewhere
|
||||
for (const auto & column : prewhere_actions->getResultColumns())
|
||||
required_columns.erase(column.name);
|
||||
// std::cerr << fmt::format("prewhere_actions = \n{}", prewhere_actions->dumpDAG()) << std::endl;
|
||||
// Prewhere_action should not add missing keys.
|
||||
prewhere_required_columns = prewhere_actions->foldActionsByProjection(
|
||||
prewhere_required_columns, projection.sample_block_for_keys, prewhere_info->prewhere_column_name, false);
|
||||
// std::cerr << fmt::format("prewhere_actions = \n{}", prewhere_actions->dumpDAG()) << std::endl;
|
||||
// std::cerr << fmt::format("prewhere_required_columns = \n{}", fmt::join(prewhere_required_columns, ", ")) << std::endl;
|
||||
if (prewhere_required_columns.empty())
|
||||
return false;
|
||||
candidate.prewhere_info->prewhere_actions = std::make_shared<ExpressionActions>(prewhere_actions, actions_settings);
|
||||
|
||||
if (prewhere_info->row_level_filter_actions)
|
||||
{
|
||||
auto row_level_filter_actions = prewhere_info->row_level_filter_actions->clone();
|
||||
prewhere_required_columns = row_level_filter_actions->foldActionsByProjection(
|
||||
prewhere_required_columns, projection.sample_block_for_keys, prewhere_info->row_level_column_name, false);
|
||||
// std::cerr << fmt::format("row_level_filter_required_columns = \n{}", fmt::join(prewhere_required_columns, ", ")) << std::endl;
|
||||
if (prewhere_required_columns.empty())
|
||||
return false;
|
||||
candidate.prewhere_info->row_level_filter
|
||||
= std::make_shared<ExpressionActions>(row_level_filter_actions, actions_settings);
|
||||
}
|
||||
|
||||
if (prewhere_info->alias_actions)
|
||||
{
|
||||
auto alias_actions = prewhere_info->alias_actions->clone();
|
||||
// std::cerr << fmt::format("alias_actions = \n{}", alias_actions->dumpDAG()) << std::endl;
|
||||
prewhere_required_columns
|
||||
= alias_actions->foldActionsByProjection(prewhere_required_columns, projection.sample_block_for_keys, {}, false);
|
||||
// std::cerr << fmt::format("alias_actions = \n{}", alias_actions->dumpDAG()) << std::endl;
|
||||
// std::cerr << fmt::format("alias_required_columns = \n{}", fmt::join(prewhere_required_columns, ", ")) << std::endl;
|
||||
if (prewhere_required_columns.empty())
|
||||
return false;
|
||||
candidate.prewhere_info->alias_actions = std::make_shared<ExpressionActions>(alias_actions, actions_settings);
|
||||
}
|
||||
required_columns.insert(prewhere_required_columns.begin(), prewhere_required_columns.end());
|
||||
}
|
||||
|
||||
bool match = true;
|
||||
for (const auto & column : required_columns)
|
||||
{
|
||||
/// There are still missing columns, fail to match
|
||||
if (!source_block.has(column))
|
||||
{
|
||||
match = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
return match;
|
||||
};
|
||||
|
||||
for (const auto & projection : metadata_snapshot->projections)
|
||||
{
|
||||
ProjectionCandidate candidate{};
|
||||
candidate.desc = &projection;
|
||||
|
||||
if (projection.type == ProjectionDescription::Type::Aggregate && analysis_result.need_aggregate && can_use_aggregate_projection)
|
||||
{
|
||||
// std::cerr << fmt::format("====== aggregate projection analysis: {} ======", projection.name) << std::endl;
|
||||
bool match = true;
|
||||
Block aggregates;
|
||||
// Let's first check if all aggregates are provided by current projection
|
||||
for (const auto & aggregate : select.getQueryAnalyzer()->aggregates())
|
||||
{
|
||||
const auto * column = projection.sample_block.findByName(aggregate.column_name);
|
||||
if (column)
|
||||
{
|
||||
aggregates.insert(*column);
|
||||
}
|
||||
else
|
||||
{
|
||||
match = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!match)
|
||||
continue;
|
||||
|
||||
// Check if all aggregation keys can be either provided by some action, or by current
|
||||
// projection directly. Reshape the `before_aggregation` action DAG so that it only
|
||||
// needs to provide aggregation keys, and certain children DAG might be substituted by
|
||||
// some keys in projection.
|
||||
candidate.before_aggregation = analysis_result.before_aggregation->clone();
|
||||
// std::cerr << fmt::format("keys = {}", fmt::join(keys, ", ")) << std::endl;
|
||||
// std::cerr << fmt::format("before_aggregation = \n{}", candidate.before_aggregation->dumpDAG()) << std::endl;
|
||||
auto required_columns = candidate.before_aggregation->foldActionsByProjection(keys, projection.sample_block_for_keys);
|
||||
// std::cerr << fmt::format("before_aggregation = \n{}", candidate.before_aggregation->dumpDAG()) << std::endl;
|
||||
// std::cerr << fmt::format("aggregate_required_columns = \n{}", fmt::join(required_columns, ", ")) << std::endl;
|
||||
if (required_columns.empty())
|
||||
continue;
|
||||
|
||||
if (analysis_result.optimize_aggregation_in_order)
|
||||
{
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
auto actions_dag = analysis_result.before_aggregation->clone();
|
||||
actions_dag->foldActionsByProjection({key}, projection.sample_block_for_keys);
|
||||
candidate.group_by_elements_actions.emplace_back(std::make_shared<ExpressionActions>(actions_dag, actions_settings));
|
||||
}
|
||||
}
|
||||
|
||||
// Reorder aggregation keys and attach aggregates
|
||||
candidate.before_aggregation->reorderAggregationKeysForProjection(key_name_pos_map);
|
||||
candidate.before_aggregation->addAggregatesViaProjection(aggregates);
|
||||
|
||||
if (rewrite_before_where(candidate, projection, required_columns, projection.sample_block_for_keys, aggregates))
|
||||
{
|
||||
candidate.required_columns = {required_columns.begin(), required_columns.end()};
|
||||
for (const auto & aggregate : aggregates)
|
||||
candidate.required_columns.push_back(aggregate.name);
|
||||
candidates.push_back(std::move(candidate));
|
||||
}
|
||||
// std::cerr << fmt::format("====== aggregate projection analysis end: {} ======", projection.name) << std::endl;
|
||||
}
|
||||
|
||||
if (projection.type == ProjectionDescription::Type::Normal && (analysis_result.hasWhere() || analysis_result.hasPrewhere()))
|
||||
{
|
||||
// std::cerr << fmt::format("====== normal projection analysis: {} ======", projection.name) << std::endl;
|
||||
const auto & actions
|
||||
= analysis_result.before_aggregation ? analysis_result.before_aggregation : analysis_result.before_order_by;
|
||||
NameSet required_columns;
|
||||
for (const auto & column : actions->getRequiredColumns())
|
||||
required_columns.insert(column.name);
|
||||
|
||||
if (rewrite_before_where(candidate, projection, required_columns, projection.sample_block, {}))
|
||||
{
|
||||
candidate.required_columns = {required_columns.begin(), required_columns.end()};
|
||||
candidates.push_back(std::move(candidate));
|
||||
}
|
||||
// std::cerr << fmt::format("====== normal projection analysis end: {} ======", projection.name) << std::endl;
|
||||
}
|
||||
}
|
||||
|
||||
// Let's select the best projection to execute the query.
|
||||
if (!candidates.empty())
|
||||
{
|
||||
// First build a MergeTreeDataSelectCache to check if a projection is indeed better than base
|
||||
query_info.merge_tree_data_select_cache = std::make_unique<MergeTreeDataSelectCache>();
|
||||
|
||||
std::unique_ptr<PartitionIdToMaxBlock> max_added_blocks;
|
||||
if (settings.select_sequential_consistency)
|
||||
{
|
||||
if (const StorageReplicatedMergeTree * replicated = dynamic_cast<const StorageReplicatedMergeTree *>(this))
|
||||
max_added_blocks = std::make_unique<PartitionIdToMaxBlock>(replicated->getMaxAddedBlocks());
|
||||
}
|
||||
|
||||
auto parts = getDataPartsVector();
|
||||
MergeTreeDataSelectExecutor reader(*this);
|
||||
reader.readFromParts(
|
||||
parts,
|
||||
analysis_result.required_columns,
|
||||
metadata_snapshot,
|
||||
metadata_snapshot,
|
||||
query_info, // TODO syntax_analysis_result set in index
|
||||
query_context,
|
||||
0, // max_block_size is unused when getting cache
|
||||
settings.max_threads,
|
||||
max_added_blocks.get(),
|
||||
query_info.merge_tree_data_select_cache.get());
|
||||
|
||||
// Add 1 to base sum_marks so that we prefer projections even when they have equal number of marks to read.
|
||||
size_t min_sum_marks = query_info.merge_tree_data_select_cache->sum_marks + 1;
|
||||
ProjectionCandidate * selected_candidate = nullptr;
|
||||
/// Favor aggregate projections
|
||||
for (auto & candidate : candidates)
|
||||
{
|
||||
if (candidate.desc->type == ProjectionDescription::Type::Aggregate)
|
||||
{
|
||||
selectBestProjection(
|
||||
reader,
|
||||
metadata_snapshot,
|
||||
query_info,
|
||||
analysis_result.required_columns,
|
||||
candidate,
|
||||
query_context,
|
||||
max_added_blocks.get(),
|
||||
settings,
|
||||
parts,
|
||||
selected_candidate,
|
||||
min_sum_marks);
|
||||
}
|
||||
}
|
||||
|
||||
/// Select the best normal projection if no aggregate projection is available
|
||||
if (!selected_candidate)
|
||||
{
|
||||
for (auto & candidate : candidates)
|
||||
{
|
||||
if (candidate.desc->type == ProjectionDescription::Type::Normal)
|
||||
{
|
||||
selectBestProjection(
|
||||
reader,
|
||||
metadata_snapshot,
|
||||
query_info,
|
||||
analysis_result.required_columns,
|
||||
candidate,
|
||||
query_context,
|
||||
max_added_blocks.get(),
|
||||
settings,
|
||||
parts,
|
||||
selected_candidate,
|
||||
min_sum_marks);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!selected_candidate)
|
||||
return false;
|
||||
|
||||
if (selected_candidate->desc->type == ProjectionDescription::Type::Aggregate)
|
||||
{
|
||||
selected_candidate->aggregation_keys = select.getQueryAnalyzer()->aggregationKeys();
|
||||
selected_candidate->aggregate_descriptions = select.getQueryAnalyzer()->aggregates();
|
||||
}
|
||||
|
||||
query_info.projection = std::move(*selected_candidate);
|
||||
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage(
|
||||
ContextPtr query_context,
|
||||
QueryProcessingStage::Enum to_stage,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
SelectQueryInfo & query_info) const
|
||||
{
|
||||
if (to_stage >= QueryProcessingStage::Enum::WithMergeableState)
|
||||
{
|
||||
if (getQueryProcessingStageWithAggregateProjection(query_context, metadata_snapshot, query_info))
|
||||
{
|
||||
if (query_info.projection->desc->type == ProjectionDescription::Type::Aggregate)
|
||||
return QueryProcessingStage::Enum::WithMergeableState;
|
||||
}
|
||||
}
|
||||
|
||||
return QueryProcessingStage::Enum::FetchColumns;
|
||||
}
|
||||
|
||||
|
||||
MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const
|
||||
{
|
||||
MergeTreeData * src_data = dynamic_cast<MergeTreeData *>(&source_table);
|
||||
@ -4267,6 +4738,18 @@ bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagge
|
||||
return true;
|
||||
}
|
||||
|
||||
bool MergeTreeData::partsContainSameProjections(const DataPartPtr & left, const DataPartPtr & right)
|
||||
{
|
||||
if (left->getProjectionParts().size() != right->getProjectionParts().size())
|
||||
return false;
|
||||
for (const auto & [name, _] : left->getProjectionParts())
|
||||
{
|
||||
if (!right->hasProjection(name))
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, String * out_reason) const
|
||||
{
|
||||
if (!canUseAdaptiveGranularity())
|
||||
|
@ -51,6 +51,7 @@ struct EmergingPartInfo
|
||||
|
||||
struct CurrentlySubmergingEmergingTagger;
|
||||
|
||||
struct SelectQueryOptions;
|
||||
class ExpressionActions;
|
||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
using ManyExpressionActions = std::vector<ExpressionActionsPtr>;
|
||||
@ -216,15 +217,15 @@ public:
|
||||
/// After this method setColumns must be called
|
||||
MutableDataPartPtr createPart(const String & name,
|
||||
MergeTreeDataPartType type, const MergeTreePartInfo & part_info,
|
||||
const VolumePtr & volume, const String & relative_path) const;
|
||||
const VolumePtr & volume, const String & relative_path, const IMergeTreeDataPart * parent_part = nullptr) const;
|
||||
|
||||
/// Create part, that already exists on filesystem.
|
||||
/// After this methods 'loadColumnsChecksumsIndexes' must be called.
|
||||
MutableDataPartPtr createPart(const String & name,
|
||||
const VolumePtr & volume, const String & relative_path) const;
|
||||
const VolumePtr & volume, const String & relative_path, const IMergeTreeDataPart * parent_part = nullptr) const;
|
||||
|
||||
MutableDataPartPtr createPart(const String & name, const MergeTreePartInfo & part_info,
|
||||
const VolumePtr & volume, const String & relative_path) const;
|
||||
const VolumePtr & volume, const String & relative_path, const IMergeTreeDataPart * parent_part = nullptr) const;
|
||||
|
||||
/// Auxiliary object to add a set of parts into the working set in two steps:
|
||||
/// * First, as PreCommitted parts (the parts are ready, but not yet in the active set).
|
||||
@ -357,6 +358,18 @@ public:
|
||||
bool attach,
|
||||
BrokenPartCallback broken_part_callback_ = [](const String &){});
|
||||
|
||||
bool getQueryProcessingStageWithAggregateProjection(
|
||||
ContextPtr query_context, const StorageMetadataPtr & metadata_snapshot, SelectQueryInfo & query_info) const;
|
||||
|
||||
QueryProcessingStage::Enum getQueryProcessingStage(
|
||||
ContextPtr query_context,
|
||||
QueryProcessingStage::Enum to_stage,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
SelectQueryInfo & info) const override;
|
||||
|
||||
ReservationPtr reserveSpace(UInt64 expected_size, VolumePtr & volume) const;
|
||||
|
||||
static bool partsContainSameProjections(const DataPartPtr & left, const DataPartPtr & right);
|
||||
|
||||
StoragePolicyPtr getStoragePolicy() const override;
|
||||
|
||||
@ -388,10 +401,11 @@ public:
|
||||
DataParts getDataParts(const DataPartStates & affordable_states) const;
|
||||
/// Returns sorted list of the parts with specified states
|
||||
/// out_states will contain snapshot of each part state
|
||||
DataPartsVector getDataPartsVector(const DataPartStates & affordable_states, DataPartStateVector * out_states = nullptr) const;
|
||||
DataPartsVector getDataPartsVector(
|
||||
const DataPartStates & affordable_states, DataPartStateVector * out_states = nullptr, bool require_projection_parts = false) const;
|
||||
|
||||
/// Returns absolutely all parts (and snapshot of their states)
|
||||
DataPartsVector getAllDataPartsVector(DataPartStateVector * out_states = nullptr) const;
|
||||
DataPartsVector getAllDataPartsVector(DataPartStateVector * out_states = nullptr, bool require_projection_parts = false) const;
|
||||
|
||||
/// Returns all detached parts
|
||||
DetachedPartsInfo getDetachedParts() const;
|
||||
|
@ -7,12 +7,14 @@
|
||||
#include <Storages/MergeTree/AllMergeSelector.h>
|
||||
#include <Storages/MergeTree/TTLMergeSelector.h>
|
||||
#include <Storages/MergeTree/MergeList.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataWriter.h>
|
||||
#include <Storages/MergeTree/StorageFromMergeTreeDataPart.h>
|
||||
#include <DataStreams/TTLBlockInputStream.h>
|
||||
#include <DataStreams/DistinctSortedBlockInputStream.h>
|
||||
#include <DataStreams/ExpressionBlockInputStream.h>
|
||||
#include <DataStreams/MaterializingBlockInputStream.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
#include <DataStreams/SquashingBlockInputStream.h>
|
||||
#include <Processors/Merges/MergingSortedTransform.h>
|
||||
#include <Processors/Merges/CollapsingSortedTransform.h>
|
||||
#include <Processors/Merges/SummingSortedTransform.h>
|
||||
@ -20,6 +22,7 @@
|
||||
#include <Processors/Merges/GraphiteRollupSortedTransform.h>
|
||||
#include <Processors/Merges/AggregatingSortedTransform.h>
|
||||
#include <Processors/Merges/VersionedCollapsingTransform.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include <Processors/Transforms/ExpressionTransform.h>
|
||||
#include <Processors/Transforms/MaterializingTransform.h>
|
||||
#include <Processors/Executors/PipelineExecutingBlockInputStream.h>
|
||||
@ -472,6 +475,7 @@ static void extractMergingAndGatheringColumns(
|
||||
const NamesAndTypesList & storage_columns,
|
||||
const ExpressionActionsPtr & sorting_key_expr,
|
||||
const IndicesDescription & indexes,
|
||||
const ProjectionsDescription & projections,
|
||||
const MergeTreeData::MergingParams & merging_params,
|
||||
NamesAndTypesList & gathering_columns, Names & gathering_column_names,
|
||||
NamesAndTypesList & merging_columns, Names & merging_column_names)
|
||||
@ -485,6 +489,13 @@ static void extractMergingAndGatheringColumns(
|
||||
std::inserter(key_columns, key_columns.end()));
|
||||
}
|
||||
|
||||
for (const auto & projection : projections)
|
||||
{
|
||||
Names projection_columns_vec = projection.required_columns;
|
||||
std::copy(projection_columns_vec.cbegin(), projection_columns_vec.cend(),
|
||||
std::inserter(key_columns, key_columns.end()));
|
||||
}
|
||||
|
||||
/// Force sign column for Collapsing mode
|
||||
if (merging_params.mode == MergeTreeData::MergingParams::Collapsing)
|
||||
key_columns.emplace(merging_params.sign_column);
|
||||
@ -649,14 +660,17 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
const FutureMergedMutatedPart & future_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
MergeList::Entry & merge_entry,
|
||||
TableLockHolder &,
|
||||
TableLockHolder & holder,
|
||||
time_t time_of_merge,
|
||||
ContextPtr context,
|
||||
const ReservationPtr & space_reservation,
|
||||
bool deduplicate,
|
||||
const Names & deduplicate_by_columns)
|
||||
const Names & deduplicate_by_columns,
|
||||
const MergeTreeData::MergingParams & merging_params,
|
||||
const IMergeTreeDataPart * parent_part,
|
||||
const String & prefix)
|
||||
{
|
||||
static const String TMP_PREFIX = "tmp_merge_";
|
||||
const String tmp_prefix = parent_part ? prefix : "tmp_merge_";
|
||||
|
||||
if (merges_blocker.isCancelled())
|
||||
throw Exception("Cancelled merging parts", ErrorCodes::ABORTED);
|
||||
@ -679,7 +693,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
|
||||
auto disk = space_reservation->getDisk();
|
||||
String part_path = data.relative_data_path;
|
||||
String new_part_tmp_path = part_path + TMP_PREFIX + future_part.name + "/";
|
||||
String new_part_tmp_path = part_path + tmp_prefix + future_part.name + (parent_part ? ".proj" : "") + "/";
|
||||
if (disk->exists(new_part_tmp_path))
|
||||
throw Exception("Directory " + fullPath(disk, new_part_tmp_path) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS);
|
||||
|
||||
@ -695,7 +709,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
storage_columns,
|
||||
metadata_snapshot->getSortingKey().expression,
|
||||
metadata_snapshot->getSecondaryIndices(),
|
||||
data.merging_params,
|
||||
metadata_snapshot->getProjections(),
|
||||
merging_params,
|
||||
gathering_columns,
|
||||
gathering_column_names,
|
||||
merging_columns,
|
||||
@ -707,12 +722,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
future_part.type,
|
||||
future_part.part_info,
|
||||
single_disk_volume,
|
||||
TMP_PREFIX + future_part.name);
|
||||
tmp_prefix + future_part.name + (parent_part ? ".proj" : ""),
|
||||
parent_part);
|
||||
|
||||
new_data_part->uuid = future_part.uuid;
|
||||
new_data_part->setColumns(storage_columns);
|
||||
new_data_part->partition.assign(future_part.getPartition());
|
||||
new_data_part->is_temp = true;
|
||||
new_data_part->is_temp = parent_part == nullptr;
|
||||
|
||||
bool need_remove_expired_values = false;
|
||||
bool force_ttl = false;
|
||||
@ -739,7 +755,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
|
||||
size_t sum_input_rows_upper_bound = merge_entry->total_rows_count;
|
||||
size_t sum_compressed_bytes_upper_bound = merge_entry->total_size_bytes_compressed;
|
||||
MergeAlgorithm chosen_merge_algorithm = chooseMergeAlgorithm(parts, sum_input_rows_upper_bound, gathering_columns, deduplicate, need_remove_expired_values);
|
||||
MergeAlgorithm chosen_merge_algorithm = chooseMergeAlgorithm(
|
||||
parts, sum_input_rows_upper_bound, gathering_columns, deduplicate, need_remove_expired_values, merging_params);
|
||||
merge_entry->merge_algorithm.store(chosen_merge_algorithm, std::memory_order_relaxed);
|
||||
|
||||
LOG_DEBUG(log, "Selected MergeAlgorithm: {}", toString(chosen_merge_algorithm));
|
||||
@ -852,7 +869,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
bool blocks_are_granules_size = (chosen_merge_algorithm == MergeAlgorithm::Vertical);
|
||||
|
||||
UInt64 merge_block_size = data_settings->merge_max_block_size;
|
||||
switch (data.merging_params.mode)
|
||||
switch (merging_params.mode)
|
||||
{
|
||||
case MergeTreeData::MergingParams::Ordinary:
|
||||
merged_transform = std::make_unique<MergingSortedTransform>(
|
||||
@ -861,13 +878,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
|
||||
case MergeTreeData::MergingParams::Collapsing:
|
||||
merged_transform = std::make_unique<CollapsingSortedTransform>(
|
||||
header, pipes.size(), sort_description, data.merging_params.sign_column, false,
|
||||
header, pipes.size(), sort_description, merging_params.sign_column, false,
|
||||
merge_block_size, rows_sources_write_buf.get(), blocks_are_granules_size);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Summing:
|
||||
merged_transform = std::make_unique<SummingSortedTransform>(
|
||||
header, pipes.size(), sort_description, data.merging_params.columns_to_sum, partition_key_columns, merge_block_size);
|
||||
header, pipes.size(), sort_description, merging_params.columns_to_sum, partition_key_columns, merge_block_size);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Aggregating:
|
||||
@ -877,19 +894,19 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
|
||||
case MergeTreeData::MergingParams::Replacing:
|
||||
merged_transform = std::make_unique<ReplacingSortedTransform>(
|
||||
header, pipes.size(), sort_description, data.merging_params.version_column,
|
||||
header, pipes.size(), sort_description, merging_params.version_column,
|
||||
merge_block_size, rows_sources_write_buf.get(), blocks_are_granules_size);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Graphite:
|
||||
merged_transform = std::make_unique<GraphiteRollupSortedTransform>(
|
||||
header, pipes.size(), sort_description, merge_block_size,
|
||||
data.merging_params.graphite_params, time_of_merge);
|
||||
merging_params.graphite_params, time_of_merge);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::VersionedCollapsing:
|
||||
merged_transform = std::make_unique<VersionedCollapsingTransform>(
|
||||
header, pipes.size(), sort_description, data.merging_params.sign_column,
|
||||
header, pipes.size(), sort_description, merging_params.sign_column,
|
||||
merge_block_size, rows_sources_write_buf.get(), blocks_are_granules_size);
|
||||
break;
|
||||
}
|
||||
@ -1084,6 +1101,55 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
ReadableSize(merge_entry->bytes_read_uncompressed / elapsed_seconds));
|
||||
}
|
||||
|
||||
for (const auto & projection : metadata_snapshot->getProjections())
|
||||
{
|
||||
MergeTreeData::DataPartsVector projection_parts;
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
auto it = part->getProjectionParts().find(projection.name);
|
||||
if (it != part->getProjectionParts().end())
|
||||
projection_parts.push_back(it->second);
|
||||
}
|
||||
if (projection_parts.size() < parts.size())
|
||||
{
|
||||
LOG_DEBUG(log, "Projection {} is not merged because some parts don't have it", projection.name);
|
||||
continue;
|
||||
}
|
||||
|
||||
LOG_DEBUG(
|
||||
log,
|
||||
"Selected {} projection_parts from {} to {}",
|
||||
projection_parts.size(),
|
||||
projection_parts.front()->name,
|
||||
projection_parts.back()->name);
|
||||
|
||||
FutureMergedMutatedPart projection_future_part;
|
||||
projection_future_part.assign(std::move(projection_parts));
|
||||
projection_future_part.name = projection.name;
|
||||
projection_future_part.path = future_part.path + "/" + projection.name + ".proj/";
|
||||
projection_future_part.part_info = {"all", 0, 0, 0};
|
||||
|
||||
MergeTreeData::MergingParams projection_merging_params;
|
||||
projection_merging_params.mode = MergeTreeData::MergingParams::Ordinary;
|
||||
if (projection.type == ProjectionDescription::Type::Aggregate)
|
||||
projection_merging_params.mode = MergeTreeData::MergingParams::Aggregating;
|
||||
|
||||
// TODO Should we use a new merge_entry for projection?
|
||||
auto merged_projection_part = mergePartsToTemporaryPart(
|
||||
projection_future_part,
|
||||
projection.metadata,
|
||||
merge_entry,
|
||||
holder,
|
||||
time_of_merge,
|
||||
context,
|
||||
space_reservation,
|
||||
deduplicate,
|
||||
deduplicate_by_columns,
|
||||
projection_merging_params,
|
||||
new_data_part.get());
|
||||
new_data_part->addProjectionPart(projection.name, std::move(merged_projection_part));
|
||||
}
|
||||
|
||||
if (chosen_merge_algorithm != MergeAlgorithm::Vertical)
|
||||
to.writeSuffixAndFinalizePart(new_data_part, need_sync);
|
||||
else
|
||||
@ -1101,7 +1167,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
time_t time_of_mutation,
|
||||
ContextPtr context,
|
||||
const ReservationPtr & space_reservation,
|
||||
TableLockHolder &)
|
||||
TableLockHolder & holder)
|
||||
{
|
||||
checkOperationIsNotCanceled(merge_entry);
|
||||
|
||||
@ -1153,11 +1219,18 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
MergeStageProgress stage_progress(1.0);
|
||||
|
||||
NamesAndTypesList storage_columns = metadata_snapshot->getColumns().getAllPhysical();
|
||||
NameSet materialized_indices;
|
||||
NameSet materialized_projections;
|
||||
MutationsInterpreter::MutationKind::MutationKindEnum mutation_kind
|
||||
= MutationsInterpreter::MutationKind::MutationKindEnum::MUTATE_UNKNOWN;
|
||||
|
||||
if (!for_interpreter.empty())
|
||||
{
|
||||
interpreter = std::make_unique<MutationsInterpreter>(
|
||||
storage_from_source_part, metadata_snapshot, for_interpreter, context_for_reading, true);
|
||||
materialized_indices = interpreter->grabMaterializedIndices();
|
||||
materialized_projections = interpreter->grabMaterializedProjections();
|
||||
mutation_kind = interpreter->getMutationKind();
|
||||
in = interpreter->execute();
|
||||
updated_header = interpreter->getUpdatedHeader();
|
||||
in->setProgressCallback(MergeProgressCallback(merge_entry, watch_prev_elapsed, stage_progress));
|
||||
@ -1179,8 +1252,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
auto disk = new_data_part->volume->getDisk();
|
||||
String new_part_tmp_path = new_data_part->getFullRelativePath();
|
||||
|
||||
disk->createDirectories(new_part_tmp_path);
|
||||
|
||||
SyncGuardPtr sync_guard;
|
||||
if (data.getSettings()->fsync_part_directory)
|
||||
sync_guard = disk->getDirectorySyncGuard(new_part_tmp_path);
|
||||
@ -1195,8 +1266,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
need_remove_expired_values = true;
|
||||
|
||||
/// All columns from part are changed and may be some more that were missing before in part
|
||||
if (!isWidePart(source_part) || (interpreter && interpreter->isAffectingAllColumns()))
|
||||
if (!isWidePart(source_part)
|
||||
|| (mutation_kind == MutationsInterpreter::MutationKind::MUTATE_OTHER && interpreter && interpreter->isAffectingAllColumns()))
|
||||
{
|
||||
disk->createDirectories(new_part_tmp_path);
|
||||
|
||||
/// Note: this is done before creating input streams, because otherwise data.data_parts_mutex
|
||||
/// (which is locked in data.getTotalActiveSizeInBytes())
|
||||
/// (which is locked in shared mode when input streams are created) and when inserting new data
|
||||
@ -1205,16 +1279,22 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
auto compression_codec = data.getCompressionCodecForPart(source_part->getBytesOnDisk(), source_part->ttl_infos, time_of_mutation);
|
||||
|
||||
auto part_indices = getIndicesForNewDataPart(metadata_snapshot->getSecondaryIndices(), for_file_renames);
|
||||
auto part_projections = getProjectionsForNewDataPart(metadata_snapshot->getProjections(), for_file_renames);
|
||||
|
||||
mutateAllPartColumns(
|
||||
new_data_part,
|
||||
metadata_snapshot,
|
||||
part_indices,
|
||||
part_projections,
|
||||
in,
|
||||
time_of_mutation,
|
||||
compression_codec,
|
||||
merge_entry,
|
||||
need_remove_expired_values,
|
||||
need_sync);
|
||||
need_sync,
|
||||
space_reservation,
|
||||
holder,
|
||||
context);
|
||||
|
||||
/// no finalization required, because mutateAllPartColumns use
|
||||
/// MergedBlockOutputStream which finilaze all part fields itself
|
||||
@ -1222,13 +1302,39 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
else /// TODO: check that we modify only non-key columns in this case.
|
||||
{
|
||||
/// We will modify only some of the columns. Other columns and key values can be copied as-is.
|
||||
auto indices_to_recalc = getIndicesToRecalculate(in, updated_header.getNamesAndTypesList(), metadata_snapshot, context);
|
||||
NameSet updated_columns;
|
||||
if (mutation_kind != MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION)
|
||||
{
|
||||
for (const auto & name_type : updated_header.getNamesAndTypesList())
|
||||
updated_columns.emplace(name_type.name);
|
||||
}
|
||||
|
||||
NameSet files_to_skip = collectFilesToSkip(source_part, updated_header, indices_to_recalc, mrk_extension);
|
||||
auto indices_to_recalc = getIndicesToRecalculate(
|
||||
in, updated_columns, metadata_snapshot, context, materialized_indices, source_part);
|
||||
auto projections_to_recalc = getProjectionsToRecalculate(
|
||||
updated_columns, metadata_snapshot, materialized_projections, source_part);
|
||||
|
||||
NameSet files_to_skip = collectFilesToSkip(
|
||||
source_part,
|
||||
mutation_kind == MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION ? Block{} : updated_header,
|
||||
indices_to_recalc,
|
||||
mrk_extension,
|
||||
projections_to_recalc);
|
||||
NameToNameVector files_to_rename = collectFilesForRenames(source_part, for_file_renames, mrk_extension);
|
||||
|
||||
if (indices_to_recalc.empty() && projections_to_recalc.empty() && mutation_kind != MutationsInterpreter::MutationKind::MUTATE_OTHER
|
||||
&& files_to_rename.empty())
|
||||
{
|
||||
LOG_TRACE(
|
||||
log, "Part {} doesn't change up to mutation version {} (optimized)", source_part->name, future_part.part_info.mutation);
|
||||
return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info, metadata_snapshot);
|
||||
}
|
||||
|
||||
if (need_remove_expired_values)
|
||||
files_to_skip.insert("ttl.txt");
|
||||
|
||||
disk->createDirectories(new_part_tmp_path);
|
||||
|
||||
/// Create hardlinks for unchanged files
|
||||
for (auto it = disk->iterateDirectory(source_part->getFullRelativePath()); it->isValid(); it->next())
|
||||
{
|
||||
@ -1249,7 +1355,20 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
destination += it->name();
|
||||
}
|
||||
|
||||
disk->createHardLink(it->path(), destination);
|
||||
if (!disk->isDirectory(it->path()))
|
||||
disk->createHardLink(it->path(), destination);
|
||||
else if (!startsWith("tmp_", it->name())) // ignore projection tmp merge dir
|
||||
{
|
||||
// it's a projection part directory
|
||||
disk->createDirectories(destination);
|
||||
for (auto p_it = disk->iterateDirectory(it->path()); p_it->isValid(); p_it->next())
|
||||
{
|
||||
String p_destination = destination + "/";
|
||||
String p_file_name = p_it->name();
|
||||
p_destination += p_it->name();
|
||||
disk->createHardLink(p_it->path(), p_destination);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
merge_entry->columns_written = storage_columns.size() - updated_header.columns();
|
||||
@ -1264,14 +1383,18 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
source_part,
|
||||
metadata_snapshot,
|
||||
indices_to_recalc,
|
||||
updated_header,
|
||||
projections_to_recalc,
|
||||
mutation_kind == MutationsInterpreter::MutationKind::MUTATE_INDEX_PROJECTION ? Block{} : updated_header,
|
||||
new_data_part,
|
||||
in,
|
||||
time_of_mutation,
|
||||
compression_codec,
|
||||
merge_entry,
|
||||
need_remove_expired_values,
|
||||
need_sync);
|
||||
need_sync,
|
||||
space_reservation,
|
||||
holder,
|
||||
context);
|
||||
}
|
||||
|
||||
for (const auto & [rename_from, rename_to] : files_to_rename)
|
||||
@ -1296,8 +1419,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
|
||||
|
||||
MergeAlgorithm MergeTreeDataMergerMutator::chooseMergeAlgorithm(
|
||||
const MergeTreeData::DataPartsVector & parts, size_t sum_rows_upper_bound,
|
||||
const NamesAndTypesList & gathering_columns, bool deduplicate, bool need_remove_expired_values) const
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
size_t sum_rows_upper_bound,
|
||||
const NamesAndTypesList & gathering_columns,
|
||||
bool deduplicate,
|
||||
bool need_remove_expired_values,
|
||||
const MergeTreeData::MergingParams & merging_params) const
|
||||
{
|
||||
const auto data_settings = data.getSettings();
|
||||
|
||||
@ -1313,10 +1440,10 @@ MergeAlgorithm MergeTreeDataMergerMutator::chooseMergeAlgorithm(
|
||||
return MergeAlgorithm::Horizontal;
|
||||
|
||||
bool is_supported_storage =
|
||||
data.merging_params.mode == MergeTreeData::MergingParams::Ordinary ||
|
||||
data.merging_params.mode == MergeTreeData::MergingParams::Collapsing ||
|
||||
data.merging_params.mode == MergeTreeData::MergingParams::Replacing ||
|
||||
data.merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing;
|
||||
merging_params.mode == MergeTreeData::MergingParams::Ordinary ||
|
||||
merging_params.mode == MergeTreeData::MergingParams::Collapsing ||
|
||||
merging_params.mode == MergeTreeData::MergingParams::Replacing ||
|
||||
merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing;
|
||||
|
||||
bool enough_ordinary_cols = gathering_columns.size() >= data_settings->vertical_merge_algorithm_min_columns_to_activate;
|
||||
|
||||
@ -1405,6 +1532,7 @@ void MergeTreeDataMergerMutator::splitMutationCommands(
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (command.type == MutationCommand::Type::MATERIALIZE_INDEX
|
||||
|| command.type == MutationCommand::Type::MATERIALIZE_PROJECTION
|
||||
|| command.type == MutationCommand::Type::MATERIALIZE_TTL
|
||||
|| command.type == MutationCommand::Type::DELETE
|
||||
|| command.type == MutationCommand::Type::UPDATE)
|
||||
@ -1413,7 +1541,7 @@ void MergeTreeDataMergerMutator::splitMutationCommands(
|
||||
for (const auto & [column_name, expr] : command.column_to_update_expression)
|
||||
mutated_columns.emplace(column_name);
|
||||
}
|
||||
else if (command.type == MutationCommand::Type::DROP_INDEX)
|
||||
else if (command.type == MutationCommand::Type::DROP_INDEX || command.type == MutationCommand::Type::DROP_PROJECTION)
|
||||
{
|
||||
for_file_renames.push_back(command);
|
||||
}
|
||||
@ -1435,7 +1563,7 @@ void MergeTreeDataMergerMutator::splitMutationCommands(
|
||||
}
|
||||
}
|
||||
}
|
||||
/// If it's compact part than we don't need to actually remove files
|
||||
/// If it's compact part, then we don't need to actually remove files
|
||||
/// from disk we just don't read dropped columns
|
||||
for (const auto & column : part->getColumns())
|
||||
{
|
||||
@ -1449,13 +1577,14 @@ void MergeTreeDataMergerMutator::splitMutationCommands(
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (command.type == MutationCommand::Type::MATERIALIZE_INDEX
|
||||
|| command.type == MutationCommand::Type::MATERIALIZE_PROJECTION
|
||||
|| command.type == MutationCommand::Type::MATERIALIZE_TTL
|
||||
|| command.type == MutationCommand::Type::DELETE
|
||||
|| command.type == MutationCommand::Type::UPDATE)
|
||||
{
|
||||
for_interpreter.push_back(command);
|
||||
}
|
||||
else if (command.type == MutationCommand::Type::DROP_INDEX)
|
||||
else if (command.type == MutationCommand::Type::DROP_INDEX || command.type == MutationCommand::Type::DROP_PROJECTION)
|
||||
{
|
||||
for_file_renames.push_back(command);
|
||||
}
|
||||
@ -1499,13 +1628,21 @@ NameToNameVector MergeTreeDataMergerMutator::collectFilesForRenames(
|
||||
}
|
||||
|
||||
NameToNameVector rename_vector;
|
||||
/// Remove old indices
|
||||
/// Remove old data
|
||||
for (const auto & command : commands_for_removes)
|
||||
{
|
||||
if (command.type == MutationCommand::Type::DROP_INDEX)
|
||||
{
|
||||
rename_vector.emplace_back("skp_idx_" + command.column_name + ".idx", "");
|
||||
rename_vector.emplace_back("skp_idx_" + command.column_name + mrk_extension, "");
|
||||
if (source_part->checksums.has(INDEX_FILE_PREFIX + command.column_name + ".idx"))
|
||||
{
|
||||
rename_vector.emplace_back(INDEX_FILE_PREFIX + command.column_name + ".idx", "");
|
||||
rename_vector.emplace_back(INDEX_FILE_PREFIX + command.column_name + mrk_extension, "");
|
||||
}
|
||||
}
|
||||
else if (command.type == MutationCommand::Type::DROP_PROJECTION)
|
||||
{
|
||||
if (source_part->checksums.has(command.column_name + ".proj"))
|
||||
rename_vector.emplace_back(command.column_name + ".proj", "");
|
||||
}
|
||||
else if (command.type == MutationCommand::Type::DROP_COLUMN)
|
||||
{
|
||||
@ -1561,7 +1698,8 @@ NameSet MergeTreeDataMergerMutator::collectFilesToSkip(
|
||||
const MergeTreeDataPartPtr & source_part,
|
||||
const Block & updated_header,
|
||||
const std::set<MergeTreeIndexPtr> & indices_to_recalc,
|
||||
const String & mrk_extension)
|
||||
const String & mrk_extension,
|
||||
const std::set<MergeTreeProjectionPtr> & projections_to_recalc)
|
||||
{
|
||||
NameSet files_to_skip = source_part->getFileNamesWithoutChecksums();
|
||||
|
||||
@ -1583,6 +1721,10 @@ NameSet MergeTreeDataMergerMutator::collectFilesToSkip(
|
||||
files_to_skip.insert(index->getFileName() + ".idx");
|
||||
files_to_skip.insert(index->getFileName() + mrk_extension);
|
||||
}
|
||||
for (const auto & projection : projections_to_recalc)
|
||||
{
|
||||
files_to_skip.insert(projection->getDirectoryName());
|
||||
}
|
||||
|
||||
return files_to_skip;
|
||||
}
|
||||
@ -1686,27 +1828,65 @@ MergeTreeIndices MergeTreeDataMergerMutator::getIndicesForNewDataPart(
|
||||
return new_indices;
|
||||
}
|
||||
|
||||
MergeTreeProjections MergeTreeDataMergerMutator::getProjectionsForNewDataPart(
|
||||
const ProjectionsDescription & all_projections,
|
||||
const MutationCommands & commands_for_removes)
|
||||
{
|
||||
NameSet removed_projections;
|
||||
for (const auto & command : commands_for_removes)
|
||||
if (command.type == MutationCommand::DROP_PROJECTION)
|
||||
removed_projections.insert(command.column_name);
|
||||
|
||||
MergeTreeProjections new_projections;
|
||||
for (const auto & projection : all_projections)
|
||||
if (!removed_projections.count(projection.name))
|
||||
new_projections.push_back(MergeTreeProjectionFactory::instance().get(projection));
|
||||
|
||||
return new_projections;
|
||||
}
|
||||
|
||||
std::set<MergeTreeIndexPtr> MergeTreeDataMergerMutator::getIndicesToRecalculate(
|
||||
BlockInputStreamPtr & input_stream,
|
||||
const NamesAndTypesList & updated_columns,
|
||||
const NameSet & updated_columns,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
ContextPtr context)
|
||||
ContextPtr context,
|
||||
const NameSet & materialized_indices,
|
||||
const MergeTreeData::DataPartPtr & source_part)
|
||||
{
|
||||
/// Checks if columns used in skipping indexes modified.
|
||||
const auto & index_factory = MergeTreeIndexFactory::instance();
|
||||
std::set<MergeTreeIndexPtr> indices_to_recalc;
|
||||
ASTPtr indices_recalc_expr_list = std::make_shared<ASTExpressionList>();
|
||||
for (const auto & col : updated_columns.getNames())
|
||||
{
|
||||
const auto & indices = metadata_snapshot->getSecondaryIndices();
|
||||
for (size_t i = 0; i < indices.size(); ++i)
|
||||
{
|
||||
const auto & index = indices[i];
|
||||
const auto & index_cols = index.expression->getRequiredColumns();
|
||||
auto it = std::find(std::cbegin(index_cols), std::cend(index_cols), col);
|
||||
const auto & indices = metadata_snapshot->getSecondaryIndices();
|
||||
|
||||
if (it != std::cend(index_cols)
|
||||
&& indices_to_recalc.insert(index_factory.get(index)).second)
|
||||
for (size_t i = 0; i < indices.size(); ++i)
|
||||
{
|
||||
const auto & index = indices[i];
|
||||
|
||||
// If we ask to materialize and it already exists
|
||||
if (!source_part->checksums.has(INDEX_FILE_PREFIX + index.name + ".idx") && materialized_indices.count(index.name))
|
||||
{
|
||||
if (indices_to_recalc.insert(index_factory.get(index)).second)
|
||||
{
|
||||
ASTPtr expr_list = index.expression_list_ast->clone();
|
||||
for (const auto & expr : expr_list->children)
|
||||
indices_recalc_expr_list->children.push_back(expr->clone());
|
||||
}
|
||||
}
|
||||
// If some dependent columns gets mutated
|
||||
else
|
||||
{
|
||||
bool mutate = false;
|
||||
const auto & index_cols = index.expression->getRequiredColumns();
|
||||
for (const auto & col : index_cols)
|
||||
{
|
||||
if (updated_columns.count(col))
|
||||
{
|
||||
mutate = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (mutate && indices_to_recalc.insert(index_factory.get(index)).second)
|
||||
{
|
||||
ASTPtr expr_list = index.expression_list_ast->clone();
|
||||
for (const auto & expr : expr_list->children)
|
||||
@ -1733,6 +1913,42 @@ std::set<MergeTreeIndexPtr> MergeTreeDataMergerMutator::getIndicesToRecalculate(
|
||||
return indices_to_recalc;
|
||||
}
|
||||
|
||||
std::set<MergeTreeProjectionPtr> MergeTreeDataMergerMutator::getProjectionsToRecalculate(
|
||||
const NameSet & updated_columns,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const NameSet & materialized_projections,
|
||||
const MergeTreeData::DataPartPtr & source_part)
|
||||
{
|
||||
/// Checks if columns used in projections modified.
|
||||
const auto & projection_factory = MergeTreeProjectionFactory::instance();
|
||||
std::set<MergeTreeProjectionPtr> projections_to_recalc;
|
||||
for (const auto & projection : metadata_snapshot->getProjections())
|
||||
{
|
||||
// If we ask to materialize and it doesn't exist
|
||||
if (!source_part->checksums.has(projection.name + ".proj") && materialized_projections.count(projection.name))
|
||||
{
|
||||
projections_to_recalc.insert(projection_factory.get(projection));
|
||||
}
|
||||
else
|
||||
{
|
||||
// If some dependent columns gets mutated
|
||||
bool mutate = false;
|
||||
const auto & projection_cols = projection.required_columns;
|
||||
for (const auto & col : projection_cols)
|
||||
{
|
||||
if (updated_columns.count(col))
|
||||
{
|
||||
mutate = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (mutate)
|
||||
projections_to_recalc.insert(projection_factory.get(projection));
|
||||
}
|
||||
}
|
||||
return projections_to_recalc;
|
||||
}
|
||||
|
||||
bool MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const Names & columns, const MutationCommands & commands)
|
||||
{
|
||||
if (!metadata_snapshot->hasAnyTTL())
|
||||
@ -1750,16 +1966,182 @@ bool MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadataPtr & met
|
||||
return false;
|
||||
}
|
||||
|
||||
// 1. get projection pipeline and a sink to write parts
|
||||
// 2. build an executor that can write block to the input stream (actually we can write through it to generate as many parts as possible)
|
||||
// 3. finalize the pipeline so that all parts are merged into one part
|
||||
void MergeTreeDataMergerMutator::writeWithProjections(
|
||||
MergeTreeData::MutableDataPartPtr new_data_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeProjections & projections_to_build,
|
||||
BlockInputStreamPtr mutating_stream,
|
||||
IMergedBlockOutputStream & out,
|
||||
time_t time_of_mutation,
|
||||
MergeListEntry & merge_entry,
|
||||
const ReservationPtr & space_reservation,
|
||||
TableLockHolder & holder,
|
||||
ContextPtr context,
|
||||
IMergeTreeDataPart::MinMaxIndex * minmax_idx)
|
||||
{
|
||||
size_t block_num = 0;
|
||||
std::map<String, MergeTreeData::MutableDataPartsVector> projection_parts;
|
||||
Block block;
|
||||
std::vector<SquashingTransform> projection_squashes;
|
||||
for (size_t i = 0, size = projections_to_build.size(); i < size; ++i)
|
||||
{
|
||||
projection_squashes.emplace_back(65536, 65536 * 256);
|
||||
}
|
||||
while (checkOperationIsNotCanceled(merge_entry) && (block = mutating_stream->read()))
|
||||
{
|
||||
if (minmax_idx)
|
||||
minmax_idx->update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey()));
|
||||
|
||||
out.write(block);
|
||||
|
||||
for (size_t i = 0, size = projections_to_build.size(); i < size; ++i)
|
||||
{
|
||||
const auto & projection = projections_to_build[i]->projection;
|
||||
auto in = InterpreterSelectQuery(
|
||||
projection.query_ast,
|
||||
context,
|
||||
Pipe(std::make_shared<SourceFromSingleChunk>(block, Chunk(block.getColumns(), block.rows()))),
|
||||
SelectQueryOptions{
|
||||
projection.type == ProjectionDescription::Type::Normal ? QueryProcessingStage::FetchColumns : QueryProcessingStage::WithMergeableState})
|
||||
.execute()
|
||||
.getInputStream();
|
||||
in = std::make_shared<SquashingBlockInputStream>(in, block.rows(), std::numeric_limits<UInt64>::max());
|
||||
in->readPrefix();
|
||||
auto & projection_squash = projection_squashes[i];
|
||||
auto projection_block = projection_squash.add(in->read());
|
||||
if (in->read())
|
||||
throw Exception("Projection cannot increase the number of rows in a block", ErrorCodes::LOGICAL_ERROR);
|
||||
in->readSuffix();
|
||||
if (projection_block)
|
||||
{
|
||||
projection_parts[projection.name].emplace_back(
|
||||
MergeTreeDataWriter::writeTempProjectionPart(data, log, projection_block, projection, new_data_part.get(), ++block_num));
|
||||
}
|
||||
}
|
||||
|
||||
merge_entry->rows_written += block.rows();
|
||||
merge_entry->bytes_written_uncompressed += block.bytes();
|
||||
}
|
||||
|
||||
// Write the last block
|
||||
for (size_t i = 0, size = projections_to_build.size(); i < size; ++i)
|
||||
{
|
||||
const auto & projection = projections_to_build[i]->projection;
|
||||
auto & projection_squash = projection_squashes[i];
|
||||
auto projection_block = projection_squash.add({});
|
||||
if (projection_block)
|
||||
{
|
||||
projection_parts[projection.name].emplace_back(
|
||||
MergeTreeDataWriter::writeTempProjectionPart(data, log, projection_block, projection, new_data_part.get(), ++block_num));
|
||||
}
|
||||
}
|
||||
|
||||
const auto & projections = metadata_snapshot->projections;
|
||||
|
||||
for (auto && [name, parts] : projection_parts)
|
||||
{
|
||||
LOG_DEBUG(log, "Selected {} projection_parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name);
|
||||
|
||||
const auto & projection = projections.get(name);
|
||||
|
||||
std::map<size_t, MergeTreeData::MutableDataPartsVector> level_parts;
|
||||
size_t current_level = 0;
|
||||
size_t next_level = 1;
|
||||
level_parts[current_level] = std::move(parts);
|
||||
size_t max_parts_to_merge_in_one_level = 10;
|
||||
for (;;)
|
||||
{
|
||||
auto & current_level_parts = level_parts[current_level];
|
||||
auto & next_level_parts = level_parts[next_level];
|
||||
|
||||
MergeTreeData::MutableDataPartsVector selected_parts;
|
||||
while (selected_parts.size() < max_parts_to_merge_in_one_level && !current_level_parts.empty())
|
||||
{
|
||||
selected_parts.push_back(std::move(current_level_parts.back()));
|
||||
current_level_parts.pop_back();
|
||||
}
|
||||
|
||||
if (selected_parts.empty())
|
||||
{
|
||||
if (next_level_parts.empty())
|
||||
{
|
||||
LOG_WARNING(log, "There is no projection parts merged");
|
||||
break;
|
||||
}
|
||||
current_level = next_level;
|
||||
++next_level;
|
||||
}
|
||||
else if (selected_parts.size() == 1)
|
||||
{
|
||||
if (next_level_parts.empty())
|
||||
{
|
||||
LOG_DEBUG(log, "Merged a projection part in level {}", current_level);
|
||||
selected_parts[0]->renameTo(projection.name + ".proj", true);
|
||||
selected_parts[0]->name = projection.name;
|
||||
selected_parts[0]->is_temp = false;
|
||||
new_data_part->addProjectionPart(name, std::move(selected_parts[0]));
|
||||
break;
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_DEBUG(log, "Forwarded part {} in level {} to next level", selected_parts[0]->name, current_level);
|
||||
next_level_parts.push_back(std::move(selected_parts[0]));
|
||||
}
|
||||
}
|
||||
else if (selected_parts.size() > 1)
|
||||
{
|
||||
// Generate a unique part name
|
||||
++block_num;
|
||||
FutureMergedMutatedPart projection_future_part;
|
||||
MergeTreeData::DataPartsVector const_selected_parts(
|
||||
std::make_move_iterator(selected_parts.begin()), std::make_move_iterator(selected_parts.end()));
|
||||
projection_future_part.assign(std::move(const_selected_parts));
|
||||
projection_future_part.name = fmt::format("{}_{}", projection.name, ++block_num);
|
||||
projection_future_part.part_info = {"all", 0, 0, 0};
|
||||
|
||||
MergeTreeData::MergingParams projection_merging_params;
|
||||
projection_merging_params.mode = MergeTreeData::MergingParams::Ordinary;
|
||||
if (projection.type == ProjectionDescription::Type::Aggregate)
|
||||
projection_merging_params.mode = MergeTreeData::MergingParams::Aggregating;
|
||||
|
||||
LOG_DEBUG(log, "Merged {} parts in level {} to {}", selected_parts.size(), current_level, projection_future_part.name);
|
||||
next_level_parts.push_back(mergePartsToTemporaryPart(
|
||||
projection_future_part,
|
||||
projection.metadata,
|
||||
merge_entry,
|
||||
holder,
|
||||
time_of_mutation,
|
||||
context,
|
||||
space_reservation,
|
||||
false, // TODO Do we need deduplicate for projections
|
||||
{},
|
||||
projection_merging_params,
|
||||
new_data_part.get(),
|
||||
"tmp_merge_"));
|
||||
|
||||
next_level_parts.back()->is_temp = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeDataMergerMutator::mutateAllPartColumns(
|
||||
MergeTreeData::MutableDataPartPtr new_data_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeIndices & skip_indices,
|
||||
const MergeTreeProjections & projections_to_build,
|
||||
BlockInputStreamPtr mutating_stream,
|
||||
time_t time_of_mutation,
|
||||
const CompressionCodecPtr & compression_codec,
|
||||
MergeListEntry & merge_entry,
|
||||
bool need_remove_expired_values,
|
||||
bool need_sync) const
|
||||
bool need_sync,
|
||||
const ReservationPtr & space_reservation,
|
||||
TableLockHolder & holder,
|
||||
ContextPtr context)
|
||||
{
|
||||
if (mutating_stream == nullptr)
|
||||
throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -1783,15 +2165,18 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns(
|
||||
mutating_stream->readPrefix();
|
||||
out.writePrefix();
|
||||
|
||||
Block block;
|
||||
while (checkOperationIsNotCanceled(merge_entry) && (block = mutating_stream->read()))
|
||||
{
|
||||
minmax_idx.update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey()));
|
||||
out.write(block);
|
||||
|
||||
merge_entry->rows_written += block.rows();
|
||||
merge_entry->bytes_written_uncompressed += block.bytes();
|
||||
}
|
||||
writeWithProjections(
|
||||
new_data_part,
|
||||
metadata_snapshot,
|
||||
projections_to_build,
|
||||
mutating_stream,
|
||||
out,
|
||||
time_of_mutation,
|
||||
merge_entry,
|
||||
space_reservation,
|
||||
holder,
|
||||
context,
|
||||
&minmax_idx);
|
||||
|
||||
new_data_part->minmax_idx = std::move(minmax_idx);
|
||||
mutating_stream->readSuffix();
|
||||
@ -1802,6 +2187,7 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns(
|
||||
const MergeTreeDataPartPtr & source_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const std::set<MergeTreeIndexPtr> & indices_to_recalc,
|
||||
const std::set<MergeTreeProjectionPtr> & projections_to_recalc,
|
||||
const Block & mutation_header,
|
||||
MergeTreeData::MutableDataPartPtr new_data_part,
|
||||
BlockInputStreamPtr mutating_stream,
|
||||
@ -1809,7 +2195,10 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns(
|
||||
const CompressionCodecPtr & compression_codec,
|
||||
MergeListEntry & merge_entry,
|
||||
bool need_remove_expired_values,
|
||||
bool need_sync) const
|
||||
bool need_sync,
|
||||
const ReservationPtr & space_reservation,
|
||||
TableLockHolder & holder,
|
||||
ContextPtr context)
|
||||
{
|
||||
if (mutating_stream == nullptr)
|
||||
throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -1832,14 +2221,18 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns(
|
||||
mutating_stream->readPrefix();
|
||||
out.writePrefix();
|
||||
|
||||
Block block;
|
||||
while (checkOperationIsNotCanceled(merge_entry) && (block = mutating_stream->read()))
|
||||
{
|
||||
out.write(block);
|
||||
|
||||
merge_entry->rows_written += block.rows();
|
||||
merge_entry->bytes_written_uncompressed += block.bytes();
|
||||
}
|
||||
std::vector<MergeTreeProjectionPtr> projections_to_build(projections_to_recalc.begin(), projections_to_recalc.end());
|
||||
writeWithProjections(
|
||||
new_data_part,
|
||||
metadata_snapshot,
|
||||
projections_to_build,
|
||||
mutating_stream,
|
||||
out,
|
||||
time_of_mutation,
|
||||
merge_entry,
|
||||
space_reservation,
|
||||
holder,
|
||||
context);
|
||||
|
||||
mutating_stream->readSuffix();
|
||||
|
||||
@ -1897,6 +2290,7 @@ void MergeTreeDataMergerMutator::finalizeMutatedPart(
|
||||
new_data_part->index = source_part->index;
|
||||
new_data_part->minmax_idx = source_part->minmax_idx;
|
||||
new_data_part->modification_time = time(nullptr);
|
||||
new_data_part->loadProjections(false, false);
|
||||
new_data_part->setBytesOnDisk(
|
||||
MergeTreeData::DataPart::calculateTotalSizeOnDisk(new_data_part->volume->getDisk(), new_data_part->getFullRelativePath()));
|
||||
new_data_part->default_codec = codec;
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Storages/MergeTree/TTLMergeSelector.h>
|
||||
#include <Storages/MergeTree/MergeAlgorithm.h>
|
||||
#include <Storages/MergeTree/MergeType.h>
|
||||
#include <Storages/MergeTree/IMergedBlockOutputStream.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -128,7 +129,10 @@ public:
|
||||
ContextPtr context,
|
||||
const ReservationPtr & space_reservation,
|
||||
bool deduplicate,
|
||||
const Names & deduplicate_by_columns);
|
||||
const Names & deduplicate_by_columns,
|
||||
const MergeTreeData::MergingParams & merging_params,
|
||||
const IMergeTreeDataPart * parent_part = nullptr,
|
||||
const String & prefix = "");
|
||||
|
||||
/// Mutate a single data part with the specified commands. Will create and return a temporary part.
|
||||
MergeTreeData::MutableDataPartPtr mutatePartToTemporaryPart(
|
||||
@ -177,7 +181,8 @@ private:
|
||||
const MergeTreeDataPartPtr & source_part,
|
||||
const Block & updated_header,
|
||||
const std::set<MergeTreeIndexPtr> & indices_to_recalc,
|
||||
const String & mrk_extension);
|
||||
const String & mrk_extension,
|
||||
const std::set<MergeTreeProjectionPtr> & projections_to_recalc);
|
||||
|
||||
/// Get the columns list of the resulting part in the same order as storage_columns.
|
||||
static NamesAndTypesList getColumnsForNewDataPart(
|
||||
@ -191,41 +196,74 @@ private:
|
||||
const IndicesDescription & all_indices,
|
||||
const MutationCommands & commands_for_removes);
|
||||
|
||||
static MergeTreeProjections getProjectionsForNewDataPart(
|
||||
const ProjectionsDescription & all_projections,
|
||||
const MutationCommands & commands_for_removes);
|
||||
|
||||
static bool shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const Names & columns, const MutationCommands & commands);
|
||||
|
||||
/// Return set of indices which should be recalculated during mutation also
|
||||
/// wraps input stream into additional expression stream
|
||||
static std::set<MergeTreeIndexPtr> getIndicesToRecalculate(
|
||||
BlockInputStreamPtr & input_stream,
|
||||
const NamesAndTypesList & updated_columns,
|
||||
const NameSet & updated_columns,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
ContextPtr context);
|
||||
ContextPtr context,
|
||||
const NameSet & materialized_indices,
|
||||
const MergeTreeData::DataPartPtr & source_part);
|
||||
|
||||
static std::set<MergeTreeProjectionPtr> getProjectionsToRecalculate(
|
||||
const NameSet & updated_columns,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const NameSet & materialized_projections,
|
||||
const MergeTreeData::DataPartPtr & source_part);
|
||||
|
||||
void writeWithProjections(
|
||||
MergeTreeData::MutableDataPartPtr new_data_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeProjections & projections_to_build,
|
||||
BlockInputStreamPtr mutating_stream,
|
||||
IMergedBlockOutputStream & out,
|
||||
time_t time_of_mutation,
|
||||
MergeListEntry & merge_entry,
|
||||
const ReservationPtr & space_reservation,
|
||||
TableLockHolder & holder,
|
||||
ContextPtr context,
|
||||
IMergeTreeDataPart::MinMaxIndex * minmax_idx = nullptr);
|
||||
|
||||
/// Override all columns of new part using mutating_stream
|
||||
void mutateAllPartColumns(
|
||||
MergeTreeData::MutableDataPartPtr new_data_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeIndices & skip_indices,
|
||||
const MergeTreeProjections & projections_to_build,
|
||||
BlockInputStreamPtr mutating_stream,
|
||||
time_t time_of_mutation,
|
||||
const CompressionCodecPtr & codec,
|
||||
const CompressionCodecPtr & compression_codec,
|
||||
MergeListEntry & merge_entry,
|
||||
bool need_remove_expired_values,
|
||||
bool need_sync) const;
|
||||
bool need_sync,
|
||||
const ReservationPtr & space_reservation,
|
||||
TableLockHolder & holder,
|
||||
ContextPtr context);
|
||||
|
||||
/// Mutate some columns of source part with mutation_stream
|
||||
void mutateSomePartColumns(
|
||||
const MergeTreeDataPartPtr & source_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const std::set<MergeTreeIndexPtr> & indices_to_recalc,
|
||||
const std::set<MergeTreeProjectionPtr> & projections_to_recalc,
|
||||
const Block & mutation_header,
|
||||
MergeTreeData::MutableDataPartPtr new_data_part,
|
||||
BlockInputStreamPtr mutating_stream,
|
||||
time_t time_of_mutation,
|
||||
const CompressionCodecPtr & codec,
|
||||
const CompressionCodecPtr & compression_codec,
|
||||
MergeListEntry & merge_entry,
|
||||
bool need_remove_expired_values,
|
||||
bool need_sync) const;
|
||||
bool need_sync,
|
||||
const ReservationPtr & space_reservation,
|
||||
TableLockHolder & holder,
|
||||
ContextPtr context);
|
||||
|
||||
/// Initialize and write to disk new part fields like checksums, columns,
|
||||
/// etc.
|
||||
@ -246,7 +284,11 @@ private:
|
||||
|
||||
MergeAlgorithm chooseMergeAlgorithm(
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
size_t rows_upper_bound, const NamesAndTypesList & gathering_columns, bool deduplicate, bool need_remove_expired_values) const;
|
||||
size_t rows_upper_bound,
|
||||
const NamesAndTypesList & gathering_columns,
|
||||
bool deduplicate,
|
||||
bool need_remove_expired_values,
|
||||
const MergeTreeData::MergingParams & merging_params) const;
|
||||
|
||||
bool checkOperationIsNotCanceled(const MergeListEntry & merge_entry) const;
|
||||
|
||||
|
@ -48,6 +48,9 @@ void MergeTreeDataPartChecksum::checkSize(const DiskPtr & disk, const String & p
|
||||
{
|
||||
if (!disk->exists(path))
|
||||
throw Exception(fullPath(disk, path) + " doesn't exist", ErrorCodes::FILE_DOESNT_EXIST);
|
||||
if (disk->isDirectory(path))
|
||||
// This is a projection, no need to check its size.
|
||||
return;
|
||||
UInt64 size = disk->getFileSize(path);
|
||||
if (size != file_size)
|
||||
throw Exception(fullPath(disk, path) + " has unexpected size: " + toString(size) + " instead of " + toString(file_size),
|
||||
@ -306,6 +309,24 @@ String MergeTreeDataPartChecksums::getTotalChecksumHex() const
|
||||
return getHexUIntUppercase(hi) + getHexUIntUppercase(lo);
|
||||
}
|
||||
|
||||
MergeTreeDataPartChecksums::Checksum::uint128 MergeTreeDataPartChecksums::getTotalChecksumUInt128() const
|
||||
{
|
||||
SipHash hash_of_all_files;
|
||||
|
||||
for (const auto & elem : files)
|
||||
{
|
||||
const String & name = elem.first;
|
||||
const auto & checksum = elem.second;
|
||||
|
||||
updateHash(hash_of_all_files, name);
|
||||
hash_of_all_files.update(checksum.file_hash);
|
||||
}
|
||||
|
||||
MergeTreeDataPartChecksums::Checksum::uint128 ret;
|
||||
hash_of_all_files.get128(reinterpret_cast<char *>(&ret));
|
||||
return ret;
|
||||
}
|
||||
|
||||
void MinimalisticDataPartChecksums::serialize(WriteBuffer & to) const
|
||||
{
|
||||
writeString("checksums format version: 5\n", to);
|
||||
|
@ -52,6 +52,8 @@ struct MergeTreeDataPartChecksums
|
||||
|
||||
void add(MergeTreeDataPartChecksums && rhs_checksums);
|
||||
|
||||
bool has(const String & file_name) const { return files.find(file_name) != files.end(); }
|
||||
|
||||
bool empty() const
|
||||
{
|
||||
return files.empty();
|
||||
@ -83,6 +85,8 @@ struct MergeTreeDataPartChecksums
|
||||
/// SipHash of all all files hashes represented as hex string
|
||||
String getTotalChecksumHex() const;
|
||||
|
||||
Checksum::uint128 getTotalChecksumUInt128() const;
|
||||
|
||||
String getSerializedString() const;
|
||||
static MergeTreeDataPartChecksums deserializeFrom(const String & s);
|
||||
|
||||
|
@ -21,8 +21,9 @@ MergeTreeDataPartCompact::MergeTreeDataPartCompact(
|
||||
MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const VolumePtr & volume_,
|
||||
const std::optional<String> & relative_path_)
|
||||
: IMergeTreeDataPart(storage_, name_, volume_, relative_path_, Type::COMPACT)
|
||||
const std::optional<String> & relative_path_,
|
||||
const IMergeTreeDataPart * parent_part_)
|
||||
: IMergeTreeDataPart(storage_, name_, volume_, relative_path_, Type::COMPACT, parent_part_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -31,8 +32,9 @@ MergeTreeDataPartCompact::MergeTreeDataPartCompact(
|
||||
const String & name_,
|
||||
const MergeTreePartInfo & info_,
|
||||
const VolumePtr & volume_,
|
||||
const std::optional<String> & relative_path_)
|
||||
: IMergeTreeDataPart(storage_, name_, info_, volume_, relative_path_, Type::COMPACT)
|
||||
const std::optional<String> & relative_path_,
|
||||
const IMergeTreeDataPart * parent_part_)
|
||||
: IMergeTreeDataPart(storage_, name_, info_, volume_, relative_path_, Type::COMPACT, parent_part_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -26,13 +26,15 @@ public:
|
||||
const String & name_,
|
||||
const MergeTreePartInfo & info_,
|
||||
const VolumePtr & volume_,
|
||||
const std::optional<String> & relative_path_ = {});
|
||||
const std::optional<String> & relative_path_ = {},
|
||||
const IMergeTreeDataPart * parent_part_ = nullptr);
|
||||
|
||||
MergeTreeDataPartCompact(
|
||||
MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const VolumePtr & volume_,
|
||||
const std::optional<String> & relative_path_ = {});
|
||||
const std::optional<String> & relative_path_ = {},
|
||||
const IMergeTreeDataPart * parent_part_ = nullptr);
|
||||
|
||||
MergeTreeReaderPtr getReader(
|
||||
const NamesAndTypesList & columns,
|
||||
|
@ -22,8 +22,9 @@ MergeTreeDataPartInMemory::MergeTreeDataPartInMemory(
|
||||
MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const VolumePtr & volume_,
|
||||
const std::optional<String> & relative_path_)
|
||||
: IMergeTreeDataPart(storage_, name_, volume_, relative_path_, Type::IN_MEMORY)
|
||||
const std::optional<String> & relative_path_,
|
||||
const IMergeTreeDataPart * parent_part_)
|
||||
: IMergeTreeDataPart(storage_, name_, volume_, relative_path_, Type::IN_MEMORY, parent_part_)
|
||||
{
|
||||
default_codec = CompressionCodecFactory::instance().get("NONE", {});
|
||||
}
|
||||
@ -33,8 +34,9 @@ MergeTreeDataPartInMemory::MergeTreeDataPartInMemory(
|
||||
const String & name_,
|
||||
const MergeTreePartInfo & info_,
|
||||
const VolumePtr & volume_,
|
||||
const std::optional<String> & relative_path_)
|
||||
: IMergeTreeDataPart(storage_, name_, info_, volume_, relative_path_, Type::IN_MEMORY)
|
||||
const std::optional<String> & relative_path_,
|
||||
const IMergeTreeDataPart * parent_part_)
|
||||
: IMergeTreeDataPart(storage_, name_, info_, volume_, relative_path_, Type::IN_MEMORY, parent_part_)
|
||||
{
|
||||
default_codec = CompressionCodecFactory::instance().get("NONE", {});
|
||||
}
|
||||
|
@ -15,13 +15,15 @@ public:
|
||||
const String & name_,
|
||||
const MergeTreePartInfo & info_,
|
||||
const VolumePtr & volume_,
|
||||
const std::optional<String> & relative_path_ = {});
|
||||
const std::optional<String> & relative_path_ = {},
|
||||
const IMergeTreeDataPart * parent_part_ = nullptr);
|
||||
|
||||
MergeTreeDataPartInMemory(
|
||||
MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const VolumePtr & volume_,
|
||||
const std::optional<String> & relative_path_ = {});
|
||||
const std::optional<String> & relative_path_ = {},
|
||||
const IMergeTreeDataPart * parent_part_ = nullptr);
|
||||
|
||||
MergeTreeReaderPtr getReader(
|
||||
const NamesAndTypesList & columns,
|
||||
|
@ -23,8 +23,9 @@ MergeTreeDataPartWide::MergeTreeDataPartWide(
|
||||
MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const VolumePtr & volume_,
|
||||
const std::optional<String> & relative_path_)
|
||||
: IMergeTreeDataPart(storage_, name_, volume_, relative_path_, Type::WIDE)
|
||||
const std::optional<String> & relative_path_,
|
||||
const IMergeTreeDataPart * parent_part_)
|
||||
: IMergeTreeDataPart(storage_, name_, volume_, relative_path_, Type::WIDE, parent_part_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -33,8 +34,9 @@ MergeTreeDataPartWide::MergeTreeDataPartWide(
|
||||
const String & name_,
|
||||
const MergeTreePartInfo & info_,
|
||||
const VolumePtr & volume_,
|
||||
const std::optional<String> & relative_path_)
|
||||
: IMergeTreeDataPart(storage_, name_, info_, volume_, relative_path_, Type::WIDE)
|
||||
const std::optional<String> & relative_path_,
|
||||
const IMergeTreeDataPart * parent_part_)
|
||||
: IMergeTreeDataPart(storage_, name_, info_, volume_, relative_path_, Type::WIDE, parent_part_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -20,13 +20,15 @@ public:
|
||||
const String & name_,
|
||||
const MergeTreePartInfo & info_,
|
||||
const VolumePtr & volume,
|
||||
const std::optional<String> & relative_path = {});
|
||||
const std::optional<String> & relative_path_ = {},
|
||||
const IMergeTreeDataPart * parent_part_ = nullptr);
|
||||
|
||||
MergeTreeDataPartWide(
|
||||
MergeTreeData & storage_,
|
||||
const String & name_,
|
||||
const VolumePtr & volume,
|
||||
const std::optional<String> & relative_path = {});
|
||||
const std::optional<String> & relative_path_ = {},
|
||||
const IMergeTreeDataPart * parent_part_ = nullptr);
|
||||
|
||||
MergeTreeReaderPtr getReader(
|
||||
const NamesAndTypesList & columns,
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -13,6 +13,22 @@ namespace DB
|
||||
|
||||
class KeyCondition;
|
||||
|
||||
struct MergeTreeDataSelectSamplingData
|
||||
{
|
||||
bool use_sampling;
|
||||
std::shared_ptr<ASTFunction> filter_function;
|
||||
ActionsDAGPtr filter_expression;
|
||||
};
|
||||
|
||||
struct MergeTreeDataSelectCache
|
||||
{
|
||||
RangesInDataParts parts_with_ranges;
|
||||
MergeTreeDataSelectSamplingData sampling;
|
||||
std::unique_ptr<ReadFromMergeTree::IndexStats> index_stats;
|
||||
size_t sum_marks = 0;
|
||||
size_t sum_ranges = 0;
|
||||
bool use_cache = false;
|
||||
};
|
||||
|
||||
/** Executes SELECT queries on data from the merge tree.
|
||||
*/
|
||||
@ -33,17 +49,20 @@ public:
|
||||
ContextPtr context,
|
||||
UInt64 max_block_size,
|
||||
unsigned num_streams,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const;
|
||||
|
||||
QueryPlanPtr readFromParts(
|
||||
MergeTreeData::DataPartsVector parts,
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot_base,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
UInt64 max_block_size,
|
||||
unsigned num_streams,
|
||||
const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const;
|
||||
const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr,
|
||||
MergeTreeDataSelectCache * cache = nullptr) const;
|
||||
|
||||
private:
|
||||
const MergeTreeData & data;
|
||||
@ -79,7 +98,8 @@ private:
|
||||
const Settings & settings,
|
||||
const MergeTreeReaderSettings & reader_settings,
|
||||
ActionsDAGPtr & out_projection,
|
||||
const String & query_id) const;
|
||||
const String & query_id,
|
||||
const InputOrderInfoPtr & input_order_info) const;
|
||||
|
||||
QueryPlanPtr spreadMarkRangesAmongStreamsFinal(
|
||||
RangesInDataParts && parts,
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Disks/createVolume.h>
|
||||
#include <Interpreters/AggregationCommon.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <IO/HashingWriteBuffer.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
@ -13,6 +14,8 @@
|
||||
#include <Poco/File.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataStreams/ITTLAlgorithm.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <DataStreams/SquashingBlockInputStream.h>
|
||||
|
||||
#include <Parsers/queryToString.h>
|
||||
|
||||
@ -23,6 +26,7 @@
|
||||
#include <Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h>
|
||||
#include <Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h>
|
||||
#include <Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
@ -31,6 +35,11 @@ namespace ProfileEvents
|
||||
extern const Event MergeTreeDataWriterRows;
|
||||
extern const Event MergeTreeDataWriterUncompressedBytes;
|
||||
extern const Event MergeTreeDataWriterCompressedBytes;
|
||||
extern const Event MergeTreeDataProjectionWriterBlocks;
|
||||
extern const Event MergeTreeDataProjectionWriterBlocksAlreadySorted;
|
||||
extern const Event MergeTreeDataProjectionWriterRows;
|
||||
extern const Event MergeTreeDataProjectionWriterUncompressedBytes;
|
||||
extern const Event MergeTreeDataProjectionWriterCompressedBytes;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
@ -258,7 +267,8 @@ Block MergeTreeDataWriter::mergeBlock(const Block & block, SortDescription sort_
|
||||
return block.cloneWithColumns(status.chunk.getColumns());
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPartition & block_with_partition, const StorageMetadataPtr & metadata_snapshot, bool optimize_on_insert)
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(
|
||||
BlockWithPartition & block_with_partition, const StorageMetadataPtr & metadata_snapshot, ContextPtr context)
|
||||
{
|
||||
Block & block = block_with_partition.block;
|
||||
|
||||
@ -321,7 +331,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
}
|
||||
|
||||
Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names;
|
||||
if (optimize_on_insert)
|
||||
if (context->getSettingsRef().optimize_on_insert)
|
||||
block = mergeBlock(block, sort_description, partition_key_columns, perm_ptr);
|
||||
|
||||
/// Size of part would not be greater than block.bytes() + epsilon
|
||||
@ -376,6 +386,31 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
sync_guard = disk->getDirectorySyncGuard(full_path);
|
||||
}
|
||||
|
||||
if (metadata_snapshot->hasProjections())
|
||||
{
|
||||
for (const auto & projection : metadata_snapshot->getProjections())
|
||||
{
|
||||
auto in = InterpreterSelectQuery(
|
||||
projection.query_ast,
|
||||
context,
|
||||
Pipe(std::make_shared<SourceFromSingleChunk>(block, Chunk(block.getColumns(), block.rows()))),
|
||||
SelectQueryOptions{
|
||||
projection.type == ProjectionDescription::Type::Normal ? QueryProcessingStage::FetchColumns : QueryProcessingStage::WithMergeableState})
|
||||
.execute()
|
||||
.getInputStream();
|
||||
in = std::make_shared<SquashingBlockInputStream>(in, block.rows(), std::numeric_limits<UInt64>::max());
|
||||
in->readPrefix();
|
||||
auto projection_block = in->read();
|
||||
if (in->read())
|
||||
throw Exception("Projection cannot grow block rows", ErrorCodes::LOGICAL_ERROR);
|
||||
in->readSuffix();
|
||||
if (projection_block.rows())
|
||||
{
|
||||
new_data_part->addProjectionPart(projection.name, writeProjectionPart(projection_block, projection, new_data_part.get()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (metadata_snapshot->hasRowsTTL())
|
||||
updateTTL(metadata_snapshot->getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true);
|
||||
|
||||
@ -413,4 +448,128 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
return new_data_part;
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeProjectionPartImpl(
|
||||
MergeTreeData & data,
|
||||
Poco::Logger * log,
|
||||
Block block,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
MergeTreeData::MutableDataPartPtr && new_data_part)
|
||||
{
|
||||
NamesAndTypesList columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames());
|
||||
MergeTreePartition partition{};
|
||||
IMergeTreeDataPart::MinMaxIndex minmax_idx{};
|
||||
new_data_part->setColumns(columns);
|
||||
new_data_part->partition = std::move(partition);
|
||||
new_data_part->minmax_idx = std::move(minmax_idx);
|
||||
|
||||
if (new_data_part->isStoredOnDisk())
|
||||
{
|
||||
/// The name could be non-unique in case of stale files from previous runs.
|
||||
String full_path = new_data_part->getFullRelativePath();
|
||||
|
||||
if (new_data_part->volume->getDisk()->exists(full_path))
|
||||
{
|
||||
LOG_WARNING(log, "Removing old temporary directory {}", fullPath(new_data_part->volume->getDisk(), full_path));
|
||||
new_data_part->volume->getDisk()->removeRecursive(full_path);
|
||||
}
|
||||
|
||||
new_data_part->volume->getDisk()->createDirectories(full_path);
|
||||
}
|
||||
|
||||
/// If we need to calculate some columns to sort.
|
||||
if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices())
|
||||
data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot)->execute(block);
|
||||
|
||||
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
|
||||
SortDescription sort_description;
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
|
||||
for (size_t i = 0; i < sort_columns_size; ++i)
|
||||
sort_description.emplace_back(block.getPositionByName(sort_columns[i]), 1, 1);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterBlocks);
|
||||
|
||||
/// Sort
|
||||
IColumn::Permutation * perm_ptr = nullptr;
|
||||
IColumn::Permutation perm;
|
||||
if (!sort_description.empty())
|
||||
{
|
||||
if (!isAlreadySorted(block, sort_description))
|
||||
{
|
||||
stableGetPermutation(block, sort_description, perm);
|
||||
perm_ptr = &perm;
|
||||
}
|
||||
else
|
||||
ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterBlocksAlreadySorted);
|
||||
}
|
||||
|
||||
/// This effectively chooses minimal compression method:
|
||||
/// either default lz4 or compression method with zero thresholds on absolute and relative part size.
|
||||
auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0);
|
||||
|
||||
MergedBlockOutputStream out(
|
||||
new_data_part,
|
||||
metadata_snapshot,
|
||||
columns,
|
||||
{},
|
||||
compression_codec);
|
||||
|
||||
out.writePrefix();
|
||||
out.writeWithPermutation(block, perm_ptr);
|
||||
out.writeSuffixAndFinalizePart(new_data_part);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterRows, block.rows());
|
||||
ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterUncompressedBytes, block.bytes());
|
||||
ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterCompressedBytes, new_data_part->getBytesOnDisk());
|
||||
|
||||
return std::move(new_data_part);
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr
|
||||
MergeTreeDataWriter::writeProjectionPart(Block block, const ProjectionDescription & projection, const IMergeTreeDataPart * parent_part)
|
||||
{
|
||||
/// Size of part would not be greater than block.bytes() + epsilon
|
||||
size_t expected_size = block.bytes();
|
||||
|
||||
// just check if there is enough space on parent volume
|
||||
data.reserveSpace(expected_size, parent_part->volume);
|
||||
|
||||
String part_name = projection.name;
|
||||
MergeTreePartInfo new_part_info("all", 0, 0, 0);
|
||||
auto new_data_part = data.createPart(
|
||||
part_name, data.choosePartType(expected_size, block.rows()), new_part_info, parent_part->volume, part_name + ".proj", parent_part);
|
||||
new_data_part->is_temp = false; // clean up will be done on parent part
|
||||
|
||||
return writeProjectionPartImpl(data, log, block, projection.metadata, std::move(new_data_part));
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempProjectionPart(
|
||||
MergeTreeData & data,
|
||||
Poco::Logger * log,
|
||||
Block block,
|
||||
const ProjectionDescription & projection,
|
||||
const IMergeTreeDataPart * parent_part,
|
||||
size_t block_num)
|
||||
{
|
||||
/// Size of part would not be greater than block.bytes() + epsilon
|
||||
size_t expected_size = block.bytes();
|
||||
|
||||
// just check if there is enough space on parent volume
|
||||
data.reserveSpace(expected_size, parent_part->volume);
|
||||
|
||||
String part_name = fmt::format("{}_{}", projection.name, block_num);
|
||||
MergeTreePartInfo new_part_info("all", 0, 0, 0);
|
||||
auto new_data_part = data.createPart(
|
||||
part_name,
|
||||
data.choosePartType(expected_size, block.rows()),
|
||||
new_part_info,
|
||||
parent_part->volume,
|
||||
"tmp_insert_" + part_name + ".proj",
|
||||
parent_part);
|
||||
new_data_part->is_temp = true; // It's part for merge
|
||||
|
||||
return writeProjectionPartImpl(data, log, block, projection.metadata, std::move(new_data_part));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -47,9 +47,30 @@ public:
|
||||
*/
|
||||
MergeTreeData::MutableDataPartPtr writeTempPart(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot, bool optimize_on_insert);
|
||||
|
||||
MergeTreeData::MutableDataPartPtr
|
||||
writeTempPart(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot, ContextPtr context);
|
||||
|
||||
MergeTreeData::MutableDataPartPtr writeProjectionPart(
|
||||
Block block, const ProjectionDescription & projection, const IMergeTreeDataPart * parent_part);
|
||||
|
||||
static MergeTreeData::MutableDataPartPtr writeTempProjectionPart(
|
||||
MergeTreeData & data,
|
||||
Poco::Logger * log,
|
||||
Block block,
|
||||
const ProjectionDescription & projection,
|
||||
const IMergeTreeDataPart * parent_part,
|
||||
size_t block_num);
|
||||
|
||||
Block mergeBlock(const Block & block, SortDescription sort_description, Names & partition_key_columns, IColumn::Permutation *& permutation);
|
||||
|
||||
private:
|
||||
static MergeTreeData::MutableDataPartPtr writeProjectionPartImpl(
|
||||
MergeTreeData & data,
|
||||
Poco::Logger * log,
|
||||
Block block,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
MergeTreeData::MutableDataPartPtr && new_data_part);
|
||||
|
||||
MergeTreeData & data;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
78
src/Storages/MergeTree/MergeTreeProjections.cpp
Normal file
78
src/Storages/MergeTree/MergeTreeProjections.cpp
Normal file
@ -0,0 +1,78 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Storages/MergeTree/MergeTreeProjections.h>
|
||||
|
||||
#include <numeric>
|
||||
|
||||
#include <boost/algorithm/string.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int INCORRECT_QUERY;
|
||||
}
|
||||
|
||||
void MergeTreeProjectionFactory::registerCreator(ProjectionDescription::Type projection_type, Creator creator)
|
||||
{
|
||||
if (!creators.emplace(projection_type, std::move(creator)).second)
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"MergeTreeProjectionFactory: the Projection creator name '{}' is not unique",
|
||||
ProjectionDescription::typeToString(projection_type));
|
||||
}
|
||||
|
||||
MergeTreeProjectionPtr MergeTreeProjectionFactory::get(const ProjectionDescription & projection) const
|
||||
{
|
||||
auto it = creators.find(projection.type);
|
||||
if (it == creators.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Projection type {} is not registered",
|
||||
ProjectionDescription::typeToString(projection.type));
|
||||
|
||||
return it->second(projection);
|
||||
}
|
||||
|
||||
|
||||
MergeTreeProjections MergeTreeProjectionFactory::getMany(const std::vector<ProjectionDescription> & projections) const
|
||||
{
|
||||
MergeTreeProjections result;
|
||||
for (const auto & projection : projections)
|
||||
result.emplace_back(get(projection));
|
||||
return result;
|
||||
}
|
||||
|
||||
void MergeTreeProjectionFactory::validate(const ProjectionDescription & projection) const
|
||||
{
|
||||
if (startsWith(projection.name, "tmp_"))
|
||||
throw Exception("Projection's name cannot start with 'tmp_'", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
get(projection);
|
||||
}
|
||||
|
||||
MergeTreeProjectionPtr normalProjectionCreator(const ProjectionDescription & projection)
|
||||
{
|
||||
return std::make_shared<MergeTreeProjectionNormal>(projection);
|
||||
}
|
||||
|
||||
MergeTreeProjectionPtr aggregateProjectionCreator(const ProjectionDescription & projection)
|
||||
{
|
||||
return std::make_shared<MergeTreeProjectionAggregate>(projection);
|
||||
}
|
||||
|
||||
MergeTreeProjectionFactory::MergeTreeProjectionFactory()
|
||||
{
|
||||
registerCreator(ProjectionDescription::Type::Normal, normalProjectionCreator);
|
||||
registerCreator(ProjectionDescription::Type::Aggregate, aggregateProjectionCreator);
|
||||
}
|
||||
|
||||
MergeTreeProjectionFactory & MergeTreeProjectionFactory::instance()
|
||||
{
|
||||
static MergeTreeProjectionFactory instance;
|
||||
return instance;
|
||||
}
|
||||
|
||||
}
|
88
src/Storages/MergeTree/MergeTreeProjections.h
Normal file
88
src/Storages/MergeTree/MergeTreeProjections.h
Normal file
@ -0,0 +1,88 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <unordered_map>
|
||||
#include <vector>
|
||||
#include <Core/Block.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Parsers/ASTProjectionDeclaration.h>
|
||||
#include <Storages/MergeTree/MarkRange.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
|
||||
#include <Storages/ProjectionsDescription.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/// Condition on the projection.
|
||||
class IMergeTreeProjectionCondition
|
||||
{
|
||||
public:
|
||||
virtual ~IMergeTreeProjectionCondition() = default;
|
||||
/// Checks if this projection is useful for query.
|
||||
virtual bool canHandleQuery() const = 0;
|
||||
};
|
||||
|
||||
using MergeTreeProjectionConditionPtr = std::shared_ptr<IMergeTreeProjectionCondition>;
|
||||
|
||||
struct IMergeTreeProjection
|
||||
{
|
||||
IMergeTreeProjection(const ProjectionDescription & projection_) : projection(projection_) { }
|
||||
|
||||
virtual ~IMergeTreeProjection() = default;
|
||||
|
||||
/// gets directory name
|
||||
String getDirectoryName() const { return projection.name + ".proj"; }
|
||||
|
||||
const String & getName() const { return projection.name; }
|
||||
|
||||
Names getColumnsRequiredForProjectionCalc() const { return projection.required_columns; }
|
||||
|
||||
const ProjectionDescription & projection;
|
||||
};
|
||||
|
||||
using MergeTreeProjectionPtr = std::shared_ptr<const IMergeTreeProjection>;
|
||||
using MergeTreeProjections = std::vector<MergeTreeProjectionPtr>;
|
||||
|
||||
class MergeTreeProjectionNormal : public IMergeTreeProjection
|
||||
{
|
||||
public:
|
||||
explicit MergeTreeProjectionNormal(const ProjectionDescription & projection_) : IMergeTreeProjection(projection_) { }
|
||||
|
||||
~MergeTreeProjectionNormal() override = default;
|
||||
};
|
||||
|
||||
class MergeTreeProjectionAggregate : public IMergeTreeProjection
|
||||
{
|
||||
public:
|
||||
explicit MergeTreeProjectionAggregate(const ProjectionDescription & projection_) : IMergeTreeProjection(projection_) { }
|
||||
|
||||
~MergeTreeProjectionAggregate() override = default;
|
||||
};
|
||||
|
||||
class MergeTreeProjectionFactory : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
static MergeTreeProjectionFactory & instance();
|
||||
|
||||
using Creator = std::function<MergeTreeProjectionPtr(const ProjectionDescription & projection)>;
|
||||
|
||||
void validate(const ProjectionDescription & projection) const;
|
||||
|
||||
MergeTreeProjectionPtr get(const ProjectionDescription & projection) const;
|
||||
|
||||
MergeTreeProjections getMany(const std::vector<ProjectionDescription> & projections) const;
|
||||
|
||||
void registerCreator(ProjectionDescription::Type projection_type, Creator creator);
|
||||
|
||||
protected:
|
||||
MergeTreeProjectionFactory();
|
||||
|
||||
private:
|
||||
using Creators = std::unordered_map<ProjectionDescription::Type, Creator>;
|
||||
Creators creators;
|
||||
};
|
||||
|
||||
}
|
@ -637,17 +637,14 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
|
||||
return read_result;
|
||||
|
||||
bool has_columns = false;
|
||||
for (auto & column : columns)
|
||||
{
|
||||
if (column)
|
||||
has_columns = true;
|
||||
}
|
||||
|
||||
size_t total_bytes = 0;
|
||||
for (auto & column : columns)
|
||||
{
|
||||
if (column)
|
||||
{
|
||||
total_bytes += column->byteSize();
|
||||
has_columns = true;
|
||||
}
|
||||
}
|
||||
|
||||
read_result.addNumBytesRead(total_bytes);
|
||||
|
@ -23,8 +23,7 @@ MergeTreeThreadSelectBlockInputProcessor::MergeTreeThreadSelectBlockInputProcess
|
||||
const Names & virt_column_names_)
|
||||
:
|
||||
MergeTreeBaseSelectProcessor{
|
||||
pool_->getHeader(), storage_, metadata_snapshot_, prewhere_info_,
|
||||
max_block_size_rows_,
|
||||
pool_->getHeader(), storage_, metadata_snapshot_, prewhere_info_, max_block_size_rows_,
|
||||
preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_,
|
||||
reader_settings_, use_uncompressed_cache_, virt_column_names_},
|
||||
thread{thread_},
|
||||
@ -61,7 +60,7 @@ bool MergeTreeThreadSelectBlockInputProcessor::getNewTask()
|
||||
return false;
|
||||
}
|
||||
|
||||
const std::string part_name = task->data_part->name;
|
||||
const std::string part_name = task->data_part->isProjectionPart() ? task->data_part->getParentPart()->name : task->data_part->name;
|
||||
|
||||
/// Allows pool to reduce number of threads in case of too slow reads.
|
||||
auto profile_callback = [this](ReadBufferFromFileBase::ProfileInfo info_) { pool->profileFeedback(info_); };
|
||||
|
@ -72,6 +72,12 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart(
|
||||
/// Finish columns serialization.
|
||||
writer->finish(checksums, sync);
|
||||
|
||||
for (const auto & [projection_name, projection_part] : new_part->getProjectionParts())
|
||||
checksums.addFile(
|
||||
projection_name + ".proj",
|
||||
projection_part->checksums.getTotalSizeOnDisk(),
|
||||
projection_part->checksums.getTotalChecksumUInt128());
|
||||
|
||||
NamesAndTypesList part_columns;
|
||||
if (!total_columns_list)
|
||||
part_columns = columns_list;
|
||||
@ -100,36 +106,52 @@ void MergedBlockOutputStream::finalizePartOnDisk(
|
||||
MergeTreeData::DataPart::Checksums & checksums,
|
||||
bool sync)
|
||||
{
|
||||
if (new_part->uuid != UUIDHelpers::Nil)
|
||||
|
||||
if (new_part->isProjectionPart())
|
||||
{
|
||||
auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::UUID_FILE_NAME, 4096);
|
||||
HashingWriteBuffer out_hashing(*out);
|
||||
writeUUIDText(new_part->uuid, out_hashing);
|
||||
checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_size = out_hashing.count();
|
||||
checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_hash = out_hashing.getHash();
|
||||
out->finalize();
|
||||
if (sync)
|
||||
out->sync();
|
||||
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part))
|
||||
{
|
||||
auto count_out = volume->getDisk()->writeFile(part_path + "count.txt", 4096);
|
||||
HashingWriteBuffer count_out_hashing(*count_out);
|
||||
writeIntText(rows_count, count_out_hashing);
|
||||
count_out_hashing.next();
|
||||
checksums.files["count.txt"].file_size = count_out_hashing.count();
|
||||
checksums.files["count.txt"].file_hash = count_out_hashing.getHash();
|
||||
}
|
||||
}
|
||||
|
||||
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part))
|
||||
else
|
||||
{
|
||||
new_part->partition.store(storage, volume->getDisk(), part_path, checksums);
|
||||
if (new_part->minmax_idx.initialized)
|
||||
new_part->minmax_idx.store(storage, volume->getDisk(), part_path, checksums);
|
||||
else if (rows_count)
|
||||
throw Exception("MinMax index was not initialized for new non-empty part " + new_part->name
|
||||
+ ". It is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
if (new_part->uuid != UUIDHelpers::Nil)
|
||||
{
|
||||
auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::UUID_FILE_NAME, 4096);
|
||||
HashingWriteBuffer out_hashing(*out);
|
||||
writeUUIDText(new_part->uuid, out_hashing);
|
||||
checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_size = out_hashing.count();
|
||||
checksums.files[IMergeTreeDataPart::UUID_FILE_NAME].file_hash = out_hashing.getHash();
|
||||
out->finalize();
|
||||
if (sync)
|
||||
out->sync();
|
||||
}
|
||||
|
||||
auto count_out = volume->getDisk()->writeFile(part_path + "count.txt", 4096);
|
||||
HashingWriteBuffer count_out_hashing(*count_out);
|
||||
writeIntText(rows_count, count_out_hashing);
|
||||
count_out_hashing.next();
|
||||
checksums.files["count.txt"].file_size = count_out_hashing.count();
|
||||
checksums.files["count.txt"].file_hash = count_out_hashing.getHash();
|
||||
count_out->finalize();
|
||||
if (sync)
|
||||
count_out->sync();
|
||||
if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part))
|
||||
{
|
||||
new_part->partition.store(storage, volume->getDisk(), part_path, checksums);
|
||||
if (new_part->minmax_idx.initialized)
|
||||
new_part->minmax_idx.store(storage, volume->getDisk(), part_path, checksums);
|
||||
else if (rows_count)
|
||||
throw Exception("MinMax index was not initialized for new non-empty part " + new_part->name
|
||||
+ ". It is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto count_out = volume->getDisk()->writeFile(part_path + "count.txt", 4096);
|
||||
HashingWriteBuffer count_out_hashing(*count_out);
|
||||
writeIntText(rows_count, count_out_hashing);
|
||||
count_out_hashing.next();
|
||||
checksums.files["count.txt"].file_size = count_out_hashing.count();
|
||||
checksums.files["count.txt"].file_hash = count_out_hashing.getHash();
|
||||
count_out->finalize();
|
||||
if (sync)
|
||||
count_out->sync();
|
||||
}
|
||||
}
|
||||
|
||||
if (!new_part->ttl_infos.empty())
|
||||
|
@ -68,6 +68,12 @@ MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums(
|
||||
MergeTreeData::DataPart::Checksums checksums;
|
||||
writer->finish(checksums, sync);
|
||||
|
||||
for (const auto & [projection_name, projection_part] : new_part->getProjectionParts())
|
||||
checksums.addFile(
|
||||
projection_name + ".proj",
|
||||
projection_part->checksums.getTotalSizeOnDisk(),
|
||||
projection_part->checksums.getTotalChecksumUInt128());
|
||||
|
||||
auto columns = new_part->getColumns();
|
||||
|
||||
auto removed_files = removeEmptyColumnsFromPart(new_part, columns, checksums);
|
||||
|
@ -3,17 +3,17 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool PartitionPruner::canBePruned(const DataPartPtr & part)
|
||||
bool PartitionPruner::canBePruned(const DataPart & part)
|
||||
{
|
||||
if (part->isEmpty())
|
||||
if (part.isEmpty())
|
||||
return true;
|
||||
const auto & partition_id = part->info.partition_id;
|
||||
const auto & partition_id = part.info.partition_id;
|
||||
bool is_valid;
|
||||
if (auto it = partition_filter_map.find(partition_id); it != partition_filter_map.end())
|
||||
is_valid = it->second;
|
||||
else
|
||||
{
|
||||
const auto & partition_value = part->partition.value;
|
||||
const auto & partition_value = part.partition.value;
|
||||
std::vector<FieldRef> index_value(partition_value.begin(), partition_value.end());
|
||||
is_valid = partition_condition.mayBeTrueInRange(
|
||||
partition_value.size(), index_value.data(), index_value.data(), partition_key.data_types);
|
||||
|
@ -29,7 +29,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
bool canBePruned(const DataPartPtr & part);
|
||||
bool canBePruned(const DataPart & part);
|
||||
|
||||
bool isUseless() const { return useless; }
|
||||
|
||||
|
@ -41,7 +41,7 @@ ReplicatedMergeTreeBlockOutputStream::ReplicatedMergeTreeBlockOutputStream(
|
||||
size_t max_parts_per_block_,
|
||||
bool quorum_parallel_,
|
||||
bool deduplicate_,
|
||||
bool optimize_on_insert_,
|
||||
ContextPtr context_,
|
||||
bool is_attach_)
|
||||
: storage(storage_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
@ -52,7 +52,7 @@ ReplicatedMergeTreeBlockOutputStream::ReplicatedMergeTreeBlockOutputStream(
|
||||
, quorum_parallel(quorum_parallel_)
|
||||
, deduplicate(deduplicate_)
|
||||
, log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)"))
|
||||
, optimize_on_insert(optimize_on_insert_)
|
||||
, context(context_)
|
||||
{
|
||||
/// The quorum value `1` has the same meaning as if it is disabled.
|
||||
if (quorum == 1)
|
||||
@ -144,7 +144,7 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block)
|
||||
|
||||
/// Write part to the filesystem under temporary name. Calculate a checksum.
|
||||
|
||||
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, metadata_snapshot, optimize_on_insert);
|
||||
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, metadata_snapshot, context);
|
||||
|
||||
/// If optimize_on_insert setting is true, current_block could become empty after merge
|
||||
/// and we didn't create part.
|
||||
|
@ -30,7 +30,7 @@ public:
|
||||
size_t max_parts_per_block_,
|
||||
bool quorum_parallel_,
|
||||
bool deduplicate_,
|
||||
bool optimize_on_insert,
|
||||
ContextPtr context_,
|
||||
// special flag to determine the ALTER TABLE ATTACH PART without the query context,
|
||||
// needed to set the special LogEntryType::ATTACH_PART
|
||||
bool is_attach_ = false);
|
||||
@ -83,7 +83,7 @@ private:
|
||||
using Logger = Poco::Logger;
|
||||
Poco::Logger * log;
|
||||
|
||||
bool optimize_on_insert;
|
||||
ContextPtr context;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -2007,7 +2007,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts(
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
return MergeTreeData::partsContainSameProjections(left, right);
|
||||
}
|
||||
|
||||
bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart(
|
||||
|
@ -56,6 +56,9 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr
|
||||
ttl_table = formattedAST(metadata_snapshot->getTableTTLs().definition_ast);
|
||||
|
||||
skip_indices = metadata_snapshot->getSecondaryIndices().toString();
|
||||
|
||||
projections = metadata_snapshot->getProjections().toString();
|
||||
|
||||
if (data.canUseAdaptiveGranularity())
|
||||
index_granularity_bytes = data_settings->index_granularity_bytes;
|
||||
else
|
||||
@ -89,6 +92,9 @@ void ReplicatedMergeTreeTableMetadata::write(WriteBuffer & out) const
|
||||
if (!skip_indices.empty())
|
||||
out << "indices: " << skip_indices << "\n";
|
||||
|
||||
if (!projections.empty())
|
||||
out << "projections: " << projections << "\n";
|
||||
|
||||
if (index_granularity_bytes != 0)
|
||||
out << "granularity bytes: " << index_granularity_bytes << "\n";
|
||||
|
||||
@ -130,6 +136,9 @@ void ReplicatedMergeTreeTableMetadata::read(ReadBuffer & in)
|
||||
if (checkString("indices: ", in))
|
||||
in >> skip_indices >> "\n";
|
||||
|
||||
if (checkString("projections: ", in))
|
||||
in >> projections >> "\n";
|
||||
|
||||
if (checkString("granularity bytes: ", in))
|
||||
{
|
||||
in >> index_granularity_bytes >> "\n";
|
||||
@ -243,6 +252,17 @@ void ReplicatedMergeTreeTableMetadata::checkEquals(const ReplicatedMergeTreeTabl
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
}
|
||||
|
||||
String parsed_zk_projections = ProjectionsDescription::parse(from_zk.projections, columns, context).toString();
|
||||
if (projections != parsed_zk_projections)
|
||||
{
|
||||
throw Exception(
|
||||
"Existing table metadata in ZooKeeper differs in projections."
|
||||
" Stored in ZooKeeper: " + from_zk.projections +
|
||||
", parsed from ZooKeeper: " + parsed_zk_projections +
|
||||
", local: " + projections,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
}
|
||||
|
||||
String parsed_zk_constraints = ConstraintsDescription::parse(from_zk.constraints).toString();
|
||||
if (constraints != parsed_zk_constraints)
|
||||
{
|
||||
@ -293,6 +313,12 @@ ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTabl
|
||||
diff.new_skip_indices = from_zk.skip_indices;
|
||||
}
|
||||
|
||||
if (projections != from_zk.projections)
|
||||
{
|
||||
diff.projections_changed = true;
|
||||
diff.new_projections = from_zk.projections;
|
||||
}
|
||||
|
||||
if (constraints != from_zk.constraints)
|
||||
{
|
||||
diff.constraints_changed = true;
|
||||
|
@ -27,6 +27,7 @@ struct ReplicatedMergeTreeTableMetadata
|
||||
String partition_key;
|
||||
String sorting_key;
|
||||
String skip_indices;
|
||||
String projections;
|
||||
String constraints;
|
||||
String ttl_table;
|
||||
UInt64 index_granularity_bytes;
|
||||
@ -54,12 +55,16 @@ struct ReplicatedMergeTreeTableMetadata
|
||||
bool constraints_changed = false;
|
||||
String new_constraints;
|
||||
|
||||
bool projections_changed = false;
|
||||
String new_projections;
|
||||
|
||||
bool ttl_table_changed = false;
|
||||
String new_ttl_table;
|
||||
|
||||
bool empty() const
|
||||
{
|
||||
return !sorting_key_changed && !sampling_expression_changed && !skip_indices_changed && !ttl_table_changed && !constraints_changed;
|
||||
return !sorting_key_changed && !sampling_expression_changed && !skip_indices_changed && !projections_changed
|
||||
&& !ttl_table_changed && !constraints_changed;
|
||||
}
|
||||
};
|
||||
|
||||
|
75
src/Storages/MergeTree/StorageFromBasePartsOfProjection.h
Normal file
75
src/Storages/MergeTree/StorageFromBasePartsOfProjection.h
Normal file
@ -0,0 +1,75 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Defines.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
||||
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/// A Storage that allows reading from a single MergeTree data part.
|
||||
class StorageFromBasePartsOfProjection final : public ext::shared_ptr_helper<StorageFromBasePartsOfProjection>, public IStorage
|
||||
{
|
||||
friend struct ext::shared_ptr_helper<StorageFromBasePartsOfProjection>;
|
||||
|
||||
public:
|
||||
String getName() const override { return "FromBasePartsOfProjection"; }
|
||||
|
||||
Pipe read(
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum /*processed_stage*/,
|
||||
size_t max_block_size,
|
||||
unsigned num_streams) override
|
||||
{
|
||||
// NOTE: It's used to read normal parts only
|
||||
QueryPlan query_plan = std::move(*MergeTreeDataSelectExecutor(storage).readFromParts(
|
||||
{},
|
||||
column_names,
|
||||
metadata_snapshot,
|
||||
metadata_snapshot,
|
||||
query_info,
|
||||
context,
|
||||
max_block_size,
|
||||
num_streams,
|
||||
nullptr,
|
||||
query_info.projection ? query_info.projection->merge_tree_data_select_base_cache.get()
|
||||
: query_info.merge_tree_data_select_cache.get()));
|
||||
|
||||
return query_plan.convertToPipe(
|
||||
QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
|
||||
}
|
||||
|
||||
|
||||
bool supportsIndexForIn() const override { return true; }
|
||||
|
||||
bool mayBenefitFromIndexForIn(
|
||||
const ASTPtr & left_in_operand, ContextPtr query_context, const StorageMetadataPtr & metadata_snapshot) const override
|
||||
{
|
||||
return storage.mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot);
|
||||
}
|
||||
|
||||
NamesAndTypesList getVirtuals() const override { return storage.getVirtuals(); }
|
||||
|
||||
protected:
|
||||
StorageFromBasePartsOfProjection(const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot)
|
||||
: IStorage(storage_.getStorageID()), storage(storage_)
|
||||
{
|
||||
setInMemoryMetadata(*metadata_snapshot);
|
||||
}
|
||||
|
||||
|
||||
private:
|
||||
const MergeTreeData & storage;
|
||||
};
|
||||
|
||||
}
|
@ -31,11 +31,23 @@ public:
|
||||
size_t max_block_size,
|
||||
unsigned num_streams) override
|
||||
{
|
||||
QueryPlan query_plan =
|
||||
std::move(*MergeTreeDataSelectExecutor(part->storage)
|
||||
.readFromParts({part}, column_names, metadata_snapshot, query_info, context, max_block_size, num_streams));
|
||||
// NOTE: It's used to read normal parts only
|
||||
QueryPlan query_plan = std::move(*MergeTreeDataSelectExecutor(parts.front()->storage)
|
||||
.readFromParts(
|
||||
parts,
|
||||
column_names,
|
||||
metadata_snapshot,
|
||||
metadata_snapshot,
|
||||
query_info,
|
||||
context,
|
||||
max_block_size,
|
||||
num_streams,
|
||||
nullptr,
|
||||
query_info.projection ? query_info.projection->merge_tree_data_select_base_cache.get()
|
||||
: query_info.merge_tree_data_select_cache.get()));
|
||||
|
||||
return query_plan.convertToPipe(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
|
||||
return query_plan.convertToPipe(
|
||||
QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
|
||||
}
|
||||
|
||||
|
||||
@ -44,40 +56,54 @@ public:
|
||||
bool mayBenefitFromIndexForIn(
|
||||
const ASTPtr & left_in_operand, ContextPtr query_context, const StorageMetadataPtr & metadata_snapshot) const override
|
||||
{
|
||||
return part->storage.mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot);
|
||||
return parts.front()->storage.mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot);
|
||||
}
|
||||
|
||||
NamesAndTypesList getVirtuals() const override
|
||||
{
|
||||
return part->storage.getVirtuals();
|
||||
return parts.front()->storage.getVirtuals();
|
||||
}
|
||||
|
||||
String getPartitionId() const
|
||||
{
|
||||
return part->info.partition_id;
|
||||
return parts.front()->info.partition_id;
|
||||
}
|
||||
|
||||
String getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr context) const
|
||||
{
|
||||
return part->storage.getPartitionIDFromQuery(ast, context);
|
||||
return parts.front()->storage.getPartitionIDFromQuery(ast, context);
|
||||
}
|
||||
|
||||
protected:
|
||||
StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_)
|
||||
: IStorage(getIDFromPart(part_))
|
||||
, part(part_)
|
||||
, parts({part_})
|
||||
{
|
||||
setInMemoryMetadata(part_->storage.getInMemoryMetadata());
|
||||
}
|
||||
|
||||
StorageFromMergeTreeDataPart(MergeTreeData::DataPartsVector && parts_)
|
||||
: IStorage(getIDFromParts(parts_))
|
||||
, parts(std::move(parts_))
|
||||
{
|
||||
setInMemoryMetadata(parts.front()->storage.getInMemoryMetadata());
|
||||
}
|
||||
|
||||
private:
|
||||
MergeTreeData::DataPartPtr part;
|
||||
MergeTreeData::DataPartsVector parts;
|
||||
|
||||
static StorageID getIDFromPart(const MergeTreeData::DataPartPtr & part_)
|
||||
{
|
||||
auto table_id = part_->storage.getStorageID();
|
||||
return StorageID(table_id.database_name, table_id.table_name + " (part " + part_->name + ")");
|
||||
}
|
||||
|
||||
static StorageID getIDFromParts(const MergeTreeData::DataPartsVector & parts_)
|
||||
{
|
||||
assert(!parts_.empty());
|
||||
auto table_id = parts_.front()->storage.getStorageID();
|
||||
return StorageID(table_id.database_name, table_id.table_name + " (parts)");
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -29,6 +29,7 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_ALLOCATE_MEMORY;
|
||||
extern const int CANNOT_MUNMAP;
|
||||
extern const int CANNOT_MREMAP;
|
||||
extern const int UNEXPECTED_FILE_IN_DATA_PART;
|
||||
}
|
||||
|
||||
|
||||
@ -45,6 +46,7 @@ bool isNotEnoughMemoryErrorCode(int code)
|
||||
|
||||
|
||||
IMergeTreeDataPart::Checksums checkDataPart(
|
||||
MergeTreeData::DataPartPtr data_part,
|
||||
const DiskPtr & disk,
|
||||
const String & full_relative_path,
|
||||
const NamesAndTypesList & columns_list,
|
||||
@ -98,12 +100,100 @@ IMergeTreeDataPart::Checksums checkDataPart(
|
||||
};
|
||||
|
||||
/// This function calculates only checksum of file content (compressed or uncompressed).
|
||||
auto checksum_file = [](const DiskPtr & disk_, const String & file_path)
|
||||
/// It also calculates checksum of projections.
|
||||
auto checksum_file = [&](const String & file_path, const String & file_name)
|
||||
{
|
||||
auto file_buf = disk_->readFile(file_path);
|
||||
HashingReadBuffer hashing_buf(*file_buf);
|
||||
hashing_buf.ignoreAll();
|
||||
return IMergeTreeDataPart::Checksums::Checksum{hashing_buf.count(), hashing_buf.getHash()};
|
||||
if (disk->isDirectory(file_path) && endsWith(file_name, ".proj") && !startsWith(file_name, "tmp_")) // ignore projection tmp merge dir
|
||||
{
|
||||
auto projection_name = file_name.substr(0, file_name.size() - sizeof(".proj") + 1);
|
||||
auto pit = data_part->getProjectionParts().find(projection_name);
|
||||
if (pit == data_part->getProjectionParts().end())
|
||||
{
|
||||
if (require_checksums)
|
||||
throw Exception("Unexpected file " + file_name + " in data part", ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART);
|
||||
else
|
||||
return;
|
||||
}
|
||||
|
||||
const auto & projection = pit->second;
|
||||
IMergeTreeDataPart::Checksums projection_checksums_data;
|
||||
const auto & projection_path = file_path;
|
||||
|
||||
if (part_type == MergeTreeDataPartType::COMPACT)
|
||||
{
|
||||
auto proj_path = file_path + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION;
|
||||
auto file_buf = disk->readFile(proj_path);
|
||||
HashingReadBuffer hashing_buf(*file_buf);
|
||||
hashing_buf.ignoreAll();
|
||||
projection_checksums_data.files[MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION] = IMergeTreeDataPart::Checksums::Checksum(hashing_buf.count(), hashing_buf.getHash());
|
||||
}
|
||||
else
|
||||
{
|
||||
const NamesAndTypesList & projection_columns_list = projection->getColumns();
|
||||
for (const auto & projection_column : projection_columns_list)
|
||||
{
|
||||
auto serialization = IDataType::getSerialization(projection_column, [&](const String & stream_name)
|
||||
{
|
||||
return disk->exists(stream_name + IMergeTreeDataPart::DATA_FILE_EXTENSION);
|
||||
});
|
||||
|
||||
serialization->enumerateStreams(
|
||||
[&](const ISerialization::SubstreamPath & substream_path)
|
||||
{
|
||||
String projection_file_name = ISerialization::getFileNameForStream(projection_column, substream_path) + ".bin";
|
||||
checksums_data.files[projection_file_name] = checksum_compressed_file(disk, projection_path + projection_file_name);
|
||||
},
|
||||
{});
|
||||
}
|
||||
}
|
||||
|
||||
IMergeTreeDataPart::Checksums projection_checksums_txt;
|
||||
|
||||
if (require_checksums || disk->exists(projection_path + "checksums.txt"))
|
||||
{
|
||||
auto buf = disk->readFile(projection_path + "checksums.txt");
|
||||
projection_checksums_txt.read(*buf);
|
||||
assertEOF(*buf);
|
||||
}
|
||||
|
||||
const auto & projection_checksum_files_txt = projection_checksums_txt.files;
|
||||
for (auto projection_it = disk->iterateDirectory(projection_path); projection_it->isValid(); projection_it->next())
|
||||
{
|
||||
const String & projection_file_name = projection_it->name();
|
||||
auto projection_checksum_it = projection_checksums_data.files.find(projection_file_name);
|
||||
|
||||
/// Skip files that we already calculated. Also skip metadata files that are not checksummed.
|
||||
if (projection_checksum_it == projection_checksums_data.files.end() && !files_without_checksums.count(projection_file_name))
|
||||
{
|
||||
auto projection_txt_checksum_it = projection_checksum_files_txt.find(file_name);
|
||||
if (projection_txt_checksum_it == projection_checksum_files_txt.end()
|
||||
|| projection_txt_checksum_it->second.uncompressed_size == 0)
|
||||
{
|
||||
auto projection_file_buf = disk->readFile(projection_it->path());
|
||||
HashingReadBuffer projection_hashing_buf(*projection_file_buf);
|
||||
projection_hashing_buf.ignoreAll();
|
||||
projection_checksums_data.files[projection_file_name] = IMergeTreeDataPart::Checksums::Checksum(
|
||||
projection_hashing_buf.count(), projection_hashing_buf.getHash());
|
||||
}
|
||||
else
|
||||
{
|
||||
projection_checksums_data.files[projection_file_name] = checksum_compressed_file(disk, projection_it->path());
|
||||
}
|
||||
}
|
||||
}
|
||||
checksums_data.files[file_name] = IMergeTreeDataPart::Checksums::Checksum(
|
||||
projection_checksums_data.getTotalSizeOnDisk(), projection_checksums_data.getTotalChecksumUInt128());
|
||||
|
||||
if (require_checksums || !projection_checksums_txt.files.empty())
|
||||
projection_checksums_txt.checkEqual(projection_checksums_data, false);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto file_buf = disk->readFile(file_path);
|
||||
HashingReadBuffer hashing_buf(*file_buf);
|
||||
hashing_buf.ignoreAll();
|
||||
checksums_data.files[file_name] = IMergeTreeDataPart::Checksums::Checksum(hashing_buf.count(), hashing_buf.getHash());
|
||||
}
|
||||
};
|
||||
|
||||
bool check_uncompressed = true;
|
||||
@ -111,7 +201,7 @@ IMergeTreeDataPart::Checksums checkDataPart(
|
||||
if (part_type == MergeTreeDataPartType::COMPACT)
|
||||
{
|
||||
const auto & file_name = MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION;
|
||||
checksums_data.files[file_name] = checksum_file(disk, path + file_name);
|
||||
checksum_file(path + file_name, file_name);
|
||||
/// Uncompressed checksums in compact parts are computed in a complex way.
|
||||
/// We check only checksum of compressed file.
|
||||
check_uncompressed = false;
|
||||
@ -161,9 +251,9 @@ IMergeTreeDataPart::Checksums checkDataPart(
|
||||
if (txt_checksum_it == checksum_files_txt.end() || txt_checksum_it->second.uncompressed_size == 0)
|
||||
{
|
||||
/// The file is not compressed.
|
||||
checksums_data.files[file_name] = checksum_file(disk, it->path());
|
||||
checksum_file(it->path(), file_name);
|
||||
}
|
||||
else /// If we have both compressed and uncompressed in txt, than calculate them
|
||||
else /// If we have both compressed and uncompressed in txt, then calculate them
|
||||
{
|
||||
checksums_data.files[file_name] = checksum_compressed_file(disk, it->path());
|
||||
}
|
||||
@ -196,6 +286,7 @@ IMergeTreeDataPart::Checksums checkDataPart(
|
||||
return checkDataPartInMemory(part_in_memory);
|
||||
|
||||
return checkDataPart(
|
||||
data_part,
|
||||
data_part->volume->getDisk(),
|
||||
data_part->getFullRelativePath(),
|
||||
data_part->getColumns(),
|
||||
|
@ -290,7 +290,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
|
||||
bool is_extended_storage_def = args.storage_def->partition_by || args.storage_def->primary_key || args.storage_def->order_by
|
||||
|| args.storage_def->sample_by || (args.query.columns_list->indices && !args.query.columns_list->indices->children.empty())
|
||||
|| args.storage_def->settings;
|
||||
|| (args.query.columns_list->projections && !args.query.columns_list->projections->children.empty()) || args.storage_def->settings;
|
||||
|
||||
String name_part = args.engine_name.substr(0, args.engine_name.size() - strlen("MergeTree"));
|
||||
|
||||
@ -688,6 +688,13 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
for (auto & index : args.query.columns_list->indices->children)
|
||||
metadata.secondary_indices.push_back(IndexDescription::getIndexFromAST(index, args.columns, args.getContext()));
|
||||
|
||||
if (args.query.columns_list && args.query.columns_list->projections)
|
||||
for (auto & projection_ast : args.query.columns_list->projections->children)
|
||||
{
|
||||
auto projection = ProjectionDescription::getProjectionFromAST(projection_ast, args.columns, args.getContext());
|
||||
metadata.projections.add(std::move(projection));
|
||||
}
|
||||
|
||||
if (args.query.columns_list && args.query.columns_list->constraints)
|
||||
for (auto & constraint : args.query.columns_list->constraints->children)
|
||||
metadata.constraints.constraints.push_back(constraint);
|
||||
@ -802,6 +809,7 @@ void registerStorageMergeTree(StorageFactory & factory)
|
||||
StorageFactory::StorageFeatures features{
|
||||
.supports_settings = true,
|
||||
.supports_skipping_indices = true,
|
||||
.supports_projections = true,
|
||||
.supports_sort_order = true,
|
||||
.supports_ttl = true,
|
||||
.supports_parallel_insert = true,
|
||||
|
@ -65,6 +65,16 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
|
||||
res.index_name = command->index->as<ASTIdentifier &>().name();
|
||||
return res;
|
||||
}
|
||||
else if (command->type == ASTAlterCommand::MATERIALIZE_PROJECTION)
|
||||
{
|
||||
MutationCommand res;
|
||||
res.ast = command->ptr();
|
||||
res.type = MATERIALIZE_PROJECTION;
|
||||
res.partition = command->partition;
|
||||
res.predicate = nullptr;
|
||||
res.projection_name = command->projection->as<ASTIdentifier &>().name();
|
||||
return res;
|
||||
}
|
||||
else if (parse_alter_commands && command->type == ASTAlterCommand::MODIFY_COLUMN)
|
||||
{
|
||||
MutationCommand res;
|
||||
@ -100,6 +110,18 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
|
||||
res.clear = true;
|
||||
return res;
|
||||
}
|
||||
else if (parse_alter_commands && command->type == ASTAlterCommand::DROP_PROJECTION)
|
||||
{
|
||||
MutationCommand res;
|
||||
res.ast = command->ptr();
|
||||
res.type = MutationCommand::Type::DROP_PROJECTION;
|
||||
res.column_name = command->projection->as<ASTIdentifier &>().name();
|
||||
if (command->partition)
|
||||
res.partition = command->partition;
|
||||
if (command->clear_projection)
|
||||
res.clear = true;
|
||||
return res;
|
||||
}
|
||||
else if (parse_alter_commands && command->type == ASTAlterCommand::RENAME_COLUMN)
|
||||
{
|
||||
MutationCommand res;
|
||||
|
@ -28,9 +28,11 @@ struct MutationCommand
|
||||
DELETE,
|
||||
UPDATE,
|
||||
MATERIALIZE_INDEX,
|
||||
MATERIALIZE_PROJECTION,
|
||||
READ_COLUMN, /// Read column and apply conversions (MODIFY COLUMN alter query).
|
||||
DROP_COLUMN,
|
||||
DROP_INDEX,
|
||||
DROP_PROJECTION,
|
||||
MATERIALIZE_TTL,
|
||||
RENAME_COLUMN,
|
||||
};
|
||||
@ -43,8 +45,9 @@ struct MutationCommand
|
||||
/// Columns with corresponding actions
|
||||
std::unordered_map<String, ASTPtr> column_to_update_expression;
|
||||
|
||||
/// For MATERIALIZE INDEX.
|
||||
/// For MATERIALIZE INDEX and PROJECTION
|
||||
String index_name;
|
||||
String projection_name;
|
||||
|
||||
/// For MATERIALIZE INDEX, UPDATE and DELETE.
|
||||
ASTPtr partition;
|
||||
|
290
src/Storages/ProjectionsDescription.cpp
Normal file
290
src/Storages/ProjectionsDescription.cpp
Normal file
@ -0,0 +1,290 @@
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Storages/ProjectionsDescription.h>
|
||||
|
||||
#include <Parsers/ASTProjectionDeclaration.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
|
||||
#include <Core/Defines.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Parsers/ASTProjectionSelectQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Processors/Pipe.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_QUERY;
|
||||
extern const int NO_SUCH_PROJECTION_IN_TABLE;
|
||||
extern const int ILLEGAL_PROJECTION;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
};
|
||||
|
||||
const char * ProjectionDescription::typeToString(Type type)
|
||||
{
|
||||
switch (type)
|
||||
{
|
||||
case Type::Normal:
|
||||
return "normal";
|
||||
case Type::Aggregate:
|
||||
return "aggregate";
|
||||
}
|
||||
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
|
||||
bool ProjectionDescription::isPrimaryKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const
|
||||
{
|
||||
const String column_name = node->getColumnName();
|
||||
|
||||
for (const auto & key_name : metadata->getPrimaryKeyColumns())
|
||||
if (column_name == key_name)
|
||||
return true;
|
||||
|
||||
if (const auto * func = node->as<ASTFunction>())
|
||||
if (func->arguments->children.size() == 1)
|
||||
return isPrimaryKeyColumnPossiblyWrappedInFunctions(func->arguments->children.front());
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
ProjectionDescription ProjectionDescription::clone() const
|
||||
{
|
||||
ProjectionDescription other;
|
||||
if (definition_ast)
|
||||
other.definition_ast = definition_ast->clone();
|
||||
if (query_ast)
|
||||
other.query_ast = query_ast->clone();
|
||||
|
||||
other.name = name;
|
||||
other.type = type;
|
||||
other.required_columns = required_columns;
|
||||
other.column_names = column_names;
|
||||
other.data_types = data_types;
|
||||
other.sample_block = sample_block;
|
||||
other.sample_block_for_keys = sample_block_for_keys;
|
||||
other.metadata = metadata;
|
||||
other.key_size = key_size;
|
||||
|
||||
return other;
|
||||
}
|
||||
|
||||
ProjectionsDescription ProjectionsDescription::clone() const
|
||||
{
|
||||
ProjectionsDescription other;
|
||||
for (const auto & projection : projections)
|
||||
other.add(projection.clone());
|
||||
|
||||
return other;
|
||||
}
|
||||
|
||||
bool ProjectionDescription::operator==(const ProjectionDescription & other) const
|
||||
{
|
||||
return name == other.name && queryToString(definition_ast) == queryToString(other.definition_ast);
|
||||
}
|
||||
|
||||
ProjectionDescription
|
||||
ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr query_context)
|
||||
{
|
||||
const auto * projection_definition = definition_ast->as<ASTProjectionDeclaration>();
|
||||
|
||||
if (!projection_definition)
|
||||
throw Exception("Cannot create projection from non ASTProjectionDeclaration AST", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
if (projection_definition->name.empty())
|
||||
throw Exception("Projection must have name in definition.", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
if (!projection_definition->query)
|
||||
throw Exception("QUERY is required for projection", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
ProjectionDescription result;
|
||||
result.definition_ast = projection_definition->clone();
|
||||
result.name = projection_definition->name;
|
||||
|
||||
auto query = projection_definition->query->as<ASTProjectionSelectQuery &>();
|
||||
result.query_ast = query.cloneToASTSelect();
|
||||
|
||||
auto external_storage_holder = std::make_shared<TemporaryTableHolder>(query_context, columns, ConstraintsDescription{});
|
||||
StoragePtr storage = external_storage_holder->getTable();
|
||||
InterpreterSelectQuery select(
|
||||
result.query_ast, query_context, storage, {}, SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias());
|
||||
|
||||
result.required_columns = select.getRequiredColumns();
|
||||
result.sample_block = select.getSampleBlock();
|
||||
|
||||
const auto & analysis_result = select.getAnalysisResult();
|
||||
if (analysis_result.need_aggregate)
|
||||
{
|
||||
for (const auto & key : select.getQueryAnalyzer()->aggregationKeys())
|
||||
result.sample_block_for_keys.insert({nullptr, key.type, key.name});
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < result.sample_block.columns(); ++i)
|
||||
{
|
||||
const auto & column_with_type_name = result.sample_block.getByPosition(i);
|
||||
|
||||
if (column_with_type_name.column && isColumnConst(*column_with_type_name.column))
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Projections cannot contain constant columns: {}", column_with_type_name.name);
|
||||
|
||||
result.column_names.emplace_back(column_with_type_name.name);
|
||||
result.data_types.emplace_back(column_with_type_name.type);
|
||||
}
|
||||
|
||||
StorageInMemoryMetadata metadata;
|
||||
metadata.setColumns(ColumnsDescription(result.sample_block.getNamesAndTypesList()));
|
||||
metadata.partition_key = KeyDescription::getSortingKeyFromAST({}, metadata.columns, query_context, {});
|
||||
|
||||
const auto & query_select = result.query_ast->as<const ASTSelectQuery &>();
|
||||
if (select.hasAggregation())
|
||||
{
|
||||
result.type = ProjectionDescription::Type::Aggregate;
|
||||
if (const auto & group_expression_list = query_select.groupBy())
|
||||
{
|
||||
ASTPtr order_expression;
|
||||
if (group_expression_list->children.size() == 1)
|
||||
{
|
||||
result.key_size = 1;
|
||||
order_expression = std::make_shared<ASTIdentifier>(group_expression_list->children.front()->getColumnName());
|
||||
}
|
||||
else
|
||||
{
|
||||
auto function_node = std::make_shared<ASTFunction>();
|
||||
function_node->name = "tuple";
|
||||
function_node->arguments = group_expression_list->clone();
|
||||
result.key_size = function_node->arguments->children.size();
|
||||
for (auto & child : function_node->arguments->children)
|
||||
child = std::make_shared<ASTIdentifier>(child->getColumnName());
|
||||
function_node->children.push_back(function_node->arguments);
|
||||
order_expression = function_node;
|
||||
}
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST(order_expression, metadata.columns, query_context, {});
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(order_expression, metadata.columns, query_context);
|
||||
}
|
||||
else
|
||||
{
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST({}, metadata.columns, query_context, {});
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST({}, metadata.columns, query_context);
|
||||
}
|
||||
if (query_select.orderBy())
|
||||
throw Exception(
|
||||
"When aggregation is used in projection, ORDER BY cannot be specified", ErrorCodes::ILLEGAL_PROJECTION);
|
||||
}
|
||||
else
|
||||
{
|
||||
result.type = ProjectionDescription::Type::Normal;
|
||||
metadata.sorting_key = KeyDescription::getSortingKeyFromAST(query_select.orderBy(), metadata.columns, query_context, {});
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(query_select.orderBy(), metadata.columns, query_context);
|
||||
}
|
||||
metadata.primary_key.definition_ast = nullptr;
|
||||
result.metadata = std::make_shared<StorageInMemoryMetadata>(metadata);
|
||||
return result;
|
||||
}
|
||||
|
||||
void ProjectionDescription::recalculateWithNewColumns(const ColumnsDescription & new_columns, ContextPtr query_context)
|
||||
{
|
||||
*this = getProjectionFromAST(definition_ast, new_columns, query_context);
|
||||
}
|
||||
|
||||
String ProjectionsDescription::toString() const
|
||||
{
|
||||
if (empty())
|
||||
return {};
|
||||
|
||||
ASTExpressionList list;
|
||||
for (const auto & projection : projections)
|
||||
list.children.push_back(projection.definition_ast);
|
||||
|
||||
return serializeAST(list, true);
|
||||
}
|
||||
|
||||
ProjectionsDescription ProjectionsDescription::parse(const String & str, const ColumnsDescription & columns, ContextPtr query_context)
|
||||
{
|
||||
ProjectionsDescription result;
|
||||
if (str.empty())
|
||||
return result;
|
||||
|
||||
ParserProjectionDeclarationList parser;
|
||||
ASTPtr list = parseQuery(parser, str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||
|
||||
for (const auto & projection_ast : list->children)
|
||||
{
|
||||
auto projection = ProjectionDescription::getProjectionFromAST(projection_ast, columns, query_context);
|
||||
result.add(std::move(projection));
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
bool ProjectionsDescription::has(const String & projection_name) const
|
||||
{
|
||||
return map.count(projection_name) > 0;
|
||||
}
|
||||
|
||||
const ProjectionDescription & ProjectionsDescription::get(const String & projection_name) const
|
||||
{
|
||||
auto it = map.find(projection_name);
|
||||
if (it == map.end())
|
||||
throw Exception("There is no projection " + projection_name + " in table", ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE);
|
||||
|
||||
return *(it->second);
|
||||
}
|
||||
|
||||
void ProjectionsDescription::add(ProjectionDescription && projection, const String & after_projection, bool first, bool if_not_exists)
|
||||
{
|
||||
if (has(projection.name))
|
||||
{
|
||||
if (if_not_exists)
|
||||
return;
|
||||
throw Exception(
|
||||
"Cannot add projection " + projection.name + ": projection with this name already exists", ErrorCodes::ILLEGAL_PROJECTION);
|
||||
}
|
||||
|
||||
auto insert_it = projections.cend();
|
||||
|
||||
if (first)
|
||||
insert_it = projections.cbegin();
|
||||
else if (!after_projection.empty())
|
||||
{
|
||||
auto it = std::find_if(projections.cbegin(), projections.cend(), [&after_projection](const auto & projection_)
|
||||
{
|
||||
return projection_.name == after_projection;
|
||||
});
|
||||
if (it != projections.cend())
|
||||
++it;
|
||||
insert_it = it;
|
||||
}
|
||||
|
||||
auto it = projections.insert(insert_it, std::move(projection));
|
||||
map[it->name] = it;
|
||||
}
|
||||
|
||||
void ProjectionsDescription::remove(const String & projection_name)
|
||||
{
|
||||
auto it = map.find(projection_name);
|
||||
if (it == map.end())
|
||||
throw Exception("There is no projection " + projection_name + " in table.", ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE);
|
||||
|
||||
projections.erase(it->second);
|
||||
map.erase(it);
|
||||
}
|
||||
|
||||
ExpressionActionsPtr
|
||||
ProjectionsDescription::getSingleExpressionForProjections(const ColumnsDescription & columns, ContextPtr query_context) const
|
||||
{
|
||||
ASTPtr combined_expr_list = std::make_shared<ASTExpressionList>();
|
||||
for (const auto & projection : projections)
|
||||
for (const auto & projection_expr : projection.query_ast->children)
|
||||
combined_expr_list->children.push_back(projection_expr->clone());
|
||||
|
||||
auto syntax_result = TreeRewriter(query_context).analyze(combined_expr_list, columns.getAllPhysical());
|
||||
return ExpressionAnalyzer(combined_expr_list, syntax_result, query_context).getActions(false);
|
||||
}
|
||||
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user