mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Info about frozen parts in system.parts (#5471)
* wip * wip * wip * Clean * fix test * clean * fix test * fix test
This commit is contained in:
parent
261dd5668e
commit
1b0d1a4450
@ -1,7 +1,9 @@
|
||||
#include <Common/localBackup.h>
|
||||
#include "localBackup.h"
|
||||
|
||||
#include <Common/createHardLink.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <Poco/Path.h>
|
||||
#include <Poco/File.h>
|
||||
#include <string>
|
||||
#include <iostream>
|
||||
|
@ -1,8 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Path.h>
|
||||
#include <optional>
|
||||
|
||||
namespace Poco { class Path; }
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -2829,6 +2829,7 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String &
|
||||
String backup_part_absolute_path = part_absolute_path;
|
||||
backup_part_absolute_path.replace(0, clickhouse_path.size(), backup_path);
|
||||
localBackup(part_absolute_path, backup_part_absolute_path);
|
||||
part->is_frozen = true;
|
||||
++parts_processed;
|
||||
}
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Storages/MergeTree/MergeTreeDataMergerMutator.h>
|
||||
#include "MergeTreeDataMergerMutator.h"
|
||||
|
||||
#include <Storages/MergeTree/MergeTreeSequentialBlockInputStream.h>
|
||||
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
||||
#include <Storages/MergeTree/DiskSpaceMonitor.h>
|
||||
@ -25,12 +26,9 @@
|
||||
#include <Common/SimpleIncrement.h>
|
||||
#include <Common/interpolate.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/localBackup.h>
|
||||
#include <Common/createHardLink.h>
|
||||
|
||||
#include <Poco/File.h>
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
|
||||
#include <cmath>
|
||||
#include <numeric>
|
||||
#include <iomanip>
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <optional>
|
||||
#include "MergeTreeDataPart.h"
|
||||
|
||||
#include <optional>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
@ -14,13 +15,10 @@
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/localBackup.h>
|
||||
#include <Compression/CompressionInfo.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
|
||||
#include <Poco/File.h>
|
||||
#include <Poco/Path.h>
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
#include <common/JSON.h>
|
||||
|
||||
@ -728,7 +726,8 @@ void MergeTreeDataPart::accumulateColumnSizes(ColumnToSize & column_to_size) con
|
||||
void MergeTreeDataPart::loadColumns(bool require)
|
||||
{
|
||||
String path = getFullPath() + "columns.txt";
|
||||
if (!Poco::File(path).exists())
|
||||
Poco::File poco_file_path{path};
|
||||
if (!poco_file_path.exists())
|
||||
{
|
||||
if (require)
|
||||
throw Exception("No columns.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
@ -750,6 +749,8 @@ void MergeTreeDataPart::loadColumns(bool require)
|
||||
return;
|
||||
}
|
||||
|
||||
is_frozen = !poco_file_path.canWrite();
|
||||
|
||||
ReadBufferFromFile file = openForReading(path);
|
||||
columns.readText(file);
|
||||
}
|
||||
|
@ -106,6 +106,9 @@ struct MergeTreeDataPart
|
||||
/// If true it means that there are no ZooKeeper node for this part, so it should be deleted only from filesystem
|
||||
bool is_duplicate = false;
|
||||
|
||||
/// Frozen by ALTER TABLE ... FREEZE ...
|
||||
mutable bool is_frozen = false;
|
||||
|
||||
/**
|
||||
* Part state is a stage of its lifetime. States are ordered and state of a part could be increased only.
|
||||
* Part state should be modified under data_parts mutex.
|
||||
|
@ -1,30 +1,24 @@
|
||||
#include <Databases/IDatabase.h>
|
||||
#include "StorageMergeTree.h"
|
||||
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/localBackup.h>
|
||||
|
||||
#include <Interpreters/InterpreterAlterQuery.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/ActiveDataPartSet.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/PartitionCommands.h>
|
||||
#include <Storages/StorageMergeTree.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockOutputStream.h>
|
||||
#include <Storages/MergeTree/DiskSpaceMonitor.h>
|
||||
#include <Storages/MergeTree/MergeList.h>
|
||||
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <Poco/File.h>
|
||||
|
||||
#include <optional>
|
||||
|
||||
|
||||
|
@ -1,3 +1,5 @@
|
||||
#include "StorageSystemParts.h"
|
||||
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
@ -5,7 +7,6 @@
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <Storages/System/StorageSystemParts.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
|
||||
@ -38,6 +39,7 @@ StorageSystemParts::StorageSystemParts(const std::string & name)
|
||||
{"data_version", std::make_shared<DataTypeUInt64>()},
|
||||
{"primary_key_bytes_in_memory", std::make_shared<DataTypeUInt64>()},
|
||||
{"primary_key_bytes_in_memory_allocated", std::make_shared<DataTypeUInt64>()},
|
||||
{"is_frozen", std::make_shared<DataTypeUInt8>()},
|
||||
|
||||
{"database", std::make_shared<DataTypeString>()},
|
||||
{"table", std::make_shared<DataTypeString>()},
|
||||
@ -96,6 +98,7 @@ void StorageSystemParts::processNextStorage(MutableColumns & columns, const Stor
|
||||
columns[i++]->insert(static_cast<UInt64>(part->info.getDataVersion()));
|
||||
columns[i++]->insert(part->getIndexSizeInBytes());
|
||||
columns[i++]->insert(part->getIndexSizeInAllocatedBytes());
|
||||
columns[i++]->insert(part->is_frozen);
|
||||
|
||||
columns[i++]->insert(info.database);
|
||||
columns[i++]->insert(info.table);
|
||||
|
@ -1,3 +1,5 @@
|
||||
#include "StorageSystemPartsColumns.h"
|
||||
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
@ -5,7 +7,6 @@
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <Storages/System/StorageSystemPartsColumns.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreePartHeader.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/Exception.h>
|
||||
|
@ -15,7 +15,6 @@
|
||||
88cdc31ded355e7572d68d8cde525d3a shadow/1/data/test/partition_428/19700201_19700201_1_1_0/p.bin
|
||||
9e688c58a5487b8eaf69c9e1005ad0bf shadow/1/data/test/partition_428/19700102_19700102_2_2_0/primary.idx
|
||||
b01e3d4df40467db3f1c2d029f59f6a2 shadow/1/data/test/partition_428/19700201_19700201_1_1_0/checksums.txt
|
||||
b026324c6904b2a9cb4b88d6d61c81d1 shadow/increment.txt
|
||||
cfcb770c3ecd0990dcceb1bde129e6c6 shadow/1/data/test/partition_428/19700102_19700102_2_2_0/p.bin
|
||||
e2af3bef1fd129aea73a890ede1e7a30 shadow/1/data/test/partition_428/19700201_19700201_1_1_0/k.bin
|
||||
e6654eba9e88b001280d3bdd21ccc417 shadow/1/data/test/partition_428/19700102_19700102_2_2_0/checksums.txt
|
||||
@ -36,7 +35,6 @@ e6654eba9e88b001280d3bdd21ccc417 shadow/1/data/test/partition_428/19700102_1970
|
||||
88cdc31ded355e7572d68d8cde525d3a shadow/1/data/test/partition_428/19700201_19700201_1_1_0/p.bin
|
||||
9e688c58a5487b8eaf69c9e1005ad0bf shadow/1/data/test/partition_428/19700102_19700102_2_2_0/primary.idx
|
||||
b01e3d4df40467db3f1c2d029f59f6a2 shadow/1/data/test/partition_428/19700201_19700201_1_1_0/checksums.txt
|
||||
b026324c6904b2a9cb4b88d6d61c81d1 shadow/increment.txt
|
||||
cfcb770c3ecd0990dcceb1bde129e6c6 shadow/1/data/test/partition_428/19700102_19700102_2_2_0/p.bin
|
||||
e2af3bef1fd129aea73a890ede1e7a30 shadow/1/data/test/partition_428/19700201_19700201_1_1_0/k.bin
|
||||
e6654eba9e88b001280d3bdd21ccc417 shadow/1/data/test/partition_428/19700102_19700102_2_2_0/checksums.txt
|
||||
|
@ -26,7 +26,8 @@ done
|
||||
$chl "ALTER TABLE test.partition_428 FREEZE"
|
||||
|
||||
# Do `cd` for consistent output for reference
|
||||
cd $ch_dir && find shadow -type f -exec md5sum {} \; | sort
|
||||
# Do not check increment.txt - it can be changed by other tests with FREEZE
|
||||
cd $ch_dir && find shadow -type f -exec md5sum {} \; | grep "partition_428" | sed 's!shadow/[0-9]*/data/[a-z0-9_-]*/!shadow/1/data/test/!g' | sort | uniq
|
||||
|
||||
$chl "ALTER TABLE test.partition_428 DETACH PARTITION 197001"
|
||||
$chl "ALTER TABLE test.partition_428 ATTACH PARTITION 197001"
|
||||
@ -40,7 +41,7 @@ done
|
||||
$chl "ALTER TABLE test.partition_428 MODIFY COLUMN v1 Int8"
|
||||
|
||||
# Check the backup hasn't changed
|
||||
cd $ch_dir && find shadow -type f -exec md5sum {} \; | sort
|
||||
cd $ch_dir && find shadow -type f -exec md5sum {} \; | grep "partition_428" | sed 's!shadow/[0-9]*/data/[a-z0-9_-]*/!shadow/1/data/test/!g' | sort | uniq
|
||||
|
||||
$chl "OPTIMIZE TABLE test.partition_428"
|
||||
|
||||
|
@ -0,0 +1,19 @@
|
||||
19701001_1_1_0 0
|
||||
19701002_2_2_0 0
|
||||
19701003_3_3_0 0
|
||||
freeze one
|
||||
19701001_1_1_0 0
|
||||
19701002_2_2_0 1
|
||||
19701003_3_3_0 0
|
||||
freeze all
|
||||
19701001_1_1_0 1
|
||||
19701002_2_2_0 1
|
||||
19701003_3_3_0 1
|
||||
1970-10-01 00:00:01
|
||||
1970-10-02 00:00:01
|
||||
1970-10-02 00:00:02
|
||||
1970-10-03 00:00:01
|
||||
19701001_1_1_0 1
|
||||
19701002_2_2_0 1
|
||||
19701002_4_4_0 0
|
||||
19701003_3_3_0 1
|
14
dbms/tests/queries/0_stateless/00952_part_frozen_info.sql
Normal file
14
dbms/tests/queries/0_stateless/00952_part_frozen_info.sql
Normal file
@ -0,0 +1,14 @@
|
||||
DROP TABLE IF EXISTS part_info;
|
||||
CREATE TABLE part_info (t DateTime) ENGINE = MergeTree PARTITION BY toDate(t) ORDER BY (t);
|
||||
INSERT INTO part_info VALUES (toDateTime('1970-10-01 00:00:01')), (toDateTime('1970-10-02 00:00:01')), (toDateTime('1970-10-03 00:00:01'));
|
||||
SELECT name, is_frozen FROM system.parts WHERE `database` = currentDatabase() AND `table` = 'part_info';
|
||||
SELECT 'freeze one';
|
||||
ALTER TABLE part_info FREEZE PARTITION toDate('1970-10-02');
|
||||
SELECT name, is_frozen FROM system.parts WHERE `database` = currentDatabase() AND `table` = 'part_info';
|
||||
SELECT 'freeze all';
|
||||
ALTER TABLE part_info FREEZE;
|
||||
SELECT name, is_frozen FROM system.parts WHERE `database` = currentDatabase() AND `table` = 'part_info';
|
||||
INSERT INTO part_info VALUES (toDateTime('1970-10-02 00:00:02'));
|
||||
select * from part_info order by t;
|
||||
SELECT name, is_frozen FROM system.parts WHERE `database` = currentDatabase() AND `table` = 'part_info';
|
||||
DROP TABLE part_info;
|
Loading…
Reference in New Issue
Block a user