Cosmetics, pt. VIII

This commit is contained in:
Robert Schulze 2024-05-23 18:00:53 +00:00
parent 5d82a94615
commit 12f60a4969
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A

View File

@ -42,6 +42,13 @@ constexpr uint64_t machine_seq_num_mask = (1ull << machine_seq_num_bits_count) -
/// max values
constexpr uint64_t max_machine_seq_num = machine_seq_num_mask;
uint64_t getTimestamp()
{
auto now = std::chrono::system_clock::now();
auto ticks_since_epoch = std::chrono::duration_cast<std::chrono::milliseconds>(now.time_since_epoch()).count();
return static_cast<uint64_t>(ticks_since_epoch) & ((1ull << timestamp_bits_count) - 1);
}
uint64_t getMachineId()
{
UUID server_uuid = ServerUUID::get();
@ -52,31 +59,24 @@ uint64_t getMachineId()
return (((hi * 11) ^ (lo * 17)) & machine_id_mask) >> machine_seq_num_bits_count;
}
uint64_t getTimestamp()
{
auto now = std::chrono::system_clock::now();
auto ticks_since_epoch = std::chrono::duration_cast<std::chrono::milliseconds>(now.time_since_epoch()).count();
return static_cast<uint64_t>(ticks_since_epoch) & ((1ull << timestamp_bits_count) - 1);
}
struct SnowflakeId
{
uint64_t timestamp;
uint64_t machind_id;
uint64_t machine_id;
uint64_t machine_seq_num;
};
SnowflakeId toSnowflakeId(uint64_t snowflake)
{
return {.timestamp = (snowflake >> (machine_id_bits_count + machine_seq_num_bits_count)),
.machind_id = ((snowflake & machine_id_mask) >> machine_seq_num_bits_count),
.machine_id = ((snowflake & machine_id_mask) >> machine_seq_num_bits_count),
.machine_seq_num = (snowflake & machine_seq_num_mask)};
}
uint64_t fromSnowflakeId(SnowflakeId components)
{
return (components.timestamp << (machine_id_bits_count + machine_seq_num_bits_count) |
components.machind_id << (machine_seq_num_bits_count) |
components.machine_id << (machine_seq_num_bits_count) |
components.machine_seq_num);
}
@ -93,7 +93,7 @@ struct SnowflakeIdRange
SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, size_t input_rows_count)
{
/// 1. `now`
SnowflakeId begin = {.timestamp = getTimestamp(), .machind_id = getMachineId(), .machine_seq_num = 0};
SnowflakeId begin = {.timestamp = getTimestamp(), .machine_id = getMachineId(), .machine_seq_num = 0};
/// 2. `begin`
if (begin.timestamp <= available.timestamp)
@ -111,7 +111,7 @@ SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, size_t in
else
end.timestamp = begin.timestamp;
end.machind_id = begin.machind_id;
end.machine_id = begin.machine_id;
end.machine_seq_num = (begin.machine_seq_num + input_rows_count) & machine_seq_num_mask;
return {begin, end};