diff --git a/dbms/src/Core/AccurateComparison.h b/dbms/src/Core/AccurateComparison.h index af1ea285a89..e26269b136d 100644 --- a/dbms/src/Core/AccurateComparison.h +++ b/dbms/src/Core/AccurateComparison.h @@ -430,6 +430,13 @@ inline bool_if_safe_conversion greaterOrEqualsOp(A a, B b) template inline bool NO_SANITIZE_UNDEFINED convertNumeric(From value, To & result) { + /// If the type is actually the same it's not necessary to do any checks. + if constexpr (std::is_same_v) + { + result = value; + return true; + } + /// Note that NaNs doesn't compare equal to anything, but they are still in range of any Float type. if (isNaN(value) && std::is_floating_point_v) { diff --git a/dbms/src/Functions/DummyJSONParser.h b/dbms/src/Functions/DummyJSONParser.h new file mode 100644 index 00000000000..3b66d4607dc --- /dev/null +++ b/dbms/src/Functions/DummyJSONParser.h @@ -0,0 +1,46 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +/// This class can be used as an argument for the template class FunctionJSON when we unable to parse JSONs. +/// It can't do anything useful and just throws an exception. +struct DummyJSONParser +{ + static constexpr bool need_preallocate = false; + void preallocate(size_t) {} + bool parse(const char *, size_t) { throw Exception{"Functions JSON* are not supported without AVX2", ErrorCodes::NOT_IMPLEMENTED}; } + + using Iterator = std::nullptr_t; + Iterator getRoot() const { return nullptr; } + + static bool downToArray(Iterator &) { return false; } + static bool downToObject(Iterator &) { return false; } + static bool downToObject(Iterator &, StringRef &) { return false; } + static bool parentScopeIsObject(const Iterator &) { return false; } + static bool next(Iterator &) { return false; } + static bool nextKeyValue(Iterator &) { return false; } + static bool nextKeyValue(Iterator &, StringRef &) { return false; } + static bool isInteger(const Iterator &) { return false; } + static bool isFloat(const Iterator &) { return false; } + static bool isString(const Iterator &) { return false; } + static bool isArray(const Iterator &) { return false; } + static bool isObject(const Iterator &) { return false; } + static bool isBool(const Iterator &) { return false; } + static bool isNull(const Iterator &) { return false; } + static StringRef getKey(const Iterator &) { return {}; } + static StringRef getString(const Iterator &) { return {}; } + static Int64 getInteger(const Iterator &) { return 0; } + static double getFloat(const Iterator &) { return 0; } + static bool getBool(const Iterator &) { return false; } +}; + +} diff --git a/dbms/src/Functions/FunctionsJSON.cpp b/dbms/src/Functions/FunctionsJSON.cpp index b5ec5fd6c64..9383b58b05a 100644 --- a/dbms/src/Functions/FunctionsJSON.cpp +++ b/dbms/src/Functions/FunctionsJSON.cpp @@ -1,459 +1,7 @@ #include -#include -#include +#include +#include -#if USE_SIMDJSON -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -template -class JSONNullableImplBase -{ -public: - static DataTypePtr getType() { return std::make_shared(std::make_shared()); } - - static Field getDefault() { return {}; } -}; - -class JSONHasImpl : public JSONNullableImplBase -{ -public: - static constexpr auto name{"JSONHas"}; - - static Field getValue(ParsedJson::iterator &) { return {1}; } -}; - -class JSONLengthImpl : public JSONNullableImplBase -{ -public: - static constexpr auto name{"JSONLength"}; - - static Field getValue(ParsedJson::iterator & pjh) - { - if (!pjh.is_object_or_array()) - return getDefault(); - - size_t size = 0; - - if (pjh.down()) - { - ++size; - while (pjh.next()) - ++size; - if (pjh.get_scope_type() == '{') - size /= 2; - } - - return {size}; - } -}; - -class JSONTypeImpl -{ -public: - static constexpr auto name{"JSONType"}; - - static DataTypePtr getType() - { - static const std::vector> values = { - {"Array", '['}, - {"Object", '{'}, - {"String", '"'}, - {"Int", 'l'}, - {"Float",'d'}, - {"Bool", 'b'}, - {"Null",'n'}, - }; - return std::make_shared(std::make_shared>(values)); - } - - static Field getDefault() { return {}; } - - static Field getValue(ParsedJson::iterator & pjh) - { - switch (pjh.get_type()) - { - case '[': - case '{': - case '"': - case 'l': - case 'd': - case 'n': - return {pjh.get_type()}; - case 't': - case 'f': - return {'b'}; - default: - return {}; - } - } -}; - -class JSONExtractImpl -{ -public: - static constexpr auto name{"JSONExtract"}; - - static DataTypePtr getType(const DataTypePtr & type) - { - WhichDataType which{type}; - - if (which.isNativeUInt() || which.isNativeInt() || which.isFloat() || which.isEnum() || which.isDateOrDateTime() - || which.isStringOrFixedString() || which.isInterval()) - return std::make_shared(type); - - if (which.isArray()) - { - auto array_type = static_cast(type.get()); - - return std::make_shared(getType(array_type->getNestedType())); - } - - if (which.isTuple()) - { - auto tuple_type = static_cast(type.get()); - - DataTypes types; - types.reserve(tuple_type->getElements().size()); - - for (const DataTypePtr & element : tuple_type->getElements()) - { - types.push_back(getType(element)); - } - - return std::make_shared(std::move(types)); - } - - throw Exception{"Unsupported return type schema: " + type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - } - - static Field getDefault(const DataTypePtr & type) - { - WhichDataType which{type}; - - if (which.isNativeUInt() || which.isNativeInt() || which.isFloat() || which.isEnum() || which.isDateOrDateTime() - || which.isStringOrFixedString() || which.isInterval()) - return {}; - - if (which.isArray()) - return {Array{}}; - - if (which.isTuple()) - { - auto tuple_type = static_cast(type.get()); - - Tuple tuple; - tuple.toUnderType().reserve(tuple_type->getElements().size()); - - for (const DataTypePtr & element : tuple_type->getElements()) - tuple.toUnderType().push_back(getDefault(element)); - - return {tuple}; - } - - // should not reach - throw Exception{"Unsupported return type schema: " + type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - } - - static Field getValue(ParsedJson::iterator & pjh, const DataTypePtr & type) - { - WhichDataType which{type}; - - if (which.isNativeUInt() || which.isNativeInt() || which.isEnum() || which.isDateOrDateTime() || which.isInterval()) - { - if (pjh.is_integer()) - return {pjh.get_integer()}; - else - return getDefault(type); - } - - if (which.isFloat()) - { - if (pjh.is_integer()) - return {static_cast(pjh.get_integer())}; - else if (pjh.is_double()) - return {pjh.get_double()}; - else - return getDefault(type); - } - - if (which.isStringOrFixedString()) - { - if (pjh.is_string()) - return {String{pjh.get_string()}}; - else - return getDefault(type); - } - - if (which.isArray()) - { - if (!pjh.is_object_or_array()) - return getDefault(type); - - auto array_type = static_cast(type.get()); - - Array array; - - bool first = true; - - while (first ? pjh.down() : pjh.next()) - { - first = false; - - ParsedJson::iterator pjh1{pjh}; - - array.push_back(getValue(pjh1, array_type->getNestedType())); - } - - return {array}; - } - - if (which.isTuple()) - { - if (!pjh.is_object_or_array()) - return getDefault(type); - - auto tuple_type = static_cast(type.get()); - - Tuple tuple; - tuple.toUnderType().reserve(tuple_type->getElements().size()); - - bool valid = true; - bool first = true; - - for (const DataTypePtr & element : tuple_type->getElements()) - { - if (valid) - { - valid &= first ? pjh.down() : pjh.next(); - first = false; - - ParsedJson::iterator pjh1{pjh}; - - tuple.toUnderType().push_back(getValue(pjh1, element)); - } - else - tuple.toUnderType().push_back(getDefault(element)); - } - - return {tuple}; - } - - // should not reach - throw Exception{"Unsupported return type schema: " + type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - } -}; - -class JSONExtractUIntImpl : public JSONNullableImplBase -{ -public: - static constexpr auto name{"JSONExtractUInt"}; - - static Field getValue(ParsedJson::iterator & pjh) - { - if (pjh.is_integer()) - return {pjh.get_integer()}; - else - return getDefault(); - } -}; - -class JSONExtractIntImpl : public JSONNullableImplBase -{ -public: - static constexpr auto name{"JSONExtractInt"}; - - static Field getValue(ParsedJson::iterator & pjh) - { - if (pjh.is_integer()) - return {pjh.get_integer()}; - else - return getDefault(); - } -}; - -class JSONExtractFloatImpl : public JSONNullableImplBase -{ -public: - static constexpr auto name{"JSONExtractFloat"}; - - static Field getValue(ParsedJson::iterator & pjh) - { - if (pjh.is_double()) - return {pjh.get_double()}; - else - return getDefault(); - } -}; - -class JSONExtractBoolImpl : public JSONNullableImplBase -{ -public: - static constexpr auto name{"JSONExtractBool"}; - - static Field getValue(ParsedJson::iterator & pjh) - { - if (pjh.get_type() == 't') - return {1}; - else if (pjh.get_type() == 'f') - return {0}; - else - return getDefault(); - } -}; - -class JSONExtractRawImpl: public JSONNullableImplBase -{ -public: - static constexpr auto name {"JSONExtractRaw"}; - - static Field getValue(ParsedJson::iterator & pjh) - { - WriteBufferFromOwnString buf; - traverse(pjh, buf); - return {std::move(buf.str())}; - } - -private: - static void traverse(ParsedJson::iterator & pjh, WriteBuffer & buf) - { - switch (pjh.get_type()) - { - case '{': - { - writeChar('{', buf); - if (pjh.down()) - { - writeJSONString(pjh.get_string(), pjh.get_string() + pjh.get_string_length(), buf, format_settings()); - writeChar(':', buf); - pjh.next(); - traverse(pjh, buf); - while (pjh.next()) - { - writeChar(',', buf); - writeJSONString(pjh.get_string(), pjh.get_string() + pjh.get_string_length(), buf, format_settings()); - writeChar(':', buf); - pjh.next(); - traverse(pjh, buf); - } - pjh.up(); - } - writeChar('}', buf); - break; - } - case '[': - { - writeChar('[', buf); - if (pjh.down()) - { - traverse(pjh, buf); - while (pjh.next()) - { - writeChar(',', buf); - traverse(pjh, buf); - } - pjh.up(); - } - writeChar(']', buf); - break; - } - case '"': - { - writeJSONString(pjh.get_string(), pjh.get_string() + pjh.get_string_length(), buf, format_settings()); - break; - } - case 'l': - { - writeIntText(pjh.get_integer(), buf); - break; - } - case 'd': - { - writeFloatText(pjh.get_double(), buf); - break; - } - case 't': - { - writeCString("true", buf); - break; - } - case 'f': - { - writeCString("false", buf); - break; - } - case 'n': - { - writeCString("null", buf); - break; - } - } - } - - static const FormatSettings & format_settings() - { - static const FormatSettings the_instance = [] - { - FormatSettings settings; - settings.json.escape_forward_slashes = false; - return settings; - }(); - return the_instance; - } -}; - -class JSONExtractStringImpl : public JSONNullableImplBase -{ -public: - static constexpr auto name{"JSONExtractString"}; - - static Field getValue(ParsedJson::iterator & pjh) - { - if (pjh.is_string()) - return {String{pjh.get_string()}}; - else - return getDefault(); - } -}; - -class JSONExtractKeyImpl : public JSONNullableImplBase -{ -public: - static constexpr auto name{"JSONExtractKey"}; - - static Field getValue(ParsedJson::iterator & pjh) - { - if (pjh.get_scope_type() == '{' && pjh.prev() && pjh.is_string()) - return {String{pjh.get_string()}}; - else - return getDefault(); - } -}; - -} -#else -namespace DB -{ -struct JSONHasImpl { static constexpr auto name{"JSONHas"}; }; -struct JSONLengthImpl { static constexpr auto name{"JSONLength"}; }; -struct JSONTypeImpl { static constexpr auto name{"JSONType"}; }; -struct JSONExtractImpl { static constexpr auto name{"JSONExtract"}; }; -struct JSONExtractUIntImpl { static constexpr auto name{"JSONExtractUInt"}; }; -struct JSONExtractIntImpl { static constexpr auto name{"JSONExtractInt"}; }; -struct JSONExtractFloatImpl { static constexpr auto name{"JSONExtractFloat"}; }; -struct JSONExtractBoolImpl { static constexpr auto name{"JSONExtractBool"}; }; -struct JSONExtractRawImpl { static constexpr auto name {"JSONExtractRaw"}; }; -struct JSONExtractStringImpl { static constexpr auto name{"JSONExtractString"}; }; -struct JSONExtractKeyImpl { static constexpr auto name{"JSONExtractKey"}; }; -} -#endif namespace DB { @@ -463,31 +11,11 @@ void registerFunctionsJSON(FunctionFactory & factory) #if USE_SIMDJSON if (__builtin_cpu_supports("avx2")) { - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); + registerFunctionsJSONTemplate(factory); return; } #endif - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); + registerFunctionsJSONTemplate(factory); } } diff --git a/dbms/src/Functions/FunctionsJSON.h b/dbms/src/Functions/FunctionsJSON.h index fb13c990e27..bcab236bcdd 100644 --- a/dbms/src/Functions/FunctionsJSON.h +++ b/dbms/src/Functions/FunctionsJSON.h @@ -1,168 +1,68 @@ #pragma once #include -#include - -#if USE_SIMDJSON - +#include +#include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include -#include +#include +#include +#include #include -#ifdef __clang__ - #pragma clang diagnostic push - #pragma clang diagnostic ignored "-Wold-style-cast" - #pragma clang diagnostic ignored "-Wnewline-eof" -#endif - -#include - -#ifdef __clang__ - #pragma clang diagnostic pop -#endif namespace DB { namespace ErrorCodes { - extern const int CANNOT_ALLOCATE_MEMORY; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -template -class FunctionJSONBase : public IFunction + +/// Functions to parse JSONs and extract values from it. +/// The first argument of all these functions gets a JSON, +/// after that there are any number of arguments specifying path to a desired part from the JSON's root. +/// For example, +/// select JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', b, 1) = -100 +template typename Impl, typename JSONParser> +class FunctionJSON : public IFunction { -private: - enum class Action - { - key = 1, - index = 2, - }; - - mutable std::vector actions; - mutable DataTypePtr virtual_type; - - bool tryMove(ParsedJson::iterator & pjh, Action action, const Field & accessor) - { - switch (action) - { - case Action::key: - if (!pjh.is_object() || !pjh.move_to_key(accessor.get().data())) - return false; - - break; - case Action::index: - if (!pjh.is_object_or_array() || !pjh.down()) - return false; - - int index = accessor.get(); - size_t steps; - if (index > 0) - { - if (pjh.get_scope_type() == '{') - steps = index * 2 - 1; - else - steps = index - 1; - } - else if (index < 0) - { - size_t steps_to_end = 0; - ParsedJson::iterator pjh1{pjh}; - while (pjh1.next()) - ++steps_to_end; - - if (pjh.get_scope_type() == '{') - steps = index * 2 + steps_to_end + 2; - else - steps = index + steps_to_end + 1; - } - else - return false; - - for (const auto i : ext::range(0, steps)) - { - (void)i; - - if (!pjh.next()) - return false; - } - - break; - } - - return true; - } - public: - static constexpr auto name = Impl::name; - - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return Impl::name; } - + static constexpr auto name = Name::name; + static FunctionPtr create(const Context &) { return std::make_shared(); } + String getName() const override { return Name::name; } bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - - bool useDefaultImplementationForConstants() const override { return true; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - if constexpr (ExtraArg) - { - if (arguments.size() < 2) - throw Exception{"Function " + getName() + " requires at least two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - auto col_type_const = typeid_cast(arguments[arguments.size() - 1].column.get()); - - if (!col_type_const) - throw Exception{"Illegal non-const column " + arguments[arguments.size() - 1].column->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN}; - - virtual_type = DataTypeFactory::instance().get(col_type_const->getValue()); - } - else - { - if (arguments.size() < 1) - throw Exception{"Function " + getName() + " requires at least one arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - } - - if (!isString(arguments[0].type)) - throw Exception{"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - actions.reserve(arguments.size() - 1 - ExtraArg); - - for (const auto i : ext::range(1, arguments.size() - ExtraArg)) - { - if (isString(arguments[i].type)) - actions.push_back(Action::key); - else if (isInteger(arguments[i].type)) - actions.push_back(Action::index); - else - throw Exception{"Illegal type " + arguments[i].type->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - } - - if constexpr (ExtraArg) - return Impl::getType(virtual_type); - else - return Impl::getType(); - } + bool useDefaultImplementationForConstants() const override { return false; } + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return Impl::getType(arguments); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result_pos, size_t input_rows_count) override { MutableColumnPtr to{block.getByPosition(result_pos).type->createColumn()}; to->reserve(input_rows_count); - const ColumnPtr & arg_json = block.getByPosition(arguments[0]).column; + if (arguments.size() < 1) + throw Exception{"Function " + getName() + " requires at least one arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + const auto & first_column = block.getByPosition(arguments[0]); + if (!isString(first_column.type)) + throw Exception{"Illegal type " + first_column.type->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + const ColumnPtr & arg_json = first_column.column; auto col_json_const = typeid_cast(arg_json.get()); - auto col_json_string = typeid_cast(col_json_const ? col_json_const->getDataColumnPtr().get() : arg_json.get()); @@ -172,80 +72,888 @@ public: const ColumnString::Chars & chars = col_json_string->getChars(); const ColumnString::Offsets & offsets = col_json_string->getOffsets(); - size_t max_size = 1; + std::vector moves; + constexpr size_t num_extra_arguments = Impl::num_extra_arguments; + const size_t num_moves = arguments.size() - num_extra_arguments - 1; + moves.reserve(num_moves); + for (const auto i : ext::range(0, num_moves)) + { + const auto & column = block.getByPosition(arguments[1 + i]); + if (column.column->isColumnConst()) + { + const auto & column_const = static_cast(*column.column); + if (isString(column.type)) + moves.emplace_back(MoveType::ConstKey, column_const.getField().get()); + else if (isInteger(column.type)) + moves.emplace_back(MoveType::ConstIndex, column_const.getField().get()); + else + throw Exception{"Illegal type " + column.type->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } + else + { + if (isString(column.type)) + moves.emplace_back(MoveType::Key, ""); + else if (isInteger(column.type)) + moves.emplace_back(MoveType::Index, 0); + else + throw Exception{"Illegal type " + column.type->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } + } - for (const auto i : ext::range(0, input_rows_count)) - if (max_size < offsets[i] - offsets[i - 1] - 1) - max_size = offsets[i] - offsets[i - 1] - 1; + JSONParser parser; + if (parser.need_preallocate) + { + size_t max_size = 1; - ParsedJson pj; - if (!pj.allocateCapacity(max_size)) - throw Exception{"Can not allocate memory for " + std::to_string(max_size) + " units when parsing JSON", - ErrorCodes::CANNOT_ALLOCATE_MEMORY}; + for (const auto i : ext::range(0, input_rows_count)) + if (max_size < offsets[i] - offsets[i - 1] - 1) + max_size = offsets[i] - offsets[i - 1] - 1; + + parser.preallocate(max_size); + } + + Impl impl; + impl.prepare(block, arguments, result_pos); for (const auto i : ext::range(0, input_rows_count)) { - bool ok = json_parse(&chars[offsets[i - 1]], offsets[i] - offsets[i - 1] - 1, pj) == 0; + bool ok = parser.parse(reinterpret_cast(&chars[offsets[i - 1]]), offsets[i] - offsets[i - 1] - 1); - ParsedJson::iterator pjh{pj}; - - for (const auto j : ext::range(0, actions.size())) + auto it = parser.getRoot(); + for (const auto j : ext::range(0, moves.size())) { if (!ok) break; - ok = tryMove(pjh, actions[j], (*block.getByPosition(arguments[j + 1]).column)[i]); + switch (moves[j].type) + { + case MoveType::ConstIndex: + ok = moveIteratorToElementByIndex(it, moves[j].index); + break; + case MoveType::ConstKey: + ok = moveIteratorToElementByKey(it, moves[j].key); + break; + case MoveType::Index: + { + const Field field = (*block.getByPosition(arguments[j + 1]).column)[i]; + ok = moveIteratorToElementByIndex(it, field.get()); + break; + } + case MoveType::Key: + { + const Field field = (*block.getByPosition(arguments[j + 1]).column)[i]; + ok = moveIteratorToElementByKey(it, field.get().data()); + break; + } + } } if (ok) + ok = impl.addValueToColumn(*to, it); + + if (!ok) + to->insertDefault(); + } + block.getByPosition(result_pos).column = std::move(to); + } + +private: + enum class MoveType + { + Key, + Index, + ConstKey, + ConstIndex, + }; + struct Move + { + Move(MoveType type_, size_t index_ = 0) : type(type_), index(index_) {} + Move(MoveType type_, const String & key_) : type(type_), key(key_) {} + MoveType type; + size_t index = 0; + String key; + }; + + using Iterator = typename JSONParser::Iterator; + bool moveIteratorToElementByIndex(Iterator & it, int index) + { + if (JSONParser::isArray(it)) + { + if (!JSONParser::downToArray(it)) + return false; + size_t steps; + if (index > 0) { - if constexpr (ExtraArg) - to->insert(Impl::getValue(pjh, virtual_type)); - else - to->insert(Impl::getValue(pjh)); + steps = index - 1; } else { - if constexpr (ExtraArg) - to->insert(Impl::getDefault(virtual_type)); - else - to->insert(Impl::getDefault()); + size_t length = 1; + Iterator it2 = it; + while (JSONParser::next(it2)) + ++length; + steps = index + length; + } + while (steps--) + { + if (!JSONParser::next(it)) + return false; + } + return true; + } + if (JSONParser::isObject(it)) + { + if (!JSONParser::downToObject(it)) + return false; + size_t steps; + if (index > 0) + { + steps = index - 1; + } + else + { + size_t length = 1; + Iterator it2 = it; + while (JSONParser::nextKeyValue(it2)) + ++length; + steps = index + length; + } + while (steps--) + { + if (!JSONParser::nextKeyValue(it)) + return false; + } + return true; + } + return false; + } + + bool moveIteratorToElementByKey(Iterator & it, const String & key) + { + if (JSONParser::isObject(it)) + { + StringRef current_key; + if (!JSONParser::downToObject(it, current_key)) + return false; + do + { + if (current_key == key) + return true; + } while (JSONParser::nextKeyValue(it, current_key)); + } + return false; + } +}; + + +struct NameJSONHas { static constexpr auto name{"JSONHas"}; }; +struct NameJSONLength { static constexpr auto name{"JSONLength"}; }; +struct NameJSONKey { static constexpr auto name{"JSONKey"}; }; +struct NameJSONType { static constexpr auto name{"JSONType"}; }; +struct NameJSONExtractInt { static constexpr auto name{"JSONExtractInt"}; }; +struct NameJSONExtractUInt { static constexpr auto name{"JSONExtractUInt"}; }; +struct NameJSONExtractFloat { static constexpr auto name{"JSONExtractFloat"}; }; +struct NameJSONExtractBool { static constexpr auto name{"JSONExtractBool"}; }; +struct NameJSONExtractString { static constexpr auto name{"JSONExtractString"}; }; +struct NameJSONExtractRaw { static constexpr auto name{"JSONExtractRaw"}; }; +struct NameJSONExtract { static constexpr auto name{"JSONExtract"}; }; + + +template +class JSONHasImpl +{ +public: + static DataTypePtr getType(const ColumnsWithTypeAndName &) { return std::make_shared(); } + + using Iterator = typename JSONParser::Iterator; + static bool addValueToColumn(IColumn & dest, const Iterator &) + { + ColumnVector & col_vec = static_cast &>(dest); + col_vec.insertValue(1); + return true; + } + + static constexpr size_t num_extra_arguments = 0; + static void prepare(const Block &, const ColumnNumbers &, size_t) {} +}; + + +template +class JSONLengthImpl +{ +public: + static DataTypePtr getType(const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + using Iterator = typename JSONParser::Iterator; + static bool addValueToColumn(IColumn & dest, const Iterator & it) + { + size_t size; + if (JSONParser::isArray(it)) + { + size = 0; + Iterator it2 = it; + if (JSONParser::downToArray(it2)) + { + do + ++size; + while (JSONParser::next(it2)); + } + } + else if (JSONParser::isObject(it)) + { + size = 0; + Iterator it2 = it; + if (JSONParser::downToObject(it2)) + { + do + ++size; + while (JSONParser::nextKeyValue(it2)); + } + } + else + return false; + + ColumnVector & col_vec = static_cast &>(dest); + col_vec.insertValue(size); + return true; + } + + static constexpr size_t num_extra_arguments = 0; + static void prepare(const Block &, const ColumnNumbers &, size_t) {} +}; + + +template +class JSONKeyImpl +{ +public: + static DataTypePtr getType(const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + using Iterator = typename JSONParser::Iterator; + static bool addValueToColumn(IColumn & dest, const Iterator & it) + { + if (!JSONParser::parentScopeIsObject(it)) + return false; + StringRef key = JSONParser::getKey(it); + ColumnString & col_str = static_cast(dest); + col_str.insertData(key.data, key.size); + return true; + } + + static constexpr size_t num_extra_arguments = 0; + static void prepare(const Block &, const ColumnNumbers &, size_t) {} +}; + + +template +class JSONTypeImpl +{ +public: + static DataTypePtr getType(const ColumnsWithTypeAndName &) + { + static const std::vector> values = { + {"Array", '['}, + {"Object", '{'}, + {"String", '"'}, + {"Integer", 'l'}, + {"Float", 'd'}, + {"Bool", 'b'}, + {"Null", 0}, + }; + return std::make_shared>(values); + } + + using Iterator = typename JSONParser::Iterator; + static bool addValueToColumn(IColumn & dest, const Iterator & it) + { + UInt8 type; + if (JSONParser::isInteger(it)) + type = 'l'; + else if (JSONParser::isFloat(it)) + type = 'd'; + else if (JSONParser::isBool(it)) + type = 'b'; + else if (JSONParser::isString(it)) + type = '"'; + else if (JSONParser::isArray(it)) + type = '['; + else if (JSONParser::isObject(it)) + type = '{'; + else if (JSONParser::isNull(it)) + type = 0; + else + return false; + + ColumnVector & col_vec = static_cast &>(dest); + col_vec.insertValue(type); + return true; + } + + static constexpr size_t num_extra_arguments = 0; + static void prepare(const Block &, const ColumnNumbers &, size_t) {} +}; + + +template +class JSONExtractNumericImpl +{ +public: + static DataTypePtr getType(const ColumnsWithTypeAndName &) + { + return std::make_shared>(); + } + + using Iterator = typename JSONParser::Iterator; + static bool addValueToColumn(IColumn & dest, const Iterator & it) + { + NumberType value; + + if (JSONParser::isInteger(it)) + { + if (!accurate::convertNumeric(JSONParser::getInteger(it), value)) + return false; + } + else if (JSONParser::isFloat(it)) + { + if (!accurate::convertNumeric(JSONParser::getFloat(it), value)) + return false; + } + else if (JSONParser::isBool(it) && std::is_integral_v && convert_bool_to_integer) + value = static_cast(JSONParser::getBool(it)); + else + return false; + + auto & col_vec = static_cast &>(dest); + col_vec.insertValue(value); + return true; + } + + static constexpr size_t num_extra_arguments = 0; + static void prepare(const Block &, const ColumnNumbers &, size_t) {} +}; + +template +using JSONExtractInt8Impl = JSONExtractNumericImpl; +template +using JSONExtractUInt8Impl = JSONExtractNumericImpl; +template +using JSONExtractInt16Impl = JSONExtractNumericImpl; +template +using JSONExtractUInt16Impl = JSONExtractNumericImpl; +template +using JSONExtractInt32Impl = JSONExtractNumericImpl; +template +using JSONExtractUInt32Impl = JSONExtractNumericImpl; +template +using JSONExtractInt64Impl = JSONExtractNumericImpl; +template +using JSONExtractUInt64Impl = JSONExtractNumericImpl; +template +using JSONExtractFloat32Impl = JSONExtractNumericImpl; +template +using JSONExtractFloat64Impl = JSONExtractNumericImpl; + + +template +class JSONExtractBoolImpl +{ +public: + static DataTypePtr getType(const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + using Iterator = typename JSONParser::Iterator; + static bool addValueToColumn(IColumn & dest, const Iterator & it) + { + if (!JSONParser::isBool(it)) + return false; + + auto & col_vec = static_cast &>(dest); + col_vec.insertValue(static_cast(JSONParser::getBool(it))); + return true; + } + + static constexpr size_t num_extra_arguments = 0; + static void prepare(const Block &, const ColumnNumbers &, size_t) {} +}; + + +template +class JSONExtractStringImpl +{ +public: + static DataTypePtr getType(const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + using Iterator = typename JSONParser::Iterator; + static bool addValueToColumn(IColumn & dest, const Iterator & it) + { + if (!JSONParser::isString(it)) + return false; + + StringRef str = JSONParser::getString(it); + ColumnString & col_str = static_cast(dest); + col_str.insertData(str.data, str.size); + return true; + } + + static constexpr size_t num_extra_arguments = 0; + static void prepare(const Block &, const ColumnNumbers &, size_t) {} +}; + + +template +class JSONExtractRawImpl +{ +public: + static DataTypePtr getType(const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + using Iterator = typename JSONParser::Iterator; + static bool addValueToColumn(IColumn & dest, const Iterator & it) + { + ColumnString & col_str = static_cast(dest); + auto & chars = col_str.getChars(); + WriteBufferFromVector buf(chars, WriteBufferFromVector::AppendModeTag()); + traverse(it, buf); + buf.finish(); + chars.push_back(0); + col_str.getOffsets().push_back(chars.size()); + return true; + } + + static constexpr size_t num_extra_arguments = 0; + static void prepare(const Block &, const ColumnNumbers &, size_t) {} + +private: + static void traverse(const Iterator & it, WriteBuffer & buf) + { + if (JSONParser::isInteger(it)) + { + writeIntText(JSONParser::getInteger(it), buf); + return; + } + if (JSONParser::isFloat(it)) + { + writeFloatText(JSONParser::getFloat(it), buf); + return; + } + if (JSONParser::isBool(it)) + { + if (JSONParser::getBool(it)) + writeCString("true", buf); + else + writeCString("false", buf); + return; + } + if (JSONParser::isString(it)) + { + writeJSONString(JSONParser::getString(it), buf, format_settings()); + return; + } + if (JSONParser::isArray(it)) + { + writeChar('[', buf); + Iterator it2 = it; + if (JSONParser::downToArray(it2)) + { + traverse(it2, buf); + while (JSONParser::next(it2)) + { + writeChar(',', buf); + traverse(it2, buf); + } + } + writeChar(']', buf); + return; + } + if (JSONParser::isObject(it)) + { + writeChar('{', buf); + Iterator it2 = it; + StringRef key; + if (JSONParser::downToObject(it2, key)) + { + writeJSONString(key, buf, format_settings()); + writeChar(':', buf); + traverse(it2, buf); + while (JSONParser::nextKeyValue(it2, key)) + { + writeChar(',', buf); + writeJSONString(key, buf, format_settings()); + writeChar(':', buf); + traverse(it2, buf); + } + } + writeChar('}', buf); + return; + } + if (JSONParser::isNull(it)) + { + writeCString("null", buf); + return; + } + } + + static const FormatSettings & format_settings() + { + static const FormatSettings the_instance = [] + { + FormatSettings settings; + settings.json.escape_forward_slashes = false; + return settings; + }(); + return the_instance; + } +}; + + +template +class JSONExtractImpl +{ +public: + static constexpr size_t num_extra_arguments = 1; + + static DataTypePtr getType(const ColumnsWithTypeAndName & arguments) + { + if (arguments.size() < 2) + throw Exception{"Function JSONExtract requires at least two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + + auto col_type_const = typeid_cast(arguments.back().column.get()); + if (!col_type_const) + throw Exception{"Illegal non-const column " + arguments.back().column->getName() + " of the last argument of function JSONExtract", + ErrorCodes::ILLEGAL_COLUMN}; + + return DataTypeFactory::instance().get(col_type_const->getValue()); + } + + void prepare(const Block & block, const ColumnNumbers &, size_t result_pos) + { + extract_tree = buildExtractTree(block.getByPosition(result_pos).type); + } + + using Iterator = typename JSONParser::Iterator; + bool addValueToColumn(IColumn & dest, const Iterator & it) + { + return extract_tree->addValueToColumn(dest, it); + } + +private: + class Node + { + public: + Node() {} + virtual ~Node() {} + virtual bool addValueToColumn(IColumn &, const Iterator &) = 0; + }; + + template + class NumericNode : public Node + { + public: + bool addValueToColumn(IColumn & dest, const Iterator & it) override + { + return JSONExtractNumericImpl::addValueToColumn(dest, it); + } + }; + + class StringNode : public Node + { + public: + bool addValueToColumn(IColumn & dest, const Iterator & it) override + { + return JSONExtractStringImpl::addValueToColumn(dest, it); + } + }; + + class FixedStringNode : public Node + { + public: + bool addValueToColumn(IColumn & dest, const Iterator & it) override + { + if (!JSONParser::isString(it)) + return false; + auto & col_str = static_cast(dest); + StringRef str = JSONParser::getString(it); + if (str.size > col_str.getN()) + return false; + col_str.insertData(str.data, str.size); + return true; + } + }; + + template + class EnumNode : public Node + { + public: + EnumNode(const std::vector> & name_value_pairs_) : name_value_pairs(name_value_pairs_) + { + for (const auto & name_value_pair : name_value_pairs) + { + name_to_value_map.emplace(name_value_pair.first, name_value_pair.second); + only_values.emplace(name_value_pair.second); } } - block.getByPosition(result_pos).column = std::move(to); - } -}; -} -#endif + bool addValueToColumn(IColumn & dest, const Iterator & it) override + { + auto & col_vec = static_cast &>(dest); -namespace DB -{ -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} + if (JSONParser::isInteger(it)) + { + size_t value = static_cast(JSONParser::getInteger(it)); + if (!only_values.count(value)) + return false; + col_vec.insertValue(value); + return true; + } -template -class FunctionJSONDummy : public IFunction -{ -public: - static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + if (JSONParser::isString(it)) + { + auto value = name_to_value_map.find(JSONParser::getString(it)); + if (value == name_to_value_map.end()) + return false; + col_vec.insertValue(value->second); + return true; + } - String getName() const override { return Impl::name; } - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } + return false; + } - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName &) const override + private: + std::vector> name_value_pairs; + std::unordered_map name_to_value_map; + std::unordered_set only_values; + }; + + class NullableNode : public Node { - throw Exception{"Function " + getName() + " is not supported without AVX2", ErrorCodes::NOT_IMPLEMENTED}; + public: + NullableNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} + + bool addValueToColumn(IColumn & dest, const Iterator & it) override + { + ColumnNullable & col_null = static_cast(dest); + if (!nested->addValueToColumn(col_null.getNestedColumn(), it)) + return false; + col_null.getNullMapColumn().insertValue(0); + return true; + } + + private: + std::unique_ptr nested; + }; + + class ArrayNode : public Node + { + public: + ArrayNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} + + bool addValueToColumn(IColumn & dest, const Iterator & it) override + { + if (!JSONParser::isArray(it)) + return false; + + Iterator it2 = it; + if (!JSONParser::downToArray(it2)) + return false; + + ColumnArray & col_arr = static_cast(dest); + auto & data = col_arr.getData(); + size_t old_size = data.size(); + bool were_valid_elements = false; + + do + { + if (nested->addValueToColumn(data, it2)) + were_valid_elements = true; + else + data.insertDefault(); + } + while (JSONParser::next(it2)); + + if (!were_valid_elements) + { + data.popBack(data.size() - old_size); + return false; + } + + col_arr.getOffsets().push_back(data.size()); + return true; + } + + private: + std::unique_ptr nested; + }; + + class TupleNode : public Node + { + public: + TupleNode(std::vector> nested_, const std::vector & explicit_names_) : nested(std::move(nested_)), explicit_names(explicit_names_) + { + for (size_t i = 0; i != explicit_names.size(); ++i) + name_to_index_map.emplace(explicit_names[i], i); + } + + bool addValueToColumn(IColumn & dest, const Iterator & it) override + { + ColumnTuple & tuple = static_cast(dest); + size_t old_size = dest.size(); + bool were_valid_elements = false; + + auto set_size = [&](size_t size) + { + for (size_t i = 0; i != tuple.tupleSize(); ++i) + { + auto & col = tuple.getColumn(i); + if (col.size() != size) + { + if (col.size() > size) + col.popBack(col.size() - size); + else + while (col.size() < size) + col.insertDefault(); + } + } + }; + + if (JSONParser::isArray(it)) + { + Iterator it2 = it; + if (!JSONParser::downToArray(it2)) + return false; + + size_t index = 0; + do + { + if (nested[index]->addValueToColumn(tuple.getColumn(index), it2)) + were_valid_elements = true; + else + tuple.getColumn(index).insertDefault(); + ++index; + } + while (JSONParser::next(it2)); + + set_size(old_size + static_cast(were_valid_elements)); + return were_valid_elements; + } + + if (JSONParser::isObject(it)) + { + if (name_to_index_map.empty()) + { + Iterator it2 = it; + if (!JSONParser::downToObject(it2)) + return false; + + size_t index = 0; + do + { + if (nested[index]->addValueToColumn(tuple.getColumn(index), it2)) + were_valid_elements = true; + else + tuple.getColumn(index).insertDefault(); + ++index; + } + while (JSONParser::nextKeyValue(it2)); + } + else + { + Iterator it2 = it; + StringRef key; + if (!JSONParser::downToObject(it2, key)) + return false; + + do + { + auto index = name_to_index_map.find(key); + if (index != name_to_index_map.end()) + { + if (nested[index->second]->addValueToColumn(tuple.getColumn(index->second), it2)) + were_valid_elements = true; + } + } + while (JSONParser::nextKeyValue(it2, key)); + } + + set_size(old_size + static_cast(were_valid_elements)); + return were_valid_elements; + } + + return false; + } + + private: + std::vector> nested; + std::vector explicit_names; + std::unordered_map name_to_index_map; + }; + + std::unique_ptr buildExtractTree(const DataTypePtr & type) + { + switch (type->getTypeId()) + { + case TypeIndex::UInt8: return std::make_unique>(); + case TypeIndex::UInt16: return std::make_unique>(); + case TypeIndex::UInt32: return std::make_unique>(); + case TypeIndex::UInt64: return std::make_unique>(); + case TypeIndex::Int8: return std::make_unique>(); + case TypeIndex::Int16: return std::make_unique>(); + case TypeIndex::Int32: return std::make_unique>(); + case TypeIndex::Int64: return std::make_unique>(); + case TypeIndex::Float32: return std::make_unique>(); + case TypeIndex::Float64: return std::make_unique>(); + case TypeIndex::String: return std::make_unique(); + case TypeIndex::FixedString: return std::make_unique(); + case TypeIndex::Enum8: return std::make_unique>(static_cast(*type).getValues()); + case TypeIndex::Enum16: return std::make_unique>(static_cast(*type).getValues()); + case TypeIndex::Nullable: return std::make_unique(buildExtractTree(static_cast(*type).getNestedType())); + case TypeIndex::Array: return std::make_unique(buildExtractTree(static_cast(*type).getNestedType())); + case TypeIndex::Tuple: + { + const auto & tuple = static_cast(*type); + const auto & tuple_elements = tuple.getElements(); + std::vector> elements; + for (const auto & tuple_element : tuple_elements) + elements.emplace_back(buildExtractTree(tuple_element)); + return std::make_unique(std::move(elements), tuple.haveExplicitNames() ? tuple.getElementNames() : Strings{}); + } + default: + throw Exception{"Unsupported return type schema: " + type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } } - void executeImpl(Block &, const ColumnNumbers &, size_t, size_t) override - { - throw Exception{"Function " + getName() + " is not supported without AVX2", ErrorCodes::NOT_IMPLEMENTED}; - } + std::unique_ptr extract_tree; }; + +template +void registerFunctionsJSONTemplate(FunctionFactory & factory) +{ + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); +} + } diff --git a/dbms/src/Functions/SimdJSONParser.h b/dbms/src/Functions/SimdJSONParser.h new file mode 100644 index 00000000000..103b4a4b9c0 --- /dev/null +++ b/dbms/src/Functions/SimdJSONParser.h @@ -0,0 +1,110 @@ +#pragma once + +#include +#if USE_SIMDJSON + +#include +#include +#include + +#ifdef __clang__ + #pragma clang diagnostic push + #pragma clang diagnostic ignored "-Wold-style-cast" + #pragma clang diagnostic ignored "-Wnewline-eof" +#endif + +#include + +#ifdef __clang__ + #pragma clang diagnostic pop +#endif + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_ALLOCATE_MEMORY; +} + +/// This class can be used as an argument for the template class FunctionJSON. +/// It provides ability to parse JSONs using simdjson library. +struct SimdJSONParser +{ + static constexpr bool need_preallocate = true; + + void preallocate(size_t max_size) + { + if (!pj.allocateCapacity(max_size)) + throw Exception{"Can not allocate memory for " + std::to_string(max_size) + " units when parsing JSON", + ErrorCodes::CANNOT_ALLOCATE_MEMORY}; + } + + bool parse(const char * data, size_t size) { return !json_parse(data, size, pj); } + + using Iterator = ParsedJson::iterator; + Iterator getRoot() { return Iterator{pj}; } + + static bool downToArray(Iterator & it) { return it.down(); } + + static bool downToObject(Iterator & it) { return it.down() && it.next(); } + + static bool downToObject(Iterator & it, StringRef & first_key) + { + if (!it.down()) + return false; + first_key.data = it.get_string(); + first_key.size = it.get_string_length(); + return it.next(); + } + + static bool parentScopeIsObject(const Iterator & it) { return it.get_scope_type() == '{'; } + + static bool next(Iterator & it) { return it.next(); } + + static bool nextKeyValue(Iterator & it) { return it.next() && it.next(); } + + static bool nextKeyValue(Iterator & it, StringRef & key) + { + if (!it.next()) + return false; + key.data = it.get_string(); + key.size = it.get_string_length(); + return it.next(); + } + + static bool isInteger(const Iterator & it) { return it.is_integer(); } + + static bool isFloat(const Iterator & it) { return it.is_double(); } + + static bool isString(const Iterator & it) { return it.is_string(); } + + static bool isArray(const Iterator & it) { return it.is_array(); } + + static bool isObject(const Iterator & it) { return it.is_object(); } + + static bool isBool(const Iterator & it) { return it.get_type() == 't' || it.get_type() == 'f'; } + + static bool isNull(const Iterator & it) { return it.get_type() == 'n'; } + + static StringRef getKey(const Iterator & it) + { + Iterator it2 = it; + it2.prev(); + return StringRef{it2.get_string(), it2.get_string_length()}; + } + + static StringRef getString(const Iterator & it) { return StringRef{it.get_string(), it.get_string_length()}; } + + static Int64 getInteger(const Iterator & it) { return it.get_integer(); } + + static double getFloat(const Iterator & it) { return it.get_double(); } + + static bool getBool(const Iterator & it) { return it.get_type() == 't'; } + +private: + ParsedJson pj; +}; + +} +#endif diff --git a/dbms/src/IO/WriteBufferFromVector.h b/dbms/src/IO/WriteBufferFromVector.h index e52ff7d8411..2c5dedfd664 100644 --- a/dbms/src/IO/WriteBufferFromVector.h +++ b/dbms/src/IO/WriteBufferFromVector.h @@ -38,18 +38,28 @@ private: working_buffer = internal_buffer; } + static constexpr size_t initial_size = 32; + public: WriteBufferFromVector(VectorType & vector_) : WriteBuffer(reinterpret_cast(vector_.data()), vector_.size()), vector(vector_) { if (vector.empty()) { - static constexpr size_t initial_size = 32; vector.resize(initial_size); set(reinterpret_cast(vector.data()), vector.size()); } } + struct AppendModeTag {}; + WriteBufferFromVector(VectorType & vector_, AppendModeTag) + : WriteBuffer(nullptr, 0), vector(vector_) + { + size_t old_size = vector.size(); + vector.resize(vector.capacity() < initial_size ? initial_size : vector.capacity()); + set(reinterpret_cast(vector.data() + old_size), (vector.size() - old_size) * sizeof(typename VectorType::value_type)); + } + void finish() { if (is_finished) diff --git a/dbms/tests/queries/0_stateless/00918_json_functions_avx2.reference b/dbms/tests/queries/0_stateless/00918_json_functions_avx2.reference index ba6d2eeaa4a..8d98d3202db 100644 --- a/dbms/tests/queries/0_stateless/00918_json_functions_avx2.reference +++ b/dbms/tests/queries/0_stateless/00918_json_functions_avx2.reference @@ -1,26 +1,52 @@ +--JSONLength-- 2 -Object -1 -1 -a -b -b -a -hello -hello 3 +0 +--JSONHas-- +1 +1 +0 +--JSONKey-- +a +b +b +a +--JSONType-- +Object Array +--JSONExtract-- +hello +hello -100 200 300 -('a','hello','b',[-100,200,300]) -[-100,NULL,300] -['a','hello','b',NULL] -[(NULL,NULL,NULL),(NULL,NULL,NULL),(NULL,NULL,NULL),(-100,200,44)] -{"a":"hello","b":[-100,200,300],"c":{"d":[121,144]}} +1 +--JSONExtract (generic)-- +('hello',[-100,200,300]) +('hello',[-100,200,300]) +([-100,200,300],'hello') +('hello\0',0) +hello +[-100,200,300] +(-100,200,300) +[-100,0,0] +[-100,NULL,NULL] +[0,200,0] +[NULL,200,NULL] +-100 +200 +\N +Thursday +Friday +--JSONExtractRaw-- +{"a":"hello","b":[-100,200,300]} "hello" [-100,200,300] -100 +{"a":"hello","b":[-100,200,300],"c":{"d":[121,144]}} {"d":[121,144]} [121,144] 144 + +{"passed":true} +{} diff --git a/dbms/tests/queries/0_stateless/00918_json_functions_avx2.sql b/dbms/tests/queries/0_stateless/00918_json_functions_avx2.sql index 438a256ecf8..5654b3528a5 100644 --- a/dbms/tests/queries/0_stateless/00918_json_functions_avx2.sql +++ b/dbms/tests/queries/0_stateless/00918_json_functions_avx2.sql @@ -1,26 +1,58 @@ +SELECT '--JSONLength--'; SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}'); -SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}'); +SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); +SELECT JSONLength('{}'); + +SELECT '--JSONHas--'; SELECT JSONHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'a'); SELECT JSONHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); -SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', 1); -SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', 2); -SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -1); -SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -2); -SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1); -SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 'a'); -SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); +SELECT JSONHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'c'); + +SELECT '--JSONKey--'; +SELECT JSONKey('{"a": "hello", "b": [-100, 200.0, 300]}', 1); +SELECT JSONKey('{"a": "hello", "b": [-100, 200.0, 300]}', 2); +SELECT JSONKey('{"a": "hello", "b": [-100, 200.0, 300]}', -1); +SELECT JSONKey('{"a": "hello", "b": [-100, 200.0, 300]}', -2); + +SELECT '--JSONType--'; +SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}'); SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); + +SELECT '--JSONExtract--'; +SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 'a'); +SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1); SELECT JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1); SELECT JSONExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2); SELECT JSONExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1); -SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Tuple(String, String, String, Array(Float64))'); -SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 'Array(Int32)'); -SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Array(String)'); -SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Array(Tuple(Int16, Float32, UInt8))'); +SELECT JSONExtractBool('{"passed": true}', 'passed'); + +SELECT '--JSONExtract (generic)--'; +SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Tuple(String, Array(Float64))'); +SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Tuple(a String, b Array(Float64))'); +SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Tuple(b Array(Float64), a String)'); +SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Tuple(a FixedString(6), c UInt8)'); +SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'a', 'String'); +SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 'Array(Float32)'); +SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 'Tuple(Int8, Float32, UInt16)'); +SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 'Array(Int8)'); +SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 'Array(Nullable(Int8))'); +SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 'Array(UInt8)'); +SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 'Array(Nullable(UInt8))'); +SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1, 'Int8'); +SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2, 'Int32'); +SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 4, 'Nullable(Int64)'); +SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)'); +SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)'); + +SELECT '--JSONExtractRaw--'; +SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}'); +SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'a'); +SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); +SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1); SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}'); -SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}', 'a'); -SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}', 'b'); -SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}', 'b', 1); SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}', 'c'); SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}', 'c', 'd'); SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}', 'c', 'd', 2); +SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}', 'c', 'd', 3); +SELECT JSONExtractRaw('{"passed": true}'); +SELECT JSONExtractRaw('{}');