ClickHouse/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

580 lines
18 KiB
C++
Raw Normal View History

#include <Processors/Formats/Impl/MsgPackRowInputFormat.h>
2020-07-10 19:08:18 +00:00
#if USE_MSGPACK
Suppress clang-analyzer-cplusplus.NewDelete in MsgPackRowInputFormat Appartently there is some issue with clang-15, since even the following example shows error [1]. [1]: https://gist.github.com/azat/027f0e949ea836fc2e6269113ceb8752 clang-tidy report [1]: FAILED: src/CMakeFiles/dbms.dir/Processors/Formats/Impl/MsgPackRowInputFormat.cpp.o /usr/bin/cmake -E __run_co_compile --launcher="prlimit;--as=10000000000;--data=5000000000;--cpu=1000;/usr/bin/ccache" --tidy=/usr/bin/clang-tidy-15 --source=/ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp -- /usr/bin/clang++-15 --target=x86_64-linux-gnu --sysroot=/ch/cmake/linux/../../contrib/sysroot/linux-x86_64/x86_64-linux-gnu/libc -DAWS_SDK_VERSION_MAJOR=1 -DAWS_SDK_VERSION_MINOR=7 -DAWS_SDK_VERSION_PATCH=231 -DBOOST_ASIO_HAS_STD_INVOKE_RESULT=1 -DBOOST_ASIO_STANDALONE=1 -DCARES_STATICLIB -DCONFIGDIR=\"\" -DENABLE_MULTITARGET_CODE=1 -DENABLE_OPENSSL_ENCRYPTION -DHAS_RESERVED_IDENTIFIER -DHAVE_CONFIG_H -DLIBSASL_EXPORTS=1 -DLZ4_DISABLE_DEPRECATE_WARNINGS=1 -DOBSOLETE_CRAM_ATTR=1 -DOBSOLETE_DIGEST_ATTR=1 -DPLUGINDIR=\"\" -DPOCO_ENABLE_CPP11 -DPOCO_HAVE_FD_EPOLL -DPOCO_OS_FAMILY_UNIX -DSASLAUTHD_CONF_FILE_DEFAULT=\"\" -DSNAPPY_CODEC_AVAILABLE -DSTD_EXCEPTION_HAS_STACK_TRACE=1 -DUNALIGNED_OK -DWITH_COVERAGE=0 -DWITH_GZFILEOP -DX86_64 -DZLIB_COMPAT -D_LIBCPP_ENABLE_THREAD_SAFETY_ANNOTATIONS -Iincludes/configs -I/ch/src -Isrc -Isrc/Core/include -I/ch/base/glibc-compatibility/memcpy -I/ch/base/base/.. -Ibase/base/.. -I/ch/contrib/cctz/include -I/ch/base/pcg-random/. -I/ch/contrib/miniselect/include -I/ch/contrib/zstd/lib -Icontrib/cyrus-sasl-cmake -I/ch/contrib/lz4/lib -I/ch/src/Common/mysqlxx/. -Icontrib/c-ares -I/ch/contrib/c-ares -I/ch/contrib/c-ares/include -isystem /ch/contrib/libcxx/include -isystem /ch/contrib/libcxxabi/include -isystem /ch/contrib/libunwind/include -isystem /ch/contrib/libdivide/. -isystem /ch/contrib/jemalloc-cmake/include -isystem /ch/contrib/llvm/llvm/include -isystem contrib/llvm/llvm/include -isystem /ch/contrib/abseil-cpp -isystem /ch/contrib/croaring/cpp -isystem /ch/contrib/croaring/include -isystem /ch/contrib/cityhash102/include -isystem /ch/contrib/boost -isystem /ch/contrib/poco/Net/include -isystem /ch/contrib/poco/Foundation/include -isystem /ch/contrib/poco/NetSSL_OpenSSL/include -isystem /ch/contrib/poco/Crypto/include -isystem /ch/contrib/boringssl/include -isystem /ch/contrib/poco/Util/include -isystem /ch/contrib/poco/JSON/include -isystem /ch/contrib/poco/XML/include -isystem /ch/contrib/replxx/include -isystem /ch/contrib/fmtlib-cmake/../fmtlib/include -isystem /ch/contrib/magic_enum/include -isystem /ch/contrib/double-conversion -isystem /ch/contrib/dragonbox/include -isystem /ch/contrib/re2 -isystem contrib/re2-cmake -isystem /ch/contrib/zlib-ng -isystem contrib/zlib-ng-cmake -isystem /ch/contrib/pdqsort -isystem /ch/contrib/xz/src/liblzma/api -isystem /ch/contrib/aws-c-common/include -isystem /ch/contrib/aws-c-event-stream/include -isystem /ch/contrib/aws/aws-cpp-sdk-s3/include -isystem /ch/contrib/aws/aws-cpp-sdk-core/include -isystem contrib/aws-s3-cmake/include -isystem /ch/contrib/snappy -isystem contrib/snappy-cmake -isystem /ch/contrib/msgpack-c/include -isystem /ch/contrib/fast_float/include -isystem /ch/contrib/librdkafka-cmake/include -isystem /ch/contrib/librdkafka/src -isystem contrib/librdkafka-cmake/auxdir -isystem /ch/contrib/cppkafka/include -isystem /ch/contrib/nats-io/src -isystem /ch/contrib/nats-io/src/adapters -isystem /ch/contrib/nats-io/src/include -isystem /ch/contrib/nats-io/src/unix -isystem /ch/contrib/libuv/include -isystem /ch/contrib/krb5/src/include -isystem contrib/krb5-cmake/include -isystem /ch/contrib/NuRaft/include -isystem /ch/contrib/poco/MongoDB/include -isystem contrib/mariadb-connector-c-cmake/include-public -isystem /ch/contrib/mariadb-connector-c/include -isystem /ch/contrib/mariadb-connector-c/libmariadb -isystem /ch/contrib/icu/icu4c/source/i18n -isystem /ch/contrib/icu/icu4c/source/common -isystem /ch/contrib/capnproto/c++/src -isystem /ch/contrib/arrow/cpp/src -isystem /ch/contrib/arrow-cmake/cpp/src -isystem contrib/arrow-cmake/cpp/src -isystem contrib/arrow-cmake/../orc/c++/include -isystem /ch/contrib/orc/c++/include -isystem contrib/avro-cmake/include -isystem /ch/contrib/avro/lang/c++/api -isystem /ch/contrib/openldap-cmake/linux_x86_64/include -isystem /ch/contrib/openldap/include -isystem /ch/contrib/sparsehash-c11 -isystem /ch/contrib/protobuf/src -isystem src/Server/grpc_protos -isystem /ch/contrib/grpc/include -isystem /ch/contrib/libhdfs3/include -isystem /ch/contrib/hive-metastore -isystem /ch/contrib/thrift/lib/cpp/src -isystem contrib/thrift-cmake -isystem /ch/contrib/azure/sdk/core/azure-core/inc-isystem /ch/contrib/azure/sdk/identity/azure-identity/inc -isystem /ch/contrib/azure/sdk/storage/azure-storage-common/inc -isystem /ch/contrib/azure/sdk/storage/azure-storage-blobs/inc -isystem /ch/contrib/s2geometry/src -isystem /ch/contrib/AMQP-CPP/include -isystem /ch/contrib/AMQP-CPP -isystem /ch/contrib/sqlite-amalgamation -isystem /ch/contrib/rocksdb/include -isystem /ch/contrib/libpqxx/include -isystem /ch/contrib/libpq -isystem /ch/contrib/libpq/include -isystem /ch/contrib/libstemmer_c/include -isystem /ch/contrib/wordnet-blast -isystem /ch/contrib/lemmagen-c/include -isystem /ch/contrib/simdjson/include -isystem /ch/contrib/rapidjson/include -isystem /ch/contrib/consistent-hashing --gcc-toolchain=/ch/cmake/linux/../../contrib/sysroot/linux-x86_64 -std=c++20 -fdiagnostics-color=always -Xclang -fuse-ctor-homing -fsized-deallocation -UNDEBUG -gdwarf-aranges -pipe -mssse3 -msse4.1 -msse4.2 -mpclmul -mpopcnt -fasynchronous-unwind-tables -falign-functions=32 -mbranches-within-32B-boundaries -fdiagnostics-absolute-paths -fstrict-vtable-pointers -fexperimental-new-pass-manager -Wall -Wextra -Weverything -Wpedantic -Wno-zero -length-array -Wno-c++98-compat-pedantic -Wno-c++98-compat -Wno-c++20-compat -Wno-conversion -Wno-ctad-maybe-unsupported -Wno-disabled-macro-expansion -Wno-documentation-unknown-command -Wno-double-promotion -Wno-exit-time-destructors -Wno-float-equal -Wno-global-constructors -Wno-missing-prototypes -Wno-missing-variable-declarations -Wno-padded -Wno-switch-enum -Wno-undefined-func-template -Wno-unused-template -Wno-vla -Wno-weak-template-vtables -Wno-weak-vtables -Wno-thread-safety-negative -g -O0 -g -gdwarf-4 -fno-inline -D_LIBCPP_DEBUG=0 -D OS_LINUX -I/ch/base -I/ch/contrib/magic_enum/include -include /ch/src/Core/iostream_debug_helpers.h -Werror -nostdinc++ -std=gnu++2a -MD -MT src/CMakeFiles/dbms.dir/Processors/Formats/Impl/MsgPackRowInputFormat.cpp.o -MF src/CMakeFiles/dbms.dir/Processors/Formats/Impl/MsgPackRowInputFormat.cpp.o.d -o src/CMakeFiles/dbms.dir/Processors/Formats/Impl/MsgPackRowInputFormat.cpp.o -c /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp /ch/contrib/msgpack-c/include/msgpack/v1/detail/cpp11_zone.hpp:195:9: error: Attempt to free released memory [clang-analyzer-cplusplus.NewDelete,-warnings-as-errors] ::free(p); ^ /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp:509:5: note: Taking false branch if (buf.eof()) ^ /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp:514:24: note: Assuming 'i' is not equal to field 'number_of_columns' for (size_t i = 0; i != number_of_columns; ++i) ^~~~~~~~~~~~~~~~~~~~~~ /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp:514:5: note: Loop condition is true. Entering loop body for (size_t i = 0; i != number_of_columns; ++i) ^ /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp:516:30: note: Calling 'MsgPackSchemaReader::readObject' auto object_handle = readObject(); ^~~~~~~~~~~~ /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp:426:5: note: Taking false branch if (buf.eof()) ^ /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp:433:5: note: Loop condition is true. Entering loop body while (need_more_data) ^ /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp:438:29: note: Calling 'unpack' object_handle = msgpack::unpack(buf.position(), buf.buffer().end() - buf.position(), offset); ^~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ /ch/contrib/msgpack-c/include/msgpack/v3/unpack.hpp:52:12: note: Calling 'unpack' return msgpack::v3::unpack(data, len, off, referenced, f, user_data, limit); ^~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ /ch/contrib/msgpack-c/include/msgpack/v3/unpack.hpp:35:5: note: Control jumps to the 'default' case at line 40 switch(ret) { ^ /ch/contrib/msgpack-c/include/msgpack/v3/unpack.hpp:41:9: note: Execution continues on line 43 break; ^ /ch/contrib/msgpack-c/include/msgpack/v3/unpack.hpp:43:35: note: Calling '~unique_ptr' return msgpack::object_handle(); ^ /ch/contrib/libcxx/include/__memory/unique_ptr.h:269:19: note: Calling 'unique_ptr::reset' ~unique_ptr() { reset(); } ^~~~~~~ /ch/contrib/libcxx/include/__memory/unique_ptr.h:314:9: note: '__tmp' is non-null if (__tmp) ^~~~~ /ch/contrib/libcxx/include/__memory/unique_ptr.h:314:5: note: Taking true branch if (__tmp) ^ /ch/contrib/libcxx/include/__memory/unique_ptr.h:315:7: note: Calling 'default_delete::operator()' __ptr_.second()(__tmp); ^~~~~~~~~~~~~~~~~~~~~~ /ch/contrib/libcxx/include/__memory/unique_ptr.h:54:5: note: Memory is released delete __ptr; ^~~~~~~~~~~~ /ch/contrib/libcxx/include/__memory/unique_ptr.h:54:5: note: Calling 'zone::operator delete' delete __ptr; ^~~~~~~~~~~~ /ch/contrib/msgpack-c/include/msgpack/v1/detail/cpp11_zone.hpp:195:9: note: Attempt to free released memory ::free(p); ^~~~~~~~~ [1]: https://s3.amazonaws.com/clickhouse-builds/41046/9677898b3b234a5ba0371edaf719ea8890d084ff/binary_tidy/build_log.log Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-09-10 11:11:54 +00:00
/// FIXME: there is some issue with clang-15, that incorrectly detect a
/// "Attempt to free released memory" in msgpack::unpack(), because of delete
/// operator for zone (from msgpack/v1/detail/cpp11_zone.hpp), hence NOLINT
///
/// NOTE: that I was not able to suppress it locally, only with
/// NOLINTBEGIN/NOLINTEND
//
// NOLINTBEGIN(clang-analyzer-cplusplus.NewDelete)
2020-07-10 19:08:18 +00:00
#include <cstdlib>
#include <Common/assert_cast.h>
#include <IO/ReadHelpers.h>
2022-01-27 16:54:15 +00:00
#include <IO/ReadBufferFromMemory.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeNullable.h>
2021-09-16 11:25:45 +00:00
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeString.h>
2022-01-27 16:54:15 +00:00
#include <DataTypes/DataTypeUUID.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
2021-09-16 11:25:45 +00:00
#include <Columns/ColumnMap.h>
#include <Columns/ColumnLowCardinality.h>
2022-02-07 14:51:16 +00:00
#include <Formats/MsgPackExtensionTypes.h>
#include <Formats/EscapingRuleUtils.h>
2022-02-07 14:51:16 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int INCORRECT_DATA;
extern const int BAD_ARGUMENTS;
extern const int UNEXPECTED_END_OF_FILE;
}
MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_)
: MsgPackRowInputFormat(header_, std::make_unique<PeekableReadBuffer>(in_), params_) {}
MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, Params params_)
: IRowInputFormat(header_, *buf_, std::move(params_)), buf(std::move(buf_)), parser(visitor), data_types(header_.getDataTypes()) {}
void MsgPackRowInputFormat::resetParser()
{
IRowInputFormat::resetParser();
2021-12-10 18:37:08 +00:00
buf->reset();
visitor.reset();
}
2020-04-18 11:26:57 +00:00
void MsgPackVisitor::set_info(IColumn & column, DataTypePtr type) // NOLINT
{
2020-04-17 09:35:38 +00:00
while (!info_stack.empty())
{
2020-04-17 09:35:38 +00:00
info_stack.pop();
}
2020-04-17 09:35:38 +00:00
info_stack.push(Info{column, type});
}
void MsgPackVisitor::reset()
{
info_stack = {};
}
2021-09-16 14:05:07 +00:00
template <typename InsertFunc>
static bool checkAndInsertNullable(IColumn & column, DataTypePtr type, InsertFunc insert_func)
{
if (type->isNullable())
{
auto & nullable_column = assert_cast<ColumnNullable &>(column);
auto & nested_column = nullable_column.getNestedColumn();
2021-09-17 14:32:23 +00:00
const auto & nested_type = assert_cast<const DataTypeNullable *>(type.get())->getNestedType();
2021-09-16 14:05:07 +00:00
insert_func(nested_column, nested_type);
nullable_column.getNullMapColumn().insertValue(0);
return true;
}
2021-09-16 11:25:45 +00:00
2021-09-16 14:05:07 +00:00
return false;
}
2021-09-16 11:25:45 +00:00
2021-09-16 14:05:07 +00:00
template <typename InsertFunc>
static bool checkAndInsertLowCardinality(IColumn & column, DataTypePtr type, InsertFunc insert_func)
{
2021-09-22 10:47:03 +00:00
if (type->lowCardinality())
2021-09-16 14:05:07 +00:00
{
auto & lc_column = assert_cast<ColumnLowCardinality &>(column);
auto tmp_column = lc_column.getDictionary().getNestedColumn()->cloneEmpty();
auto dict_type = assert_cast<const DataTypeLowCardinality *>(type.get())->getDictionaryType();
insert_func(*tmp_column, dict_type);
lc_column.insertFromFullColumn(*tmp_column, 0);
return true;
}
return false;
}
static void insertInteger(IColumn & column, DataTypePtr type, UInt64 value)
{
auto insert_func = [&](IColumn & column_, DataTypePtr type_)
{
insertInteger(column_, type_, value);
};
if (checkAndInsertNullable(column, type, insert_func) || checkAndInsertLowCardinality(column, type, insert_func))
return;
switch (type->getTypeId())
{
case TypeIndex::UInt8:
{
2021-09-16 14:05:07 +00:00
assert_cast<ColumnUInt8 &>(column).insertValue(value);
2020-04-17 09:35:38 +00:00
break;
}
case TypeIndex::Date: [[fallthrough]];
case TypeIndex::UInt16:
{
2021-09-16 14:05:07 +00:00
assert_cast<ColumnUInt16 &>(column).insertValue(value);
2020-04-17 09:35:38 +00:00
break;
}
case TypeIndex::DateTime: [[fallthrough]];
case TypeIndex::UInt32:
{
assert_cast<ColumnUInt32 &>(column).insertValue(static_cast<UInt32>(value));
2020-04-17 09:35:38 +00:00
break;
}
case TypeIndex::UInt64:
{
2021-09-16 14:05:07 +00:00
assert_cast<ColumnUInt64 &>(column).insertValue(value);
2020-04-17 09:35:38 +00:00
break;
}
case TypeIndex::Int8:
{
2021-09-16 14:05:07 +00:00
assert_cast<ColumnInt8 &>(column).insertValue(value);
2020-04-17 09:35:38 +00:00
break;
}
case TypeIndex::Int16:
{
2021-09-16 14:05:07 +00:00
assert_cast<ColumnInt16 &>(column).insertValue(value);
2020-04-17 09:35:38 +00:00
break;
}
case TypeIndex::Int32:
{
assert_cast<ColumnInt32 &>(column).insertValue(static_cast<Int32>(value));
2020-04-17 09:35:38 +00:00
break;
}
case TypeIndex::Int64:
{
2021-09-16 14:05:07 +00:00
assert_cast<ColumnInt64 &>(column).insertValue(value);
2020-04-17 09:35:38 +00:00
break;
}
case TypeIndex::DateTime64:
{
2021-09-16 14:05:07 +00:00
assert_cast<DataTypeDateTime64::ColumnType &>(column).insertValue(value);
2021-09-16 11:25:45 +00:00
break;
}
default:
2021-09-16 14:05:07 +00:00
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack integer into column with type {}.", type->getName());
}
2020-04-17 09:35:38 +00:00
}
2022-01-27 16:54:15 +00:00
static void insertString(IColumn & column, DataTypePtr type, const char * value, size_t size, bool bin)
2021-09-16 14:05:07 +00:00
{
auto insert_func = [&](IColumn & column_, DataTypePtr type_)
{
2022-01-27 16:54:15 +00:00
insertString(column_, type_, value, size, bin);
2021-09-16 14:05:07 +00:00
};
if (checkAndInsertNullable(column, type, insert_func) || checkAndInsertLowCardinality(column, type, insert_func))
return;
2022-01-27 16:54:15 +00:00
if (isUUID(type))
{
ReadBufferFromMemory buf(value, size);
UUID uuid;
if (bin)
readBinary(uuid, buf);
else
readUUIDText(uuid, buf);
assert_cast<ColumnUUID &>(column).insertValue(uuid);
return;
}
2021-09-16 14:05:07 +00:00
if (!isStringOrFixedString(type))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack string into column with type {}.", type->getName());
column.insertData(value, size);
}
static void insertFloat32(IColumn & column, DataTypePtr type, Float32 value) // NOLINT
{
auto insert_func = [&](IColumn & column_, DataTypePtr type_)
{
insertFloat32(column_, type_, value);
};
if (checkAndInsertNullable(column, type, insert_func) || checkAndInsertLowCardinality(column, type, insert_func))
return;
if (!WhichDataType(type).isFloat32())
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack float32 into column with type {}.", type->getName());
assert_cast<ColumnFloat32 &>(column).insertValue(value);
}
static void insertFloat64(IColumn & column, DataTypePtr type, Float64 value) // NOLINT
{
auto insert_func = [&](IColumn & column_, DataTypePtr type_)
{
insertFloat64(column_, type_, value);
};
if (checkAndInsertNullable(column, type, insert_func) || checkAndInsertLowCardinality(column, type, insert_func))
return;
if (!WhichDataType(type).isFloat64())
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack float64 into column with type {}.", type->getName());
assert_cast<ColumnFloat64 &>(column).insertValue(value);
}
static void insertNull(IColumn & column, DataTypePtr type)
{
auto insert_func = [&](IColumn & column_, DataTypePtr type_)
{
insertNull(column_, type_);
};
2021-09-17 14:32:23 +00:00
/// LowCardinality(Nullable(...))
2021-09-16 14:05:07 +00:00
if (checkAndInsertLowCardinality(column, type, insert_func))
return;
if (!type->isNullable())
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack null into non-nullable column with type {}.", type->getName());
assert_cast<ColumnNullable &>(column).insertDefault();
}
static void insertUUID(IColumn & column, DataTypePtr type, const char * value, size_t size)
2022-01-27 16:54:15 +00:00
{
if (!isUUID(type))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack UUID into column with type {}.", type->getName());
2022-01-27 16:54:15 +00:00
ReadBufferFromMemory buf(value, size);
UUID uuid;
2022-02-07 14:07:44 +00:00
readBinaryBigEndian(uuid.toUnderType().items[0], buf);
readBinaryBigEndian(uuid.toUnderType().items[1], buf);
2022-01-27 16:54:15 +00:00
assert_cast<ColumnUUID &>(column).insertValue(uuid);
}
2020-04-18 11:26:57 +00:00
bool MsgPackVisitor::visit_positive_integer(UInt64 value) // NOLINT
2020-04-17 09:35:38 +00:00
{
2021-09-16 14:05:07 +00:00
insertInteger(info_stack.top().column, info_stack.top().type, value);
2020-04-17 09:35:38 +00:00
return true;
}
2020-04-18 11:26:57 +00:00
bool MsgPackVisitor::visit_negative_integer(Int64 value) // NOLINT
2020-04-17 09:35:38 +00:00
{
2021-09-16 14:05:07 +00:00
insertInteger(info_stack.top().column, info_stack.top().type, value);
2020-04-17 09:35:38 +00:00
return true;
}
2021-09-16 14:05:07 +00:00
bool MsgPackVisitor::visit_str(const char * value, size_t size) // NOLINT
2020-04-17 09:35:38 +00:00
{
2022-01-27 16:54:15 +00:00
insertString(info_stack.top().column, info_stack.top().type, value, size, false);
2020-04-17 09:35:38 +00:00
return true;
}
2021-09-17 14:32:23 +00:00
bool MsgPackVisitor::visit_bin(const char * value, size_t size) // NOLINT
2020-04-17 09:35:38 +00:00
{
2022-01-27 16:54:15 +00:00
insertString(info_stack.top().column, info_stack.top().type, value, size, true);
2021-09-16 14:05:07 +00:00
return true;
}
2021-09-16 11:25:45 +00:00
2021-09-16 14:05:07 +00:00
bool MsgPackVisitor::visit_float32(Float32 value) // NOLINT
{
insertFloat32(info_stack.top().column, info_stack.top().type, value);
2020-04-17 09:35:38 +00:00
return true;
}
2020-04-18 11:26:57 +00:00
bool MsgPackVisitor::visit_float64(Float64 value) // NOLINT
2020-04-17 09:35:38 +00:00
{
2021-09-16 14:05:07 +00:00
insertFloat64(info_stack.top().column, info_stack.top().type, value);
2020-04-17 09:35:38 +00:00
return true;
}
2021-09-17 14:32:23 +00:00
bool MsgPackVisitor::visit_boolean(bool value)
{
insertInteger(info_stack.top().column, info_stack.top().type, UInt64(value));
return true;
}
2020-04-18 11:26:57 +00:00
bool MsgPackVisitor::start_array(size_t size) // NOLINT
2020-04-17 09:35:38 +00:00
{
2021-09-16 11:25:45 +00:00
if (!isArray(info_stack.top().type))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack array into column with type {}.", info_stack.top().type->getName());
2020-04-17 09:35:38 +00:00
auto nested_type = assert_cast<const DataTypeArray &>(*info_stack.top().type).getNestedType();
ColumnArray & column_array = assert_cast<ColumnArray &>(info_stack.top().column);
ColumnArray::Offsets & offsets = column_array.getOffsets();
IColumn & nested_column = column_array.getData();
offsets.push_back(offsets.back() + size);
info_stack.push(Info{nested_column, nested_type});
return true;
}
2020-04-18 11:26:57 +00:00
bool MsgPackVisitor::end_array() // NOLINT
2020-04-17 09:35:38 +00:00
{
info_stack.pop();
return true;
}
2021-09-16 11:25:45 +00:00
bool MsgPackVisitor::start_map(uint32_t size) // NOLINT
{
if (!isMap(info_stack.top().type))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack map into column with type {}.", info_stack.top().type->getName());
ColumnArray & column_array = assert_cast<ColumnMap &>(info_stack.top().column).getNestedColumn();
ColumnArray::Offsets & offsets = column_array.getOffsets();
offsets.push_back(offsets.back() + size);
return true;
}
bool MsgPackVisitor::start_map_key() // NOLINT
{
auto key_column = assert_cast<ColumnMap &>(info_stack.top().column).getNestedData().getColumns()[0];
2021-09-16 14:05:07 +00:00
auto key_type = assert_cast<const DataTypeMap &>(*info_stack.top().type).getKeyType();
2021-09-16 11:25:45 +00:00
info_stack.push(Info{*key_column, key_type});
return true;
}
bool MsgPackVisitor::end_map_key() // NOLINT
{
info_stack.pop();
return true;
}
bool MsgPackVisitor::start_map_value() // NOLINT
{
auto value_column = assert_cast<ColumnMap &>(info_stack.top().column).getNestedData().getColumns()[1];
2021-09-16 14:05:07 +00:00
auto value_type = assert_cast<const DataTypeMap &>(*info_stack.top().type).getValueType();
2021-09-16 11:25:45 +00:00
info_stack.push(Info{*value_column, value_type});
return true;
}
bool MsgPackVisitor::end_map_value() // NOLINT
{
info_stack.pop();
return true;
}
bool MsgPackVisitor::visit_nil()
{
2021-09-16 14:05:07 +00:00
insertNull(info_stack.top().column, info_stack.top().type);
return true;
2021-09-16 11:25:45 +00:00
}
2022-01-27 16:54:15 +00:00
bool MsgPackVisitor::visit_ext(const char * value, uint32_t size)
{
2022-02-07 14:07:44 +00:00
int8_t type = *value;
if (*value == int8_t(MsgPackExtensionTypes::UUIDType))
2022-01-27 16:54:15 +00:00
{
insertUUID(info_stack.top().column, info_stack.top().type, value + 1, size - 1);
return true;
}
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported MsgPack extension type: {:x}", type);
2022-01-27 16:54:15 +00:00
}
2020-04-18 11:26:57 +00:00
void MsgPackVisitor::parse_error(size_t, size_t) // NOLINT
2020-04-17 09:35:38 +00:00
{
throw Exception("Error occurred while parsing msgpack data.", ErrorCodes::INCORRECT_DATA);
}
bool MsgPackRowInputFormat::readObject()
{
if (buf->eof())
2020-04-17 09:35:38 +00:00
return false;
PeekableReadBufferCheckpoint checkpoint{*buf};
2020-04-17 09:35:38 +00:00
size_t offset = 0;
while (!parser.execute(buf->position(), buf->available(), offset))
2020-04-17 09:35:38 +00:00
{
buf->position() = buf->buffer().end();
if (buf->eof())
2020-04-17 09:35:38 +00:00
throw Exception("Unexpected end of file while parsing msgpack object.", ErrorCodes::INCORRECT_DATA);
buf->position() = buf->buffer().end();
buf->makeContinuousMemoryFromCheckpointToPos();
buf->rollbackToCheckpoint();
2020-04-17 09:35:38 +00:00
}
buf->position() += offset;
2020-04-17 09:35:38 +00:00
return true;
}
bool MsgPackRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &)
{
size_t column_index = 0;
bool has_more_data = true;
for (; column_index != columns.size(); ++column_index)
{
2020-04-17 09:35:38 +00:00
visitor.set_info(*columns[column_index], data_types[column_index]);
has_more_data = readObject();
if (!has_more_data)
break;
}
if (!has_more_data)
{
if (column_index != 0)
throw Exception("Not enough values to complete the row.", ErrorCodes::INCORRECT_DATA);
return false;
}
return true;
}
void MsgPackRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf = std::make_unique<PeekableReadBuffer>(in_);
IInputFormat::setReadBuffer(in_);
}
2021-12-28 06:40:36 +00:00
MsgPackSchemaReader::MsgPackSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
: IRowSchemaReader(buf, format_settings_), buf(in_), number_of_columns(format_settings_.msgpack.number_of_columns)
{
if (!number_of_columns)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "You must specify setting input_format_msgpack_number_of_columns to extract table schema from MsgPack data");
}
msgpack::object_handle MsgPackSchemaReader::readObject()
{
if (buf.eof())
throw Exception(ErrorCodes::UNEXPECTED_END_OF_FILE, "Unexpected eof while parsing msgpack object");
PeekableReadBufferCheckpoint checkpoint{buf};
size_t offset = 0;
bool need_more_data = true;
msgpack::object_handle object_handle;
while (need_more_data)
{
offset = 0;
try
{
object_handle = msgpack::unpack(buf.position(), buf.buffer().end() - buf.position(), offset);
need_more_data = false;
}
catch (msgpack::insufficient_bytes &)
{
buf.position() = buf.buffer().end();
if (buf.eof())
throw Exception("Unexpected end of file while parsing msgpack object", ErrorCodes::UNEXPECTED_END_OF_FILE);
buf.position() = buf.buffer().end();
buf.makeContinuousMemoryFromCheckpointToPos();
buf.rollbackToCheckpoint();
}
}
buf.position() += offset;
return object_handle;
}
DataTypePtr MsgPackSchemaReader::getDataType(const msgpack::object & object)
{
switch (object.type)
{
2021-12-17 15:34:13 +00:00
case msgpack::type::object_type::POSITIVE_INTEGER: [[fallthrough]];
case msgpack::type::object_type::NEGATIVE_INTEGER:
return makeNullable(std::make_shared<DataTypeInt64>());
case msgpack::type::object_type::FLOAT32:
return makeNullable(std::make_shared<DataTypeFloat32>());
case msgpack::type::object_type::FLOAT64:
return makeNullable(std::make_shared<DataTypeFloat64>());
case msgpack::type::object_type::BOOLEAN:
return makeNullable(std::make_shared<DataTypeUInt8>());
case msgpack::type::object_type::BIN: [[fallthrough]];
case msgpack::type::object_type::STR:
return makeNullable(std::make_shared<DataTypeString>());
case msgpack::type::object_type::ARRAY:
{
msgpack::object_array object_array = object.via.array;
if (object_array.size)
{
auto nested_type = getDataType(object_array.ptr[0]);
if (nested_type)
return std::make_shared<DataTypeArray>(getDataType(object_array.ptr[0]));
}
return nullptr;
}
case msgpack::type::object_type::MAP:
{
msgpack::object_map object_map = object.via.map;
if (object_map.size)
{
2021-12-17 15:34:13 +00:00
auto key_type = removeNullable(getDataType(object_map.ptr[0].key));
auto value_type = getDataType(object_map.ptr[0].val);
if (key_type && value_type)
return std::make_shared<DataTypeMap>(key_type, value_type);
}
return nullptr;
}
case msgpack::type::object_type::NIL:
return nullptr;
2022-01-27 16:54:15 +00:00
case msgpack::type::object_type::EXT:
{
msgpack::object_ext object_ext = object.via.ext;
if (object_ext.type() == int8_t(MsgPackExtensionTypes::UUIDType))
2022-01-27 16:54:15 +00:00
return std::make_shared<DataTypeUUID>();
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Msgpack extension type {:x} is not supported", object_ext.type());
2022-01-27 16:54:15 +00:00
}
}
UNREACHABLE();
}
DataTypes MsgPackSchemaReader::readRowAndGetDataTypes()
{
if (buf.eof())
return {};
DataTypes data_types;
data_types.reserve(number_of_columns);
for (size_t i = 0; i != number_of_columns; ++i)
{
auto object_handle = readObject();
data_types.push_back(getDataType(object_handle.get()));
}
return data_types;
}
2021-10-11 16:11:50 +00:00
void registerInputFormatMsgPack(FormatFactory & factory)
2020-03-26 21:11:33 +00:00
{
2021-10-11 16:11:50 +00:00
factory.registerInputFormat("MsgPack", [](
2020-04-03 20:44:13 +00:00
ReadBuffer & buf,
const Block & sample,
const RowInputFormatParams & params,
2020-03-26 21:11:33 +00:00
const FormatSettings &)
{
return std::make_shared<MsgPackRowInputFormat>(sample, buf, params);
});
factory.registerFileExtension("messagepack", "MsgPack");
}
void registerMsgPackSchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader("MsgPack", [](ReadBuffer & buf, const FormatSettings & settings)
{
return std::make_shared<MsgPackSchemaReader>(buf, settings);
});
factory.registerAdditionalInfoForSchemaCacheGetter("MsgPack", [](const FormatSettings & settings)
{
String result = getAdditionalFormatInfoForAllRowBasedFormats(settings);
return result + fmt::format(", number_of_columns={}", settings.msgpack.number_of_columns);
});
}
2020-03-26 21:11:33 +00:00
}
2020-07-10 19:08:18 +00:00
Suppress clang-analyzer-cplusplus.NewDelete in MsgPackRowInputFormat Appartently there is some issue with clang-15, since even the following example shows error [1]. [1]: https://gist.github.com/azat/027f0e949ea836fc2e6269113ceb8752 clang-tidy report [1]: FAILED: src/CMakeFiles/dbms.dir/Processors/Formats/Impl/MsgPackRowInputFormat.cpp.o /usr/bin/cmake -E __run_co_compile --launcher="prlimit;--as=10000000000;--data=5000000000;--cpu=1000;/usr/bin/ccache" --tidy=/usr/bin/clang-tidy-15 --source=/ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp -- /usr/bin/clang++-15 --target=x86_64-linux-gnu --sysroot=/ch/cmake/linux/../../contrib/sysroot/linux-x86_64/x86_64-linux-gnu/libc -DAWS_SDK_VERSION_MAJOR=1 -DAWS_SDK_VERSION_MINOR=7 -DAWS_SDK_VERSION_PATCH=231 -DBOOST_ASIO_HAS_STD_INVOKE_RESULT=1 -DBOOST_ASIO_STANDALONE=1 -DCARES_STATICLIB -DCONFIGDIR=\"\" -DENABLE_MULTITARGET_CODE=1 -DENABLE_OPENSSL_ENCRYPTION -DHAS_RESERVED_IDENTIFIER -DHAVE_CONFIG_H -DLIBSASL_EXPORTS=1 -DLZ4_DISABLE_DEPRECATE_WARNINGS=1 -DOBSOLETE_CRAM_ATTR=1 -DOBSOLETE_DIGEST_ATTR=1 -DPLUGINDIR=\"\" -DPOCO_ENABLE_CPP11 -DPOCO_HAVE_FD_EPOLL -DPOCO_OS_FAMILY_UNIX -DSASLAUTHD_CONF_FILE_DEFAULT=\"\" -DSNAPPY_CODEC_AVAILABLE -DSTD_EXCEPTION_HAS_STACK_TRACE=1 -DUNALIGNED_OK -DWITH_COVERAGE=0 -DWITH_GZFILEOP -DX86_64 -DZLIB_COMPAT -D_LIBCPP_ENABLE_THREAD_SAFETY_ANNOTATIONS -Iincludes/configs -I/ch/src -Isrc -Isrc/Core/include -I/ch/base/glibc-compatibility/memcpy -I/ch/base/base/.. -Ibase/base/.. -I/ch/contrib/cctz/include -I/ch/base/pcg-random/. -I/ch/contrib/miniselect/include -I/ch/contrib/zstd/lib -Icontrib/cyrus-sasl-cmake -I/ch/contrib/lz4/lib -I/ch/src/Common/mysqlxx/. -Icontrib/c-ares -I/ch/contrib/c-ares -I/ch/contrib/c-ares/include -isystem /ch/contrib/libcxx/include -isystem /ch/contrib/libcxxabi/include -isystem /ch/contrib/libunwind/include -isystem /ch/contrib/libdivide/. -isystem /ch/contrib/jemalloc-cmake/include -isystem /ch/contrib/llvm/llvm/include -isystem contrib/llvm/llvm/include -isystem /ch/contrib/abseil-cpp -isystem /ch/contrib/croaring/cpp -isystem /ch/contrib/croaring/include -isystem /ch/contrib/cityhash102/include -isystem /ch/contrib/boost -isystem /ch/contrib/poco/Net/include -isystem /ch/contrib/poco/Foundation/include -isystem /ch/contrib/poco/NetSSL_OpenSSL/include -isystem /ch/contrib/poco/Crypto/include -isystem /ch/contrib/boringssl/include -isystem /ch/contrib/poco/Util/include -isystem /ch/contrib/poco/JSON/include -isystem /ch/contrib/poco/XML/include -isystem /ch/contrib/replxx/include -isystem /ch/contrib/fmtlib-cmake/../fmtlib/include -isystem /ch/contrib/magic_enum/include -isystem /ch/contrib/double-conversion -isystem /ch/contrib/dragonbox/include -isystem /ch/contrib/re2 -isystem contrib/re2-cmake -isystem /ch/contrib/zlib-ng -isystem contrib/zlib-ng-cmake -isystem /ch/contrib/pdqsort -isystem /ch/contrib/xz/src/liblzma/api -isystem /ch/contrib/aws-c-common/include -isystem /ch/contrib/aws-c-event-stream/include -isystem /ch/contrib/aws/aws-cpp-sdk-s3/include -isystem /ch/contrib/aws/aws-cpp-sdk-core/include -isystem contrib/aws-s3-cmake/include -isystem /ch/contrib/snappy -isystem contrib/snappy-cmake -isystem /ch/contrib/msgpack-c/include -isystem /ch/contrib/fast_float/include -isystem /ch/contrib/librdkafka-cmake/include -isystem /ch/contrib/librdkafka/src -isystem contrib/librdkafka-cmake/auxdir -isystem /ch/contrib/cppkafka/include -isystem /ch/contrib/nats-io/src -isystem /ch/contrib/nats-io/src/adapters -isystem /ch/contrib/nats-io/src/include -isystem /ch/contrib/nats-io/src/unix -isystem /ch/contrib/libuv/include -isystem /ch/contrib/krb5/src/include -isystem contrib/krb5-cmake/include -isystem /ch/contrib/NuRaft/include -isystem /ch/contrib/poco/MongoDB/include -isystem contrib/mariadb-connector-c-cmake/include-public -isystem /ch/contrib/mariadb-connector-c/include -isystem /ch/contrib/mariadb-connector-c/libmariadb -isystem /ch/contrib/icu/icu4c/source/i18n -isystem /ch/contrib/icu/icu4c/source/common -isystem /ch/contrib/capnproto/c++/src -isystem /ch/contrib/arrow/cpp/src -isystem /ch/contrib/arrow-cmake/cpp/src -isystem contrib/arrow-cmake/cpp/src -isystem contrib/arrow-cmake/../orc/c++/include -isystem /ch/contrib/orc/c++/include -isystem contrib/avro-cmake/include -isystem /ch/contrib/avro/lang/c++/api -isystem /ch/contrib/openldap-cmake/linux_x86_64/include -isystem /ch/contrib/openldap/include -isystem /ch/contrib/sparsehash-c11 -isystem /ch/contrib/protobuf/src -isystem src/Server/grpc_protos -isystem /ch/contrib/grpc/include -isystem /ch/contrib/libhdfs3/include -isystem /ch/contrib/hive-metastore -isystem /ch/contrib/thrift/lib/cpp/src -isystem contrib/thrift-cmake -isystem /ch/contrib/azure/sdk/core/azure-core/inc-isystem /ch/contrib/azure/sdk/identity/azure-identity/inc -isystem /ch/contrib/azure/sdk/storage/azure-storage-common/inc -isystem /ch/contrib/azure/sdk/storage/azure-storage-blobs/inc -isystem /ch/contrib/s2geometry/src -isystem /ch/contrib/AMQP-CPP/include -isystem /ch/contrib/AMQP-CPP -isystem /ch/contrib/sqlite-amalgamation -isystem /ch/contrib/rocksdb/include -isystem /ch/contrib/libpqxx/include -isystem /ch/contrib/libpq -isystem /ch/contrib/libpq/include -isystem /ch/contrib/libstemmer_c/include -isystem /ch/contrib/wordnet-blast -isystem /ch/contrib/lemmagen-c/include -isystem /ch/contrib/simdjson/include -isystem /ch/contrib/rapidjson/include -isystem /ch/contrib/consistent-hashing --gcc-toolchain=/ch/cmake/linux/../../contrib/sysroot/linux-x86_64 -std=c++20 -fdiagnostics-color=always -Xclang -fuse-ctor-homing -fsized-deallocation -UNDEBUG -gdwarf-aranges -pipe -mssse3 -msse4.1 -msse4.2 -mpclmul -mpopcnt -fasynchronous-unwind-tables -falign-functions=32 -mbranches-within-32B-boundaries -fdiagnostics-absolute-paths -fstrict-vtable-pointers -fexperimental-new-pass-manager -Wall -Wextra -Weverything -Wpedantic -Wno-zero -length-array -Wno-c++98-compat-pedantic -Wno-c++98-compat -Wno-c++20-compat -Wno-conversion -Wno-ctad-maybe-unsupported -Wno-disabled-macro-expansion -Wno-documentation-unknown-command -Wno-double-promotion -Wno-exit-time-destructors -Wno-float-equal -Wno-global-constructors -Wno-missing-prototypes -Wno-missing-variable-declarations -Wno-padded -Wno-switch-enum -Wno-undefined-func-template -Wno-unused-template -Wno-vla -Wno-weak-template-vtables -Wno-weak-vtables -Wno-thread-safety-negative -g -O0 -g -gdwarf-4 -fno-inline -D_LIBCPP_DEBUG=0 -D OS_LINUX -I/ch/base -I/ch/contrib/magic_enum/include -include /ch/src/Core/iostream_debug_helpers.h -Werror -nostdinc++ -std=gnu++2a -MD -MT src/CMakeFiles/dbms.dir/Processors/Formats/Impl/MsgPackRowInputFormat.cpp.o -MF src/CMakeFiles/dbms.dir/Processors/Formats/Impl/MsgPackRowInputFormat.cpp.o.d -o src/CMakeFiles/dbms.dir/Processors/Formats/Impl/MsgPackRowInputFormat.cpp.o -c /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp /ch/contrib/msgpack-c/include/msgpack/v1/detail/cpp11_zone.hpp:195:9: error: Attempt to free released memory [clang-analyzer-cplusplus.NewDelete,-warnings-as-errors] ::free(p); ^ /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp:509:5: note: Taking false branch if (buf.eof()) ^ /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp:514:24: note: Assuming 'i' is not equal to field 'number_of_columns' for (size_t i = 0; i != number_of_columns; ++i) ^~~~~~~~~~~~~~~~~~~~~~ /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp:514:5: note: Loop condition is true. Entering loop body for (size_t i = 0; i != number_of_columns; ++i) ^ /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp:516:30: note: Calling 'MsgPackSchemaReader::readObject' auto object_handle = readObject(); ^~~~~~~~~~~~ /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp:426:5: note: Taking false branch if (buf.eof()) ^ /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp:433:5: note: Loop condition is true. Entering loop body while (need_more_data) ^ /ch/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp:438:29: note: Calling 'unpack' object_handle = msgpack::unpack(buf.position(), buf.buffer().end() - buf.position(), offset); ^~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ /ch/contrib/msgpack-c/include/msgpack/v3/unpack.hpp:52:12: note: Calling 'unpack' return msgpack::v3::unpack(data, len, off, referenced, f, user_data, limit); ^~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ /ch/contrib/msgpack-c/include/msgpack/v3/unpack.hpp:35:5: note: Control jumps to the 'default' case at line 40 switch(ret) { ^ /ch/contrib/msgpack-c/include/msgpack/v3/unpack.hpp:41:9: note: Execution continues on line 43 break; ^ /ch/contrib/msgpack-c/include/msgpack/v3/unpack.hpp:43:35: note: Calling '~unique_ptr' return msgpack::object_handle(); ^ /ch/contrib/libcxx/include/__memory/unique_ptr.h:269:19: note: Calling 'unique_ptr::reset' ~unique_ptr() { reset(); } ^~~~~~~ /ch/contrib/libcxx/include/__memory/unique_ptr.h:314:9: note: '__tmp' is non-null if (__tmp) ^~~~~ /ch/contrib/libcxx/include/__memory/unique_ptr.h:314:5: note: Taking true branch if (__tmp) ^ /ch/contrib/libcxx/include/__memory/unique_ptr.h:315:7: note: Calling 'default_delete::operator()' __ptr_.second()(__tmp); ^~~~~~~~~~~~~~~~~~~~~~ /ch/contrib/libcxx/include/__memory/unique_ptr.h:54:5: note: Memory is released delete __ptr; ^~~~~~~~~~~~ /ch/contrib/libcxx/include/__memory/unique_ptr.h:54:5: note: Calling 'zone::operator delete' delete __ptr; ^~~~~~~~~~~~ /ch/contrib/msgpack-c/include/msgpack/v1/detail/cpp11_zone.hpp:195:9: note: Attempt to free released memory ::free(p); ^~~~~~~~~ [1]: https://s3.amazonaws.com/clickhouse-builds/41046/9677898b3b234a5ba0371edaf719ea8890d084ff/binary_tidy/build_log.log Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-09-10 11:11:54 +00:00
// NOLINTEND(clang-analyzer-cplusplus.NewDelete)
2020-07-10 19:27:12 +00:00
#else
namespace DB
{
class FormatFactory;
2021-10-11 16:11:50 +00:00
void registerInputFormatMsgPack(FormatFactory &)
2020-07-10 19:27:12 +00:00
{
}
void registerMsgPackSchemaReader(FormatFactory &)
{
}
2020-07-10 19:27:12 +00:00
}
2020-07-10 19:08:18 +00:00
#endif