post-review changes

This commit is contained in:
Alexis Arnaud 2024-08-06 19:31:30 +02:00
parent 3c586d80c8
commit 229fffcd56
8 changed files with 154 additions and 310 deletions

View File

@ -89,6 +89,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"optimize_functions_to_subcolumns", false, true, "Enabled settings by default"},
{"allow_experimental_vector_similarity_index", false, false, "Added new setting to allow experimental vector similarity indexes"},
{"input_format_json_empty_as_default", true, false, "Added new setting to allow to treat empty fields in JSON input as default values."}
{"input_format_json_empty_as_default", false, false, "Added new setting to allow to treat empty fields in JSON input as default values."}
}
},
{"24.7",

View File

@ -5,13 +5,13 @@
#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>
#include <IO/WriteBufferFromString.h>
#include <Formats/FormatSettings.h>
#include <Formats/JSONUtils.h>
namespace DB
{
@ -620,108 +620,16 @@ 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
auto deserializer = [&nested](IColumn & column_, ReadBuffer & istr_, auto && deserialize_nested) -> ReturnType
{
if (check_for_empty_string(buf))
auto adapter = [&deserialize_nested, &istr_, &nested](IColumn & nested_column) -> ReturnType
{
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;
}
return deserialize_nested(nested_column, istr_, nested);
};
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_, adapter, false);
};
return deserializeTextImpl<ReturnType>(column, istr,
[&deserialize_nested, &istr](IColumn & nested_column) -> ReturnType
{
return deserialize_nested(nested_column, istr);
}, false);
return JSONUtils::deserializeEmpyStringAsDefaultOrNested<ReturnType>(column, istr, settings, deserializer);
}
}

View File

@ -6,9 +6,9 @@
#include <Columns/ColumnMap.h>
#include <Core/Field.h>
#include <Formats/FormatSettings.h>
#include <Formats/JSONUtils.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>
@ -319,103 +319,16 @@ void SerializationMap::serializeTextJSONPretty(const IColumn & column, size_t ro
template <typename ReturnType>
ReturnType SerializationMap::deserializeTextJSONMapImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
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
auto deserializer = [this](IColumn & column_, ReadBuffer & istr_, auto && deserialize_nested) -> ReturnType
{
if (check_for_empty_string(buf))
auto adapter = [&deserialize_nested](ReadBuffer & buf, const SerializationPtr & nested_column_serialization, IColumn & nested_column) -> ReturnType
{
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;
}
return deserialize_nested(nested_column, buf, nested_column_serialization);
};
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 this->deserializeTextImpl<ReturnType>(column_, istr_, adapter);
};
return deserializeTextImpl<ReturnType>(column, istr, deserialize_subcolumn);
return JSONUtils::deserializeEmpyStringAsDefaultOrNested<ReturnType>(column, istr, settings, deserializer);
}
void SerializationMap::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const

View File

@ -5,7 +5,7 @@
#include <Core/Field.h>
#include <Columns/ColumnTuple.h>
#include <Common/assert_cast.h>
#include <IO/PeekableReadBuffer.h>
#include <Formats/JSONUtils.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromString.h>
@ -314,103 +314,9 @@ void SerializationTuple::serializeTextJSONPretty(const IColumn & column, size_t
}
template <typename ReturnType>
ReturnType SerializationTuple::deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
ReturnType SerializationTuple::deserializeTextJSONTupleImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, auto && deserialize_element) const
{
static constexpr bool 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_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, buf, settings, elems[element_pos]);
else
elems[element_pos]->deserializeTextJSON(element_column, buf, settings);
}
else
{
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column))
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);
};
static constexpr auto throw_exception = std::is_same_v<ReturnType, void>;
if (settings.json.read_named_tuples_as_objects
&& have_explicit_names)
@ -583,6 +489,22 @@ ReturnType SerializationTuple::deserializeTextJSONImpl(IColumn & column, ReadBuf
}
}
template <typename ReturnType>
ReturnType SerializationTuple::deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
auto deserializer = [&settings, this](IColumn & column_, ReadBuffer & istr_, auto && deserialize_nested) -> ReturnType
{
auto adapter = [&deserialize_nested, &istr_, this](IColumn & nested_column, size_t element_pos) -> ReturnType
{
return deserialize_nested(nested_column, istr_, this->elems[element_pos]);
};
return this->deserializeTextJSONTupleImpl<ReturnType>(column_, istr_, settings, adapter);
};
return JSONUtils::deserializeEmpyStringAsDefaultOrNested<ReturnType>(column, istr, settings, deserializer);
}
void SerializationTuple::deserializeTextJSON(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings) const
{
deserializeTextJSONImpl(column, istr, settings);

View File

@ -81,6 +81,9 @@ private:
template <typename ReturnType = void>
ReturnType deserializeTextImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const;
template <typename ReturnType>
ReturnType deserializeTextJSONTupleImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, auto && deserialize_element) const;
template <typename ReturnType = void>
ReturnType deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const;

View File

@ -151,7 +151,6 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
format_settings.json.try_infer_objects_as_tuples = settings.input_format_json_try_infer_named_tuples_from_objects;
format_settings.json.throw_on_bad_escape_sequence = settings.input_format_json_throw_on_bad_escape_sequence;
format_settings.json.ignore_unnecessary_fields = settings.input_format_json_ignore_unnecessary_fields;
format_settings.json.case_insensitive_column_matching = settings.input_format_json_case_insensitive_column_matching;
format_settings.json.empty_as_default = settings.input_format_json_empty_as_default;
format_settings.null_as_default = settings.input_format_null_as_default;
format_settings.force_null_for_omitted_fields = settings.input_format_force_null_for_omitted_fields;

View File

@ -3,10 +3,7 @@
#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>
#include <DataTypes/DataTypeObject.h>
#include <DataTypes/DataTypeFactory.h>
@ -311,8 +308,8 @@ namespace JSONUtils
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 (!format_settings.json.empty_as_default || in.eof() || *in.position() != EMPTY_STRING[0])
return deserialize_impl(column, in);
if (in.available() >= EMPTY_STRING_LENGTH)
{
@ -322,11 +319,8 @@ namespace JSONUtils
auto * pos = buf_.position();
if (checkString(EMPTY_STRING, buf_))
return true;
else
{
buf_.position() = pos;
return false;
}
buf_.position() = pos;
return false;
};
return do_deserialize(column, in, check_for_empty_string, deserialize_impl);
@ -343,22 +337,17 @@ namespace JSONUtils
SCOPE_EXIT(peekable_buf.dropCheckpoint());
if (checkString(EMPTY_STRING, peekable_buf))
return true;
else
{
peekable_buf.rollbackToCheckpoint();
return false;
}
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;
bool res = deserialize_impl(column_, peekable_buf);
if (unlikely(peekable_buf.hasUnreadData()))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect state while parsing JSON: PeekableReadBuffer has unread data in own memory: {}", String(peekable_buf.position(), peekable_buf.available()));
return res;
};
PeekableReadBuffer peekable_buf(in, true);

View File

@ -2,11 +2,15 @@
#include <DataTypes/IDataType.h>
#include <DataTypes/Serializations/ISerialization.h>
#include <DataTypes/Serializations/SerializationNullable.h>
#include <Formats/FormatSettings.h>
#include <IO/BufferWithOwnMemory.h>
#include <IO/PeekableReadBuffer.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/Progress.h>
#include <Core/NamesAndTypes.h>
#include <Common/assert_cast.h>
#include <Common/Stopwatch.h>
#include <utility>
@ -16,6 +20,11 @@ namespace DB
class Block;
struct JSONInferenceInfo;
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace JSONUtils
{
std::pair<bool, size_t> fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_bytes, size_t max_rows);
@ -136,6 +145,106 @@ namespace JSONUtils
bool skipUntilFieldInObject(ReadBuffer & in, const String & desired_field_name, const FormatSettings::JSON & settings);
void skipTheRestOfObject(ReadBuffer & in, const FormatSettings::JSON & settings);
template <typename ReturnType>
ReturnType deserializeEmpyStringAsDefaultOrNested(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, auto && deserializer)
{
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, const SerializationPtr & nested_column_serialization) -> ReturnType
{
if (check_for_empty_string(buf))
{
nested_column.insertDefault();
return ReturnType(true);
}
return deserialize(nested_column, buf, nested_column_serialization);
};
auto deserialize_nested_impl = [&settings](IColumn & nested_column, ReadBuffer & buf, const SerializationPtr & nested_column_serialization) -> ReturnType
{
if constexpr (throw_exception)
{
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column))
SerializationNullable::deserializeNullAsDefaultOrNestedTextJSON(nested_column, buf, settings, nested_column_serialization);
else
nested_column_serialization->deserializeTextJSON(nested_column, buf, settings);
}
else
{
if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(nested_column))
return SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextJSON(nested_column, buf, settings, nested_column_serialization);
return nested_column_serialization->tryDeserializeTextJSON(nested_column, buf, settings);
}
};
auto deserialize_nested = [&settings, &do_deserialize_nested, &deserialize_nested_impl](IColumn & nested_column, ReadBuffer & buf, const SerializationPtr & nested_column_serialization) -> ReturnType
{
if (!settings.json.empty_as_default || buf.eof() || *buf.position() != EMPTY_STRING[0])
return deserialize_nested_impl(nested_column, buf, nested_column_serialization);
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;
buf_.position() = pos;
return false;
};
return do_deserialize_nested(nested_column, buf, check_for_empty_string, deserialize_nested_impl, nested_column_serialization);
}
/// 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;
peekable_buf.rollbackToCheckpoint();
return false;
};
auto deserialize_nested_impl_with_check = [&deserialize_nested_impl](IColumn & nested_column_, ReadBuffer & buf_, const SerializationPtr & nested_column_serialization_) -> ReturnType
{
auto & peekable_buf = assert_cast<PeekableReadBuffer &>(buf_);
auto enforceNoUnreadData = [&peekable_buf]() -> void
{
if (unlikely(peekable_buf.hasUnreadData()))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect state while parsing JSON: PeekableReadBuffer has unread data in own memory: {}", String(peekable_buf.position(), peekable_buf.available()));
};
if constexpr (throw_exception)
{
deserialize_nested_impl(nested_column_, peekable_buf, nested_column_serialization_);
enforceNoUnreadData();
}
else
{
bool res = deserialize_nested_impl(nested_column_, peekable_buf, nested_column_serialization_);
enforceNoUnreadData();
return res;
}
};
PeekableReadBuffer peekable_buf(buf, true);
return do_deserialize_nested(nested_column, peekable_buf, check_for_empty_string, deserialize_nested_impl_with_check, nested_column_serialization);
};
return deserializer(column, istr, deserialize_nested);
}
}
}