mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Remove generateSnowflakeIDThreadMonotonic
This commit is contained in:
parent
5a35010a83
commit
70302fe309
@ -746,71 +746,6 @@ SELECT generateSnowflakeID(1), generateSnowflakeID(2);
|
||||
└────────────────────────┴────────────────────────┘
|
||||
```
|
||||
|
||||
## generateSnowflakeIDThreadMonotonic
|
||||
|
||||
Generates a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID).
|
||||
|
||||
The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond.
|
||||
For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes.
|
||||
In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0.
|
||||
|
||||
This function behaves like `generateSnowflakeID` but gives no guarantee on counter monotony across different simultaneous requests.
|
||||
Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate Snowflake IDs.
|
||||
|
||||
```
|
||||
0 1 2 3
|
||||
0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1
|
||||
├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤
|
||||
|0| timestamp |
|
||||
├─┼ ┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤
|
||||
| | machine_id | machine_seq_num |
|
||||
└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘
|
||||
```
|
||||
|
||||
**Syntax**
|
||||
|
||||
``` sql
|
||||
generateSnowflakeIDThreadMonotonic([expr])
|
||||
```
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `expr` — An arbitrary [expression](../../sql-reference/syntax.md#syntax-expressions) used to bypass [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in a query. The value of the expression has no effect on the returned Snowflake ID. Optional.
|
||||
|
||||
**Returned value**
|
||||
|
||||
A value of type UInt64.
|
||||
|
||||
**Example**
|
||||
|
||||
First, create a table with a column of type UInt64, then insert a generated Snowflake ID into the table.
|
||||
|
||||
``` sql
|
||||
CREATE TABLE tab (id UInt64) ENGINE = Memory;
|
||||
|
||||
INSERT INTO tab SELECT generateSnowflakeIDThreadMonotonic();
|
||||
|
||||
SELECT * FROM tab;
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```response
|
||||
┌──────────────────id─┐
|
||||
│ 7199082832006627328 │
|
||||
└─────────────────────┘
|
||||
```
|
||||
|
||||
**Example with multiple Snowflake IDs generated per row**
|
||||
|
||||
```sql
|
||||
SELECT generateSnowflakeIDThreadMonotonic(1), generateSnowflakeIDThreadMonotonic(2);
|
||||
|
||||
┌─generateSnowflakeIDThreadMonotonic(1)─┬─generateSnowflakeIDThreadMonotonic(2)─┐
|
||||
│ 7199082940311945216 │ 7199082940316139520 │
|
||||
└───────────────────────────────────────┴───────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## snowflakeToDateTime
|
||||
|
||||
Extracts the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) in [DateTime](../data-types/datetime.md) format.
|
||||
|
@ -123,61 +123,37 @@ SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, size_t in
|
||||
return {begin, end};
|
||||
}
|
||||
|
||||
struct GlobalCounterPolicy
|
||||
struct Data
|
||||
{
|
||||
static constexpr auto name = "generateSnowflakeID";
|
||||
static constexpr auto description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function generateSnowflakeID guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)";
|
||||
|
||||
/// Guarantee counter monotonicity within one timestamp across all threads generating Snowflake IDs simultaneously.
|
||||
struct Data
|
||||
static inline std::atomic<uint64_t> lowest_available_snowflake_id = 0;
|
||||
|
||||
SnowflakeId reserveRange(size_t input_rows_count)
|
||||
{
|
||||
static inline std::atomic<uint64_t> lowest_available_snowflake_id = 0;
|
||||
|
||||
SnowflakeId reserveRange(size_t input_rows_count)
|
||||
uint64_t available_snowflake_id = lowest_available_snowflake_id.load();
|
||||
SnowflakeIdRange range;
|
||||
do
|
||||
{
|
||||
uint64_t available_snowflake_id = lowest_available_snowflake_id.load();
|
||||
SnowflakeIdRange range;
|
||||
do
|
||||
{
|
||||
range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), input_rows_count);
|
||||
}
|
||||
while (!lowest_available_snowflake_id.compare_exchange_weak(available_snowflake_id, fromSnowflakeId(range.end)));
|
||||
/// if CAS failed --> another thread updated `lowest_available_snowflake_id` and we re-try
|
||||
/// else --> our thread reserved ID range [begin, end) and return the beginning of the range
|
||||
|
||||
return range.begin;
|
||||
range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), input_rows_count);
|
||||
}
|
||||
};
|
||||
};
|
||||
while (!lowest_available_snowflake_id.compare_exchange_weak(available_snowflake_id, fromSnowflakeId(range.end)));
|
||||
/// CAS failed --> another thread updated `lowest_available_snowflake_id` and we re-try
|
||||
/// else --> our thread reserved ID range [begin, end) and return the beginning of the range
|
||||
|
||||
struct ThreadLocalCounterPolicy
|
||||
{
|
||||
static constexpr auto name = "generateSnowflakeIDThreadMonotonic";
|
||||
static constexpr auto description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. This function behaves like generateSnowflakeID but gives no guarantee on counter monotony across different simultaneous requests. Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate Snowflake IDs.)";
|
||||
|
||||
/// Guarantee counter monotonicity within one timestamp within the same thread. Faster than GlobalCounterPolicy if a query uses multiple threads.
|
||||
struct Data
|
||||
{
|
||||
static inline thread_local uint64_t lowest_available_snowflake_id = 0;
|
||||
|
||||
SnowflakeId reserveRange(size_t input_rows_count)
|
||||
{
|
||||
SnowflakeIdRange range = getRangeOfAvailableIds(toSnowflakeId(lowest_available_snowflake_id), input_rows_count);
|
||||
lowest_available_snowflake_id = fromSnowflakeId(range.end);
|
||||
return range.begin;
|
||||
}
|
||||
};
|
||||
return range.begin;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
template <typename FillPolicy>
|
||||
class FunctionGenerateSnowflakeID : public IFunction, public FillPolicy
|
||||
class FunctionGenerateSnowflakeID : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "generateSnowflakeID";
|
||||
|
||||
static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared<FunctionGenerateSnowflakeID>(); }
|
||||
|
||||
String getName() const override { return FillPolicy::name; }
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
bool isDeterministic() const override { return false; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return false; }
|
||||
@ -205,7 +181,7 @@ public:
|
||||
{
|
||||
vec_to.resize(input_rows_count);
|
||||
|
||||
typename FillPolicy::Data data;
|
||||
Data data;
|
||||
SnowflakeId snowflake_id = data.reserveRange(input_rows_count); /// returns begin of available snowflake ids range
|
||||
|
||||
for (UInt64 & to_row : vec_to)
|
||||
@ -229,27 +205,16 @@ public:
|
||||
|
||||
};
|
||||
|
||||
template<typename FillPolicy>
|
||||
void registerSnowflakeIDGenerator(auto & factory)
|
||||
{
|
||||
static constexpr auto doc_syntax_format = "{}([expression])";
|
||||
static constexpr auto example_format = "SELECT {}()";
|
||||
static constexpr auto multiple_example_format = "SELECT {f}(1), {f}(2)";
|
||||
|
||||
FunctionDocumentation::Description description = FillPolicy::description;
|
||||
FunctionDocumentation::Syntax syntax = fmt::format(doc_syntax_format, FillPolicy::name);
|
||||
FunctionDocumentation::Arguments arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}};
|
||||
FunctionDocumentation::ReturnedValue returned_value = "A value of type UInt64";
|
||||
FunctionDocumentation::Examples examples = {{"single", fmt::format(example_format, FillPolicy::name), ""}, {"multiple", fmt::format(multiple_example_format, fmt::arg("f", FillPolicy::name)), ""}};
|
||||
FunctionDocumentation::Categories categories = {"Snowflake ID"};
|
||||
|
||||
factory.template registerFunction<FunctionGenerateSnowflakeID<FillPolicy>>({description, syntax, arguments, returned_value, examples, categories}, FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
REGISTER_FUNCTION(GenerateSnowflakeID)
|
||||
{
|
||||
registerSnowflakeIDGenerator<GlobalCounterPolicy>(factory);
|
||||
registerSnowflakeIDGenerator<ThreadLocalCounterPolicy>(factory);
|
||||
FunctionDocumentation::Description description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function generateSnowflakeID guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)";
|
||||
FunctionDocumentation::Syntax syntax = "generateSnowflakeID([expression])";
|
||||
FunctionDocumentation::Arguments arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}};
|
||||
FunctionDocumentation::ReturnedValue returned_value = "A value of type UInt64";
|
||||
FunctionDocumentation::Examples examples = {{"single", "SELECT generateSnowflakeID()", "7201148511606784000"}, {"multiple", "SELECT generateSnowflakeID(1), generateSnowflakeID(2)", ""}};
|
||||
FunctionDocumentation::Categories categories = {"Snowflake ID"};
|
||||
|
||||
factory.registerFunction<FunctionGenerateSnowflakeID>({description, syntax, arguments, returned_value, examples, categories});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,9 +1,5 @@
|
||||
-- generateSnowflakeID
|
||||
1
|
||||
0
|
||||
0
|
||||
1
|
||||
100
|
||||
-- generateSnowflakeIDThreadMonotonic
|
||||
1
|
||||
100
|
||||
|
@ -1,4 +1,4 @@
|
||||
SELECT '-- generateSnowflakeID';
|
||||
-- Test SQL function 'generateSnowflakeID'
|
||||
|
||||
SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeID()), 63), 1) = 0; -- check first bit is zero
|
||||
|
||||
@ -14,16 +14,3 @@ FROM
|
||||
SELECT DISTINCT generateSnowflakeID()
|
||||
FROM numbers(100)
|
||||
);
|
||||
|
||||
SELECT '-- generateSnowflakeIDThreadMonotonic';
|
||||
|
||||
SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeIDThreadMonotonic()), 63), 1) = 0; -- check first bit is zero
|
||||
|
||||
SELECT generateSnowflakeIDThreadMonotonic(1, 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
|
||||
|
||||
SELECT count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT DISTINCT generateSnowflakeIDThreadMonotonic()
|
||||
FROM numbers(100)
|
||||
);
|
||||
|
@ -1619,7 +1619,6 @@ generateRandom
|
||||
generateRandomStructure
|
||||
generateSeries
|
||||
generateSnowflakeID
|
||||
generateSnowflakeIDThreadMonotonic
|
||||
generateULID
|
||||
generateUUIDv
|
||||
geoDistance
|
||||
|
Loading…
Reference in New Issue
Block a user