mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #11745 from ClickHouse/atomic_metadata5
Atomic metadata.
This commit is contained in:
commit
55eee9bfa1
@ -167,7 +167,7 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header,
|
||||
auto temporary_table = TemporaryTableHolder(context, ColumnsDescription{columns}, {});
|
||||
auto storage = temporary_table.getTable();
|
||||
context.addExternalTable(data->table_name, std::move(temporary_table));
|
||||
BlockOutputStreamPtr output = storage->write(ASTPtr(), context);
|
||||
BlockOutputStreamPtr output = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), context);
|
||||
|
||||
/// Write data
|
||||
auto sink = std::make_shared<SinkToOutputStream>(std::move(output));
|
||||
|
@ -49,11 +49,11 @@ std::ostream & operator<<(std::ostream & stream, const IStorage & what)
|
||||
{
|
||||
auto table_id = what.getStorageID();
|
||||
stream << "IStorage(name = " << what.getName() << ", tableName = " << table_id.table_name << ") {"
|
||||
<< what.getColumns().getAllPhysical().toString() << "}";
|
||||
<< what.getInMemoryMetadataPtr()->getColumns().getAllPhysical().toString() << "}";
|
||||
return stream;
|
||||
}
|
||||
|
||||
std::ostream & operator<<(std::ostream & stream, const TableStructureReadLock &)
|
||||
std::ostream & operator<<(std::ostream & stream, const TableLockHolder &)
|
||||
{
|
||||
stream << "TableStructureReadLock()";
|
||||
return stream;
|
||||
|
@ -22,9 +22,6 @@ std::ostream & operator<<(std::ostream & stream, const IDataType & what);
|
||||
class IStorage;
|
||||
std::ostream & operator<<(std::ostream & stream, const IStorage & what);
|
||||
|
||||
class TableStructureReadLock;
|
||||
std::ostream & operator<<(std::ostream & stream, const TableStructureReadLock & what);
|
||||
|
||||
class IFunctionOverloadResolver;
|
||||
std::ostream & operator<<(std::ostream & stream, const IFunctionOverloadResolver & what);
|
||||
|
||||
|
@ -101,7 +101,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery)
|
||||
|
||||
BlockOutputStreamPtr table_out;
|
||||
if (subquery.table)
|
||||
table_out = subquery.table->write({}, context);
|
||||
table_out = subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), context);
|
||||
|
||||
bool done_with_set = !subquery.set;
|
||||
bool done_with_join = !subquery.join;
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <DataStreams/SizeLimits.h>
|
||||
#include <DataStreams/ExecutionSpeedLimits.h>
|
||||
#include <IO/Progress.h>
|
||||
#include <Storages/TableStructureLockHolder.h>
|
||||
#include <Storages/TableLockHolder.h>
|
||||
#include <Common/TypePromotion.h>
|
||||
|
||||
#include <atomic>
|
||||
@ -109,7 +109,7 @@ public:
|
||||
size_t checkDepth(size_t max_depth) const { return checkDepthImpl(max_depth, max_depth); }
|
||||
|
||||
/// Do not allow to change the table while the blocks stream and its children are alive.
|
||||
void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); }
|
||||
void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); }
|
||||
|
||||
/// Get information about execution speed.
|
||||
const BlockStreamProfileInfo & getProfileInfo() const { return info; }
|
||||
@ -229,7 +229,7 @@ public:
|
||||
protected:
|
||||
/// Order is important: `table_locks` must be destroyed after `children` so that tables from
|
||||
/// which child streams read are protected by the locks during the lifetime of the child streams.
|
||||
std::vector<TableStructureReadLockHolder> table_locks;
|
||||
std::vector<TableLockHolder> table_locks;
|
||||
|
||||
BlockInputStreams children;
|
||||
std::shared_mutex children_mutex;
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <DataStreams/IBlockStream_fwd.h>
|
||||
#include <Storages/TableStructureLockHolder.h>
|
||||
#include <Storages/TableLockHolder.h>
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
@ -61,10 +61,10 @@ public:
|
||||
|
||||
/** Don't let to alter table while instance of stream is alive.
|
||||
*/
|
||||
void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); }
|
||||
void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); }
|
||||
|
||||
private:
|
||||
std::vector<TableStructureReadLockHolder> table_locks;
|
||||
std::vector<TableLockHolder> table_locks;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -21,7 +21,11 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery(
|
||||
const ASTPtr & ast, ReadBuffer * input_buffer_tail_part, const Block & header, const Context & context, const ASTPtr & input_function)
|
||||
const ASTPtr & ast,
|
||||
ReadBuffer * input_buffer_tail_part,
|
||||
const Block & header,
|
||||
const Context & context,
|
||||
const ASTPtr & input_function)
|
||||
{
|
||||
const auto * ast_insert_query = ast->as<ASTInsertQuery>();
|
||||
|
||||
@ -59,7 +63,8 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery(
|
||||
if (context.getSettingsRef().input_format_defaults_for_omitted_fields && ast_insert_query->table_id && !input_function)
|
||||
{
|
||||
StoragePtr storage = DatabaseCatalog::instance().getTable(ast_insert_query->table_id, context);
|
||||
auto column_defaults = storage->getColumns().getDefaults();
|
||||
auto metadata_snapshot = storage->getInMemoryMetadataPtr();
|
||||
auto column_defaults = metadata_snapshot->getColumns().getDefaults();
|
||||
if (!column_defaults.empty())
|
||||
res_stream = std::make_shared<AddingDefaultsBlockInputStream>(res_stream, column_defaults, context);
|
||||
}
|
||||
|
@ -11,6 +11,8 @@ namespace DB
|
||||
|
||||
struct BlockIO;
|
||||
class Context;
|
||||
struct StorageInMemoryMetadata;
|
||||
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
||||
|
||||
/** Prepares an input stream which produce data containing in INSERT query
|
||||
* Head of inserting data could be stored in INSERT ast directly
|
||||
@ -19,7 +21,8 @@ class Context;
|
||||
class InputStreamFromASTInsertQuery : public IBlockInputStream
|
||||
{
|
||||
public:
|
||||
InputStreamFromASTInsertQuery(const ASTPtr & ast,
|
||||
InputStreamFromASTInsertQuery(
|
||||
const ASTPtr & ast,
|
||||
ReadBuffer * input_buffer_tail_part,
|
||||
const Block & header,
|
||||
const Context & context,
|
||||
|
@ -19,15 +19,21 @@ namespace DB
|
||||
|
||||
PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
const StoragePtr & storage_,
|
||||
const Context & context_, const ASTPtr & query_ptr_, bool no_destination)
|
||||
: storage(storage_), context(context_), query_ptr(query_ptr_)
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const Context & context_,
|
||||
const ASTPtr & query_ptr_,
|
||||
bool no_destination)
|
||||
: storage(storage_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, context(context_)
|
||||
, query_ptr(query_ptr_)
|
||||
{
|
||||
/** TODO This is a very important line. At any insertion into the table one of streams should own lock.
|
||||
* Although now any insertion into the table is done via PushingToViewsBlockOutputStream,
|
||||
* but it's clear that here is not the best place for this functionality.
|
||||
*/
|
||||
addTableLock(
|
||||
storage->lockStructureForShare(true, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout));
|
||||
storage->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout));
|
||||
|
||||
/// If the "root" table deduplactes blocks, there are no need to make deduplication for children
|
||||
/// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks
|
||||
@ -60,6 +66,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
for (const auto & database_table : dependencies)
|
||||
{
|
||||
auto dependent_table = DatabaseCatalog::instance().getTable(database_table, context);
|
||||
auto dependent_metadata_snapshot = dependent_table->getInMemoryMetadataPtr();
|
||||
|
||||
ASTPtr query;
|
||||
BlockOutputStreamPtr out;
|
||||
@ -67,12 +74,12 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
if (auto * materialized_view = dynamic_cast<StorageMaterializedView *>(dependent_table.get()))
|
||||
{
|
||||
addTableLock(
|
||||
materialized_view->lockStructureForShare(
|
||||
true, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout));
|
||||
materialized_view->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout));
|
||||
|
||||
StoragePtr inner_table = materialized_view->getTargetTable();
|
||||
auto inner_table_id = inner_table->getStorageID();
|
||||
query = materialized_view->getSelectQuery().inner_query;
|
||||
auto inner_metadata_snapshot = inner_table->getInMemoryMetadataPtr();
|
||||
query = dependent_metadata_snapshot->getSelectQuery().inner_query;
|
||||
|
||||
std::unique_ptr<ASTInsertQuery> insert = std::make_unique<ASTInsertQuery>();
|
||||
insert->table_id = inner_table_id;
|
||||
@ -83,7 +90,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
|
||||
/// Insert only columns returned by select.
|
||||
auto list = std::make_shared<ASTExpressionList>();
|
||||
const auto & inner_table_columns = inner_table->getColumns();
|
||||
const auto & inner_table_columns = inner_metadata_snapshot->getColumns();
|
||||
for (auto & column : header)
|
||||
/// But skip columns which storage doesn't have.
|
||||
if (inner_table_columns.hasPhysical(column.name))
|
||||
@ -97,9 +104,11 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
out = io.out;
|
||||
}
|
||||
else if (dynamic_cast<const StorageLiveView *>(dependent_table.get()))
|
||||
out = std::make_shared<PushingToViewsBlockOutputStream>(dependent_table, *insert_context, ASTPtr(), true);
|
||||
out = std::make_shared<PushingToViewsBlockOutputStream>(
|
||||
dependent_table, dependent_metadata_snapshot, *insert_context, ASTPtr(), true);
|
||||
else
|
||||
out = std::make_shared<PushingToViewsBlockOutputStream>(dependent_table, *insert_context, ASTPtr());
|
||||
out = std::make_shared<PushingToViewsBlockOutputStream>(
|
||||
dependent_table, dependent_metadata_snapshot, *insert_context, ASTPtr());
|
||||
|
||||
views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out), nullptr});
|
||||
}
|
||||
@ -107,7 +116,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
/// Do not push to destination table if the flag is set
|
||||
if (!no_destination)
|
||||
{
|
||||
output = storage->write(query_ptr, context);
|
||||
output = storage->write(query_ptr, storage->getInMemoryMetadataPtr(), context);
|
||||
replicated_output = dynamic_cast<ReplicatedMergeTreeBlockOutputStream *>(output.get());
|
||||
}
|
||||
}
|
||||
@ -118,9 +127,9 @@ Block PushingToViewsBlockOutputStream::getHeader() const
|
||||
/// If we don't write directly to the destination
|
||||
/// then expect that we're inserting with precalculated virtual columns
|
||||
if (output)
|
||||
return storage->getSampleBlock();
|
||||
return metadata_snapshot->getSampleBlock();
|
||||
else
|
||||
return storage->getSampleBlockWithVirtuals();
|
||||
return metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals());
|
||||
}
|
||||
|
||||
|
||||
@ -314,7 +323,7 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n
|
||||
Context local_context = *select_context;
|
||||
local_context.addViewSource(
|
||||
StorageValues::create(
|
||||
storage->getStorageID(), storage->getColumns(), block, storage->getVirtuals()));
|
||||
storage->getStorageID(), metadata_snapshot->getColumns(), block, storage->getVirtuals()));
|
||||
select.emplace(view.query, local_context, SelectQueryOptions());
|
||||
in = std::make_shared<MaterializingBlockInputStream>(select->execute().getInputStream());
|
||||
|
||||
|
@ -17,8 +17,12 @@ class ReplicatedMergeTreeBlockOutputStream;
|
||||
class PushingToViewsBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
PushingToViewsBlockOutputStream(const StoragePtr & storage_,
|
||||
const Context & context_, const ASTPtr & query_ptr_, bool no_destination = false);
|
||||
PushingToViewsBlockOutputStream(
|
||||
const StoragePtr & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const Context & context_,
|
||||
const ASTPtr & query_ptr_,
|
||||
bool no_destination = false);
|
||||
|
||||
Block getHeader() const override;
|
||||
void write(const Block & block) override;
|
||||
@ -29,6 +33,7 @@ public:
|
||||
|
||||
private:
|
||||
StoragePtr storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
BlockOutputStreamPtr output;
|
||||
ReplicatedMergeTreeBlockOutputStream * replicated_output = nullptr;
|
||||
|
||||
|
@ -319,18 +319,22 @@ void RemoteQueryExecutor::sendExternalTables()
|
||||
for (const auto & table : external_tables)
|
||||
{
|
||||
StoragePtr cur = table.second;
|
||||
auto metadata_snapshot = cur->getInMemoryMetadataPtr();
|
||||
QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage(context);
|
||||
|
||||
Pipes pipes;
|
||||
|
||||
pipes = cur->read(cur->getColumns().getNamesOfPhysical(), {}, context,
|
||||
read_from_table_stage, DEFAULT_BLOCK_SIZE, 1);
|
||||
pipes = cur->read(
|
||||
metadata_snapshot->getColumns().getNamesOfPhysical(),
|
||||
metadata_snapshot, {}, context,
|
||||
read_from_table_stage, DEFAULT_BLOCK_SIZE, 1);
|
||||
|
||||
auto data = std::make_unique<ExternalTableData>();
|
||||
data->table_name = table.first;
|
||||
|
||||
if (pipes.empty())
|
||||
data->pipe = std::make_unique<Pipe>(std::make_shared<SourceFromSingleChunk>(cur->getSampleBlock(), Chunk()));
|
||||
data->pipe = std::make_unique<Pipe>(
|
||||
std::make_shared<SourceFromSingleChunk>(metadata_snapshot->getSampleBlock(), Chunk()));
|
||||
else if (pipes.size() == 1)
|
||||
data->pipe = std::make_unique<Pipe>(std::move(pipes.front()));
|
||||
else
|
||||
|
@ -20,10 +20,12 @@ namespace ErrorCodes
|
||||
TTLBlockInputStream::TTLBlockInputStream(
|
||||
const BlockInputStreamPtr & input_,
|
||||
const MergeTreeData & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const MergeTreeData::MutableDataPartPtr & data_part_,
|
||||
time_t current_time_,
|
||||
bool force_)
|
||||
: storage(storage_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, data_part(data_part_)
|
||||
, current_time(current_time_)
|
||||
, force(force_)
|
||||
@ -34,11 +36,11 @@ TTLBlockInputStream::TTLBlockInputStream(
|
||||
children.push_back(input_);
|
||||
header = children.at(0)->getHeader();
|
||||
|
||||
const auto & storage_columns = storage.getColumns();
|
||||
const auto & storage_columns = metadata_snapshot->getColumns();
|
||||
const auto & column_defaults = storage_columns.getDefaults();
|
||||
|
||||
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
|
||||
for (const auto & [name, _] : storage.getColumnTTLs())
|
||||
for (const auto & [name, _] : metadata_snapshot->getColumnTTLs())
|
||||
{
|
||||
auto it = column_defaults.find(name);
|
||||
if (it != column_defaults.end())
|
||||
@ -65,13 +67,12 @@ TTLBlockInputStream::TTLBlockInputStream(
|
||||
|
||||
if (!default_expr_list->children.empty())
|
||||
{
|
||||
auto syntax_result = SyntaxAnalyzer(storage.global_context).analyze(
|
||||
default_expr_list, storage.getColumns().getAllPhysical());
|
||||
auto syntax_result = SyntaxAnalyzer(storage.global_context).analyze(default_expr_list, metadata_snapshot->getColumns().getAllPhysical());
|
||||
defaults_expression = ExpressionAnalyzer{default_expr_list, syntax_result, storage.global_context}.getActions(true);
|
||||
}
|
||||
|
||||
auto storage_rows_ttl = storage.getRowsTTL();
|
||||
if (storage.hasRowsTTL() && storage_rows_ttl.mode == TTLMode::GROUP_BY)
|
||||
auto storage_rows_ttl = metadata_snapshot->getRowsTTL();
|
||||
if (metadata_snapshot->hasRowsTTL() && storage_rows_ttl.mode == TTLMode::GROUP_BY)
|
||||
{
|
||||
current_key_value.resize(storage_rows_ttl.group_by_keys.size());
|
||||
|
||||
@ -106,14 +107,15 @@ bool TTLBlockInputStream::isTTLExpired(time_t ttl) const
|
||||
Block TTLBlockInputStream::readImpl()
|
||||
{
|
||||
/// Skip all data if table ttl is expired for part
|
||||
auto storage_rows_ttl = storage.getRowsTTL();
|
||||
if (storage.hasRowsTTL() && !storage_rows_ttl.where_expression &&
|
||||
storage_rows_ttl.mode != TTLMode::GROUP_BY && isTTLExpired(old_ttl_infos.table_ttl.max))
|
||||
auto storage_rows_ttl = metadata_snapshot->getRowsTTL();
|
||||
if (metadata_snapshot->hasRowsTTL() && !storage_rows_ttl.where_expression && storage_rows_ttl.mode != TTLMode::GROUP_BY
|
||||
&& isTTLExpired(old_ttl_infos.table_ttl.max))
|
||||
{
|
||||
rows_removed = data_part->rows_count;
|
||||
return {};
|
||||
}
|
||||
|
||||
|
||||
Block block = children.at(0)->read();
|
||||
if (!block)
|
||||
{
|
||||
@ -127,7 +129,7 @@ Block TTLBlockInputStream::readImpl()
|
||||
return block;
|
||||
}
|
||||
|
||||
if (storage.hasRowsTTL() && (force || isTTLExpired(old_ttl_infos.table_ttl.min)))
|
||||
if (metadata_snapshot->hasRowsTTL() && (force || isTTLExpired(old_ttl_infos.table_ttl.min)))
|
||||
removeRowsWithExpiredTableTTL(block);
|
||||
|
||||
removeValuesWithExpiredColumnTTL(block);
|
||||
@ -153,7 +155,7 @@ void TTLBlockInputStream::readSuffixImpl()
|
||||
|
||||
void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block)
|
||||
{
|
||||
auto rows_ttl = storage.getRowsTTL();
|
||||
auto rows_ttl = metadata_snapshot->getRowsTTL();
|
||||
|
||||
rows_ttl.expression->execute(block);
|
||||
if (rows_ttl.where_expression)
|
||||
@ -201,7 +203,7 @@ void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block)
|
||||
size_t rows_aggregated = 0;
|
||||
size_t current_key_start = 0;
|
||||
size_t rows_with_current_key = 0;
|
||||
auto storage_rows_ttl = storage.getRowsTTL();
|
||||
auto storage_rows_ttl = metadata_snapshot->getRowsTTL();
|
||||
for (size_t i = 0; i < block.rows(); ++i)
|
||||
{
|
||||
UInt32 cur_ttl = getTimestampByIndex(ttl_column, i);
|
||||
@ -278,7 +280,7 @@ void TTLBlockInputStream::finalizeAggregates(MutableColumns & result_columns)
|
||||
if (!agg_result.empty())
|
||||
{
|
||||
auto aggregated_res = aggregator->convertToBlocks(agg_result, true, 1);
|
||||
auto storage_rows_ttl = storage.getRowsTTL();
|
||||
auto storage_rows_ttl = metadata_snapshot->getRowsTTL();
|
||||
for (auto & agg_block : aggregated_res)
|
||||
{
|
||||
for (const auto & it : storage_rows_ttl.set_parts)
|
||||
@ -310,7 +312,7 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block)
|
||||
}
|
||||
|
||||
std::vector<String> columns_to_remove;
|
||||
for (const auto & [name, ttl_entry] : storage.getColumnTTLs())
|
||||
for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs())
|
||||
{
|
||||
/// If we read not all table columns. E.g. while mutation.
|
||||
if (!block.has(name))
|
||||
@ -371,7 +373,7 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block)
|
||||
void TTLBlockInputStream::updateMovesTTL(Block & block)
|
||||
{
|
||||
std::vector<String> columns_to_remove;
|
||||
for (const auto & ttl_entry : storage.getMoveTTLs())
|
||||
for (const auto & ttl_entry : metadata_snapshot->getMoveTTLs())
|
||||
{
|
||||
auto & new_ttl_info = new_ttl_infos.moves_ttl[ttl_entry.result_column];
|
||||
|
||||
|
@ -16,6 +16,7 @@ public:
|
||||
TTLBlockInputStream(
|
||||
const BlockInputStreamPtr & input_,
|
||||
const MergeTreeData & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const MergeTreeData::MutableDataPartPtr & data_part_,
|
||||
time_t current_time,
|
||||
bool force_
|
||||
@ -33,6 +34,7 @@ protected:
|
||||
|
||||
private:
|
||||
const MergeTreeData & storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
|
||||
/// ttl_infos and empty_columns are updating while reading
|
||||
const MergeTreeData::MutableDataPartPtr & data_part;
|
||||
|
@ -139,7 +139,8 @@ static ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr
|
||||
create_table_query->table = table_id.table_name;
|
||||
create_table_query->database = table_id.database_name;
|
||||
|
||||
for (const auto & column_type_and_name : storage->getColumns().getOrdinary())
|
||||
auto metadata_snapshot = storage->getInMemoryMetadataPtr();
|
||||
for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary())
|
||||
{
|
||||
const auto & column_declaration = std::make_shared<ASTColumnDeclaration>();
|
||||
column_declaration->name = column_type_and_name.name;
|
||||
@ -361,7 +362,7 @@ void DatabaseMySQL::cleanOutdatedTables()
|
||||
++iterator;
|
||||
else
|
||||
{
|
||||
const auto table_lock = (*iterator)->lockAlterIntention(RWLockImpl::NO_QUERY, lock_acquire_timeout);
|
||||
const auto table_lock = (*iterator)->lockExclusively(RWLockImpl::NO_QUERY, lock_acquire_timeout);
|
||||
|
||||
(*iterator)->shutdown();
|
||||
(*iterator)->is_dropped = true;
|
||||
|
@ -266,7 +266,7 @@ void DatabaseOnDisk::renameTable(
|
||||
}
|
||||
|
||||
auto table_data_relative_path = getTableDataPath(table_name);
|
||||
TableStructureWriteLockHolder table_lock;
|
||||
TableExclusiveLockHolder table_lock;
|
||||
String table_metadata_path;
|
||||
ASTPtr attach_query;
|
||||
/// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case.
|
||||
|
@ -22,7 +22,6 @@ class Context;
|
||||
struct Settings;
|
||||
struct ConstraintsDescription;
|
||||
struct IndicesDescription;
|
||||
struct TableStructureWriteLockHolder;
|
||||
class ASTCreateQuery;
|
||||
using Dictionaries = std::vector<String>;
|
||||
|
||||
@ -237,7 +236,7 @@ public:
|
||||
using ASTModifier = std::function<void(IAST &)>;
|
||||
|
||||
/// Change the table structure in metadata.
|
||||
/// You must call under the TableStructureLock of the corresponding table . If engine_modifier is empty, then engine does not change.
|
||||
/// You must call under the alter_lock of the corresponding table . If engine_modifier is empty, then engine does not change.
|
||||
virtual void alterTable(
|
||||
const Context & /*context*/,
|
||||
const StorageID & /*table_id*/,
|
||||
|
@ -67,8 +67,7 @@ FunctionBaseImplPtr JoinGetOverloadResolver<or_null>::build(const ColumnsWithTyp
|
||||
auto join = storage_join->getJoin();
|
||||
DataTypes data_types(arguments.size());
|
||||
|
||||
auto table_lock = storage_join->lockStructureForShare(
|
||||
false, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
auto table_lock = storage_join->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
data_types[i] = arguments[i].type;
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Storages/TableStructureLockHolder.h>
|
||||
#include <Storages/TableLockHolder.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -37,7 +37,7 @@ class FunctionJoinGet final : public IFunctionBaseImpl
|
||||
public:
|
||||
static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet";
|
||||
|
||||
FunctionJoinGet(TableStructureReadLockHolder table_lock_, StoragePtr storage_join_,
|
||||
FunctionJoinGet(TableLockHolder table_lock_, StoragePtr storage_join_,
|
||||
HashJoinPtr join_, String attr_name_,
|
||||
DataTypes argument_types_, DataTypePtr return_type_)
|
||||
: table_lock(std::move(table_lock_))
|
||||
@ -57,7 +57,7 @@ public:
|
||||
ExecutableFunctionImplPtr prepare(const Block & sample_block, const ColumnNumbers & arguments, size_t result) const override;
|
||||
|
||||
private:
|
||||
TableStructureReadLockHolder table_lock;
|
||||
TableLockHolder table_lock;
|
||||
StoragePtr storage_join;
|
||||
HashJoinPtr join;
|
||||
const String attr_name;
|
||||
|
@ -114,7 +114,8 @@ void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers &
|
||||
if (host_name.empty())
|
||||
{
|
||||
const StoragePtr & table = DatabaseCatalog::instance().getTable({database_name, table_name}, global_context);
|
||||
has_column = table->getColumns().hasPhysical(column_name);
|
||||
auto table_metadata = table->getInMemoryMetadataPtr();
|
||||
has_column = table_metadata->getColumns().hasPhysical(column_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -363,7 +363,7 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
|
||||
const IAST & args = *func->arguments;
|
||||
const ASTPtr & left_in_operand = args.children.at(0);
|
||||
|
||||
if (storage()->mayBenefitFromIndexForIn(left_in_operand, context))
|
||||
if (storage()->mayBenefitFromIndexForIn(left_in_operand, context, metadata_snapshot))
|
||||
{
|
||||
const ASTPtr & arg = args.children.at(1);
|
||||
if (arg->as<ASTSubquery>() || arg->as<ASTIdentifier>())
|
||||
@ -1018,6 +1018,7 @@ ExpressionActionsPtr SelectQueryExpressionAnalyzer::simpleSelectActions()
|
||||
|
||||
ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||
SelectQueryExpressionAnalyzer & query_analyzer,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
bool first_stage_,
|
||||
bool second_stage_,
|
||||
bool only_types,
|
||||
@ -1068,14 +1069,14 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
|
||||
|
||||
if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1))
|
||||
{
|
||||
Names columns_for_sampling = storage->getColumnsRequiredForSampling();
|
||||
Names columns_for_sampling = metadata_snapshot->getColumnsRequiredForSampling();
|
||||
additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(),
|
||||
columns_for_sampling.begin(), columns_for_sampling.end());
|
||||
}
|
||||
|
||||
if (storage && query.final())
|
||||
{
|
||||
Names columns_for_final = storage->getColumnsRequiredForFinal();
|
||||
Names columns_for_final = metadata_snapshot->getColumnsRequiredForFinal();
|
||||
additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(),
|
||||
columns_for_final.begin(), columns_for_final.end());
|
||||
}
|
||||
|
@ -11,7 +11,6 @@
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -32,6 +31,9 @@ class ASTExpressionList;
|
||||
class ASTSelectQuery;
|
||||
struct ASTTablesInSelectQueryElement;
|
||||
|
||||
struct StorageInMemoryMetadata;
|
||||
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
||||
|
||||
/// Create columns in block or return false if not possible
|
||||
bool sanitizeBlock(Block & block, bool throw_if_cannot_create_column = false);
|
||||
|
||||
@ -202,6 +204,7 @@ struct ExpressionAnalysisResult
|
||||
|
||||
ExpressionAnalysisResult(
|
||||
SelectQueryExpressionAnalyzer & query_analyzer,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
bool first_stage,
|
||||
bool second_stage,
|
||||
bool only_types,
|
||||
@ -232,11 +235,14 @@ public:
|
||||
const ASTPtr & query_,
|
||||
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_,
|
||||
const Context & context_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const NameSet & required_result_columns_ = {},
|
||||
bool do_global_ = false,
|
||||
const SelectQueryOptions & options_ = {})
|
||||
: ExpressionAnalyzer(query_, syntax_analyzer_result_, context_, options_.subquery_depth, do_global_)
|
||||
, required_result_columns(required_result_columns_), query_options(options_)
|
||||
: ExpressionAnalyzer(query_, syntax_analyzer_result_, context_, options_.subquery_depth, do_global_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, required_result_columns(required_result_columns_)
|
||||
, query_options(options_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -260,6 +266,7 @@ public:
|
||||
void appendProjectResult(ExpressionActionsChain & chain) const;
|
||||
|
||||
private:
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
/// If non-empty, ignore all expressions not from this list.
|
||||
NameSet required_result_columns;
|
||||
SelectQueryOptions query_options;
|
||||
|
@ -43,6 +43,8 @@ BlockIO InterpreterAlterQuery::execute()
|
||||
context.checkAccess(getRequiredAccess());
|
||||
auto table_id = context.resolveStorageID(alter, Context::ResolveOrdinary);
|
||||
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context);
|
||||
auto alter_lock = table->lockForAlter(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
|
||||
/// Add default database to table identifiers that we can encounter in e.g. default expressions,
|
||||
/// mutation expression, etc.
|
||||
@ -68,7 +70,7 @@ BlockIO InterpreterAlterQuery::execute()
|
||||
}
|
||||
else if (auto mut_command = MutationCommand::parse(command_ast))
|
||||
{
|
||||
if (mut_command->type == MutationCommand::MATERIALIZE_TTL && !table->hasAnyTTL())
|
||||
if (mut_command->type == MutationCommand::MATERIALIZE_TTL && !metadata_snapshot->hasAnyTTL())
|
||||
throw Exception("Cannot MATERIALIZE TTL as there is no TTL set for table "
|
||||
+ table->getStorageID().getNameForLogs(), ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
@ -82,16 +84,13 @@ BlockIO InterpreterAlterQuery::execute()
|
||||
|
||||
if (!mutation_commands.empty())
|
||||
{
|
||||
auto table_lock_holder = table->lockStructureForShare(
|
||||
false /* because mutation is executed asyncronously */,
|
||||
context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
MutationsInterpreter(table, mutation_commands, context, false).validate(table_lock_holder);
|
||||
MutationsInterpreter(table, metadata_snapshot, mutation_commands, context, false).validate();
|
||||
table->mutate(mutation_commands, context);
|
||||
}
|
||||
|
||||
if (!partition_commands.empty())
|
||||
{
|
||||
table->alterPartition(query_ptr, partition_commands, context);
|
||||
table->alterPartition(query_ptr, metadata_snapshot, partition_commands, context);
|
||||
}
|
||||
|
||||
if (!live_view_commands.empty())
|
||||
@ -111,13 +110,11 @@ BlockIO InterpreterAlterQuery::execute()
|
||||
|
||||
if (!alter_commands.empty())
|
||||
{
|
||||
auto table_lock_holder = table->lockAlterIntention(
|
||||
context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
StorageInMemoryMetadata metadata = table->getInMemoryMetadata();
|
||||
alter_commands.validate(metadata, context);
|
||||
alter_commands.prepare(metadata);
|
||||
table->checkAlterIsPossible(alter_commands, context.getSettingsRef());
|
||||
table->alter(alter_commands, context, table_lock_holder);
|
||||
table->alter(alter_commands, context, alter_lock);
|
||||
}
|
||||
|
||||
return {};
|
||||
|
@ -405,7 +405,7 @@ ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(const A
|
||||
InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(ASTCreateQuery & create) const
|
||||
{
|
||||
TableProperties properties;
|
||||
TableStructureReadLockHolder as_storage_lock;
|
||||
TableLockHolder as_storage_lock;
|
||||
|
||||
if (create.columns_list)
|
||||
{
|
||||
@ -428,16 +428,16 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS
|
||||
StoragePtr as_storage = DatabaseCatalog::instance().getTable({as_database_name, create.as_table}, context);
|
||||
|
||||
/// as_storage->getColumns() and setEngine(...) must be called under structure lock of other_table for CREATE ... AS other_table.
|
||||
as_storage_lock = as_storage->lockStructureForShare(
|
||||
false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
properties.columns = as_storage->getColumns();
|
||||
as_storage_lock = as_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
auto as_storage_metadata = as_storage->getInMemoryMetadataPtr();
|
||||
properties.columns = as_storage_metadata->getColumns();
|
||||
|
||||
/// Secondary indices make sense only for MergeTree family of storage engines.
|
||||
/// We should not copy them for other storages.
|
||||
if (create.storage && endsWith(create.storage->engine->name, "MergeTree"))
|
||||
properties.indices = as_storage->getSecondaryIndices();
|
||||
properties.indices = as_storage_metadata->getSecondaryIndices();
|
||||
|
||||
properties.constraints = as_storage->getConstraints();
|
||||
properties.constraints = as_storage_metadata->getConstraints();
|
||||
}
|
||||
else if (create.select)
|
||||
{
|
||||
|
@ -89,9 +89,9 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
|
||||
table = DatabaseCatalog::instance().getTable(table_id, context);
|
||||
}
|
||||
|
||||
auto table_lock = table->lockStructureForShare(
|
||||
false, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
columns = table->getColumns();
|
||||
auto table_lock = table->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
columns = metadata_snapshot->getColumns();
|
||||
}
|
||||
|
||||
Block sample_block = getSampleBlock();
|
||||
|
@ -93,7 +93,7 @@ BlockIO InterpreterDropQuery::executeToTable(
|
||||
{
|
||||
context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id);
|
||||
table->shutdown();
|
||||
TableStructureWriteLockHolder table_lock;
|
||||
TableExclusiveLockHolder table_lock;
|
||||
if (database->getEngineName() != "Atomic")
|
||||
table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
/// Drop table from memory, don't touch data and metadata
|
||||
@ -105,8 +105,9 @@ BlockIO InterpreterDropQuery::executeToTable(
|
||||
table->checkTableCanBeDropped();
|
||||
|
||||
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
/// Drop table data, don't touch metadata
|
||||
table->truncate(query_ptr, context, table_lock);
|
||||
table->truncate(query_ptr, metadata_snapshot, context, table_lock);
|
||||
}
|
||||
else if (query.kind == ASTDropQuery::Kind::Drop)
|
||||
{
|
||||
@ -115,7 +116,7 @@ BlockIO InterpreterDropQuery::executeToTable(
|
||||
|
||||
table->shutdown();
|
||||
|
||||
TableStructureWriteLockHolder table_lock;
|
||||
TableExclusiveLockHolder table_lock;
|
||||
if (database->getEngineName() != "Atomic")
|
||||
table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
|
||||
@ -187,7 +188,8 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(const String & table_name,
|
||||
{
|
||||
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
/// Drop table data, don't touch metadata
|
||||
table->truncate(query_ptr, context, table_lock);
|
||||
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
table->truncate(query_ptr, metadata_snapshot, context, table_lock);
|
||||
}
|
||||
else if (kind == ASTDropQuery::Kind::Drop)
|
||||
{
|
||||
|
@ -68,19 +68,22 @@ StoragePtr InterpreterInsertQuery::getTable(ASTInsertQuery & query)
|
||||
return DatabaseCatalog::instance().getTable(query.table_id, context);
|
||||
}
|
||||
|
||||
Block InterpreterInsertQuery::getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table) const
|
||||
Block InterpreterInsertQuery::getSampleBlock(
|
||||
const ASTInsertQuery & query,
|
||||
const StoragePtr & table,
|
||||
const StorageMetadataPtr & metadata_snapshot) const
|
||||
{
|
||||
Block table_sample_non_materialized = table->getSampleBlockNonMaterialized();
|
||||
Block table_sample_non_materialized = metadata_snapshot->getSampleBlockNonMaterialized();
|
||||
/// If the query does not include information about columns
|
||||
if (!query.columns)
|
||||
{
|
||||
if (no_destination)
|
||||
return table->getSampleBlockWithVirtuals();
|
||||
return metadata_snapshot->getSampleBlockWithVirtuals(table->getVirtuals());
|
||||
else
|
||||
return table_sample_non_materialized;
|
||||
}
|
||||
|
||||
Block table_sample = table->getSampleBlock();
|
||||
Block table_sample = metadata_snapshot->getSampleBlock();
|
||||
/// Form the block based on the column names from the query
|
||||
Block res;
|
||||
for (const auto & identifier : query.columns->children)
|
||||
@ -110,10 +113,10 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
BlockIO res;
|
||||
|
||||
StoragePtr table = getTable(query);
|
||||
auto table_lock = table->lockStructureForShare(
|
||||
true, context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
auto table_lock = table->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
|
||||
auto query_sample_block = getSampleBlock(query, table);
|
||||
auto query_sample_block = getSampleBlock(query, table, metadata_snapshot);
|
||||
if (!query.table_function)
|
||||
context.checkAccess(AccessType::INSERT, query.table_id, query_sample_block.getNames());
|
||||
|
||||
@ -221,21 +224,21 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
/// NOTE: we explicitly ignore bound materialized views when inserting into Kafka Storage.
|
||||
/// Otherwise we'll get duplicates when MV reads same rows again from Kafka.
|
||||
if (table->noPushingToViews() && !no_destination)
|
||||
out = table->write(query_ptr, context);
|
||||
out = table->write(query_ptr, metadata_snapshot, context);
|
||||
else
|
||||
out = std::make_shared<PushingToViewsBlockOutputStream>(table, context, query_ptr, no_destination);
|
||||
out = std::make_shared<PushingToViewsBlockOutputStream>(table, metadata_snapshot, context, query_ptr, no_destination);
|
||||
|
||||
/// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order.
|
||||
|
||||
/// Checking constraints. It must be done after calculation of all defaults, so we can check them on calculated columns.
|
||||
if (const auto & constraints = table->getConstraints(); !constraints.empty())
|
||||
if (const auto & constraints = metadata_snapshot->getConstraints(); !constraints.empty())
|
||||
out = std::make_shared<CheckConstraintsBlockOutputStream>(
|
||||
query.table_id, out, out->getHeader(), table->getConstraints(), context);
|
||||
query.table_id, out, out->getHeader(), metadata_snapshot->getConstraints(), context);
|
||||
|
||||
/// Actually we don't know structure of input blocks from query/table,
|
||||
/// because some clients break insertion protocol (columns != header)
|
||||
out = std::make_shared<AddingDefaultBlockOutputStream>(
|
||||
out, query_sample_block, out->getHeader(), table->getColumns().getDefaults(), context);
|
||||
out, query_sample_block, out->getHeader(), metadata_snapshot->getColumns().getDefaults(), context);
|
||||
|
||||
/// It's important to squash blocks as early as possible (before other transforms),
|
||||
/// because other transforms may work inefficient if block size is small.
|
||||
@ -286,7 +289,7 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
|
||||
if (!allow_materialized)
|
||||
{
|
||||
for (const auto & column : table->getColumns())
|
||||
for (const auto & column : metadata_snapshot->getColumns())
|
||||
if (column.default_desc.kind == ColumnDefaultKind::Materialized && header.has(column.name))
|
||||
throw Exception("Cannot insert column " + column.name + ", because it is MATERIALIZED column.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <DataStreams/BlockIO.h>
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -34,7 +35,7 @@ public:
|
||||
|
||||
private:
|
||||
StoragePtr getTable(ASTInsertQuery & query);
|
||||
Block getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table) const;
|
||||
Block getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const;
|
||||
|
||||
ASTPtr query_ptr;
|
||||
const Context & context;
|
||||
|
@ -26,7 +26,8 @@ BlockIO InterpreterOptimizeQuery::execute()
|
||||
|
||||
auto table_id = context.resolveStorageID(ast, Context::ResolveOrdinary);
|
||||
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context);
|
||||
table->optimize(query_ptr, ast.partition, ast.final, ast.deduplicate, context);
|
||||
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
table->optimize(query_ptr, metadata_snapshot, ast.partition, ast.final, ast.deduplicate, context);
|
||||
return {};
|
||||
}
|
||||
|
||||
|
@ -130,8 +130,8 @@ String InterpreterSelectQuery::generateFilterActions(
|
||||
table_expr->children.push_back(table_expr->database_and_table_name);
|
||||
|
||||
/// Using separate expression analyzer to prevent any possible alias injection
|
||||
auto syntax_result = SyntaxAnalyzer(*context).analyzeSelect(query_ast, SyntaxAnalyzerResult({}, storage));
|
||||
SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, *context);
|
||||
auto syntax_result = SyntaxAnalyzer(*context).analyzeSelect(query_ast, SyntaxAnalyzerResult({}, storage, metadata_snapshot));
|
||||
SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, *context, metadata_snapshot);
|
||||
actions = analyzer.simpleSelectActions();
|
||||
|
||||
return expr_list->children.at(0)->getColumnName();
|
||||
@ -166,8 +166,9 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
const ASTPtr & query_ptr_,
|
||||
const Context & context_,
|
||||
const StoragePtr & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const SelectQueryOptions & options_)
|
||||
: InterpreterSelectQuery(query_ptr_, context_, nullptr, std::nullopt, storage_, options_.copy().noSubquery())
|
||||
: InterpreterSelectQuery(query_ptr_, context_, nullptr, std::nullopt, storage_, options_.copy().noSubquery(), {}, metadata_snapshot_)
|
||||
{}
|
||||
|
||||
InterpreterSelectQuery::~InterpreterSelectQuery() = default;
|
||||
@ -215,7 +216,8 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
std::optional<Pipe> input_pipe_,
|
||||
const StoragePtr & storage_,
|
||||
const SelectQueryOptions & options_,
|
||||
const Names & required_result_column_names)
|
||||
const Names & required_result_column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot_)
|
||||
: options(options_)
|
||||
/// NOTE: the query almost always should be cloned because it will be modified during analysis.
|
||||
, query_ptr(options.modify_inplace ? query_ptr_ : query_ptr_->clone())
|
||||
@ -224,6 +226,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
, input(input_)
|
||||
, input_pipe(std::move(input_pipe_))
|
||||
, log(&Poco::Logger::get("InterpreterSelectQuery"))
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
{
|
||||
checkStackSize();
|
||||
|
||||
@ -253,13 +256,14 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
|
||||
if (storage)
|
||||
{
|
||||
table_lock = storage->lockStructureForShare(
|
||||
false, context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout);
|
||||
table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout);
|
||||
table_id = storage->getStorageID();
|
||||
if (metadata_snapshot == nullptr)
|
||||
metadata_snapshot = storage->getInMemoryMetadataPtr();
|
||||
}
|
||||
|
||||
if (has_input || !joined_tables.resolveTables())
|
||||
joined_tables.makeFakeTable(storage, source_header);
|
||||
joined_tables.makeFakeTable(storage, metadata_snapshot, source_header);
|
||||
|
||||
/// Rewrite JOINs
|
||||
if (!has_input && joined_tables.tablesCount() > 1)
|
||||
@ -273,7 +277,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
{
|
||||
/// Rewritten with subquery. Free storage locks here.
|
||||
storage = {};
|
||||
table_lock.release();
|
||||
table_lock.reset();
|
||||
table_id = StorageID::createEmpty();
|
||||
}
|
||||
}
|
||||
@ -304,11 +308,12 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
/// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it.
|
||||
ASTPtr view_table;
|
||||
if (view)
|
||||
view->replaceWithSubquery(getSelectQuery(), view_table);
|
||||
view->replaceWithSubquery(getSelectQuery(), view_table, metadata_snapshot);
|
||||
|
||||
syntax_analyzer_result = SyntaxAnalyzer(*context).analyzeSelect(
|
||||
query_ptr, SyntaxAnalyzerResult(source_header.getNamesAndTypesList(), storage),
|
||||
options, joined_tables.tablesWithColumns(), required_result_column_names, table_join);
|
||||
query_ptr,
|
||||
SyntaxAnalyzerResult(source_header.getNamesAndTypesList(), storage, metadata_snapshot),
|
||||
options, joined_tables.tablesWithColumns(), required_result_column_names, table_join);
|
||||
|
||||
/// Save scalar sub queries's results in the query context
|
||||
if (!options.only_analyze && context->hasQueryContext())
|
||||
@ -331,12 +336,12 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
current_info.query = query_ptr;
|
||||
current_info.syntax_analyzer_result = syntax_analyzer_result;
|
||||
|
||||
MergeTreeWhereOptimizer{current_info, *context, *merge_tree, syntax_analyzer_result->requiredSourceColumns(), log};
|
||||
MergeTreeWhereOptimizer{current_info, *context, *merge_tree, metadata_snapshot, syntax_analyzer_result->requiredSourceColumns(), log};
|
||||
}
|
||||
}
|
||||
|
||||
query_analyzer = std::make_unique<SelectQueryExpressionAnalyzer>(
|
||||
query_ptr, syntax_analyzer_result, *context,
|
||||
query_ptr, syntax_analyzer_result, *context, metadata_snapshot,
|
||||
NameSet(required_result_column_names.begin(), required_result_column_names.end()),
|
||||
!options.only_analyze, options);
|
||||
|
||||
@ -377,14 +382,15 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
|
||||
if (storage)
|
||||
{
|
||||
source_header = storage->getSampleBlockForColumns(required_columns);
|
||||
source_header = metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID());
|
||||
|
||||
/// Fix source_header for filter actions.
|
||||
if (row_policy_filter)
|
||||
{
|
||||
filter_info = std::make_shared<FilterInfo>();
|
||||
filter_info->column_name = generateFilterActions(filter_info->actions, row_policy_filter, required_columns);
|
||||
source_header = storage->getSampleBlockForColumns(filter_info->actions->getRequiredColumns());
|
||||
source_header = metadata_snapshot->getSampleBlockForColumns(
|
||||
filter_info->actions->getRequiredColumns(), storage->getVirtuals(), storage->getStorageID());
|
||||
}
|
||||
}
|
||||
|
||||
@ -497,6 +503,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
|
||||
|
||||
analysis_result = ExpressionAnalysisResult(
|
||||
*query_analyzer,
|
||||
metadata_snapshot,
|
||||
first_stage,
|
||||
second_stage,
|
||||
options.only_analyze,
|
||||
@ -1105,7 +1112,7 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
|
||||
/// Detect, if ALIAS columns are required for query execution
|
||||
auto alias_columns_required = false;
|
||||
const ColumnsDescription & storage_columns = storage->getColumns();
|
||||
const ColumnsDescription & storage_columns = metadata_snapshot->getColumns();
|
||||
for (const auto & column_name : required_columns)
|
||||
{
|
||||
auto column_default = storage_columns.getDefault(column_name);
|
||||
@ -1197,7 +1204,7 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
= ext::map<NameSet>(required_columns_after_prewhere, [](const auto & it) { return it.name; });
|
||||
}
|
||||
|
||||
auto syntax_result = SyntaxAnalyzer(*context).analyze(required_columns_all_expr, required_columns_after_prewhere, storage);
|
||||
auto syntax_result = SyntaxAnalyzer(*context).analyze(required_columns_all_expr, required_columns_after_prewhere, storage, metadata_snapshot);
|
||||
alias_actions = ExpressionAnalyzer(required_columns_all_expr, syntax_result, *context).getActions(true);
|
||||
|
||||
/// The set of required columns could be added as a result of adding an action to calculate ALIAS.
|
||||
@ -1228,7 +1235,7 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
prewhere_info->prewhere_actions = std::move(new_actions);
|
||||
|
||||
auto analyzed_result
|
||||
= SyntaxAnalyzer(*context).analyze(required_columns_from_prewhere_expr, storage->getColumns().getAllPhysical());
|
||||
= SyntaxAnalyzer(*context).analyze(required_columns_from_prewhere_expr, metadata_snapshot->getColumns().getAllPhysical());
|
||||
prewhere_info->alias_actions
|
||||
= ExpressionAnalyzer(required_columns_from_prewhere_expr, analyzed_result, *context).getActions(true, false);
|
||||
|
||||
@ -1332,7 +1339,6 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
else if (storage)
|
||||
{
|
||||
/// Table.
|
||||
|
||||
if (max_streams == 0)
|
||||
throw Exception("Logical error: zero number of streams requested", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
@ -1360,11 +1366,11 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
getSortDescriptionFromGroupBy(query),
|
||||
query_info.syntax_analyzer_result);
|
||||
|
||||
query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage);
|
||||
query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage, metadata_snapshot);
|
||||
}
|
||||
|
||||
auto read_step = std::make_unique<ReadFromStorageStep>(
|
||||
table_lock, options, storage,
|
||||
table_lock, metadata_snapshot, options, storage,
|
||||
required_columns, query_info, context, processing_stage, max_block_size, max_streams);
|
||||
|
||||
read_step->setStepDescription("Read from " + storage->getName());
|
||||
|
@ -10,7 +10,7 @@
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Interpreters/SelectQueryOptions.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/TableStructureLockHolder.h>
|
||||
#include <Storages/TableLockHolder.h>
|
||||
#include <Storages/ReadInOrderOptimizer.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
|
||||
@ -70,6 +70,7 @@ public:
|
||||
const ASTPtr & query_ptr_,
|
||||
const Context & context_,
|
||||
const StoragePtr & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_ = nullptr,
|
||||
const SelectQueryOptions & = {});
|
||||
|
||||
~InterpreterSelectQuery() override;
|
||||
@ -101,7 +102,8 @@ private:
|
||||
std::optional<Pipe> input_pipe,
|
||||
const StoragePtr & storage_,
|
||||
const SelectQueryOptions &,
|
||||
const Names & required_result_column_names = {});
|
||||
const Names & required_result_column_names = {},
|
||||
const StorageMetadataPtr & metadata_snapshot_= nullptr);
|
||||
|
||||
ASTSelectQuery & getSelectQuery() { return query_ptr->as<ASTSelectQuery &>(); }
|
||||
|
||||
@ -186,13 +188,14 @@ private:
|
||||
/// Table from where to read data, if not subquery.
|
||||
StoragePtr storage;
|
||||
StorageID table_id = StorageID::createEmpty(); /// Will be initialized if storage is not nullptr
|
||||
TableStructureReadLockHolder table_lock;
|
||||
TableLockHolder table_lock;
|
||||
|
||||
/// Used when we read from prepared input, not table or subquery.
|
||||
BlockInputStreamPtr input;
|
||||
std::optional<Pipe> input_pipe;
|
||||
|
||||
Poco::Logger * log;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -47,7 +47,7 @@ BlockIO InterpreterWatchQuery::execute()
|
||||
ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
/// List of columns to read to execute the query.
|
||||
Names required_columns = storage->getColumns().getNamesOfPhysical();
|
||||
Names required_columns = storage->getInMemoryMetadataPtr()->getColumns().getNamesOfPhysical();
|
||||
context.checkAccess(AccessType::SELECT, table_id, required_columns);
|
||||
|
||||
/// Get context settings for this query
|
||||
|
@ -207,11 +207,11 @@ bool JoinedTables::resolveTables()
|
||||
return !tables_with_columns.empty();
|
||||
}
|
||||
|
||||
void JoinedTables::makeFakeTable(StoragePtr storage, const Block & source_header)
|
||||
void JoinedTables::makeFakeTable(StoragePtr storage, const StorageMetadataPtr & metadata_snapshot, const Block & source_header)
|
||||
{
|
||||
if (storage)
|
||||
{
|
||||
const ColumnsDescription & storage_columns = storage->getColumns();
|
||||
const ColumnsDescription & storage_columns = metadata_snapshot->getColumns();
|
||||
tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, storage_columns.getOrdinary());
|
||||
|
||||
auto & table = tables_with_columns.back();
|
||||
|
@ -13,6 +13,8 @@ namespace DB
|
||||
class ASTSelectQuery;
|
||||
class TableJoin;
|
||||
struct SelectQueryOptions;
|
||||
struct StorageInMemoryMetadata;
|
||||
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
||||
|
||||
/// Joined tables' columns resolver.
|
||||
/// We want to get each table structure at most once per table occurance. Or even better once per table.
|
||||
@ -31,7 +33,7 @@ public:
|
||||
bool resolveTables();
|
||||
|
||||
/// Make fake tables_with_columns[0] in case we have predefined input in InterpreterSelectQuery
|
||||
void makeFakeTable(StoragePtr storage, const Block & source_header);
|
||||
void makeFakeTable(StoragePtr storage, const StorageMetadataPtr & metadata_snapshot, const Block & source_header);
|
||||
std::shared_ptr<TableJoin> makeTableJoin(const ASTSelectQuery & select_query);
|
||||
|
||||
const TablesWithColumns & tablesWithColumns() const { return tables_with_columns; }
|
||||
|
@ -137,13 +137,13 @@ ASTPtr prepareQueryAffectedAST(const std::vector<MutationCommand> & commands)
|
||||
return select;
|
||||
}
|
||||
|
||||
ColumnDependencies getAllColumnDependencies(const StoragePtr & storage, const NameSet & updated_columns)
|
||||
ColumnDependencies getAllColumnDependencies(const StorageMetadataPtr & metadata_snapshot, const NameSet & updated_columns)
|
||||
{
|
||||
NameSet new_updated_columns = updated_columns;
|
||||
ColumnDependencies dependencies;
|
||||
while (!new_updated_columns.empty())
|
||||
{
|
||||
auto new_dependencies = storage->getColumnDependencies(new_updated_columns);
|
||||
auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns);
|
||||
new_updated_columns.clear();
|
||||
for (const auto & dependency : new_dependencies)
|
||||
{
|
||||
@ -163,6 +163,7 @@ ColumnDependencies getAllColumnDependencies(const StoragePtr & storage, const Na
|
||||
|
||||
bool isStorageTouchedByMutations(
|
||||
StoragePtr storage,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const std::vector<MutationCommand> & commands,
|
||||
Context context_copy)
|
||||
{
|
||||
@ -183,7 +184,7 @@ bool isStorageTouchedByMutations(
|
||||
/// Interpreter must be alive, when we use result of execute() method.
|
||||
/// For some reason it may copy context and and give it into ExpressionBlockInputStream
|
||||
/// after that we will use context from destroyed stack frame in our stream.
|
||||
InterpreterSelectQuery interpreter(select_query, context_copy, storage, SelectQueryOptions().ignoreLimits());
|
||||
InterpreterSelectQuery interpreter(select_query, context_copy, storage, metadata_snapshot, SelectQueryOptions().ignoreLimits());
|
||||
BlockInputStreamPtr in = interpreter.execute().getInputStream();
|
||||
|
||||
Block block = in->read();
|
||||
@ -200,20 +201,22 @@ bool isStorageTouchedByMutations(
|
||||
|
||||
MutationsInterpreter::MutationsInterpreter(
|
||||
StoragePtr storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
MutationCommands commands_,
|
||||
const Context & context_,
|
||||
bool can_execute_)
|
||||
: storage(std::move(storage_))
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, commands(std::move(commands_))
|
||||
, context(context_)
|
||||
, can_execute(can_execute_)
|
||||
{
|
||||
mutation_ast = prepare(!can_execute);
|
||||
SelectQueryOptions limits = SelectQueryOptions().analyze(!can_execute).ignoreLimits();
|
||||
select_interpreter = std::make_unique<InterpreterSelectQuery>(mutation_ast, context, storage, limits);
|
||||
select_interpreter = std::make_unique<InterpreterSelectQuery>(mutation_ast, context, storage, metadata_snapshot_, limits);
|
||||
}
|
||||
|
||||
static NameSet getKeyColumns(const StoragePtr & storage)
|
||||
static NameSet getKeyColumns(const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot)
|
||||
{
|
||||
const MergeTreeData * merge_tree_data = dynamic_cast<const MergeTreeData *>(storage.get());
|
||||
if (!merge_tree_data)
|
||||
@ -221,10 +224,10 @@ static NameSet getKeyColumns(const StoragePtr & storage)
|
||||
|
||||
NameSet key_columns;
|
||||
|
||||
for (const String & col : merge_tree_data->getColumnsRequiredForPartitionKey())
|
||||
for (const String & col : metadata_snapshot->getColumnsRequiredForPartitionKey())
|
||||
key_columns.insert(col);
|
||||
|
||||
for (const String & col : merge_tree_data->getColumnsRequiredForSortingKey())
|
||||
for (const String & col : metadata_snapshot->getColumnsRequiredForSortingKey())
|
||||
key_columns.insert(col);
|
||||
/// We don't process sample_by_ast separately because it must be among the primary key columns.
|
||||
|
||||
@ -238,15 +241,16 @@ static NameSet getKeyColumns(const StoragePtr & storage)
|
||||
}
|
||||
|
||||
static void validateUpdateColumns(
|
||||
const StoragePtr & storage, const NameSet & updated_columns,
|
||||
const StoragePtr & storage,
|
||||
const StorageMetadataPtr & metadata_snapshot, const NameSet & updated_columns,
|
||||
const std::unordered_map<String, Names> & column_to_affected_materialized)
|
||||
{
|
||||
NameSet key_columns = getKeyColumns(storage);
|
||||
NameSet key_columns = getKeyColumns(storage, metadata_snapshot);
|
||||
|
||||
for (const String & column_name : updated_columns)
|
||||
{
|
||||
auto found = false;
|
||||
for (const auto & col : storage->getColumns().getOrdinary())
|
||||
for (const auto & col : metadata_snapshot->getColumns().getOrdinary())
|
||||
{
|
||||
if (col.name == column_name)
|
||||
{
|
||||
@ -257,7 +261,7 @@ static void validateUpdateColumns(
|
||||
|
||||
if (!found)
|
||||
{
|
||||
for (const auto & col : storage->getColumns().getMaterialized())
|
||||
for (const auto & col : metadata_snapshot->getColumns().getMaterialized())
|
||||
{
|
||||
if (col.name == column_name)
|
||||
throw Exception("Cannot UPDATE materialized column " + backQuote(column_name), ErrorCodes::CANNOT_UPDATE_COLUMN);
|
||||
@ -293,8 +297,8 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
throw Exception("Empty mutation commands list", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
|
||||
const ColumnsDescription & columns_desc = storage->getColumns();
|
||||
const IndicesDescription & indices_desc = storage->getSecondaryIndices();
|
||||
const ColumnsDescription & columns_desc = metadata_snapshot->getColumns();
|
||||
const IndicesDescription & indices_desc = metadata_snapshot->getSecondaryIndices();
|
||||
NamesAndTypesList all_columns = columns_desc.getAllPhysical();
|
||||
|
||||
NameSet updated_columns;
|
||||
@ -325,11 +329,11 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
}
|
||||
}
|
||||
|
||||
validateUpdateColumns(storage, updated_columns, column_to_affected_materialized);
|
||||
validateUpdateColumns(storage, metadata_snapshot, updated_columns, column_to_affected_materialized);
|
||||
}
|
||||
|
||||
/// Columns, that we need to read for calculation of skip indices or TTL expressions.
|
||||
auto dependencies = getAllColumnDependencies(storage, updated_columns);
|
||||
auto dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns);
|
||||
|
||||
/// First, break a sequence of commands into stages.
|
||||
for (const auto & command : commands)
|
||||
@ -407,7 +411,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
}
|
||||
else if (command.type == MutationCommand::MATERIALIZE_TTL)
|
||||
{
|
||||
if (storage->hasRowsTTL())
|
||||
if (metadata_snapshot->hasRowsTTL())
|
||||
{
|
||||
for (const auto & column : all_columns)
|
||||
dependencies.emplace(column.name, ColumnDependency::TTL_TARGET);
|
||||
@ -415,7 +419,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
else
|
||||
{
|
||||
NameSet new_updated_columns;
|
||||
auto column_ttls = storage->getColumns().getColumnTTLs();
|
||||
auto column_ttls = metadata_snapshot->getColumns().getColumnTTLs();
|
||||
for (const auto & elem : column_ttls)
|
||||
{
|
||||
dependencies.emplace(elem.first, ColumnDependency::TTL_TARGET);
|
||||
@ -423,7 +427,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
}
|
||||
|
||||
auto all_columns_vec = all_columns.getNames();
|
||||
auto all_dependencies = getAllColumnDependencies(storage, NameSet(all_columns_vec.begin(), all_columns_vec.end()));
|
||||
auto all_dependencies = getAllColumnDependencies(metadata_snapshot, NameSet(all_columns_vec.begin(), all_columns_vec.end()));
|
||||
|
||||
for (const auto & dependency : all_dependencies)
|
||||
{
|
||||
@ -432,7 +436,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
}
|
||||
|
||||
/// Recalc only skip indices of columns, that could be updated by TTL.
|
||||
auto new_dependencies = storage->getColumnDependencies(new_updated_columns);
|
||||
auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns);
|
||||
for (const auto & dependency : new_dependencies)
|
||||
{
|
||||
if (dependency.kind == ColumnDependency::SKIP_INDEX)
|
||||
@ -502,7 +506,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
}
|
||||
|
||||
const ASTPtr select_query = prepareInterpreterSelectQuery(stages_copy, /* dry_run = */ true);
|
||||
InterpreterSelectQuery interpreter{select_query, context, storage, SelectQueryOptions().analyze(/* dry_run = */ false).ignoreLimits()};
|
||||
InterpreterSelectQuery interpreter{select_query, context, storage, metadata_snapshot, SelectQueryOptions().analyze(/* dry_run = */ false).ignoreLimits()};
|
||||
|
||||
auto first_stage_header = interpreter.getSampleBlock();
|
||||
auto in = std::make_shared<NullBlockInputStream>(first_stage_header);
|
||||
@ -524,7 +528,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
|
||||
ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector<Stage> & prepared_stages, bool dry_run)
|
||||
{
|
||||
NamesAndTypesList all_columns = storage->getColumns().getAllPhysical();
|
||||
NamesAndTypesList all_columns = metadata_snapshot->getColumns().getAllPhysical();
|
||||
|
||||
|
||||
/// Next, for each stage calculate columns changed by this and previous stages.
|
||||
@ -667,7 +671,7 @@ BlockInputStreamPtr MutationsInterpreter::addStreamsForLaterStages(const std::ve
|
||||
return in;
|
||||
}
|
||||
|
||||
void MutationsInterpreter::validate(TableStructureReadLockHolder &)
|
||||
void MutationsInterpreter::validate()
|
||||
{
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
|
||||
@ -692,7 +696,7 @@ void MutationsInterpreter::validate(TableStructureReadLockHolder &)
|
||||
addStreamsForLaterStages(stages, in)->getHeader();
|
||||
}
|
||||
|
||||
BlockInputStreamPtr MutationsInterpreter::execute(TableStructureReadLockHolder &)
|
||||
BlockInputStreamPtr MutationsInterpreter::execute()
|
||||
{
|
||||
if (!can_execute)
|
||||
throw Exception("Cannot execute mutations interpreter because can_execute flag set to false", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -729,7 +733,7 @@ size_t MutationsInterpreter::evaluateCommandsSize()
|
||||
|
||||
std::optional<SortDescription> MutationsInterpreter::getStorageSortDescriptionIfPossible(const Block & header) const
|
||||
{
|
||||
Names sort_columns = storage->getSortingKeyColumns();
|
||||
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
|
||||
SortDescription sort_description;
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
|
@ -15,7 +15,8 @@ namespace DB
|
||||
class Context;
|
||||
|
||||
/// Return false if the data isn't going to be changed by mutations.
|
||||
bool isStorageTouchedByMutations(StoragePtr storage, const std::vector<MutationCommand> & commands, Context context_copy);
|
||||
bool isStorageTouchedByMutations(
|
||||
StoragePtr storage, const StorageMetadataPtr & metadata_snapshot, const std::vector<MutationCommand> & commands, Context context_copy);
|
||||
|
||||
/// Create an input stream that will read data from storage and apply mutation commands (UPDATEs, DELETEs, MATERIALIZEs)
|
||||
/// to this data.
|
||||
@ -24,14 +25,19 @@ class MutationsInterpreter
|
||||
public:
|
||||
/// Storage to mutate, array of mutations commands and context. If you really want to execute mutation
|
||||
/// use can_execute = true, in other cases (validation, amount of commands) it can be false
|
||||
MutationsInterpreter(StoragePtr storage_, MutationCommands commands_, const Context & context_, bool can_execute_);
|
||||
MutationsInterpreter(
|
||||
StoragePtr storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
MutationCommands commands_,
|
||||
const Context & context_,
|
||||
bool can_execute_);
|
||||
|
||||
void validate(TableStructureReadLockHolder & table_lock_holder);
|
||||
void validate();
|
||||
|
||||
size_t evaluateCommandsSize();
|
||||
|
||||
/// The resulting stream will return blocks containing only changed columns and columns, that we need to recalculate indices.
|
||||
BlockInputStreamPtr execute(TableStructureReadLockHolder & table_lock_holder);
|
||||
BlockInputStreamPtr execute();
|
||||
|
||||
/// Only changed columns.
|
||||
const Block & getUpdatedHeader() const;
|
||||
@ -47,6 +53,7 @@ private:
|
||||
std::optional<SortDescription> getStorageSortDescriptionIfPossible(const Block & header) const;
|
||||
|
||||
StoragePtr storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
MutationCommands commands;
|
||||
Context context;
|
||||
bool can_execute;
|
||||
|
@ -714,7 +714,7 @@ void SyntaxAnalyzerResult::collectSourceColumns(bool add_special)
|
||||
{
|
||||
if (storage)
|
||||
{
|
||||
const ColumnsDescription & columns = storage->getColumns();
|
||||
const ColumnsDescription & columns = metadata_snapshot->getColumns();
|
||||
|
||||
auto columns_from_storage = add_special ? columns.getAll() : columns.getAllPhysical();
|
||||
if (source_columns.empty())
|
||||
@ -1005,14 +1005,19 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyzeSelect(
|
||||
return std::make_shared<const SyntaxAnalyzerResult>(result);
|
||||
}
|
||||
|
||||
SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(ASTPtr & query, const NamesAndTypesList & source_columns, ConstStoragePtr storage, bool allow_aggregations) const
|
||||
SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
||||
ASTPtr & query,
|
||||
const NamesAndTypesList & source_columns,
|
||||
ConstStoragePtr storage,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
bool allow_aggregations) const
|
||||
{
|
||||
if (query->as<ASTSelectQuery>())
|
||||
throw Exception("Not select analyze for select asts.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
const auto & settings = context.getSettingsRef();
|
||||
|
||||
SyntaxAnalyzerResult result(source_columns, storage, false);
|
||||
SyntaxAnalyzerResult result(source_columns, storage, metadata_snapshot, false);
|
||||
|
||||
normalize(query, result.aliases, settings);
|
||||
|
||||
|
@ -16,10 +16,13 @@ class Context;
|
||||
struct Settings;
|
||||
struct SelectQueryOptions;
|
||||
using Scalars = std::map<String, Block>;
|
||||
struct StorageInMemoryMetadata;
|
||||
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
||||
|
||||
struct SyntaxAnalyzerResult
|
||||
{
|
||||
ConstStoragePtr storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
std::shared_ptr<TableJoin> analyzed_join;
|
||||
|
||||
NamesAndTypesList source_columns;
|
||||
@ -51,8 +54,13 @@ struct SyntaxAnalyzerResult
|
||||
/// Results of scalar sub queries
|
||||
Scalars scalars;
|
||||
|
||||
SyntaxAnalyzerResult(const NamesAndTypesList & source_columns_, ConstStoragePtr storage_ = {}, bool add_special = true)
|
||||
SyntaxAnalyzerResult(
|
||||
const NamesAndTypesList & source_columns_,
|
||||
ConstStoragePtr storage_ = {},
|
||||
const StorageMetadataPtr & metadata_snapshot_ = {},
|
||||
bool add_special = true)
|
||||
: storage(storage_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, source_columns(source_columns_)
|
||||
{
|
||||
collectSourceColumns(add_special);
|
||||
@ -86,7 +94,12 @@ public:
|
||||
{}
|
||||
|
||||
/// Analyze and rewrite not select query
|
||||
SyntaxAnalyzerResultPtr analyze(ASTPtr & query, const NamesAndTypesList & source_columns_, ConstStoragePtr storage = {}, bool allow_aggregations = false) const;
|
||||
SyntaxAnalyzerResultPtr analyze(
|
||||
ASTPtr & query,
|
||||
const NamesAndTypesList & source_columns_,
|
||||
ConstStoragePtr storage = {},
|
||||
const StorageMetadataPtr & metadata_snapshot = {},
|
||||
bool allow_aggregations = false) const;
|
||||
|
||||
/// Analyze and rewrite select query
|
||||
SyntaxAnalyzerResultPtr analyzeSelect(
|
||||
|
@ -457,8 +457,9 @@ void SystemLog<LogElement>::prepareTable()
|
||||
|
||||
if (table)
|
||||
{
|
||||
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
const Block expected = LogElement::createBlock();
|
||||
const Block actual = table->getSampleBlockNonMaterialized();
|
||||
const Block actual = metadata_snapshot->getSampleBlockNonMaterialized();
|
||||
|
||||
if (!blocksHaveEqualStructure(actual, expected))
|
||||
{
|
||||
|
@ -351,8 +351,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
{
|
||||
StoragePtr storage = context.executeTableFunction(input_function);
|
||||
auto & input_storage = dynamic_cast<StorageInput &>(*storage);
|
||||
BlockInputStreamPtr input_stream = std::make_shared<InputStreamFromASTInsertQuery>(ast, istr,
|
||||
input_storage.getSampleBlock(), context, input_function);
|
||||
auto input_metadata_snapshot = input_storage.getInMemoryMetadataPtr();
|
||||
BlockInputStreamPtr input_stream = std::make_shared<InputStreamFromASTInsertQuery>(
|
||||
ast, istr, input_metadata_snapshot->getSampleBlock(), context, input_function);
|
||||
input_storage.setInputStream(input_stream);
|
||||
}
|
||||
}
|
||||
|
@ -87,7 +87,8 @@ static NamesAndTypesList getColumnsFromTableExpression(const ASTTableExpression
|
||||
const auto table_function = table_expression.table_function;
|
||||
auto * query_context = const_cast<Context *>(&context.getQueryContext());
|
||||
const auto & function_storage = query_context->executeTableFunction(table_function);
|
||||
const auto & columns = function_storage->getColumns();
|
||||
auto function_metadata_snapshot = function_storage->getInMemoryMetadataPtr();
|
||||
const auto & columns = function_metadata_snapshot->getColumns();
|
||||
names_and_type_list = columns.getOrdinary();
|
||||
materialized = columns.getMaterialized();
|
||||
aliases = columns.getAliases();
|
||||
@ -97,7 +98,8 @@ static NamesAndTypesList getColumnsFromTableExpression(const ASTTableExpression
|
||||
{
|
||||
auto table_id = context.resolveStorageID(table_expression.database_and_table_name);
|
||||
const auto & table = DatabaseCatalog::instance().getTable(table_id, context);
|
||||
const auto & columns = table->getColumns();
|
||||
auto table_metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
const auto & columns = table_metadata_snapshot->getColumns();
|
||||
names_and_type_list = columns.getOrdinary();
|
||||
materialized = columns.getMaterialized();
|
||||
aliases = columns.getAliases();
|
||||
|
@ -90,14 +90,14 @@ std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
|
||||
{
|
||||
auto * query_context = const_cast<Context *>(&context.getQueryContext());
|
||||
const auto & storage = query_context->executeTableFunction(table_expression);
|
||||
columns = storage->getColumns().getOrdinary();
|
||||
columns = storage->getInMemoryMetadataPtr()->getColumns().getOrdinary();
|
||||
select_query->addTableFunction(*const_cast<ASTPtr *>(&table_expression)); // XXX: const_cast should be avoided!
|
||||
}
|
||||
else
|
||||
{
|
||||
auto table_id = context.resolveStorageID(table_expression);
|
||||
const auto & storage = DatabaseCatalog::instance().getTable(table_id, context);
|
||||
columns = storage->getColumns().getOrdinary();
|
||||
columns = storage->getInMemoryMetadataPtr()->getColumns().getOrdinary();
|
||||
select_query->replaceDatabaseAndTable(table_id);
|
||||
}
|
||||
|
||||
|
@ -62,12 +62,12 @@ public:
|
||||
|
||||
/// Do not allow to change the table while the processors of pipe are alive.
|
||||
/// TODO: move it to pipeline.
|
||||
void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); }
|
||||
void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); }
|
||||
/// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible.
|
||||
void addInterpreterContext(std::shared_ptr<Context> context) { interpreter_context.emplace_back(std::move(context)); }
|
||||
void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); }
|
||||
|
||||
const std::vector<TableStructureReadLockHolder> & getTableLocks() const { return table_locks; }
|
||||
const std::vector<TableLockHolder> & getTableLocks() const { return table_locks; }
|
||||
const std::vector<std::shared_ptr<Context>> & getContexts() const { return interpreter_context; }
|
||||
const std::vector<StoragePtr> & getStorageHolders() const { return storage_holders; }
|
||||
|
||||
@ -80,7 +80,7 @@ private:
|
||||
/// It is the max number of processors which can be executed in parallel for each step. See QueryPipeline::Streams.
|
||||
size_t max_parallel_streams = 0;
|
||||
|
||||
std::vector<TableStructureReadLockHolder> table_locks;
|
||||
std::vector<TableLockHolder> table_locks;
|
||||
|
||||
/// Some processors may implicitly use Context or temporary Storage created by Interpreter.
|
||||
/// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here,
|
||||
|
@ -7,14 +7,13 @@
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Storages/TableLockHolder.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class TableStructureReadLock;
|
||||
using TableStructureReadLockPtr = std::shared_ptr<TableStructureReadLock>;
|
||||
using TableStructureReadLocks = std::vector<TableStructureReadLockHolder>;
|
||||
|
||||
using TableLockHolders = std::vector<TableLockHolder>;
|
||||
class Context;
|
||||
|
||||
class IOutputFormat;
|
||||
@ -146,7 +145,7 @@ public:
|
||||
|
||||
const Block & getHeader() const { return current_header; }
|
||||
|
||||
void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); }
|
||||
void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); }
|
||||
void addInterpreterContext(std::shared_ptr<Context> context) { interpreter_context.emplace_back(std::move(context)); }
|
||||
void addStorageHolder(StoragePtr storage) { storage_holders.emplace_back(std::move(storage)); }
|
||||
|
||||
@ -180,7 +179,7 @@ private:
|
||||
/// because QueryPipeline is alive until query is finished.
|
||||
std::vector<std::shared_ptr<Context>> interpreter_context;
|
||||
std::vector<StoragePtr> storage_holders;
|
||||
TableStructureReadLocks table_locks;
|
||||
TableLockHolders table_locks;
|
||||
|
||||
/// Common header for each stream.
|
||||
Block current_header;
|
||||
|
@ -13,7 +13,8 @@ namespace DB
|
||||
{
|
||||
|
||||
ReadFromStorageStep::ReadFromStorageStep(
|
||||
TableStructureReadLockHolder table_lock_,
|
||||
TableLockHolder table_lock_,
|
||||
StorageMetadataPtr & metadata_snapshot_,
|
||||
SelectQueryOptions options_,
|
||||
StoragePtr storage_,
|
||||
const Names & required_columns_,
|
||||
@ -23,6 +24,7 @@ ReadFromStorageStep::ReadFromStorageStep(
|
||||
size_t max_block_size_,
|
||||
size_t max_streams_)
|
||||
: table_lock(std::move(table_lock_))
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, options(std::move(options_))
|
||||
, storage(std::move(storage_))
|
||||
, required_columns(required_columns_)
|
||||
@ -35,11 +37,11 @@ ReadFromStorageStep::ReadFromStorageStep(
|
||||
/// Note: we read from storage in constructor of step because we don't know real header before reading.
|
||||
/// It will be fixed when storage return QueryPlanStep itself.
|
||||
|
||||
Pipes pipes = storage->read(required_columns, query_info, *context, processing_stage, max_block_size, max_streams);
|
||||
Pipes pipes = storage->read(required_columns, metadata_snapshot, query_info, *context, processing_stage, max_block_size, max_streams);
|
||||
|
||||
if (pipes.empty())
|
||||
{
|
||||
Pipe pipe(std::make_shared<NullSource>(storage->getSampleBlockForColumns(required_columns)));
|
||||
Pipe pipe(std::make_shared<NullSource>(metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID())));
|
||||
|
||||
if (query_info.prewhere_info)
|
||||
{
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Processors/QueryPlan/IQueryPlanStep.h>
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
#include <Storages/TableStructureLockHolder.h>
|
||||
#include <Storages/TableLockHolder.h>
|
||||
#include <Interpreters/SelectQueryOptions.h>
|
||||
|
||||
namespace DB
|
||||
@ -9,6 +9,9 @@ namespace DB
|
||||
class IStorage;
|
||||
using StoragePtr = std::shared_ptr<IStorage>;
|
||||
|
||||
struct StorageInMemoryMetadata;
|
||||
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
||||
|
||||
struct SelectQueryInfo;
|
||||
|
||||
struct PrewhereInfo;
|
||||
@ -18,7 +21,8 @@ class ReadFromStorageStep : public IQueryPlanStep
|
||||
{
|
||||
public:
|
||||
ReadFromStorageStep(
|
||||
TableStructureReadLockHolder table_lock,
|
||||
TableLockHolder table_lock,
|
||||
StorageMetadataPtr & metadata_snapshot,
|
||||
SelectQueryOptions options,
|
||||
StoragePtr storage,
|
||||
const Names & required_columns,
|
||||
@ -35,7 +39,8 @@ public:
|
||||
QueryPipelinePtr updatePipeline(QueryPipelines) override;
|
||||
|
||||
private:
|
||||
TableStructureReadLockHolder table_lock;
|
||||
TableLockHolder table_lock;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
SelectQueryOptions options;
|
||||
|
||||
StoragePtr storage;
|
||||
|
@ -66,7 +66,7 @@ void CreatingSetsTransform::startSubquery(SubqueryForSet & subquery)
|
||||
elapsed_nanoseconds = 0;
|
||||
|
||||
if (subquery.table)
|
||||
table_out = subquery.table->write({}, context);
|
||||
table_out = subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), context);
|
||||
|
||||
done_with_set = !subquery.set;
|
||||
done_with_join = !subquery.join;
|
||||
|
@ -254,7 +254,8 @@ void MySQLHandler::comFieldList(ReadBuffer & payload)
|
||||
packet.readPayload(payload);
|
||||
String database = connection_context.getCurrentDatabase();
|
||||
StoragePtr table_ptr = DatabaseCatalog::instance().getTable({database, packet.table}, connection_context);
|
||||
for (const NameAndTypePair & column: table_ptr->getColumns().getAll())
|
||||
auto metadata_snapshot = table_ptr->getInMemoryMetadataPtr();
|
||||
for (const NameAndTypePair & column : metadata_snapshot->getColumns().getAll())
|
||||
{
|
||||
ColumnDefinition column_definition(
|
||||
database, packet.table, packet.table, column.name, column.name, CharacterSet::binary, 100, ColumnType::MYSQL_TYPE_STRING, 0, 0
|
||||
|
@ -214,17 +214,18 @@ void TCPHandler::runImpl()
|
||||
if (&context != &query_context.value())
|
||||
throw Exception("Unexpected context in Input initializer", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto metadata_snapshot = input_storage->getInMemoryMetadataPtr();
|
||||
state.need_receive_data_for_input = true;
|
||||
|
||||
/// Send ColumnsDescription for input storage.
|
||||
if (client_revision >= DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA
|
||||
&& query_context->getSettingsRef().input_format_defaults_for_omitted_fields)
|
||||
{
|
||||
sendTableColumns(input_storage->getColumns());
|
||||
sendTableColumns(metadata_snapshot->getColumns());
|
||||
}
|
||||
|
||||
/// Send block to the client - input storage structure.
|
||||
state.input_header = input_storage->getSampleBlock();
|
||||
state.input_header = metadata_snapshot->getSampleBlock();
|
||||
sendData(state.input_header);
|
||||
});
|
||||
|
||||
@ -475,7 +476,10 @@ void TCPHandler::processInsertQuery(const Settings & connection_settings)
|
||||
if (query_context->getSettingsRef().input_format_defaults_for_omitted_fields)
|
||||
{
|
||||
if (!table_id.empty())
|
||||
sendTableColumns(DatabaseCatalog::instance().getTable(table_id, *query_context)->getColumns());
|
||||
{
|
||||
auto storage_ptr = DatabaseCatalog::instance().getTable(table_id, *query_context);
|
||||
sendTableColumns(storage_ptr->getInMemoryMetadataPtr()->getColumns());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -953,8 +957,9 @@ bool TCPHandler::receiveData(bool scalar)
|
||||
storage = temporary_table.getTable();
|
||||
query_context->addExternalTable(temporary_id.table_name, std::move(temporary_table));
|
||||
}
|
||||
auto metadata_snapshot = storage->getInMemoryMetadataPtr();
|
||||
/// The data will be written directly to the table.
|
||||
state.io.out = storage->write(ASTPtr(), *query_context);
|
||||
state.io.out = storage->write(ASTPtr(), metadata_snapshot, *query_context);
|
||||
}
|
||||
if (state.need_receive_data_for_input)
|
||||
state.block_for_input = block;
|
||||
|
@ -83,18 +83,29 @@ static void writeBlockConvert(const BlockOutputStreamPtr & out, const Block & bl
|
||||
|
||||
|
||||
DistributedBlockOutputStream::DistributedBlockOutputStream(
|
||||
const Context & context_, StorageDistributed & storage_, const ASTPtr & query_ast_, const ClusterPtr & cluster_,
|
||||
bool insert_sync_, UInt64 insert_timeout_)
|
||||
: context(context_), storage(storage_), query_ast(query_ast_), query_string(queryToString(query_ast_)),
|
||||
cluster(cluster_), insert_sync(insert_sync_),
|
||||
insert_timeout(insert_timeout_), log(&Poco::Logger::get("DistributedBlockOutputStream"))
|
||||
const Context & context_,
|
||||
StorageDistributed & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const ASTPtr & query_ast_,
|
||||
const ClusterPtr & cluster_,
|
||||
bool insert_sync_,
|
||||
UInt64 insert_timeout_)
|
||||
: context(context_)
|
||||
, storage(storage_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, query_ast(query_ast_)
|
||||
, query_string(queryToString(query_ast_))
|
||||
, cluster(cluster_)
|
||||
, insert_sync(insert_sync_)
|
||||
, insert_timeout(insert_timeout_)
|
||||
, log(&Poco::Logger::get("DistributedBlockOutputStream"))
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
Block DistributedBlockOutputStream::getHeader() const
|
||||
{
|
||||
return storage.getSampleBlock();
|
||||
return metadata_snapshot->getSampleBlock();
|
||||
}
|
||||
|
||||
|
||||
@ -109,7 +120,7 @@ void DistributedBlockOutputStream::write(const Block & block)
|
||||
|
||||
/* They are added by the AddingDefaultBlockOutputStream, and we will get
|
||||
* different number of columns eventually */
|
||||
for (const auto & col : storage.getColumns().getMaterialized())
|
||||
for (const auto & col : metadata_snapshot->getColumns().getMaterialized())
|
||||
{
|
||||
if (ordinary_block.has(col.name))
|
||||
{
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/Throttler.h>
|
||||
@ -36,8 +37,14 @@ class StorageDistributed;
|
||||
class DistributedBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
DistributedBlockOutputStream(const Context & context_, StorageDistributed & storage_, const ASTPtr & query_ast_,
|
||||
const ClusterPtr & cluster_, bool insert_sync_, UInt64 insert_timeout_);
|
||||
DistributedBlockOutputStream(
|
||||
const Context & context_,
|
||||
StorageDistributed & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const ASTPtr & query_ast_,
|
||||
const ClusterPtr & cluster_,
|
||||
bool insert_sync_,
|
||||
UInt64 insert_timeout_);
|
||||
|
||||
Block getHeader() const override;
|
||||
void write(const Block & block) override;
|
||||
@ -79,6 +86,7 @@ private:
|
||||
private:
|
||||
const Context & context;
|
||||
StorageDistributed & storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
ASTPtr query_ast;
|
||||
String query_string;
|
||||
ClusterPtr cluster;
|
||||
|
@ -19,295 +19,15 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int COLUMN_QUERIED_MORE_THAN_ONCE;
|
||||
extern const int DUPLICATE_COLUMN;
|
||||
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
||||
extern const int EMPTY_LIST_OF_COLUMNS_QUERIED;
|
||||
extern const int NO_SUCH_COLUMN_IN_TABLE;
|
||||
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
|
||||
extern const int TYPE_MISMATCH;
|
||||
extern const int TABLE_IS_DROPPED;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int DEADLOCK_AVOIDED;
|
||||
}
|
||||
|
||||
const ColumnsDescription & IStorage::getColumns() const
|
||||
{
|
||||
return metadata.columns;
|
||||
}
|
||||
|
||||
const IndicesDescription & IStorage::getSecondaryIndices() const
|
||||
{
|
||||
return metadata.secondary_indices;
|
||||
}
|
||||
|
||||
bool IStorage::hasSecondaryIndices() const
|
||||
{
|
||||
return !metadata.secondary_indices.empty();
|
||||
}
|
||||
|
||||
const ConstraintsDescription & IStorage::getConstraints() const
|
||||
{
|
||||
return metadata.constraints;
|
||||
}
|
||||
|
||||
Block IStorage::getSampleBlock() const
|
||||
{
|
||||
Block res;
|
||||
|
||||
for (const auto & column : getColumns().getAllPhysical())
|
||||
res.insert({column.type->createColumn(), column.type, column.name});
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
Block IStorage::getSampleBlockWithVirtuals() const
|
||||
{
|
||||
auto res = getSampleBlock();
|
||||
|
||||
/// Virtual columns must be appended after ordinary, because user can
|
||||
/// override them.
|
||||
for (const auto & column : getVirtuals())
|
||||
res.insert({column.type->createColumn(), column.type, column.name});
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
Block IStorage::getSampleBlockNonMaterialized() const
|
||||
{
|
||||
Block res;
|
||||
|
||||
for (const auto & column : getColumns().getOrdinary())
|
||||
res.insert({column.type->createColumn(), column.type, column.name});
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
Block IStorage::getSampleBlockForColumns(const Names & column_names) const
|
||||
{
|
||||
Block res;
|
||||
|
||||
std::unordered_map<String, DataTypePtr> columns_map;
|
||||
|
||||
NamesAndTypesList all_columns = getColumns().getAll();
|
||||
for (const auto & elem : all_columns)
|
||||
columns_map.emplace(elem.name, elem.type);
|
||||
|
||||
/// Virtual columns must be appended after ordinary, because user can
|
||||
/// override them.
|
||||
for (const auto & column : getVirtuals())
|
||||
columns_map.emplace(column.name, column.type);
|
||||
|
||||
for (const auto & name : column_names)
|
||||
{
|
||||
auto it = columns_map.find(name);
|
||||
if (it != columns_map.end())
|
||||
{
|
||||
res.insert({it->second->createColumn(), it->second, it->first});
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
"Column " + backQuote(name) + " not found in table " + getStorageID().getNameForLogs(), ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK);
|
||||
}
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
using NamesAndTypesMap = google::dense_hash_map<StringRef, const IDataType *, StringRefHash>;
|
||||
using UniqueStrings = google::dense_hash_set<StringRef, StringRefHash>;
|
||||
#else
|
||||
using NamesAndTypesMap = google::sparsehash::dense_hash_map<StringRef, const IDataType *, StringRefHash>;
|
||||
using UniqueStrings = google::sparsehash::dense_hash_set<StringRef, StringRefHash>;
|
||||
#endif
|
||||
|
||||
String listOfColumns(const NamesAndTypesList & available_columns)
|
||||
{
|
||||
std::stringstream ss;
|
||||
for (auto it = available_columns.begin(); it != available_columns.end(); ++it)
|
||||
{
|
||||
if (it != available_columns.begin())
|
||||
ss << ", ";
|
||||
ss << it->name;
|
||||
}
|
||||
return ss.str();
|
||||
}
|
||||
|
||||
NamesAndTypesMap getColumnsMap(const NamesAndTypesList & columns)
|
||||
{
|
||||
NamesAndTypesMap res;
|
||||
res.set_empty_key(StringRef());
|
||||
|
||||
for (const auto & column : columns)
|
||||
res.insert({column.name, column.type.get()});
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
UniqueStrings initUniqueStrings()
|
||||
{
|
||||
UniqueStrings strings;
|
||||
strings.set_empty_key(StringRef());
|
||||
return strings;
|
||||
}
|
||||
}
|
||||
|
||||
void IStorage::check(const Names & column_names, bool include_virtuals) const
|
||||
{
|
||||
NamesAndTypesList available_columns = getColumns().getAllPhysical();
|
||||
if (include_virtuals)
|
||||
{
|
||||
auto virtuals = getVirtuals();
|
||||
available_columns.insert(available_columns.end(), virtuals.begin(), virtuals.end());
|
||||
}
|
||||
|
||||
const String list_of_columns = listOfColumns(available_columns);
|
||||
|
||||
if (column_names.empty())
|
||||
throw Exception("Empty list of columns queried. There are columns: " + list_of_columns, ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED);
|
||||
|
||||
const auto columns_map = getColumnsMap(available_columns);
|
||||
|
||||
auto unique_names = initUniqueStrings();
|
||||
for (const auto & name : column_names)
|
||||
{
|
||||
if (columns_map.end() == columns_map.find(name))
|
||||
throw Exception(
|
||||
"There is no column with name " + backQuote(name) + " in table " + getStorageID().getNameForLogs() + ". There are columns: " + list_of_columns,
|
||||
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
|
||||
if (unique_names.end() != unique_names.find(name))
|
||||
throw Exception("Column " + name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE);
|
||||
unique_names.insert(name);
|
||||
}
|
||||
}
|
||||
|
||||
void IStorage::check(const NamesAndTypesList & provided_columns) const
|
||||
{
|
||||
const NamesAndTypesList & available_columns = getColumns().getAllPhysical();
|
||||
const auto columns_map = getColumnsMap(available_columns);
|
||||
|
||||
auto unique_names = initUniqueStrings();
|
||||
for (const NameAndTypePair & column : provided_columns)
|
||||
{
|
||||
auto it = columns_map.find(column.name);
|
||||
if (columns_map.end() == it)
|
||||
throw Exception(
|
||||
"There is no column with name " + column.name + ". There are columns: " + listOfColumns(available_columns),
|
||||
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
|
||||
if (!column.type->equals(*it->second))
|
||||
throw Exception(
|
||||
"Type mismatch for column " + column.name + ". Column has type " + it->second->getName() + ", got type "
|
||||
+ column.type->getName(),
|
||||
ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
if (unique_names.end() != unique_names.find(column.name))
|
||||
throw Exception("Column " + column.name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE);
|
||||
unique_names.insert(column.name);
|
||||
}
|
||||
}
|
||||
|
||||
void IStorage::check(const NamesAndTypesList & provided_columns, const Names & column_names) const
|
||||
{
|
||||
const NamesAndTypesList & available_columns = getColumns().getAllPhysical();
|
||||
const auto available_columns_map = getColumnsMap(available_columns);
|
||||
const auto & provided_columns_map = getColumnsMap(provided_columns);
|
||||
|
||||
if (column_names.empty())
|
||||
throw Exception(
|
||||
"Empty list of columns queried. There are columns: " + listOfColumns(available_columns),
|
||||
ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED);
|
||||
|
||||
auto unique_names = initUniqueStrings();
|
||||
for (const String & name : column_names)
|
||||
{
|
||||
auto it = provided_columns_map.find(name);
|
||||
if (provided_columns_map.end() == it)
|
||||
continue;
|
||||
|
||||
auto jt = available_columns_map.find(name);
|
||||
if (available_columns_map.end() == jt)
|
||||
throw Exception(
|
||||
"There is no column with name " + name + ". There are columns: " + listOfColumns(available_columns),
|
||||
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
|
||||
if (!it->second->equals(*jt->second))
|
||||
throw Exception(
|
||||
"Type mismatch for column " + name + ". Column has type " + jt->second->getName() + ", got type " + it->second->getName(),
|
||||
ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
if (unique_names.end() != unique_names.find(name))
|
||||
throw Exception("Column " + name + " queried more than once", ErrorCodes::COLUMN_QUERIED_MORE_THAN_ONCE);
|
||||
unique_names.insert(name);
|
||||
}
|
||||
}
|
||||
|
||||
void IStorage::check(const Block & block, bool need_all) const
|
||||
{
|
||||
const NamesAndTypesList & available_columns = getColumns().getAllPhysical();
|
||||
const auto columns_map = getColumnsMap(available_columns);
|
||||
|
||||
NameSet names_in_block;
|
||||
|
||||
block.checkNumberOfRows();
|
||||
|
||||
for (const auto & column : block)
|
||||
{
|
||||
if (names_in_block.count(column.name))
|
||||
throw Exception("Duplicate column " + column.name + " in block", ErrorCodes::DUPLICATE_COLUMN);
|
||||
|
||||
names_in_block.insert(column.name);
|
||||
|
||||
auto it = columns_map.find(column.name);
|
||||
if (columns_map.end() == it)
|
||||
throw Exception(
|
||||
"There is no column with name " + column.name + ". There are columns: " + listOfColumns(available_columns),
|
||||
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
|
||||
if (!column.type->equals(*it->second))
|
||||
throw Exception(
|
||||
"Type mismatch for column " + column.name + ". Column has type " + it->second->getName() + ", got type "
|
||||
+ column.type->getName(),
|
||||
ErrorCodes::TYPE_MISMATCH);
|
||||
}
|
||||
|
||||
if (need_all && names_in_block.size() < columns_map.size())
|
||||
{
|
||||
for (const auto & available_column : available_columns)
|
||||
{
|
||||
if (!names_in_block.count(available_column.name))
|
||||
throw Exception("Expected column " + available_column.name, ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void IStorage::setColumns(ColumnsDescription columns_)
|
||||
{
|
||||
if (columns_.getAllPhysical().empty())
|
||||
throw Exception("Empty list of columns passed", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
|
||||
metadata.columns = std::move(columns_);
|
||||
}
|
||||
|
||||
void IStorage::setSecondaryIndices(IndicesDescription secondary_indices_)
|
||||
{
|
||||
metadata.secondary_indices = std::move(secondary_indices_);
|
||||
}
|
||||
|
||||
void IStorage::setConstraints(ConstraintsDescription constraints_)
|
||||
{
|
||||
metadata.constraints = std::move(constraints_);
|
||||
}
|
||||
|
||||
bool IStorage::isVirtualColumn(const String & column_name) const
|
||||
bool IStorage::isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const
|
||||
{
|
||||
/// Virtual column maybe overriden by real column
|
||||
return !getColumns().has(column_name) && getVirtuals().contains(column_name);
|
||||
return !metadata_snapshot->getColumns().has(column_name) && getVirtuals().contains(column_name);
|
||||
}
|
||||
|
||||
RWLockImpl::LockHolder IStorage::tryLockTimed(
|
||||
@ -326,63 +46,48 @@ RWLockImpl::LockHolder IStorage::tryLockTimed(
|
||||
return lock_holder;
|
||||
}
|
||||
|
||||
TableStructureReadLockHolder IStorage::lockStructureForShare(bool will_add_new_data, const String & query_id, const SettingSeconds & acquire_timeout)
|
||||
TableLockHolder IStorage::lockForShare(const String & query_id, const SettingSeconds & acquire_timeout)
|
||||
{
|
||||
TableStructureReadLockHolder result;
|
||||
if (will_add_new_data)
|
||||
result.new_data_structure_lock = tryLockTimed(new_data_structure_lock, RWLockImpl::Read, query_id, acquire_timeout);
|
||||
result.structure_lock = tryLockTimed(structure_lock, RWLockImpl::Read, query_id, acquire_timeout);
|
||||
TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout);
|
||||
|
||||
if (is_dropped)
|
||||
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
TableStructureWriteLockHolder IStorage::lockAlterIntention(const String & query_id, const SettingSeconds & acquire_timeout)
|
||||
TableLockHolder IStorage::lockForAlter(const String & query_id, const SettingSeconds & acquire_timeout)
|
||||
{
|
||||
TableStructureWriteLockHolder result;
|
||||
result.alter_intention_lock = tryLockTimed(alter_intention_lock, RWLockImpl::Write, query_id, acquire_timeout);
|
||||
TableLockHolder result = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout);
|
||||
|
||||
if (is_dropped)
|
||||
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
void IStorage::lockStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id, const SettingSeconds & acquire_timeout)
|
||||
{
|
||||
if (!lock_holder.alter_intention_lock)
|
||||
throw Exception("Alter intention lock for table " + getStorageID().getNameForLogs() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!lock_holder.new_data_structure_lock)
|
||||
lock_holder.new_data_structure_lock = tryLockTimed(new_data_structure_lock, RWLockImpl::Write, query_id, acquire_timeout);
|
||||
lock_holder.structure_lock = tryLockTimed(structure_lock, RWLockImpl::Write, query_id, acquire_timeout);
|
||||
}
|
||||
|
||||
TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id, const SettingSeconds & acquire_timeout)
|
||||
TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, const SettingSeconds & acquire_timeout)
|
||||
{
|
||||
TableStructureWriteLockHolder result;
|
||||
result.alter_intention_lock = tryLockTimed(alter_intention_lock, RWLockImpl::Write, query_id, acquire_timeout);
|
||||
TableExclusiveLockHolder result;
|
||||
result.alter_lock = tryLockTimed(alter_lock, RWLockImpl::Write, query_id, acquire_timeout);
|
||||
|
||||
if (is_dropped)
|
||||
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
|
||||
|
||||
result.new_data_structure_lock = tryLockTimed(new_data_structure_lock, RWLockImpl::Write, query_id, acquire_timeout);
|
||||
result.structure_lock = tryLockTimed(structure_lock, RWLockImpl::Write, query_id, acquire_timeout);
|
||||
result.drop_lock = tryLockTimed(drop_lock, RWLockImpl::Write, query_id, acquire_timeout);
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
void IStorage::alter(
|
||||
const AlterCommands & params,
|
||||
const Context & context,
|
||||
TableStructureWriteLockHolder & table_lock_holder)
|
||||
const AlterCommands & params, const Context & context, TableLockHolder &)
|
||||
{
|
||||
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
auto table_id = getStorageID();
|
||||
StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
|
||||
params.apply(new_metadata, context);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata);
|
||||
setColumns(std::move(new_metadata.columns));
|
||||
setInMemoryMetadata(new_metadata);
|
||||
}
|
||||
|
||||
|
||||
@ -415,277 +120,4 @@ NamesAndTypesList IStorage::getVirtuals() const
|
||||
return {};
|
||||
}
|
||||
|
||||
const KeyDescription & IStorage::getPartitionKey() const
|
||||
{
|
||||
return metadata.partition_key;
|
||||
}
|
||||
|
||||
void IStorage::setPartitionKey(const KeyDescription & partition_key_)
|
||||
{
|
||||
metadata.partition_key = partition_key_;
|
||||
}
|
||||
|
||||
bool IStorage::isPartitionKeyDefined() const
|
||||
{
|
||||
return metadata.partition_key.definition_ast != nullptr;
|
||||
}
|
||||
|
||||
bool IStorage::hasPartitionKey() const
|
||||
{
|
||||
return !metadata.partition_key.column_names.empty();
|
||||
}
|
||||
|
||||
Names IStorage::getColumnsRequiredForPartitionKey() const
|
||||
{
|
||||
if (hasPartitionKey())
|
||||
return metadata.partition_key.expression->getRequiredColumns();
|
||||
return {};
|
||||
}
|
||||
|
||||
const KeyDescription & IStorage::getSortingKey() const
|
||||
{
|
||||
return metadata.sorting_key;
|
||||
}
|
||||
|
||||
void IStorage::setSortingKey(const KeyDescription & sorting_key_)
|
||||
{
|
||||
metadata.sorting_key = sorting_key_;
|
||||
}
|
||||
|
||||
bool IStorage::isSortingKeyDefined() const
|
||||
{
|
||||
return metadata.sorting_key.definition_ast != nullptr;
|
||||
}
|
||||
|
||||
bool IStorage::hasSortingKey() const
|
||||
{
|
||||
return !metadata.sorting_key.column_names.empty();
|
||||
}
|
||||
|
||||
Names IStorage::getColumnsRequiredForSortingKey() const
|
||||
{
|
||||
if (hasSortingKey())
|
||||
return metadata.sorting_key.expression->getRequiredColumns();
|
||||
return {};
|
||||
}
|
||||
|
||||
Names IStorage::getSortingKeyColumns() const
|
||||
{
|
||||
if (hasSortingKey())
|
||||
return metadata.sorting_key.column_names;
|
||||
return {};
|
||||
}
|
||||
|
||||
const KeyDescription & IStorage::getPrimaryKey() const
|
||||
{
|
||||
return metadata.primary_key;
|
||||
}
|
||||
|
||||
void IStorage::setPrimaryKey(const KeyDescription & primary_key_)
|
||||
{
|
||||
metadata.primary_key = primary_key_;
|
||||
}
|
||||
|
||||
bool IStorage::isPrimaryKeyDefined() const
|
||||
{
|
||||
return metadata.primary_key.definition_ast != nullptr;
|
||||
}
|
||||
|
||||
bool IStorage::hasPrimaryKey() const
|
||||
{
|
||||
return !metadata.primary_key.column_names.empty();
|
||||
}
|
||||
|
||||
Names IStorage::getColumnsRequiredForPrimaryKey() const
|
||||
{
|
||||
if (hasPrimaryKey())
|
||||
return metadata.primary_key.expression->getRequiredColumns();
|
||||
return {};
|
||||
}
|
||||
|
||||
Names IStorage::getPrimaryKeyColumns() const
|
||||
{
|
||||
if (!metadata.primary_key.column_names.empty())
|
||||
return metadata.primary_key.column_names;
|
||||
return {};
|
||||
}
|
||||
|
||||
const KeyDescription & IStorage::getSamplingKey() const
|
||||
{
|
||||
return metadata.sampling_key;
|
||||
}
|
||||
|
||||
void IStorage::setSamplingKey(const KeyDescription & sampling_key_)
|
||||
{
|
||||
metadata.sampling_key = sampling_key_;
|
||||
}
|
||||
|
||||
|
||||
bool IStorage::isSamplingKeyDefined() const
|
||||
{
|
||||
return metadata.sampling_key.definition_ast != nullptr;
|
||||
}
|
||||
|
||||
bool IStorage::hasSamplingKey() const
|
||||
{
|
||||
return !metadata.sampling_key.column_names.empty();
|
||||
}
|
||||
|
||||
Names IStorage::getColumnsRequiredForSampling() const
|
||||
{
|
||||
if (hasSamplingKey())
|
||||
return metadata.sampling_key.expression->getRequiredColumns();
|
||||
return {};
|
||||
}
|
||||
|
||||
TTLTableDescription IStorage::getTableTTLs() const
|
||||
{
|
||||
std::lock_guard lock(ttl_mutex);
|
||||
return metadata.table_ttl;
|
||||
}
|
||||
|
||||
void IStorage::setTableTTLs(const TTLTableDescription & table_ttl_)
|
||||
{
|
||||
std::lock_guard lock(ttl_mutex);
|
||||
metadata.table_ttl = table_ttl_;
|
||||
}
|
||||
|
||||
bool IStorage::hasAnyTableTTL() const
|
||||
{
|
||||
return hasAnyMoveTTL() || hasRowsTTL();
|
||||
}
|
||||
|
||||
TTLColumnsDescription IStorage::getColumnTTLs() const
|
||||
{
|
||||
std::lock_guard lock(ttl_mutex);
|
||||
return metadata.column_ttls_by_name;
|
||||
}
|
||||
|
||||
void IStorage::setColumnTTLs(const TTLColumnsDescription & column_ttls_by_name_)
|
||||
{
|
||||
std::lock_guard lock(ttl_mutex);
|
||||
metadata.column_ttls_by_name = column_ttls_by_name_;
|
||||
}
|
||||
|
||||
bool IStorage::hasAnyColumnTTL() const
|
||||
{
|
||||
std::lock_guard lock(ttl_mutex);
|
||||
return !metadata.column_ttls_by_name.empty();
|
||||
}
|
||||
|
||||
TTLDescription IStorage::getRowsTTL() const
|
||||
{
|
||||
std::lock_guard lock(ttl_mutex);
|
||||
return metadata.table_ttl.rows_ttl;
|
||||
}
|
||||
|
||||
bool IStorage::hasRowsTTL() const
|
||||
{
|
||||
std::lock_guard lock(ttl_mutex);
|
||||
return metadata.table_ttl.rows_ttl.expression != nullptr;
|
||||
}
|
||||
|
||||
TTLDescriptions IStorage::getMoveTTLs() const
|
||||
{
|
||||
std::lock_guard lock(ttl_mutex);
|
||||
return metadata.table_ttl.move_ttl;
|
||||
}
|
||||
|
||||
bool IStorage::hasAnyMoveTTL() const
|
||||
{
|
||||
std::lock_guard lock(ttl_mutex);
|
||||
return !metadata.table_ttl.move_ttl.empty();
|
||||
}
|
||||
|
||||
|
||||
ColumnDependencies IStorage::getColumnDependencies(const NameSet & updated_columns) const
|
||||
{
|
||||
if (updated_columns.empty())
|
||||
return {};
|
||||
|
||||
ColumnDependencies res;
|
||||
|
||||
NameSet indices_columns;
|
||||
NameSet required_ttl_columns;
|
||||
NameSet updated_ttl_columns;
|
||||
|
||||
auto add_dependent_columns = [&updated_columns](const auto & expression, auto & to_set)
|
||||
{
|
||||
auto requiered_columns = expression->getRequiredColumns();
|
||||
for (const auto & dependency : requiered_columns)
|
||||
{
|
||||
if (updated_columns.count(dependency))
|
||||
{
|
||||
to_set.insert(requiered_columns.begin(), requiered_columns.end());
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
};
|
||||
|
||||
for (const auto & index : getSecondaryIndices())
|
||||
add_dependent_columns(index.expression, indices_columns);
|
||||
|
||||
if (hasRowsTTL())
|
||||
{
|
||||
auto rows_expression = getRowsTTL().expression;
|
||||
if (add_dependent_columns(rows_expression, required_ttl_columns))
|
||||
{
|
||||
/// Filter all columns, if rows TTL expression have to be recalculated.
|
||||
for (const auto & column : getColumns().getAllPhysical())
|
||||
updated_ttl_columns.insert(column.name);
|
||||
}
|
||||
}
|
||||
|
||||
for (const auto & [name, entry] : getColumnTTLs())
|
||||
{
|
||||
if (add_dependent_columns(entry.expression, required_ttl_columns))
|
||||
updated_ttl_columns.insert(name);
|
||||
}
|
||||
|
||||
for (const auto & entry : getMoveTTLs())
|
||||
add_dependent_columns(entry.expression, required_ttl_columns);
|
||||
|
||||
for (const auto & column : indices_columns)
|
||||
res.emplace(column, ColumnDependency::SKIP_INDEX);
|
||||
for (const auto & column : required_ttl_columns)
|
||||
res.emplace(column, ColumnDependency::TTL_EXPRESSION);
|
||||
for (const auto & column : updated_ttl_columns)
|
||||
res.emplace(column, ColumnDependency::TTL_TARGET);
|
||||
|
||||
return res;
|
||||
|
||||
}
|
||||
|
||||
ASTPtr IStorage::getSettingsChanges() const
|
||||
{
|
||||
if (metadata.settings_changes)
|
||||
return metadata.settings_changes->clone();
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
void IStorage::setSettingsChanges(const ASTPtr & settings_changes_)
|
||||
{
|
||||
if (settings_changes_)
|
||||
metadata.settings_changes = settings_changes_->clone();
|
||||
else
|
||||
metadata.settings_changes = nullptr;
|
||||
}
|
||||
|
||||
const SelectQueryDescription & IStorage::getSelectQuery() const
|
||||
{
|
||||
return metadata.select;
|
||||
}
|
||||
|
||||
void IStorage::setSelectQuery(const SelectQueryDescription & select_)
|
||||
{
|
||||
metadata.select = select_;
|
||||
}
|
||||
|
||||
bool IStorage::hasSelectQuery() const
|
||||
{
|
||||
return metadata.select.select_query != nullptr;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -8,7 +8,7 @@
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/TableStructureLockHolder.h>
|
||||
#include <Storages/TableLockHolder.h>
|
||||
#include <Storages/CheckResults.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
#include <Storages/ColumnDependency.h>
|
||||
@ -80,9 +80,10 @@ class IStorage : public std::enable_shared_from_this<IStorage>, public TypePromo
|
||||
{
|
||||
public:
|
||||
IStorage() = delete;
|
||||
/// Storage fields should be initialized in separate methods like setColumns
|
||||
/// or setTableTTLs.
|
||||
explicit IStorage(StorageID storage_id_) : storage_id(std::move(storage_id_)) {} //-V730
|
||||
/// Storage metadata can be set separately in setInMemoryMetadata method
|
||||
explicit IStorage(StorageID storage_id_)
|
||||
: storage_id(std::move(storage_id_))
|
||||
, metadata(std::make_unique<StorageInMemoryMetadata>()) {} //-V730
|
||||
|
||||
virtual ~IStorage() = default;
|
||||
IStorage(const IStorage &) = delete;
|
||||
@ -101,7 +102,7 @@ public:
|
||||
virtual bool isView() const { return false; }
|
||||
|
||||
/// Returns true if the storage supports queries with the SAMPLE section.
|
||||
virtual bool supportsSampling() const { return hasSamplingKey(); }
|
||||
virtual bool supportsSampling() const { return getInMemoryMetadataPtr()->hasSamplingKey(); }
|
||||
|
||||
/// Returns true if the storage supports queries with the FINAL section.
|
||||
virtual bool supportsFinal() const { return false; }
|
||||
@ -129,58 +130,32 @@ public:
|
||||
/// Example is StorageSystemNumbers.
|
||||
virtual bool hasEvenlyDistributedRead() const { return false; }
|
||||
|
||||
/// Returns true if there is set table TTL, any column TTL or any move TTL.
|
||||
virtual bool hasAnyTTL() const { return hasAnyColumnTTL() || hasAnyTableTTL(); }
|
||||
|
||||
/// Optional size information of each physical column.
|
||||
/// Currently it's only used by the MergeTree family for query optimizations.
|
||||
using ColumnSizeByName = std::unordered_map<std::string, ColumnSize>;
|
||||
virtual ColumnSizeByName getColumnSizes() const { return {}; }
|
||||
|
||||
public: /// thread-unsafe part. lockStructure must be acquired
|
||||
public:
|
||||
|
||||
const ColumnsDescription & getColumns() const; /// returns combined set of columns
|
||||
void setColumns(ColumnsDescription columns_); /// sets only real columns, possibly overwrites virtual ones.
|
||||
/// Get mutable version (snapshot) of storage metadata. Metadata object is
|
||||
/// multiversion, so it can be concurrently chaged, but returned copy can be
|
||||
/// used without any locks.
|
||||
StorageInMemoryMetadata getInMemoryMetadata() const { return *metadata.get(); }
|
||||
|
||||
void setSecondaryIndices(IndicesDescription secondary_indices_);
|
||||
const IndicesDescription & getSecondaryIndices() const;
|
||||
/// Has at least one non primary index
|
||||
bool hasSecondaryIndices() const;
|
||||
/// Get immutable version (snapshot) of storage metadata. Metadata object is
|
||||
/// multiversion, so it can be concurrently chaged, but returned copy can be
|
||||
/// used without any locks.
|
||||
StorageMetadataPtr getInMemoryMetadataPtr() const { return metadata.get(); }
|
||||
|
||||
const ConstraintsDescription & getConstraints() const;
|
||||
void setConstraints(ConstraintsDescription constraints_);
|
||||
/// Update storage metadata. Used in ALTER or initialization of Storage.
|
||||
/// Metadata object is multiversion, so this method can be called without
|
||||
/// any locks.
|
||||
void setInMemoryMetadata(const StorageInMemoryMetadata & metadata_)
|
||||
{
|
||||
metadata.set(std::make_unique<StorageInMemoryMetadata>(metadata_));
|
||||
}
|
||||
|
||||
/// Storage settings
|
||||
ASTPtr getSettingsChanges() const;
|
||||
void setSettingsChanges(const ASTPtr & settings_changes_);
|
||||
bool hasSettingsChanges() const { return metadata.settings_changes != nullptr; }
|
||||
|
||||
/// Select query for *View storages.
|
||||
const SelectQueryDescription & getSelectQuery() const;
|
||||
void setSelectQuery(const SelectQueryDescription & select_);
|
||||
bool hasSelectQuery() const;
|
||||
|
||||
StorageInMemoryMetadata getInMemoryMetadata() const { return metadata; }
|
||||
|
||||
Block getSampleBlock() const; /// ordinary + materialized.
|
||||
Block getSampleBlockWithVirtuals() const; /// ordinary + materialized + virtuals.
|
||||
Block getSampleBlockNonMaterialized() const; /// ordinary.
|
||||
Block getSampleBlockForColumns(const Names & column_names) const; /// ordinary + materialized + aliases + virtuals.
|
||||
|
||||
/// Verify that all the requested names are in the table and are set correctly:
|
||||
/// list of names is not empty and the names do not repeat.
|
||||
void check(const Names & column_names, bool include_virtuals = false) const;
|
||||
|
||||
/// Check that all the requested names are in the table and have the correct types.
|
||||
void check(const NamesAndTypesList & columns) const;
|
||||
|
||||
/// Check that all names from the intersection of `names` and `columns` are in the table and have the same types.
|
||||
void check(const NamesAndTypesList & columns, const Names & column_names) const;
|
||||
|
||||
/// Check that the data block contains all the columns of the table with the correct types,
|
||||
/// contains only the columns of the table, and all the columns are different.
|
||||
/// If |need_all| is set, then checks that all the columns of the table are in the block.
|
||||
void check(const Block & block, bool need_all = false) const;
|
||||
|
||||
/// Return list of virtual columns (like _part, _table, etc). In the vast
|
||||
/// majority of cases virtual columns are static constant part of Storage
|
||||
@ -198,36 +173,39 @@ protected:
|
||||
|
||||
/// Returns whether the column is virtual - by default all columns are real.
|
||||
/// Initially reserved virtual column name may be shadowed by real column.
|
||||
bool isVirtualColumn(const String & column_name) const;
|
||||
bool isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const;
|
||||
|
||||
|
||||
private:
|
||||
StorageID storage_id;
|
||||
mutable std::mutex id_mutex;
|
||||
|
||||
/// TODO (alesap) just use multiversion for atomic metadata
|
||||
mutable std::mutex ttl_mutex;
|
||||
StorageInMemoryMetadata metadata;
|
||||
/// Multiversion storage metadata. Allows to read/write storage metadata
|
||||
/// without locks.
|
||||
MultiVersionStorageMetadataPtr metadata;
|
||||
private:
|
||||
RWLockImpl::LockHolder tryLockTimed(
|
||||
const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const SettingSeconds & acquire_timeout) const;
|
||||
|
||||
public:
|
||||
/// Acquire this lock if you need the table structure to remain constant during the execution of
|
||||
/// the query. If will_add_new_data is true, this means that the query will add new data to the table
|
||||
/// (INSERT or a parts merge).
|
||||
TableStructureReadLockHolder lockStructureForShare(bool will_add_new_data, const String & query_id, const SettingSeconds & acquire_timeout);
|
||||
/// Lock table for share. This lock must be acuqired if you want to be sure,
|
||||
/// that table will be not dropped while you holding this lock. It's used in
|
||||
/// variety of cases starting from SELECT queries to background merges in
|
||||
/// MergeTree.
|
||||
TableLockHolder lockForShare(const String & query_id, const SettingSeconds & acquire_timeout);
|
||||
|
||||
/// Acquire this lock at the start of ALTER to lock out other ALTERs and make sure that only you
|
||||
/// can modify the table structure. It can later be upgraded to the exclusive lock.
|
||||
TableStructureWriteLockHolder lockAlterIntention(const String & query_id, const SettingSeconds & acquire_timeout);
|
||||
/// Lock table for alter. This lock must be acuqired in ALTER queries to be
|
||||
/// sure, that we execute only one simultaneous alter. Doesn't affect share lock.
|
||||
TableLockHolder lockForAlter(const String & query_id, const SettingSeconds & acquire_timeout);
|
||||
|
||||
/// Upgrade alter intention lock to the full exclusive structure lock. This is done by ALTER queries
|
||||
/// to ensure that no other query uses the table structure and it can be safely changed.
|
||||
void lockStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id, const SettingSeconds & acquire_timeout);
|
||||
|
||||
/// Acquire the full exclusive lock immediately. No other queries can run concurrently.
|
||||
TableStructureWriteLockHolder lockExclusively(const String & query_id, const SettingSeconds & acquire_timeout);
|
||||
/// Lock table exclusively. This lock must be acuired if you want to be
|
||||
/// sure, that no other thread (SELECT, merge, ALTER, etc.) doing something
|
||||
/// with table. For example it allows to wait all threads before DROP or
|
||||
/// truncate query.
|
||||
///
|
||||
/// NOTE: You have to be 100% sure that you need this lock. It's extremely
|
||||
/// heavyweight and makes table irresponsive.
|
||||
TableExclusiveLockHolder lockExclusively(const String & query_id, const SettingSeconds & acquire_timeout);
|
||||
|
||||
/** 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,
|
||||
@ -293,10 +271,14 @@ public:
|
||||
* num_streams - a recommendation, how many streams to return,
|
||||
* if the storage can return a different number of streams.
|
||||
*
|
||||
* It is guaranteed that the structure of the table will not change over the lifetime of the returned streams (that is, there will not be ALTER, RENAME and DROP).
|
||||
* metadata_snapshot is consistent snapshot of table metadata, it should be
|
||||
* passed in all parts of the returned pipeline. Storage metadata can be
|
||||
* changed during lifetime of the returned pipeline, but the snapshot is
|
||||
* guaranteed to be immutable.
|
||||
*/
|
||||
virtual Pipes read(
|
||||
const Names & /*column_names*/,
|
||||
const StorageMetadataPtr & /*metadata_snapshot*/,
|
||||
const SelectQueryInfo & /*query_info*/,
|
||||
const Context & /*context*/,
|
||||
QueryProcessingStage::Enum /*processed_stage*/,
|
||||
@ -310,10 +292,14 @@ public:
|
||||
* Receives a description of the query, which can contain information about the data write method.
|
||||
* Returns an object by which you can write data sequentially.
|
||||
*
|
||||
* It is guaranteed that the table structure will not change over the lifetime of the returned streams (that is, there will not be ALTER, RENAME and DROP).
|
||||
* metadata_snapshot is consistent snapshot of table metadata, it should be
|
||||
* passed in all parts of the returned streams. Storage metadata can be
|
||||
* changed during lifetime of the returned streams, but the snapshot is
|
||||
* guaranteed to be immutable.
|
||||
*/
|
||||
virtual BlockOutputStreamPtr write(
|
||||
const ASTPtr & /*query*/,
|
||||
const StorageMetadataPtr & /*metadata_snapshot*/,
|
||||
const Context & /*context*/)
|
||||
{
|
||||
throw Exception("Method write is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
@ -328,9 +314,13 @@ public:
|
||||
virtual void drop() {}
|
||||
|
||||
/** Clear the table data and leave it empty.
|
||||
* Must be called under lockForAlter.
|
||||
* Must be called under exclusive lock (lockExclusively).
|
||||
*/
|
||||
virtual void truncate(const ASTPtr & /*query*/, const Context & /* context */, TableStructureWriteLockHolder &)
|
||||
virtual void truncate(
|
||||
const ASTPtr & /*query*/,
|
||||
const StorageMetadataPtr & /* metadata_snapshot */,
|
||||
const Context & /* context */,
|
||||
TableExclusiveLockHolder &)
|
||||
{
|
||||
throw Exception("Truncate is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
@ -352,11 +342,10 @@ public:
|
||||
*/
|
||||
virtual void renameInMemory(const StorageID & new_table_id);
|
||||
|
||||
/** ALTER tables in the form of column changes that do not affect the change to Storage or its parameters.
|
||||
* This method must fully execute the ALTER query, taking care of the locks itself.
|
||||
* To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata.
|
||||
/** ALTER tables in the form of column changes that do not affect the change
|
||||
* to Storage or its parameters. Executes under alter lock (lockForAlter).
|
||||
*/
|
||||
virtual void alter(const AlterCommands & params, const Context & context, TableStructureWriteLockHolder & table_lock_holder);
|
||||
virtual void alter(const AlterCommands & params, const Context & context, TableLockHolder & alter_lock_holder);
|
||||
|
||||
/** Checks that alter commands can be applied to storage. For example, columns can be modified,
|
||||
* or primary key can be changes, etc.
|
||||
@ -366,7 +355,7 @@ public:
|
||||
/** ALTER tables with regard to its partitions.
|
||||
* Should handle locks for each command on its own.
|
||||
*/
|
||||
virtual void alterPartition(const ASTPtr & /* query */, const PartitionCommands & /* commands */, const Context & /* context */)
|
||||
virtual void alterPartition(const ASTPtr & /* query */, const StorageMetadataPtr & /* metadata_snapshot */, const PartitionCommands & /* commands */, const Context & /* context */)
|
||||
{
|
||||
throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
@ -374,7 +363,13 @@ public:
|
||||
/** Perform any background work. For example, combining parts in a MergeTree type table.
|
||||
* Returns whether any work has been done.
|
||||
*/
|
||||
virtual bool optimize(const ASTPtr & /*query*/, const ASTPtr & /*partition*/, bool /*final*/, bool /*deduplicate*/, const Context & /*context*/)
|
||||
virtual bool optimize(
|
||||
const ASTPtr & /*query*/,
|
||||
const StorageMetadataPtr & /*metadata_snapshot*/,
|
||||
const ASTPtr & /*partition*/,
|
||||
bool /*final*/,
|
||||
bool /*deduplicate*/,
|
||||
const Context & /*context*/)
|
||||
{
|
||||
throw Exception("Method optimize is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
@ -421,7 +416,7 @@ public:
|
||||
virtual bool supportsIndexForIn() const { return false; }
|
||||
|
||||
/// Provides a hint that the storage engine may evaluate the IN-condition by using an index.
|
||||
virtual bool mayBenefitFromIndexForIn(const ASTPtr & /* left_in_operand */, const Context & /* query_context */) const { return false; }
|
||||
virtual bool mayBenefitFromIndexForIn(const ASTPtr & /* left_in_operand */, const Context & /* query_context */, const StorageMetadataPtr & /* metadata_snapshot */) const { return false; }
|
||||
|
||||
/// Checks validity of the data
|
||||
virtual CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) { throw Exception("Check query is not supported for " + getName() + " storage", ErrorCodes::NOT_IMPLEMENTED); }
|
||||
@ -439,99 +434,9 @@ public:
|
||||
/// Returns data paths if storage supports it, empty vector otherwise.
|
||||
virtual Strings getDataPaths() const { return {}; }
|
||||
|
||||
/// Returns structure with partition key.
|
||||
const KeyDescription & getPartitionKey() const;
|
||||
/// Set partition key for storage (methods bellow, are just wrappers for this
|
||||
/// struct).
|
||||
void setPartitionKey(const KeyDescription & partition_key_);
|
||||
/// Returns ASTExpressionList of partition key expression for storage or nullptr if there is none.
|
||||
ASTPtr getPartitionKeyAST() const { return metadata.partition_key.definition_ast; }
|
||||
/// Storage has user-defined (in CREATE query) partition key.
|
||||
bool isPartitionKeyDefined() const;
|
||||
/// Storage has partition key.
|
||||
bool hasPartitionKey() const;
|
||||
/// Returns column names that need to be read to calculate partition key.
|
||||
Names getColumnsRequiredForPartitionKey() const;
|
||||
|
||||
|
||||
/// Returns structure with sorting key.
|
||||
const KeyDescription & getSortingKey() const;
|
||||
/// Set sorting key for storage (methods bellow, are just wrappers for this
|
||||
/// struct).
|
||||
void setSortingKey(const KeyDescription & sorting_key_);
|
||||
/// Returns ASTExpressionList of sorting key expression for storage or nullptr if there is none.
|
||||
ASTPtr getSortingKeyAST() const { return metadata.sorting_key.definition_ast; }
|
||||
/// Storage has user-defined (in CREATE query) sorting key.
|
||||
bool isSortingKeyDefined() const;
|
||||
/// Storage has sorting key. It means, that it contains at least one column.
|
||||
bool hasSortingKey() const;
|
||||
/// Returns column names that need to be read to calculate sorting key.
|
||||
Names getColumnsRequiredForSortingKey() const;
|
||||
/// Returns columns names in sorting key specified by user in ORDER BY
|
||||
/// expression. For example: 'a', 'x * y', 'toStartOfMonth(date)', etc.
|
||||
Names getSortingKeyColumns() const;
|
||||
|
||||
/// Returns structure with primary key.
|
||||
const KeyDescription & getPrimaryKey() const;
|
||||
/// Set primary key for storage (methods bellow, are just wrappers for this
|
||||
/// struct).
|
||||
void setPrimaryKey(const KeyDescription & primary_key_);
|
||||
/// Returns ASTExpressionList of primary key expression for storage or nullptr if there is none.
|
||||
ASTPtr getPrimaryKeyAST() const { return metadata.primary_key.definition_ast; }
|
||||
/// Storage has user-defined (in CREATE query) sorting key.
|
||||
bool isPrimaryKeyDefined() const;
|
||||
/// Storage has primary key (maybe part of some other key). It means, that
|
||||
/// it contains at least one column.
|
||||
bool hasPrimaryKey() const;
|
||||
/// Returns column names that need to be read to calculate primary key.
|
||||
Names getColumnsRequiredForPrimaryKey() const;
|
||||
/// Returns columns names in sorting key specified by. For example: 'a', 'x
|
||||
/// * y', 'toStartOfMonth(date)', etc.
|
||||
Names getPrimaryKeyColumns() const;
|
||||
|
||||
/// Returns structure with sampling key.
|
||||
const KeyDescription & getSamplingKey() const;
|
||||
/// Set sampling key for storage (methods bellow, are just wrappers for this
|
||||
/// struct).
|
||||
void setSamplingKey(const KeyDescription & sampling_key_);
|
||||
/// Returns sampling expression AST for storage or nullptr if there is none.
|
||||
ASTPtr getSamplingKeyAST() const { return metadata.sampling_key.definition_ast; }
|
||||
/// Storage has user-defined (in CREATE query) sampling key.
|
||||
bool isSamplingKeyDefined() const;
|
||||
/// Storage has sampling key.
|
||||
bool hasSamplingKey() const;
|
||||
/// Returns column names that need to be read to calculate sampling key.
|
||||
Names getColumnsRequiredForSampling() const;
|
||||
|
||||
/// Returns column names that need to be read for FINAL to work.
|
||||
Names getColumnsRequiredForFinal() const { return getColumnsRequiredForSortingKey(); }
|
||||
|
||||
/// Returns columns, which will be needed to calculate dependencies (skip
|
||||
/// indices, TTL expressions) if we update @updated_columns set of columns.
|
||||
ColumnDependencies getColumnDependencies(const NameSet & updated_columns) const;
|
||||
|
||||
/// Returns storage policy if storage supports it.
|
||||
virtual StoragePolicyPtr getStoragePolicy() const { return {}; }
|
||||
|
||||
/// Common tables TTLs (for rows and moves).
|
||||
TTLTableDescription getTableTTLs() const;
|
||||
void setTableTTLs(const TTLTableDescription & table_ttl_);
|
||||
bool hasAnyTableTTL() const;
|
||||
|
||||
/// Separate TTLs for columns.
|
||||
TTLColumnsDescription getColumnTTLs() const;
|
||||
void setColumnTTLs(const TTLColumnsDescription & column_ttls_by_name_);
|
||||
bool hasAnyColumnTTL() const;
|
||||
|
||||
/// Just wrapper for table TTLs, return rows part of table TTLs.
|
||||
TTLDescription getRowsTTL() const;
|
||||
bool hasRowsTTL() const;
|
||||
|
||||
/// Just wrapper for table TTLs, return moves (to disks or volumes) parts of
|
||||
/// table TTL.
|
||||
TTLDescriptions getMoveTTLs() const;
|
||||
bool hasAnyMoveTTL() const;
|
||||
|
||||
/// If it is possible to quickly determine exact number of rows in the table at this moment of time, then return it.
|
||||
/// Used for:
|
||||
/// - Simple count() opimization
|
||||
@ -558,22 +463,18 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
/// You always need to take the next three locks in this order.
|
||||
/// Lock required for alter queries (lockForAlter). Always taken for write
|
||||
/// (actually can be replaced with std::mutex, but for consistency we use
|
||||
/// RWLock). Allows to execute only one simultaneous alter query. Also it
|
||||
/// should be taken by DROP-like queries, to be sure, that all alters are
|
||||
/// finished.
|
||||
mutable RWLock alter_lock = RWLockImpl::create();
|
||||
|
||||
/// If you hold this lock exclusively, you can be sure that no other structure modifying queries
|
||||
/// (e.g. ALTER, DROP) are concurrently executing. But queries that only read table structure
|
||||
/// (e.g. SELECT, INSERT) can continue to execute.
|
||||
mutable RWLock alter_intention_lock = RWLockImpl::create();
|
||||
|
||||
/// It is taken for share for the entire INSERT query and the entire merge of the parts (for MergeTree).
|
||||
/// ALTER COLUMN queries acquire an exclusive lock to ensure that no new parts with the old structure
|
||||
/// are added to the table and thus the set of parts to modify doesn't change.
|
||||
mutable RWLock new_data_structure_lock = RWLockImpl::create();
|
||||
|
||||
/// Lock for the table column structure (names, types, etc.) and data path.
|
||||
/// It is taken in exclusive mode by queries that modify them (e.g. RENAME, ALTER and DROP)
|
||||
/// and in share mode by other queries.
|
||||
mutable RWLock structure_lock = RWLockImpl::create();
|
||||
/// Lock required for drop queries. Every thread that want to ensure, that
|
||||
/// table is not dropped have to tabke this lock for read (lockForShare).
|
||||
/// DROP-like queries take this lock for write (lockExclusively), to be sure
|
||||
/// that all table threads finished.
|
||||
mutable RWLock drop_lock = RWLockImpl::create();
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -19,15 +19,23 @@ namespace ErrorCodes
|
||||
const auto MAX_FAILED_POLL_ATTEMPTS = 10;
|
||||
|
||||
KafkaBlockInputStream::KafkaBlockInputStream(
|
||||
StorageKafka & storage_, const std::shared_ptr<Context> & context_, const Names & columns, Poco::Logger * log_, size_t max_block_size_, bool commit_in_suffix_)
|
||||
StorageKafka & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const std::shared_ptr<Context> & context_,
|
||||
const Names & columns,
|
||||
Poco::Logger * log_,
|
||||
size_t max_block_size_,
|
||||
bool commit_in_suffix_)
|
||||
: storage(storage_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, context(context_)
|
||||
, column_names(columns)
|
||||
, log(log_)
|
||||
, max_block_size(max_block_size_)
|
||||
, commit_in_suffix(commit_in_suffix_)
|
||||
, non_virtual_header(storage.getSampleBlockNonMaterialized())
|
||||
, virtual_header(storage.getSampleBlockForColumns({"_topic", "_key", "_offset", "_partition", "_timestamp","_timestamp_ms","_headers.name","_headers.value"}))
|
||||
, non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized())
|
||||
, virtual_header(metadata_snapshot->getSampleBlockForColumns(
|
||||
{"_topic", "_key", "_offset", "_partition", "_timestamp", "_timestamp_ms", "_headers.name", "_headers.value"}, storage.getVirtuals(), storage.getStorageID()))
|
||||
{
|
||||
}
|
||||
|
||||
@ -44,7 +52,7 @@ KafkaBlockInputStream::~KafkaBlockInputStream()
|
||||
|
||||
Block KafkaBlockInputStream::getHeader() const
|
||||
{
|
||||
return storage.getSampleBlockForColumns(column_names);
|
||||
return metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID());
|
||||
}
|
||||
|
||||
void KafkaBlockInputStream::readPrefixImpl()
|
||||
|
@ -18,7 +18,13 @@ class KafkaBlockInputStream : public IBlockInputStream
|
||||
{
|
||||
public:
|
||||
KafkaBlockInputStream(
|
||||
StorageKafka & storage_, const std::shared_ptr<Context> & context_, const Names & columns, Poco::Logger * log_, size_t max_block_size_, bool commit_in_suffix = true);
|
||||
StorageKafka & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const std::shared_ptr<Context> & context_,
|
||||
const Names & columns,
|
||||
Poco::Logger * log_,
|
||||
size_t max_block_size_,
|
||||
bool commit_in_suffix = true);
|
||||
~KafkaBlockInputStream() override;
|
||||
|
||||
String getName() const override { return storage.getName(); }
|
||||
@ -33,6 +39,7 @@ public:
|
||||
|
||||
private:
|
||||
StorageKafka & storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
const std::shared_ptr<Context> context;
|
||||
Names column_names;
|
||||
Poco::Logger * log;
|
||||
|
@ -11,13 +11,19 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_CREATE_IO_BUFFER;
|
||||
}
|
||||
|
||||
KafkaBlockOutputStream::KafkaBlockOutputStream(StorageKafka & storage_, const std::shared_ptr<Context> & context_) : storage(storage_), context(context_)
|
||||
KafkaBlockOutputStream::KafkaBlockOutputStream(
|
||||
StorageKafka & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const std::shared_ptr<Context> & context_)
|
||||
: storage(storage_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
Block KafkaBlockOutputStream::getHeader() const
|
||||
{
|
||||
return storage.getSampleBlockNonMaterialized();
|
||||
return metadata_snapshot->getSampleBlockNonMaterialized();
|
||||
}
|
||||
|
||||
void KafkaBlockOutputStream::writePrefix()
|
||||
|
@ -10,7 +10,10 @@ namespace DB
|
||||
class KafkaBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
explicit KafkaBlockOutputStream(StorageKafka & storage_, const std::shared_ptr<Context> & context_);
|
||||
explicit KafkaBlockOutputStream(
|
||||
StorageKafka & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const std::shared_ptr<Context> & context_);
|
||||
|
||||
Block getHeader() const override;
|
||||
|
||||
@ -22,6 +25,7 @@ public:
|
||||
|
||||
private:
|
||||
StorageKafka & storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
const std::shared_ptr<Context> context;
|
||||
ProducerBufferPtr buffer;
|
||||
BlockOutputStreamPtr child;
|
||||
|
@ -140,7 +140,9 @@ StorageKafka::StorageKafka(
|
||||
, intermediate_commit(kafka_settings->kafka_commit_every_batch.value)
|
||||
, settings_adjustments(createSettingsAdjustments())
|
||||
{
|
||||
setColumns(columns_);
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
task = global_context.getSchedulePool().createTask(log->name(), [this]{ threadFunc(); });
|
||||
task->deactivate();
|
||||
|
||||
@ -202,6 +204,7 @@ String StorageKafka::getDefaultClientId(const StorageID & table_id_)
|
||||
|
||||
Pipes StorageKafka::read(
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const SelectQueryInfo & /* query_info */,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum /* processed_stage */,
|
||||
@ -224,7 +227,7 @@ Pipes StorageKafka::read(
|
||||
/// TODO: probably that leads to awful performance.
|
||||
/// FIXME: seems that doesn't help with extra reading and committing unprocessed messages.
|
||||
/// TODO: rewrite KafkaBlockInputStream to KafkaSource. Now it is used in other place.
|
||||
pipes.emplace_back(std::make_shared<SourceFromInputStream>(std::make_shared<KafkaBlockInputStream>(*this, modified_context, column_names, log, 1)));
|
||||
pipes.emplace_back(std::make_shared<SourceFromInputStream>(std::make_shared<KafkaBlockInputStream>(*this, metadata_snapshot, modified_context, column_names, log, 1)));
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Starting reading {} streams", pipes.size());
|
||||
@ -232,14 +235,14 @@ Pipes StorageKafka::read(
|
||||
}
|
||||
|
||||
|
||||
BlockOutputStreamPtr StorageKafka::write(const ASTPtr &, const Context & context)
|
||||
BlockOutputStreamPtr StorageKafka::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context & context)
|
||||
{
|
||||
auto modified_context = std::make_shared<Context>(context);
|
||||
modified_context->applySettingsChanges(settings_adjustments);
|
||||
|
||||
if (topics.size() > 1)
|
||||
throw Exception("Can't write to Kafka table with multiple topics!", ErrorCodes::NOT_IMPLEMENTED);
|
||||
return std::make_shared<KafkaBlockOutputStream>(*this, modified_context);
|
||||
return std::make_shared<KafkaBlockOutputStream>(*this, metadata_snapshot, modified_context);
|
||||
}
|
||||
|
||||
|
||||
@ -519,6 +522,7 @@ bool StorageKafka::streamToViews()
|
||||
auto table = DatabaseCatalog::instance().getTable(table_id, global_context);
|
||||
if (!table)
|
||||
throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR);
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
|
||||
// Create an INSERT query for streaming data
|
||||
auto insert = std::make_shared<ASTInsertQuery>();
|
||||
@ -537,8 +541,7 @@ bool StorageKafka::streamToViews()
|
||||
|
||||
for (size_t i = 0; i < num_created_consumers; ++i)
|
||||
{
|
||||
auto stream
|
||||
= std::make_shared<KafkaBlockInputStream>(*this, kafka_context, block_io.out->getHeader().getNames(), log, block_size, false);
|
||||
auto stream = std::make_shared<KafkaBlockInputStream>(*this, metadata_snapshot, kafka_context, block_io.out->getHeader().getNames(), log, block_size, false);
|
||||
streams.emplace_back(stream);
|
||||
|
||||
// Limit read batch to maximum block size to allow DDL
|
||||
|
@ -39,6 +39,7 @@ public:
|
||||
|
||||
Pipes read(
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & /*metadata_snapshot*/,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
@ -47,6 +48,7 @@ public:
|
||||
|
||||
BlockOutputStreamPtr write(
|
||||
const ASTPtr & query,
|
||||
const StorageMetadataPtr & /*metadata_snapshot*/,
|
||||
const Context & context) override;
|
||||
|
||||
void pushReadBuffer(ConsumerBufferPtr buf);
|
||||
|
@ -18,7 +18,9 @@ public:
|
||||
QueryProcessingStage::Enum to_stage_)
|
||||
: IStorage(table_id_), pipes(std::move(pipes_)), to_stage(to_stage_)
|
||||
{
|
||||
setColumns(columns_);
|
||||
StorageInMemoryMetadata metadata_;
|
||||
metadata_.setColumns(columns_);
|
||||
setInMemoryMetadata(metadata_);
|
||||
}
|
||||
static StoragePtr createStorage(const StorageID & table_id,
|
||||
const ColumnsDescription & columns, Pipes pipes, QueryProcessingStage::Enum to_stage)
|
||||
@ -35,6 +37,7 @@ public:
|
||||
|
||||
Pipes read(
|
||||
const Names & /*column_names*/,
|
||||
const StorageMetadataPtr & /*metadata_snapshot*/,
|
||||
const SelectQueryInfo & /*query_info*/,
|
||||
const Context & /*context*/,
|
||||
QueryProcessingStage::Enum /*processed_stage*/,
|
||||
|
@ -142,12 +142,14 @@ BlockInputStreamPtr StorageLiveView::completeQuery(Pipes pipes)
|
||||
|
||||
auto creator = [&](const StorageID & blocks_id_global)
|
||||
{
|
||||
return StorageBlocks::createStorage(blocks_id_global, getParentStorage()->getColumns(),
|
||||
std::move(pipes), QueryProcessingStage::WithMergeableState);
|
||||
auto parent_table_metadata = getParentStorage()->getInMemoryMetadataPtr();
|
||||
return StorageBlocks::createStorage(
|
||||
blocks_id_global, parent_table_metadata->getColumns(),
|
||||
std::move(pipes), QueryProcessingStage::WithMergeableState);
|
||||
};
|
||||
block_context->addExternalTable(getBlocksTableName(), TemporaryTableHolder(global_context, creator));
|
||||
|
||||
InterpreterSelectQuery select(getInnerBlocksQuery(), *block_context, StoragePtr(), SelectQueryOptions(QueryProcessingStage::Complete));
|
||||
InterpreterSelectQuery select(getInnerBlocksQuery(), *block_context, StoragePtr(), nullptr, SelectQueryOptions(QueryProcessingStage::Complete));
|
||||
BlockInputStreamPtr data = std::make_shared<MaterializingBlockInputStream>(select.execute().getInputStream());
|
||||
|
||||
/// Squashing is needed here because the view query can generate a lot of blocks
|
||||
@ -209,12 +211,14 @@ void StorageLiveView::writeIntoLiveView(
|
||||
|
||||
auto creator = [&](const StorageID & blocks_id_global)
|
||||
{
|
||||
return StorageBlocks::createStorage(blocks_id_global, live_view.getParentStorage()->getColumns(),
|
||||
std::move(pipes), QueryProcessingStage::FetchColumns);
|
||||
auto parent_metadata = live_view.getParentStorage()->getInMemoryMetadataPtr();
|
||||
return StorageBlocks::createStorage(
|
||||
blocks_id_global, parent_metadata->getColumns(),
|
||||
std::move(pipes), QueryProcessingStage::FetchColumns);
|
||||
};
|
||||
TemporaryTableHolder blocks_storage(context, creator);
|
||||
|
||||
InterpreterSelectQuery select_block(mergeable_query, context, blocks_storage.getTable(),
|
||||
InterpreterSelectQuery select_block(mergeable_query, context, blocks_storage.getTable(), blocks_storage.getTable()->getInMemoryMetadataPtr(),
|
||||
QueryProcessingStage::WithMergeableState);
|
||||
|
||||
auto data_mergeable_stream = std::make_shared<MaterializingBlockInputStream>(
|
||||
@ -251,7 +255,9 @@ StorageLiveView::StorageLiveView(
|
||||
live_view_context = std::make_unique<Context>(global_context);
|
||||
live_view_context->makeQueryContext();
|
||||
|
||||
setColumns(columns_);
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
|
||||
if (!query.select)
|
||||
throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY);
|
||||
@ -508,7 +514,7 @@ void StorageLiveView::drop()
|
||||
|
||||
void StorageLiveView::refresh(const Context & context)
|
||||
{
|
||||
auto alter_lock = lockAlterIntention(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
auto table_lock = lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
if (getNewBlocks())
|
||||
@ -518,6 +524,7 @@ void StorageLiveView::refresh(const Context & context)
|
||||
|
||||
Pipes StorageLiveView::read(
|
||||
const Names & /*column_names*/,
|
||||
const StorageMetadataPtr & /*metadata_snapshot*/,
|
||||
const SelectQueryInfo & /*query_info*/,
|
||||
const Context & /*context*/,
|
||||
QueryProcessingStage::Enum /*processed_stage*/,
|
||||
|
@ -126,6 +126,7 @@ public:
|
||||
|
||||
Pipes read(
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & /*metadata_snapshot*/,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
|
@ -352,9 +352,9 @@ size_t IMergeTreeDataPart::getFileSizeOrZero(const String & file_name) const
|
||||
return checksum->second.file_size;
|
||||
}
|
||||
|
||||
String IMergeTreeDataPart::getColumnNameWithMinumumCompressedSize() const
|
||||
String IMergeTreeDataPart::getColumnNameWithMinumumCompressedSize(const StorageMetadataPtr & metadata_snapshot) const
|
||||
{
|
||||
const auto & storage_columns = storage.getColumns().getAllPhysical();
|
||||
const auto & storage_columns = metadata_snapshot->getColumns().getAllPhysical();
|
||||
auto alter_conversions = storage.getAlterConversionsForPart(shared_from_this());
|
||||
|
||||
std::optional<std::string> minimum_size_column;
|
||||
@ -437,7 +437,8 @@ void IMergeTreeDataPart::loadIndex()
|
||||
if (!index_granularity.isInitialized())
|
||||
throw Exception("Index granularity is not loaded before index loading", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
const auto & primary_key = storage.getPrimaryKey();
|
||||
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
|
||||
const auto & primary_key = metadata_snapshot->getPrimaryKey();
|
||||
size_t key_size = primary_key.column_names.size();
|
||||
|
||||
if (key_size)
|
||||
@ -496,7 +497,8 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex()
|
||||
minmax_idx.load(storage, volume->getDisk(), path);
|
||||
}
|
||||
|
||||
String calculated_partition_id = partition.getID(storage.getPartitionKey().sample_block);
|
||||
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
|
||||
String calculated_partition_id = partition.getID(metadata_snapshot->getPartitionKey().sample_block);
|
||||
if (calculated_partition_id != info.partition_id)
|
||||
throw Exception(
|
||||
"While loading part " + getFullPath() + ": calculated partition ID: " + calculated_partition_id
|
||||
@ -611,6 +613,7 @@ void IMergeTreeDataPart::loadTTLInfos()
|
||||
void IMergeTreeDataPart::loadColumns(bool require)
|
||||
{
|
||||
String path = getFullRelativePath() + "columns.txt";
|
||||
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
|
||||
if (!volume->getDisk()->exists(path))
|
||||
{
|
||||
/// We can get list of columns only from columns.txt in compact parts.
|
||||
@ -618,7 +621,7 @@ void IMergeTreeDataPart::loadColumns(bool require)
|
||||
throw Exception("No columns.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
/// If there is no file with a list of columns, write it down.
|
||||
for (const NameAndTypePair & column : storage.getColumns().getAllPhysical())
|
||||
for (const NameAndTypePair & column : metadata_snapshot->getColumns().getAllPhysical())
|
||||
if (volume->getDisk()->exists(getFullRelativePath() + getFileNameForColumn(column) + ".bin"))
|
||||
columns.push_back(column);
|
||||
|
||||
@ -840,7 +843,8 @@ void IMergeTreeDataPart::checkConsistencyBase() const
|
||||
{
|
||||
String path = getFullRelativePath();
|
||||
|
||||
const auto & pk = storage.getPrimaryKey();
|
||||
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
|
||||
const auto & pk = metadata_snapshot->getPrimaryKey();
|
||||
if (!checksums.empty())
|
||||
{
|
||||
if (!pk.column_names.empty() && !checksums.files.count("primary.idx"))
|
||||
@ -851,7 +855,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const
|
||||
if (!checksums.files.count("count.txt"))
|
||||
throw Exception("No checksum for count.txt", ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
if (storage.hasPartitionKey() && !checksums.files.count("partition.dat"))
|
||||
if (metadata_snapshot->hasPartitionKey() && !checksums.files.count("partition.dat"))
|
||||
throw Exception("No checksum for partition.dat", ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
if (!isEmpty())
|
||||
@ -884,7 +888,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const
|
||||
{
|
||||
check_file_not_empty(volume->getDisk(), path + "count.txt");
|
||||
|
||||
if (storage.hasPartitionKey())
|
||||
if (metadata_snapshot->hasPartitionKey())
|
||||
check_file_not_empty(volume->getDisk(), path + "partition.dat");
|
||||
|
||||
for (const String & col_name : storage.minmax_idx_columns)
|
||||
|
@ -77,6 +77,7 @@ public:
|
||||
|
||||
virtual MergeTreeReaderPtr getReader(
|
||||
const NamesAndTypesList & columns_,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MarkRanges & mark_ranges,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
MarkCache * mark_cache,
|
||||
@ -86,6 +87,7 @@ public:
|
||||
|
||||
virtual MergeTreeWriterPtr getWriter(
|
||||
const NamesAndTypesList & columns_list,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
|
||||
const CompressionCodecPtr & default_codec_,
|
||||
const MergeTreeWriterSettings & writer_settings,
|
||||
@ -142,7 +144,7 @@ public:
|
||||
|
||||
/// Returns the name of a column with minimum compressed size (as returned by getColumnSize()).
|
||||
/// If no checksums are present returns the name of the first physically existing column.
|
||||
String getColumnNameWithMinumumCompressedSize() const;
|
||||
String getColumnNameWithMinumumCompressedSize(const StorageMetadataPtr & metadata_snapshot) const;
|
||||
|
||||
bool contains(const IMergeTreeDataPart & other) const { return info.contains(other.info); }
|
||||
|
||||
|
@ -65,6 +65,7 @@ void IMergeTreeDataPartWriter::Stream::addToChecksums(MergeTreeData::DataPart::C
|
||||
IMergeTreeDataPartWriter::IMergeTreeDataPartWriter(
|
||||
const MergeTreeData::DataPartPtr & data_part_,
|
||||
const NamesAndTypesList & columns_list_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const std::vector<MergeTreeIndexPtr> & indices_to_recalc_,
|
||||
const String & marks_file_extension_,
|
||||
const CompressionCodecPtr & default_codec_,
|
||||
@ -73,6 +74,7 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter(
|
||||
: data_part(data_part_)
|
||||
, part_path(data_part_->getFullRelativePath())
|
||||
, storage(data_part_->storage)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, columns_list(columns_list_)
|
||||
, marks_file_extension(marks_file_extension_)
|
||||
, index_granularity(index_granularity_)
|
||||
@ -162,7 +164,7 @@ void IMergeTreeDataPartWriter::fillIndexGranularity(size_t index_granularity_for
|
||||
|
||||
void IMergeTreeDataPartWriter::initPrimaryIndex()
|
||||
{
|
||||
if (storage.hasPrimaryKey())
|
||||
if (metadata_snapshot->hasPrimaryKey())
|
||||
{
|
||||
index_file_stream = data_part->volume->getDisk()->writeFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite);
|
||||
index_stream = std::make_unique<HashingWriteBuffer>(*index_file_stream);
|
||||
@ -221,7 +223,7 @@ void IMergeTreeDataPartWriter::calculateAndSerializePrimaryIndex(const Block & p
|
||||
|
||||
while (index_mark < total_marks && current_row < rows)
|
||||
{
|
||||
if (storage.hasPrimaryKey())
|
||||
if (metadata_snapshot->hasPrimaryKey())
|
||||
{
|
||||
for (size_t j = 0; j < primary_columns_num; ++j)
|
||||
{
|
||||
|
@ -63,6 +63,7 @@ public:
|
||||
IMergeTreeDataPartWriter(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const NamesAndTypesList & columns_list,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
|
||||
const String & marks_file_extension,
|
||||
const CompressionCodecPtr & default_codec,
|
||||
@ -119,6 +120,7 @@ protected:
|
||||
MergeTreeData::DataPartPtr data_part;
|
||||
String part_path;
|
||||
const MergeTreeData & storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
NamesAndTypesList columns_list;
|
||||
const String marks_file_extension;
|
||||
|
||||
|
@ -22,13 +22,23 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
IMergeTreeReader::IMergeTreeReader(const MergeTreeData::DataPartPtr & data_part_,
|
||||
const NamesAndTypesList & columns_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_,
|
||||
const MarkRanges & all_mark_ranges_, const MergeTreeReaderSettings & settings_,
|
||||
IMergeTreeReader::IMergeTreeReader(
|
||||
const MergeTreeData::DataPartPtr & data_part_,
|
||||
const NamesAndTypesList & columns_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
UncompressedCache * uncompressed_cache_,
|
||||
MarkCache * mark_cache_,
|
||||
const MarkRanges & all_mark_ranges_,
|
||||
const MergeTreeReaderSettings & settings_,
|
||||
const ValueSizeMap & avg_value_size_hints_)
|
||||
: data_part(data_part_), avg_value_size_hints(avg_value_size_hints_)
|
||||
, columns(columns_), uncompressed_cache(uncompressed_cache_), mark_cache(mark_cache_)
|
||||
, settings(settings_), storage(data_part_->storage)
|
||||
: data_part(data_part_)
|
||||
, avg_value_size_hints(avg_value_size_hints_)
|
||||
, columns(columns_)
|
||||
, uncompressed_cache(uncompressed_cache_)
|
||||
, mark_cache(mark_cache_)
|
||||
, settings(settings_)
|
||||
, storage(data_part_->storage)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, all_mark_ranges(all_mark_ranges_)
|
||||
, alter_conversions(storage.getAlterConversionsForPart(data_part))
|
||||
{
|
||||
@ -112,7 +122,7 @@ void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_e
|
||||
|
||||
if (res_columns[i] == nullptr)
|
||||
{
|
||||
if (storage.getColumns().hasDefault(name))
|
||||
if (metadata_snapshot->getColumns().hasDefault(name))
|
||||
{
|
||||
should_evaluate_missing_defaults = true;
|
||||
continue;
|
||||
@ -170,7 +180,7 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns
|
||||
additional_columns.insert({res_columns[pos], name_and_type->type, name_and_type->name});
|
||||
}
|
||||
|
||||
DB::evaluateMissingDefaults(additional_columns, columns, storage.getColumns().getDefaults(), storage.global_context);
|
||||
DB::evaluateMissingDefaults(additional_columns, columns, metadata_snapshot->getColumns().getDefaults(), storage.global_context);
|
||||
|
||||
/// Move columns from block.
|
||||
name_and_type = columns.begin();
|
||||
|
@ -18,8 +18,10 @@ public:
|
||||
using ValueSizeMap = std::map<std::string, double>;
|
||||
using DeserializeBinaryBulkStateMap = std::map<std::string, IDataType::DeserializeBinaryBulkStatePtr>;
|
||||
|
||||
IMergeTreeReader(const MergeTreeData::DataPartPtr & data_part_,
|
||||
IMergeTreeReader(
|
||||
const MergeTreeData::DataPartPtr & data_part_,
|
||||
const NamesAndTypesList & columns_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
UncompressedCache * uncompressed_cache_,
|
||||
MarkCache * mark_cache_,
|
||||
const MarkRanges & all_mark_ranges_,
|
||||
@ -75,6 +77,7 @@ protected:
|
||||
MergeTreeReaderSettings settings;
|
||||
|
||||
const MergeTreeData & storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
MarkRanges all_mark_ranges;
|
||||
|
||||
friend class MergeTreeRangeReader::DelayedStream;
|
||||
|
@ -5,10 +5,11 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
IMergedBlockOutputStream::IMergedBlockOutputStream(
|
||||
const MergeTreeDataPartPtr & data_part)
|
||||
const MergeTreeDataPartPtr & data_part,
|
||||
const StorageMetadataPtr & metadata_snapshot_)
|
||||
: storage(data_part->storage)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, volume(data_part->volume)
|
||||
, part_path(data_part->getFullRelativePath())
|
||||
{
|
||||
|
@ -13,7 +13,8 @@ class IMergedBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
IMergedBlockOutputStream(
|
||||
const MergeTreeDataPartPtr & data_part);
|
||||
const MergeTreeDataPartPtr & data_part,
|
||||
const StorageMetadataPtr & metadata_snapshot_);
|
||||
|
||||
using WrittenOffsetColumns = std::set<std::string>;
|
||||
|
||||
@ -36,6 +37,7 @@ protected:
|
||||
|
||||
protected:
|
||||
const MergeTreeData & storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
|
||||
VolumePtr volume;
|
||||
String part_path;
|
||||
|
@ -20,6 +20,7 @@ namespace ErrorCodes
|
||||
MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor(
|
||||
Block header,
|
||||
const MergeTreeData & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
UInt64 max_block_size_rows_,
|
||||
UInt64 preferred_block_size_bytes_,
|
||||
@ -27,16 +28,16 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor(
|
||||
const MergeTreeReaderSettings & reader_settings_,
|
||||
bool use_uncompressed_cache_,
|
||||
const Names & virt_column_names_)
|
||||
:
|
||||
SourceWithProgress(getHeader(std::move(header), prewhere_info_, virt_column_names_)),
|
||||
storage(storage_),
|
||||
prewhere_info(prewhere_info_),
|
||||
max_block_size_rows(max_block_size_rows_),
|
||||
preferred_block_size_bytes(preferred_block_size_bytes_),
|
||||
preferred_max_column_in_block_size_bytes(preferred_max_column_in_block_size_bytes_),
|
||||
reader_settings(reader_settings_),
|
||||
use_uncompressed_cache(use_uncompressed_cache_),
|
||||
virt_column_names(virt_column_names_)
|
||||
: SourceWithProgress(getHeader(std::move(header), prewhere_info_, virt_column_names_))
|
||||
, storage(storage_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, prewhere_info(prewhere_info_)
|
||||
, max_block_size_rows(max_block_size_rows_)
|
||||
, preferred_block_size_bytes(preferred_block_size_bytes_)
|
||||
, preferred_max_column_in_block_size_bytes(preferred_max_column_in_block_size_bytes_)
|
||||
, reader_settings(reader_settings_)
|
||||
, use_uncompressed_cache(use_uncompressed_cache_)
|
||||
, virt_column_names(virt_column_names_)
|
||||
{
|
||||
header_without_virtual_columns = getPort().getHeader();
|
||||
|
||||
|
@ -22,6 +22,7 @@ public:
|
||||
MergeTreeBaseSelectProcessor(
|
||||
Block header,
|
||||
const MergeTreeData & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
UInt64 max_block_size_rows_,
|
||||
UInt64 preferred_block_size_bytes_,
|
||||
@ -54,6 +55,7 @@ protected:
|
||||
|
||||
protected:
|
||||
const MergeTreeData & storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
|
||||
PrewhereInfoPtr prewhere_info;
|
||||
|
||||
|
@ -8,7 +8,7 @@ namespace DB
|
||||
|
||||
Block MergeTreeBlockOutputStream::getHeader() const
|
||||
{
|
||||
return storage.getSampleBlock();
|
||||
return metadata_snapshot->getSampleBlock();
|
||||
}
|
||||
|
||||
|
||||
@ -16,12 +16,12 @@ void MergeTreeBlockOutputStream::write(const Block & block)
|
||||
{
|
||||
storage.delayInsertOrThrowIfNeeded();
|
||||
|
||||
auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block);
|
||||
auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot);
|
||||
for (auto & current_block : part_blocks)
|
||||
{
|
||||
Stopwatch watch;
|
||||
|
||||
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block);
|
||||
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, metadata_snapshot);
|
||||
storage.renameTempPartAndAdd(part, &storage.increment);
|
||||
|
||||
PartLog::addNewPart(storage.global_context, part, watch.elapsed());
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -13,14 +14,22 @@ class StorageMergeTree;
|
||||
class MergeTreeBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
MergeTreeBlockOutputStream(StorageMergeTree & storage_, size_t max_parts_per_block_)
|
||||
: storage(storage_), max_parts_per_block(max_parts_per_block_) {}
|
||||
MergeTreeBlockOutputStream(
|
||||
StorageMergeTree & storage_,
|
||||
const StorageMetadataPtr metadata_snapshot_,
|
||||
size_t max_parts_per_block_)
|
||||
: storage(storage_)
|
||||
, metadata_snapshot(metadata_snapshot_)
|
||||
, max_parts_per_block(max_parts_per_block_)
|
||||
{
|
||||
}
|
||||
|
||||
Block getHeader() const override;
|
||||
void write(const Block & block) override;
|
||||
|
||||
private:
|
||||
StorageMergeTree & storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
size_t max_parts_per_block;
|
||||
};
|
||||
|
||||
|
@ -13,14 +13,14 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
NameSet injectRequiredColumns(const MergeTreeData & storage, const MergeTreeData::DataPartPtr & part, Names & columns)
|
||||
NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData::DataPartPtr & part, Names & columns)
|
||||
{
|
||||
NameSet required_columns{std::begin(columns), std::end(columns)};
|
||||
NameSet injected_columns;
|
||||
|
||||
auto all_column_files_missing = true;
|
||||
|
||||
const auto & storage_columns = storage.getColumns();
|
||||
const auto & storage_columns = metadata_snapshot->getColumns();
|
||||
auto alter_conversions = storage.getAlterConversionsForPart(part);
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
@ -66,7 +66,7 @@ NameSet injectRequiredColumns(const MergeTreeData & storage, const MergeTreeData
|
||||
*/
|
||||
if (all_column_files_missing)
|
||||
{
|
||||
const auto minimum_size_column_name = part->getColumnNameWithMinumumCompressedSize();
|
||||
const auto minimum_size_column_name = part->getColumnNameWithMinumumCompressedSize(metadata_snapshot);
|
||||
columns.push_back(minimum_size_column_name);
|
||||
/// correctly report added column
|
||||
injected_columns.insert(columns.back());
|
||||
@ -214,14 +214,19 @@ void MergeTreeBlockSizePredictor::update(const Block & sample_block, const Colum
|
||||
}
|
||||
|
||||
|
||||
MergeTreeReadTaskColumns getReadTaskColumns(const MergeTreeData & storage, const MergeTreeData::DataPartPtr & data_part,
|
||||
const Names & required_columns, const PrewhereInfoPtr & prewhere_info, bool check_columns)
|
||||
MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
const MergeTreeData & storage,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const Names & required_columns,
|
||||
const PrewhereInfoPtr & prewhere_info,
|
||||
bool check_columns)
|
||||
{
|
||||
Names column_names = required_columns;
|
||||
Names pre_column_names;
|
||||
|
||||
/// inject columns required for defaults evaluation
|
||||
bool should_reorder = !injectRequiredColumns(storage, data_part, column_names).empty();
|
||||
bool should_reorder = !injectRequiredColumns(storage, metadata_snapshot, data_part, column_names).empty();
|
||||
|
||||
if (prewhere_info)
|
||||
{
|
||||
@ -233,7 +238,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(const MergeTreeData & storage, const
|
||||
if (pre_column_names.empty())
|
||||
pre_column_names.push_back(column_names[0]);
|
||||
|
||||
const auto injected_pre_columns = injectRequiredColumns(storage, data_part, pre_column_names);
|
||||
const auto injected_pre_columns = injectRequiredColumns(storage, metadata_snapshot, data_part, pre_column_names);
|
||||
if (!injected_pre_columns.empty())
|
||||
should_reorder = true;
|
||||
|
||||
@ -251,7 +256,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(const MergeTreeData & storage, const
|
||||
|
||||
if (check_columns)
|
||||
{
|
||||
const NamesAndTypesList & physical_columns = storage.getColumns().getAllPhysical();
|
||||
const NamesAndTypesList & physical_columns = metadata_snapshot->getColumns().getAllPhysical();
|
||||
result.pre_columns = physical_columns.addTypes(pre_column_names);
|
||||
result.columns = physical_columns.addTypes(column_names);
|
||||
}
|
||||
|
@ -22,7 +22,7 @@ using MergeTreeBlockSizePredictorPtr = std::unique_ptr<MergeTreeBlockSizePredict
|
||||
* so that you can calculate the DEFAULT expression for these columns.
|
||||
* Adds them to the `columns`.
|
||||
*/
|
||||
NameSet injectRequiredColumns(const MergeTreeData & storage, const MergeTreeData::DataPartPtr & part, Names & columns);
|
||||
NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData::DataPartPtr & part, Names & columns);
|
||||
|
||||
|
||||
/// A batch of work for MergeTreeThreadSelectBlockInputStream
|
||||
@ -73,8 +73,13 @@ struct MergeTreeReadTaskColumns
|
||||
bool should_reorder;
|
||||
};
|
||||
|
||||
MergeTreeReadTaskColumns getReadTaskColumns(const MergeTreeData & storage, const MergeTreeData::DataPartPtr & data_part,
|
||||
const Names & required_columns, const PrewhereInfoPtr & prewhere_info, bool check_columns);
|
||||
MergeTreeReadTaskColumns getReadTaskColumns(
|
||||
const MergeTreeData & storage,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const Names & required_columns,
|
||||
const PrewhereInfoPtr & prewhere_info,
|
||||
bool check_columns);
|
||||
|
||||
struct MergeTreeBlockSizePredictor
|
||||
{
|
||||
|
@ -142,32 +142,12 @@ MergeTreeData::MergeTreeData(
|
||||
if (relative_data_path.empty())
|
||||
throw Exception("MergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||
|
||||
setSettingsChanges(metadata_.settings_changes);
|
||||
const auto settings = getSettings();
|
||||
setProperties(metadata_, attach);
|
||||
|
||||
/// NOTE: using the same columns list as is read when performing actual merges.
|
||||
merging_params.check(getColumns().getAllPhysical());
|
||||
|
||||
if (metadata_.sampling_key.definition_ast != nullptr)
|
||||
{
|
||||
const auto & pk_sample_block = getPrimaryKey().sample_block;
|
||||
if (!pk_sample_block.has(metadata_.sampling_key.column_names[0]) && !attach
|
||||
&& !settings->compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility.
|
||||
throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
setSamplingKey(metadata_.sampling_key);
|
||||
}
|
||||
|
||||
MergeTreeDataFormatVersion min_format_version(0);
|
||||
if (!date_column_name.empty())
|
||||
{
|
||||
try
|
||||
{
|
||||
auto partition_by_ast = makeASTFunction("toYYYYMM", std::make_shared<ASTIdentifier>(date_column_name));
|
||||
auto partition_key = KeyDescription::getKeyFromAST(partition_by_ast, getColumns(), global_context);
|
||||
initPartitionKey(partition_key);
|
||||
|
||||
checkPartitionKeyAndInitMinMax(metadata_.partition_key);
|
||||
if (minmax_idx_date_column_pos == -1)
|
||||
throw Exception("Could not find Date column", ErrorCodes::BAD_TYPE_OF_FIELD);
|
||||
}
|
||||
@ -181,11 +161,26 @@ MergeTreeData::MergeTreeData(
|
||||
else
|
||||
{
|
||||
is_custom_partitioned = true;
|
||||
initPartitionKey(metadata_.partition_key);
|
||||
checkPartitionKeyAndInitMinMax(metadata_.partition_key);
|
||||
min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING;
|
||||
}
|
||||
|
||||
setTTLExpressions(metadata_);
|
||||
setProperties(metadata_, metadata_, attach);
|
||||
const auto settings = getSettings();
|
||||
|
||||
/// NOTE: using the same columns list as is read when performing actual merges.
|
||||
merging_params.check(metadata_.getColumns().getAllPhysical());
|
||||
|
||||
if (metadata_.sampling_key.definition_ast != nullptr)
|
||||
{
|
||||
const auto & pk_sample_block = metadata_.getPrimaryKey().sample_block;
|
||||
if (!pk_sample_block.has(metadata_.sampling_key.column_names[0]) && !attach
|
||||
&& !settings->compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility.
|
||||
throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
|
||||
checkTTLExpressions(metadata_, metadata_);
|
||||
|
||||
/// format_file always contained on any data path
|
||||
PathWithDisk version_file;
|
||||
@ -275,7 +270,7 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata, bool attach) const
|
||||
void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach) const
|
||||
{
|
||||
if (!new_metadata.sorting_key.definition_ast)
|
||||
throw Exception("ORDER BY cannot be empty", ErrorCodes::BAD_ARGUMENTS);
|
||||
@ -313,7 +308,7 @@ void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata
|
||||
auto all_columns = new_metadata.columns.getAllPhysical();
|
||||
|
||||
/// Order by check AST
|
||||
if (hasSortingKey())
|
||||
if (old_metadata.hasSortingKey())
|
||||
{
|
||||
/// This is ALTER, not CREATE/ATTACH TABLE. Let us check that all new columns used in the sorting key
|
||||
/// expression have just been added (so that the sorting order is guaranteed to be valid with the new key).
|
||||
@ -322,7 +317,7 @@ void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata
|
||||
Names new_sorting_key_columns = new_sorting_key.column_names;
|
||||
|
||||
ASTPtr added_key_column_expr_list = std::make_shared<ASTExpressionList>();
|
||||
const auto & old_sorting_key_columns = getSortingKeyColumns();
|
||||
const auto & old_sorting_key_columns = old_metadata.getSortingKeyColumns();
|
||||
for (size_t new_i = 0, old_i = 0; new_i < sorting_key_size; ++new_i)
|
||||
{
|
||||
if (old_i < old_sorting_key_columns.size())
|
||||
@ -343,7 +338,7 @@ void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata
|
||||
|
||||
NamesAndTypesList deleted_columns;
|
||||
NamesAndTypesList added_columns;
|
||||
getColumns().getAllPhysical().getDifference(all_columns, deleted_columns, added_columns);
|
||||
old_metadata.getColumns().getAllPhysical().getDifference(all_columns, deleted_columns, added_columns);
|
||||
|
||||
for (const String & col : used_columns)
|
||||
{
|
||||
@ -382,16 +377,10 @@ void MergeTreeData::checkProperties(const StorageInMemoryMetadata & new_metadata
|
||||
|
||||
}
|
||||
|
||||
void MergeTreeData::setProperties(const StorageInMemoryMetadata & new_metadata, bool attach)
|
||||
void MergeTreeData::setProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach)
|
||||
{
|
||||
checkProperties(new_metadata, attach);
|
||||
|
||||
/// Other parts of metadata initialized is separate methods
|
||||
setColumns(std::move(new_metadata.columns));
|
||||
setSecondaryIndices(std::move(new_metadata.secondary_indices));
|
||||
setConstraints(std::move(new_metadata.constraints));
|
||||
setSortingKey(std::move(new_metadata.sorting_key));
|
||||
setPrimaryKey(std::move(new_metadata.primary_key));
|
||||
checkProperties(new_metadata, old_metadata, attach);
|
||||
setInMemoryMetadata(new_metadata);
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -415,18 +404,18 @@ ExpressionActionsPtr getCombinedIndicesExpression(
|
||||
|
||||
}
|
||||
|
||||
ExpressionActionsPtr MergeTreeData::getPrimaryKeyAndSkipIndicesExpression() const
|
||||
ExpressionActionsPtr MergeTreeData::getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const
|
||||
{
|
||||
return getCombinedIndicesExpression(getPrimaryKey(), getSecondaryIndices(), getColumns(), global_context);
|
||||
return getCombinedIndicesExpression(metadata_snapshot->getPrimaryKey(), metadata_snapshot->getSecondaryIndices(), metadata_snapshot->getColumns(), global_context);
|
||||
}
|
||||
|
||||
ExpressionActionsPtr MergeTreeData::getSortingKeyAndSkipIndicesExpression() const
|
||||
ExpressionActionsPtr MergeTreeData::getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const
|
||||
{
|
||||
return getCombinedIndicesExpression(getSortingKey(), getSecondaryIndices(), getColumns(), global_context);
|
||||
return getCombinedIndicesExpression(metadata_snapshot->getSortingKey(), metadata_snapshot->getSecondaryIndices(), metadata_snapshot->getColumns(), global_context);
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::initPartitionKey(const KeyDescription & new_partition_key)
|
||||
void MergeTreeData::checkPartitionKeyAndInitMinMax(const KeyDescription & new_partition_key)
|
||||
{
|
||||
if (new_partition_key.expression_list_ast->children.empty())
|
||||
return;
|
||||
@ -479,11 +468,10 @@ void MergeTreeData::initPartitionKey(const KeyDescription & new_partition_key)
|
||||
}
|
||||
}
|
||||
}
|
||||
setPartitionKey(new_partition_key);
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_metadata) const
|
||||
void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata) const
|
||||
{
|
||||
auto new_column_ttls = new_metadata.column_ttls_by_name;
|
||||
|
||||
@ -491,12 +479,12 @@ void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_meta
|
||||
{
|
||||
NameSet columns_ttl_forbidden;
|
||||
|
||||
if (hasPartitionKey())
|
||||
for (const auto & col : getColumnsRequiredForPartitionKey())
|
||||
if (old_metadata.hasPartitionKey())
|
||||
for (const auto & col : old_metadata.getColumnsRequiredForPartitionKey())
|
||||
columns_ttl_forbidden.insert(col);
|
||||
|
||||
if (hasSortingKey())
|
||||
for (const auto & col : getColumnsRequiredForSortingKey())
|
||||
if (old_metadata.hasSortingKey())
|
||||
for (const auto & col : old_metadata.getColumnsRequiredForSortingKey())
|
||||
columns_ttl_forbidden.insert(col);
|
||||
|
||||
for (const auto & [name, ttl_description] : new_column_ttls)
|
||||
@ -524,14 +512,6 @@ void MergeTreeData::checkTTLExpressions(const StorageInMemoryMetadata & new_meta
|
||||
}
|
||||
}
|
||||
|
||||
/// Todo replace columns with TTL for columns
|
||||
void MergeTreeData::setTTLExpressions(const StorageInMemoryMetadata & new_metadata)
|
||||
{
|
||||
checkTTLExpressions(new_metadata);
|
||||
setColumnTTLs(new_metadata.column_ttls_by_name);
|
||||
setTableTTLs(new_metadata.table_ttl);
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::checkStoragePolicy(const StoragePolicyPtr & new_storage_policy) const
|
||||
{
|
||||
@ -1245,9 +1225,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
{
|
||||
/// Check that needed transformations can be applied to the list of columns without considering type conversions.
|
||||
StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
|
||||
StorageInMemoryMetadata old_metadata = getInMemoryMetadata();
|
||||
commands.apply(new_metadata, global_context);
|
||||
if (getSecondaryIndices().empty() && !new_metadata.secondary_indices.empty() &&
|
||||
!settings.allow_experimental_data_skipping_indices)
|
||||
if (old_metadata.getSecondaryIndices().empty() && !new_metadata.secondary_indices.empty()
|
||||
&& !settings.allow_experimental_data_skipping_indices)
|
||||
throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \
|
||||
"before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
@ -1258,24 +1239,24 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
/// (and not as a part of some expression) and if the ALTER only affects column metadata.
|
||||
NameSet columns_alter_type_metadata_only;
|
||||
|
||||
if (hasPartitionKey())
|
||||
if (old_metadata.hasPartitionKey())
|
||||
{
|
||||
/// Forbid altering partition key columns because it can change partition ID format.
|
||||
/// TODO: in some cases (e.g. adding an Enum value) a partition key column can still be ALTERed.
|
||||
/// We should allow it.
|
||||
for (const String & col : getColumnsRequiredForPartitionKey())
|
||||
for (const String & col : old_metadata.getColumnsRequiredForPartitionKey())
|
||||
columns_alter_type_forbidden.insert(col);
|
||||
}
|
||||
|
||||
for (const auto & index : getSecondaryIndices())
|
||||
for (const auto & index : old_metadata.getSecondaryIndices())
|
||||
{
|
||||
for (const String & col : index.expression->getRequiredColumns())
|
||||
columns_alter_type_forbidden.insert(col);
|
||||
}
|
||||
|
||||
if (hasSortingKey())
|
||||
if (old_metadata.hasSortingKey())
|
||||
{
|
||||
auto sorting_key_expr = getSortingKey().expression;
|
||||
auto sorting_key_expr = old_metadata.getSortingKey().expression;
|
||||
for (const ExpressionAction & action : sorting_key_expr->getActions())
|
||||
{
|
||||
auto action_columns = action.getNeededColumns();
|
||||
@ -1291,7 +1272,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
columns_alter_type_forbidden.insert(merging_params.sign_column);
|
||||
|
||||
std::map<String, const IDataType *> old_types;
|
||||
for (const auto & column : getColumns().getAllPhysical())
|
||||
for (const auto & column : old_metadata.getColumns().getAllPhysical())
|
||||
old_types.emplace(column.name, column.type.get());
|
||||
|
||||
|
||||
@ -1336,14 +1317,14 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
}
|
||||
}
|
||||
|
||||
checkProperties(new_metadata);
|
||||
checkProperties(new_metadata, old_metadata);
|
||||
|
||||
checkTTLExpressions(new_metadata);
|
||||
checkTTLExpressions(new_metadata, old_metadata);
|
||||
|
||||
if (hasSettingsChanges())
|
||||
if (old_metadata.hasSettingsChanges())
|
||||
{
|
||||
|
||||
const auto current_changes = getSettingsChanges()->as<const ASTSetQuery &>().changes;
|
||||
const auto current_changes = old_metadata.getSettingsChanges()->as<const ASTSetQuery &>().changes;
|
||||
const auto & new_changes = new_metadata.settings_changes->as<const ASTSetQuery &>().changes;
|
||||
for (const auto & changed_setting : new_changes)
|
||||
{
|
||||
@ -1443,7 +1424,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createPart(
|
||||
|
||||
void MergeTreeData::changeSettings(
|
||||
const ASTPtr & new_settings,
|
||||
TableStructureWriteLockHolder & /* table_lock_holder */)
|
||||
TableLockHolder & /* table_lock_holder */)
|
||||
{
|
||||
if (new_settings)
|
||||
{
|
||||
@ -1482,11 +1463,13 @@ void MergeTreeData::changeSettings(
|
||||
MergeTreeSettings copy = *getSettings();
|
||||
copy.applyChanges(new_changes);
|
||||
storage_settings.set(std::make_unique<const MergeTreeSettings>(copy));
|
||||
setSettingsChanges(new_settings);
|
||||
StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
|
||||
new_metadata.setSettingsChanges(new_settings);
|
||||
setInMemoryMetadata(new_metadata);
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeData::freezeAll(const String & with_name, const Context & context, TableStructureReadLockHolder &)
|
||||
void MergeTreeData::freezeAll(const String & with_name, const Context & context, TableLockHolder &)
|
||||
{
|
||||
freezePartitionsByMatcher([] (const DataPartPtr &){ return true; }, with_name, context);
|
||||
}
|
||||
@ -2294,7 +2277,7 @@ void MergeTreeData::removePartContributionToColumnSizes(const DataPartPtr & part
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String & with_name, const Context & context, TableStructureReadLockHolder &)
|
||||
void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String & with_name, const Context & context, TableLockHolder &)
|
||||
{
|
||||
std::optional<String> prefix;
|
||||
String partition_id;
|
||||
@ -2455,7 +2438,8 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context
|
||||
|
||||
/// Re-parse partition key fields using the information about expected field types.
|
||||
|
||||
size_t fields_count = getPartitionKey().sample_block.columns();
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
size_t fields_count = metadata_snapshot->getPartitionKey().sample_block.columns();
|
||||
if (partition_ast.fields_count != fields_count)
|
||||
throw Exception(
|
||||
"Wrong number of fields in the partition expression: " + toString(partition_ast.fields_count) +
|
||||
@ -2472,7 +2456,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context
|
||||
ReadBufferFromMemory right_paren_buf(")", 1);
|
||||
ConcatReadBuffer buf({&left_paren_buf, &fields_buf, &right_paren_buf});
|
||||
|
||||
auto input_stream = FormatFactory::instance().getInput("Values", buf, getPartitionKey().sample_block, context, context.getSettingsRef().max_block_size);
|
||||
auto input_stream = FormatFactory::instance().getInput("Values", buf, metadata_snapshot->getPartitionKey().sample_block, context, context.getSettingsRef().max_block_size);
|
||||
|
||||
auto block = input_stream->read();
|
||||
if (!block || !block.rows())
|
||||
@ -2805,8 +2789,9 @@ MergeTreeData::selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & tt
|
||||
{
|
||||
time_t max_max_ttl = 0;
|
||||
TTLDescriptions::const_iterator best_entry_it;
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
|
||||
const auto & move_ttl_entries = getMoveTTLs();
|
||||
const auto & move_ttl_entries = metadata_snapshot->getMoveTTLs();
|
||||
for (auto ttl_entry_it = move_ttl_entries.begin(); ttl_entry_it != move_ttl_entries.end(); ++ttl_entry_it)
|
||||
{
|
||||
auto ttl_info_it = ttl_infos.moves_ttl.find(ttl_entry_it->result_column);
|
||||
@ -2919,11 +2904,12 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData:
|
||||
return total_covered_parts;
|
||||
}
|
||||
|
||||
bool MergeTreeData::isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const
|
||||
bool MergeTreeData::isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(
|
||||
const ASTPtr & node, const StorageMetadataPtr & metadata_snapshot) const
|
||||
{
|
||||
const String column_name = node->getColumnName();
|
||||
|
||||
for (const auto & name : getPrimaryKeyColumns())
|
||||
for (const auto & name : metadata_snapshot->getPrimaryKeyColumns())
|
||||
if (column_name == name)
|
||||
return true;
|
||||
|
||||
@ -2933,12 +2919,13 @@ bool MergeTreeData::isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const A
|
||||
|
||||
if (const auto * func = node->as<ASTFunction>())
|
||||
if (func->arguments->children.size() == 1)
|
||||
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(func->arguments->children.front());
|
||||
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(func->arguments->children.front(), metadata_snapshot);
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context &) const
|
||||
bool MergeTreeData::mayBenefitFromIndexForIn(
|
||||
const ASTPtr & left_in_operand, const Context &, const StorageMetadataPtr & metadata_snapshot) const
|
||||
{
|
||||
/// Make sure that the left side of the IN operator contain part of the key.
|
||||
/// If there is a tuple on the left side of the IN operator, at least one item of the tuple
|
||||
@ -2949,26 +2936,26 @@ bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, con
|
||||
{
|
||||
for (const auto & item : left_in_operand_tuple->arguments->children)
|
||||
{
|
||||
if (isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(item))
|
||||
if (isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(item, metadata_snapshot))
|
||||
return true;
|
||||
for (const auto & index : getSecondaryIndices())
|
||||
for (const auto & index : metadata_snapshot->getSecondaryIndices())
|
||||
if (index_wrapper_factory.get(index)->mayBenefitFromIndexForIn(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);
|
||||
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand, metadata_snapshot);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const auto & index : getSecondaryIndices())
|
||||
for (const auto & index : metadata_snapshot->getSecondaryIndices())
|
||||
if (index_wrapper_factory.get(index)->mayBenefitFromIndexForIn(left_in_operand))
|
||||
return true;
|
||||
|
||||
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand);
|
||||
return isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(left_in_operand, metadata_snapshot);
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & source_table) const
|
||||
MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const
|
||||
{
|
||||
MergeTreeData * src_data = dynamic_cast<MergeTreeData *>(&source_table);
|
||||
if (!src_data)
|
||||
@ -2976,7 +2963,7 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & sour
|
||||
" supports attachPartitionFrom only for MergeTree family of table engines."
|
||||
" Got " + source_table.getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
if (getColumns().getAllPhysical().sizeOfDifference(src_data->getColumns().getAllPhysical()))
|
||||
if (my_snapshot->getColumns().getAllPhysical().sizeOfDifference(src_snapshot->getColumns().getAllPhysical()))
|
||||
throw Exception("Tables have different structure", ErrorCodes::INCOMPATIBLE_COLUMNS);
|
||||
|
||||
auto query_to_string = [] (const ASTPtr & ast)
|
||||
@ -2984,10 +2971,10 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & sour
|
||||
return ast ? queryToString(ast) : "";
|
||||
};
|
||||
|
||||
if (query_to_string(getSortingKeyAST()) != query_to_string(src_data->getSortingKeyAST()))
|
||||
if (query_to_string(my_snapshot->getSortingKeyAST()) != query_to_string(src_snapshot->getSortingKeyAST()))
|
||||
throw Exception("Tables have different ordering", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (query_to_string(getPartitionKeyAST()) != query_to_string(src_data->getPartitionKeyAST()))
|
||||
if (query_to_string(my_snapshot->getPartitionKeyAST()) != query_to_string(src_snapshot->getPartitionKeyAST()))
|
||||
throw Exception("Tables have different partition key", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (format_version != src_data->format_version)
|
||||
@ -2996,9 +2983,10 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & sour
|
||||
return *src_data;
|
||||
}
|
||||
|
||||
MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(const StoragePtr & source_table) const
|
||||
MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(
|
||||
const StoragePtr & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const
|
||||
{
|
||||
return checkStructureAndGetMergeTreeData(*source_table);
|
||||
return checkStructureAndGetMergeTreeData(*source_table, src_snapshot, my_snapshot);
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(const MergeTreeData::DataPartPtr & src_part,
|
||||
@ -3236,11 +3224,12 @@ bool MergeTreeData::selectPartsAndMove()
|
||||
bool MergeTreeData::areBackgroundMovesNeeded() const
|
||||
{
|
||||
auto policy = getStoragePolicy();
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
|
||||
if (policy->getVolumes().size() > 1)
|
||||
return true;
|
||||
|
||||
return policy->getVolumes().size() == 1 && policy->getVolumes()[0]->getDisks().size() > 1 && hasAnyMoveTTL();
|
||||
return policy->getVolumes().size() == 1 && policy->getVolumes()[0]->getDisks().size() > 1 && metadata_snapshot->hasAnyMoveTTL();
|
||||
}
|
||||
|
||||
bool MergeTreeData::movePartsToSpace(const DataPartsVector & parts, SpacePtr space)
|
||||
|
@ -350,7 +350,8 @@ public:
|
||||
bool supportsSettings() const override { return true; }
|
||||
NamesAndTypesList getVirtuals() const override;
|
||||
|
||||
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context &) const override;
|
||||
bool
|
||||
mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context &, const StorageMetadataPtr & metadata_snapshot) const override;
|
||||
|
||||
/// Load the set of data parts from disk. Call once - immediately after the object is created.
|
||||
void loadDataParts(bool skip_sanity_checks);
|
||||
@ -476,7 +477,7 @@ public:
|
||||
|
||||
/// Delete all directories which names begin with "tmp"
|
||||
/// Set non-negative parameter value to override MergeTreeSettings temporary_directories_lifetime
|
||||
/// Must be called with locked lockStructureForShare().
|
||||
/// Must be called with locked lockForShare() because use relative_data_path.
|
||||
void clearOldTemporaryDirectories(ssize_t custom_directories_lifetime_seconds = -1);
|
||||
|
||||
/// After the call to dropAllData() no method can be called.
|
||||
@ -486,9 +487,9 @@ public:
|
||||
/// Drop data directories if they are empty. It is safe to call this method if table creation was unsuccessful.
|
||||
void dropIfEmpty();
|
||||
|
||||
/// Moves the entire data directory.
|
||||
/// Flushes the uncompressed blocks cache and the marks cache.
|
||||
/// Must be called with locked lockStructureForAlter().
|
||||
/// Moves the entire data directory. Flushes the uncompressed blocks cache
|
||||
/// and the marks cache. Must be called with locked lockExclusively()
|
||||
/// because changes relative_data_path.
|
||||
void rename(const String & new_table_path, const StorageID & new_table_id) override;
|
||||
|
||||
/// Check if the ALTER can be performed:
|
||||
@ -501,10 +502,10 @@ public:
|
||||
/// Change MergeTreeSettings
|
||||
void changeSettings(
|
||||
const ASTPtr & new_settings,
|
||||
TableStructureWriteLockHolder & table_lock_holder);
|
||||
TableLockHolder & table_lock_holder);
|
||||
|
||||
/// Freezes all parts.
|
||||
void freezeAll(const String & with_name, const Context & context, TableStructureReadLockHolder & table_lock_holder);
|
||||
void freezeAll(const String & with_name, const Context & context, TableLockHolder & table_lock_holder);
|
||||
|
||||
/// Should be called if part data is suspected to be corrupted.
|
||||
void reportBrokenPart(const String & name) const
|
||||
@ -526,7 +527,7 @@ public:
|
||||
* Backup is created in directory clickhouse_dir/shadow/i/, where i - incremental number,
|
||||
* or if 'with_name' is specified - backup is created in directory with specified name.
|
||||
*/
|
||||
void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context, TableStructureReadLockHolder & table_lock_holder);
|
||||
void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context, TableLockHolder & table_lock_holder);
|
||||
|
||||
|
||||
public:
|
||||
@ -555,8 +556,8 @@ public:
|
||||
/// Extracts MergeTreeData of other *MergeTree* storage
|
||||
/// and checks that their structure suitable for ALTER TABLE ATTACH PARTITION FROM
|
||||
/// Tables structure should be locked.
|
||||
MergeTreeData & checkStructureAndGetMergeTreeData(const StoragePtr & source_table) const;
|
||||
MergeTreeData & checkStructureAndGetMergeTreeData(IStorage & source_table) const;
|
||||
MergeTreeData & checkStructureAndGetMergeTreeData(const StoragePtr & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const;
|
||||
MergeTreeData & checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const;
|
||||
|
||||
MergeTreeData::MutableDataPartPtr cloneAndLoadDataPartOnSameDisk(
|
||||
const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info);
|
||||
@ -606,14 +607,18 @@ public:
|
||||
static ReservationPtr tryReserveSpace(UInt64 expected_size, SpacePtr space);
|
||||
|
||||
/// Reserves space at least 1MB preferring best destination according to `ttl_infos`.
|
||||
ReservationPtr reserveSpacePreferringTTLRules(UInt64 expected_size,
|
||||
const IMergeTreeDataPart::TTLInfos & ttl_infos,
|
||||
time_t time_of_move,
|
||||
size_t min_volume_index = 0) const;
|
||||
ReservationPtr tryReserveSpacePreferringTTLRules(UInt64 expected_size,
|
||||
const IMergeTreeDataPart::TTLInfos & ttl_infos,
|
||||
time_t time_of_move,
|
||||
size_t min_volume_index = 0) const;
|
||||
ReservationPtr reserveSpacePreferringTTLRules(
|
||||
UInt64 expected_size,
|
||||
const IMergeTreeDataPart::TTLInfos & ttl_infos,
|
||||
time_t time_of_move,
|
||||
size_t min_volume_index = 0) const;
|
||||
|
||||
ReservationPtr tryReserveSpacePreferringTTLRules(
|
||||
UInt64 expected_size,
|
||||
const IMergeTreeDataPart::TTLInfos & ttl_infos,
|
||||
time_t time_of_move,
|
||||
size_t min_volume_index = 0) const;
|
||||
|
||||
/// Choose disk with max available free space
|
||||
/// Reserves 0 bytes
|
||||
ReservationPtr makeEmptyReservationOnLargestDisk() { return getStoragePolicy()->makeEmptyReservationOnLargestDisk(); }
|
||||
@ -643,8 +648,8 @@ public:
|
||||
Int64 minmax_idx_date_column_pos = -1; /// In a common case minmax index includes a date column.
|
||||
Int64 minmax_idx_time_column_pos = -1; /// In other cases, minmax index often includes a dateTime column.
|
||||
|
||||
ExpressionActionsPtr getPrimaryKeyAndSkipIndicesExpression() const;
|
||||
ExpressionActionsPtr getSortingKeyAndSkipIndicesExpression() const;
|
||||
ExpressionActionsPtr getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const;
|
||||
ExpressionActionsPtr getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const;
|
||||
|
||||
std::optional<TTLDescription> selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const;
|
||||
|
||||
@ -780,14 +785,13 @@ protected:
|
||||
/// The same for clearOldTemporaryDirectories.
|
||||
std::mutex clear_old_temporary_directories_mutex;
|
||||
|
||||
void checkProperties(const StorageInMemoryMetadata & new_metadata, bool attach = false) const;
|
||||
void checkProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach = false) const;
|
||||
|
||||
void setProperties(const StorageInMemoryMetadata & new_metadata, bool attach = false);
|
||||
void setProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach = false);
|
||||
|
||||
void initPartitionKey(const KeyDescription & new_partition_key);
|
||||
void checkPartitionKeyAndInitMinMax(const KeyDescription & new_partition_key);
|
||||
|
||||
void checkTTLExpressions(const StorageInMemoryMetadata & new_metadata) const;
|
||||
void setTTLExpressions(const StorageInMemoryMetadata & new_metadata);
|
||||
void checkTTLExpressions(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata) const;
|
||||
|
||||
void checkStoragePolicy(const StoragePolicyPtr & new_storage_policy) const;
|
||||
|
||||
@ -811,7 +815,7 @@ protected:
|
||||
DataPartsLock & data_parts_lock) const;
|
||||
|
||||
/// Checks whether the column is in the primary key, possibly wrapped in a chain of functions with single argument.
|
||||
bool isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const;
|
||||
bool isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node, const StorageMetadataPtr & metadata_snapshot) const;
|
||||
|
||||
/// Common part for |freezePartition()| and |freezeAll()|.
|
||||
using MatcherFn = std::function<bool(const DataPartPtr &)>;
|
||||
|
@ -578,8 +578,14 @@ public:
|
||||
|
||||
/// parts should be sorted.
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart(
|
||||
const FutureMergedMutatedPart & future_part, MergeList::Entry & merge_entry, TableStructureReadLockHolder &,
|
||||
time_t time_of_merge, const ReservationPtr & space_reservation, bool deduplicate, bool force_ttl)
|
||||
const FutureMergedMutatedPart & future_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
MergeList::Entry & merge_entry,
|
||||
TableLockHolder &,
|
||||
time_t time_of_merge,
|
||||
const ReservationPtr & space_reservation,
|
||||
bool deduplicate,
|
||||
bool force_ttl)
|
||||
{
|
||||
static const String TMP_PREFIX = "tmp_merge_";
|
||||
|
||||
@ -598,16 +604,22 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
|
||||
MergeTreeData::DataPart::ColumnToSize merged_column_to_size;
|
||||
|
||||
Names all_column_names = data.getColumns().getNamesOfPhysical();
|
||||
NamesAndTypesList storage_columns = data.getColumns().getAllPhysical();
|
||||
Names all_column_names = metadata_snapshot->getColumns().getNamesOfPhysical();
|
||||
NamesAndTypesList storage_columns = metadata_snapshot->getColumns().getAllPhysical();
|
||||
const auto data_settings = data.getSettings();
|
||||
|
||||
NamesAndTypesList gathering_columns;
|
||||
NamesAndTypesList merging_columns;
|
||||
Names gathering_column_names, merging_column_names;
|
||||
extractMergingAndGatheringColumns(
|
||||
storage_columns, data.getSortingKey().expression, data.getSecondaryIndices(),
|
||||
data.merging_params, gathering_columns, gathering_column_names, merging_columns, merging_column_names);
|
||||
storage_columns,
|
||||
metadata_snapshot->getSortingKey().expression,
|
||||
metadata_snapshot->getSecondaryIndices(),
|
||||
data.merging_params,
|
||||
gathering_columns,
|
||||
gathering_column_names,
|
||||
merging_columns,
|
||||
merging_column_names);
|
||||
|
||||
auto single_disk_volume = std::make_shared<SingleDiskVolume>("volume_" + future_part.name, disk);
|
||||
MergeTreeData::MutableDataPartPtr new_data_part = data.createPart(
|
||||
@ -706,23 +718,23 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
auto input = std::make_unique<MergeTreeSequentialSource>(
|
||||
data, part, merging_column_names, read_with_direct_io, true);
|
||||
data, metadata_snapshot, part, merging_column_names, read_with_direct_io, true);
|
||||
|
||||
input->setProgressCallback(
|
||||
MergeProgressCallback(merge_entry, watch_prev_elapsed, horizontal_stage_progress));
|
||||
|
||||
Pipe pipe(std::move(input));
|
||||
|
||||
if (data.hasSortingKey())
|
||||
if (metadata_snapshot->hasSortingKey())
|
||||
{
|
||||
auto expr = std::make_shared<ExpressionTransform>(pipe.getHeader(), data.getSortingKey().expression);
|
||||
auto expr = std::make_shared<ExpressionTransform>(pipe.getHeader(), metadata_snapshot->getSortingKey().expression);
|
||||
pipe.addSimpleTransform(std::move(expr));
|
||||
}
|
||||
|
||||
pipes.emplace_back(std::move(pipe));
|
||||
}
|
||||
|
||||
Names sort_columns = data.getSortingKeyColumns();
|
||||
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
|
||||
SortDescription sort_description;
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
@ -789,21 +801,22 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
merged_stream = std::make_shared<DistinctSortedBlockInputStream>(merged_stream, SizeLimits(), 0 /*limit_hint*/, Names());
|
||||
|
||||
if (need_remove_expired_values)
|
||||
merged_stream = std::make_shared<TTLBlockInputStream>(merged_stream, data, new_data_part, time_of_merge, force_ttl);
|
||||
merged_stream = std::make_shared<TTLBlockInputStream>(merged_stream, data, metadata_snapshot, new_data_part, time_of_merge, force_ttl);
|
||||
|
||||
|
||||
if (data.hasSecondaryIndices())
|
||||
if (metadata_snapshot->hasSecondaryIndices())
|
||||
{
|
||||
const auto & indices = data.getSecondaryIndices();
|
||||
merged_stream = std::make_shared<ExpressionBlockInputStream>(merged_stream, indices.getSingleExpressionForIndices(data.getColumns(), data.global_context));
|
||||
const auto & indices = metadata_snapshot->getSecondaryIndices();
|
||||
merged_stream = std::make_shared<ExpressionBlockInputStream>(merged_stream, indices.getSingleExpressionForIndices(metadata_snapshot->getColumns(), data.global_context));
|
||||
merged_stream = std::make_shared<MaterializingBlockInputStream>(merged_stream);
|
||||
}
|
||||
|
||||
const auto & index_factory = MergeTreeIndexFactory::instance();
|
||||
MergedBlockOutputStream to{
|
||||
new_data_part,
|
||||
metadata_snapshot,
|
||||
merging_columns,
|
||||
index_factory.getMany(data.getSecondaryIndices()),
|
||||
index_factory.getMany(metadata_snapshot->getSecondaryIndices()),
|
||||
compression_codec,
|
||||
merged_column_to_size,
|
||||
data_settings->min_merge_bytes_to_use_direct_io,
|
||||
@ -892,7 +905,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
for (size_t part_num = 0; part_num < parts.size(); ++part_num)
|
||||
{
|
||||
auto column_part_source = std::make_shared<MergeTreeSequentialSource>(
|
||||
data, parts[part_num], column_names, read_with_direct_io, true);
|
||||
data, metadata_snapshot, parts[part_num], column_names, read_with_direct_io, true);
|
||||
|
||||
column_part_source->setProgressCallback(
|
||||
MergeProgressCallback(merge_entry, watch_prev_elapsed, column_progress));
|
||||
@ -906,6 +919,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
|
||||
MergedColumnOnlyOutputStream column_to(
|
||||
new_data_part,
|
||||
metadata_snapshot,
|
||||
column_gathered_stream.getHeader(),
|
||||
compression_codec,
|
||||
/// we don't need to recalc indices here
|
||||
@ -975,12 +989,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTemporaryPart(
|
||||
const FutureMergedMutatedPart & future_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MutationCommands & commands,
|
||||
MergeListEntry & merge_entry,
|
||||
time_t time_of_mutation,
|
||||
const Context & context,
|
||||
const ReservationPtr & space_reservation,
|
||||
TableStructureReadLockHolder & table_lock_holder)
|
||||
TableLockHolder &)
|
||||
{
|
||||
checkOperationIsNotCanceled(merge_entry);
|
||||
|
||||
@ -1004,7 +1019,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
commands_for_part.emplace_back(command);
|
||||
}
|
||||
|
||||
if (!isStorageTouchedByMutations(storage_from_source_part, commands_for_part, context_for_reading))
|
||||
if (!isStorageTouchedByMutations(storage_from_source_part, metadata_snapshot, commands_for_part, context_for_reading))
|
||||
{
|
||||
LOG_TRACE(log, "Part {} doesn't change up to mutation version {}", source_part->name, future_part.part_info.mutation);
|
||||
return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info);
|
||||
@ -1027,12 +1042,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
UInt64 watch_prev_elapsed = 0;
|
||||
MergeStageProgress stage_progress(1.0);
|
||||
|
||||
NamesAndTypesList storage_columns = data.getColumns().getAllPhysical();
|
||||
NamesAndTypesList storage_columns = metadata_snapshot->getColumns().getAllPhysical();
|
||||
|
||||
if (!for_interpreter.empty())
|
||||
{
|
||||
interpreter.emplace(storage_from_source_part, for_interpreter, context_for_reading, true);
|
||||
in = interpreter->execute(table_lock_holder);
|
||||
interpreter.emplace(storage_from_source_part, metadata_snapshot, for_interpreter, context_for_reading, true);
|
||||
in = interpreter->execute();
|
||||
updated_header = interpreter->getUpdatedHeader();
|
||||
in->setProgressCallback(MergeProgressCallback(merge_entry, watch_prev_elapsed, stage_progress));
|
||||
}
|
||||
@ -1069,15 +1084,16 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
|
||||
bool need_remove_expired_values = false;
|
||||
|
||||
if (in && shouldExecuteTTL(in->getHeader().getNamesAndTypesList().getNames(), commands_for_part))
|
||||
if (in && shouldExecuteTTL(metadata_snapshot, in->getHeader().getNamesAndTypesList().getNames(), commands_for_part))
|
||||
need_remove_expired_values = true;
|
||||
|
||||
/// All columns from part are changed and may be some more that were missing before in part
|
||||
if (isCompactPart(source_part) || source_part->getColumns().isSubsetOf(updated_header.getNamesAndTypesList()))
|
||||
{
|
||||
auto part_indices = getIndicesForNewDataPart(data.getSecondaryIndices(), for_file_renames);
|
||||
auto part_indices = getIndicesForNewDataPart(metadata_snapshot->getSecondaryIndices(), for_file_renames);
|
||||
mutateAllPartColumns(
|
||||
new_data_part,
|
||||
metadata_snapshot,
|
||||
part_indices,
|
||||
in,
|
||||
time_of_mutation,
|
||||
@ -1091,7 +1107,7 @@ 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(), context);
|
||||
auto indices_to_recalc = getIndicesToRecalculate(in, updated_header.getNamesAndTypesList(), metadata_snapshot, context);
|
||||
|
||||
NameSet files_to_skip = collectFilesToSkip(updated_header, indices_to_recalc, mrk_extension);
|
||||
NameToNameVector files_to_rename = collectFilesForRenames(source_part, for_file_renames, mrk_extension);
|
||||
@ -1130,6 +1146,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
{
|
||||
mutateSomePartColumns(
|
||||
source_part,
|
||||
metadata_snapshot,
|
||||
indices_to_recalc,
|
||||
updated_header,
|
||||
new_data_part,
|
||||
@ -1513,7 +1530,8 @@ MergeTreeIndices MergeTreeDataMergerMutator::getIndicesForNewDataPart(
|
||||
std::set<MergeTreeIndexPtr> MergeTreeDataMergerMutator::getIndicesToRecalculate(
|
||||
BlockInputStreamPtr & input_stream,
|
||||
const NamesAndTypesList & updated_columns,
|
||||
const Context & context) const
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const Context & context)
|
||||
{
|
||||
/// Checks if columns used in skipping indexes modified.
|
||||
const auto & index_factory = MergeTreeIndexFactory::instance();
|
||||
@ -1521,7 +1539,7 @@ std::set<MergeTreeIndexPtr> MergeTreeDataMergerMutator::getIndicesToRecalculate(
|
||||
ASTPtr indices_recalc_expr_list = std::make_shared<ASTExpressionList>();
|
||||
for (const auto & col : updated_columns.getNames())
|
||||
{
|
||||
const auto & indices = data.getSecondaryIndices();
|
||||
const auto & indices = metadata_snapshot->getSecondaryIndices();
|
||||
for (size_t i = 0; i < indices.size(); ++i)
|
||||
{
|
||||
const auto & index = indices[i];
|
||||
@ -1556,16 +1574,16 @@ std::set<MergeTreeIndexPtr> MergeTreeDataMergerMutator::getIndicesToRecalculate(
|
||||
return indices_to_recalc;
|
||||
}
|
||||
|
||||
bool MergeTreeDataMergerMutator::shouldExecuteTTL(const Names & columns, const MutationCommands & commands) const
|
||||
bool MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const Names & columns, const MutationCommands & commands)
|
||||
{
|
||||
if (!data.hasAnyTTL())
|
||||
if (!metadata_snapshot->hasAnyTTL())
|
||||
return false;
|
||||
|
||||
for (const auto & command : commands)
|
||||
if (command.type == MutationCommand::MATERIALIZE_TTL)
|
||||
return true;
|
||||
|
||||
auto dependencies = data.getColumnDependencies(NameSet(columns.begin(), columns.end()));
|
||||
auto dependencies = metadata_snapshot->getColumnDependencies(NameSet(columns.begin(), columns.end()));
|
||||
for (const auto & dependency : dependencies)
|
||||
if (dependency.kind == ColumnDependency::TTL_EXPRESSION || dependency.kind == ColumnDependency::TTL_TARGET)
|
||||
return true;
|
||||
@ -1575,6 +1593,7 @@ bool MergeTreeDataMergerMutator::shouldExecuteTTL(const Names & columns, const M
|
||||
|
||||
void MergeTreeDataMergerMutator::mutateAllPartColumns(
|
||||
MergeTreeData::MutableDataPartPtr new_data_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeIndices & skip_indices,
|
||||
BlockInputStreamPtr mutating_stream,
|
||||
time_t time_of_mutation,
|
||||
@ -1585,17 +1604,18 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns(
|
||||
if (mutating_stream == nullptr)
|
||||
throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (data.hasPrimaryKey() || data.hasSecondaryIndices())
|
||||
if (metadata_snapshot->hasPrimaryKey() || metadata_snapshot->hasSecondaryIndices())
|
||||
mutating_stream = std::make_shared<MaterializingBlockInputStream>(
|
||||
std::make_shared<ExpressionBlockInputStream>(mutating_stream, data.getPrimaryKeyAndSkipIndicesExpression()));
|
||||
std::make_shared<ExpressionBlockInputStream>(mutating_stream, data.getPrimaryKeyAndSkipIndicesExpression(metadata_snapshot)));
|
||||
|
||||
if (need_remove_expired_values)
|
||||
mutating_stream = std::make_shared<TTLBlockInputStream>(mutating_stream, data, new_data_part, time_of_mutation, true);
|
||||
mutating_stream = std::make_shared<TTLBlockInputStream>(mutating_stream, data, metadata_snapshot, new_data_part, time_of_mutation, true);
|
||||
|
||||
IMergeTreeDataPart::MinMaxIndex minmax_idx;
|
||||
|
||||
MergedBlockOutputStream out{
|
||||
new_data_part,
|
||||
metadata_snapshot,
|
||||
new_data_part->getColumns(),
|
||||
skip_indices,
|
||||
compression_codec};
|
||||
@ -1622,6 +1642,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns(
|
||||
|
||||
void MergeTreeDataMergerMutator::mutateSomePartColumns(
|
||||
const MergeTreeDataPartPtr & source_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const std::set<MergeTreeIndexPtr> & indices_to_recalc,
|
||||
const Block & mutation_header,
|
||||
MergeTreeData::MutableDataPartPtr new_data_part,
|
||||
@ -1635,11 +1656,12 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns(
|
||||
throw Exception("Cannot mutate part columns with uninitialized mutations stream. It's a bug", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (need_remove_expired_values)
|
||||
mutating_stream = std::make_shared<TTLBlockInputStream>(mutating_stream, data, new_data_part, time_of_mutation, true);
|
||||
mutating_stream = std::make_shared<TTLBlockInputStream>(mutating_stream, data, metadata_snapshot, new_data_part, time_of_mutation, true);
|
||||
|
||||
IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets;
|
||||
MergedColumnOnlyOutputStream out(
|
||||
new_data_part,
|
||||
metadata_snapshot,
|
||||
mutation_header,
|
||||
compression_codec,
|
||||
std::vector<MergeTreeIndexPtr>(indices_to_recalc.begin(), indices_to_recalc.end()),
|
||||
|
@ -105,18 +105,24 @@ public:
|
||||
*/
|
||||
MergeTreeData::MutableDataPartPtr mergePartsToTemporaryPart(
|
||||
const FutureMergedMutatedPart & future_part,
|
||||
MergeListEntry & merge_entry, TableStructureReadLockHolder & table_lock_holder, time_t time_of_merge,
|
||||
const ReservationPtr & space_reservation, bool deduplicate, bool force_ttl);
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
MergeListEntry & merge_entry,
|
||||
TableLockHolder & table_lock_holder,
|
||||
time_t time_of_merge,
|
||||
const ReservationPtr & space_reservation,
|
||||
bool deduplicate,
|
||||
bool force_ttl);
|
||||
|
||||
/// Mutate a single data part with the specified commands. Will create and return a temporary part.
|
||||
MergeTreeData::MutableDataPartPtr mutatePartToTemporaryPart(
|
||||
const FutureMergedMutatedPart & future_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MutationCommands & commands,
|
||||
MergeListEntry & merge_entry,
|
||||
time_t time_of_mutation,
|
||||
const Context & context,
|
||||
const ReservationPtr & space_reservation,
|
||||
TableStructureReadLockHolder & table_lock_holder);
|
||||
TableLockHolder & table_lock_holder);
|
||||
|
||||
MergeTreeData::DataPartPtr renameMergedTemporaryPart(
|
||||
MergeTreeData::MutableDataPartPtr & new_data_part,
|
||||
@ -164,18 +170,20 @@ private:
|
||||
const IndicesDescription & all_indices,
|
||||
const MutationCommands & commands_for_removes);
|
||||
|
||||
bool shouldExecuteTTL(const Names & columns, const MutationCommands & commands) const;
|
||||
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
|
||||
std::set<MergeTreeIndexPtr> getIndicesToRecalculate(
|
||||
static std::set<MergeTreeIndexPtr> getIndicesToRecalculate(
|
||||
BlockInputStreamPtr & input_stream,
|
||||
const NamesAndTypesList & updated_columns,
|
||||
const Context & context) const;
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const Context & context);
|
||||
|
||||
/// Override all columns of new part using mutating_stream
|
||||
void mutateAllPartColumns(
|
||||
MergeTreeData::MutableDataPartPtr new_data_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeIndices & skip_indices,
|
||||
BlockInputStreamPtr mutating_stream,
|
||||
time_t time_of_mutation,
|
||||
@ -186,6 +194,7 @@ private:
|
||||
/// 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 Block & mutation_header,
|
||||
MergeTreeData::MutableDataPartPtr new_data_part,
|
||||
|
@ -38,6 +38,7 @@ MergeTreeDataPartCompact::MergeTreeDataPartCompact(
|
||||
|
||||
IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader(
|
||||
const NamesAndTypesList & columns_to_read,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MarkRanges & mark_ranges,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
MarkCache * mark_cache,
|
||||
@ -47,13 +48,14 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader(
|
||||
{
|
||||
auto ptr = std::static_pointer_cast<const MergeTreeDataPartCompact>(shared_from_this());
|
||||
return std::make_unique<MergeTreeReaderCompact>(
|
||||
ptr, columns_to_read, uncompressed_cache,
|
||||
ptr, columns_to_read, metadata_snapshot, uncompressed_cache,
|
||||
mark_cache, mark_ranges, reader_settings,
|
||||
avg_value_size_hints, profile_callback);
|
||||
}
|
||||
|
||||
IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter(
|
||||
const NamesAndTypesList & columns_list,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
|
||||
const CompressionCodecPtr & default_codec,
|
||||
const MergeTreeWriterSettings & writer_settings,
|
||||
@ -68,8 +70,8 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter(
|
||||
{ return *getColumnPosition(lhs.name) < *getColumnPosition(rhs.name); });
|
||||
|
||||
return std::make_unique<MergeTreeDataPartWriterCompact>(
|
||||
shared_from_this(), ordered_columns_list, indices_to_recalc,
|
||||
index_granularity_info.marks_file_extension,
|
||||
shared_from_this(), ordered_columns_list, metadata_snapshot,
|
||||
indices_to_recalc, index_granularity_info.marks_file_extension,
|
||||
default_codec, writer_settings, computed_index_granularity);
|
||||
}
|
||||
|
||||
|
@ -37,6 +37,7 @@ public:
|
||||
|
||||
MergeTreeReaderPtr getReader(
|
||||
const NamesAndTypesList & columns,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MarkRanges & mark_ranges,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
MarkCache * mark_cache,
|
||||
@ -46,6 +47,7 @@ public:
|
||||
|
||||
MergeTreeWriterPtr getWriter(
|
||||
const NamesAndTypesList & columns_list,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
|
||||
const CompressionCodecPtr & default_codec_,
|
||||
const MergeTreeWriterSettings & writer_settings,
|
||||
|
@ -37,6 +37,7 @@ MergeTreeDataPartWide::MergeTreeDataPartWide(
|
||||
|
||||
IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader(
|
||||
const NamesAndTypesList & columns_to_read,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MarkRanges & mark_ranges,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
MarkCache * mark_cache,
|
||||
@ -46,20 +47,21 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader(
|
||||
{
|
||||
auto ptr = std::static_pointer_cast<const MergeTreeDataPartWide>(shared_from_this());
|
||||
return std::make_unique<MergeTreeReaderWide>(
|
||||
ptr, columns_to_read, uncompressed_cache,
|
||||
ptr, columns_to_read, metadata_snapshot, uncompressed_cache,
|
||||
mark_cache, mark_ranges, reader_settings,
|
||||
avg_value_size_hints, profile_callback);
|
||||
}
|
||||
|
||||
IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter(
|
||||
const NamesAndTypesList & columns_list,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
|
||||
const CompressionCodecPtr & default_codec,
|
||||
const MergeTreeWriterSettings & writer_settings,
|
||||
const MergeTreeIndexGranularity & computed_index_granularity) const
|
||||
{
|
||||
return std::make_unique<MergeTreeDataPartWriterWide>(
|
||||
shared_from_this(), columns_list, indices_to_recalc,
|
||||
shared_from_this(), columns_list, metadata_snapshot, indices_to_recalc,
|
||||
index_granularity_info.marks_file_extension,
|
||||
default_codec, writer_settings, computed_index_granularity);
|
||||
}
|
||||
|
@ -30,6 +30,7 @@ public:
|
||||
|
||||
MergeTreeReaderPtr getReader(
|
||||
const NamesAndTypesList & columns,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MarkRanges & mark_ranges,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
MarkCache * mark_cache,
|
||||
@ -39,6 +40,7 @@ public:
|
||||
|
||||
MergeTreeWriterPtr getWriter(
|
||||
const NamesAndTypesList & columns_list,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
|
||||
const CompressionCodecPtr & default_codec_,
|
||||
const MergeTreeWriterSettings & writer_settings,
|
||||
|
@ -3,19 +3,17 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact(
|
||||
const MergeTreeData::DataPartPtr & data_part_,
|
||||
const NamesAndTypesList & columns_list_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const std::vector<MergeTreeIndexPtr> & indices_to_recalc_,
|
||||
const String & marks_file_extension_,
|
||||
const CompressionCodecPtr & default_codec_,
|
||||
const MergeTreeWriterSettings & settings_,
|
||||
const MergeTreeIndexGranularity & index_granularity_)
|
||||
: IMergeTreeDataPartWriter(data_part_, columns_list_,
|
||||
indices_to_recalc_, marks_file_extension_,
|
||||
default_codec_, settings_, index_granularity_)
|
||||
: IMergeTreeDataPartWriter(
|
||||
data_part_, columns_list_, metadata_snapshot_, indices_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_)
|
||||
{
|
||||
using DataPart = MergeTreeDataPartCompact;
|
||||
String data_file_name = DataPart::DATA_FILE_NAME;
|
||||
|
@ -10,6 +10,7 @@ public:
|
||||
MergeTreeDataPartWriterCompact(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const NamesAndTypesList & columns_list,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
|
||||
const String & marks_file_extension,
|
||||
const CompressionCodecPtr & default_codec,
|
||||
|
@ -16,16 +16,16 @@ namespace
|
||||
MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide(
|
||||
const MergeTreeData::DataPartPtr & data_part_,
|
||||
const NamesAndTypesList & columns_list_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const std::vector<MergeTreeIndexPtr> & indices_to_recalc_,
|
||||
const String & marks_file_extension_,
|
||||
const CompressionCodecPtr & default_codec_,
|
||||
const MergeTreeWriterSettings & settings_,
|
||||
const MergeTreeIndexGranularity & index_granularity_)
|
||||
: IMergeTreeDataPartWriter(data_part_, columns_list_,
|
||||
indices_to_recalc_, marks_file_extension_,
|
||||
default_codec_, settings_, index_granularity_)
|
||||
: IMergeTreeDataPartWriter(
|
||||
data_part_, columns_list_, metadata_snapshot_, indices_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_)
|
||||
{
|
||||
const auto & columns = storage.getColumns();
|
||||
const auto & columns = metadata_snapshot->getColumns();
|
||||
for (const auto & it : columns_list)
|
||||
addStreams(it.name, *it.type, columns.getCodecOrDefault(it.name, default_codec), settings.estimated_size);
|
||||
}
|
||||
|
@ -13,6 +13,7 @@ public:
|
||||
MergeTreeDataPartWriterWide(
|
||||
const MergeTreeData::DataPartPtr & data_part,
|
||||
const NamesAndTypesList & columns_list,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const std::vector<MergeTreeIndexPtr> & indices_to_recalc,
|
||||
const String & marks_file_extension,
|
||||
const CompressionCodecPtr & default_codec,
|
||||
|
@ -100,7 +100,10 @@ static Block getBlockWithPartColumn(const MergeTreeData::DataPartsVector & parts
|
||||
|
||||
|
||||
size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead(
|
||||
const MergeTreeData::DataPartsVector & parts, const KeyCondition & key_condition, const Settings & settings) const
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const KeyCondition & key_condition,
|
||||
const Settings & settings) const
|
||||
{
|
||||
size_t rows_count = 0;
|
||||
|
||||
@ -109,7 +112,7 @@ size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead(
|
||||
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
MarkRanges ranges = markRangesFromPKRange(part, key_condition, settings);
|
||||
MarkRanges ranges = markRangesFromPKRange(part, metadata_snapshot, key_condition, settings);
|
||||
|
||||
/** In order to get a lower bound on the number of rows that match the condition on PK,
|
||||
* consider only guaranteed full marks.
|
||||
@ -147,6 +150,7 @@ static RelativeSize convertAbsoluteSampleSizeToRelative(const ASTPtr & node, siz
|
||||
|
||||
Pipes MergeTreeDataSelectExecutor::read(
|
||||
const Names & column_names_to_return,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
const UInt64 max_block_size,
|
||||
@ -154,13 +158,15 @@ Pipes MergeTreeDataSelectExecutor::read(
|
||||
const PartitionIdToMaxBlock * max_block_numbers_to_read) const
|
||||
{
|
||||
return readFromParts(
|
||||
data.getDataPartsVector(), column_names_to_return, query_info, context,
|
||||
max_block_size, num_streams, max_block_numbers_to_read);
|
||||
data.getDataPartsVector(), column_names_to_return, metadata_snapshot,
|
||||
query_info, context, max_block_size, num_streams,
|
||||
max_block_numbers_to_read);
|
||||
}
|
||||
|
||||
Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
MergeTreeData::DataPartsVector parts,
|
||||
const Names & column_names_to_return,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
const UInt64 max_block_size,
|
||||
@ -205,7 +211,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
}
|
||||
}
|
||||
|
||||
NamesAndTypesList available_real_columns = data.getColumns().getAllPhysical();
|
||||
NamesAndTypesList available_real_columns = metadata_snapshot->getColumns().getAllPhysical();
|
||||
|
||||
/// If there are only virtual columns in the query, you must request at least one non-virtual one.
|
||||
if (real_column_names.empty())
|
||||
@ -218,10 +224,10 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
std::multiset<String> part_values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
|
||||
|
||||
data.check(real_column_names);
|
||||
metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID());
|
||||
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const auto & primary_key = data.getPrimaryKey();
|
||||
const auto & primary_key = metadata_snapshot->getPrimaryKey();
|
||||
Names primary_key_columns = primary_key.column_names;
|
||||
|
||||
KeyCondition key_condition(query_info, context, primary_key_columns, primary_key.expression);
|
||||
@ -323,7 +329,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
/// Convert absolute value of the sampling (in form `SAMPLE 1000000` - how many rows to read) into the relative `SAMPLE 0.1` (how much data to read).
|
||||
size_t approx_total_rows = 0;
|
||||
if (relative_sample_size > 1 || relative_sample_offset > 1)
|
||||
approx_total_rows = getApproximateTotalRowsToRead(parts, key_condition, settings);
|
||||
approx_total_rows = getApproximateTotalRowsToRead(parts, metadata_snapshot, key_condition, settings);
|
||||
|
||||
if (relative_sample_size > 1)
|
||||
{
|
||||
@ -387,7 +393,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
used_sample_factor = 1.0 / boost::rational_cast<Float64>(relative_sample_size);
|
||||
|
||||
RelativeSize size_of_universum = 0;
|
||||
const auto & sampling_key = data.getSamplingKey();
|
||||
const auto & sampling_key = metadata_snapshot->getSamplingKey();
|
||||
DataTypePtr sampling_column_type = sampling_key.data_types[0];
|
||||
|
||||
if (typeid_cast<const DataTypeUInt64 *>(sampling_column_type.get()))
|
||||
@ -456,7 +462,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
/// If sample and final are used together no need to calculate sampling expression twice.
|
||||
/// The first time it was calculated for final, because sample key is a part of the PK.
|
||||
/// So, assume that we already have calculated column.
|
||||
ASTPtr sampling_key_ast = data.getSamplingKeyAST();
|
||||
ASTPtr sampling_key_ast = metadata_snapshot->getSamplingKeyAST();
|
||||
|
||||
if (select.final())
|
||||
{
|
||||
@ -547,7 +553,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
std::vector<std::pair<MergeTreeIndexPtr, MergeTreeIndexConditionPtr>> useful_indices;
|
||||
|
||||
for (const auto & index : data.getSecondaryIndices())
|
||||
for (const auto & index : metadata_snapshot->getSecondaryIndices())
|
||||
{
|
||||
auto index_helper = MergeTreeIndexFactory::instance().get(index);
|
||||
auto condition = index_helper->createIndexCondition(query_info, context);
|
||||
@ -562,8 +568,8 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
{
|
||||
RangesInDataPart ranges(part, part_index++);
|
||||
|
||||
if (data.hasPrimaryKey())
|
||||
ranges.ranges = markRangesFromPKRange(part, key_condition, settings);
|
||||
if (metadata_snapshot->hasPrimaryKey())
|
||||
ranges.ranges = markRangesFromPKRange(part, metadata_snapshot, key_condition, settings);
|
||||
else
|
||||
{
|
||||
size_t total_marks_count = part->getMarksCount();
|
||||
@ -614,7 +620,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
if (select.final())
|
||||
{
|
||||
/// Add columns needed to calculate the sorting expression and the sign.
|
||||
std::vector<String> add_columns = data.getColumnsRequiredForSortingKey();
|
||||
std::vector<String> add_columns = metadata_snapshot->getColumnsRequiredForSortingKey();
|
||||
column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end());
|
||||
|
||||
if (!data.merging_params.sign_column.empty())
|
||||
@ -629,6 +635,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
std::move(parts_with_ranges),
|
||||
num_streams,
|
||||
column_names_to_read,
|
||||
metadata_snapshot,
|
||||
max_block_size,
|
||||
settings.use_uncompressed_cache,
|
||||
query_info,
|
||||
@ -640,16 +647,17 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
else if ((settings.optimize_read_in_order || settings.optimize_aggregation_in_order) && query_info.input_order_info)
|
||||
{
|
||||
size_t prefix_size = query_info.input_order_info->order_key_prefix_descr.size();
|
||||
auto order_key_prefix_ast = data.getSortingKey().expression_list_ast->clone();
|
||||
auto order_key_prefix_ast = metadata_snapshot->getSortingKey().expression_list_ast->clone();
|
||||
order_key_prefix_ast->children.resize(prefix_size);
|
||||
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(order_key_prefix_ast, data.getColumns().getAllPhysical());
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(order_key_prefix_ast, metadata_snapshot->getColumns().getAllPhysical());
|
||||
auto sorting_key_prefix_expr = ExpressionAnalyzer(order_key_prefix_ast, syntax_result, context).getActions(false);
|
||||
|
||||
res = spreadMarkRangesAmongStreamsWithOrder(
|
||||
std::move(parts_with_ranges),
|
||||
num_streams,
|
||||
column_names_to_read,
|
||||
metadata_snapshot,
|
||||
max_block_size,
|
||||
settings.use_uncompressed_cache,
|
||||
query_info,
|
||||
@ -665,6 +673,7 @@ Pipes MergeTreeDataSelectExecutor::readFromParts(
|
||||
std::move(parts_with_ranges),
|
||||
num_streams,
|
||||
column_names_to_read,
|
||||
metadata_snapshot,
|
||||
max_block_size,
|
||||
settings.use_uncompressed_cache,
|
||||
query_info,
|
||||
@ -727,6 +736,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams(
|
||||
RangesInDataParts && parts,
|
||||
size_t num_streams,
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
UInt64 max_block_size,
|
||||
bool use_uncompressed_cache,
|
||||
const SelectQueryInfo & query_info,
|
||||
@ -783,8 +793,18 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams(
|
||||
num_streams = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts.size());
|
||||
|
||||
MergeTreeReadPoolPtr pool = std::make_shared<MergeTreeReadPool>(
|
||||
num_streams, sum_marks, min_marks_for_concurrent_read, parts, data, query_info.prewhere_info, true,
|
||||
column_names, MergeTreeReadPool::BackoffSettings(settings), settings.preferred_block_size_bytes, false);
|
||||
num_streams,
|
||||
sum_marks,
|
||||
min_marks_for_concurrent_read,
|
||||
parts,
|
||||
data,
|
||||
metadata_snapshot,
|
||||
query_info.prewhere_info,
|
||||
true,
|
||||
column_names,
|
||||
MergeTreeReadPool::BackoffSettings(settings),
|
||||
settings.preferred_block_size_bytes,
|
||||
false);
|
||||
|
||||
/// Let's estimate total number of rows for progress bar.
|
||||
LOG_TRACE(log, "Reading approx. {} rows with {} streams", total_rows, num_streams);
|
||||
@ -792,8 +812,9 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams(
|
||||
for (size_t i = 0; i < num_streams; ++i)
|
||||
{
|
||||
auto source = std::make_shared<MergeTreeThreadSelectBlockInputProcessor>(
|
||||
i, pool, min_marks_for_concurrent_read, max_block_size, settings.preferred_block_size_bytes,
|
||||
settings.preferred_max_column_in_block_size_bytes, data, use_uncompressed_cache,
|
||||
i, pool, min_marks_for_concurrent_read, max_block_size,
|
||||
settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes,
|
||||
data, metadata_snapshot, use_uncompressed_cache,
|
||||
query_info.prewhere_info, reader_settings, virt_columns);
|
||||
|
||||
if (i == 0)
|
||||
@ -812,7 +833,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams(
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
auto source = std::make_shared<MergeTreeSelectProcessor>(
|
||||
data, part.data_part, max_block_size, settings.preferred_block_size_bytes,
|
||||
data, metadata_snapshot, part.data_part, max_block_size, settings.preferred_block_size_bytes,
|
||||
settings.preferred_max_column_in_block_size_bytes, column_names, part.ranges, use_uncompressed_cache,
|
||||
query_info.prewhere_info, true, reader_settings, virt_columns, part.part_index_in_query);
|
||||
|
||||
@ -845,6 +866,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder(
|
||||
RangesInDataParts && parts,
|
||||
size_t num_streams,
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
UInt64 max_block_size,
|
||||
bool use_uncompressed_cache,
|
||||
const SelectQueryInfo & query_info,
|
||||
@ -1004,18 +1026,38 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder(
|
||||
if (input_order_info->direction == 1)
|
||||
{
|
||||
pipes.emplace_back(std::make_shared<MergeTreeSelectProcessor>(
|
||||
data, part.data_part, max_block_size, settings.preferred_block_size_bytes,
|
||||
settings.preferred_max_column_in_block_size_bytes, column_names, ranges_to_get_from_part,
|
||||
use_uncompressed_cache, query_info.prewhere_info, true, reader_settings,
|
||||
virt_columns, part.part_index_in_query));
|
||||
data,
|
||||
metadata_snapshot,
|
||||
part.data_part,
|
||||
max_block_size,
|
||||
settings.preferred_block_size_bytes,
|
||||
settings.preferred_max_column_in_block_size_bytes,
|
||||
column_names,
|
||||
ranges_to_get_from_part,
|
||||
use_uncompressed_cache,
|
||||
query_info.prewhere_info,
|
||||
true,
|
||||
reader_settings,
|
||||
virt_columns,
|
||||
part.part_index_in_query));
|
||||
}
|
||||
else
|
||||
{
|
||||
pipes.emplace_back(std::make_shared<MergeTreeReverseSelectProcessor>(
|
||||
data, part.data_part, max_block_size, settings.preferred_block_size_bytes,
|
||||
settings.preferred_max_column_in_block_size_bytes, column_names, ranges_to_get_from_part,
|
||||
use_uncompressed_cache, query_info.prewhere_info, true, reader_settings,
|
||||
virt_columns, part.part_index_in_query));
|
||||
data,
|
||||
metadata_snapshot,
|
||||
part.data_part,
|
||||
max_block_size,
|
||||
settings.preferred_block_size_bytes,
|
||||
settings.preferred_max_column_in_block_size_bytes,
|
||||
column_names,
|
||||
ranges_to_get_from_part,
|
||||
use_uncompressed_cache,
|
||||
query_info.prewhere_info,
|
||||
true,
|
||||
reader_settings,
|
||||
virt_columns,
|
||||
part.part_index_in_query));
|
||||
|
||||
pipes.back().addSimpleTransform(std::make_shared<ReverseTransform>(pipes.back().getHeader()));
|
||||
}
|
||||
@ -1025,7 +1067,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder(
|
||||
{
|
||||
SortDescription sort_description;
|
||||
for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j)
|
||||
sort_description.emplace_back(data.getSortingKey().column_names[j],
|
||||
sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j],
|
||||
input_order_info->direction, 1);
|
||||
|
||||
/// Drop temporary columns, added by 'sorting_key_prefix_expr'
|
||||
@ -1050,6 +1092,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
|
||||
RangesInDataParts && parts,
|
||||
size_t num_streams,
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
UInt64 max_block_size,
|
||||
bool use_uncompressed_cache,
|
||||
const SelectQueryInfo & query_info,
|
||||
@ -1088,7 +1131,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
|
||||
for (const auto & part : parts)
|
||||
{
|
||||
auto source_processor = std::make_shared<MergeTreeSelectProcessor>(
|
||||
data, part.data_part, max_block_size, settings.preferred_block_size_bytes,
|
||||
data, metadata_snapshot, part.data_part, max_block_size, settings.preferred_block_size_bytes,
|
||||
settings.preferred_max_column_in_block_size_bytes, column_names, part.ranges, use_uncompressed_cache,
|
||||
query_info.prewhere_info, true, reader_settings,
|
||||
virt_columns, part.part_index_in_query);
|
||||
@ -1098,11 +1141,11 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
|
||||
if (!out_projection)
|
||||
out_projection = createProjection(pipe, data);
|
||||
|
||||
pipe.addSimpleTransform(std::make_shared<ExpressionTransform>(pipe.getHeader(), data.getSortingKey().expression));
|
||||
pipe.addSimpleTransform(std::make_shared<ExpressionTransform>(pipe.getHeader(), metadata_snapshot->getSortingKey().expression));
|
||||
pipes.emplace_back(std::move(pipe));
|
||||
}
|
||||
|
||||
Names sort_columns = data.getSortingKeyColumns();
|
||||
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
|
||||
SortDescription sort_description;
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
@ -1231,33 +1274,13 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
|
||||
return pipes;
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeDataSelectExecutor::createPositiveSignCondition(
|
||||
ExpressionActionsPtr & out_expression, String & out_column, const Context & context) const
|
||||
{
|
||||
auto function = std::make_shared<ASTFunction>();
|
||||
auto arguments = std::make_shared<ASTExpressionList>();
|
||||
auto sign = std::make_shared<ASTIdentifier>(data.merging_params.sign_column);
|
||||
auto one = std::make_shared<ASTLiteral>(1);
|
||||
|
||||
function->name = "equals";
|
||||
function->arguments = arguments;
|
||||
function->children.push_back(arguments);
|
||||
|
||||
arguments->children.push_back(sign);
|
||||
arguments->children.push_back(one);
|
||||
|
||||
ASTPtr query = function;
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(query, data.getColumns().getAllPhysical());
|
||||
out_expression = ExpressionAnalyzer(query, syntax_result, context).getActions(false);
|
||||
out_column = function->getColumnName();
|
||||
}
|
||||
|
||||
|
||||
/// Calculates a set of mark ranges, that could possibly contain keys, required by condition.
|
||||
/// In other words, it removes subranges from whole range, that definitely could not contain required keys.
|
||||
MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
|
||||
const MergeTreeData::DataPartPtr & part, const KeyCondition & key_condition, const Settings & settings) const
|
||||
const MergeTreeData::DataPartPtr & part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const KeyCondition & key_condition,
|
||||
const Settings & settings)
|
||||
{
|
||||
MarkRanges res;
|
||||
|
||||
@ -1295,7 +1318,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
|
||||
std::function<void(size_t, size_t, FieldRef &)> create_field_ref;
|
||||
/// If there are no monotonic functions, there is no need to save block reference.
|
||||
/// Passing explicit field to FieldRef allows to optimize ranges and shows better performance.
|
||||
const auto & primary_key = data.getPrimaryKey();
|
||||
const auto & primary_key = metadata_snapshot->getPrimaryKey();
|
||||
if (key_condition.hasMonotonicFunctionsChain())
|
||||
{
|
||||
auto index_block = std::make_shared<Block>();
|
||||
|
@ -26,6 +26,7 @@ public:
|
||||
|
||||
Pipes read(
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
UInt64 max_block_size,
|
||||
@ -35,6 +36,7 @@ public:
|
||||
Pipes readFromParts(
|
||||
MergeTreeData::DataPartsVector parts,
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
UInt64 max_block_size,
|
||||
@ -50,6 +52,7 @@ private:
|
||||
RangesInDataParts && parts,
|
||||
size_t num_streams,
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
UInt64 max_block_size,
|
||||
bool use_uncompressed_cache,
|
||||
const SelectQueryInfo & query_info,
|
||||
@ -62,6 +65,7 @@ private:
|
||||
RangesInDataParts && parts,
|
||||
size_t num_streams,
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
UInt64 max_block_size,
|
||||
bool use_uncompressed_cache,
|
||||
const SelectQueryInfo & query_info,
|
||||
@ -75,6 +79,7 @@ private:
|
||||
RangesInDataParts && parts,
|
||||
size_t num_streams,
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
UInt64 max_block_size,
|
||||
bool use_uncompressed_cache,
|
||||
const SelectQueryInfo & query_info,
|
||||
@ -86,19 +91,15 @@ private:
|
||||
/// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index.
|
||||
size_t getApproximateTotalRowsToRead(
|
||||
const MergeTreeData::DataPartsVector & parts,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const KeyCondition & key_condition,
|
||||
const Settings & settings) const;
|
||||
|
||||
/// Create the expression "Sign == 1".
|
||||
void createPositiveSignCondition(
|
||||
ExpressionActionsPtr & out_expression,
|
||||
String & out_column,
|
||||
const Context & context) const;
|
||||
|
||||
MarkRanges markRangesFromPKRange(
|
||||
static MarkRanges markRangesFromPKRange(
|
||||
const MergeTreeData::DataPartPtr & part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const KeyCondition & key_condition,
|
||||
const Settings & settings) const;
|
||||
const Settings & settings);
|
||||
|
||||
MarkRanges filterMarksUsingIndex(
|
||||
MergeTreeIndexPtr index_helper,
|
||||
|
@ -132,22 +132,22 @@ void updateTTL(
|
||||
|
||||
}
|
||||
|
||||
BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block, size_t max_parts)
|
||||
BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot)
|
||||
{
|
||||
BlocksWithPartition result;
|
||||
if (!block || !block.rows())
|
||||
return result;
|
||||
|
||||
data.check(block, true);
|
||||
metadata_snapshot->check(block, true);
|
||||
|
||||
if (!data.hasPartitionKey()) /// Table is not partitioned.
|
||||
if (!metadata_snapshot->hasPartitionKey()) /// Table is not partitioned.
|
||||
{
|
||||
result.emplace_back(Block(block), Row());
|
||||
return result;
|
||||
}
|
||||
|
||||
Block block_copy = block;
|
||||
const auto & partition_key = data.getPartitionKey();
|
||||
const auto & partition_key = metadata_snapshot->getPartitionKey();
|
||||
partition_key.expression->execute(block_copy);
|
||||
|
||||
ColumnRawPtrs partition_columns;
|
||||
@ -192,7 +192,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block
|
||||
return result;
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPartition & block_with_partition)
|
||||
MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPartition & block_with_partition, const StorageMetadataPtr & metadata_snapshot)
|
||||
{
|
||||
Block & block = block_with_partition.block;
|
||||
|
||||
@ -206,7 +206,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
|
||||
MergeTreePartition partition(std::move(block_with_partition.partition));
|
||||
|
||||
MergeTreePartInfo new_part_info(partition.getID(data.getPartitionKey().sample_block), temp_index, temp_index, 0);
|
||||
MergeTreePartInfo new_part_info(partition.getID(metadata_snapshot->getPartitionKey().sample_block), temp_index, temp_index, 0);
|
||||
String part_name;
|
||||
if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
|
||||
{
|
||||
@ -230,11 +230,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
size_t expected_size = block.bytes();
|
||||
|
||||
DB::IMergeTreeDataPart::TTLInfos move_ttl_infos;
|
||||
const auto & move_ttl_entries = data.getMoveTTLs();
|
||||
const auto & move_ttl_entries = metadata_snapshot->getMoveTTLs();
|
||||
for (const auto & ttl_entry : move_ttl_entries)
|
||||
updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false);
|
||||
|
||||
NamesAndTypesList columns = data.getColumns().getAllPhysical().filter(block.getNames());
|
||||
NamesAndTypesList columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames());
|
||||
ReservationPtr reservation = data.reserveSpacePreferringTTLRules(expected_size, move_ttl_infos, time(nullptr));
|
||||
VolumePtr volume = data.getStoragePolicy()->getVolume(0);
|
||||
|
||||
@ -262,10 +262,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
new_data_part->volume->getDisk()->createDirectories(full_path);
|
||||
|
||||
/// If we need to calculate some columns to sort.
|
||||
if (data.hasSortingKey() || data.hasSecondaryIndices())
|
||||
data.getSortingKeyAndSkipIndicesExpression()->execute(block);
|
||||
if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices())
|
||||
data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot)->execute(block);
|
||||
|
||||
Names sort_columns = data.getSortingKeyColumns();
|
||||
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
|
||||
SortDescription sort_description;
|
||||
size_t sort_columns_size = sort_columns.size();
|
||||
sort_description.reserve(sort_columns_size);
|
||||
@ -289,10 +289,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocksAlreadySorted);
|
||||
}
|
||||
|
||||
if (data.hasRowsTTL())
|
||||
updateTTL(data.getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true);
|
||||
if (metadata_snapshot->hasRowsTTL())
|
||||
updateTTL(metadata_snapshot->getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true);
|
||||
|
||||
for (const auto & [name, ttl_entry] : data.getColumnTTLs())
|
||||
for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs())
|
||||
updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true);
|
||||
|
||||
new_data_part->ttl_infos.update(move_ttl_infos);
|
||||
@ -302,7 +302,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
|
||||
auto compression_codec = data.global_context.chooseCompressionCodec(0, 0);
|
||||
|
||||
const auto & index_factory = MergeTreeIndexFactory::instance();
|
||||
MergedBlockOutputStream out(new_data_part, columns, index_factory.getMany(data.getSecondaryIndices()), compression_codec);
|
||||
MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec);
|
||||
|
||||
out.writePrefix();
|
||||
out.writeWithPermutation(block, perm_ptr);
|
||||
|
@ -40,12 +40,12 @@ public:
|
||||
* (split rows by partition)
|
||||
* Works deterministically: if same block was passed, function will return same result in same order.
|
||||
*/
|
||||
BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts);
|
||||
static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot);
|
||||
|
||||
/** All rows must correspond to same partition.
|
||||
* Returns part with unique name starting with 'tmp_', yet not added to MergeTreeData.
|
||||
*/
|
||||
MergeTreeData::MutableDataPartPtr writeTempPart(BlockWithPartition & block);
|
||||
MergeTreeData::MutableDataPartPtr writeTempPart(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot);
|
||||
|
||||
private:
|
||||
MergeTreeData & data;
|
||||
|
@ -26,7 +26,7 @@ static std::unique_ptr<ReadBufferFromFileBase> openForReading(const DiskPtr & di
|
||||
|
||||
String MergeTreePartition::getID(const MergeTreeData & storage) const
|
||||
{
|
||||
return getID(storage.getPartitionKey().sample_block);
|
||||
return getID(storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block);
|
||||
}
|
||||
|
||||
/// NOTE: This ID is used to create part names which are then persisted in ZK and as directory names on the file system.
|
||||
@ -89,7 +89,7 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const
|
||||
|
||||
void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const
|
||||
{
|
||||
const auto & partition_key_sample = storage.getPartitionKey().sample_block;
|
||||
const auto & partition_key_sample = storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block;
|
||||
size_t key_size = partition_key_sample.columns();
|
||||
|
||||
if (key_size == 0)
|
||||
@ -124,10 +124,11 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe
|
||||
|
||||
void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path)
|
||||
{
|
||||
if (!storage.hasPartitionKey())
|
||||
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
|
||||
if (!metadata_snapshot->hasPartitionKey())
|
||||
return;
|
||||
|
||||
const auto & partition_key_sample = storage.getPartitionKey().sample_block;
|
||||
const auto & partition_key_sample = metadata_snapshot->getPartitionKey().sample_block;
|
||||
auto partition_file_path = part_path + "partition.dat";
|
||||
auto file = openForReading(disk, partition_file_path);
|
||||
value.resize(partition_key_sample.columns());
|
||||
@ -137,7 +138,7 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis
|
||||
|
||||
void MergeTreePartition::store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const
|
||||
{
|
||||
store(storage.getPartitionKey().sample_block, disk, part_path, checksums);
|
||||
store(storage.getInMemoryMetadataPtr()->getPartitionKey().sample_block, disk, part_path, checksums);
|
||||
}
|
||||
|
||||
void MergeTreePartition::store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const
|
||||
|
@ -17,17 +17,28 @@ namespace ErrorCodes
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
MergeTreeReadPool::MergeTreeReadPool(
|
||||
const size_t threads_, const size_t sum_marks_, const size_t min_marks_for_concurrent_read_,
|
||||
RangesInDataParts parts_, const MergeTreeData & data_, const PrewhereInfoPtr & prewhere_info_,
|
||||
const bool check_columns_, const Names & column_names_,
|
||||
const BackoffSettings & backoff_settings_, size_t preferred_block_size_bytes_,
|
||||
const size_t threads_,
|
||||
const size_t sum_marks_,
|
||||
const size_t min_marks_for_concurrent_read_,
|
||||
RangesInDataParts parts_,
|
||||
const MergeTreeData & data_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const PrewhereInfoPtr & prewhere_info_,
|
||||
const bool check_columns_,
|
||||
const Names & column_names_,
|
||||
const BackoffSettings & backoff_settings_,
|
||||
size_t preferred_block_size_bytes_,
|
||||
const bool do_not_steal_tasks_)
|
||||
: backoff_settings{backoff_settings_}, backoff_state{threads_}, data{data_},
|
||||
column_names{column_names_}, do_not_steal_tasks{do_not_steal_tasks_},
|
||||
predict_block_size_bytes{preferred_block_size_bytes_ > 0}, prewhere_info{prewhere_info_}, parts_ranges{parts_}
|
||||
: backoff_settings{backoff_settings_}
|
||||
, backoff_state{threads_}
|
||||
, data{data_}
|
||||
, metadata_snapshot{metadata_snapshot_}
|
||||
, column_names{column_names_}
|
||||
, do_not_steal_tasks{do_not_steal_tasks_}
|
||||
, predict_block_size_bytes{preferred_block_size_bytes_ > 0}
|
||||
, prewhere_info{prewhere_info_}
|
||||
, parts_ranges{parts_}
|
||||
{
|
||||
/// parts don't contain duplicate MergeTreeDataPart's.
|
||||
const auto per_part_sum_marks = fillPerPartInfo(parts_, check_columns_);
|
||||
@ -139,7 +150,7 @@ MarkRanges MergeTreeReadPool::getRestMarks(const IMergeTreeDataPart & part, cons
|
||||
|
||||
Block MergeTreeReadPool::getHeader() const
|
||||
{
|
||||
return data.getSampleBlockForColumns(column_names);
|
||||
return metadata_snapshot->getSampleBlockForColumns(column_names, data.getVirtuals(), data.getStorageID());
|
||||
}
|
||||
|
||||
void MergeTreeReadPool::profileFeedback(const ReadBufferFromFileBase::ProfileInfo info)
|
||||
@ -186,7 +197,7 @@ std::vector<size_t> MergeTreeReadPool::fillPerPartInfo(
|
||||
RangesInDataParts & parts, const bool check_columns)
|
||||
{
|
||||
std::vector<size_t> per_part_sum_marks;
|
||||
Block sample_block = data.getSampleBlock();
|
||||
Block sample_block = metadata_snapshot->getSampleBlock();
|
||||
|
||||
for (const auto i : ext::range(0, parts.size()))
|
||||
{
|
||||
@ -200,7 +211,7 @@ std::vector<size_t> MergeTreeReadPool::fillPerPartInfo(
|
||||
per_part_sum_marks.push_back(sum_marks);
|
||||
|
||||
auto [required_columns, required_pre_columns, should_reorder] =
|
||||
getReadTaskColumns(data, part.data_part, column_names, prewhere_info, check_columns);
|
||||
getReadTaskColumns(data, metadata_snapshot, part.data_part, column_names, prewhere_info, check_columns);
|
||||
|
||||
/// will be used to distinguish between PREWHERE and WHERE columns when applying filter
|
||||
const auto & required_column_names = required_columns.getNames();
|
||||
|
@ -68,7 +68,7 @@ private:
|
||||
public:
|
||||
MergeTreeReadPool(
|
||||
const size_t threads_, const size_t sum_marks_, const size_t min_marks_for_concurrent_read_,
|
||||
RangesInDataParts parts_, const MergeTreeData & data_, const PrewhereInfoPtr & prewhere_info_,
|
||||
RangesInDataParts parts_, const MergeTreeData & data_, const StorageMetadataPtr & metadata_snapshot_, const PrewhereInfoPtr & prewhere_info_,
|
||||
const bool check_columns_, const Names & column_names_,
|
||||
const BackoffSettings & backoff_settings_, size_t preferred_block_size_bytes_,
|
||||
const bool do_not_steal_tasks_ = false);
|
||||
@ -95,6 +95,7 @@ private:
|
||||
RangesInDataParts & parts, const size_t min_marks_for_concurrent_read);
|
||||
|
||||
const MergeTreeData & data;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
Names column_names;
|
||||
bool do_not_steal_tasks;
|
||||
bool predict_block_size_bytes;
|
||||
|
@ -17,6 +17,7 @@ namespace ErrorCodes
|
||||
MergeTreeReaderCompact::MergeTreeReaderCompact(
|
||||
DataPartCompactPtr data_part_,
|
||||
NamesAndTypesList columns_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
UncompressedCache * uncompressed_cache_,
|
||||
MarkCache * mark_cache_,
|
||||
MarkRanges mark_ranges_,
|
||||
@ -24,15 +25,23 @@ MergeTreeReaderCompact::MergeTreeReaderCompact(
|
||||
ValueSizeMap avg_value_size_hints_,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
|
||||
clockid_t clock_type_)
|
||||
: IMergeTreeReader(std::move(data_part_), std::move(columns_),
|
||||
uncompressed_cache_, mark_cache_, std::move(mark_ranges_),
|
||||
std::move(settings_), std::move(avg_value_size_hints_))
|
||||
: IMergeTreeReader(
|
||||
std::move(data_part_),
|
||||
std::move(columns_),
|
||||
metadata_snapshot_,
|
||||
uncompressed_cache_,
|
||||
mark_cache_,
|
||||
std::move(mark_ranges_),
|
||||
std::move(settings_),
|
||||
std::move(avg_value_size_hints_))
|
||||
, marks_loader(
|
||||
data_part->volume->getDisk(),
|
||||
mark_cache,
|
||||
data_part->index_granularity_info.getMarksFilePath(data_part->getFullRelativePath() + MergeTreeDataPartCompact::DATA_FILE_NAME),
|
||||
data_part->getMarksCount(), data_part->index_granularity_info,
|
||||
settings.save_marks_in_cache, data_part->getColumns().size())
|
||||
data_part->volume->getDisk(),
|
||||
mark_cache,
|
||||
data_part->index_granularity_info.getMarksFilePath(data_part->getFullRelativePath() + MergeTreeDataPartCompact::DATA_FILE_NAME),
|
||||
data_part->getMarksCount(),
|
||||
data_part->index_granularity_info,
|
||||
settings.save_marks_in_cache,
|
||||
data_part->getColumns().size())
|
||||
{
|
||||
size_t buffer_size = settings.max_read_buffer_size;
|
||||
const String full_data_path = data_part->getFullRelativePath() + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION;
|
||||
|
@ -17,6 +17,7 @@ public:
|
||||
MergeTreeReaderCompact(
|
||||
DataPartCompactPtr data_part_,
|
||||
NamesAndTypesList columns_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
UncompressedCache * uncompressed_cache_,
|
||||
MarkCache * mark_cache_,
|
||||
MarkRanges mark_ranges_,
|
||||
|
@ -28,6 +28,7 @@ namespace ErrorCodes
|
||||
MergeTreeReaderWide::MergeTreeReaderWide(
|
||||
DataPartWidePtr data_part_,
|
||||
NamesAndTypesList columns_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
UncompressedCache * uncompressed_cache_,
|
||||
MarkCache * mark_cache_,
|
||||
MarkRanges mark_ranges_,
|
||||
@ -36,8 +37,14 @@ MergeTreeReaderWide::MergeTreeReaderWide(
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
|
||||
clockid_t clock_type_)
|
||||
: IMergeTreeReader(
|
||||
std::move(data_part_), std::move(columns_), uncompressed_cache_, std::move(mark_cache_),
|
||||
std::move(mark_ranges_), std::move(settings_), std::move(avg_value_size_hints_))
|
||||
std::move(data_part_),
|
||||
std::move(columns_),
|
||||
metadata_snapshot_,
|
||||
uncompressed_cache_,
|
||||
std::move(mark_cache_),
|
||||
std::move(mark_ranges_),
|
||||
std::move(settings_),
|
||||
std::move(avg_value_size_hints_))
|
||||
{
|
||||
try
|
||||
{
|
||||
|
@ -17,6 +17,7 @@ public:
|
||||
MergeTreeReaderWide(
|
||||
DataPartWidePtr data_part_,
|
||||
NamesAndTypesList columns_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
UncompressedCache * uncompressed_cache_,
|
||||
MarkCache * mark_cache_,
|
||||
MarkRanges mark_ranges_,
|
||||
|
@ -34,6 +34,7 @@ static Block replaceTypes(Block && header, const MergeTreeData::DataPartPtr & da
|
||||
|
||||
MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor(
|
||||
const MergeTreeData & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const MergeTreeData::DataPartPtr & owned_data_part_,
|
||||
UInt64 max_block_size_rows_,
|
||||
size_t preferred_block_size_bytes_,
|
||||
@ -49,8 +50,8 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor(
|
||||
bool quiet)
|
||||
:
|
||||
MergeTreeBaseSelectProcessor{
|
||||
replaceTypes(storage_.getSampleBlockForColumns(required_columns_), owned_data_part_),
|
||||
storage_, prewhere_info_, max_block_size_rows_,
|
||||
replaceTypes(metadata_snapshot_->getSampleBlockForColumns(required_columns_, storage_.getVirtuals(), storage_.getStorageID()), owned_data_part_),
|
||||
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_},
|
||||
required_columns{std::move(required_columns_)},
|
||||
@ -75,7 +76,7 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor(
|
||||
|
||||
ordered_names = header_without_virtual_columns.getNames();
|
||||
|
||||
task_columns = getReadTaskColumns(storage, data_part, required_columns, prewhere_info, check_columns);
|
||||
task_columns = getReadTaskColumns(storage, metadata_snapshot, data_part, required_columns, prewhere_info, check_columns);
|
||||
|
||||
/// will be used to distinguish between PREWHERE and WHERE columns when applying filter
|
||||
const auto & column_names = task_columns.columns.getNames();
|
||||
@ -86,11 +87,12 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor(
|
||||
|
||||
owned_mark_cache = storage.global_context.getMarkCache();
|
||||
|
||||
reader = data_part->getReader(task_columns.columns, all_mark_ranges,
|
||||
owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings);
|
||||
reader = data_part->getReader(task_columns.columns, metadata_snapshot,
|
||||
all_mark_ranges, owned_uncompressed_cache.get(),
|
||||
owned_mark_cache.get(), reader_settings);
|
||||
|
||||
if (prewhere_info)
|
||||
pre_reader = data_part->getReader(task_columns.pre_columns, all_mark_ranges,
|
||||
pre_reader = data_part->getReader(task_columns.pre_columns, metadata_snapshot, all_mark_ranges,
|
||||
owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings);
|
||||
}
|
||||
|
||||
@ -114,7 +116,7 @@ try
|
||||
|
||||
auto size_predictor = (preferred_block_size_bytes == 0)
|
||||
? nullptr
|
||||
: std::make_unique<MergeTreeBlockSizePredictor>(data_part, ordered_names, data_part->storage.getSampleBlock());
|
||||
: std::make_unique<MergeTreeBlockSizePredictor>(data_part, ordered_names, metadata_snapshot->getSampleBlock());
|
||||
|
||||
task = std::make_unique<MergeTreeReadTask>(
|
||||
data_part, mark_ranges_for_task, part_index_in_query, ordered_names, column_name_set,
|
||||
|
@ -18,6 +18,7 @@ class MergeTreeReverseSelectProcessor : public MergeTreeBaseSelectProcessor
|
||||
public:
|
||||
MergeTreeReverseSelectProcessor(
|
||||
const MergeTreeData & storage,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const MergeTreeData::DataPartPtr & owned_data_part,
|
||||
UInt64 max_block_size_rows,
|
||||
size_t preferred_block_size_bytes,
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user