2012-07-16 20:25:19 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-06-06 17:18:32 +00:00
|
|
|
#include <ext/shared_ptr_helper.h>
|
2016-08-26 21:25:05 +00:00
|
|
|
|
2018-03-06 20:18:34 +00:00
|
|
|
#include <Storages/IStorage.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeData.h>
|
|
|
|
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
|
|
|
#include <Storages/MergeTree/MergeTreeDataWriter.h>
|
|
|
|
#include <Storages/MergeTree/MergeTreeDataMerger.h>
|
|
|
|
#include <Storages/MergeTree/DiskSpaceMonitor.h>
|
|
|
|
#include <Storages/MergeTree/BackgroundProcessingPool.h>
|
2017-05-10 06:49:19 +00:00
|
|
|
#include <Common/SimpleIncrement.h>
|
2015-04-16 06:12:35 +00:00
|
|
|
|
2012-07-16 20:25:19 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** See the description of the data structure in MergeTreeData.
|
2012-07-16 20:25:19 +00:00
|
|
|
*/
|
2017-06-06 18:36:13 +00:00
|
|
|
class StorageMergeTree : public ext::shared_ptr_helper<StorageMergeTree>, public IStorage
|
2012-07-16 20:25:19 +00:00
|
|
|
{
|
2014-03-13 12:48:07 +00:00
|
|
|
friend class MergeTreeBlockOutputStream;
|
|
|
|
|
2012-07-16 20:25:19 +00:00
|
|
|
public:
|
2017-06-06 17:06:14 +00:00
|
|
|
void startup() override;
|
2017-04-01 07:20:54 +00:00
|
|
|
void shutdown() override;
|
|
|
|
~StorageMergeTree() override;
|
|
|
|
|
|
|
|
std::string getName() const override
|
|
|
|
{
|
|
|
|
return data.merging_params.getModeName() + "MergeTree";
|
|
|
|
}
|
|
|
|
|
|
|
|
std::string getTableName() const override { return table_name; }
|
2018-03-19 14:29:40 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
bool supportsSampling() const override { return data.supportsSampling(); }
|
|
|
|
bool supportsPrewhere() const override { return data.supportsPrewhere(); }
|
2018-03-19 14:29:40 +00:00
|
|
|
bool supportsFinal() const override { return data.supportsFinal(); }
|
|
|
|
bool supportsIndexForIn() const override { return true; }
|
|
|
|
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand) const override { return data.mayBenefitFromIndexForIn(left_in_operand); }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-03-13 14:18:11 +00:00
|
|
|
const ColumnsDescription & getColumns() const override { return data.getColumns(); }
|
|
|
|
void setColumns(ColumnsDescription columns_) override { return data.setColumns(std::move(columns_)); }
|
|
|
|
|
2017-12-25 21:57:29 +00:00
|
|
|
NameAndTypePair getColumn(const String & column_name) const override
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
return data.getColumn(column_name);
|
|
|
|
}
|
|
|
|
|
|
|
|
bool hasColumn(const String & column_name) const override
|
|
|
|
{
|
|
|
|
return data.hasColumn(column_name);
|
|
|
|
}
|
|
|
|
|
|
|
|
BlockInputStreams read(
|
|
|
|
const Names & column_names,
|
2017-07-15 03:48:36 +00:00
|
|
|
const SelectQueryInfo & query_info,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & context,
|
|
|
|
QueryProcessingStage::Enum & processed_stage,
|
2017-06-02 15:54:39 +00:00
|
|
|
size_t max_block_size,
|
|
|
|
unsigned num_streams) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-05-21 22:25:25 +00:00
|
|
|
BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-04-16 15:00:33 +00:00
|
|
|
/** Perform the next step in combining the parts.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2017-09-06 20:34:26 +00:00
|
|
|
bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-09-06 20:34:26 +00:00
|
|
|
void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context) override;
|
|
|
|
void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context) override;
|
|
|
|
void attachPartition(const ASTPtr & partition, bool part, const Context & context) override;
|
|
|
|
void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
void drop() override;
|
|
|
|
|
|
|
|
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override;
|
|
|
|
|
|
|
|
void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) override;
|
|
|
|
|
|
|
|
bool checkTableCanBeDropped() const override;
|
|
|
|
|
|
|
|
MergeTreeData & getData() { return data; }
|
|
|
|
const MergeTreeData & getData() const { return data; }
|
2014-07-29 15:21:03 +00:00
|
|
|
|
2018-02-21 19:26:59 +00:00
|
|
|
String getDataPath() const override { return full_path; }
|
|
|
|
|
2012-07-16 20:25:19 +00:00
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
String path;
|
|
|
|
String database_name;
|
|
|
|
String table_name;
|
|
|
|
String full_path;
|
|
|
|
|
|
|
|
Context & context;
|
|
|
|
BackgroundProcessingPool & background_pool;
|
|
|
|
|
|
|
|
MergeTreeData data;
|
|
|
|
MergeTreeDataSelectExecutor reader;
|
|
|
|
MergeTreeDataWriter writer;
|
|
|
|
MergeTreeDataMerger merger;
|
|
|
|
|
|
|
|
/// For block numbers.
|
|
|
|
SimpleIncrement increment{0};
|
|
|
|
|
|
|
|
/// For clearOldParts, clearOldTemporaryDirectories.
|
2018-02-25 02:43:27 +00:00
|
|
|
AtomicStopwatch time_after_previous_cleanup;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
MergeTreeData::DataParts currently_merging;
|
|
|
|
std::mutex currently_merging_mutex;
|
|
|
|
|
|
|
|
Logger * log;
|
|
|
|
|
|
|
|
std::atomic<bool> shutdown_called {false};
|
|
|
|
|
|
|
|
BackgroundProcessingPool::TaskHandle merge_task_handle;
|
|
|
|
|
|
|
|
friend struct CurrentlyMergingPartsTagger;
|
|
|
|
|
2017-11-04 03:20:18 +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.
|
|
|
|
*/
|
2018-01-12 17:30:21 +00:00
|
|
|
bool merge(size_t aio_threshold, bool aggressive, const String & partition_id, bool final, bool deduplicate,
|
|
|
|
String * out_disable_reason = nullptr);
|
2017-11-04 03:20:18 +00:00
|
|
|
|
|
|
|
bool mergeTask();
|
|
|
|
|
|
|
|
protected:
|
2017-06-06 18:36:13 +00:00
|
|
|
/** Attach the table with the appropriate name, along the appropriate path (with / at the end),
|
|
|
|
* (correctness of names and paths are not checked)
|
|
|
|
* consisting of the specified columns.
|
|
|
|
*
|
|
|
|
* primary_expr_ast - expression for sorting;
|
2017-09-08 18:11:09 +00:00
|
|
|
* date_column_name - if not empty, the name of the column with the date used for partitioning by month;
|
|
|
|
otherwise, partition_expr_ast is used as the partitioning expression;
|
2017-06-06 18:36:13 +00:00
|
|
|
*/
|
2017-04-01 07:20:54 +00:00
|
|
|
StorageMergeTree(
|
|
|
|
const String & path_,
|
|
|
|
const String & database_name_,
|
|
|
|
const String & table_name_,
|
2018-03-06 20:18:34 +00:00
|
|
|
const ColumnsDescription & columns_,
|
2017-04-01 07:20:54 +00:00
|
|
|
bool attach,
|
|
|
|
Context & context_,
|
2017-09-08 13:17:38 +00:00
|
|
|
const ASTPtr & primary_expr_ast_,
|
2018-02-09 10:53:50 +00:00
|
|
|
const ASTPtr & secondary_sorting_expr_list_,
|
2017-09-08 18:11:09 +00:00
|
|
|
const String & date_column_name,
|
|
|
|
const ASTPtr & partition_expr_ast_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported.
|
|
|
|
const MergeTreeData::MergingParams & merging_params_,
|
2017-09-19 20:42:42 +00:00
|
|
|
const MergeTreeSettings & settings_,
|
|
|
|
bool has_force_restore_data_flag);
|
2012-07-16 20:25:19 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|