mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
tests
This commit is contained in:
parent
e4212bfe59
commit
3ba26aba43
@ -347,7 +347,7 @@ UInt64 MergeTreeDataPart::calculateTotalSizeOnDisk(const String & from)
|
|||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergeTreeDataPart::remove() const
|
void MergeTreeDataPart::remove(bool force_recursive /*= false*/) const
|
||||||
{
|
{
|
||||||
if (relative_path.empty())
|
if (relative_path.empty())
|
||||||
throw Exception("Part relative_path cannot be empty. This is bug.", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Part relative_path cannot be empty. This is bug.", ErrorCodes::LOGICAL_ERROR);
|
||||||
@ -398,6 +398,13 @@ void MergeTreeDataPart::remove() const
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (force_recursive)
|
||||||
|
{
|
||||||
|
/// Part is not loaded (we don't know which files are there), so remove dir recursively.
|
||||||
|
to_dir.remove(true);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
/// Remove each expected file in directory, then remove directory itself.
|
/// Remove each expected file in directory, then remove directory itself.
|
||||||
|
@ -241,7 +241,7 @@ struct MergeTreeDataPart
|
|||||||
/// Calculate the total size of the entire directory with all the files
|
/// Calculate the total size of the entire directory with all the files
|
||||||
static UInt64 calculateTotalSizeOnDisk(const String & from);
|
static UInt64 calculateTotalSizeOnDisk(const String & from);
|
||||||
|
|
||||||
void remove() const;
|
void remove(bool force_recursive = false) const;
|
||||||
|
|
||||||
/// Makes checks and move part to new directory
|
/// Makes checks and move part to new directory
|
||||||
/// Changes only relative_dir_name, you need to update other metadata (name, is_temp) explicitly
|
/// Changes only relative_dir_name, you need to update other metadata (name, is_temp) explicitly
|
||||||
|
@ -1007,7 +1007,7 @@ void StorageMergeTree::dropDetached(const ASTPtr & partition, bool part, const C
|
|||||||
MergeTreeDataPart detached_part(*this, part_id, info);
|
MergeTreeDataPart detached_part(*this, part_id, info);
|
||||||
detached_part.relative_path = "detached/" + part_id;
|
detached_part.relative_path = "detached/" + part_id;
|
||||||
|
|
||||||
detached_part.remove();
|
detached_part.remove(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & context)
|
void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & context)
|
||||||
@ -1070,9 +1070,9 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par
|
|||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
tryLogCurrentException(log, String(__PRETTY_FUNCTION__) + ": cannot attach part " + source_part_name);
|
LOG_INFO(log, "Cannot attach part " << source_part_name << " :" << getCurrentExceptionMessage(false));
|
||||||
|
|
||||||
if (part->relative_path == "detached/attaching_" + source_part_name)
|
if (part && part->relative_path == "detached/attaching_" + source_part_name)
|
||||||
{
|
{
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
|
@ -1,6 +0,0 @@
|
|||||||
OK
|
|
||||||
0_1_1_0
|
|
||||||
1_2_2_0
|
|
||||||
2_3_3_0
|
|
||||||
3_4_4_0
|
|
||||||
16
|
|
@ -1,12 +0,0 @@
|
|||||||
#!/usr/bin/env bash
|
|
||||||
|
|
||||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|
||||||
. $CURDIR/../shell_config.sh
|
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS attach_bug";
|
|
||||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE attach_bug (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n";
|
|
||||||
$CLICKHOUSE_CLIENT --query="INSERT INTO attach_bug SELECT number FROM system.numbers LIMIT 16";
|
|
||||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_bug ATTACH PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK'
|
|
||||||
$CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_bug' ORDER BY name FORMAT TSV";
|
|
||||||
$CLICKHOUSE_CLIENT --query="SElECT count() FROM attach_bug FORMAT TSV";
|
|
||||||
$CLICKHOUSE_CLIENT --query="DROP TABLE attach_bug";
|
|
@ -0,0 +1,17 @@
|
|||||||
|
=== cannot attach active ===
|
||||||
|
OK
|
||||||
|
0_1_1_0
|
||||||
|
1_2_2_0
|
||||||
|
2_3_3_0
|
||||||
|
3_4_4_0
|
||||||
|
16 120
|
||||||
|
=== attach all valid parts ===
|
||||||
|
0_5_5_0
|
||||||
|
0_6_6_0
|
||||||
|
1_2_2_0
|
||||||
|
1_4_4_0
|
||||||
|
16 120
|
||||||
|
=== detached ===
|
||||||
|
0_5_5_0
|
||||||
|
delete_tmp_0_7_7
|
||||||
|
attaching_0_6_6
|
41
dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh
Executable file
41
dbms/tests/queries/0_stateless/00974_attach_invalid_parts.sh
Executable file
@ -0,0 +1,41 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
|
||||||
|
set -e
|
||||||
|
|
||||||
|
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
. $CURDIR/../shell_config.sh
|
||||||
|
|
||||||
|
ch_dir=`${CLICKHOUSE_EXTRACT_CONFIG} -k path`
|
||||||
|
cur_db=`${CLICKHOUSE_CLIENT} --query "SELECT currentDatabase()"`
|
||||||
|
|
||||||
|
echo '=== cannot attach active ===';
|
||||||
|
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS attach_active";
|
||||||
|
$CLICKHOUSE_CLIENT --query="CREATE TABLE attach_active (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n";
|
||||||
|
$CLICKHOUSE_CLIENT --query="INSERT INTO attach_active SELECT number FROM system.numbers LIMIT 16";
|
||||||
|
$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_active ATTACH PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK'
|
||||||
|
$CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_active' AND database='${cur_db}' ORDER BY name FORMAT TSV";
|
||||||
|
$CLICKHOUSE_CLIENT --query="SElECT count(), sum(n) FROM attach_active FORMAT TSV";
|
||||||
|
$CLICKHOUSE_CLIENT --query="DROP TABLE attach_active";
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
echo '=== attach all valid parts ===';
|
||||||
|
$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES";
|
||||||
|
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS attach_partitions";
|
||||||
|
$CLICKHOUSE_CLIENT --query="CREATE TABLE attach_partitions (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n";
|
||||||
|
$CLICKHOUSE_CLIENT --query="INSERT INTO attach_partitions SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8";
|
||||||
|
$CLICKHOUSE_CLIENT --query="INSERT INTO attach_partitions SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8";
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions DETACH PARTITION 0";
|
||||||
|
mkdir $ch_dir/data/$cur_db/attach_partitions/detached/0_5_5_0/ # broken part
|
||||||
|
cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_1_1_0/ $ch_dir/data/$cur_db/attach_partitions/detached/attaching_0_6_6_0/
|
||||||
|
cp -r $ch_dir/data/$cur_db/attach_partitions/detached/0_3_3_0/ $ch_dir/data/$cur_db/attach_partitions/detached/delete_tmp_0_7_7_0/
|
||||||
|
$CLICKHOUSE_CLIENT --query="ALTER TABLE attach_partitions ATTACH PARTITION 0";
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT --query="SElECT name FROM system.parts WHERE table='attach_partitions' AND database='${cur_db}' ORDER BY name FORMAT TSV";
|
||||||
|
$CLICKHOUSE_CLIENT --query="SElECT count(), sum(n) FROM attach_partitions FORMAT TSV";
|
||||||
|
echo '=== detached ===';
|
||||||
|
$CLICKHOUSE_CLIENT --query="SELECT directory_name FROM system.detached_parts WHERE table='attach_partitions' AND database='${cur_db}' FORMAT TSV";
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT --query="DROP TABLE attach_partitions";
|
||||||
|
$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES";
|
@ -0,0 +1,2 @@
|
|||||||
|
OK
|
||||||
|
0_3_3_0
|
30
dbms/tests/queries/0_stateless/00975_drop_detached.sh
Executable file
30
dbms/tests/queries/0_stateless/00975_drop_detached.sh
Executable file
@ -0,0 +1,30 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
|
||||||
|
set -e
|
||||||
|
|
||||||
|
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
. $CURDIR/../shell_config.sh
|
||||||
|
|
||||||
|
ch_dir=`${CLICKHOUSE_EXTRACT_CONFIG} -k path`
|
||||||
|
cur_db=`${CLICKHOUSE_CLIENT} --query "SELECT currentDatabase()"`
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES";
|
||||||
|
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS drop_detached";
|
||||||
|
$CLICKHOUSE_CLIENT --query="CREATE TABLE drop_detached (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n";
|
||||||
|
$CLICKHOUSE_CLIENT --query="INSERT INTO drop_detached SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8";
|
||||||
|
$CLICKHOUSE_CLIENT --query="INSERT INTO drop_detached SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 8";
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT --query="ALTER TABLE drop_detached DETACH PARTITION 0";
|
||||||
|
mkdir $ch_dir/data/$cur_db/drop_detached/detached/attaching_0_6_6_0/
|
||||||
|
mkdir $ch_dir/data/$cur_db/drop_detached/detached/delete_tmp_0_7_7_0/
|
||||||
|
mkdir $ch_dir/data/$cur_db/drop_detached/detached/any_other_name/
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '../1_2_2_0'" 2>&1 | grep "Invalid part name" > /dev/null && echo 'OK'
|
||||||
|
$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART '0_1_1_0'"
|
||||||
|
$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'attaching_0_6_6_0'"
|
||||||
|
$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'delete_tmp_0_7_7_0'"
|
||||||
|
$CLICKHOUSE_CLIENT --allow_drop_detached=1 --query="ALTER TABLE drop_detached DROP DETACHED PART 'any_other_name'"
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT --query="SElECT directory_name FROM system.detached_parts WHERE table='drop_detached' AND database='${cur_db}' FORMAT TSV";
|
||||||
|
$CLICKHOUSE_CLIENT --query="DROP TABLE drop_detached";
|
||||||
|
$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES";
|
Loading…
Reference in New Issue
Block a user