mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
add fallback for renameat2
This commit is contained in:
parent
a6d0aacd53
commit
58067438cd
@ -1,23 +1,52 @@
|
||||
#include <Common/rename.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Poco/File.h>
|
||||
|
||||
#if defined(_GNU_SOURCE)
|
||||
#if defined(linux) || defined(__linux) || defined(__linux__)
|
||||
#include <unistd.h>
|
||||
#include <sys/syscall.h>
|
||||
#include <linux/fs.h>
|
||||
#include <sys/utsname.h>
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
#if defined(__NR_renameat2)
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ATOMIC_RENAME_FAIL;
|
||||
extern const int SYSTEM_ERROR;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
extern const int FILE_ALREADY_EXISTS;
|
||||
}
|
||||
|
||||
static bool supportsRenameat2Impl()
|
||||
{
|
||||
#if defined(__NR_renameat2)
|
||||
/// renameat2 is available in linux since 3.15
|
||||
struct utsname sysinfo;
|
||||
if (uname(&sysinfo))
|
||||
return false;
|
||||
char * point = nullptr;
|
||||
long v_major = strtol(sysinfo.release, &point, 10);
|
||||
|
||||
errno = 0;
|
||||
if (errno || *point != '.' || v_major < 3)
|
||||
return false;
|
||||
if (3 < v_major)
|
||||
return true;
|
||||
|
||||
errno = 0;
|
||||
long v_minor = strtol(point + 1, nullptr, 10);
|
||||
return !errno && 15 <= v_minor;
|
||||
#else
|
||||
return false;
|
||||
#endif
|
||||
}
|
||||
|
||||
#if defined(__NR_renameat2)
|
||||
|
||||
static void renameat2(const std::string & old_path, const std::string & new_path, int flags)
|
||||
{
|
||||
if (old_path.empty() || new_path.empty())
|
||||
@ -39,29 +68,55 @@ static void renameat2(const std::string & old_path, const std::string & new_path
|
||||
}
|
||||
|
||||
#else
|
||||
#define RENAME_NOREPLACE 0
|
||||
#define RENAME_EXCHANGE 0
|
||||
#define RENAME_NOREPLACE -1
|
||||
#define RENAME_EXCHANGE -1
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
|
||||
[[noreturn]] static void renameat2(const std::string &, const std::string &, int)
|
||||
[[noreturn]]
|
||||
static void renameat2(const std::string &, const std::string &, int)
|
||||
{
|
||||
throw Exception("Compiled without renameat2() support", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
#endif
|
||||
|
||||
static void renameNoReplaceFallback(const std::string & old_path, const std::string & new_path)
|
||||
{
|
||||
/// NOTE it's unsafe
|
||||
if (Poco::File{new_path}.exists())
|
||||
throw Exception("File " + new_path + " exists", ErrorCodes::FILE_ALREADY_EXISTS);
|
||||
Poco::File{old_path}.renameTo(new_path);
|
||||
}
|
||||
|
||||
/// Do not use [[noreturn]] to avoid warnings like "code will never be executed" in other places
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wmissing-noreturn"
|
||||
static void renameExchangeFallback(const std::string &, const std::string &)
|
||||
{
|
||||
throw Exception("System call renameat2() is not supported", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
|
||||
bool supportsRenameat2()
|
||||
{
|
||||
static bool supports = supportsRenameat2Impl();
|
||||
return supports;
|
||||
}
|
||||
|
||||
void renameNoReplace(const std::string & old_path, const std::string & new_path)
|
||||
{
|
||||
renameat2(old_path, new_path, RENAME_NOREPLACE);
|
||||
if (supportsRenameat2())
|
||||
renameat2(old_path, new_path, RENAME_NOREPLACE);
|
||||
else
|
||||
renameNoReplaceFallback(old_path, new_path);
|
||||
}
|
||||
|
||||
void renameExchange(const std::string & old_path, const std::string & new_path)
|
||||
{
|
||||
renameat2(old_path, new_path, RENAME_EXCHANGE);
|
||||
if (supportsRenameat2())
|
||||
renameat2(old_path, new_path, RENAME_EXCHANGE);
|
||||
else
|
||||
renameExchangeFallback(old_path, new_path);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,23 +1,17 @@
|
||||
#pragma once
|
||||
#include <string>
|
||||
|
||||
#if defined(_GNU_SOURCE)
|
||||
#include <sys/syscall.h>
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Returns true, if the following functions supported by the system
|
||||
bool supportsRenameat2();
|
||||
|
||||
/// Atomically rename old_path to new_path. If new_path exists, do not overwrite it and throw exception
|
||||
#if !defined(__NR_renameat2)
|
||||
[[noreturn]]
|
||||
#endif
|
||||
void renameNoReplace(const std::string & old_path, const std::string & new_path);
|
||||
|
||||
/// Atomically exchange oldpath and newpath. Throw exception if some of them does not exist
|
||||
#if !defined(__NR_renameat2)
|
||||
[[noreturn]]
|
||||
#endif
|
||||
void renameExchange(const std::string & old_path, const std::string & new_path);
|
||||
|
||||
}
|
||||
|
@ -23,10 +23,10 @@ fi
|
||||
echo
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT _table, d FROM merge('${CLICKHOUSE_DATABASE}', '^[abc]\$') ORDER BY _table"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE root NO DELAY"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE root"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE a NO DELAY"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE b NO DELAY"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE c NO DELAY"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE b"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE c"
|
||||
sleep 2
|
||||
|
||||
# Deduplication check for non-replicated root table
|
||||
@ -36,6 +36,6 @@ ${CLICKHOUSE_CLIENT} --query "CREATE MATERIALIZED VIEW a (d UInt64) ENGINE = Rep
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO root VALUES (1)";
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO root VALUES (1)";
|
||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM a";
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE root NO DELAY"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE root"
|
||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE a NO DELAY"
|
||||
sleep 1
|
||||
|
Loading…
Reference in New Issue
Block a user