mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
Cosmetics, pt. I
This commit is contained in:
parent
d72cd45bd8
commit
1293a0f795
@ -18,8 +18,7 @@ namespace ErrorCodes
|
||||
namespace
|
||||
{
|
||||
|
||||
/*
|
||||
Snowflake ID
|
||||
/* Snowflake ID
|
||||
https://en.wikipedia.org/wiki/Snowflake_ID
|
||||
|
||||
0 1 2 3
|
||||
@ -30,35 +29,34 @@ namespace
|
||||
| | machine_id | machine_seq_num |
|
||||
├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤
|
||||
|
||||
- The first 41 (+ 1 top zero bit) bits is timestamp in Unix time milliseconds
|
||||
- The middle 10 bits are the machine ID.
|
||||
- The last 12 bits decode to number of ids processed by the machine at the given millisecond.
|
||||
- The first 41 (+ 1 top zero bit) bits is the timestamp (millisecond since Unix epoch 1 Jan 1970)
|
||||
- The middle 10 bits are the machine ID
|
||||
- The last 12 bits are a counter to disambiguate multiple snowflakeIDs generated within the same millisecond by differen processes
|
||||
*/
|
||||
|
||||
constexpr auto timestamp_size = 41;
|
||||
constexpr auto machine_id_size = 10;
|
||||
constexpr auto machine_seq_num_size = 12;
|
||||
constexpr auto timestamp_bits_count = 41;
|
||||
constexpr auto machine_id_bits_count = 10;
|
||||
constexpr auto machine_seq_num_bits_count = 12;
|
||||
|
||||
constexpr int64_t timestamp_mask = ((1LL << timestamp_size) - 1) << (machine_id_size + machine_seq_num_size);
|
||||
constexpr int64_t machine_id_mask = ((1LL << machine_id_size) - 1) << machine_seq_num_size;
|
||||
constexpr int64_t machine_seq_num_mask = (1LL << machine_seq_num_size) - 1;
|
||||
constexpr int64_t timestamp_mask = ((1LL << timestamp_bits_count) - 1) << (machine_id_bits_count + machine_seq_num_bits_count);
|
||||
constexpr int64_t machine_id_mask = ((1LL << machine_id_bits_count) - 1) << machine_seq_num_bits_count;
|
||||
constexpr int64_t machine_seq_num_mask = (1LL << machine_seq_num_bits_count) - 1;
|
||||
constexpr int64_t max_machine_seq_num = machine_seq_num_mask;
|
||||
|
||||
Int64 getMachineID()
|
||||
{
|
||||
auto serverUUID = ServerUUID::get();
|
||||
|
||||
// hash serverUUID into 64 bits
|
||||
Int64 h = UUIDHelpers::getHighBytes(serverUUID);
|
||||
Int64 l = UUIDHelpers::getLowBytes(serverUUID);
|
||||
return ((h * 11) ^ (l * 17)) & machine_id_mask;
|
||||
UUID server_uuid = ServerUUID::get();
|
||||
/// hash into 64 bits
|
||||
UInt64 hi = UUIDHelpers::getHighBytes(server_uuid);
|
||||
UInt64 lo = UUIDHelpers::getLowBytes(server_uuid);
|
||||
return ((hi * 11) ^ (lo * 17)) & machine_id_mask;
|
||||
}
|
||||
|
||||
Int64 getTimestamp()
|
||||
{
|
||||
const auto tm_point = std::chrono::system_clock::now();
|
||||
return std::chrono::duration_cast<std::chrono::milliseconds>(
|
||||
tm_point.time_since_epoch()).count() & ((1LL << timestamp_size) - 1);
|
||||
auto now = std::chrono::system_clock::now();
|
||||
auto ticks_since_epoch = std::chrono::duration_cast<std::chrono::milliseconds>(now.time_since_epoch()).count();
|
||||
return ticks_since_epoch & ((1LL << timestamp_bits_count) - 1);
|
||||
}
|
||||
|
||||
}
|
||||
@ -66,16 +64,11 @@ Int64 getTimestamp()
|
||||
class FunctionSnowflakeID : public IFunction
|
||||
{
|
||||
private:
|
||||
mutable std::atomic<Int64> lowest_available_snowflake_id{0};
|
||||
// 1 atomic value because we don't want to use mutex
|
||||
mutable std::atomic<Int64> lowest_available_snowflake_id = 0; /// atomic to avoid a mutex
|
||||
|
||||
public:
|
||||
static constexpr auto name = "generateSnowflakeID";
|
||||
|
||||
static FunctionPtr create(ContextPtr /*context*/)
|
||||
{
|
||||
return std::make_shared<FunctionSnowflakeID>();
|
||||
}
|
||||
static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared<FunctionSnowflakeID>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
@ -95,31 +88,34 @@ public:
|
||||
return std::make_shared<DataTypeInt64>();
|
||||
}
|
||||
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & /*arguments*/, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
auto col_res = ColumnVector<Int64>::create();
|
||||
typename ColumnVector<Int64>::Container & vec_to = col_res->getData();
|
||||
Int64 size64 = static_cast<Int64>(input_rows_count);
|
||||
|
||||
vec_to.resize(input_rows_count);
|
||||
|
||||
if (input_rows_count == 0) {
|
||||
return col_res;
|
||||
}
|
||||
|
||||
Int64 machine_id = getMachineID();
|
||||
const Int64 machine_id = getMachineID();
|
||||
Int64 current_timestamp = getTimestamp();
|
||||
Int64 current_machine_seq_num;
|
||||
|
||||
Int64 available_id, next_available_id;
|
||||
Int64 available_snowflake_id, next_available_snowflake_id;
|
||||
|
||||
const Int64 size64 = static_cast<Int64>(input_rows_count);
|
||||
|
||||
do
|
||||
{
|
||||
available_id = lowest_available_snowflake_id.load();
|
||||
Int64 available_timestamp = (available_id & timestamp_mask) >> (machine_id_size + machine_seq_num_size);
|
||||
Int64 available_machine_seq_num = available_id & machine_seq_num_mask;
|
||||
available_snowflake_id = lowest_available_snowflake_id.load();
|
||||
const Int64 available_timestamp = (available_snowflake_id & timestamp_mask) >> (machine_id_bits_count + machine_seq_num_bits_count);
|
||||
const Int64 available_machine_seq_num = available_snowflake_id & machine_seq_num_mask;
|
||||
|
||||
if (current_timestamp > available_timestamp)
|
||||
{
|
||||
/// handle overflow
|
||||
current_machine_seq_num = 0;
|
||||
}
|
||||
else
|
||||
@ -128,24 +124,23 @@ public:
|
||||
current_machine_seq_num = available_machine_seq_num;
|
||||
}
|
||||
|
||||
// calculate new `lowest_available_snowflake_id`
|
||||
/// calculate new lowest_available_snowflake_id
|
||||
const Int64 seq_nums_in_current_timestamp_left = (max_machine_seq_num - current_machine_seq_num + 1);
|
||||
Int64 new_timestamp;
|
||||
Int64 seq_nums_in_current_timestamp_left = (max_machine_seq_num - current_machine_seq_num + 1);
|
||||
if (size64 >= seq_nums_in_current_timestamp_left) {
|
||||
if (size64 >= seq_nums_in_current_timestamp_left)
|
||||
new_timestamp = current_timestamp + 1 + (size64 - seq_nums_in_current_timestamp_left) / max_machine_seq_num;
|
||||
} else {
|
||||
else
|
||||
new_timestamp = current_timestamp;
|
||||
}
|
||||
Int64 new_machine_seq_num = (current_machine_seq_num + size64) & machine_seq_num_mask;
|
||||
next_available_id = (new_timestamp << (machine_id_size + machine_seq_num_size)) | machine_id | new_machine_seq_num;
|
||||
const Int64 new_machine_seq_num = (current_machine_seq_num + size64) & machine_seq_num_mask;
|
||||
next_available_snowflake_id = (new_timestamp << (machine_id_bits_count + machine_seq_num_bits_count)) | machine_id | new_machine_seq_num;
|
||||
}
|
||||
while (!lowest_available_snowflake_id.compare_exchange_strong(available_id, next_available_id));
|
||||
// failed CAS => another thread updated `lowest_available_snowflake_id`
|
||||
// successful CAS => we have our range of exclusive values
|
||||
while (!lowest_available_snowflake_id.compare_exchange_strong(available_snowflake_id, next_available_snowflake_id));
|
||||
/// failed CAS => another thread updated `lowest_available_snowflake_id`
|
||||
/// successful CAS => we have our range of exclusive values
|
||||
|
||||
for (Int64 & el : vec_to)
|
||||
for (Int64 & to_row : vec_to)
|
||||
{
|
||||
el = (current_timestamp << (machine_id_size + machine_seq_num_size)) | machine_id | current_machine_seq_num;
|
||||
to_row = (current_timestamp << (machine_id_bits_count + machine_seq_num_bits_count)) | machine_id | current_machine_seq_num;
|
||||
if (current_machine_seq_num++ == max_machine_seq_num)
|
||||
{
|
||||
current_machine_seq_num = 0;
|
||||
@ -163,10 +158,10 @@ REGISTER_FUNCTION(GenerateSnowflakeID)
|
||||
factory.registerFunction<FunctionSnowflakeID>(FunctionDocumentation
|
||||
{
|
||||
.description=R"(
|
||||
Generates Snowflake ID -- unique identificators contains:
|
||||
- The first 41 (+ 1 top zero bit) bits is timestamp in Unix time milliseconds
|
||||
- The middle 10 bits are the machine ID.
|
||||
- The last 12 bits decode to number of ids processed by the machine at the given millisecond.
|
||||
Generates a SnowflakeID -- unique identificators contains:
|
||||
- The first 41 (+ 1 top zero bit) bits is the timestamp (millisecond since Unix epoch 1 Jan 1970)
|
||||
- The middle 10 bits are the machine ID
|
||||
- The last 12 bits are a counter to disambiguate multiple snowflakeIDs generated within the same millisecond by differen processes
|
||||
|
||||
In case the number of ids processed overflows, the timestamp field is incremented by 1 and the counter is reset to 0.
|
||||
This function guarantees strict monotony on 1 machine and differences in values obtained on different machines.
|
||||
|
Loading…
Reference in New Issue
Block a user