mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
post-review changes
This commit is contained in:
parent
5d6d378f24
commit
0ff90e4a5c
@ -64,8 +64,6 @@ public:
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
bool supportsSparseSerialization() const override { return false; }
|
||||
|
||||
bool isNonTriviallySerializedAsStringJSON() const override { return true; }
|
||||
|
||||
bool isVersioned() const;
|
||||
|
||||
/// Version is not empty only if it was parsed from AST or implicitly cast to 0 or version according
|
||||
|
@ -17,7 +17,6 @@ public:
|
||||
|
||||
bool canBeUsedAsVersion() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
bool isNonTriviallySerializedAsStringJSON() const override { return true; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
|
@ -18,7 +18,6 @@ public:
|
||||
|
||||
bool canBeUsedAsVersion() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
bool isNonTriviallySerializedAsStringJSON() const override { return true; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
|
@ -44,7 +44,6 @@ public:
|
||||
|
||||
bool canBeUsedAsVersion() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
bool isNonTriviallySerializedAsStringJSON() const override { return true; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
|
@ -39,8 +39,6 @@ public:
|
||||
|
||||
bool isSummable() const override { return false; }
|
||||
|
||||
bool isNonTriviallySerializedAsStringJSON() const override { return true; }
|
||||
|
||||
protected:
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
|
@ -46,7 +46,6 @@ public:
|
||||
size_t getSizeOfValueInMemory() const override { return sizeof(IPv4); }
|
||||
bool isCategorial() const override { return true; }
|
||||
bool canBeInsideLowCardinality() const override { return true; }
|
||||
bool isNonTriviallySerializedAsStringJSON() const override { return true; }
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override { return std::make_shared<SerializationIP<IPv4>>(); }
|
||||
};
|
||||
@ -85,7 +84,6 @@ public:
|
||||
size_t getSizeOfValueInMemory() const override { return sizeof(IPv6); }
|
||||
bool isCategorial() const override { return true; }
|
||||
bool canBeInsideLowCardinality() const override { return true; }
|
||||
bool isNonTriviallySerializedAsStringJSON() const override { return true; }
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override { return std::make_shared<SerializationIP<IPv6>>(); }
|
||||
};
|
||||
|
@ -42,7 +42,6 @@ public:
|
||||
size_t getSizeOfValueInMemory() const override { return sizeof(UUID); }
|
||||
bool isCategorial() const override { return true; }
|
||||
bool canBeInsideLowCardinality() const override { return true; }
|
||||
bool isNonTriviallySerializedAsStringJSON() const override { return true; }
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
|
@ -328,9 +328,6 @@ public:
|
||||
/// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column.
|
||||
static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint);
|
||||
|
||||
/// Checks if this is a non-numeric non-string type which is serialized as a JSON string.
|
||||
virtual bool isNonTriviallySerializedAsStringJSON() const { return false; }
|
||||
|
||||
protected:
|
||||
friend class DataTypeFactory;
|
||||
friend class AggregateFunctionSimpleState;
|
||||
|
@ -17,7 +17,6 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
class IDataType;
|
||||
@ -399,20 +398,12 @@ public:
|
||||
virtual void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
virtual void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
virtual bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
|
||||
/// The following two methods are implemented only for non-numeric non-string simple data types.
|
||||
virtual void deserializeTextNoEmptyCheckJSON(IColumn & /*column*/, ReadBuffer & /*istr*/, const FormatSettings &) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method deserializeTextNoEmptyCheckJSON is not supported");
|
||||
}
|
||||
virtual bool tryDeserializeTextNoEmptyCheckJSON(IColumn & /*column*/, ReadBuffer & /*istr*/, const FormatSettings &) const
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method tryDeserializeTextNoEmptyCheckJSON is not supported");
|
||||
}
|
||||
virtual void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t /*indent*/) const
|
||||
{
|
||||
serializeTextJSON(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
|
||||
/** Text serialization for putting into the XML format.
|
||||
*/
|
||||
virtual void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
|
@ -182,7 +182,7 @@ void SerializationAggregateFunction::serializeTextJSON(const IColumn & column, s
|
||||
}
|
||||
|
||||
|
||||
void SerializationAggregateFunction::deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
void SerializationAggregateFunction::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
String s;
|
||||
readJSONString(s, istr, settings.json);
|
||||
|
@ -3,13 +3,12 @@
|
||||
#include <AggregateFunctions/IAggregateFunction_fwd.h>
|
||||
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
#include <DataTypes/Serializations/SerializationAsStringNonTrivialJSON.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class SerializationAggregateFunction final : public SerializationAsStringNonTrivialJSON<ISerialization>
|
||||
class SerializationAggregateFunction final : public ISerialization
|
||||
{
|
||||
private:
|
||||
AggregateFunctionPtr function;
|
||||
@ -38,7 +37,7 @@ public:
|
||||
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
@ -614,29 +615,124 @@ void SerializationArray::serializeTextJSONPretty(const IColumn & column, size_t
|
||||
writeChar(']', ostr);
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
template <typename ReturnType>
|
||||
ReturnType deserializeTextJSONArrayImpl(IColumn & column, ReadBuffer & istr, const SerializationPtr & nested, const FormatSettings & settings)
|
||||
{
|
||||
static constexpr auto throw_exception = std::is_same_v<ReturnType, void>;
|
||||
|
||||
static constexpr auto EMPTY_STRING = "\"\"";
|
||||
static constexpr auto EMPTY_STRING_LENGTH = std::string_view(EMPTY_STRING).length();
|
||||
|
||||
auto do_deserialize_nested = [](IColumn & nested_column, ReadBuffer & buf, auto && check_for_empty_string, auto && deserialize) -> ReturnType
|
||||
{
|
||||
if (check_for_empty_string(buf))
|
||||
{
|
||||
nested_column.insertDefault();
|
||||
return ReturnType(true);
|
||||
}
|
||||
return deserialize(nested_column, buf);
|
||||
};
|
||||
|
||||
auto deserialize_nested_impl = [&settings, &nested](IColumn & nested_column, ReadBuffer & buf) -> ReturnType
|
||||
{
|
||||
if constexpr (throw_exception)
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column))
|
||||
SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(nested_column, buf, settings, nested);
|
||||
else
|
||||
nested->deserializeTextJSON(nested_column, buf, settings);
|
||||
return;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column))
|
||||
return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(nested_column, buf, settings, nested);
|
||||
else
|
||||
return nested->tryDeserializeTextJSON(nested_column, buf, settings);
|
||||
}
|
||||
};
|
||||
|
||||
auto deserialize_nested = [&settings, &do_deserialize_nested, &deserialize_nested_impl](IColumn & nested_column, ReadBuffer & buf) -> ReturnType
|
||||
{
|
||||
if (!settings.json.empty_as_default || buf.eof() || *buf.position() != EMPTY_STRING[0])
|
||||
return deserialize_nested_impl(nested_column, buf);
|
||||
|
||||
if (buf.available() >= EMPTY_STRING_LENGTH)
|
||||
{
|
||||
/// We have enough data in buffer to check if we have an empty string.
|
||||
auto check_for_empty_string = [](ReadBuffer & buf_) -> bool
|
||||
{
|
||||
auto * pos = buf_.position();
|
||||
if (checkString(EMPTY_STRING, buf_))
|
||||
return true;
|
||||
else
|
||||
{
|
||||
buf_.position() = pos;
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
return do_deserialize_nested(nested_column, buf, check_for_empty_string, deserialize_nested_impl);
|
||||
}
|
||||
|
||||
/// We don't have enough data in buffer to check if we have an empty string.
|
||||
/// Use PeekableReadBuffer to make a checkpoint before checking for an
|
||||
/// empty string and rollback if check was failed.
|
||||
|
||||
auto check_for_empty_string = [](ReadBuffer & buf_) -> bool
|
||||
{
|
||||
auto & peekable_buf = assert_cast<PeekableReadBuffer &>(buf_);
|
||||
peekable_buf.setCheckpoint();
|
||||
SCOPE_EXIT(peekable_buf.dropCheckpoint());
|
||||
if (checkString(EMPTY_STRING, peekable_buf))
|
||||
return true;
|
||||
else
|
||||
{
|
||||
peekable_buf.rollbackToCheckpoint();
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
auto deserialize_nested_impl_with_check = [&deserialize_nested_impl](IColumn & nested_column_, ReadBuffer & buf_) -> ReturnType
|
||||
{
|
||||
auto & peekable_buf = assert_cast<PeekableReadBuffer &>(buf_);
|
||||
if constexpr (throw_exception)
|
||||
{
|
||||
deserialize_nested_impl(nested_column_, peekable_buf);
|
||||
assert(!peekable_buf.hasUnreadData());
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!deserialize_nested_impl(nested_column_, peekable_buf))
|
||||
return false;
|
||||
if (likely(!peekable_buf.hasUnreadData()))
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
PeekableReadBuffer peekable_buf(buf, true);
|
||||
return do_deserialize_nested(nested_column, peekable_buf, check_for_empty_string, deserialize_nested_impl_with_check);
|
||||
};
|
||||
|
||||
return deserializeTextImpl<ReturnType>(column, istr,
|
||||
[&deserialize_nested, &istr](IColumn & nested_column) -> ReturnType
|
||||
{
|
||||
return deserialize_nested(nested_column, istr);
|
||||
}, false);
|
||||
}
|
||||
}
|
||||
|
||||
void SerializationArray::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeTextImpl(column, istr,
|
||||
[&](IColumn & nested_column)
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column))
|
||||
SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(nested_column, istr, settings, nested);
|
||||
else
|
||||
nested->deserializeTextJSON(nested_column, istr, settings);
|
||||
}, false);
|
||||
deserializeTextJSONArrayImpl<void>(column, istr, nested, settings);
|
||||
}
|
||||
|
||||
bool SerializationArray::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
auto read_nested = [&](IColumn & nested_column)
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column))
|
||||
return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(nested_column, istr, settings, nested);
|
||||
return nested->tryDeserializeTextJSON(nested_column, istr, settings);
|
||||
};
|
||||
|
||||
return deserializeTextImpl<bool>(column, istr, std::move(read_nested), false);
|
||||
return deserializeTextJSONArrayImpl<bool>(column, istr, nested, settings);
|
||||
}
|
||||
|
||||
|
||||
|
@ -1,61 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types_fwd.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
#include <concepts>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
/** Serialization for non-numeric non-string data types serialized as JSON strings
|
||||
* For these data types, we support an option, input_format_json_empty_as_default, which, when set to 1,
|
||||
* allows for JSON deserialization to treat an encountered empty string as a default value for the specified type.
|
||||
* Derived classes must implement the following methods:
|
||||
* deserializeTextNoEmptyCheckJSON() and tryDeserializeTextNoEmptyCheckJSON()
|
||||
* instead of deserializeTextJSON() and tryDeserializeTextJSON() respectively.
|
||||
*/
|
||||
template <typename T>
|
||||
requires std::derived_from<T, ISerialization>
|
||||
class SerializationAsStringNonTrivialJSON : public T
|
||||
{
|
||||
public:
|
||||
using T::T;
|
||||
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & format_settings) const override
|
||||
{
|
||||
if (format_settings.json.empty_as_default && tryMatchEmptyString(istr))
|
||||
column.insertDefault();
|
||||
else
|
||||
deserializeTextNoEmptyCheckJSON(column, istr, format_settings);
|
||||
}
|
||||
|
||||
bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & format_settings) const override
|
||||
{
|
||||
if (format_settings.json.empty_as_default && tryMatchEmptyString(istr))
|
||||
{
|
||||
column.insertDefault();
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return tryDeserializeTextNoEmptyCheckJSON(column, istr, format_settings);
|
||||
}
|
||||
|
||||
void deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override = 0;
|
||||
|
||||
bool tryDeserializeTextNoEmptyCheckJSON(IColumn & /*column*/, ReadBuffer & /*istr*/, const FormatSettings &) const override
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method tryDeserializeTextNoEmptyCheckJSON is not supported");
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -85,7 +85,7 @@ void SerializationDate::serializeTextJSON(const IColumn & column, size_t row_num
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
|
||||
void SerializationDate::deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
void SerializationDate::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
DayNum x;
|
||||
assertChar('"', istr);
|
||||
@ -94,7 +94,7 @@ void SerializationDate::deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBu
|
||||
assert_cast<ColumnUInt16 &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
bool SerializationDate::tryDeserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
bool SerializationDate::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
DayNum x;
|
||||
if (!checkChar('"', istr) || !tryReadDateText(x, istr, time_zone) || !checkChar('"', istr))
|
||||
|
@ -1,13 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SerializationNumber.h>
|
||||
#include <DataTypes/Serializations/SerializationAsStringNonTrivialJSON.h>
|
||||
#include <Common/DateLUT.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class SerializationDate final : public SerializationAsStringNonTrivialJSON<SerializationNumber<UInt16>>
|
||||
class SerializationDate final : public SerializationNumber<UInt16>
|
||||
{
|
||||
public:
|
||||
explicit SerializationDate(const DateLUTImpl & time_zone_ = DateLUT::instance());
|
||||
@ -22,8 +21,8 @@ public:
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
@ -83,7 +83,7 @@ void SerializationDate32::serializeTextJSON(const IColumn & column, size_t row_n
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
|
||||
void SerializationDate32::deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
void SerializationDate32::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
ExtendedDayNum x;
|
||||
assertChar('"', istr);
|
||||
@ -92,7 +92,7 @@ void SerializationDate32::deserializeTextNoEmptyCheckJSON(IColumn & column, Read
|
||||
assert_cast<ColumnInt32 &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
bool SerializationDate32::tryDeserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
bool SerializationDate32::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
ExtendedDayNum x;
|
||||
if (!checkChar('"', istr) || !tryReadDateText(x, istr, time_zone) || !checkChar('"', istr))
|
||||
|
@ -1,12 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SerializationNumber.h>
|
||||
#include <DataTypes/Serializations/SerializationAsStringNonTrivialJSON.h>
|
||||
#include <Common/DateLUT.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class SerializationDate32 final : public SerializationAsStringNonTrivialJSON<SerializationNumber<Int32>>
|
||||
class SerializationDate32 final : public SerializationNumber<Int32>
|
||||
{
|
||||
public:
|
||||
explicit SerializationDate32(const DateLUTImpl & time_zone_ = DateLUT::instance());
|
||||
@ -21,8 +20,8 @@ public:
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
@ -180,7 +180,7 @@ void SerializationDateTime::serializeTextJSON(const IColumn & column, size_t row
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
|
||||
void SerializationDateTime::deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
void SerializationDateTime::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
time_t x = 0;
|
||||
if (checkChar('"', istr))
|
||||
@ -196,7 +196,7 @@ void SerializationDateTime::deserializeTextNoEmptyCheckJSON(IColumn & column, Re
|
||||
assert_cast<ColumnType &>(column).getData().push_back(static_cast<UInt32>(x));
|
||||
}
|
||||
|
||||
bool SerializationDateTime::tryDeserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
bool SerializationDateTime::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
time_t x = 0;
|
||||
if (checkChar('"', istr))
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SerializationNumber.h>
|
||||
#include <DataTypes/Serializations/SerializationAsStringNonTrivialJSON.h>
|
||||
#include <DataTypes/TimezoneMixin.h>
|
||||
|
||||
class DateLUTImpl;
|
||||
@ -9,7 +8,7 @@ class DateLUTImpl;
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class SerializationDateTime final : public SerializationAsStringNonTrivialJSON<SerializationNumber<UInt32>>, public TimezoneMixin
|
||||
class SerializationDateTime final : public SerializationNumber<UInt32>, public TimezoneMixin
|
||||
{
|
||||
public:
|
||||
explicit SerializationDateTime(const TimezoneMixin & time_zone_);
|
||||
@ -24,8 +23,8 @@ public:
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
@ -15,7 +15,7 @@ namespace DB
|
||||
|
||||
SerializationDateTime64::SerializationDateTime64(
|
||||
UInt32 scale_, const TimezoneMixin & time_zone_)
|
||||
: SerializationAsStringNonTrivialJSON<SerializationDecimalBase<DateTime64>>(DecimalUtils::max_precision<DateTime64>, scale_)
|
||||
: SerializationDecimalBase<DateTime64>(DecimalUtils::max_precision<DateTime64>, scale_)
|
||||
, TimezoneMixin(time_zone_)
|
||||
{
|
||||
}
|
||||
@ -170,7 +170,7 @@ void SerializationDateTime64::serializeTextJSON(const IColumn & column, size_t r
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
|
||||
void SerializationDateTime64::deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
void SerializationDateTime64::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
DateTime64 x = 0;
|
||||
if (checkChar('"', istr))
|
||||
@ -185,7 +185,7 @@ void SerializationDateTime64::deserializeTextNoEmptyCheckJSON(IColumn & column,
|
||||
assert_cast<ColumnType &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
bool SerializationDateTime64::tryDeserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
bool SerializationDateTime64::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
DateTime64 x = 0;
|
||||
if (checkChar('"', istr))
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SerializationDecimalBase.h>
|
||||
#include <DataTypes/Serializations/SerializationAsStringNonTrivialJSON.h>
|
||||
#include <DataTypes/TimezoneMixin.h>
|
||||
|
||||
class DateLUTImpl;
|
||||
@ -9,7 +8,7 @@ class DateLUTImpl;
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class SerializationDateTime64 final : public SerializationAsStringNonTrivialJSON<SerializationDecimalBase<DateTime64>>, public TimezoneMixin
|
||||
class SerializationDateTime64 final : public SerializationDecimalBase<DateTime64>, public TimezoneMixin
|
||||
{
|
||||
public:
|
||||
SerializationDateTime64(UInt32 scale_, const TimezoneMixin & time_zone_);
|
||||
@ -26,8 +25,8 @@ public:
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
@ -69,7 +69,7 @@ void SerializationIP<IPv>::serializeTextJSON(const DB::IColumn & column, size_t
|
||||
}
|
||||
|
||||
template <typename IPv>
|
||||
void SerializationIP<IPv>::deserializeTextNoEmptyCheckJSON(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const
|
||||
void SerializationIP<IPv>::deserializeTextJSON(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const
|
||||
{
|
||||
IPv x;
|
||||
assertChar('"', istr);
|
||||
@ -84,7 +84,7 @@ void SerializationIP<IPv>::deserializeTextNoEmptyCheckJSON(DB::IColumn & column,
|
||||
}
|
||||
|
||||
template <typename IPv>
|
||||
bool SerializationIP<IPv>::tryDeserializeTextNoEmptyCheckJSON(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings &) const
|
||||
bool SerializationIP<IPv>::tryDeserializeTextJSON(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings &) const
|
||||
{
|
||||
IPv x;
|
||||
if (!checkChar('"', istr) || !tryReadText(x, istr) || !checkChar('"', istr))
|
||||
|
@ -4,14 +4,13 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/Serializations/SerializationAsStringNonTrivialJSON.h>
|
||||
#include <DataTypes/Serializations/SimpleTextSerialization.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename IPv>
|
||||
class SerializationIP : public SerializationAsStringNonTrivialJSON<SimpleTextSerialization>
|
||||
class SerializationIP : public SimpleTextSerialization
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
@ -23,8 +22,8 @@ public:
|
||||
bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
void deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
bool tryDeserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &/* settings*/) const override;
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
@ -315,29 +316,116 @@ void SerializationMap::serializeTextJSONPretty(const IColumn & column, size_t ro
|
||||
writeChar('}', ostr);
|
||||
}
|
||||
|
||||
|
||||
void SerializationMap::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
template <typename ReturnType>
|
||||
ReturnType SerializationMap::deserializeTextJSONMapImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeTextImpl(column, istr,
|
||||
[&settings](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn)
|
||||
static constexpr auto throw_exception = std::is_same_v<ReturnType, void>;
|
||||
|
||||
static constexpr auto EMPTY_STRING = "\"\"";
|
||||
static constexpr auto EMPTY_STRING_LENGTH = std::string_view(EMPTY_STRING).length();
|
||||
|
||||
auto do_deserialize_subcolumn = [](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn, auto && check_for_empty_string, auto && deserialize) -> ReturnType
|
||||
{
|
||||
if (check_for_empty_string(buf))
|
||||
{
|
||||
subcolumn.insertDefault();
|
||||
return ReturnType(true);
|
||||
}
|
||||
return deserialize(buf, subcolumn_serialization, subcolumn);
|
||||
};
|
||||
|
||||
auto deserialize_subcolumn_impl = [&settings](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn) -> ReturnType
|
||||
{
|
||||
if constexpr (throw_exception)
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(subcolumn))
|
||||
SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(subcolumn, buf, settings, subcolumn_serialization);
|
||||
else
|
||||
subcolumn_serialization->deserializeTextJSON(subcolumn, buf, settings);
|
||||
});
|
||||
}
|
||||
else
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(subcolumn))
|
||||
return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(subcolumn, buf, settings, subcolumn_serialization);
|
||||
else
|
||||
return subcolumn_serialization->tryDeserializeTextJSON(subcolumn, buf, settings);
|
||||
}
|
||||
};
|
||||
|
||||
auto deserialize_subcolumn = [&settings, &do_deserialize_subcolumn, &deserialize_subcolumn_impl](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn) -> ReturnType
|
||||
{
|
||||
if (!settings.json.empty_as_default || buf.eof() || *buf.position() != EMPTY_STRING[0])
|
||||
return do_deserialize_subcolumn(buf, subcolumn_serialization, subcolumn, [](ReadBuffer &){ return false; }, deserialize_subcolumn_impl);
|
||||
|
||||
if (buf.available() >= EMPTY_STRING_LENGTH)
|
||||
{
|
||||
/// We have enough data in buffer to check if we have an empty string.
|
||||
auto check_for_empty_string = [](ReadBuffer & buf_) -> bool
|
||||
{
|
||||
auto * pos = buf_.position();
|
||||
if (checkString(EMPTY_STRING, buf_))
|
||||
return true;
|
||||
else
|
||||
{
|
||||
buf_.position() = pos;
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
return do_deserialize_subcolumn(buf, subcolumn_serialization, subcolumn, check_for_empty_string, deserialize_subcolumn_impl);
|
||||
}
|
||||
|
||||
/// We don't have enough data in buffer to check if we have an empty string.
|
||||
/// Use PeekableReadBuffer to make a checkpoint before checking for an
|
||||
/// empty string and rollback if check was failed.
|
||||
|
||||
auto check_for_empty_string = [](ReadBuffer & buf_) -> bool
|
||||
{
|
||||
auto & peekable_buf = assert_cast<PeekableReadBuffer &>(buf_);
|
||||
peekable_buf.setCheckpoint();
|
||||
SCOPE_EXIT(peekable_buf.dropCheckpoint());
|
||||
if (checkString(EMPTY_STRING, peekable_buf))
|
||||
return true;
|
||||
else
|
||||
{
|
||||
peekable_buf.rollbackToCheckpoint();
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
auto deserialize_subcolumn_impl_with_check = [&deserialize_subcolumn_impl](ReadBuffer & buf_, const SerializationPtr & subcolumn_serialization_, IColumn & subcolumn_) -> ReturnType
|
||||
{
|
||||
auto & peekable_buf = assert_cast<PeekableReadBuffer &>(buf_);
|
||||
if constexpr (throw_exception)
|
||||
{
|
||||
deserialize_subcolumn_impl(peekable_buf, subcolumn_serialization_, subcolumn_);
|
||||
assert(!peekable_buf.hasUnreadData());
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!deserialize_subcolumn_impl(peekable_buf, subcolumn_serialization_, subcolumn_))
|
||||
return false;
|
||||
if (likely(!peekable_buf.hasUnreadData()))
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
PeekableReadBuffer peekable_buf(buf, true);
|
||||
return do_deserialize_subcolumn(peekable_buf, subcolumn_serialization, subcolumn, check_for_empty_string, deserialize_subcolumn_impl_with_check);
|
||||
};
|
||||
|
||||
return deserializeTextImpl<ReturnType>(column, istr, deserialize_subcolumn);
|
||||
}
|
||||
|
||||
void SerializationMap::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeTextJSONMapImpl<void>(column, istr, settings);
|
||||
}
|
||||
|
||||
bool SerializationMap::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
auto reader = [&settings](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn)
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(subcolumn))
|
||||
return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(subcolumn, buf, settings, subcolumn_serialization);
|
||||
return subcolumn_serialization->tryDeserializeTextJSON(subcolumn, buf, settings);
|
||||
};
|
||||
|
||||
return deserializeTextImpl<bool>(column, istr, reader);
|
||||
return deserializeTextJSONMapImpl<bool>(column, istr, settings);
|
||||
}
|
||||
|
||||
void SerializationMap::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
|
@ -74,6 +74,9 @@ private:
|
||||
|
||||
template <typename ReturnType = void, typename Reader>
|
||||
ReturnType deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && reader) const;
|
||||
|
||||
template <typename ReturnType>
|
||||
ReturnType deserializeTextJSONMapImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -844,52 +844,25 @@ bool SerializationNullable::tryDeserializeNullJSON(DB::ReadBuffer & istr)
|
||||
return checkString("null", istr);
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
enum class Strategy : uint8_t
|
||||
{
|
||||
Deserialize,
|
||||
DeserializeNoEmptyCheck,
|
||||
TryDeserialize
|
||||
};
|
||||
|
||||
template <Strategy> struct ReturnTypeImpl;
|
||||
template <> struct ReturnTypeImpl<Strategy::Deserialize> { using Type = void; };
|
||||
template <> struct ReturnTypeImpl<Strategy::TryDeserialize> { using Type = bool; };
|
||||
template <> struct ReturnTypeImpl<Strategy::DeserializeNoEmptyCheck> { using Type = void; };
|
||||
|
||||
template <Strategy strategy>
|
||||
using ReturnType = typename ReturnTypeImpl<strategy>::Type;
|
||||
|
||||
template <Strategy> struct AlwaysFalse : std::false_type {};
|
||||
|
||||
template <Strategy strategy>
|
||||
ReturnType<strategy> deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested, bool & is_null)
|
||||
template<typename ReturnType>
|
||||
ReturnType deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested, bool & is_null)
|
||||
{
|
||||
auto check_for_null = [](ReadBuffer & buf){ return checkStringByFirstCharacterAndAssertTheRest("null", buf); };
|
||||
auto deserialize_nested = [&nested, &settings](IColumn & nested_column, ReadBuffer & buf) -> ReturnType<strategy>
|
||||
auto deserialize_nested = [&nested, &settings](IColumn & nested_column, ReadBuffer & buf)
|
||||
{
|
||||
if constexpr (strategy == Strategy::TryDeserialize)
|
||||
if constexpr (std::is_same_v<ReturnType, bool>)
|
||||
return nested->tryDeserializeTextJSON(nested_column, buf, settings);
|
||||
else if constexpr (strategy == Strategy::Deserialize)
|
||||
nested->deserializeTextJSON(nested_column, buf, settings);
|
||||
else if constexpr (strategy == Strategy::DeserializeNoEmptyCheck)
|
||||
nested->deserializeTextNoEmptyCheckJSON(nested_column, buf, settings);
|
||||
else
|
||||
static_assert(AlwaysFalse<strategy>::value);
|
||||
nested->deserializeTextJSON(nested_column, buf, settings);
|
||||
};
|
||||
|
||||
return deserializeImpl<ReturnType<strategy>>(column, istr, check_for_null, deserialize_nested, is_null);
|
||||
}
|
||||
|
||||
return deserializeImpl<ReturnType>(column, istr, check_for_null, deserialize_nested, is_null);
|
||||
}
|
||||
|
||||
void SerializationNullable::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
ColumnNullable & col = assert_cast<ColumnNullable &>(column);
|
||||
bool is_null;
|
||||
deserializeTextJSONImpl<Strategy::Deserialize>(col.getNestedColumn(), istr, settings, nested, is_null);
|
||||
deserializeTextJSONImpl<void>(col.getNestedColumn(), istr, settings, nested, is_null);
|
||||
safeAppendToNullMap<void>(col, is_null);
|
||||
}
|
||||
|
||||
@ -897,27 +870,20 @@ bool SerializationNullable::tryDeserializeTextJSON(IColumn & column, ReadBuffer
|
||||
{
|
||||
ColumnNullable & col = assert_cast<ColumnNullable &>(column);
|
||||
bool is_null;
|
||||
return deserializeTextJSONImpl<Strategy::TryDeserialize>(col.getNestedColumn(), istr, settings, nested, is_null) && safeAppendToNullMap<bool>(col, is_null);
|
||||
return deserializeTextJSONImpl<bool>(col.getNestedColumn(), istr, settings, nested, is_null) && safeAppendToNullMap<bool>(col, is_null);
|
||||
}
|
||||
|
||||
bool SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(DB::IColumn & nested_column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, const DB::SerializationPtr & nested_serialization)
|
||||
{
|
||||
bool is_null;
|
||||
deserializeTextJSONImpl<Strategy::Deserialize>(nested_column, istr, settings, nested_serialization, is_null);
|
||||
return !is_null;
|
||||
}
|
||||
|
||||
bool SerializationNullable::deserializeNullAsDefaultOrNestedTextNoEmptyCheckJSON(DB::IColumn & nested_column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, const DB::SerializationPtr & nested_serialization)
|
||||
{
|
||||
bool is_null;
|
||||
deserializeTextJSONImpl<Strategy::DeserializeNoEmptyCheck>(nested_column, istr, settings, nested_serialization, is_null);
|
||||
deserializeTextJSONImpl<void>(nested_column, istr, settings, nested_serialization, is_null);
|
||||
return !is_null;
|
||||
}
|
||||
|
||||
bool SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(DB::IColumn & nested_column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, const DB::SerializationPtr & nested_serialization)
|
||||
{
|
||||
bool is_null;
|
||||
return deserializeTextJSONImpl<Strategy::TryDeserialize>(nested_column, istr, settings, nested_serialization, is_null);
|
||||
return deserializeTextJSONImpl<bool>(nested_column, istr, settings, nested_serialization, is_null);
|
||||
}
|
||||
|
||||
void SerializationNullable::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
|
@ -88,7 +88,6 @@ public:
|
||||
static bool deserializeNullAsDefaultOrNestedTextQuoted(IColumn & nested_column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested_serialization);
|
||||
static bool deserializeNullAsDefaultOrNestedTextCSV(IColumn & nested_column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization);
|
||||
static bool deserializeNullAsDefaultOrNestedTextJSON(IColumn & nested_column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested_serialization);
|
||||
static bool deserializeNullAsDefaultOrNestedTextNoEmptyCheckJSON(IColumn & nested_column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested_serialization);
|
||||
static bool deserializeNullAsDefaultOrNestedTextRaw(IColumn & nested_column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested_serialization);
|
||||
|
||||
/// If Check for NULL and deserialize value into non-nullable column or insert default value of nested type.
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Core/Field.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
@ -317,24 +318,100 @@ ReturnType SerializationTuple::deserializeTextJSONImpl(IColumn & column, ReadBuf
|
||||
{
|
||||
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
|
||||
|
||||
auto deserialize_element = [&](IColumn & element_column, size_t element_pos)
|
||||
static constexpr auto EMPTY_STRING = "\"\"";
|
||||
static constexpr auto EMPTY_STRING_LENGTH = std::string_view(EMPTY_STRING).length();
|
||||
|
||||
auto do_deserialize_element = [](IColumn & element_column, size_t element_pos, ReadBuffer & buf, auto && check_for_empty_string, auto && deserialize) -> ReturnType
|
||||
{
|
||||
if (check_for_empty_string(buf))
|
||||
{
|
||||
element_column.insertDefault();
|
||||
return ReturnType(true);
|
||||
}
|
||||
|
||||
return deserialize(element_column, element_pos, buf);
|
||||
};
|
||||
|
||||
auto deserialize_element_impl = [&settings, this](IColumn & element_column, size_t element_pos, ReadBuffer & buf) -> ReturnType
|
||||
{
|
||||
if constexpr (throw_exception)
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column))
|
||||
SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(element_column, istr, settings, elems[element_pos]);
|
||||
SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(element_column, buf, settings, elems[element_pos]);
|
||||
else
|
||||
elems[element_pos]->deserializeTextJSON(element_column, istr, settings);
|
||||
return true;
|
||||
elems[element_pos]->deserializeTextJSON(element_column, buf, settings);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column))
|
||||
return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(element_column, istr, settings, elems[element_pos]);
|
||||
return elems[element_pos]->tryDeserializeTextJSON(element_column, istr, settings);
|
||||
return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(element_column, buf, settings, elems[element_pos]);
|
||||
return elems[element_pos]->tryDeserializeTextJSON(element_column, buf, settings);
|
||||
}
|
||||
};
|
||||
|
||||
auto deserialize_element = [&settings, &do_deserialize_element, &deserialize_element_impl, &istr](IColumn & element_column, size_t element_pos) -> ReturnType
|
||||
{
|
||||
if (!settings.json.empty_as_default || istr.eof() || *istr.position() != EMPTY_STRING[0])
|
||||
return do_deserialize_element(element_column, element_pos, istr, [](ReadBuffer &) { return false; }, deserialize_element_impl);
|
||||
|
||||
if (istr.available() >= EMPTY_STRING_LENGTH)
|
||||
{
|
||||
/// We have enough data in buffer to check if we have an empty string.
|
||||
auto check_for_empty_string = [](ReadBuffer & buf_) -> bool
|
||||
{
|
||||
auto * pos = buf_.position();
|
||||
if (checkString(EMPTY_STRING, buf_))
|
||||
return true;
|
||||
else
|
||||
{
|
||||
buf_.position() = pos;
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
return do_deserialize_element(element_column, element_pos, istr, check_for_empty_string, deserialize_element_impl);
|
||||
}
|
||||
|
||||
/// We don't have enough data in buffer to check if we have an empty string.
|
||||
/// Use PeekableReadBuffer to make a checkpoint before checking for an
|
||||
/// empty string and rollback if check was failed.
|
||||
|
||||
auto check_for_empty_string = [](ReadBuffer & buf_) -> bool
|
||||
{
|
||||
auto & peekable_buf = assert_cast<PeekableReadBuffer &>(buf_);
|
||||
peekable_buf.setCheckpoint();
|
||||
SCOPE_EXIT(peekable_buf.dropCheckpoint());
|
||||
if (checkString(EMPTY_STRING, peekable_buf))
|
||||
return true;
|
||||
else
|
||||
{
|
||||
peekable_buf.rollbackToCheckpoint();
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
auto deserialize_element_impl_with_check = [&deserialize_element_impl](IColumn & element_column_, size_t element_pos_, ReadBuffer & buf_) -> ReturnType
|
||||
{
|
||||
auto & peekable_buf = assert_cast<PeekableReadBuffer &>(buf_);
|
||||
if constexpr (throw_exception)
|
||||
{
|
||||
deserialize_element_impl(element_column_, element_pos_, peekable_buf);
|
||||
assert(!peekable_buf.hasUnreadData());
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!deserialize_element_impl(element_column_, element_pos_, peekable_buf))
|
||||
return false;
|
||||
if (likely(!peekable_buf.hasUnreadData()))
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
PeekableReadBuffer peekable_buf(istr, true);
|
||||
return do_deserialize_element(element_column, element_pos, peekable_buf, check_for_empty_string, deserialize_element_impl_with_check);
|
||||
};
|
||||
|
||||
if (settings.json.read_named_tuples_as_objects
|
||||
&& have_explicit_names)
|
||||
{
|
||||
|
@ -94,7 +94,7 @@ void SerializationUUID::serializeTextJSON(const IColumn & column, size_t row_num
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
|
||||
void SerializationUUID::deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
void SerializationUUID::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
UUID x;
|
||||
assertChar('"', istr);
|
||||
@ -103,7 +103,7 @@ void SerializationUUID::deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBu
|
||||
assert_cast<ColumnUUID &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
bool SerializationUUID::tryDeserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
bool SerializationUUID::tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
UUID x;
|
||||
if (!checkChar('"', istr) || !tryReadText(x, istr) || !checkChar('"', istr))
|
||||
|
@ -1,12 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SerializationNumber.h>
|
||||
#include <DataTypes/Serializations/SerializationAsStringNonTrivialJSON.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class SerializationUUID : public SerializationAsStringNonTrivialJSON<SimpleTextSerialization>
|
||||
class SerializationUUID : public SimpleTextSerialization
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
@ -16,8 +15,8 @@ public:
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextNoEmptyCheckJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
bool tryDeserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
bool tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
@ -3,6 +3,8 @@
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Formats/EscapingRuleUtils.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferValidUTF8.h>
|
||||
#include <DataTypes/Serializations/SerializationNullable.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
@ -268,6 +270,9 @@ namespace JSONUtils
|
||||
const FormatSettings & format_settings,
|
||||
bool yield_strings)
|
||||
{
|
||||
static constexpr auto EMPTY_STRING = "\"\"";
|
||||
static constexpr auto EMPTY_STRING_LENGTH = std::string_view(EMPTY_STRING).length();
|
||||
|
||||
try
|
||||
{
|
||||
bool as_nullable = format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type);
|
||||
@ -286,33 +291,78 @@ namespace JSONUtils
|
||||
return true;
|
||||
}
|
||||
|
||||
if (format_settings.json.empty_as_default && type->isNonTriviallySerializedAsStringJSON())
|
||||
auto do_deserialize = [](IColumn & column_, ReadBuffer & buf_, auto && check_for_empty_string, auto && deserialize) -> bool
|
||||
{
|
||||
/// We have a non-numeric non-string data type at the top level.
|
||||
/// At first glance, it looks like we sort of duplicate the work done in
|
||||
/// SerializationAsStringNonTrivialJSON. Actually we need to proceed as
|
||||
/// done here because we want to return false if we inserted a default
|
||||
/// value on purpose, which the ISerialization interface does not allow for.
|
||||
if (tryMatchEmptyString(in))
|
||||
if (check_for_empty_string(buf_))
|
||||
{
|
||||
column.insertDefault();
|
||||
column_.insertDefault();
|
||||
return false;
|
||||
}
|
||||
else
|
||||
return deserialize(column_, buf_);
|
||||
};
|
||||
|
||||
if (as_nullable)
|
||||
return SerializationNullable::deserializeNullAsDefaultOrNestedTextNoEmptyCheckJSON(column, in, format_settings, serialization);
|
||||
|
||||
serialization->deserializeTextNoEmptyCheckJSON(column, in, format_settings);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
auto deserialize_impl = [as_nullable, &format_settings, &serialization](IColumn & column_, ReadBuffer & buf_) -> bool
|
||||
{
|
||||
if (as_nullable)
|
||||
return SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(column, in, format_settings, serialization);
|
||||
return SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(column_, buf_, format_settings, serialization);
|
||||
|
||||
serialization->deserializeTextJSON(column, in, format_settings);
|
||||
serialization->deserializeTextJSON(column_, buf_, format_settings);
|
||||
return true;
|
||||
};
|
||||
|
||||
if (!format_settings.json.empty_as_default || in.eof() || *in.position() != EMPTY_STRING[0])
|
||||
return do_deserialize(column, in, [](ReadBuffer &) { return false; }, deserialize_impl);
|
||||
|
||||
if (in.available() >= EMPTY_STRING_LENGTH)
|
||||
{
|
||||
/// We have enough data in buffer to check if we have an empty string.
|
||||
auto check_for_empty_string = [](ReadBuffer & buf_)
|
||||
{
|
||||
auto * pos = buf_.position();
|
||||
if (checkString(EMPTY_STRING, buf_))
|
||||
return true;
|
||||
else
|
||||
{
|
||||
buf_.position() = pos;
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
return do_deserialize(column, in, check_for_empty_string, deserialize_impl);
|
||||
}
|
||||
|
||||
/// We don't have enough data in buffer to check if we have an empty string.
|
||||
/// Use PeekableReadBuffer to make a checkpoint before checking for an
|
||||
/// empty string and rollback if check was failed.
|
||||
|
||||
auto check_for_empty_string = [](ReadBuffer & buf_) -> bool
|
||||
{
|
||||
auto & peekable_buf = assert_cast<PeekableReadBuffer &>(buf_);
|
||||
peekable_buf.setCheckpoint();
|
||||
SCOPE_EXIT(peekable_buf.dropCheckpoint());
|
||||
if (checkString(EMPTY_STRING, peekable_buf))
|
||||
return true;
|
||||
else
|
||||
{
|
||||
peekable_buf.rollbackToCheckpoint();
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
auto deserialize_impl_with_check = [&deserialize_impl](IColumn & column_, ReadBuffer & buf_) -> bool
|
||||
{
|
||||
auto & peekable_buf = assert_cast<PeekableReadBuffer &>(buf_);
|
||||
|
||||
if (!deserialize_impl(column_, peekable_buf))
|
||||
return false;
|
||||
if (likely(!peekable_buf.hasUnreadData()))
|
||||
return true;
|
||||
return false;
|
||||
};
|
||||
|
||||
PeekableReadBuffer peekable_buf(in, true);
|
||||
return do_deserialize(column, peekable_buf, check_for_empty_string, deserialize_impl_with_check);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
|
@ -1137,23 +1137,6 @@ template void readCSVStringInto<String, false, false>(String & s, ReadBuffer & b
|
||||
template void readCSVStringInto<String, true, false>(String & s, ReadBuffer & buf, const FormatSettings::CSV & settings);
|
||||
template void readCSVStringInto<PaddedPODArray<UInt8>, false, false>(PaddedPODArray<UInt8> & s, ReadBuffer & buf, const FormatSettings::CSV & settings);
|
||||
|
||||
bool tryMatchEmptyString(ReadBuffer & buf)
|
||||
{
|
||||
if (buf.eof() || *buf.position() != '"')
|
||||
return false;
|
||||
|
||||
++buf.position();
|
||||
|
||||
if (buf.eof() || *buf.position() != '"')
|
||||
{
|
||||
--buf.position();
|
||||
return false;
|
||||
}
|
||||
|
||||
++buf.position();
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename Vector, typename ReturnType>
|
||||
ReturnType readJSONStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::JSON & settings)
|
||||
|
@ -665,10 +665,6 @@ void readStringUntilEOFInto(Vector & s, ReadBuffer & buf);
|
||||
template <typename Vector, bool include_quotes = false, bool allow_throw = true>
|
||||
void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & settings);
|
||||
|
||||
/// Consumes the current token if it is an empty string, i.e. two consecutive double quotes,
|
||||
/// Returns true if consumed.
|
||||
bool tryMatchEmptyString(ReadBuffer & buf);
|
||||
|
||||
/// ReturnType is either bool or void. If bool, the function will return false instead of throwing an exception.
|
||||
template <typename Vector, typename ReturnType = void>
|
||||
ReturnType readJSONStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::JSON & settings);
|
||||
|
@ -24,18 +24,18 @@ SELECT * FROM table1 ORDER BY address ASC;
|
||||
-- Nullable
|
||||
-- { echoOn }
|
||||
SELECT x FROM format(JSONEachRow, 'x Nullable(IPv6)', '{"x":""}');
|
||||
::
|
||||
\N
|
||||
-- Compound types
|
||||
SELECT x FROM format(JSONEachRow, 'x Array(UUID)', '{"x":["00000000-0000-0000-0000-000000000000","b15f852c-c41a-4fd6-9247-1929c841715e",""]}');
|
||||
['00000000-0000-0000-0000-000000000000','b15f852c-c41a-4fd6-9247-1929c841715e','00000000-0000-0000-0000-000000000000']
|
||||
SELECT x FROM format(JSONEachRow, 'x Array(Nullable(IPv6))', '{"x":["",""]}');
|
||||
['::','::']
|
||||
[NULL,NULL]
|
||||
SELECT x FROM format(JSONEachRow, 'x Tuple(Date, IPv4, String)', '{"x":["", "", "abc"]}');
|
||||
('1970-01-01','0.0.0.0','abc')
|
||||
SELECT x FROM format(JSONEachRow, 'x Map(String, IPv6)', '{"x":{"abc": ""}}');
|
||||
{'abc':'::'}
|
||||
SELECT x FROM format(JSONEachRow, 'x Variant(Date, UUID)', '{"x":""}');
|
||||
00000000-0000-0000-0000-000000000000
|
||||
\N
|
||||
-- Deep composition
|
||||
SELECT x FROM format(JSONEachRow, 'x Array(Array(IPv6))', '{"x":[["2001:db8:3333:4444:CCCC:DDDD:EEEE:FFFF", ""], ["", "2001:db8:3333:4444:5555:6666:7777:8888"]]}');
|
||||
[['2001:db8:3333:4444:cccc:dddd:eeee:ffff','::'],['::','2001:db8:3333:4444:5555:6666:7777:8888']]
|
||||
@ -44,4 +44,4 @@ SELECT x FROM format(JSONEachRow, 'x Variant(Date, Array(UUID))', '{"x":["", "b1
|
||||
SELECT x FROM format(JSONEachRow, 'x Tuple(Array(UUID), Tuple(UUID, Map(String, IPv6)))', '{"x":[[""], ["",{"abc":""}]]}');
|
||||
(['00000000-0000-0000-0000-000000000000'],('00000000-0000-0000-0000-000000000000',{'abc':'::'}))
|
||||
SELECT x FROM format(JSONEachRow, 'x Map(Tuple(Date,IPv4), Variant(UUID,IPv6))', '{"x":{["",""]:""}}');
|
||||
{('1970-01-01','0.0.0.0'):'00000000-0000-0000-0000-000000000000'}
|
||||
{('1970-01-01','0.0.0.0'):NULL}
|
Loading…
Reference in New Issue
Block a user