-
Notifications
You must be signed in to change notification settings - Fork 13.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-37175][table] Support JSON built-in function for JSON_OBJECT #26022
base: master
Are you sure you want to change the base?
Conversation
6cc5305
to
dc6eed5
Compare
@@ -861,9 +862,13 @@ public static ApiExpression withoutColumns(Object head, Object... tail) { | |||
* jsonObject(JsonOnNull.ABSENT, "K1", nullOf(DataTypes.STRING())) // "{}" | |||
* | |||
* // {"K1":{"K2":"V"}} | |||
* jsonObject(JsonOnNull.NULL, "K1", json('{"K2":"V"}')) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
* jsonObject(JsonOnNull.NULL, "K1", json('{"K2":"V"}')) | |
* jsonObject(JsonOnNull.NULL, "K1", json("{'K2':'V'}")) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
or escaped? not sure how strict the user-defined JSON needs to be? maybe we should check and document this.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The string doesn't have to be escaped, I've updated the javadoc for the java expression and added one example using string literals that show that there's no need to escape it
* // {"K":{"K2":{"K3":42}}}
* jsonObject(
* JsonOnNull.NULL,
* "K",
* json("""
* {
* "K2": {
* "K3": 42
* }
* }
* """))
We escape in the java code since java also uses "
to express strings. If using single quotes in python or string literals in java, there's no need to escape. If they're escaped, they are also processed properly as well and result in the same json object.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thanks for updating the JavaDocs of json()
. Seems this line still needs an update.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ops, yes
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
Outdated
Show resolved
Hide resolved
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
Outdated
Show resolved
Hide resolved
...e/flink-table-common/src/main/java/org/apache/flink/table/types/logical/LogicalTypeRoot.java
Outdated
Show resolved
Hide resolved
...-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/JsonGenerateUtils.scala
Outdated
Show resolved
Hide resolved
...-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/JsonGenerateUtils.scala
Outdated
Show resolved
Hide resolved
...flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala
Outdated
Show resolved
Hide resolved
b6516b6
to
6cc950a
Compare
6cc950a
to
dccb782
Compare
1dea414
to
42db42a
Compare
Made the changes as we discussed @twalthr. One additional thing I've changed is that we not only parse the json, but convert the json back to string before storing it. I think that makes sense, so we optimize the storage space by getting rid of unnecessary whitespaces/line breaks and so on before returning the value. It's a similar behavior to other frameworks and what JSON type probably will eventually do. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks very good! We are almost there :-)
@@ -861,9 +862,13 @@ public static ApiExpression withoutColumns(Object head, Object... tail) { | |||
* jsonObject(JsonOnNull.ABSENT, "K1", nullOf(DataTypes.STRING())) // "{}" | |||
* | |||
* // {"K1":{"K2":"V"}} | |||
* jsonObject(JsonOnNull.NULL, "K1", json('{"K2":"V"}')) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thanks for updating the JavaDocs of json()
. Seems this line still needs an update.
.kind(SCALAR) | ||
.inputTypeStrategy(sequence(logical(LogicalTypeFamily.CHARACTER_STRING))) | ||
.outputTypeStrategy(nullableIfArgs(explicit(DataTypes.STRING()))) | ||
.runtimeClass("org.apache.flink.table.runtime.functions.scalar.JsonFunction") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
replace with runtimeDeferred()
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
My first take at it gave me the impression that we would have even more boilerplate code if we go with runtimeDeferred - I can't just swap them. Would it be a bad practice using the runtime class here?
// throw exception if json function is called outside JSON_OBJECT | ||
if (isJsonFunctionOperand(call)) { | ||
throw new ValidationException( | ||
"The JSON function is currently only supported inside a JSON_OBJECT." + |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
"The JSON function is currently only supported inside a JSON_OBJECT." + | |
"The JSON() function is currently only supported inside a JSON_OBJECT() function." + |
@@ -476,6 +485,11 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, nullableInput: Boolean) | |||
call.getOperator.getReturnTypeInference == ReturnTypes.ARG0 => | |||
generateNullLiteral(resultType) | |||
|
|||
// We only support JSON function operands within JSON_OBJECT | |||
case (operand: RexNode, _) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
case (operand: RexNode, _) | |
case (operand: RexNode, i) |
also check that the index is even. otherwise JSON() function could be used for the key?
@@ -476,6 +485,11 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, nullableInput: Boolean) | |||
call.getOperator.getReturnTypeInference == ReturnTypes.ARG0 => | |||
generateNullLiteral(resultType) | |||
|
|||
// We only support JSON function operands within JSON_OBJECT | |||
case (operand: RexNode, _) | |||
if isJsonObjectOrArrayOperand(call) && isJsonFunctionOperand(operand) => |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since we use OrArrayOperand
? Do we support JSON_ARRAY?
|
||
/** Implementation of {@link BuiltInFunctionDefinitions#JSON}. */ | ||
@Internal | ||
public class JsonFunction extends BuiltInScalarFunction { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Remove?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I look at Oracle and see that they use json_object for the nested json_object.
also I see there is an example of a nested object using json_object in the flink docs:
JSON_OBJECT(
KEY 'K1'
VALUE JSON_OBJECT(
KEY 'K2'
VALUE 'V'
)
)
why do we need a new function when we can already do this with json_object? Or am I missing something?
Reviewed by Chi on 23/01/2025 Go back to the submitter with review comments. |
What is the purpose of the change
It is currently not possible to declare a SQL string that contains existing JSON as valid JSON for JSON_OBJECT. Something like JSON_OBJECT(KEY 'K' VALUE '{"value": 42}') returns {"K", "{"value": 42}"}, where the value is a string a not a json object.
This PR adds support for the JSON() function. It's the initial support for it, until this function returns the JSON datatype (what we still don't have in flink).
Example:
Brief change log
Verifying this change
This change added tests for multiple uses cases of the function. Also added tests to make sure the fucnction is only called within JSON_OBJECT.
Does this pull request potentially affect one of the following parts:
@Public(Evolving)
: yesDocumentation