Merge branch 'master' into fix-use-quota-bug

This commit is contained in:
mergify[bot] 2021-12-10 15:42:48 +00:00 committed by GitHub
commit 2ea137ba55
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 67 additions and 48 deletions

View File

@ -569,6 +569,12 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
stages.emplace_back(context);
const auto & column = columns_desc.get(command.column_name);
if (!column.default_desc.expression)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Cannot materialize column `{}` because it doesn't have default expression", column.name);
auto materialized_column = makeASTFunction(
"_CAST", column.default_desc.expression->clone(), std::make_shared<ASTLiteral>(column.type->getName()));

View File

@ -424,52 +424,59 @@ void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns)
void IMergeTreeDataPart::removeIfNeeded()
{
if (state == State::DeleteOnDestroy || is_temp)
if (!is_temp && state != State::DeleteOnDestroy)
return;
try
{
try
{
auto path = getFullRelativePath();
auto path = getFullRelativePath();
if (!volume->getDisk()->exists(path))
if (!volume->getDisk()->exists(path))
return;
if (is_temp)
{
String file_name = fileName(relative_path);
if (file_name.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "relative_path {} of part {} is invalid or not set", relative_path, name);
if (!startsWith(file_name, "tmp") && !endsWith(file_name, ".tmp_proj"))
{
LOG_ERROR(
storage.log,
"~DataPart() should remove part {} but its name doesn't start with \"tmp\" or end with \".tmp_proj\". Too "
"suspicious, keeping the part.",
path);
return;
if (is_temp)
{
String file_name = fileName(relative_path);
if (file_name.empty())
throw Exception("relative_path " + relative_path + " of part " + name + " is invalid or not set", ErrorCodes::LOGICAL_ERROR);
if (!startsWith(file_name, "tmp") && !endsWith(file_name, ".tmp_proj"))
{
LOG_ERROR(
storage.log,
"~DataPart() should remove part {} but its name doesn't start with \"tmp\" or end with \".tmp_proj\". Too "
"suspicious, keeping the part.",
path);
return;
}
}
if (parent_part)
{
std::optional<bool> keep_shared_data = keepSharedDataInDecoupledStorage();
if (!keep_shared_data.has_value())
return;
projectionRemove(parent_part->getFullRelativePath(), *keep_shared_data);
}
else
remove();
if (state == State::DeleteOnDestroy)
{
LOG_TRACE(storage.log, "Removed part from old location {}", path);
}
}
catch (...)
if (parent_part)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
std::optional<bool> keep_shared_data = keepSharedDataInDecoupledStorage();
if (!keep_shared_data.has_value())
return;
projectionRemove(parent_part->getFullRelativePath(), *keep_shared_data);
}
else
remove();
if (state == State::DeleteOnDestroy)
{
LOG_TRACE(storage.log, "Removed part from old location {}", path);
}
}
catch (...)
{
/// FIXME If part it temporary, then directory will not be removed for 1 day (temporary_directories_lifetime).
/// If it's tmp_merge_<part_name> or tmp_fetch_<part_name>,
/// then all future attempts to execute part producing operation will fail with "directory already exists".
/// Seems like it's especially important for remote disks, because removal may fail due to network issues.
tryLogCurrentException(__PRETTY_FUNCTION__);
assert(!is_temp);
assert(state != State::DeleteOnDestroy);
assert(state != State::Temporary);
}
}
@ -1157,14 +1164,17 @@ void IMergeTreeDataPart::remove() const
* And a race condition can happen that will lead to "File not found" error here.
*/
/// NOTE We rename part to delete_tmp_<relative_path> instead of delete_tmp_<name> to avoid race condition
/// when we try to remove two parts with the same name, but different relative paths,
/// for example all_1_2_1 (in Deleting state) and tmp_merge_all_1_2_1 (in Temporary state).
fs::path from = fs::path(storage.relative_data_path) / relative_path;
fs::path to = fs::path(storage.relative_data_path) / ("delete_tmp_" + name);
fs::path to = fs::path(storage.relative_data_path) / ("delete_tmp_" + relative_path);
// TODO directory delete_tmp_<name> is never removed if server crashes before returning from this function
auto disk = volume->getDisk();
if (disk->exists(to))
{
LOG_WARNING(storage.log, "Directory {} (to which part must be renamed before removing) already exists. Most likely this is due to unclean restart. Removing it.", fullPath(disk, to));
LOG_WARNING(storage.log, "Directory {} (to which part must be renamed before removing) already exists. Most likely this is due to unclean restart or race condition. Removing it.", fullPath(disk, to));
try
{
disk->removeSharedRecursive(fs::path(to) / "", *keep_shared_data);

View File

@ -198,6 +198,7 @@ public:
mutable std::atomic<time_t> remove_time { std::numeric_limits<time_t>::max() };
/// If true, the destructor will delete the directory with the part.
/// FIXME Why do we need this flag? What's difference from Temporary and DeleteOnDestroy state? Can we get rid of this?
bool is_temp = false;
/// If true it means that there are no ZooKeeper node for this part, so it should be deleted only from filesystem

View File

@ -1373,9 +1373,6 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil:
const auto storage_settings_ptr = getSettings();
String part_path = fs::path(replica_path) / "parts" / part_name;
//ops.emplace_back(zkutil::makeCheckRequest(
// zookeeper_path + "/columns", expected_columns_version));
if (storage_settings_ptr->use_minimalistic_part_header_in_zookeeper)
{
ops.emplace_back(zkutil::makeCreateRequest(
@ -1421,6 +1418,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd
Coordination::Requests new_ops;
for (const String & part_path : absent_part_paths_on_replicas)
{
/// NOTE Create request may fail with ZNONODE if replica is being dropped, we will throw an exception
new_ops.emplace_back(zkutil::makeCreateRequest(part_path, "", zkutil::CreateMode::Persistent));
new_ops.emplace_back(zkutil::makeRemoveRequest(part_path, -1));
}

View File

@ -12,9 +12,11 @@ function wait_for_query_to_start()
}
MAX_TIMEOUT=30
# TCP CLIENT
$CLICKHOUSE_CLIENT --max_execution_time 10 --query_id "test_01948_tcp_$CLICKHOUSE_DATABASE" -q \
$CLICKHOUSE_CLIENT --max_execution_time $MAX_TIMEOUT --query_id "test_01948_tcp_$CLICKHOUSE_DATABASE" -q \
"SELECT * FROM
(
SELECT a.name as n
@ -30,12 +32,12 @@ $CLICKHOUSE_CLIENT --max_execution_time 10 --query_id "test_01948_tcp_$CLICKHOUS
LIMIT 20
FORMAT Null" > /dev/null 2>&1 &
wait_for_query_to_start "test_01948_tcp_$CLICKHOUSE_DATABASE"
$CLICKHOUSE_CLIENT --max_execution_time 10 -q "KILL QUERY WHERE query_id = 'test_01948_tcp_$CLICKHOUSE_DATABASE' SYNC"
$CLICKHOUSE_CLIENT --max_execution_time $MAX_TIMEOUT -q "KILL QUERY WHERE query_id = 'test_01948_tcp_$CLICKHOUSE_DATABASE' SYNC"
# HTTP CLIENT
${CLICKHOUSE_CURL_COMMAND} -q --max-time 10 -sS "$CLICKHOUSE_URL&query_id=test_01948_http_$CLICKHOUSE_DATABASE" -d \
${CLICKHOUSE_CURL_COMMAND} -q --max-time $MAX_TIMEOUT -sS "$CLICKHOUSE_URL&query_id=test_01948_http_$CLICKHOUSE_DATABASE" -d \
"SELECT * FROM
(
SELECT a.name as n
@ -51,4 +53,4 @@ ${CLICKHOUSE_CURL_COMMAND} -q --max-time 10 -sS "$CLICKHOUSE_URL&query_id=test_0
LIMIT 20
FORMAT Null" > /dev/null 2>&1 &
wait_for_query_to_start "test_01948_http_$CLICKHOUSE_DATABASE"
$CLICKHOUSE_CURL --max-time 10 -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = 'test_01948_http_$CLICKHOUSE_DATABASE' SYNC"
$CLICKHOUSE_CURL --max-time $MAX_TIMEOUT -sS "$CLICKHOUSE_URL" -d "KILL QUERY WHERE query_id = 'test_01948_http_$CLICKHOUSE_DATABASE' SYNC"

View File

@ -5,6 +5,8 @@ SET mutations_sync = 2;
CREATE TABLE tmp (x Int64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY tuple();
INSERT INTO tmp SELECT * FROM system.numbers LIMIT 20;
ALTER TABLE tmp MATERIALIZE COLUMN x; -- { serverError 36 }
ALTER TABLE tmp ADD COLUMN s String DEFAULT toString(x);
SELECT groupArray(x), groupArray(s) FROM tmp;