[fix](json) fix json int128 overflow (#22917)

* support int128 in jsonb

* fix jsonb int128 write

* fix jsonb to json int128

* fix json functions for int128

* add nereids function jsonb_extract_largeint

* add testcase for json int128

* change docs for json int128

* add nereids function jsonb_extract_largeint

* clang format

* fix check style

* using int128_t = __int128_t for all int128

* use fmt::format_to instead of snprintf digit by digit for int128

* clang format

* delete useless check

* add warn log

* clang format
This commit is contained in:
Kang
2023-08-25 11:40:30 +08:00
committed by GitHub
parent 372f83df5c
commit 8ef6b4d996
20 changed files with 2972 additions and 36 deletions

View File

@ -17,6 +17,8 @@
#include "runtime/jsonb_value.h"
#include <fmt/format.h>
#include <string_view>
#include "util/jsonb_error.h"
@ -30,8 +32,10 @@ Status JsonBinaryValue::from_json_string(const char* s, int length) {
JsonbErrType error = JsonbErrType::E_NONE;
if (!parser.parse(s, length)) {
error = parser.getErrorCode();
return Status::InvalidArgument("json parse error: {} for value: {}",
JsonbErrMsg::getErrMsg(error), std::string_view(s, length));
auto msg = fmt::format("json parse error: {} for value: {}", JsonbErrMsg::getErrMsg(error),
std::string_view(s, length));
LOG(WARNING) << msg;
return Status::InvalidArgument(msg);
}
ptr = parser.getWriter().getOutput()->getBuffer();

View File

@ -91,6 +91,8 @@ namespace doris {
#define JSONB_VER 1
using int128_t = __int128;
// forward declaration
class JsonbValue;
class ObjectVal;
@ -517,6 +519,8 @@ public:
return "int";
case JsonbType::T_Int64:
return "bigint";
case JsonbType::T_Int128:
return "largeint";
case JsonbType::T_Double:
return "double";
case JsonbType::T_Float:
@ -624,11 +628,11 @@ inline bool JsonbInt64Val::setVal(int64_t value) {
return true;
}
typedef NumberValT<__int128_t> JsonbInt128Val;
typedef NumberValT<int128_t> JsonbInt128Val;
// override setVal for Int64Val
// override setVal for Int128Val
template <>
inline bool JsonbInt128Val::setVal(__int128_t value) {
inline bool JsonbInt128Val::setVal(int128_t value) {
if (!isInt128()) {
return false;
}
@ -666,7 +670,7 @@ inline bool JsonbFloatVal::setVal(float value) {
// A class to get an integer
class JsonbIntVal : public JsonbValue {
public:
int64_t val() const {
int128_t val() const {
switch (type_) {
case JsonbType::T_Int8:
return ((JsonbInt8Val*)this)->val();
@ -676,11 +680,13 @@ public:
return ((JsonbInt32Val*)this)->val();
case JsonbType::T_Int64:
return ((JsonbInt64Val*)this)->val();
case JsonbType::T_Int128:
return ((JsonbInt128Val*)this)->val();
default:
return 0;
}
}
bool setVal(int64_t val) {
bool setVal(int128_t val) {
switch (type_) {
case JsonbType::T_Int8:
if (val < std::numeric_limits<int8_t>::min() ||
@ -698,7 +704,9 @@ public:
return false;
return ((JsonbInt32Val*)this)->setVal((int32_t)val);
case JsonbType::T_Int64:
return ((JsonbInt64Val*)this)->setVal(val);
return ((JsonbInt64Val*)this)->setVal((int64_t)val);
case JsonbType::T_Int128:
return ((JsonbInt128Val*)this)->setVal(val);
default:
return false;
}
@ -1131,7 +1139,7 @@ inline unsigned int JsonbValue::numPackedBytes() const {
return sizeof(type_) + sizeof(float);
}
case JsonbType::T_Int128: {
return sizeof(type_) + sizeof(__int128_t);
return sizeof(type_) + sizeof(int128_t);
}
case JsonbType::T_String:
case JsonbType::T_Binary: {
@ -1168,7 +1176,7 @@ inline unsigned int JsonbValue::size() const {
return sizeof(float);
}
case JsonbType::T_Int128: {
return sizeof(__int128_t);
return sizeof(int128_t);
}
case JsonbType::T_String:
case JsonbType::T_Binary: {

View File

@ -71,6 +71,8 @@
namespace doris {
using int128_t = __int128;
/*
* Template JsonbParserTSIMD
*/
@ -296,21 +298,22 @@ public:
return;
}
} else if (num.is_int64() || num.is_uint64()) {
if (num.is_uint64() && num.get_uint64() > std::numeric_limits<int64_t>::max()) {
err_ = JsonbErrType::E_OCTAL_OVERFLOW;
LOG(WARNING) << "overflow number: " << num.get_uint64();
return;
}
int64_t val = num.is_int64() ? num.get_int64() : num.get_uint64();
int128_t val = num.is_int64() ? (int128_t)num.get_int64() : (int128_t)num.get_uint64();
int size = 0;
if (val <= std::numeric_limits<int8_t>::max()) {
if (val >= std::numeric_limits<int8_t>::min() &&
val <= std::numeric_limits<int8_t>::max()) {
size = writer_.writeInt8((int8_t)val);
} else if (val <= std::numeric_limits<int16_t>::max()) {
} else if (val >= std::numeric_limits<int16_t>::min() &&
val <= std::numeric_limits<int16_t>::max()) {
size = writer_.writeInt16((int16_t)val);
} else if (val <= std::numeric_limits<int32_t>::max()) {
} else if (val >= std::numeric_limits<int32_t>::min() &&
val <= std::numeric_limits<int32_t>::max()) {
size = writer_.writeInt32((int32_t)val);
} else { // val <= INT64_MAX
size = writer_.writeInt64(val);
} else if (val >= std::numeric_limits<int64_t>::min() &&
val <= std::numeric_limits<int64_t>::max()) {
size = writer_.writeInt64((int64_t)val);
} else { // INT128
size = writer_.writeInt128(val);
}
if (size == 0) {

View File

@ -34,16 +34,21 @@
#endif
#include <assert.h>
#include <fmt/format.h>
#include <string.h>
#include <algorithm>
#include <cinttypes>
#include <iostream>
namespace doris {
using int128_t = __int128;
// lengths includes sign
#define MAX_INT_DIGITS 11
#define MAX_INT64_DIGITS 20
#define MAX_INT128_DIGITS 40
#define MAX_DOUBLE_DIGITS 23 // 1(sign)+16(significant)+1(decimal)+5(exponent)
/*
@ -126,9 +131,15 @@ public:
size_ += len;
}
void write(__int128 l) {
// TODO
assert(false);
void write(int128_t l) {
// snprintf automatically adds a NULL, so we need one more char
if (size_ + MAX_INT128_DIGITS + 1 > capacity_) {
realloc(MAX_INT128_DIGITS + 1);
}
const auto result = fmt::format_to_n(head_ + size_, MAX_INT128_DIGITS, "{}", l);
assert(result.size > 0);
size_ += result.size;
}
// write the double to string

View File

@ -224,9 +224,8 @@ private:
if (value->isInt() && curr->isInt()) {
// Both are ints and optimization can be done here
int64_t val = ((const JsonbIntVal*)value)->val();
// setVal may fail because the new value can't fit into the current one.
if (((JsonbIntVal*)curr)->setVal(val)) {
if (((JsonbIntVal*)curr)->setVal(((const JsonbIntVal*)value)->val())) {
return JsonbErrType::E_NONE;
}
}

View File

@ -40,8 +40,11 @@ public:
// get json string
const std::string to_json_string(const char* data, size_t size) {
doris::JsonbValue* pval = doris::JsonbDocument::createDocument(data, size)->getValue();
return to_json_string(pval);
JsonbDocument* pdoc = doris::JsonbDocument::createDocument(data, size);
if (!pdoc) {
LOG(FATAL) << "invalid json binary value: " << std::string_view(data, size);
}
return to_json_string(pdoc->getValue());
}
const std::string to_json_string(const JsonbValue* val) {

View File

@ -45,6 +45,8 @@
namespace doris {
using int128_t = __int128;
template <class OS_TYPE>
class JsonbWriterT {
public:
@ -234,10 +236,11 @@ public:
return 0;
}
uint32_t writeInt128(__int128_t v) {
uint32_t writeInt128(int128_t v) {
if ((first_ && stack_.empty()) || (!stack_.empty() && verifyValueState())) {
if (!writeFirstHeader()) return 0;
os_->put((JsonbTypeUnder)JsonbType::T_Int128);
os_->write((char*)&v, sizeof(__int128_t));
os_->write((char*)&v, sizeof(int128_t));
kvState_ = WS_Value;
return sizeof(JsonbInt128Val);
}

View File

@ -616,6 +616,8 @@ struct ConvertImplNumberToJsonb {
writer.writeInt32(data[i]);
} else if constexpr (std::is_same_v<ColumnInt64, ColumnType>) {
writer.writeInt64(data[i]);
} else if constexpr (std::is_same_v<ColumnInt128, ColumnType>) {
writer.writeInt128(data[i]);
} else if constexpr (std::is_same_v<ColumnFloat64, ColumnType>) {
writer.writeDouble(data[i]);
} else {
@ -721,7 +723,7 @@ struct ConvertImplFromJsonb {
}
} else if constexpr (type_index == TypeIndex::Int8) {
if (value->isInt8()) {
res[i] = ((const JsonbIntVal*)value)->val();
res[i] = (int8_t)((const JsonbIntVal*)value)->val();
} else {
null_map[i] = 1;
res[i] = 0;
@ -743,7 +745,15 @@ struct ConvertImplFromJsonb {
} else if constexpr (type_index == TypeIndex::Int64) {
if (value->isInt8() || value->isInt16() || value->isInt32() ||
value->isInt64()) {
res[i] = ((const JsonbIntVal*)value)->val();
res[i] = (int64_t)((const JsonbIntVal*)value)->val();
} else {
null_map[i] = 1;
res[i] = 0;
}
} else if constexpr (type_index == TypeIndex::Int128) {
if (value->isInt8() || value->isInt16() || value->isInt32() ||
value->isInt64() || value->isInt128()) {
res[i] = (int128_t)((const JsonbIntVal*)value)->val();
} else {
null_map[i] = 1;
res[i] = 0;
@ -1793,6 +1803,8 @@ private:
return &ConvertImplFromJsonb<TypeIndex::Int32, ColumnInt32>::execute;
case TypeIndex::Int64:
return &ConvertImplFromJsonb<TypeIndex::Int64, ColumnInt64>::execute;
case TypeIndex::Int128:
return &ConvertImplFromJsonb<TypeIndex::Int128, ColumnInt128>::execute;
case TypeIndex::Float64:
return &ConvertImplFromJsonb<TypeIndex::Float64, ColumnFloat64>::execute;
default:
@ -1817,6 +1829,8 @@ private:
return &ConvertImplNumberToJsonb<ColumnInt32>::execute;
case TypeIndex::Int64:
return &ConvertImplNumberToJsonb<ColumnInt64>::execute;
case TypeIndex::Int128:
return &ConvertImplNumberToJsonb<ColumnInt128>::execute;
case TypeIndex::Float64:
return &ConvertImplNumberToJsonb<ColumnFloat64>::execute;
case TypeIndex::String:

View File

@ -756,7 +756,15 @@ private:
}
} else if constexpr (std::is_same_v<int64_t, typename ValueType::T>) {
if (value->isInt8() || value->isInt16() || value->isInt32() || value->isInt64()) {
res[i] = ((const JsonbIntVal*)value)->val();
res[i] = (int64_t)((const JsonbIntVal*)value)->val();
} else {
null_map[i] = 1;
res[i] = 0;
}
} else if constexpr (std::is_same_v<int128_t, typename ValueType::T>) {
if (value->isInt8() || value->isInt16() || value->isInt32() || value->isInt64() ||
value->isInt128()) {
res[i] = (int128_t)((const JsonbIntVal*)value)->val();
} else {
null_map[i] = 1;
res[i] = 0;
@ -892,6 +900,13 @@ struct JsonbTypeInt64 {
static const bool only_check_exists = false;
};
struct JsonbTypeInt128 {
using T = int128_t;
using ReturnType = DataTypeInt128;
using ColumnType = ColumnVector<T>;
static const bool only_check_exists = false;
};
struct JsonbTypeDouble {
using T = double;
using ReturnType = DataTypeFloat64;
@ -948,6 +963,11 @@ struct JsonbExtractBigInt : public JsonbExtractImpl<JsonbTypeInt64> {
static constexpr auto alias = "jsonb_extract_bigint";
};
struct JsonbExtractLargeInt : public JsonbExtractImpl<JsonbTypeInt128> {
static constexpr auto name = "json_extract_largeint";
static constexpr auto alias = "jsonb_extract_largeint";
};
struct JsonbExtractDouble : public JsonbExtractImpl<JsonbTypeDouble> {
static constexpr auto name = "json_extract_double";
static constexpr auto alias = "jsonb_extract_double";
@ -975,6 +995,7 @@ using FunctionJsonbExtractIsnull = FunctionJsonbExtract<JsonbExtractIsnull>;
using FunctionJsonbExtractBool = FunctionJsonbExtract<JsonbExtractBool>;
using FunctionJsonbExtractInt = FunctionJsonbExtract<JsonbExtractInt>;
using FunctionJsonbExtractBigInt = FunctionJsonbExtract<JsonbExtractBigInt>;
using FunctionJsonbExtractLargeInt = FunctionJsonbExtract<JsonbExtractLargeInt>;
using FunctionJsonbExtractDouble = FunctionJsonbExtract<JsonbExtractDouble>;
using FunctionJsonbExtractString = FunctionJsonbExtract<JsonbExtractString>;
using FunctionJsonbExtractJsonb = FunctionJsonbExtract<JsonbExtractJsonb>;
@ -1027,6 +1048,8 @@ void register_function_jsonb(SimpleFunctionFactory& factory) {
factory.register_alias(FunctionJsonbExtractInt::name, FunctionJsonbExtractInt::alias);
factory.register_function<FunctionJsonbExtractBigInt>();
factory.register_alias(FunctionJsonbExtractBigInt::name, FunctionJsonbExtractBigInt::alias);
factory.register_function<FunctionJsonbExtractLargeInt>();
factory.register_alias(FunctionJsonbExtractLargeInt::name, FunctionJsonbExtractLargeInt::alias);
factory.register_function<FunctionJsonbExtractDouble>();
factory.register_alias(FunctionJsonbExtractDouble::name, FunctionJsonbExtractDouble::alias);
factory.register_function<FunctionJsonbExtractString>();

View File

@ -39,6 +39,7 @@ BOOLEAN json_extract_isnull(JSON j, VARCHAR json_path)
BOOLEAN json_extract_bool(JSON j, VARCHAR json_path)
INT json_extract_int(JSON j, VARCHAR json_path)
BIGINT json_extract_bigint(JSON j, VARCHAR json_path)
LARGEINT json_extract_largeint(JSON j, VARCHAR json_path)
DOUBLE json_extract_double(JSON j, VARCHAR json_path)
STRING json_extract_string(JSON j, VARCHAR json_path)
VARCHAR json_str->json_path
@ -51,6 +52,7 @@ json_extract functions extract field specified by json_path from JSON. A series
- json_extract_bool extract and return BOOLEAN datatype
- json_extract_int extract and return INT datatype
- json_extract_bigint extract and return BIGINT datatype
- json_extract_largeint extract and return LARGEINT datatype
- json_extract_double extract and return DOUBLE datatype
- json_extract_STRING extract and return STRING datatype
@ -79,13 +81,14 @@ STRING json_type(JSON j, VARCHAR json_path)
There are two extra functions to check field existence and type
- json_exists_path check the existence of the field specified by json_path, return TRUE or FALS
- json_exists_path get the type as follows of the field specified by json_path, return NULL if it does not exist
- json_type get the type as follows of the field specified by json_path, return NULL if it does not exist
- object
- array
- null
- bool
- int
- bigint
- largeint
- double
- string
@ -145,4 +148,4 @@ mysql> SELECT '{"k1": "v1", "k2": { "k21": 6.6, "k22": [1, 2, 3] } }'->'$.k2.k22
### keywords
JSONB, JSON, json_extract, json_extract_isnull, json_extract_bool, json_extract_int, json_extract_bigint, json_extract_double, json_extract_string, json_exists_path, json_type
JSONB, JSON, json_extract, json_extract_isnull, json_extract_bool, json_extract_int, json_extract_bigint, json_extract_largeint,json_extract_double, json_extract_string, json_exists_path, json_type

View File

@ -36,6 +36,7 @@ It is used to determine the type of the field specified by json_path in JSON dat
- bool
- int
- bigint
- largeint
- double
- string

View File

@ -38,6 +38,7 @@ BOOLEAN json_extract_isnull(JSON j, VARCHAR json_path)
BOOLEAN json_extract_bool(JSON j, VARCHAR json_path)
INT json_extract_int(JSON j, VARCHAR json_path)
BIGINT json_extract_bigint(JSON j, VARCHAR json_path)
LARGEINT json_extract_largeint(JSON j, VARCHAR json_path)
DOUBLE json_extract_double(JSON j, VARCHAR json_path)
STRING json_extract_string(JSON j, VARCHAR json_path)
VARCHAR json_str->json_path
@ -52,6 +53,7 @@ json_extract是一系列函数,从JSON类型的数据中提取json_path指定
- json_extract_bool返回BOOLEAN类型
- json_extract_int返回INT类型
- json_extract_bigint返回BIGINT类型
- json_extract_largeint返回LARGEINT类型
- json_extract_double返回DOUBLE类型
- json_extract_STRING返回STRING类型
@ -121,4 +123,4 @@ mysql> SELECT '{"k1": "v1", "k2": { "k21": 6.6, "k22": [1, 2, 3] } }'->'$.k2.k22
```
### keywords
JSONB, JSON, json_extract, json_extract_isnull, json_extract_bool, json_extract_int, json_extract_bigint, json_extract_double, json_extract_string
JSONB, JSON, json_extract, json_extract_isnull, json_extract_bool, json_extract_int, json_extract_bigint, json_extract_largeint, json_extract_double, json_extract_string

View File

@ -36,6 +36,7 @@ under the License.
- bool
- int
- bigint
- largeint
- double
- string

View File

@ -0,0 +1,72 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions.functions.scalar;
import org.apache.doris.catalog.FunctionSignature;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.JsonType;
import org.apache.doris.nereids.types.LargeIntType;
import org.apache.doris.nereids.types.StringType;
import org.apache.doris.nereids.types.VarcharType;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import java.util.List;
/**
* ScalarFunction 'jsonb_extract_largeint'. This class is generated by GenerateFunction.
*/
public class JsonbExtractLargeint extends ScalarFunction
implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
FunctionSignature.ret(LargeIntType.INSTANCE).args(JsonType.INSTANCE, VarcharType.SYSTEM_DEFAULT),
FunctionSignature.ret(LargeIntType.INSTANCE).args(JsonType.INSTANCE, StringType.INSTANCE)
);
/**
* constructor with 2 arguments.
*/
public JsonbExtractLargeint(Expression arg0, Expression arg1) {
super("jsonb_extract_largeint", arg0, arg1);
}
/**
* withChildren.
*/
@Override
public JsonbExtractLargeint withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 2);
return new JsonbExtractLargeint(children.get(0), children.get(1));
}
@Override
public List<FunctionSignature> getSignatures() {
return SIGNATURES;
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitJsonbExtractLargeint(this, context);
}
}

View File

@ -172,6 +172,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbExtractB
import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbExtractDouble;
import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbExtractInt;
import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbExtractIsnull;
import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbExtractLargeint;
import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbExtractString;
import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbParse;
import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonbParseErrorToInvalid;
@ -973,6 +974,10 @@ public interface ScalarFunctionVisitor<R, C> {
return visitScalarFunction(jsonbExtractBigint, context);
}
default R visitJsonbExtractLargeint(JsonbExtractLargeint jsonbExtractLargeint, C context) {
return visitScalarFunction(jsonbExtractLargeint, context);
}
default R visitJsonbExtractBool(JsonbExtractBool jsonbExtractBool, C context) {
return visitScalarFunction(jsonbExtractBool, context);
}

View File

@ -1724,6 +1724,8 @@ visible_functions = {
[['jsonb_extract_int'], 'INT', ['JSONB', 'STRING'], 'ALWAYS_NULLABLE'],
[['jsonb_extract_bigint'], 'BIGINT', ['JSONB', 'VARCHAR'], 'ALWAYS_NULLABLE'],
[['jsonb_extract_bigint'], 'BIGINT', ['JSONB', 'STRING'], 'ALWAYS_NULLABLE'],
[['jsonb_extract_largeint'], 'LARGEINT', ['JSONB', 'VARCHAR'], 'ALWAYS_NULLABLE'],
[['jsonb_extract_largeint'], 'LARGEINT', ['JSONB', 'STRING'], 'ALWAYS_NULLABLE'],
[['jsonb_extract_double'], 'DOUBLE', ['JSONB', 'VARCHAR'], 'ALWAYS_NULLABLE'],
[['jsonb_extract_double'], 'DOUBLE', ['JSONB', 'STRING'], 'ALWAYS_NULLABLE'],
[['jsonb_extract_string'], 'STRING', ['JSONB', 'VARCHAR'], 'ALWAYS_NULLABLE'],
@ -1754,6 +1756,8 @@ visible_functions = {
[['json_extract_int'], 'INT', ['JSONB', 'STRING'], 'ALWAYS_NULLABLE'],
[['json_extract_bigint'], 'BIGINT', ['JSONB', 'VARCHAR'], 'ALWAYS_NULLABLE'],
[['json_extract_bigint'], 'BIGINT', ['JSONB', 'STRING'], 'ALWAYS_NULLABLE'],
[['json_extract_largeint'], 'LARGEINT', ['JSONB', 'VARCHAR'], 'ALWAYS_NULLABLE'],
[['json_extract_largeint'], 'LARGEINT', ['JSONB', 'STRING'], 'ALWAYS_NULLABLE'],
[['json_extract_double'], 'DOUBLE', ['JSONB', 'VARCHAR'], 'ALWAYS_NULLABLE'],
[['json_extract_double'], 'DOUBLE', ['JSONB', 'STRING'], 'ALWAYS_NULLABLE'],
[['json_extract_string'], 'STRING', ['JSONB', 'VARCHAR'], 'ALWAYS_NULLABLE'],

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -102,6 +102,12 @@ suite("test_json_load_and_function", "p0") {
sql """INSERT INTO ${testTable} VALUES(26, NULL)"""
sql """INSERT INTO ${testTable} VALUES(27, '{"k1":"v1", "k2": 200}')"""
sql """INSERT INTO ${testTable} VALUES(28, '{"a.b.c":{"k1.a1":"v31", "k2": 300},"a":"niu"}')"""
// int64 value
sql """INSERT INTO ${testTable} VALUES(29, '12524337771678448270')"""
// int64 min value
sql """INSERT INTO ${testTable} VALUES(30, '-9223372036854775808')"""
// int64 max value
sql """INSERT INTO ${testTable} VALUES(31, '18446744073709551615')"""
// insert into invalid json rows with enable_insert_strict=true
// expect excepiton and no rows not changed
@ -249,6 +255,39 @@ suite("test_json_load_and_function", "p0") {
qt_select "SELECT id, j, json_extract_bigint(j, '\$.a1[10]') FROM ${testTable} ORDER BY id"
// json_extract_largeint
qt_json_extract_largeint_select "SELECT id, j, json_extract_largeint(j, '\$') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.k1') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.k2') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$[0]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$[1]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$[2]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$[3]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$[4]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$[5]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$[6]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$[10]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[0]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[1]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[2]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[3]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[4]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[10]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[last]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, json_extract_largeint(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id"
// json_extract_double
qt_select "SELECT id, j, json_extract_double(j, '\$') FROM ${testTable} ORDER BY id"

View File

@ -100,6 +100,12 @@ suite("test_jsonb_load_and_function", "p0") {
sql """INSERT INTO ${testTable} VALUES(26, NULL)"""
sql """INSERT INTO ${testTable} VALUES(27, '{"k1":"v1", "k2": 200}')"""
sql """INSERT INTO ${testTable} VALUES(28, '{"a.b.c":{"k1.a1":"v31", "k2": 300},"a":"niu"}')"""
// int64 value
sql """INSERT INTO ${testTable} VALUES(29, '12524337771678448270')"""
// int64 min value
sql """INSERT INTO ${testTable} VALUES(30, '-9223372036854775808')"""
// int64 max value
sql """INSERT INTO ${testTable} VALUES(31, '18446744073709551615')"""
// insert into invalid json rows with enable_insert_strict=true
// expect excepiton and no rows not changed
@ -278,6 +284,39 @@ suite("test_jsonb_load_and_function", "p0") {
qt_select "SELECT id, j, jsonb_extract_bigint(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id"
// jsonb_extract_largeint
qt_jsonb_extract_largeint_select "SELECT id, j, jsonb_extract_largeint(j, '\$') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.k1') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.k2') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[0]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[1]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[2]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[3]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[4]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[5]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[6]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$[10]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[0]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[1]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[2]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[3]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[4]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[10]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last-0]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last-1]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last-2]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[last-10]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[-0]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[-1]') FROM ${testTable} ORDER BY id"
qt_select "SELECT id, j, jsonb_extract_largeint(j, '\$.a1[-10]') FROM ${testTable} ORDER BY id"
// jsonb_extract_double
qt_jsonb_extract_double_select "SELECT id, j, jsonb_extract_double(j, '\$') FROM ${testTable} ORDER BY id"