ClickHouse/src/Interpreters/InterpreterOptimizeQuery.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

97 lines
3.9 KiB
C++
Raw Normal View History

#include <Storages/IStorage.h>
#include <Parsers/ASTOptimizeQuery.h>
2017-05-23 18:24:43 +00:00
#include <Interpreters/Context.h>
2020-11-03 13:47:26 +00:00
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/InterpreterOptimizeQuery.h>
#include <Access/Common/AccessRightsElement.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
#include <Parsers/ASTExpressionList.h>
2022-10-01 22:44:46 +00:00
#include <Storages/MergeTree/MergeTreeData.h>
2016-12-12 07:24:56 +00:00
#include <Interpreters/processColumnTransformers.h>
#include <memory>
2016-12-12 07:24:56 +00:00
namespace DB
{
2017-04-08 01:32:05 +00:00
namespace ErrorCodes
{
extern const int THERE_IS_NO_COLUMN;
2017-04-08 01:32:05 +00:00
}
2016-12-12 07:24:56 +00:00
BlockIO InterpreterOptimizeQuery::execute()
{
const auto & ast = query_ptr->as<ASTOptimizeQuery &>();
2016-12-12 07:24:56 +00:00
if (!ast.cluster.empty())
{
DDLQueryOnClusterParams params;
params.access_to_check = getRequiredAccess();
return executeDDLQueryOnCluster(query_ptr, getContext(), params);
}
2020-01-24 16:20:36 +00:00
getContext()->checkAccess(getRequiredAccess());
auto table_id = getContext()->resolveStorageID(ast, Context::ResolveOrdinary);
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext());
checkStorageSupportsTransactionsIfNeeded(table, getContext());
2020-06-17 13:39:26 +00:00
auto metadata_snapshot = table->getInMemoryMetadataPtr();
auto storage_snapshot = table->getStorageSnapshot(metadata_snapshot, getContext());
// Empty list of names means we deduplicate by all columns, but user can explicitly state which columns to use.
Names column_names;
if (ast.deduplicate_by_columns)
{
// User requested custom set of columns for deduplication, possibly with Column Transformer expression.
{
// Expand asterisk, column transformers, etc into list of column names.
const auto cols
= processColumnTransformers(getContext()->getCurrentDatabase(), table, metadata_snapshot, ast.deduplicate_by_columns);
for (const auto & col : cols->children)
column_names.emplace_back(col->getColumnName());
}
storage_snapshot->check(column_names);
Names required_columns;
{
required_columns = metadata_snapshot->getColumnsRequiredForSortingKey();
const auto partitioning_cols = metadata_snapshot->getColumnsRequiredForPartitionKey();
required_columns.reserve(required_columns.size() + partitioning_cols.size());
required_columns.insert(required_columns.end(), partitioning_cols.begin(), partitioning_cols.end());
}
for (const auto & required_col : required_columns)
{
// Deduplication is performed only for adjacent rows in a block,
// and all rows in block are in the sorting key order within a single partition,
// hence deduplication always implicitly takes sorting keys and partition keys in account.
// So we just explicitly state that limitation in order to avoid confusion.
if (std::find(column_names.begin(), column_names.end(), required_col) == column_names.end())
throw Exception(ErrorCodes::THERE_IS_NO_COLUMN,
"DEDUPLICATE BY expression must include all columns used in table's"
" ORDER BY, PRIMARY KEY, or PARTITION BY but '{}' is missing."
" Expanded DEDUPLICATE BY columns expression: ['{}']",
required_col, fmt::join(column_names, "', '"));
}
}
2022-10-01 22:44:46 +00:00
if (auto * snapshot_data = dynamic_cast<MergeTreeData::SnapshotData *>(storage_snapshot->data.get()))
snapshot_data->parts = {};
[RFC] Replacing merge tree new engine (#41005) * Add new engine to ReplacingMergeTree corresponding to the ReplacingCollapsingMergeTree * Add new test for the new ReplacingMergeTree engine * Limit sign value to -1/1 * Add new engine to ReplacingMergeTree corresponding to the ReplacingCollapsingMergeTree * Add new test for the new ReplacingMergeTree engine * Limit sign value to -1/1 * Replace sign column(Int8) by is_deleted(UInt8) * Add new engine to ReplacingMergeTree corresponding to the ReplacingCollapsingMergeTree * Add new test for the new ReplacingMergeTree engine * Limit sign value to -1/1 * Replace sign column(Int8) by is_deleted(UInt8) * Add new engine to ReplacingMergeTree corresponding to the ReplacingCollapsingMergeTree * Add new test for the new ReplacingMergeTree engine * Limit sign value to -1/1 * Replace sign column(Int8) by is_deleted(UInt8) * Add keyword 'CLEANUP' when OPTIMIZE * Cleanup uniquely when it's a replacingMergeTree * Propagate CLEANUP information and change from 'with_cleanup' to 'cleanup' * Cleanup data flagged as 'is_deleted' * Fix merge when optimize and add a test * Fix OPTIMIZE and INSERT + add tests * New fix for cleanup at the merge * Cleanup debug logs * Add the SETTINGS option 'clean_deleted_rows' that can be 'never' or 'always' * Fix regression bug; Now REplicatedMergeTree can be called as before without 'is_deleted' * Add Replicated tests * Disable tag 'long' for our test and cleanup some white spaces * Update tests * Fix tests and remove additional useless whitespace * Fix replica test * Style clean && add condition check for is_deleted values * clean_deleted_rows settings is nom an enum * Add valid default value to the clean_deleted_rows settings * Update cleanup checkers to use the enum and fix typos in the test * Fix submodule contrib/AMQP-CPP pointer * Add missing messages in test reference and remove a print with non derterministic order * fix replica test reference * Fix edge case * Fix a typo for the spell checker * Fix reference * Fix a condition to raise an error if is_deleted differ from 0/1 and cleanup * Change tests file name and update number * This should fix the ReplacingMergeTree parameter set * Fix replicated parameters * Disable allow_deprecated_syntax_for_merge_tree for our new column * Fix a test * Remove non deterministic order print in the test * Test on replicas * Remove a condition, when checking optional parameters, that should not be sueful since we disabled the deprected_syntaxe * Revert "Remove a condition, when checking optional parameters, that should not be useful since we disabled the deprected_syntaxe" This reverts commit b65d64c05e482945ac20fcfcf0311e1b028ea137. * Fix replica management and limit the number of argument to two maximum, due to the possiblity of deprecated table create/attach failing otherwise * Test a fix for replicated log information error * Try to add sync to have consistent results * Change path of replicas that should cause one issue and add few prints in case it's not that * Get cleanup info on replicas only if information found * Fix style issues * Try to avoid replication error 'cannot select parts...' and and replica read/write field order * Cleanup according to PR reviews and add tests on error raised. * Update src/Storages/MergeTree/registerStorageMergeTree.cpp Co-authored-by: Alexander Tokmakov <tavplubix@gmail.com> * Select ... FINAL don't show rows with is_deleted = true * Update and fix SELECT ... FINAL merge parameter * Remove is_deleted rows only on the version inserted when merge * Fix (master) updates issues * Revert changes that should not be commited * Add changes according to review * Revert changes that should not be commited - part 2 --------- Co-authored-by: Alexander Tokmakov <tavplubix@gmail.com>
2023-02-16 13:03:16 +00:00
table->optimize(query_ptr, metadata_snapshot, ast.partition, ast.final, ast.deduplicate, column_names, ast.cleanup, getContext());
2016-12-12 07:24:56 +00:00
return {};
}
2020-01-24 16:20:36 +00:00
AccessRightsElements InterpreterOptimizeQuery::getRequiredAccess() const
{
const auto & optimize = query_ptr->as<const ASTOptimizeQuery &>();
AccessRightsElements required_access;
required_access.emplace_back(AccessType::OPTIMIZE, optimize.getDatabase(), optimize.getTable());
2020-01-24 16:20:36 +00:00
return required_access;
}
2016-12-12 07:24:56 +00:00
}