ClickHouse/dbms/include/DB/Storages/StorageMergeTree.h

169 lines
5.8 KiB
C++
Raw Normal View History

2012-07-16 20:25:19 +00:00
#pragma once
#include <ext/shared_ptr_helper.hpp>
2014-03-09 17:36:01 +00:00
#include <DB/Storages/MergeTree/MergeTreeData.h>
2014-04-03 11:48:28 +00:00
#include <DB/Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <DB/Storages/MergeTree/MergeTreeDataWriter.h>
#include <DB/Storages/MergeTree/MergeTreeDataMerger.h>
#include <DB/Storages/MergeTree/DiskSpaceMonitor.h>
2015-04-16 06:12:35 +00:00
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
2015-10-05 01:11:12 +00:00
#include <DB/Common/Increment.h>
2015-04-16 06:12:35 +00:00
2012-07-16 20:25:19 +00:00
namespace DB
{
2014-03-09 17:36:01 +00:00
/** См. описание структуры данных в MergeTreeData.
2012-07-16 20:25:19 +00:00
*/
class StorageMergeTree : private ext::shared_ptr_helper<StorageMergeTree>, public IStorage
2012-07-16 20:25:19 +00:00
{
friend class ext::shared_ptr_helper<StorageMergeTree>;
2014-03-13 12:48:07 +00:00
friend class MergeTreeBlockOutputStream;
2012-07-16 20:25:19 +00:00
public:
/** Подцепить таблицу с соответствующим именем, по соответствующему пути (с / на конце),
* (корректность имён и путей не проверяется)
* состоящую из указанных столбцов.
*
2012-11-30 00:52:45 +00:00
* primary_expr_ast - выражение для сортировки;
2012-07-16 20:25:19 +00:00
* date_column_name - имя столбца с датой;
* index_granularity - на сколько строчек пишется одно значение индекса.
*/
static StoragePtr create(
const String & path_,
const String & database_name_,
const String & table_name_,
2014-05-08 07:12:01 +00:00
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
2014-07-02 12:30:38 +00:00
Context & context_,
ASTPtr & primary_expr_ast_,
const String & date_column_name_,
2014-04-08 07:58:53 +00:00
const ASTPtr & sampling_expression_, /// nullptr, если семплирование не поддерживается.
2012-07-31 16:37:20 +00:00
size_t index_granularity_,
2016-04-15 17:13:51 +00:00
const MergeTreeData::MergingParams & merging_params_,
bool has_force_restore_data_flag,
2015-07-16 21:03:53 +00:00
const MergeTreeSettings & settings_);
2014-11-12 10:37:47 +00:00
void shutdown() override;
~StorageMergeTree() override;
2012-07-30 20:32:36 +00:00
std::string getName() const override
2013-09-30 19:54:25 +00:00
{
2016-04-24 09:44:47 +00:00
return data.merging_params.getModeName() + "MergeTree";
2013-09-30 19:54:25 +00:00
}
std::string getTableName() const override { return table_name; }
bool supportsSampling() const override { return data.supportsSampling(); }
bool supportsFinal() const override { return data.supportsFinal(); }
bool supportsPrewhere() const override { return data.supportsPrewhere(); }
bool supportsParallelReplicas() const override { return true; }
2012-07-16 20:25:19 +00:00
const NamesAndTypesList & getColumnsListImpl() const override { return data.getColumnsListNonMaterialized(); }
2012-07-16 20:25:19 +00:00
NameAndTypePair getColumn(const String & column_name) const override
2014-07-28 10:36:11 +00:00
{
return data.getColumn(column_name);
}
bool hasColumn(const String & column_name) const override
2014-07-28 10:36:11 +00:00
{
return data.hasColumn(column_name);
}
2012-07-21 03:45:48 +00:00
BlockInputStreams read(
2012-07-16 20:25:19 +00:00
const Names & column_names,
ASTPtr query,
const Context & context,
const Settings & settings,
2012-07-16 20:25:19 +00:00
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override;
2012-07-16 20:25:19 +00:00
BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override;
2012-07-16 20:25:19 +00:00
/** Выполнить очередной шаг объединения кусков.
*/
bool optimize(const String & partition, bool final, const Settings & settings) override
{
return merge(settings.min_bytes_to_use_direct_io, true, partition, final);
}
void dropPartition(ASTPtr query, const Field & partition, bool detach, bool unreplicated, const Settings & settings) override;
void attachPartition(ASTPtr query, const Field & partition, bool unreplicated, bool part, const Settings & settings) override;
void freezePartition(const Field & partition, const String & with_name, const Settings & settings) override;
2014-10-03 17:57:01 +00:00
2014-03-20 13:28:49 +00:00
void drop() override;
2014-03-09 17:36:01 +00:00
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override;
2012-07-16 20:25:19 +00:00
2016-01-28 01:00:27 +00:00
void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) override;
2013-08-07 13:07:42 +00:00
2014-04-12 15:49:36 +00:00
bool supportsIndexForIn() const override { return true; }
2014-04-10 08:57:01 +00:00
bool checkTableCanBeDropped() const override;
MergeTreeData & getData() { return data; }
const MergeTreeData & getData() const { return data; }
2012-07-16 20:25:19 +00:00
private:
2014-03-13 12:48:07 +00:00
String path;
2014-09-10 11:34:26 +00:00
String database_name;
String table_name;
2014-03-13 12:48:07 +00:00
String full_path;
2014-09-10 11:34:26 +00:00
Context & context;
2014-07-02 12:30:38 +00:00
BackgroundProcessingPool & background_pool;
2014-03-09 17:36:01 +00:00
MergeTreeData data;
2014-03-13 12:48:07 +00:00
MergeTreeDataSelectExecutor reader;
MergeTreeDataWriter writer;
MergeTreeDataMerger merger;
/// For block numbers.
Squashed commit of the following: commit e712f469a55ff34ad34b482b15cc4153b7ad7233 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:59:13 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a002823084e3a79bffcc17d479620a68eb0644b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:58:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9e06f407c8ee781ed8ddf98bdfcc31846bf2a0fe Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:55:14 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9581620f1e839f456fa7894aa1f996d5162ac6cd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:54:22 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a8564c68cb6cc3649fafaf401256d43c9a2e777 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:47:34 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit cf60632d78ec656be3304ef4565e859bb6ce80ba Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:40:09 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit ee3d1dc6e0c4ca60e3ac1e0c30d4b3ed1e66eca0 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:22:49 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 65592ef7116a90104fcd524b53ef8b7cf22640f2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:18:17 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 37972c257320d3b7e7b294e0fdeffff218647bfd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:17:06 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit dd909d149974ce5bed2456de1261aa5a368fd3ff Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:16:28 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 3cf43266ca7e30adf01212b1a739ba5fe43639fd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:15:42 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 6731a3df96d1609286e2536b6432916af7743f0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:13:35 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 1b5727e0d56415b7add4cb76110105358663602c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:11:18 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit bbcf726a55685b8e72f5b40ba0bf1904bd1c0407 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:09:04 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit c03b477d5e2e65014e8906ecfa2efb67ee295af1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:06:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2986e2fb0466bc18d73693dcdded28fccc0dc66b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:05:44 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 5d6cdef13d2e02bd5c4954983334e9162ab2635b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:04:53 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit f2b819b25ce8b2ccdcb201eefb03e1e6f5aab590 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:01:47 2017 +0300 Less dependencies [#CLICKHOUSE-2]
2017-01-14 09:00:19 +00:00
SimpleIncrement increment{0};
2015-06-02 20:22:53 +00:00
/// For clearOldParts, clearOldTemporaryDirectories.
StopwatchWithLock time_after_previous_cleanup;
2014-03-13 17:44:00 +00:00
MergeTreeData::DataParts currently_merging;
std::mutex currently_merging_mutex;
2014-03-13 17:44:00 +00:00
2014-03-13 12:48:07 +00:00
Logger * log;
2016-07-31 03:53:16 +00:00
std::atomic<bool> shutdown_called {false};
2014-04-11 13:05:17 +00:00
BackgroundProcessingPool::TaskHandle merge_task_handle;
2014-03-13 17:44:00 +00:00
2016-09-02 04:03:40 +00:00
friend struct CurrentlyMergingPartsTagger;
2014-03-13 17:44:00 +00:00
StorageMergeTree(
const String & path_,
const String & database_name_,
const String & table_name_,
NamesAndTypesListPtr columns_,
const NamesAndTypesList & materialized_columns_,
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_,
Context & context_,
ASTPtr & primary_expr_ast_,
const String & date_column_name_,
2016-08-13 01:59:09 +00:00
const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported.
size_t index_granularity_,
2016-04-15 17:13:51 +00:00
const MergeTreeData::MergingParams & merging_params_,
bool has_force_restore_data_flag,
const MergeTreeSettings & settings_);
2014-03-13 12:48:07 +00:00
/** Determines what parts should be merged and merges it.
* If aggressive - when selects parts don't takes into account their ratio size and novelty (used for OPTIMIZE query).
* Returns true if merge is finished successfully.
2014-03-13 12:48:07 +00:00
*/
bool merge(size_t aio_threshold, bool aggressive, const String & partition, bool final);
2014-03-13 12:48:07 +00:00
bool mergeTask();
2012-07-16 20:25:19 +00:00
};
}