[ 
https://issues.apache.org/jira/browse/FLINK-38053?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18003982#comment-18003982
 ] 

Yunfeng Zhou commented on FLINK-38053:
--------------------------------------

Hi [~sxnan] [~lsy] I have verified that Flink 2.1 can correctly use Variant 
type and built-in functions including `PARSE_JSON` and `TRY_PARSE_JSON`. The 
verifying Flink job is as follows.

{code:java}
public class VariantExample {
    public static void main(String[] args) throws Exception {
        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);

        executePrint(tEnv);
        executeCollect(tEnv);
    }

    private static void executePrint(StreamTableEnvironment tEnv) throws 
Exception {
        tEnv.executeSql(
                "CREATE TEMPORARY TABLE PrintSink (\n"
                        + "  `value` VARIANT\n"
                        + ") WITH (\n"
                        + "  'connector' = 'print'\n"
                        + ")");

        // The following line is printed in TM's stdout:
        // +I[{"a":1,"b":{"b0":2}}]
        tEnv.executeSql(
                        "INSERT INTO PrintSink VALUES (PARSE_JSON('{\"a\": 1, 
\"b\": {\"b0\": 2}}'))")
                .await();
    }

    private static void executeCollect(StreamTableEnvironment tEnv) throws 
Exception {
        Table table =
                tEnv.sqlQuery(
                        "SELECT * FROM (VALUES (TRY_PARSE_JSON('{\"a\": 1, 
\"b\": {\"b0\": 2}}')))");
        try (CloseableIterator<Row> iterator = table.execute().collect()) {
            Row row = iterator.next();
            Variant variant = row.getFieldAs(0);
            System.out.println(variant.getField("a")); // 1
            System.out.println(variant.getField("b").getField("b0")); // 2
        }
    }
}
{code}


> Release Testing: Verify FLIP-521: Integrating Variant Type into Flink
> ---------------------------------------------------------------------
>
>                 Key: FLINK-38053
>                 URL: https://issues.apache.org/jira/browse/FLINK-38053
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table SQL / Planner
>    Affects Versions: 2.1.0
>            Reporter: dalongliu
>            Assignee: Yunfeng Zhou
>            Priority: Blocker
>              Labels: release-testing
>             Fix For: 2.1.0
>
>
> This issue aims to partially verify FLIP-521.
> Flink SQL API now supports the Variant type. And it supports `PARSE_JSON` and 
> `TRY_PARSE_JSON` to parse a JSON string to a Variant.
> We may need to cover the following types of test cases:
> Test 1: Confirm that you can use SQL API to create a Print sink with a 
> Variant column.
> Test 2: Confirm that `PARSE_JSON` and `TRY_PARSE_JSON` can parse a JSON 
> string to a Variant.
> Follow up the test for https://issues.apache.org/jira/browse/FLINK-37922



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to