mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +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/rename.h>
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
|
#include <Poco/File.h>
|
||||||
|
|
||||||
#if defined(_GNU_SOURCE)
|
#if defined(linux) || defined(__linux) || defined(__linux__)
|
||||||
#include <unistd.h>
|
#include <unistd.h>
|
||||||
#include <sys/syscall.h>
|
#include <sys/syscall.h>
|
||||||
#include <linux/fs.h>
|
#include <linux/fs.h>
|
||||||
|
#include <sys/utsname.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
#if defined(__NR_renameat2)
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
extern const int ATOMIC_RENAME_FAIL;
|
extern const int ATOMIC_RENAME_FAIL;
|
||||||
extern const int SYSTEM_ERROR;
|
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)
|
static void renameat2(const std::string & old_path, const std::string & new_path, int flags)
|
||||||
{
|
{
|
||||||
if (old_path.empty() || new_path.empty())
|
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
|
#else
|
||||||
#define RENAME_NOREPLACE 0
|
#define RENAME_NOREPLACE -1
|
||||||
#define RENAME_EXCHANGE 0
|
#define RENAME_EXCHANGE -1
|
||||||
|
|
||||||
namespace ErrorCodes
|
[[noreturn]]
|
||||||
{
|
static void renameat2(const std::string &, const std::string &, int)
|
||||||
extern const int UNSUPPORTED_METHOD;
|
|
||||||
}
|
|
||||||
|
|
||||||
[[noreturn]] static void renameat2(const std::string &, const std::string &, int)
|
|
||||||
{
|
{
|
||||||
throw Exception("Compiled without renameat2() support", ErrorCodes::UNSUPPORTED_METHOD);
|
throw Exception("Compiled without renameat2() support", ErrorCodes::UNSUPPORTED_METHOD);
|
||||||
}
|
}
|
||||||
|
|
||||||
#endif
|
#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)
|
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)
|
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
|
#pragma once
|
||||||
#include <string>
|
#include <string>
|
||||||
|
|
||||||
#if defined(_GNU_SOURCE)
|
|
||||||
#include <sys/syscall.h>
|
|
||||||
#endif
|
|
||||||
|
|
||||||
namespace DB
|
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
|
/// 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);
|
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
|
/// 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);
|
void renameExchange(const std::string & old_path, const std::string & new_path);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -23,10 +23,10 @@ fi
|
|||||||
echo
|
echo
|
||||||
${CLICKHOUSE_CLIENT} --query "SELECT _table, d FROM merge('${CLICKHOUSE_DATABASE}', '^[abc]\$') ORDER BY _table"
|
${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 a NO DELAY"
|
||||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE b NO DELAY"
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE b"
|
||||||
${CLICKHOUSE_CLIENT} --query "DROP TABLE c NO DELAY"
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE c"
|
||||||
sleep 2
|
sleep 2
|
||||||
|
|
||||||
# Deduplication check for non-replicated root table
|
# 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 "INSERT INTO root VALUES (1)";
|
${CLICKHOUSE_CLIENT} --query "INSERT INTO root VALUES (1)";
|
||||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM a";
|
${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"
|
${CLICKHOUSE_CLIENT} --query "DROP TABLE a NO DELAY"
|
||||||
sleep 1
|
sleep 1
|
||||||
|
Loading…
Reference in New Issue
Block a user