Remove JSONEmpty function and allow JSON in empty function

This commit is contained in:
avogar 2024-08-15 15:53:09 +00:00
parent 1870f05207
commit 7f0406a726
10 changed files with 134 additions and 172 deletions

View File

@ -1359,35 +1359,3 @@ SELECT json, JSONSharedDataPathsWithTypes(json) FROM test;
│ {"a":["1","2","3"],"c":"2020-01-01"} │ {'c':'Date'} │
└──────────────────────────────────────┴────────────────────────────────────┘
```
### JSONEmpty
Checks whether the input [JSON](../data-types/newjson.md) object is empty.
``` sql
JSONEmpty(json)
```
**Arguments**
- `json` — [JSON](../data-types/newjson.md).
**Returned value**
- Returns `1` for an empty JSON object or `0` for a non-empty JSON object. [UInt8](../data-types/int-uint.md).
**Example**
``` sql
CREATE TABLE test (json JSON) ENGINE = Memory;
INSERT INTO test FORMAT JSONEachRow {"json" : {}}, {"json" : {"a" : [1, 2, 3], "b" : "2020-01-01"}}, {"json" : {}},
SELECT json, JSONEmpty(json) FROM test;
```
```text
┌─json─────────────────────────────────┬─JSONEmpty(json)─┐
│ {} │ 1 │
│ {"a":["1","2","3"],"b":"2020-01-01"} │ 0 │
│ {} │ 1 │
└──────────────────────────────────────┴─────────────────┘
```

View File

@ -1,126 +0,0 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypesNumber.h>
#include <Core/ColumnNumbers.h>
#include <Columns/ColumnObject.h>
#include <Columns/ColumnsNumber.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
}
namespace
{
/// Implements the function JSONEmpty which returns true if provided JSON object is empty and false otherwise.
class FunctionJSONEmpty : public IFunction
{
public:
static constexpr auto name = "JSONEmpty";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionJSONEmpty>(); }
std::string getName() const override
{
return name;
}
size_t getNumberOfArguments() const override { return 1; }
bool useDefaultImplementationForConstants() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
DataTypePtr getReturnTypeImpl(const DataTypes & data_types) const override
{
if (data_types.size() != 1 || data_types[0]->getTypeId() != TypeIndex::Object)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires single argument with type JSON", getName());
return std::make_shared<DataTypeUInt8>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override
{
const ColumnWithTypeAndName & elem = arguments[0];
const auto * object_column = typeid_cast<const ColumnObject *>(elem.column.get());
if (!object_column)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected column type in function {}. Expected Object column, got {}", getName(), elem.column->getName());
auto res = DataTypeUInt8().createColumn();
auto & data = typeid_cast<ColumnUInt8 &>(*res).getData();
const auto & typed_paths = object_column->getTypedPaths();
size_t size = object_column->size();
/// If object column has at least 1 typed path, it will never be empty, because these paths always have values.
if (!typed_paths.empty())
{
data.resize_fill(size, 0);
return res;
}
const auto & dynamic_paths = object_column->getDynamicPaths();
const auto & shared_data = object_column->getSharedDataPtr();
data.reserve(size);
for (size_t i = 0; i != size; ++i)
{
bool empty = true;
/// Check if there is no paths in shared data.
if (!shared_data->isDefaultAt(i))
{
empty = false;
}
/// Check that all dynamic paths have NULL value in this row.
else
{
for (const auto & [path, column] : dynamic_paths)
{
if (!column->isNullAt(i))
{
empty = false;
break;
}
}
}
data.push_back(empty);
}
return res;
}
};
}
REGISTER_FUNCTION(JSONEmpty)
{
factory.registerFunction<FunctionJSONEmpty>(FunctionDocumentation{
.description = R"(
Checks whether the input JSON object is empty.
)",
.syntax = {"JSONEmpty(json)"},
.arguments = {{"json", "JSON column"}},
.examples = {{{
"Example",
R"(
CREATE TABLE test (json JSON) ENGINE = Memory;
INSERT INTO test FORMAT JSONEachRow {"json" : {"a" : 42}}, {"json" : {}}, {"json" : {"b" : "Hello"}}, {"json" : {}}
SELECT json, JSONEmpty(json) FROM test;
)",
R"(
jsonJSONEmpty(json)
{"a":"42"} 0
{} 1
{"b":"Hello"} 0
{} 1
)"}}},
.categories{"JSON"},
});
}
}

View File

@ -2,10 +2,18 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionStringOrArrayToT.h>
#include <Functions/EmptyImpl.h>
#include <Columns/ColumnObject.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
}
namespace
{
@ -13,13 +21,126 @@ struct NameEmpty
{
static constexpr auto name = "empty";
};
using FunctionEmpty = FunctionStringOrArrayToT<EmptyImpl<false>, NameEmpty, UInt8, false>;
/// Implements the empty function for JSON type.
class ExecutableFunctionJSONEmpty : public IExecutableFunction
{
public:
std::string getName() const override { return NameEmpty::name; }
private:
bool useDefaultImplementationForConstants() const override { return true; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override
{
const ColumnWithTypeAndName & elem = arguments[0];
const auto * object_column = typeid_cast<const ColumnObject *>(elem.column.get());
if (!object_column)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected column type in function {}. Expected Object column, got {}", getName(), elem.column->getName());
auto res = DataTypeUInt8().createColumn();
auto & data = typeid_cast<ColumnUInt8 &>(*res).getData();
const auto & typed_paths = object_column->getTypedPaths();
size_t size = object_column->size();
/// If object column has at least 1 typed path, it will never be empty, because these paths always have values.
if (!typed_paths.empty())
{
data.resize_fill(size, 0);
return res;
}
const auto & dynamic_paths = object_column->getDynamicPaths();
const auto & shared_data = object_column->getSharedDataPtr();
data.reserve(size);
for (size_t i = 0; i != size; ++i)
{
bool empty = true;
/// Check if there is no paths in shared data.
if (!shared_data->isDefaultAt(i))
{
empty = false;
}
/// Check that all dynamic paths have NULL value in this row.
else
{
for (const auto & [path, column] : dynamic_paths)
{
if (!column->isNullAt(i))
{
empty = false;
break;
}
}
}
data.push_back(empty);
}
return res;
}
};
class FunctionEmptyJSON final : public IFunctionBase
{
public:
FunctionEmptyJSON(const DataTypes & argument_types_, const DataTypePtr & return_type_) : argument_types(argument_types_), return_type(return_type_) {}
String getName() const override { return NameEmpty::name; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
const DataTypes & getArgumentTypes() const override { return argument_types; }
const DataTypePtr & getResultType() const override { return return_type; }
ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override
{
return std::make_unique<ExecutableFunctionJSONEmpty>();
}
private:
DataTypes argument_types;
DataTypePtr return_type;
};
class FunctionEmptyOverloadResolver final : public IFunctionOverloadResolver
{
public:
static constexpr auto name = NameEmpty::name;
static FunctionOverloadResolverPtr create(ContextPtr)
{
return std::make_unique<FunctionEmptyOverloadResolver>();
}
String getName() const override { return NameEmpty::name; }
size_t getNumberOfArguments() const override { return 1; }
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
{
DataTypes argument_types;
argument_types.reserve(arguments.size());
for (const auto & arg : arguments)
argument_types.push_back(arg.type);
if (argument_types.size() == 1 && isObject(argument_types[0]))
return std::make_shared<FunctionEmptyJSON>(argument_types, return_type);
return std::make_shared<FunctionToFunctionBaseAdaptor>(std::make_shared<FunctionEmpty>(), argument_types, return_type);
}
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
{
return std::make_shared<DataTypeUInt8>();
}
};
}
REGISTER_FUNCTION(Empty)
{
factory.registerFunction<FunctionEmpty>();
factory.registerFunction<FunctionEmptyOverloadResolver>();
}
}

View File

@ -109,12 +109,12 @@ select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test format Null;
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test order by id format Null;
select count() from test where JSONEmpty(json.^a) and json.a.b.c == 0;
select count() from test where empty(json.^a) and json.a.b.c == 0;
select json.^a, json.a.b.c from test order by id format Null;
select json, json.^a, json.a.b.c from test format Null;
select json, json.^a, json.a.b.c from test order by id format Null;
select count() from test where JSONEmpty(json.^a) and json.a.b.d is Null;
select count() from test where empty(json.^a) and json.a.b.d is Null;
select json.^a, json.a.b.d from test order by id format Null;
select json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
select json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;

View File

@ -106,12 +106,12 @@ select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test format Null;
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test order by id format Null;
select count() from test where JSONEmpty(json.^a) and json.a.b.c == 0;
select count() from test where empty(json.^a) and json.a.b.c == 0;
select json.^a, json.a.b.c from test order by id format Null;
select json, json.^a, json.a.b.c from test format Null;
select json, json.^a, json.a.b.c from test order by id format Null;
select count() from test where JSONEmpty(json.^a) and json.a.b.d is Null;
select count() from test where empty(json.^a) and json.a.b.d is Null;
select json.^a, json.a.b.d from test order by id format Null;
select json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
select json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;

View File

@ -109,12 +109,12 @@ select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test format Null;
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test order by id format Null;
select count() from test where JSONEmpty(json.^a) and json.a.b.c == 0;
select count() from test where empty(json.^a) and json.a.b.c == 0;
select json.^a, json.a.b.c from test order by id format Null;
select json, json.^a, json.a.b.c from test format Null;
select json, json.^a, json.a.b.c from test order by id format Null;
select count() from test where JSONEmpty(json.^a) and json.a.b.d is Null;
select count() from test where empty(json.^a) and json.a.b.d is Null;
select json.^a, json.a.b.d from test order by id format Null;
select json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
select json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;

View File

@ -40,8 +40,8 @@ select json.a.r, json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.
select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0);
select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0.:Int64);
select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null;
select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null;
select count() from test where empty(arrayJoin(json.a.r[].^b)) and empty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null;
select count() from test where empty(arrayJoin(json.a.r[].^b)) and empty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null;
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null;
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null;

View File

@ -37,8 +37,8 @@ select json.a.r, json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.
select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0);
select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0.:Int64);
select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null;
select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null;
select count() from test where empty(arrayJoin(json.a.r[].^b)) and empty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null;
select count() from test where empty(arrayJoin(json.a.r[].^b)) and empty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null;
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null;
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null;

View File

@ -40,8 +40,8 @@ select json.a.r, json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.
select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0);
select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0.:Int64);
select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null;
select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null;
select count() from test where empty(arrayJoin(json.a.r[].^b)) and empty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null;
select count() from test where empty(arrayJoin(json.a.r[].^b)) and empty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null;
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null;
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null;

View File

@ -411,7 +411,6 @@ JSONDynamicPaths
JSONDynamicPathsWithTypes
JSONEachRow
JSONEachRowWithProgress
JSONEmpty
JSONExtract
JSONExtractArrayRaw
JSONExtractBool