提交 96f33e92 编写于 作者: A Alexey Arno

dbms: Server: Code cleanups. [#METR-15090]

上级 9bcb33c1
......@@ -24,7 +24,7 @@ public:
const ASTOptimizeQuery & ast = typeid_cast<const ASTOptimizeQuery &>(*query_ptr);
StoragePtr table = context.getTable(ast.database, ast.table);
auto table_lock = table->lockStructure(true);
table->optimize(&context.getSettingsRef());
table->optimize(context.getSettings());
}
private:
......
......@@ -236,9 +236,9 @@ public:
/** Выполнить какую-либо фоновую работу. Например, объединение кусков в таблице типа MergeTree.
* Возвращает - была ли выполнена какая-либо работа.
*/
bool optimize(const Settings * settings = nullptr)
virtual bool optimize(const Settings & settings)
{
return performOptimize(settings);
throw Exception("Method optimize is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/** Получить запрос CREATE TABLE, который описывает данную таблицу.
......@@ -279,12 +279,6 @@ public:
/// проверяет валидность данных
virtual bool checkData() const { throw DB::Exception("Check query is not supported for " + getName() + " storage"); }
protected:
virtual bool performOptimize(const Settings * settings)
{
throw Exception("Method optimize is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
protected:
using ITableDeclaration::ITableDeclaration;
......
......@@ -70,6 +70,7 @@ public:
/// Сбрасывает все буферы в подчинённую таблицу.
void shutdown() override;
bool optimize(const Settings & settings) override;
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; }
......@@ -123,8 +124,6 @@ private:
void writeBlockToDestination(const Block & block, StoragePtr table);
void flushThread();
bool performOptimize(const Settings * settings) override;
};
}
......@@ -32,6 +32,7 @@ public:
BlockOutputStreamPtr write(ASTPtr query) override;
void drop() override;
bool optimize(const Settings & settings) override;
BlockInputStreams read(
const Names & column_names,
......@@ -55,8 +56,6 @@ private:
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
bool attach_);
bool performOptimize(const Settings * settings) override;
};
}
......@@ -79,6 +79,13 @@ public:
BlockOutputStreamPtr write(ASTPtr query) override;
/** Выполнить очередной шаг объединения кусков.
*/
bool optimize(const Settings & settings) override
{
return merge(settings.min_bytes_to_use_direct_io, true);
}
void dropPartition(const Field & partition, bool detach, const Settings & settings) override;
void attachPartition(const Field & partition, bool unreplicated, bool part, const Settings & settings) override;
void freezePartition(const Field & partition, const Settings & settings) override;
......@@ -93,15 +100,6 @@ public:
MergeTreeData & getData() { return data; }
private:
/** Выполнить очередной шаг объединения кусков.
*/
bool performOptimize(const Settings * settings) override
{
const auto & applied_settings = (settings != nullptr) ? *settings : context.getSettings();
return merge(applied_settings.min_bytes_to_use_direct_io, true);
}
private:
String path;
String database_name;
......
......@@ -83,6 +83,8 @@ public:
BlockOutputStreamPtr write(ASTPtr query) override;
bool optimize(const Settings & settings) override;
void alter(const AlterCommands & params, const String & database_name, const String & table_name, Context & context) override;
void dropPartition(const Field & partition, bool detach, const Settings & settings) override;
......@@ -386,8 +388,6 @@ private:
*/
void waitForReplicaToProcessLogEntry(const String & replica_name, const LogEntry & entry);
bool performOptimize(const Settings * settings) override;
/// Преобразовать число в строку формате суффиксов автоинкрементных нод в ZooKeeper.
static String padIndex(UInt64 index)
{
......
......@@ -265,11 +265,11 @@ void StorageBuffer::shutdown()
if (flush_thread.joinable())
flush_thread.join();
optimize();
optimize(context.getSettings());
}
bool StorageBuffer::performOptimize(const Settings * settings)
bool StorageBuffer::optimize(const Settings & settings)
{
flushAllBuffers(false);
......@@ -442,7 +442,7 @@ void StorageBuffer::alter(const AlterCommands & params, const String & database_
auto lock = lockStructureForAlter();
/// Чтобы не осталось блоков старой структуры.
optimize();
optimize(context.getSettings());
params.apply(*columns, materialized_columns, alias_columns, column_defaults);
InterpreterAlterQuery::updateMetadata(database_name, table_name,
......
......@@ -129,7 +129,7 @@ void StorageMaterializedView::drop()
}
}
bool StorageMaterializedView::performOptimize(const Settings * settings)
bool StorageMaterializedView::optimize(const Settings & settings)
{
return data->optimize(settings);
}
......
......@@ -2060,7 +2060,7 @@ BlockOutputStreamPtr StorageReplicatedMergeTree::write(ASTPtr query)
}
bool StorageReplicatedMergeTree::performOptimize(const Settings * settings)
bool StorageReplicatedMergeTree::optimize(const Settings & settings)
{
/// Померджим какие-нибудь куски из директории unreplicated.
/// TODO: Мерджить реплицируемые куски тоже.
......@@ -2079,9 +2079,7 @@ bool StorageReplicatedMergeTree::performOptimize(const Settings * settings)
return false;
const auto & merge_entry = context.getMergeList().insert(database_name, table_name, merged_name);
const auto & applied_settings = (settings != nullptr) ? *settings : context.getSettings();
unreplicated_merger->mergeParts(parts, merged_name, *merge_entry, applied_settings.min_bytes_to_use_direct_io);
unreplicated_merger->mergeParts(parts, merged_name, *merge_entry, settings.min_bytes_to_use_direct_io);
return true;
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册