remove boost::uuid

This commit is contained in:
Alexander Tokmakov 2020-01-17 19:07:20 +03:00
parent e179500c54
commit b322aaf1b5
8 changed files with 31 additions and 17 deletions

View File

@ -2,10 +2,24 @@
#include <Common/UInt128.h>
#include <common/strong_typedef.h>
#include <Common/thread_local_rng.h>
namespace DB
{
STRONG_TYPEDEF(UInt128, UUID)
namespace UUIDHelpers
{
inline UUID generateV4()
{
UInt128 res{thread_local_rng(), thread_local_rng()};
res.low = (res.low & 0xffffffffffff0fffull) | 0x0000000000004000ull;
res.high = (res.high & 0x3fffffffffffffffull) | 0x8000000000000000ull;
return UUID{res};
}
const UUID Nil = UUID(UInt128(0, 0));
}
}

View File

@ -112,7 +112,7 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query)
create->format = nullptr;
create->out_file = nullptr;
if (create->uuid != UUID(UInt128(0, 0)))
if (create->uuid != UUIDHelpers::Nil)
create->table = TABLE_WITH_UUID_NAME_PLACEHOLDER;
std::ostringstream statement_stream;
@ -245,9 +245,9 @@ void DatabaseOnDisk::renameTable(
auto & create = ast->as<ASTCreateQuery &>();
create.table = to_table_name;
if (from_ordinary_to_atomic)
create.uuid = parseFromString<UUID>(boost::uuids::to_string(boost::uuids::random_generator()()));
create.uuid = UUIDHelpers::generateV4();
if (from_atomic_to_ordinary)
create.uuid = UUID(UInt128(0, 0));
create.uuid = UUIDHelpers::Nil;
/// Notify the table that it is renamed. If the table does not support renaming, exception is thrown.
try
@ -427,7 +427,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(const Context & context, const Str
return nullptr;
auto & create = ast->as<ASTCreateQuery &>();
if (create.uuid != UUID(UInt128(0, 0)))
if (create.uuid != UUIDHelpers::Nil)
{
String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName();
if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER)

View File

@ -620,19 +620,19 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create,
if (database->getEngineName() == "Atomic")
{
//TODO implement ATTACH FROM 'path/to/data': generate UUID and move table data to store/
if (create.attach && create.uuid == UUID(UInt128(0, 0)))
if (create.attach && create.uuid == UUIDHelpers::Nil)
throw Exception("UUID must be specified in ATTACH TABLE query for Atomic database engine", ErrorCodes::INCORRECT_QUERY);
if (!create.attach && create.uuid == UUID(UInt128(0, 0)))
create.uuid = parseFromString<UUID>(boost::uuids::to_string(boost::uuids::random_generator()()));
if (!create.attach && create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4();
}
else
{
if (create.uuid != UUID(UInt128(0, 0)))
if (create.uuid != UUIDHelpers::Nil)
throw Exception("Table UUID specified, but engine of database " + database_name + " is not Atomic", ErrorCodes::INCORRECT_QUERY);
}
if (!create.attach && create.uuid == UUID(UInt128(0, 0)) && database->getEngineName() == "Atomic")
create.uuid = parseFromString<UUID>(boost::uuids::to_string(boost::uuids::random_generator()()));
if (!create.attach && create.uuid == UUIDHelpers::Nil && database->getEngineName() == "Atomic")
create.uuid = UUIDHelpers::generateV4();
data_path = database->getTableDataPath(create);

View File

@ -71,7 +71,7 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
//FIXME temporary print create query without UUID for tests (remove it)
auto & create = create_query->as<ASTCreateQuery &>();
create.uuid = UUID{UInt128{0, 0}};
create.uuid = UUIDHelpers::Nil;
std::stringstream stream;
formatAST(*create_query, stream, false, true);

View File

@ -234,7 +234,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
<< (if_not_exists ? "IF NOT EXISTS " : "")
<< (settings.hilite ? hilite_none : "")
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
if (uuid != UUID(UInt128(0, 0)))
if (uuid != UUIDHelpers::Nil)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "")
<< quoteString(toString(uuid));
formatOnCluster(settings);

View File

@ -229,7 +229,7 @@ bool parseStorageID(IParser::Pos & pos, StorageID & res, Expected & expected)
tryGetIdentifierNameInto(database, res.database_name);
tryGetIdentifierNameInto(table, res.table_name);
//FIXME
res.uuid = uuid ? parseFromString<UUID>(uuid->as<ASTLiteral>()->value.get<String>()) : UUID(UInt128(0, 0));
res.uuid = uuid ? parseFromString<UUID>(uuid->as<ASTLiteral>()->value.get<String>()) : UUIDHelpers::Nil;
return true;
}

View File

@ -19,10 +19,10 @@ struct StorageID
{
String database_name;
String table_name;
UUID uuid = UUID{UInt128(0, 0)};
UUID uuid = UUIDHelpers::Nil;
StorageID(const String & database, const String & table, UUID uuid_ = UUID{UInt128(0, 0)})
StorageID(const String & database, const String & table, UUID uuid_ = UUIDHelpers::Nil)
: database_name(database), table_name(table), uuid(uuid_)
{
assertNotEmpty();
@ -60,7 +60,7 @@ struct StorageID
bool hasUUID() const
{
return uuid != UUID{UInt128(0, 0)};
return uuid != UUIDHelpers::Nil;
}
bool operator<(const StorageID & rhs) const

View File

@ -15,7 +15,7 @@ path_to_data = '/var/lib/clickhouse/'
def started_cluster():
try:
cluster.start()
q('CREATE DATABASE test')
q('CREATE DATABASE test ENGINE = Ordinary')
yield cluster