TTL for columns and tables (#4212)

Add TTL for columns and tables.
This commit is contained in:
Anton Popov 2019-04-15 12:30:45 +03:00 committed by alesapin
parent 80cb0f0636
commit 7f93f9ad2f
68 changed files with 1588 additions and 361 deletions

View File

@ -424,6 +424,8 @@ namespace ErrorCodes
extern const int HYPERSCAN_CANNOT_SCAN_TEXT = 447;
extern const int BROTLI_READ_FAILED = 448;
extern const int BROTLI_WRITE_FAILED = 449;
extern const int BAD_TTL_EXPRESSION = 450;
extern const int BAD_TTL_FILE = 451;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -0,0 +1,208 @@
#include <DataStreams/TTLBlockInputStream.h>
#include <DataTypes/DataTypeDate.h>
#include <Interpreters/evaluateMissingDefaults.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/ExpressionAnalyzer.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
TTLBlockInputStream::TTLBlockInputStream(
const BlockInputStreamPtr & input_,
const MergeTreeData & storage_,
const MergeTreeData::MutableDataPartPtr & data_part_,
time_t current_time_)
: storage(storage_)
, data_part(data_part_)
, current_time(current_time_)
, old_ttl_infos(data_part->ttl_infos)
, log(&Logger::get(storage.getLogName() + " (TTLBlockInputStream)"))
, date_lut(DateLUT::instance())
{
children.push_back(input_);
const auto & column_defaults = storage.getColumns().getDefaults();
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
for (const auto & [name, ttl_info] : old_ttl_infos.columns_ttl)
{
if (ttl_info.min <= current_time)
{
new_ttl_infos.columns_ttl.emplace(name, MergeTreeDataPart::TTLInfo{});
empty_columns.emplace(name);
auto it = column_defaults.find(name);
if (it != column_defaults.end())
default_expr_list->children.emplace_back(
setAlias(it->second.expression, it->first));
}
else
new_ttl_infos.columns_ttl.emplace(name, ttl_info);
}
if (old_ttl_infos.table_ttl.min > current_time)
new_ttl_infos.table_ttl = old_ttl_infos.table_ttl;
if (!default_expr_list->children.empty())
{
auto syntax_result = SyntaxAnalyzer(storage.global_context).analyze(
default_expr_list, storage.getColumns().getAllPhysical());
defaults_expression = ExpressionAnalyzer{default_expr_list, syntax_result, storage.global_context}.getActions(true);
}
}
Block TTLBlockInputStream::getHeader() const
{
return children.at(0)->getHeader();
}
Block TTLBlockInputStream::readImpl()
{
Block block = children.at(0)->read();
if (!block)
return block;
if (storage.hasTableTTL())
{
/// Skip all data if table ttl is expired for part
if (old_ttl_infos.table_ttl.max <= current_time)
{
rows_removed = data_part->rows_count;
return {};
}
if (old_ttl_infos.table_ttl.min <= current_time)
removeRowsWithExpiredTableTTL(block);
}
removeValuesWithExpiredColumnTTL(block);
return block;
}
void TTLBlockInputStream::readSuffixImpl()
{
for (const auto & elem : new_ttl_infos.columns_ttl)
new_ttl_infos.updatePartMinTTL(elem.second.min);
new_ttl_infos.updatePartMinTTL(new_ttl_infos.table_ttl.min);
data_part->ttl_infos = std::move(new_ttl_infos);
data_part->empty_columns = std::move(empty_columns);
if (rows_removed)
LOG_INFO(log, "Removed " << rows_removed << " rows with expired ttl from part " << data_part->name);
}
void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block)
{
storage.ttl_table_entry.expression->execute(block);
const auto & current = block.getByName(storage.ttl_table_entry.result_column);
const IColumn * ttl_column = current.column.get();
MutableColumns result_columns;
result_columns.reserve(getHeader().columns());
for (const auto & name : storage.getColumns().getNamesOfPhysical())
{
auto & column_with_type = block.getByName(name);
const IColumn * values_column = column_with_type.column.get();
MutableColumnPtr result_column = values_column->cloneEmpty();
result_column->reserve(block.rows());
for (size_t i = 0; i < block.rows(); ++i)
{
UInt32 cur_ttl = getTimestampByIndex(ttl_column, i);
if (cur_ttl > current_time)
{
new_ttl_infos.table_ttl.update(cur_ttl);
result_column->insertFrom(*values_column, i);
}
else
++rows_removed;
}
result_columns.emplace_back(std::move(result_column));
}
block = getHeader().cloneWithColumns(std::move(result_columns));
}
void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block)
{
Block block_with_defaults;
if (defaults_expression)
{
block_with_defaults = block;
defaults_expression->execute(block_with_defaults);
}
for (const auto & [name, ttl_entry] : storage.ttl_entries_by_name)
{
const auto & old_ttl_info = old_ttl_infos.columns_ttl[name];
auto & new_ttl_info = new_ttl_infos.columns_ttl[name];
if (old_ttl_info.min > current_time)
continue;
if (old_ttl_info.max <= current_time)
continue;
if (!block.has(ttl_entry.result_column))
ttl_entry.expression->execute(block);
ColumnPtr default_column = nullptr;
if (block_with_defaults.has(name))
default_column = block_with_defaults.getByName(name).column->convertToFullColumnIfConst();
auto & column_with_type = block.getByName(name);
const IColumn * values_column = column_with_type.column.get();
MutableColumnPtr result_column = values_column->cloneEmpty();
result_column->reserve(block.rows());
const auto & current = block.getByName(ttl_entry.result_column);
const IColumn * ttl_column = current.column.get();
for (size_t i = 0; i < block.rows(); ++i)
{
UInt32 cur_ttl = getTimestampByIndex(ttl_column, i);
if (cur_ttl <= current_time)
{
if (default_column)
result_column->insertFrom(*default_column, i);
else
result_column->insertDefault();
}
else
{
new_ttl_info.update(cur_ttl);
empty_columns.erase(name);
result_column->insertFrom(*values_column, i);
}
}
column_with_type.column = std::move(result_column);
}
for (const auto & elem : storage.ttl_entries_by_name)
if (block.has(elem.second.result_column))
block.erase(elem.second.result_column);
}
UInt32 TTLBlockInputStream::getTimestampByIndex(const IColumn * column, size_t ind)
{
if (const ColumnUInt16 * column_date = typeid_cast<const ColumnUInt16 *>(column))
return date_lut.fromDayNum(DayNum(column_date->getData()[ind]));
else if (const ColumnUInt32 * column_date_time = typeid_cast<const ColumnUInt32 *>(column))
return column_date_time->getData()[ind];
else
throw Exception("Unexpected type of result ttl column", ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -0,0 +1,60 @@
#pragma once
#include <DataStreams/IBlockInputStream.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeDataPart.h>
#include <Core/Block.h>
#include <common/DateLUT.h>
namespace DB
{
class TTLBlockInputStream : public IBlockInputStream
{
public:
TTLBlockInputStream(
const BlockInputStreamPtr & input_,
const MergeTreeData & storage_,
const MergeTreeData::MutableDataPartPtr & data_part_,
time_t current_time
);
String getName() const override { return "TTLBlockInputStream"; }
Block getHeader() const override;
protected:
Block readImpl() override;
/// Finalizes ttl infos and updates data part
void readSuffixImpl() override;
private:
const MergeTreeData & storage;
/// ttl_infos and empty_columns are updating while reading
const MergeTreeData::MutableDataPartPtr & data_part;
time_t current_time;
MergeTreeDataPart::TTLInfos old_ttl_infos;
MergeTreeDataPart::TTLInfos new_ttl_infos;
NameSet empty_columns;
size_t rows_removed = 0;
Logger * log;
DateLUTImpl date_lut;
std::unordered_map<String, String> defaults_result_column;
ExpressionActionsPtr defaults_expression;
private:
/// Removes values with expired ttl and computes new min_ttl and empty_columns for part
void removeValuesWithExpiredColumnTTL(Block & block);
/// Remove rows with expired table ttl and computes new min_ttl for part
void removeRowsWithExpiredTableTTL(Block & block);
UInt32 getTimestampByIndex(const IColumn * column, size_t ind);
};
}

View File

@ -233,6 +233,9 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
column_declaration->codec = parseQuery(codec_p, codec_desc_pos, codec_desc_end, "column codec", 0);
}
if (column.ttl)
column_declaration->ttl = column.ttl;
columns_list->children.push_back(column_declaration_ptr);
}
@ -347,6 +350,9 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpres
if (col_decl.codec)
column.codec = CompressionCodecFactory::instance().get(col_decl.codec, column.type);
if (col_decl.ttl)
column.ttl = col_decl.ttl;
res.add(std::move(column));
}

View File

@ -52,6 +52,9 @@ Block InterpreterDescribeQuery::getSampleBlock()
col.name = "codec_expression";
block.insert(col);
col.name = "ttl_expression";
block.insert(col);
return block;
}
@ -118,6 +121,11 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
res_columns[5]->insert(column.codec->getCodecDesc());
else
res_columns[5]->insertDefault();
if (column.ttl)
res_columns[6]->insert(queryToString(column.ttl));
else
res_columns[6]->insertDefault();
}
return std::make_shared<OneBlockInputStream>(sample_block.cloneWithColumns(std::move(res_columns)));

View File

@ -40,6 +40,11 @@ ASTPtr ASTAlterCommand::clone() const
res->predicate = predicate->clone();
res->children.push_back(res->predicate);
}
if (ttl)
{
res->ttl = ttl->clone();
res->children.push_back(res->ttl);
}
return res;
}
@ -174,6 +179,11 @@ void ASTAlterCommand::formatImpl(
settings.ostr << " " << (settings.hilite ? hilite_none : "");
comment->formatImpl(settings, state, frame);
}
else if (type == ASTAlterCommand::MODIFY_TTL)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY TTL " << (settings.hilite ? hilite_none : "");
ttl->formatImpl(settings, state, frame);
}
else
throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
}

View File

@ -27,6 +27,7 @@ public:
MODIFY_COLUMN,
COMMENT_COLUMN,
MODIFY_ORDER_BY,
MODIFY_TTL,
ADD_INDEX,
DROP_INDEX,
@ -84,6 +85,9 @@ public:
/// A column comment
ASTPtr comment;
/// For MODIFY TTL query
ASTPtr ttl;
bool detach = false; /// true for DETACH PARTITION
bool part = false; /// true for ATTACH PART

View File

@ -33,6 +33,12 @@ ASTPtr ASTColumnDeclaration::clone() const
res->children.push_back(res->comment);
}
if (ttl)
{
res->ttl = ttl->clone();
res->children.push_back(res->ttl);
}
return res;
}
@ -69,6 +75,12 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatSta
settings.ostr << ' ';
codec->formatImpl(settings, state, frame);
}
if (ttl)
{
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "TTL" << (settings.hilite ? hilite_none : "") << ' ';
ttl->formatImpl(settings, state, frame);
}
}
}

View File

@ -17,6 +17,7 @@ public:
ASTPtr default_expression;
ASTPtr codec;
ASTPtr comment;
ASTPtr ttl;
String getID(char delim) const override { return "ColumnDeclaration" + (delim + name); }

View File

@ -23,6 +23,8 @@ ASTPtr ASTStorage::clone() const
res->set(res->order_by, order_by->clone());
if (sample_by)
res->set(res->sample_by, sample_by->clone());
if (ttl_table)
res->set(res->ttl_table, ttl_table->clone());
if (settings)
res->set(res->settings, settings->clone());
@ -57,6 +59,11 @@ void ASTStorage::formatImpl(const FormatSettings & s, FormatState & state, Forma
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "SAMPLE BY " << (s.hilite ? hilite_none : "");
sample_by->formatImpl(s, state, frame);
}
if (ttl_table)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "TTL " << (s.hilite ? hilite_none : "");
ttl_table->formatImpl(s, state, frame);
}
if (settings)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "SETTINGS " << (s.hilite ? hilite_none : "");

View File

@ -18,6 +18,7 @@ public:
IAST * primary_key = nullptr;
IAST * order_by = nullptr;
IAST * sample_by = nullptr;
IAST * ttl_table = nullptr;
ASTSetQuery * settings = nullptr;
String getID(char) const override { return "Storage definition"; }

View File

@ -27,6 +27,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ParserKeyword s_modify_column("MODIFY COLUMN");
ParserKeyword s_comment_column("COMMENT COLUMN");
ParserKeyword s_modify_order_by("MODIFY ORDER BY");
ParserKeyword s_modify_ttl("MODIFY TTL");
ParserKeyword s_add_index("ADD INDEX");
ParserKeyword s_drop_index("DROP INDEX");
@ -282,6 +283,12 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
command->type = ASTAlterCommand::COMMENT_COLUMN;
}
else if (s_modify_ttl.ignore(pos, expected))
{
if (!parser_exp_elem.parse(pos, command->ttl, expected))
return false;
command->type = ASTAlterCommand::MODIFY_TTL;
}
else
return false;
@ -299,6 +306,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
command->children.push_back(command->update_assignments);
if (command->comment)
command->children.push_back(command->comment);
if (command->ttl)
command->children.push_back(command->ttl);
return true;
}

View File

@ -210,6 +210,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_primary_key("PRIMARY KEY");
ParserKeyword s_order_by("ORDER BY");
ParserKeyword s_sample_by("SAMPLE BY");
ParserKeyword s_ttl("TTL");
ParserKeyword s_settings("SETTINGS");
ParserIdentifierWithOptionalParameters ident_with_optional_params_p;
@ -221,6 +222,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ASTPtr primary_key;
ASTPtr order_by;
ASTPtr sample_by;
ASTPtr ttl_table;
ASTPtr settings;
if (!s_engine.ignore(pos, expected))
@ -265,6 +267,14 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
}
if (!ttl_table && s_ttl.ignore(pos, expected))
{
if (expression_p.parse(pos, ttl_table, expected))
continue;
else
return false;
}
if (s_settings.ignore(pos, expected))
{
if (!settings_p.parse(pos, settings, expected))
@ -280,6 +290,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
storage->set(storage->primary_key, primary_key);
storage->set(storage->order_by, order_by);
storage->set(storage->sample_by, sample_by);
storage->set(storage->ttl_table, ttl_table);
storage->set(storage->settings, settings);

View File

@ -123,9 +123,11 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
ParserKeyword s_alias{"ALIAS"};
ParserKeyword s_comment{"COMMENT"};
ParserKeyword s_codec{"CODEC"};
ParserKeyword s_ttl{"TTL"};
ParserTernaryOperatorExpression expr_parser;
ParserStringLiteral string_literal_parser;
ParserCodec codec_parser;
ParserExpression expression_parser;
/// mandatory column name
ASTPtr name;
@ -140,6 +142,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
ASTPtr default_expression;
ASTPtr comment_expression;
ASTPtr codec_expression;
ASTPtr ttl_expression;
if (!s_default.check_without_moving(pos, expected) &&
!s_materialized.check_without_moving(pos, expected) &&
@ -178,6 +181,12 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
return false;
}
if (s_ttl.ignore(pos, expected))
{
if (!expression_parser.parse(pos, ttl_expression, expected))
return false;
}
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
node = column_declaration;
getIdentifierName(name, column_declaration->name);
@ -207,6 +216,12 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
column_declaration->children.push_back(std::move(codec_expression));
}
if (ttl_expression)
{
column_declaration->ttl = ttl_expression;
column_declaration->children.push_back(std::move(ttl_expression));
}
return true;
}

View File

@ -17,6 +17,8 @@
#include <Common/typeid_cast.h>
#include <Compression/CompressionFactory.h>
#include <Parsers/queryToString.h>
namespace DB
{
@ -64,6 +66,9 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
if (command_ast->column)
command.after_column = *getIdentifierName(command_ast->column);
if (ast_col_decl.ttl)
command.ttl = ast_col_decl.ttl;
command.if_not_exists = command_ast->if_not_exists;
return command;
@ -104,6 +109,9 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
command.comment = ast_comment.value.get<String>();
}
if (ast_col_decl.ttl)
command.ttl = ast_col_decl.ttl;
if (ast_col_decl.codec)
command.codec = compression_codec_factory.get(ast_col_decl.codec, command.data_type);
@ -157,13 +165,20 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
return command;
}
else if (command_ast->type == ASTAlterCommand::MODIFY_TTL)
{
AlterCommand command;
command.type = AlterCommand::MODIFY_TTL;
command.ttl = command_ast->ttl;
return command;
}
else
return {};
}
void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
ASTPtr & order_by_ast, ASTPtr & primary_key_ast) const
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const
{
if (type == ADD_COLUMN)
{
@ -175,6 +190,7 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri
}
column.comment = comment;
column.codec = codec;
column.ttl = ttl;
columns_description.add(column, after_column);
@ -204,6 +220,9 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri
return;
}
if (ttl)
column.ttl = ttl;
column.type = data_type;
column.default_desc.kind = default_kind;
@ -278,6 +297,10 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri
indices_description.indices.erase(erase_it);
}
else if (type == MODIFY_TTL)
{
ttl_table_ast = ttl;
}
else
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
}
@ -293,20 +316,22 @@ bool AlterCommand::is_mutable() const
}
void AlterCommands::apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
ASTPtr & order_by_ast, ASTPtr & primary_key_ast) const
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const
{
auto new_columns_description = columns_description;
auto new_indices_description = indices_description;
auto new_order_by_ast = order_by_ast;
auto new_primary_key_ast = primary_key_ast;
auto new_ttl_table_ast = ttl_table_ast;
for (const AlterCommand & command : *this)
if (!command.ignore)
command.apply(new_columns_description, new_indices_description, new_order_by_ast, new_primary_key_ast);
command.apply(new_columns_description, new_indices_description, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
columns_description = std::move(new_columns_description);
indices_description = std::move(new_indices_description);
order_by_ast = std::move(new_order_by_ast);
primary_key_ast = std::move(new_primary_key_ast);
ttl_table_ast = std::move(new_ttl_table_ast);
}
void AlterCommands::validate(const IStorage & table, const Context & context)
@ -493,7 +518,8 @@ void AlterCommands::apply(ColumnsDescription & columns_description) const
IndicesDescription indices_description;
ASTPtr out_order_by;
ASTPtr out_primary_key;
apply(out_columns_description, indices_description, out_order_by, out_primary_key);
ASTPtr out_ttl_table;
apply(out_columns_description, indices_description, out_order_by, out_primary_key, out_ttl_table);
if (out_order_by)
throw Exception("Storage doesn't support modifying ORDER BY expression", ErrorCodes::NOT_IMPLEMENTED);
@ -501,6 +527,8 @@ void AlterCommands::apply(ColumnsDescription & columns_description) const
throw Exception("Storage doesn't support modifying PRIMARY KEY expression", ErrorCodes::NOT_IMPLEMENTED);
if (!indices_description.indices.empty())
throw Exception("Storage doesn't support modifying indices", ErrorCodes::NOT_IMPLEMENTED);
if (out_ttl_table)
throw Exception("Storage doesn't support modifying TTL expression", ErrorCodes::NOT_IMPLEMENTED);
columns_description = std::move(out_columns_description);
}

View File

@ -24,6 +24,7 @@ struct AlterCommand
MODIFY_ORDER_BY,
ADD_INDEX,
DROP_INDEX,
MODIFY_TTL,
UKNOWN_TYPE,
};
@ -60,6 +61,9 @@ struct AlterCommand
/// For ADD/DROP INDEX
String index_name;
/// For MODIFY TTL
ASTPtr ttl;
/// indicates that this command should not be applied, for example in case of if_exists=true and column doesn't exist.
bool ignore = false;
@ -79,7 +83,7 @@ struct AlterCommand
static std::optional<AlterCommand> parse(const ASTAlterCommand * command);
void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
ASTPtr & order_by_ast, ASTPtr & primary_key_ast) const;
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const;
/// Checks that not only metadata touched by that command
bool is_mutable() const;
};
@ -91,7 +95,7 @@ class AlterCommands : public std::vector<AlterCommand>
{
public:
void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description, ASTPtr & order_by_ast,
ASTPtr & primary_key_ast) const;
ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const;
/// For storages that don't support MODIFY_ORDER_BY.
void apply(ColumnsDescription & columns_description) const;

View File

@ -37,12 +37,14 @@ namespace ErrorCodes
bool ColumnDescription::operator==(const ColumnDescription & other) const
{
auto codec_str = [](const CompressionCodecPtr & codec_ptr) { return codec_ptr ? codec_ptr->getCodecDesc() : String(); };
auto ttl_str = [](const ASTPtr & ttl_ast) { return ttl_ast ? queryToString(ttl_ast) : String{}; };
return name == other.name
&& type->equals(*other.type)
&& default_desc == other.default_desc
&& comment == other.comment
&& codec_str(codec) == codec_str(other.codec);
&& codec_str(codec) == codec_str(other.codec)
&& ttl_str(ttl) == ttl_str(other.ttl);
}
void ColumnDescription::writeText(WriteBuffer & buf) const
@ -74,6 +76,13 @@ void ColumnDescription::writeText(WriteBuffer & buf) const
DB::writeText(")", buf);
}
if (ttl)
{
writeChar('\t', buf);
DB::writeText("TTL ", buf);
DB::writeText(queryToString(ttl), buf);
}
writeChar('\n', buf);
}
@ -99,6 +108,9 @@ void ColumnDescription::readText(ReadBuffer & buf)
if (col_ast->codec)
codec = CompressionCodecFactory::instance().get(col_ast->codec, type);
if (col_ast->ttl)
ttl = col_ast->ttl;
}
else
throw Exception("Cannot parse column description", ErrorCodes::CANNOT_PARSE_TEXT);
@ -388,6 +400,18 @@ CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_
return getCodecOrDefault(column_name, CompressionCodecFactory::instance().getDefaultCodec());
}
ColumnsDescription::ColumnTTLs ColumnsDescription::getColumnTTLs() const
{
ColumnTTLs ret;
for (const auto & column : columns)
{
if (column.ttl)
ret.emplace(column.name, column.ttl);
}
return ret;
}
String ColumnsDescription::toString() const
{

View File

@ -18,6 +18,7 @@ struct ColumnDescription
ColumnDefault default_desc;
String comment;
CompressionCodecPtr codec;
ASTPtr ttl;
ColumnDescription() = default;
ColumnDescription(String name_, DataTypePtr type_) : name(std::move(name_)), type(std::move(type_)) {}
@ -58,6 +59,9 @@ public:
/// ordinary + materialized + aliases.
NamesAndTypesList getAll() const;
using ColumnTTLs = std::unordered_map<String, ASTPtr>;
ColumnTTLs getColumnTTLs() const;
bool has(const String & column_name) const;
bool hasNested(const String & column_name) const;
ColumnDescription & get(const String & column_name);

View File

@ -39,6 +39,9 @@ public:
/// Opaque pointer to avoid dependencies (it is not possible to do forward declaration of typedef).
const void * data;
/// Minimal time, when we need to delete some data from this part
time_t min_ttl;
};
/// Parts are belong to partitions. Only parts within same partition could be merged.

View File

@ -84,6 +84,7 @@ namespace ErrorCodes
extern const int CANNOT_ALLOCATE_MEMORY;
extern const int CANNOT_MUNMAP;
extern const int CANNOT_MREMAP;
extern const int BAD_TTL_EXPRESSION;
}
@ -97,6 +98,7 @@ MergeTreeData::MergeTreeData(
const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_,
const ASTPtr & ttl_table_ast_,
const MergingParams & merging_params_,
const MergeTreeSettings & settings_,
bool require_part_metadata_,
@ -108,6 +110,7 @@ MergeTreeData::MergeTreeData(
settings(settings_),
partition_by_ast(partition_by_ast_),
sample_by_ast(sample_by_ast_),
ttl_table_ast(ttl_table_ast_),
require_part_metadata(require_part_metadata_),
database_name(database_), table_name(table_),
full_path(full_path_),
@ -159,6 +162,8 @@ MergeTreeData::MergeTreeData(
min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING;
}
setTTLExpressions(columns_.getColumnTTLs(), ttl_table_ast_);
auto path_exists = Poco::File(full_path).exists();
/// Creating directories, if not exist.
Poco::File(full_path).createDirectories();
@ -490,6 +495,98 @@ void MergeTreeData::initPartitionKey()
}
}
namespace
{
void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const String & result_column_name)
{
for (const auto & action : ttl_expression->getActions())
{
if (action.type == ExpressionAction::APPLY_FUNCTION)
{
IFunctionBase & func = *action.function_base;
if (!func.isDeterministic())
throw Exception("TTL expression cannot contain non-deterministic functions, "
"but contains function " + func.getName(), ErrorCodes::BAD_ARGUMENTS);
}
}
bool has_date_column = false;
for (const auto & elem : ttl_expression->getRequiredColumnsWithTypes())
{
if (typeid_cast<const DataTypeDateTime *>(elem.type.get()) || typeid_cast<const DataTypeDate *>(elem.type.get()))
{
has_date_column = true;
break;
}
}
if (!has_date_column)
throw Exception("TTL expression should use at least one Date or DateTime column", ErrorCodes::BAD_TTL_EXPRESSION);
const auto & result_column = ttl_expression->getSampleBlock().getByName(result_column_name);
if (!typeid_cast<const DataTypeDateTime *>(result_column.type.get())
&& !typeid_cast<const DataTypeDate *>(result_column.type.get()))
{
throw Exception("TTL expression result column should have DateTime or Date type, but has "
+ result_column.type->getName(), ErrorCodes::BAD_TTL_EXPRESSION);
}
}
}
void MergeTreeData::setTTLExpressions(const ColumnsDescription::ColumnTTLs & new_column_ttls,
const ASTPtr & new_ttl_table_ast, bool only_check)
{
auto create_ttl_entry = [this](ASTPtr ttl_ast) -> TTLEntry
{
auto syntax_result = SyntaxAnalyzer(global_context).analyze(ttl_ast, getColumns().getAllPhysical());
auto expr = ExpressionAnalyzer(ttl_ast, syntax_result, global_context).getActions(false);
String result_column = ttl_ast->getColumnName();
checkTTLExpression(expr, result_column);
return {expr, result_column};
};
if (!new_column_ttls.empty())
{
NameSet columns_ttl_forbidden;
if (partition_key_expr)
for (const auto & col : partition_key_expr->getRequiredColumns())
columns_ttl_forbidden.insert(col);
if (sorting_key_expr)
for (const auto & col : sorting_key_expr->getRequiredColumns())
columns_ttl_forbidden.insert(col);
for (const auto & [name, ast] : new_column_ttls)
{
if (columns_ttl_forbidden.count(name))
throw Exception("Trying to set ttl for key column " + name, ErrorCodes::ILLEGAL_COLUMN);
else
{
auto new_ttl_entry = create_ttl_entry(ast);
if (!only_check)
ttl_entries_by_name.emplace(name, new_ttl_entry);
}
}
}
if (new_ttl_table_ast)
{
auto new_ttl_table_entry = create_ttl_entry(new_ttl_table_ast);
if (!only_check)
{
ttl_table_ast = new_ttl_table_ast;
ttl_table_entry = new_ttl_table_entry;
}
}
}
void MergeTreeData::MergingParams::check(const NamesAndTypesList & columns) const
{
@ -1059,7 +1156,8 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c
auto new_indices = getIndicesDescription();
ASTPtr new_order_by_ast = order_by_ast;
ASTPtr new_primary_key_ast = primary_key_ast;
commands.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast);
ASTPtr new_ttl_table_ast = ttl_table_ast;
commands.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
if (getIndicesDescription().empty() && !new_indices.empty() &&
!context.getSettingsRef().allow_experimental_data_skipping_indices)
@ -1145,6 +1243,8 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c
setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast,
new_columns, new_indices, /* only_check = */ true);
setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast, /* only_check = */ true);
/// Check that type conversions are possible.
ExpressionActionsPtr unused_expression;
NameToNameMap unused_map;
@ -1446,6 +1546,32 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart(
return transaction;
}
void MergeTreeData::removeEmptyColumnsFromPart(MergeTreeData::MutableDataPartPtr & data_part)
{
auto & empty_columns = data_part->empty_columns;
if (empty_columns.empty())
return;
NamesAndTypesList new_columns;
for (const auto & [name, type] : data_part->columns)
if (!empty_columns.count(name))
new_columns.emplace_back(name, type);
std::stringstream log_message;
for (auto it = empty_columns.begin(); it != empty_columns.end(); ++it)
{
if (it != empty_columns.begin())
log_message << ", ";
log_message << *it;
}
LOG_INFO(log, "Removing empty columns: " << log_message.str() << " from part " << data_part->name);
if (auto transaction = alterDataPart(data_part, new_columns, getIndicesDescription().indices, false))
transaction->commit();
empty_columns.clear();
}
void MergeTreeData::freezeAll(const String & with_name, const Context & context)
{
freezePartitionsByMatcher([] (const DataPartPtr &){ return true; }, with_name, context);

View File

@ -312,6 +312,7 @@ public:
const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported.
const ASTPtr & ttl_table_ast_,
const MergingParams & merging_params_,
const MergeTreeSettings & settings_,
bool require_part_metadata_,
@ -494,6 +495,9 @@ public:
const IndicesASTs & new_indices,
bool skip_sanity_checks);
/// Remove columns, that have been markedd as empty after zeroing values with expired ttl
void removeEmptyColumnsFromPart(MergeTreeData::MutableDataPartPtr & data_part);
/// Freezes all parts.
void freezeAll(const String & with_name, const Context & context);
@ -514,6 +518,7 @@ public:
bool hasSortingKey() const { return !sorting_key_columns.empty(); }
bool hasPrimaryKey() const { return !primary_key_columns.empty(); }
bool hasSkipIndices() const { return !skip_indices.empty(); }
bool hasTableTTL() const { return ttl_table_ast != nullptr; }
ASTPtr getSortingKeyAST() const { return sorting_key_expr_ast; }
ASTPtr getPrimaryKeyAST() const { return primary_key_expr_ast; }
@ -601,6 +606,17 @@ public:
Block primary_key_sample;
DataTypes primary_key_data_types;
struct TTLEntry
{
ExpressionActionsPtr expression;
String result_column;
};
using TTLEntriesByName = std::unordered_map<String, TTLEntry>;
TTLEntriesByName ttl_entries_by_name;
TTLEntry ttl_table_entry;
String sampling_expr_column_name;
Names columns_required_for_sampling;
@ -625,6 +641,7 @@ private:
ASTPtr order_by_ast;
ASTPtr primary_key_ast;
ASTPtr sample_by_ast;
ASTPtr ttl_table_ast;
bool require_part_metadata;
@ -735,6 +752,9 @@ private:
void initPartitionKey();
void setTTLExpressions(const ColumnsDescription::ColumnTTLs & new_column_ttls,
const ASTPtr & new_ttl_table_ast, bool only_check = false);
/// Expression for column type conversion.
/// If no conversions are needed, out_expression=nullptr.
/// out_rename_map maps column files for the out_expression onto new table files.

View File

@ -4,9 +4,11 @@
#include <Storages/MergeTree/DiskSpaceMonitor.h>
#include <Storages/MergeTree/SimpleMergeSelector.h>
#include <Storages/MergeTree/AllMergeSelector.h>
#include <Storages/MergeTree/TTLMergeSelector.h>
#include <Storages/MergeTree/MergeList.h>
#include <Storages/MergeTree/StorageFromMergeTreeDataPart.h>
#include <Storages/MergeTree/BackgroundProcessingPool.h>
#include <DataStreams/TTLBlockInputStream.h>
#include <DataStreams/DistinctSortedBlockInputStream.h>
#include <DataStreams/ExpressionBlockInputStream.h>
#include <DataStreams/MergingSortedBlockInputStream.h>
@ -176,6 +178,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge(
const String * prev_partition_id = nullptr;
const MergeTreeData::DataPartPtr * prev_part = nullptr;
bool has_part_with_expired_ttl = false;
for (const MergeTreeData::DataPartPtr & part : data_parts)
{
const String & partition_id = part->info.partition_id;
@ -191,6 +194,10 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge(
part_info.age = current_time - part->modification_time;
part_info.level = part->info.level;
part_info.data = &part;
part_info.min_ttl = part->ttl_infos.part_min_ttl;
if (part_info.min_ttl && part_info.min_ttl <= current_time)
has_part_with_expired_ttl = true;
partitions.back().emplace_back(part_info);
@ -210,8 +217,17 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge(
if (aggressive)
merge_settings.base = 1;
bool can_merge_with_ttl =
(current_time - last_merge_with_ttl > data.settings.merge_with_ttl_timeout);
/// NOTE Could allow selection of different merge strategy.
merge_selector = std::make_unique<SimpleMergeSelector>(merge_settings);
if (can_merge_with_ttl && has_part_with_expired_ttl)
{
merge_selector = std::make_unique<TTLMergeSelector>(current_time);
last_merge_with_ttl = current_time;
}
else
merge_selector = std::make_unique<SimpleMergeSelector>(merge_settings);
IMergeSelector::PartsInPartition parts_to_merge = merge_selector->select(
partitions,
@ -224,7 +240,8 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge(
return false;
}
if (parts_to_merge.size() == 1)
/// Allow to "merge" part with itself if we need remove some values with expired ttl
if (parts_to_merge.size() == 1 && !has_part_with_expired_ttl)
throw Exception("Logical error: merge selector returned only one part to merge", ErrorCodes::LOGICAL_ERROR);
MergeTreeData::DataPartsVector parts;
@ -536,9 +553,17 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
new_data_part->relative_path = TMP_PREFIX + future_part.name;
new_data_part->is_temp = true;
bool need_remove_expired_values = false;
for (const MergeTreeData::DataPartPtr & part : parts)
new_data_part->ttl_infos.update(part->ttl_infos);
const auto & part_min_ttl = new_data_part->ttl_infos.part_min_ttl;
if (part_min_ttl && part_min_ttl <= time_of_merge)
need_remove_expired_values = true;
size_t sum_input_rows_upper_bound = merge_entry->total_size_marks * data.index_granularity;
MergeAlgorithm merge_alg = chooseMergeAlgorithm(parts, sum_input_rows_upper_bound, gathering_columns, deduplicate);
MergeAlgorithm merge_alg = chooseMergeAlgorithm(parts, sum_input_rows_upper_bound, gathering_columns, deduplicate, need_remove_expired_values);
LOG_DEBUG(log, "Selected MergeAlgorithm: " << ((merge_alg == MergeAlgorithm::Vertical) ? "Vertical" : "Horizontal"));
@ -599,6 +624,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
MergeStageProgress horizontal_stage_progress(
merge_alg == MergeAlgorithm::Horizontal ? 1.0 : column_sizes.keyColumnsWeight());
for (const auto & part : parts)
{
auto input = std::make_unique<MergeTreeSequentialBlockInputStream>(
@ -671,6 +697,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
if (deduplicate)
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);
MergedBlockOutputStream to{
data, new_part_tmp_path, merging_columns, compression_codec, merged_column_to_size, data.settings.min_merge_bytes_to_use_direct_io};
@ -684,6 +713,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
while (!actions_blocker.isCancelled() && (block = merged_stream->read()))
{
rows_written += block.rows();
to.write(block);
merge_entry->rows_written = merged_stream->getProfileInfo().rows;
@ -857,6 +887,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
data, future_part.name, future_part.part_info);
new_data_part->relative_path = "tmp_mut_" + future_part.name;
new_data_part->is_temp = true;
new_data_part->ttl_infos = source_part->ttl_infos;
String new_part_tmp_path = new_data_part->getFullPath();
@ -1016,12 +1047,14 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
MergeTreeDataMergerMutator::MergeAlgorithm MergeTreeDataMergerMutator::chooseMergeAlgorithm(
const MergeTreeData::DataPartsVector & parts, size_t sum_rows_upper_bound,
const NamesAndTypesList & gathering_columns, bool deduplicate) const
const NamesAndTypesList & gathering_columns, bool deduplicate, bool need_remove_expired_values) const
{
if (deduplicate)
return MergeAlgorithm::Horizontal;
if (data.settings.enable_vertical_merge_algorithm == 0)
return MergeAlgorithm::Horizontal;
if (need_remove_expired_values)
return MergeAlgorithm::Horizontal;
bool is_supported_storage =
data.merging_params.mode == MergeTreeData::MergingParams::Ordinary ||
@ -1093,7 +1126,6 @@ MergeTreeData::DataPartPtr MergeTreeDataMergerMutator::renameMergedTemporaryPart
return new_data_part;
}
size_t MergeTreeDataMergerMutator::estimateNeededDiskSpace(const MergeTreeData::DataPartsVector & source_parts)
{
size_t res = 0;

View File

@ -127,7 +127,7 @@ private:
MergeAlgorithm chooseMergeAlgorithm(
const MergeTreeData::DataPartsVector & parts,
size_t rows_upper_bound, const NamesAndTypesList & gathering_columns, bool deduplicate) const;
size_t rows_upper_bound, const NamesAndTypesList & gathering_columns, bool deduplicate, bool need_remove_expired_values) const;
private:
MergeTreeData & data;
@ -137,6 +137,9 @@ private:
/// When the last time you wrote to the log that the disk space was running out (not to write about this too often).
time_t disk_space_warning_time = 0;
/// Last time when TTLMergeSelector has been used
time_t last_merge_with_ttl = 0;
};

View File

@ -22,6 +22,7 @@
#include <Poco/DirectoryIterator.h>
#include <common/logger_useful.h>
#include <common/JSON.h>
#define MERGE_TREE_MARK_SIZE (2 * sizeof(UInt64))
@ -37,6 +38,7 @@ namespace ErrorCodes
extern const int CORRUPTED_DATA;
extern const int NOT_FOUND_EXPECTED_DATA_PART;
extern const int BAD_SIZE_OF_FILE_IN_DATA_PART;
extern const int BAD_TTL_FILE;
}
@ -198,7 +200,6 @@ size_t MergeTreeDataPart::getFileSizeOrZero(const String & file_name) const
return checksum->second.file_size;
}
/** 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.
*/
@ -479,6 +480,7 @@ void MergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checksu
loadIndex();
loadRowsCount(); /// Must be called after loadIndex() as it uses the value of `marks_count`.
loadPartitionAndMinMaxIndex();
loadTTLInfos();
if (check_consistency)
checkConsistency(require_columns_checksums);
}
@ -637,6 +639,33 @@ void MergeTreeDataPart::loadRowsCount()
}
}
void MergeTreeDataPart::loadTTLInfos()
{
String path = getFullPath() + "ttl.txt";
if (Poco::File(path).exists())
{
ReadBufferFromFile in = openForReading(path);
assertString("ttl format version: ", in);
size_t format_version;
readText(format_version, in);
assertChar('\n', in);
if (format_version == 1)
{
try
{
ttl_infos.read(in);
}
catch (const JSONException &)
{
throw Exception("Error while parsing file ttl.txt in part: " + name, ErrorCodes::BAD_TTL_FILE);
}
}
else
throw Exception("Unknown ttl format version: " + toString(format_version), ErrorCodes::BAD_TTL_FILE);
}
}
void MergeTreeDataPart::accumulateColumnSizes(ColumnToSize & column_to_size) const
{
std::shared_lock<std::shared_mutex> part_lock(columns_lock);

View File

@ -8,6 +8,7 @@
#include <Storages/MergeTree/MergeTreePartInfo.h>
#include <Storages/MergeTree/MergeTreePartition.h>
#include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
#include <Storages/MergeTree/MergeTreeDataPartTTLInfo.h>
#include <Storages/MergeTree/KeyCondition.h>
#include <Columns/IColumn.h>
@ -129,6 +130,11 @@ struct MergeTreeDataPart
Deleting /// not active data part with identity refcounter, it is deleting right now by a cleaner
};
using TTLInfo = MergeTreeDataPartTTLInfo;
using TTLInfos = MergeTreeDataPartTTLInfos;
TTLInfos ttl_infos;
/// Current state of the part. If the part is in working set already, it should be accessed via data_parts mutex
mutable State state{State::Temporary};
@ -216,6 +222,9 @@ struct MergeTreeDataPart
/// Columns description.
NamesAndTypesList columns;
/// Columns with values, that all have been zeroed by expired ttl
NameSet empty_columns;
using ColumnToSize = std::map<std::string, UInt64>;
/** It is blocked for writing when changing columns, checksums or any part files.
@ -281,6 +290,9 @@ private:
/// For the older format version calculates rows count from the size of a column with a fixed size.
void loadRowsCount();
/// Loads ttl infos in json format from file ttl.txt. If file doesn`t exists assigns ttl infos with all zeros
void loadTTLInfos();
void loadPartitionAndMinMaxIndex();
void checkConsistency(bool require_part_metadata);

View File

@ -0,0 +1,88 @@
#include <Storages/MergeTree/MergeTreeDataPartTTLInfo.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <common/JSON.h>
namespace DB
{
void MergeTreeDataPartTTLInfos::update(const MergeTreeDataPartTTLInfos & other_infos)
{
for (const auto & [name, ttl_info] : other_infos.columns_ttl)
{
columns_ttl[name].update(ttl_info);
updatePartMinTTL(ttl_info.min);
}
table_ttl.update(other_infos.table_ttl);
updatePartMinTTL(table_ttl.min);
}
void MergeTreeDataPartTTLInfos::read(ReadBuffer & in)
{
String json_str;
readString(json_str, in);
assertEOF(in);
JSON json(json_str);
if (json.has("columns"))
{
JSON columns = json["columns"];
for (auto col : columns)
{
MergeTreeDataPartTTLInfo ttl_info;
ttl_info.min = col["min"].getUInt();
ttl_info.max = col["max"].getUInt();
String name = col["name"].getString();
columns_ttl.emplace(name, ttl_info);
updatePartMinTTL(ttl_info.min);
}
}
if (json.has("table"))
{
JSON table = json["table"];
table_ttl.min = table["min"].getUInt();
table_ttl.max = table["max"].getUInt();
updatePartMinTTL(table_ttl.min);
}
}
void MergeTreeDataPartTTLInfos::write(WriteBuffer & out) const
{
writeString("ttl format version: 1\n", out);
writeString("{", out);
if (!columns_ttl.empty())
{
writeString("\"columns\":[", out);
for (auto it = columns_ttl.begin(); it != columns_ttl.end(); ++it)
{
if (it != columns_ttl.begin())
writeString(",", out);
writeString("{\"name\":\"", out);
writeString(it->first, out);
writeString("\",\"min\":", out);
writeIntText(it->second.min, out);
writeString(",\"max\":", out);
writeIntText(it->second.max, out);
writeString("}", out);
}
writeString("]", out);
}
if (table_ttl.min)
{
if (!columns_ttl.empty())
writeString(",", out);
writeString("\"table\":{\"min\":", out);
writeIntText(table_ttl.min, out);
writeString(",\"max\":", out);
writeIntText(table_ttl.max, out);
writeString("}", out);
}
writeString("}", out);
}
}

View File

@ -0,0 +1,51 @@
#pragma once
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromFile.h>
#include <unordered_map>
namespace DB
{
/// Minimal and maximal ttl for column or table
struct MergeTreeDataPartTTLInfo
{
time_t min = 0;
time_t max = 0;
void update(time_t time)
{
if (time && (!min || time < min))
min = time;
max = std::max(time, max);
}
void update(const MergeTreeDataPartTTLInfo & other_info)
{
if (other_info.min && (!min || other_info.min < min))
min = other_info.min;
max = std::max(other_info.max, max);
}
};
/// PartTTLInfo for all columns and table with minimal ttl for whole part
struct MergeTreeDataPartTTLInfos
{
std::unordered_map<String, MergeTreeDataPartTTLInfo> columns_ttl;
MergeTreeDataPartTTLInfo table_ttl;
time_t part_min_ttl = 0;
void read(ReadBuffer & in);
void write(WriteBuffer & out) const;
void update(const MergeTreeDataPartTTLInfos & other_infos);
void updatePartMinTTL(time_t time)
{
if (time && (!part_min_ttl || time < part_min_ttl))
part_min_ttl = time;
}
};
}

View File

@ -4,8 +4,11 @@
#include <Common/Exception.h>
#include <Interpreters/AggregationCommon.h>
#include <IO/HashingWriteBuffer.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDate.h>
#include <IO/WriteHelpers.h>
#include <Poco/File.h>
#include <Common/typeid_cast.h>
namespace ProfileEvents
@ -71,6 +74,34 @@ void buildScatterSelector(
}
}
/// Computes ttls and updates ttl infos
void updateTTL(const MergeTreeData::TTLEntry & ttl_entry, MergeTreeDataPart::TTLInfos & ttl_infos, Block & block, const String & column_name)
{
if (!block.has(ttl_entry.result_column))
ttl_entry.expression->execute(block);
auto & ttl_info = (column_name.empty() ? ttl_infos.table_ttl : ttl_infos.columns_ttl[column_name]);
const auto & current = block.getByName(ttl_entry.result_column);
const IColumn * column = current.column.get();
if (const ColumnUInt16 * column_date = typeid_cast<const ColumnUInt16 *>(column))
{
const auto & date_lut = DateLUT::instance();
for (const auto & val : column_date->getData())
ttl_info.update(date_lut.fromDayNum(DayNum(val)));
}
else if (const ColumnUInt32 * column_date_time = typeid_cast<const ColumnUInt32 *>(column))
{
for (const auto & val : column_date_time->getData())
ttl_info.update(val);
}
else
throw Exception("Unexpected type of result ttl column", ErrorCodes::LOGICAL_ERROR);
ttl_infos.updatePartMinTTL(ttl_info.min);
}
}
BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block, size_t max_parts)
@ -213,6 +244,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocksAlreadySorted);
}
if (data.hasTableTTL())
updateTTL(data.ttl_table_entry, new_data_part->ttl_infos, block, "");
for (const auto & [name, ttl_entry] : data.ttl_entries_by_name)
updateTTL(ttl_entry, new_data_part->ttl_infos, block, name);
/// This effectively chooses minimal compression method:
/// either default lz4 or compression method with zero thresholds on absolute and relative part size.
auto compression_codec = data.global_context.chooseCompressionCodec(0, 0);

View File

@ -160,7 +160,10 @@ struct MergeTreeSettings
M(SettingUInt64, finished_mutations_to_keep, 100) \
\
/** Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled) */ \
M(SettingUInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024)
M(SettingUInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024) \
\
/** Minimal time in seconds, when merge with TTL can be repeated */ \
M(SettingInt64, merge_with_ttl_timeout, 3600 * 24)
/// Settings that should not change after the creation of a table.
#define APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(M) \

View File

@ -392,6 +392,16 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart(
checksums.files["count.txt"].file_hash = count_out_hashing.getHash();
}
if (new_part->ttl_infos.part_min_ttl)
{
/// Write a file with ttl infos in json format.
WriteBufferFromFile out(part_path + "ttl.txt", 4096);
HashingWriteBuffer out_hashing(out);
new_part->ttl_infos.write(out_hashing);
checksums.files["ttl.txt"].file_size = out_hashing.count();
checksums.files["ttl.txt"].file_hash = out_hashing.getHash();
}
{
/// Write a file with a description of columns.
WriteBufferFromFile out(part_path + "columns.txt", 4096);

View File

@ -46,6 +46,8 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr
partition_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.partition_by_ast));
skip_indices = data.getIndicesDescription().toString();
ttl_table = formattedAST(data.ttl_table_ast);
}
void ReplicatedMergeTreeTableMetadata::write(WriteBuffer & out) const
@ -67,6 +69,9 @@ void ReplicatedMergeTreeTableMetadata::write(WriteBuffer & out) const
if (!sorting_key.empty())
out << "sorting key: " << sorting_key << "\n";
if (!ttl_table.empty())
out << "ttl: " << ttl_table << "\n";
if (!skip_indices.empty())
out << "indices: " << skip_indices << "\n";
}
@ -101,6 +106,9 @@ void ReplicatedMergeTreeTableMetadata::read(ReadBuffer & in)
if (checkString("indices: ", in))
in >> skip_indices >> "\n";
if (checkString("ttl: ", in))
in >> ttl_table >> "\n";
}
ReplicatedMergeTreeTableMetadata ReplicatedMergeTreeTableMetadata::parse(const String & s)
@ -183,6 +191,21 @@ ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTabl
ErrorCodes::METADATA_MISMATCH);
}
if (ttl_table != from_zk.ttl_table)
{
if (allow_alter)
{
diff.ttl_table_changed = true;
diff.new_ttl_table = from_zk.ttl_table;
}
else
throw Exception(
"Existing table metadata in ZooKeeper differs in ttl."
" Stored in ZooKeeper: " + from_zk.ttl_table +
", local: " + ttl_table,
ErrorCodes::METADATA_MISMATCH);
}
if (skip_indices != from_zk.skip_indices)
{
if (allow_alter)

View File

@ -26,6 +26,7 @@ struct ReplicatedMergeTreeTableMetadata
String partition_key;
String sorting_key;
String skip_indices;
String ttl_table;
ReplicatedMergeTreeTableMetadata() = default;
explicit ReplicatedMergeTreeTableMetadata(const MergeTreeData & data);
@ -44,6 +45,9 @@ struct ReplicatedMergeTreeTableMetadata
bool skip_indices_changed = false;
String new_skip_indices;
bool ttl_table_changed = false;
String new_ttl_table;
bool empty() const { return !sorting_key_changed && !skip_indices_changed; }
};

View File

@ -0,0 +1,68 @@
#include <Storages/MergeTree/TTLMergeSelector.h>
#include <cmath>
#include <algorithm>
namespace DB
{
IMergeSelector::PartsInPartition TTLMergeSelector::select(
const Partitions & partitions,
const size_t max_total_size_to_merge)
{
using Iterator = IMergeSelector::PartsInPartition::const_iterator;
Iterator best_begin;
ssize_t partition_to_merge_index = -1;
time_t partition_to_merge_min_ttl = 0;
for (size_t i = 0; i < partitions.size(); ++i)
{
for (auto it = partitions[i].begin(); it != partitions[i].end(); ++it)
{
if (it->min_ttl && (partition_to_merge_index == -1 || it->min_ttl < partition_to_merge_min_ttl))
{
partition_to_merge_min_ttl = it->min_ttl;
partition_to_merge_index = i;
best_begin = it;
}
}
}
if (partition_to_merge_index == -1 || partition_to_merge_min_ttl > current_time)
return {};
const auto & best_partition = partitions[partition_to_merge_index];
Iterator best_end = best_begin + 1;
size_t total_size = 0;
while (true)
{
if (!best_begin->min_ttl || best_begin->min_ttl > current_time
|| (max_total_size_to_merge && total_size > max_total_size_to_merge))
{
++best_begin;
break;
}
total_size += best_begin->size;
if (best_begin == best_partition.begin())
break;
--best_begin;
}
while (best_end != best_partition.end())
{
if (!best_end->min_ttl || best_end->min_ttl > current_time
|| (max_total_size_to_merge && total_size > max_total_size_to_merge))
break;
total_size += best_end->size;
++best_end;
}
return PartsInPartition(best_begin, best_end);
}
}

View File

@ -0,0 +1,26 @@
#pragma once
#include <Storages/MergeTree/MergeSelector.h>
namespace DB
{
/** Merge selector, which is used to remove values with expired ttl.
* It selects parts to merge by greedy algorithm:
* 1. Finds part with the most earliest expired ttl and includes it to result.
* 2. Tries to find the longest range of parts with expired ttl, that includes part from step 1.
*/
class TTLMergeSelector : public IMergeSelector
{
public:
explicit TTLMergeSelector(time_t current_time_) : current_time(current_time_) {}
PartsInPartition select(
const Partitions & partitions,
const size_t max_total_size_to_merge) override;
private:
time_t current_time;
};
}

View File

@ -211,14 +211,15 @@ MergeTreeData::DataPart::Checksums checkDataPart(
checksums_data.files["primary.idx"] = MergeTreeData::DataPart::Checksums::Checksum(primary_idx_size, hashing_buf.getHash());
}
/// Optional files count.txt, partition.dat, minmax_*.idx. Just calculate checksums for existing files.
/// Optional files count.txt, partition.dat, minmax_*.idx, ttl.txt. Just calculate checksums for existing files.
Poco::DirectoryIterator dir_end;
for (Poco::DirectoryIterator dir_it(path); dir_it != dir_end; ++dir_it)
{
const String & file_name = dir_it.name();
if (file_name == "count.txt"
|| file_name == "partition.dat"
|| (startsWith(file_name, "minmax_") && endsWith(file_name, ".idx")))
|| (startsWith(file_name, "minmax_") && endsWith(file_name, ".idx"))
|| file_name == "ttl.txt")
{
ReadBufferFromFile file_buf(dir_it->path());
HashingReadBuffer hashing_buf(file_buf);

View File

@ -572,6 +572,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
ASTPtr order_by_ast;
ASTPtr primary_key_ast;
ASTPtr sample_by_ast;
ASTPtr ttl_table_ast;
IndicesDescription indices_description;
MergeTreeSettings storage_settings = args.context.getMergeTreeSettings();
@ -593,11 +594,15 @@ static StoragePtr create(const StorageFactory::Arguments & args)
if (args.storage_def->sample_by)
sample_by_ast = args.storage_def->sample_by->ptr();
if (args.storage_def->ttl_table)
ttl_table_ast = args.storage_def->ttl_table->ptr();
if (args.query.columns_list && args.query.columns_list->indices)
for (const auto & index : args.query.columns_list->indices->children)
indices_description.indices.push_back(
std::dynamic_pointer_cast<ASTIndexDeclaration>(index->clone()));
storage_settings.loadFromQuery(*args.storage_def);
}
else
@ -636,12 +641,14 @@ static StoragePtr create(const StorageFactory::Arguments & args)
zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name,
args.columns, indices_description,
args.context, date_column_name, partition_by_ast, order_by_ast, primary_key_ast,
sample_by_ast, merging_params, storage_settings, args.has_force_restore_data_flag);
sample_by_ast, ttl_table_ast, merging_params, storage_settings,
args.has_force_restore_data_flag);
else
return StorageMergeTree::create(
args.data_path, args.database_name, args.table_name, args.columns, indices_description,
args.attach, args.context, date_column_name, partition_by_ast, order_by_ast,
primary_key_ast, sample_by_ast, merging_params, storage_settings, args.has_force_restore_data_flag);
primary_key_ast, sample_by_ast, ttl_table_ast, merging_params, storage_settings,
args.has_force_restore_data_flag);
}

View File

@ -59,6 +59,7 @@ StorageMergeTree::StorageMergeTree(
const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported.
const ASTPtr & ttl_table_ast_,
const MergeTreeData::MergingParams & merging_params_,
const MergeTreeSettings & settings_,
bool has_force_restore_data_flag)
@ -67,7 +68,8 @@ StorageMergeTree::StorageMergeTree(
data(database_name, table_name,
full_path, columns_, indices_,
context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_,
sample_by_ast_, merging_params_, settings_, false, attach),
sample_by_ast_, ttl_table_ast_, merging_params_,
settings_, false, attach),
reader(data), writer(data), merger_mutator(data, global_context.getBackgroundPool()),
log(&Logger::get(database_name_ + "." + table_name + " (StorageMergeTree)"))
{
@ -220,7 +222,8 @@ void StorageMergeTree::alter(
auto new_indices = data.getIndicesDescription();
ASTPtr new_order_by_ast = data.order_by_ast;
ASTPtr new_primary_key_ast = data.primary_key_ast;
params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast);
ASTPtr new_ttl_table_ast = data.ttl_table_ast;
params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
auto columns_for_parts = new_columns.getAllPhysical();
@ -242,6 +245,9 @@ void StorageMergeTree::alter(
if (new_primary_key_ast.get() != data.primary_key_ast.get())
storage_ast.set(storage_ast.primary_key, new_primary_key_ast);
if (new_ttl_table_ast.get() != data.ttl_table_ast.get())
storage_ast.set(storage_ast.ttl_table, new_ttl_table_ast);
};
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, storage_modifier);
@ -249,6 +255,8 @@ void StorageMergeTree::alter(
/// Reinitialize primary key because primary key column types might have changed.
data.setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, new_indices);
data.setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast);
for (auto & transaction : transactions)
transaction->commit();
@ -546,6 +554,7 @@ bool StorageMergeTree::merge(
future_part, *merge_entry, time(nullptr),
merging_tagger->reserved_space.get(), deduplicate);
merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr);
data.removeEmptyColumnsFromPart(new_part);
merging_tagger->is_successful = true;
write_part_log({});
@ -794,7 +803,8 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi
auto new_indices = getIndicesDescription();
ASTPtr ignored_order_by_ast;
ASTPtr ignored_primary_key_ast;
alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast);
ASTPtr ignored_ttl_table_ast;
alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast);
auto columns_for_parts = new_columns.getAllPhysical();
for (const auto & part : parts)

View File

@ -181,6 +181,7 @@ protected:
const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_, /// nullptr, if sampling is not supported.
const ASTPtr & ttl_table_ast_,
const MergeTreeData::MergingParams & merging_params_,
const MergeTreeSettings & settings_,
bool has_force_restore_data_flag);

View File

@ -54,6 +54,7 @@
#include <thread>
#include <future>
#include <Parsers/queryToString.h>
namespace ProfileEvents
{
@ -206,6 +207,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_,
const ASTPtr & ttl_table_ast_,
const MergeTreeData::MergingParams & merging_params_,
const MergeTreeSettings & settings_,
bool has_force_restore_data_flag)
@ -217,7 +219,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
data(database_name, table_name,
full_path, columns_, indices_,
context_, date_column_name, partition_by_ast_, order_by_ast_, primary_key_ast_,
sample_by_ast_, merging_params_, settings_, true, attach,
sample_by_ast_, ttl_table_ast_, merging_params_,
settings_, true, attach,
[this] (const std::string & name) { enqueuePartForCheck(name); }),
reader(data), writer(data), merger_mutator(data, global_context.getBackgroundPool()), queue(*this),
fetcher(data),
@ -424,6 +427,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
ASTPtr new_primary_key_ast = data.primary_key_ast;
ASTPtr new_order_by_ast = data.order_by_ast;
auto new_indices = data.getIndicesDescription();
ASTPtr new_ttl_table_ast = data.ttl_table_ast;
IDatabase::ASTModifier storage_modifier;
if (!metadata_diff.empty())
{
@ -452,6 +456,12 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
if (metadata_diff.skip_indices_changed)
new_indices = IndicesDescription::parse(metadata_diff.new_skip_indices);
if (metadata_diff.ttl_table_changed)
{
ParserExpression parser;
new_ttl_table_ast = parseQuery(parser, metadata_diff.new_ttl_table, 0);
}
storage_modifier = [&](IAST & ast)
{
auto & storage_ast = ast.as<ASTStorage &>();
@ -464,6 +474,9 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
if (new_primary_key_ast.get() != data.primary_key_ast.get())
storage_ast.set(storage_ast.primary_key, new_primary_key_ast);
if (new_ttl_table_ast.get() != data.ttl_table_ast.get())
storage_ast.set(storage_ast.ttl_table, new_ttl_table_ast);
storage_ast.set(storage_ast.order_by, new_order_by_ast);
};
}
@ -473,6 +486,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
/// Even if the primary/sorting keys didn't change we must reinitialize it
/// because primary key column types might have changed.
data.setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, new_indices);
data.setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast);
}
@ -1077,6 +1091,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
future_merged_part, *merge_entry, entry.create_time, reserved_space.get(), entry.deduplicate);
merger_mutator.renameMergedTemporaryPart(part, parts, &transaction);
data.removeEmptyColumnsFromPart(part);
try
{
@ -1499,7 +1514,8 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry &
auto new_indices = getIndicesDescription();
ASTPtr ignored_order_by_ast;
ASTPtr ignored_primary_key_ast;
alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast);
ASTPtr ignored_ttl_table_ast;
alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast);
size_t modified_parts = 0;
auto parts = data.getDataParts();
@ -3104,7 +3120,8 @@ void StorageReplicatedMergeTree::alter(
IndicesDescription new_indices = data.getIndicesDescription();
ASTPtr new_order_by_ast = data.order_by_ast;
ASTPtr new_primary_key_ast = data.primary_key_ast;
params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast);
ASTPtr new_ttl_table_ast = data.ttl_table_ast;
params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
String new_columns_str = new_columns.toString();
if (new_columns_str != data.getColumns().toString())
@ -3114,6 +3131,9 @@ void StorageReplicatedMergeTree::alter(
if (new_order_by_ast.get() != data.order_by_ast.get())
new_metadata.sorting_key = serializeAST(*MergeTreeData::extractKeyExpressionList(new_order_by_ast));
if (new_ttl_table_ast.get() != data.ttl_table_ast.get())
new_metadata.ttl_table = serializeAST(*new_ttl_table_ast);
String new_indices_str = new_indices.toString();
if (new_indices_str != data.getIndicesDescription().toString())
new_metadata.skip_indices = new_indices_str;

View File

@ -563,6 +563,7 @@ protected:
const ASTPtr & order_by_ast_,
const ASTPtr & primary_key_ast_,
const ASTPtr & sample_by_ast_,
const ASTPtr & table_ttl_ast_,
const MergeTreeData::MergingParams & merging_params_,
const MergeTreeSettings & settings_,
bool has_force_restore_data_flag);

View File

@ -0,0 +1,65 @@
import time
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', with_zookeeper=True)
node2 = cluster.add_instance('node2', with_zookeeper=True)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
except Exception as ex:
print ex
finally:
cluster.shutdown()
def drop_table(nodes, table_name):
for node in nodes:
node.query("DROP TABLE IF EXISTS {}".format(table_name))
def test_ttl_columns(start_cluster):
drop_table([node1, node2], "test_ttl")
for node in [node1, node2]:
node.query(
'''
CREATE TABLE test_ttl(date DateTime, id UInt32, a Int32 TTL date + INTERVAL 1 DAY, b Int32 TTL date + INTERVAL 1 MONTH)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl', '{replica}')
ORDER BY id PARTITION BY toDayOfMonth(date) SETTINGS merge_with_ttl_timeout=0;
'''.format(replica=node.name))
node1.query("INSERT INTO test_ttl VALUES (toDateTime('2000-10-10 00:00:00'), 1, 1, 3)")
node1.query("INSERT INTO test_ttl VALUES (toDateTime('2000-10-11 10:00:00'), 2, 2, 4)")
time.sleep(1) # sleep to allow use ttl merge selector for second time
node1.query("OPTIMIZE TABLE test_ttl FINAL")
expected = "1\t0\t0\n2\t0\t0\n"
assert TSV(node1.query("SELECT id, a, b FROM test_ttl ORDER BY id")) == TSV(expected)
assert TSV(node2.query("SELECT id, a, b FROM test_ttl ORDER BY id")) == TSV(expected)
def test_ttl_table(start_cluster):
drop_table([node1, node2], "test_ttl")
for node in [node1, node2]:
node.query(
'''
CREATE TABLE test_ttl(date DateTime, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl', '{replica}')
ORDER BY id PARTITION BY toDayOfMonth(date)
TTL date + INTERVAL 1 DAY SETTINGS merge_with_ttl_timeout=0;
'''.format(replica=node.name))
node1.query("INSERT INTO test_ttl VALUES (toDateTime('2000-10-10 00:00:00'), 1)")
node1.query("INSERT INTO test_ttl VALUES (toDateTime('2000-10-11 10:00:00'), 2)")
time.sleep(1) # sleep to allow use ttl merge selector for second time
node1.query("OPTIMIZE TABLE test_ttl FINAL")
assert TSV(node1.query("SELECT * FROM test_ttl")) == TSV("")
assert TSV(node2.query("SELECT * FROM test_ttl")) == TSV("")

View File

@ -1,27 +1,27 @@
CounterID UInt32
StartDate Date
UserID UInt32
VisitID UInt32
NestedColumn.A Array(UInt8)
NestedColumn.S Array(String)
ToDrop UInt32
Added0 UInt32
Added1 UInt32
Added2 UInt32
AddedNested1.A Array(UInt32)
AddedNested1.B Array(UInt64)
AddedNested1.C Array(String)
AddedNested2.A Array(UInt32)
AddedNested2.B Array(UInt64)
CounterID UInt32
StartDate Date
UserID UInt32
VisitID UInt32
Added0 String
Added1 UInt32
Added2 UInt32
AddedNested1.A Array(UInt32)
AddedNested1.C Array(String)
AddedNested2.A Array(UInt32)
AddedNested2.B Array(UInt64)
CounterID UInt32
StartDate Date
UserID UInt32
VisitID UInt32
NestedColumn.A Array(UInt8)
NestedColumn.S Array(String)
ToDrop UInt32
Added0 UInt32
Added1 UInt32
Added2 UInt32
AddedNested1.A Array(UInt32)
AddedNested1.B Array(UInt64)
AddedNested1.C Array(String)
AddedNested2.A Array(UInt32)
AddedNested2.B Array(UInt64)
CounterID UInt32
StartDate Date
UserID UInt32
VisitID UInt32
Added0 String
Added1 UInt32
Added2 UInt32
AddedNested1.A Array(UInt32)
AddedNested1.C Array(String)
AddedNested2.A Array(UInt32)
AddedNested2.B Array(UInt64)
1 2014-01-01 2 3 0 0 [] [] [] []

View File

@ -1,56 +1,56 @@
d Date
k UInt64
i32 Int32
d Date
k UInt64
i32 Int32
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32) ENGINE = MergeTree(d, k, 8192)
2015-01-01 10 42
d Date
k UInt64
i32 Int32
n.ui8 Array(UInt8)
n.s Array(String)
d Date
k UInt64
i32 Int32
n.ui8 Array(UInt8)
n.s Array(String)
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = MergeTree(d, k, 8192)
2015-01-01 8 40 [1,2,3] ['12','13','14']
2015-01-01 10 42 [] []
d Date
k UInt64
i32 Int32
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
d Date
k UInt64
i32 Int32
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
2015-01-01 7 39 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03']
2015-01-01 8 40 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00']
2015-01-01 10 42 [] [] []
d Date
k UInt64
i32 Int32
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
s String DEFAULT \'0\'
d Date
k UInt64
i32 Int32
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
s String DEFAULT \'0\'
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), `s` String DEFAULT \'0\') ENGINE = MergeTree(d, k, 8192)
2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] ['2000-01-01','2000-01-01','2000-01-03'] 100500
2015-01-01 7 39 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 0
2015-01-01 8 40 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00'] 0
2015-01-01 10 42 [] [] [] 0
d Date
k UInt64
i32 Int32
n.ui8 Array(UInt8)
n.s Array(String)
s Int64
d Date
k UInt64
i32 Int32
n.ui8 Array(UInt8)
n.s Array(String)
s Int64
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `s` Int64) ENGINE = MergeTree(d, k, 8192)
2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] 100500
2015-01-01 7 39 [10,20,30] ['120','130','140'] 0
2015-01-01 8 40 [1,2,3] ['12','13','14'] 0
2015-01-01 10 42 [] [] 0
d Date
k UInt64
i32 Int32
n.ui8 Array(UInt8)
n.s Array(String)
s UInt32
n.d Array(Date)
d Date
k UInt64
i32 Int32
n.ui8 Array(UInt8)
n.s Array(String)
s UInt32
n.d Array(Date)
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `s` UInt32, `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] 100500 ['0000-00-00','0000-00-00','0000-00-00']
2015-01-01 7 39 [10,20,30] ['120','130','140'] 0 ['0000-00-00','0000-00-00','0000-00-00']
@ -60,40 +60,40 @@ CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8)
2015-01-01 7 39 [10,20,30] ['120','130','140'] 0 ['0000-00-00','0000-00-00','0000-00-00']
2015-01-01 8 40 [1,2,3] ['12','13','14'] 0 ['0000-00-00','0000-00-00','0000-00-00']
2015-01-01 10 42 [] [] 0 []
d Date
k UInt64
i32 Int32
n.s Array(String)
s UInt32
d Date
k UInt64
i32 Int32
n.s Array(String)
s UInt32
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `n.s` Array(String), `s` UInt32) ENGINE = MergeTree(d, k, 8192)
2015-01-01 6 38 ['asd','qwe','qwe'] 100500
2015-01-01 7 39 ['120','130','140'] 0
2015-01-01 8 40 ['12','13','14'] 0
2015-01-01 10 42 [] 0
d Date
k UInt64
i32 Int32
s UInt32
d Date
k UInt64
i32 Int32
s UInt32
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `s` UInt32) ENGINE = MergeTree(d, k, 8192)
2015-01-01 6 38 100500
2015-01-01 7 39 0
2015-01-01 8 40 0
2015-01-01 10 42 0
d Date
k UInt64
i32 Int32
s UInt32
n.s Array(String)
n.d Array(Date)
d Date
k UInt64
i32 Int32
s UInt32
n.s Array(String)
n.d Array(Date)
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `s` UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
2015-01-01 6 38 100500 [] []
2015-01-01 7 39 0 [] []
2015-01-01 8 40 0 [] []
2015-01-01 10 42 0 [] []
d Date
k UInt64
i32 Int32
s UInt32
d Date
k UInt64
i32 Int32
s UInt32
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `s` UInt32) ENGINE = MergeTree(d, k, 8192)
2015-01-01 6 38 100500
2015-01-01 7 39 0

View File

@ -1,213 +1,213 @@
d Date
k UInt64
i32 Int32
d Date
k UInt64
i32 Int32
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
d Date
k UInt64
i32 Int32
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
2015-01-01 10 42
d Date
k UInt64
i32 Int32
dt DateTime
d Date
k UInt64
i32 Int32
dt DateTime
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime
d Date
k UInt64
i32 Int32
dt DateTime
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
2015-01-01 9 41 1992-01-01 08:00:00
2015-01-01 10 42 0000-00-00 00:00:00
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14']
2015-01-01 9 41 1992-01-01 08:00:00 [] []
2015-01-01 10 42 0000-00-00 00:00:00 [] []
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03']
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00']
2015-01-01 9 41 1992-01-01 08:00:00 [] [] []
2015-01-01 10 42 0000-00-00 00:00:00 [] [] []
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
s String DEFAULT \'0\'
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
s String DEFAULT \'0\'
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), `s` String DEFAULT \'0\') ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
s String DEFAULT \'0\'
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
n.d Array(Date)
s String DEFAULT \'0\'
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), `s` String DEFAULT \'0\') ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] ['2000-01-01','2000-01-01','2000-01-03'] 100500
2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 0
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00'] 0
2015-01-01 9 41 1992-01-01 08:00:00 [] [] [] 0
2015-01-01 10 42 0000-00-00 00:00:00 [] [] [] 0
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
s Int64
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
s Int64
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `s` Int64) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
s Int64
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
s Int64
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `s` Int64) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500
2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0
2015-01-01 9 41 1992-01-01 08:00:00 [] [] 0
2015-01-01 10 42 0000-00-00 00:00:00 [] [] 0
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
s UInt32
n.d Array(Date)
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
s UInt32
n.d Array(Date)
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `s` UInt32, `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
s UInt32
n.d Array(Date)
d Date
k UInt64
i32 Int32
dt DateTime
n.ui8 Array(UInt8)
n.s Array(String)
s UInt32
n.d Array(Date)
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `s` UInt32, `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500 ['0000-00-00','0000-00-00','0000-00-00']
2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0 ['0000-00-00','0000-00-00','0000-00-00']
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0 ['0000-00-00','0000-00-00','0000-00-00']
2015-01-01 9 41 1992-01-01 08:00:00 [] [] 0 []
2015-01-01 10 42 0000-00-00 00:00:00 [] [] 0 []
d Date
k UInt64
i32 Int32
dt DateTime
n.s Array(String)
s UInt32
d Date
k UInt64
i32 Int32
dt DateTime
n.s Array(String)
s UInt32
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.s` Array(String), `s` UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime
n.s Array(String)
s UInt32
d Date
k UInt64
i32 Int32
dt DateTime
n.s Array(String)
s UInt32
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.s` Array(String), `s` UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 13:26:50 ['asd','qwe','qwe'] 100500
2015-01-01 7 39 2014-07-14 13:26:50 ['120','130','140'] 0
2015-01-01 8 40 2012-12-12 12:12:12 ['12','13','14'] 0
2015-01-01 9 41 1992-01-01 08:00:00 [] 0
2015-01-01 10 42 0000-00-00 00:00:00 [] 0
d Date
k UInt64
i32 Int32
dt DateTime
s UInt32
d Date
k UInt64
i32 Int32
dt DateTime
s UInt32
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `s` UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime
s UInt32
d Date
k UInt64
i32 Int32
dt DateTime
s UInt32
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `s` UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 13:26:50 100500
2015-01-01 7 39 2014-07-14 13:26:50 0
2015-01-01 8 40 2012-12-12 12:12:12 0
2015-01-01 9 41 1992-01-01 08:00:00 0
2015-01-01 10 42 0000-00-00 00:00:00 0
d Date
k UInt64
i32 Int32
dt DateTime
s UInt32
n.s Array(String)
n.d Array(Date)
d Date
k UInt64
i32 Int32
dt DateTime
s UInt32
n.s Array(String)
n.d Array(Date)
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `s` UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime
s UInt32
n.s Array(String)
n.d Array(Date)
d Date
k UInt64
i32 Int32
dt DateTime
s UInt32
n.s Array(String)
n.d Array(Date)
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `s` UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 13:26:50 100500 [] []
2015-01-01 7 39 2014-07-14 13:26:50 0 [] []
2015-01-01 8 40 2012-12-12 12:12:12 0 [] []
2015-01-01 9 41 1992-01-01 08:00:00 0 [] []
2015-01-01 10 42 0000-00-00 00:00:00 0 [] []
d Date
k UInt64
i32 Int32
dt DateTime
s UInt32
d Date
k UInt64
i32 Int32
dt DateTime
s UInt32
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `s` UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt DateTime
s UInt32
d Date
k UInt64
i32 Int32
dt DateTime
s UInt32
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `s` UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 13:26:50 100500
2015-01-01 7 39 2014-07-14 13:26:50 0
2015-01-01 8 40 2012-12-12 12:12:12 0
2015-01-01 9 41 1992-01-01 08:00:00 0
2015-01-01 10 42 0000-00-00 00:00:00 0
d Date
k UInt64
i32 Int32
dt Date
s DateTime
d Date
k UInt64
i32 Int32
dt Date
s DateTime
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` Date, `s` DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
d Date
k UInt64
i32 Int32
dt Date
s DateTime
d Date
k UInt64
i32 Int32
dt Date
s DateTime
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` Date, `s` DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
2015-01-01 6 38 2014-07-15 1970-01-02 06:55:00
2015-01-01 7 39 2014-07-14 0000-00-00 00:00:00

View File

@ -1,42 +1,42 @@
col1 UInt8 DEFAULT 0
col1 UInt32
col2 UInt64 DEFAULT col1 + 1
col3 UInt64 MATERIALIZED col1 + 2
col4 UInt64 ALIAS col1 + 3
col1 UInt8 DEFAULT 0
col1 UInt32
col2 UInt64 DEFAULT col1 + 1
col3 UInt64 MATERIALIZED col1 + 2
col4 UInt64 ALIAS col1 + 3
10 11
12 13
99
payload String
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()
payload String
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()
hello clickhouse
payload String
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()
payload_length UInt64 MATERIALIZED length(payload)
payload String
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()
payload_length UInt64 MATERIALIZED length(payload)
hello clickhouse 16
hello clickhouse 16
some string 11
hello clickhouse 16
some string 11
payload String
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()
payload_length UInt64 DEFAULT length(payload)
payload String
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()
payload_length UInt64 DEFAULT length(payload)
hello clickhouse 16
some string 11
payload String
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()
payload_length UInt16 DEFAULT length(payload) % 65535
payload String
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()
payload_length UInt16 DEFAULT length(payload) % 65535
hello clickhouse 16
some string 11
payload String
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()
payload_length UInt16 DEFAULT CAST(length(payload), \'UInt16\')
payload String
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()
payload String
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()
payload_length UInt16 DEFAULT CAST(length(payload), \'UInt16\')
payload String
date Date MATERIALIZED today()
key UInt64 MATERIALIZED 0 * rand()
hello clickhouse
some string

View File

@ -1,9 +1,9 @@
EventDate Date
UTCEventTime DateTime
MoscowEventDate Date DEFAULT toDate(UTCEventTime)
EventDate Date
UTCEventTime DateTime
MoscowEventDate Date DEFAULT toDate(UTCEventTime)
EventDate Date
UTCEventTime DateTime
MoscowEventDate Date DEFAULT toDate(UTCEventTime)
EventDate Date
UTCEventTime DateTime
MoscowEventDate Date DEFAULT toDate(UTCEventTime)
2015-06-09 2015-06-09 01:02:03 2015-06-09
2015-06-09 2015-06-09 01:02:03 2015-06-09
2015-06-09 2015-06-09 01:02:03 2015-06-09

View File

@ -1,28 +1,28 @@
d Date DEFAULT CAST(\'2015-12-29\', \'Date\')
k UInt8 DEFAULT 0
e Enum8(\'hello\' = 1, \'world\' = 2)
sign Enum8(\'minus\' = -1, \'plus\' = 1)
letter Enum16(\'*\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2)
d Date DEFAULT CAST(\'2015-12-29\', \'Date\')
k UInt8 DEFAULT 0
e Enum8(\'hello\' = 1, \'world\' = 2)
sign Enum8(\'minus\' = -1, \'plus\' = 1)
letter Enum16(\'*\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2)
2015-12-29 0 hello minus *
d Date DEFAULT CAST(\'2015-12-29\', \'Date\')
k UInt8 DEFAULT 0
e Enum8(\'hello\' = 1, \'world\' = 2, \'!\' = 3)
sign Enum8(\'minus\' = -1, \'plus\' = 1)
letter Enum16(\'*\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2)
d Date DEFAULT CAST(\'2015-12-29\', \'Date\')
k UInt8 DEFAULT 0
e Enum8(\'hello\' = 1, \'world\' = 2, \'!\' = 3)
sign Enum8(\'minus\' = -1, \'plus\' = 1)
letter Enum16(\'*\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2)
2015-12-29 0 hello minus *
2015-12-29 0 ! plus b
d Date DEFAULT CAST(\'2015-12-29\', \'Date\')
k UInt8 DEFAULT 0
e Enum16(\'hello\' = 1, \'world\' = 2, \'!\' = 3)
sign Enum16(\'minus\' = -1, \'plus\' = 1)
letter Enum16(\'no letter\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2)
d Date DEFAULT CAST(\'2015-12-29\', \'Date\')
k UInt8 DEFAULT 0
e Enum16(\'hello\' = 1, \'world\' = 2, \'!\' = 3)
sign Enum16(\'minus\' = -1, \'plus\' = 1)
letter Enum16(\'no letter\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2)
2015-12-29 0 hello minus no letter
2015-12-29 0 ! plus b
d Date DEFAULT CAST(\'2015-12-29\', \'Date\')
k UInt8 DEFAULT 0
e Enum8(\'hello\' = 1, \'world\' = 2, \'!\' = 3)
sign Enum8(\'minus\' = -1, \'plus\' = 1)
letter Enum16(\'no letter\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2)
d Date DEFAULT CAST(\'2015-12-29\', \'Date\')
k UInt8 DEFAULT 0
e Enum8(\'hello\' = 1, \'world\' = 2, \'!\' = 3)
sign Enum8(\'minus\' = -1, \'plus\' = 1)
letter Enum16(\'no letter\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2)
2015-12-29 0 hello minus no letter
2015-12-29 0 ! plus b
2015-12-29 0 world minus c

View File

@ -6,7 +6,7 @@ performing test: union_all
performing test: bad_union_all
query failed
performing test: describe_table
dummy UInt8
dummy UInt8
performing test: clickhouse-local
2 3
performing test: http

View File

@ -1,30 +1,30 @@
date Date
val UInt64
val2 UInt8 DEFAULT 42
val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\')
val4 UInt64 ALIAS val
date Date
val UInt64
val2 UInt8 DEFAULT 42
val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\')
val4 UInt64 ALIAS val
-
date Date
val UInt64
val2 UInt8 DEFAULT 42
val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\')
val4 UInt64 ALIAS val
date Date
val UInt64
val2 UInt8 DEFAULT 42
val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\')
val4 UInt64 ALIAS val
-
date Date
val UInt64
val2 UInt8 DEFAULT 42
val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\')
val4 UInt64 ALIAS val
date Date
val UInt64
val2 UInt8 DEFAULT 42
val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\')
val4 UInt64 ALIAS val
-
date Date
val UInt64
val2 UInt8 DEFAULT 42
val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\')
val4 UInt64 ALIAS val
date Date
val UInt64
val2 UInt8 DEFAULT 42
val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\')
val4 UInt64 ALIAS val
-
1 UInt8
1 UInt8
-
1 UInt8
1 UInt8
-
number UInt64
number UInt64
-

View File

@ -1,3 +1,3 @@
x UInt8
x Int64 DEFAULT toInt64(y)
y String
x UInt8
x Int64 DEFAULT toInt64(y)
y String

View File

@ -8,6 +8,6 @@ hello
hello
1970-01-01 00:00:01
CREATE TABLE test.cast (`x` UInt8, `e` Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')) ENGINE = MergeTree ORDER BY e SETTINGS index_granularity = 8192
x UInt8
e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\')
x UInt8
e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\')
1 hello

View File

@ -1,5 +1,5 @@
CREATE TABLE test.cast1 (`x` UInt8, `e` Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_cast', 'r1') ORDER BY e SETTINGS index_granularity = 8192
x UInt8
e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\')
x UInt8
e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\')
1 hello
1 hello

View File

@ -1,9 +1,9 @@
a Decimal(9, 4) DEFAULT CAST(0, \'Decimal(9, 4)\')
b Decimal(18, 4) DEFAULT CAST(a / 2, \'Decimal(18, 4)\')
c Decimal(38, 4) DEFAULT CAST(b / 3, \'Decimal(38, 4)\')
d Decimal(9, 4) MATERIALIZED a + toDecimal32(\'0.2\', 1)
e Decimal(18, 4) ALIAS b * 2
f Decimal(38, 4) ALIAS c * 6
a Decimal(9, 4) DEFAULT CAST(0, \'Decimal(9, 4)\')
b Decimal(18, 4) DEFAULT CAST(a / 2, \'Decimal(18, 4)\')
c Decimal(38, 4) DEFAULT CAST(b / 3, \'Decimal(38, 4)\')
d Decimal(9, 4) MATERIALIZED a + toDecimal32(\'0.2\', 1)
e Decimal(18, 4) ALIAS b * 2
f Decimal(38, 4) ALIAS c * 6
0.0000 0.0000 0.0000
1.0000 0.5000 0.1666
2.0000 1.0000 0.3333

View File

@ -1,9 +1,9 @@
CREATE TABLE test.check_query_comment_column (`first_column` UInt8 DEFAULT 1 COMMENT \'comment 1\', `second_column` UInt8 MATERIALIZED first_column COMMENT \'comment 2\', `third_column` UInt8 ALIAS second_column COMMENT \'comment 3\', `fourth_column` UInt8 COMMENT \'comment 4\', `fifth_column` UInt8) ENGINE = TinyLog
first_column UInt8 DEFAULT 1 comment 1
second_column UInt8 MATERIALIZED first_column comment 2
third_column UInt8 ALIAS second_column comment 3
fourth_column UInt8 comment 4
fifth_column UInt8
first_column UInt8 DEFAULT 1 comment 1
second_column UInt8 MATERIALIZED first_column comment 2
third_column UInt8 ALIAS second_column comment 3
fourth_column UInt8 comment 4
fifth_column UInt8
┌─table──────────────────────┬─name──────────┬─comment───┐
│ check_query_comment_column │ first_column │ comment 1 │
│ check_query_comment_column │ second_column │ comment 2 │
@ -21,9 +21,9 @@ CREATE TABLE test.check_query_comment_column (`first_column` UInt8 DEFAULT 1 COM
└────────────────────────────┴───────────────┴─────────────┘
CREATE TABLE test.check_query_comment_column (`first_column` UInt8 DEFAULT 1 COMMENT \'comment 1_2\', `second_column` UInt8 MATERIALIZED first_column COMMENT \'comment 2_2\', `third_column` UInt8 ALIAS second_column COMMENT \'comment 3_2\', `fourth_column` UInt8 COMMENT \'comment 4_2\', `fifth_column` UInt8 COMMENT \'comment 5_2\') ENGINE = TinyLog
CREATE TABLE test.check_query_comment_column (`first_column` UInt8 COMMENT \'comment 1\', `second_column` UInt8 COMMENT \'comment 2\', `third_column` UInt8 COMMENT \'comment 3\') ENGINE = MergeTree() PARTITION BY second_column ORDER BY first_column SAMPLE BY first_column SETTINGS index_granularity = 8192
first_column UInt8 comment 1
second_column UInt8 comment 2
third_column UInt8 comment 3
first_column UInt8 comment 1
second_column UInt8 comment 2
third_column UInt8 comment 3
┌─table──────────────────────┬─name──────────┬─comment───┐
│ check_query_comment_column │ first_column │ comment 1 │
│ check_query_comment_column │ second_column │ comment 2 │

View File

@ -1,6 +1,6 @@
CREATE TABLE test.check_comments (`column_name1` UInt8 DEFAULT 1 COMMENT \'comment\', `column_name2` UInt8 COMMENT \'non default comment\') ENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_comments\', \'r1\') ORDER BY column_name1 SETTINGS index_granularity = 8192
column_name1 UInt8 DEFAULT 1 comment
column_name2 UInt8 non default comment
column_name1 UInt8 DEFAULT 1 comment
column_name2 UInt8 non default comment
CREATE TABLE test.check_comments (`column_name1` UInt8 DEFAULT 1 COMMENT \'another comment\', `column_name2` UInt8 COMMENT \'non default comment\') ENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_comments\', \'r1\') ORDER BY column_name1 SETTINGS index_granularity = 8192
column_name1 UInt8 DEFAULT 1 another comment
column_name2 UInt8 non default comment
column_name1 UInt8 DEFAULT 1 another comment
column_name2 UInt8 non default comment

View File

@ -1,3 +1,3 @@
x UInt32
y String MATERIALIZED toString(x)
z UInt64
x UInt32
y String MATERIALIZED toString(x)
z UInt64

View File

@ -0,0 +1,5 @@
CREATE TABLE test.ttl (`d` Date, `a` Int32) ENGINE = MergeTree PARTITION BY toDayOfMonth(d) ORDER BY a TTL d + toIntervalDay(1) SETTINGS index_granularity = 8192
2100-10-10 3
2100-10-10 4
d Date
a Int32 d + toIntervalDay(1)

View File

@ -0,0 +1,24 @@
set send_logs_level = 'none';
drop table if exists test.ttl;
create table test.ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d);
alter table test.ttl modify ttl d + interval 1 day;
show create table test.ttl;
insert into test.ttl values (toDateTime('2000-10-10 00:00:00'), 1);
insert into test.ttl values (toDateTime('2000-10-10 00:00:00'), 2);
insert into test.ttl values (toDateTime('2100-10-10 00:00:00'), 3);
insert into test.ttl values (toDateTime('2100-10-10 00:00:00'), 4);
optimize table test.ttl partition 10 final;
select * from test.ttl order by d;
alter table test.ttl modify ttl a; -- { serverError 450 }
drop table if exists test.ttl;
create table test.ttl (d Date, a Int) engine = MergeTree order by tuple() partition by toDayOfMonth(d);
alter table test.ttl modify column a Int ttl d + interval 1 day;
desc table test.ttl;
alter table test.ttl modify column d Int ttl d + interval 1 day; -- { serverError 44}

View File

@ -0,0 +1,8 @@
0
0 0
0 0
2000-10-10 00:00:00 0
2000-10-10 00:00:00 0
2000-10-10 00:00:00 0
2100-10-10 00:00:00 3
2100-10-10 2

View File

@ -0,0 +1,44 @@
drop table if exists test.ttl;
create table test.ttl (d DateTime, a Int ttl d + interval 1 second, b Int ttl d + interval 1 second) engine = MergeTree order by tuple() partition by toMinute(d);
insert into test.ttl values (now(), 1, 2);
insert into test.ttl values (now(), 3, 4);
select sleep(1.1);
optimize table test.ttl final;
select a, b from test.ttl;
drop table if exists test.ttl;
create table test.ttl (d DateTime, a Int ttl d + interval 1 DAY) engine = MergeTree order by tuple() partition by toDayOfMonth(d);
insert into test.ttl values (toDateTime('2000-10-10 00:00:00'), 1);
insert into test.ttl values (toDateTime('2000-10-10 00:00:00'), 2);
insert into test.ttl values (toDateTime('2000-10-10 00:00:00'), 3);
optimize table test.ttl final;
select * from test.ttl order by d;
drop table if exists test.ttl;
create table test.ttl (d DateTime, a Int) engine = MergeTree order by tuple() partition by tuple() ttl d + interval 1 day;
insert into test.ttl values (toDateTime('2000-10-10 00:00:00'), 1);
insert into test.ttl values (toDateTime('2000-10-10 00:00:00'), 2);
insert into test.ttl values (toDateTime('2100-10-10 00:00:00'), 3);
optimize table test.ttl final;
select * from test.ttl order by d;
drop table if exists test.ttl;
create table test.ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) ttl d + interval 1 day;
insert into test.ttl values (toDate('2000-10-10'), 1);
insert into test.ttl values (toDate('2100-10-10'), 2);
optimize table test.ttl final;
select * from test.ttl order by d;
set send_logs_level = 'none';
drop table if exists test.ttl;
create table test.ttl (d DateTime ttl d) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 44}
create table test.ttl (d DateTime, a Int ttl d) engine = MergeTree order by a partition by toSecond(d); -- { serverError 44}
create table test.ttl (d DateTime, a Int ttl 2 + 2) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 450 }
create table test.ttl (d DateTime, a Int ttl toDateTime(1)) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 450 }
create table test.ttl (d DateTime, a Int ttl d - d) engine = MergeTree order by tuple() partition by toSecond(d); -- { serverError 450 }

View File

@ -0,0 +1,12 @@
3
4
111
111
1 2
2 4
3 300
4 400
1 222
2 222
3 15
4 20

View File

@ -0,0 +1,30 @@
drop table if exists test.ttl;
create table test.ttl (d DateTime, a Int default 111 ttl d + interval 1 DAY) engine = MergeTree order by tuple() partition by toDayOfMonth(d);
insert into test.ttl values (toDateTime('2000-10-10 00:00:00'), 1);
insert into test.ttl values (toDateTime('2000-10-10 00:00:00'), 2);
insert into test.ttl values (toDateTime('2100-10-10 00:00:00'), 3);
insert into test.ttl values (toDateTime('2100-10-10 00:00:00'), 4);
optimize table test.ttl final;
select a from test.ttl order by a;
drop table if exists test.ttl;
create table test.ttl (d DateTime, a Int, b default a * 2 ttl d + interval 1 DAY) engine = MergeTree order by tuple() partition by toDayOfMonth(d);
insert into test.ttl values (toDateTime('2000-10-10 00:00:00'), 1, 100);
insert into test.ttl values (toDateTime('2000-10-10 00:00:00'), 2, 200);
insert into test.ttl values (toDateTime('2100-10-10 00:00:00'), 3, 300);
insert into test.ttl values (toDateTime('2100-10-10 00:00:00'), 4, 400);
optimize table test.ttl final;
select a, b from test.ttl order by a;
drop table if exists test.ttl;
create table test.ttl (d DateTime, a Int, b default 222 ttl d + interval 1 DAY) engine = MergeTree order by tuple() partition by toDayOfMonth(d);
insert into test.ttl values (toDateTime('2000-10-10 00:00:00'), 1, 5);
insert into test.ttl values (toDateTime('2000-10-10 00:00:00'), 2, 10);
insert into test.ttl values (toDateTime('2100-10-10 00:00:00'), 3, 15);
insert into test.ttl values (toDateTime('2100-10-10 00:00:00'), 4, 20);
optimize table test.ttl final;
select a, b from test.ttl order by a;

View File

@ -31,8 +31,8 @@ Main features:
```
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
(
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2],
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2],
...
INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1,
INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2
@ -41,6 +41,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
[ORDER BY expr]
[PRIMARY KEY expr]
[SAMPLE BY expr]
[TTL expr]
[SETTINGS name=value, ...]
```
@ -66,10 +67,18 @@ For a description of request parameters, see [request description](../../query_l
If a sampling expression is used, the primary key must contain it. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`.
- `TTL` - An expression for setting storage time for rows.
It must depends on `Date` or `DateTime` column and has one `Date` or `DateTime` column as a result. Example:
`TTL date + INTERVAL 1 DAY`
For more details, see [TTL for columns and tables](mergetree.md)
- `SETTINGS` — Additional parameters that control the behavior of the `MergeTree`:
- `index_granularity` — The granularity of an index. The number of data rows between the "marks" of an index. By default, 8192. The list of all available parameters you can see in [MergeTreeSettings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Storages/MergeTree/MergeTreeSettings.h).
- `use_minimalistic_part_header_in_zookeeper` — Storage method of the data parts headers in ZooKeeper. If `use_minimalistic_part_header_in_zookeeper=1`, then ZooKeeper stores less data. For more information refer the [setting description](../server_settings/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) in the "Server configuration parameters" chapter.
- `min_merge_bytes_to_use_direct_io` — The minimum data volume for merge operation required for using of the direct I/O access to the storage disk. During the merging of the data parts, ClickHouse calculates summary storage volume of all the data to be merged. If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, then ClickHouse reads and writes the data using direct I/O interface (`O_DIRECT` option) to the storage disk. If `min_merge_bytes_to_use_direct_io = 0`, then the direct I/O is disabled. Default value: `10 * 1024 * 1024 * 1024` bytes.
- `merge_with_ttl_timeout` - Minimal time in seconds, when merge with TTL can be repeated. Default value: 86400 (1 day).
**Example of sections setting**
@ -298,4 +307,13 @@ For concurrent table access, we use multi-versioning. In other words, when a tab
Reading from a table is automatically parallelized.
## TTL for columns and tables
Data with expired TTL is removed while executing merges.
If TTL is set for column, when it expires, value will be replaced by default. If all values in columns were zeroed in part, data for this column will be deleted from disk for part. You are not allowed to set TTL for all key columns. If TTL is set for table, when it expires, row will be deleted.
When TTL expires on some value or row in part, extra merge will be executed. To control frequency of merges with TTL you can set `merge_with_ttl_timeout`. If it is too low, many extra merges and lack of regular merges can reduce the perfomance.
[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/mergetree/) <!--hide-->

View File

@ -17,8 +17,8 @@ The `CREATE TABLE` query can have several forms.
```sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
(
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [compression_codec],
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec] [TTL expr1],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [compression_codec] [TTL expr2],
...
) ENGINE = engine
```
@ -80,6 +80,13 @@ If you add a new column to a table but later change its default expression, the
It is not possible to set default values for elements in nested data structures.
### TTL expression
Can be specified only for MergeTree-family tables. An expression for setting storage time for values. It must depends on `Date` or `DateTime` column and has one `Date` or `DateTime` column as a result. Example:
`TTL date + INTERVAL 1 DAY`
You are not allowed to set TTL for key columns. For more details, see [TTL for columns and tables](../operations/table_engines/mergetree.md)
## Column Compression Codecs
Besides default data compression, defined in [server settings](../operations/server_settings/settings.md#compression), per-column specification is also available.

View File

@ -30,8 +30,8 @@
```sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
(
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2],
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [TTL expr1],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [TTL expr2],
...
INDEX index_name1 expr1 TYPE type1(...) GRANULARITY value1,
INDEX index_name2 expr2 TYPE type2(...) GRANULARITY value2
@ -66,10 +66,18 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
Если используется выражение для сэмплирования, то первичный ключ должен содержать его. Пример:
`SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`.
- `TTL` - выражение для задания времени хранения строк.
Оно должно зависеть от стобца типа `Date` или `DateTime` и в качестве результата вычислять столбец типа `Date` или `DateTime`. Пример:
`TTL date + INTERVAL 1 DAY`
Подробнее смотрите в [TTL для стоблцов и таблиц](mergetree.md)
- `SETTINGS` — дополнительные параметры, регулирующие поведение `MergeTree`:
- `index_granularity` — гранулярность индекса. Число строк данных между «засечками» индекса. По умолчанию — 8192. Список всех доступных параметров можно посмотреть в [MergeTreeSettings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Storages/MergeTree/MergeTreeSettings.h).
- `min_merge_bytes_to_use_direct_io` — минимальный объем данных, необходимый для прямого (небуферизованного) чтения/записи (direct I/O) на диск. При слиянии частей данных ClickHouse вычисляет общий объем хранения всех данных, подлежащих слиянию. Если общий объем хранения всех данных для чтения превышает `min_bytes_to_use_direct_io` байт, тогда ClickHouse использует флаг `O_DIRECT` при чтении данных с диска. Если `min_merge_bytes_to_use_direct_io = 0`, тогда прямой ввод-вывод отключен. Значение по умолчанию: `10 * 1024 * 1024 * 1024` байт.
- `merge_with_ttl_timeout` - Минимальное время в секундах для повторного выполнения слияний с TTL. По умолчанию - 86400 (1 день).
**Пример задания секций**
@ -282,4 +290,13 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT
Чтения из таблицы автоматически распараллеливаются.
## TTL для столбцов и таблиц
Данные с истекшим TTL удаляются во время слияний.
Если TTL указан для столбца, то когда он истекает, значение заменяется на значение по умолчанию. Если все значения столбца обнулены в куске, то данные этого столбца удаляются с диска в куске. Если TTL указан для таблицы, то когда он истекает, удаляется строка.
Когда истекает TTL для какого-нибудь значения или строки в куске, назначается внеочередное слияние. Чтобы контролировать частоту слияний с TTL, вы можете задать настройку `merge_with_ttl_timeout`. Если ее значение слишком мало, то будет происходить слишком много внеочередных слияний и мало обычных, вследствие чего может ухудшиться производительность.
[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/mergetree/) <!--hide-->

View File

@ -16,8 +16,8 @@ CREATE DATABASE [IF NOT EXISTS] db_name
```sql
CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
(
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [compression_codec],
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec] [TTL expr1],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [compression_codec] [TTL expr2],
...
) ENGINE = engine
```
@ -80,6 +80,14 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ...
Отсутствует возможность задать значения по умолчанию для элементов вложенных структур данных.
### Выражение для TTL
Может быть указано только для таблиц семейства MergeTree. Выражение для указания времени хранения значений. Оно должно зависеть от стобца типа `Date` или `DateTime` и в качестве результата вычислять столбец типа `Date` или `DateTime`. Пример:
`TTL date + INTERVAL 1 DAY`
Нельзя указывать TTL для ключевых столбцов. Подробнее смотрите в [TTL для стоблцов и таблиц](../operations/table_engines/mergetree.md)
## Форматы сжатия для колонок
Помимо сжатия для колонок по умолчанию, определяемого в [настройках сервера](../operations/server_settings/settings.md#compression),