mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Remove JSONEmpty function and allow JSON in empty function
This commit is contained in:
parent
1870f05207
commit
7f0406a726
@ -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 │
|
||||
└──────────────────────────────────────┴─────────────────┘
|
||||
```
|
||||
|
@ -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"(
|
||||
┌─json──────────┬─JSONEmpty(json)─┐
|
||||
│ {"a":"42"} │ 0 │
|
||||
│ {} │ 1 │
|
||||
│ {"b":"Hello"} │ 0 │
|
||||
│ {} │ 1 │
|
||||
└───────────────┴─────────────────┘
|
||||
|
||||
)"}}},
|
||||
.categories{"JSON"},
|
||||
});
|
||||
}
|
||||
|
||||
}
|
@ -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>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -411,7 +411,6 @@ JSONDynamicPaths
|
||||
JSONDynamicPathsWithTypes
|
||||
JSONEachRow
|
||||
JSONEachRowWithProgress
|
||||
JSONEmpty
|
||||
JSONExtract
|
||||
JSONExtractArrayRaw
|
||||
JSONExtractBool
|
||||
|
Loading…
Reference in New Issue
Block a user