Added move TTL rules to metadata of ReplicatedMergeTree.

This commit is contained in:
Vladimir Chebotarev 2019-11-28 13:13:53 +03:00
parent 7407f7c39d
commit 14cf530542
4 changed files with 43 additions and 5 deletions

View File

@ -38,6 +38,7 @@
#include <Common/Increment.h>
#include <Common/SimpleIncrement.h>
#include <Common/escapeForFileName.h>
#include <Common/quoteString.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Stopwatch.h>
#include <Common/typeid_cast.h>
@ -130,7 +131,6 @@ MergeTreeData::MergeTreeData(
, merging_params(merging_params_)
, 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_)
@ -580,7 +580,7 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription::ColumnTTLs & new
String result_column = ttl_ast->getColumnName();
checkTTLExpression(expr, result_column);
return {expr, result_column, PartDestinationType::DELETE, {}};
return {expr, result_column, PartDestinationType::DELETE, {}, ttl_ast};
};
if (!new_column_ttls.empty())

View File

@ -587,7 +587,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_entry.expression != nullptr; }
bool hasTableTTL() const { return ttl_table_ast != nullptr; }
bool hasAnyColumnTTL() const { return !column_ttl_entries_by_name.empty(); }
/// Check that the part is not broken and calculate the checksums for it if they are not present.
@ -731,6 +731,8 @@ public:
/// Name and type of a destination are only valid in table-level context.
PartDestinationType destination_type;
String destination_name;
ASTPtr entry_ast;
};
using TTLEntriesByName = std::unordered_map<String, TTLEntry>;

View File

@ -5,6 +5,7 @@
#include <Parsers/ExpressionListParsers.h>
#include <IO/Operators.h>
namespace DB
{
@ -47,6 +48,16 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr
partition_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.partition_by_ast));
ttl_table = formattedAST(data.ttl_table_ast);
std::ostringstream ttl_move_stream;
for (const auto & ttl_entry : data.move_ttl_entries)
{
if (ttl_move_stream.tellp() > 0)
ttl_move_stream << ", ";
ttl_move_stream << formattedAST(ttl_entry.entry_ast);
}
ttl_move = ttl_move_stream.str();
skip_indices = data.getIndices().toString();
if (data.canUseAdaptiveGranularity())
index_granularity_bytes = data_settings->index_granularity_bytes;
@ -78,6 +89,9 @@ void ReplicatedMergeTreeTableMetadata::write(WriteBuffer & out) const
if (!ttl_table.empty())
out << "ttl: " << ttl_table << "\n";
if (!ttl_move.empty())
out << "move ttl: " << ttl_move << "\n";
if (!skip_indices.empty())
out << "indices: " << skip_indices << "\n";
@ -119,6 +133,9 @@ void ReplicatedMergeTreeTableMetadata::read(ReadBuffer & in)
if (checkString("ttl: ", in))
in >> ttl_table >> "\n";
if (checkString("move ttl: ", in))
in >> ttl_move >> "\n";
if (checkString("indices: ", in))
in >> skip_indices >> "\n";
@ -223,12 +240,27 @@ ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTabl
}
else
throw Exception(
"Existing table metadata in ZooKeeper differs in ttl."
"Existing table metadata in ZooKeeper differs in TTL."
" Stored in ZooKeeper: " + from_zk.ttl_table +
", local: " + ttl_table,
ErrorCodes::METADATA_MISMATCH);
}
if (ttl_move != from_zk.ttl_move)
{
if (allow_alter)
{
diff.ttl_move_changed = true;
diff.new_ttl_move = from_zk.ttl_move;
}
else
throw Exception(
"Existing table metadata in ZooKeeper differs in move TTL."
" Stored in ZooKeeper: " + from_zk.ttl_move +
", local: " + ttl_move,
ErrorCodes::METADATA_MISMATCH);
}
if (skip_indices != from_zk.skip_indices)
{
if (allow_alter)

View File

@ -28,6 +28,7 @@ struct ReplicatedMergeTreeTableMetadata
String skip_indices;
String constraints;
String ttl_table;
String ttl_move;
UInt64 index_granularity_bytes;
ReplicatedMergeTreeTableMetadata() = default;
@ -53,9 +54,12 @@ struct ReplicatedMergeTreeTableMetadata
bool ttl_table_changed = false;
String new_ttl_table;
bool ttl_move_changed = false;
String new_ttl_move;
bool empty() const
{
return !sorting_key_changed && !skip_indices_changed && !ttl_table_changed && !constraints_changed;
return !sorting_key_changed && !skip_indices_changed && !ttl_table_changed && !constraints_changed && !ttl_move_changed;
}
};