mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-14 03:25:15 +00:00
Merge pull request #44686 from Algunenano/fix_uuid_parsing_in_values
Don't parse beyond the quotes when reading UUIDs
This commit is contained in:
commit
c6ee4c3908
@ -1,9 +1,10 @@
|
||||
#include <DataTypes/Serializations/SerializationUUID.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/Serializations/SerializationUUID.h>
|
||||
#include <Formats/ProtobufReader.h>
|
||||
#include <Formats/ProtobufWriter.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
|
||||
@ -44,11 +45,44 @@ void SerializationUUID::serializeTextQuoted(const IColumn & column, size_t row_n
|
||||
|
||||
void SerializationUUID::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
UUID x;
|
||||
assertChar('\'', istr);
|
||||
readText(x, istr);
|
||||
assertChar('\'', istr);
|
||||
assert_cast<ColumnUUID &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
|
||||
UUID uuid;
|
||||
bool fast = false;
|
||||
if (istr.available() >= 38)
|
||||
{
|
||||
assertChar('\'', istr);
|
||||
char * next_pos = find_first_symbols<'\\', '\''>(istr.position(), istr.buffer().end());
|
||||
size_t len = next_pos - istr.position();
|
||||
if ((len == 32) && (istr.position()[32] == '\''))
|
||||
{
|
||||
parseUUIDWithoutSeparator(
|
||||
reinterpret_cast<const UInt8 *>(istr.position()), std::reverse_iterator<UInt8 *>(reinterpret_cast<UInt8 *>(&uuid) + 16));
|
||||
istr.ignore(33);
|
||||
fast = true;
|
||||
}
|
||||
else if ((len == 36) && (istr.position()[36] == '\''))
|
||||
{
|
||||
parseUUID(
|
||||
reinterpret_cast<const UInt8 *>(istr.position()), std::reverse_iterator<UInt8 *>(reinterpret_cast<UInt8 *>(&uuid) + 16));
|
||||
istr.ignore(37);
|
||||
fast = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
// It's ok to go back in the position because we haven't read from the buffer except the first char
|
||||
// and we know there were at least 38 bytes available (so no new read has been triggered)
|
||||
istr.position()--;
|
||||
}
|
||||
}
|
||||
|
||||
if (!fast)
|
||||
{
|
||||
String quoted_chars;
|
||||
readQuotedStringInto<false>(quoted_chars, istr);
|
||||
ReadBufferFromString parsed_quoted_buffer(quoted_chars);
|
||||
readText(uuid, parsed_quoted_buffer);
|
||||
}
|
||||
|
||||
assert_cast<ColumnUUID &>(column).getData().push_back(std::move(uuid)); /// It's important to do this at the end - for exception safety.
|
||||
}
|
||||
|
||||
void SerializationUUID::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
|
@ -39,8 +39,8 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
template <typename IteratorSrc, typename IteratorDst>
|
||||
void parseHex(IteratorSrc src, IteratorDst dst, const size_t num_bytes)
|
||||
template <size_t num_bytes, typename IteratorSrc, typename IteratorDst>
|
||||
inline void parseHex(IteratorSrc src, IteratorDst dst)
|
||||
{
|
||||
size_t src_pos = 0;
|
||||
size_t dst_pos = 0;
|
||||
@ -52,18 +52,18 @@ void parseUUID(const UInt8 * src36, UInt8 * dst16)
|
||||
{
|
||||
/// If string is not like UUID - implementation specific behaviour.
|
||||
|
||||
parseHex(&src36[0], &dst16[0], 4);
|
||||
parseHex(&src36[9], &dst16[4], 2);
|
||||
parseHex(&src36[14], &dst16[6], 2);
|
||||
parseHex(&src36[19], &dst16[8], 2);
|
||||
parseHex(&src36[24], &dst16[10], 6);
|
||||
parseHex<4>(&src36[0], &dst16[0]);
|
||||
parseHex<2>(&src36[9], &dst16[4]);
|
||||
parseHex<2>(&src36[14], &dst16[6]);
|
||||
parseHex<2>(&src36[19], &dst16[8]);
|
||||
parseHex<6>(&src36[24], &dst16[10]);
|
||||
}
|
||||
|
||||
void parseUUIDWithoutSeparator(const UInt8 * src36, UInt8 * dst16)
|
||||
{
|
||||
/// If string is not like UUID - implementation specific behaviour.
|
||||
|
||||
parseHex(&src36[0], &dst16[0], 16);
|
||||
parseHex<16>(&src36[0], &dst16[0]);
|
||||
}
|
||||
|
||||
/** Function used when byte ordering is important when parsing uuid
|
||||
@ -74,11 +74,11 @@ void parseUUID(const UInt8 * src36, std::reverse_iterator<UInt8 *> dst16)
|
||||
/// If string is not like UUID - implementation specific behaviour.
|
||||
|
||||
/// FIXME This code looks like trash.
|
||||
parseHex(&src36[0], dst16 + 8, 4);
|
||||
parseHex(&src36[9], dst16 + 12, 2);
|
||||
parseHex(&src36[14], dst16 + 14, 2);
|
||||
parseHex(&src36[19], dst16, 2);
|
||||
parseHex(&src36[24], dst16 + 2, 6);
|
||||
parseHex<4>(&src36[0], dst16 + 8);
|
||||
parseHex<2>(&src36[9], dst16 + 12);
|
||||
parseHex<2>(&src36[14], dst16 + 14);
|
||||
parseHex<2>(&src36[19], dst16);
|
||||
parseHex<6>(&src36[24], dst16 + 2);
|
||||
}
|
||||
|
||||
/** Function used when byte ordering is important when parsing uuid
|
||||
@ -88,8 +88,8 @@ void parseUUIDWithoutSeparator(const UInt8 * src36, std::reverse_iterator<UInt8
|
||||
{
|
||||
/// If string is not like UUID - implementation specific behaviour.
|
||||
|
||||
parseHex(&src36[0], dst16 + 8, 8);
|
||||
parseHex(&src36[16], dst16, 8);
|
||||
parseHex<8>(&src36[0], dst16 + 8);
|
||||
parseHex<8>(&src36[16], dst16);
|
||||
}
|
||||
|
||||
void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf)
|
||||
|
2
tests/queries/0_stateless/02517_uuid_parsing.reference
Normal file
2
tests/queries/0_stateless/02517_uuid_parsing.reference
Normal file
@ -0,0 +1,2 @@
|
||||
1011 1d83904a-c31d-4a6c-bbf0-217656b46444 1d83904a-c31d-4a6c-bbf0-217656b46444 -200 0 FOO BAR 1d83904a-c31d-4a6c-bbf0-217656b46444 2022-12-18 03:14:56 \N dispatcher
|
||||
10112222334444 1d83904a-c31d-4a6c-bbf0-217656b46444 1d83904a-c31d-4a6c-bbf0-217656b46444 12300 0 FOO BAR 1d83904a-c31d-4a6c-bbf0-217656b46444 2022-12-17 23:37:18 1d83904a-c31d-4a6c-bbf0-217656b46444 other
|
26
tests/queries/0_stateless/02517_uuid_parsing.sql
Normal file
26
tests/queries/0_stateless/02517_uuid_parsing.sql
Normal file
@ -0,0 +1,26 @@
|
||||
-- https://github.com/ClickHouse/ClickHouse/issues/44668
|
||||
|
||||
CREATE TABLE temp
|
||||
(
|
||||
`id` UInt64,
|
||||
`field1` UUID,
|
||||
`field2` UUID,
|
||||
`field3` Int64,
|
||||
`field4` Int64,
|
||||
`field5` LowCardinality(String),
|
||||
`field6` FixedString(3),
|
||||
`field7` String,
|
||||
`field8` Nullable(UUID),
|
||||
`event_at` DateTime('UTC'),
|
||||
`order_id` Nullable(UUID),
|
||||
`identity` LowCardinality(String)
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
PARTITION BY toYYYYMM(event_at)
|
||||
ORDER BY (field1, event_at, field2, field5, id)
|
||||
SETTINGS index_granularity = 8192;
|
||||
|
||||
INSERT INTO temp (id, field1, field2, field3, field4, field5, field6, field7, field8, event_at, order_id, identity)
|
||||
VALUES ('1011','1d83904a-c31d-4a6c-bbf0-217656b46444','1d83904a-c31d-4a6c-bbf0-217656b46444',-200,0,'FOO','BAR','','1d83904a-c31d-4a6c-bbf0-217656b46444','2022-12-18 03:14:56','','dispatcher'),('10112222334444','1d83904a-c31d-4a6c-bbf0-217656b46444','1d83904a-c31d-4a6c-bbf0-217656b46444',12300,0,'FOO','BAR','','1d83904a-c31d-4a6c-bbf0-217656b46444','2022-12-17 23:37:18','1d83904a-c31d-4a6c-bbf0-217656b46444','other');
|
||||
|
||||
SELECT * FROM temp ORDER BY id;
|
Loading…
Reference in New Issue
Block a user