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

fa zheng edited comment on FLINK-13869 at 8/27/19 8:20 AM:
-----------------------------------------------------------

the demo is as follows, it causes above issue, but works fine in batch mode.
{code:java}
//代码占位符
StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
      EnvironmentSettings bsSettings = 
EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
      StreamTableEnvironment tableEnv = 
StreamTableEnvironment.create(env,bsSettings);
//
//        ExecutionEnvironment env = 
ExecutionEnvironment.getExecutionEnvironment();
//        BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env);
        tableEnv.registerCatalog("myHive1",new 
HiveCatalog("myHive1","p1_stream","","1.2.1"));
        tableEnv.useCatalog("myHive1");
        tableEnv.useDatabase("p1_stream");
        Optional<Catalog> myHive1 = tableEnv.getCatalog("myHive1");
        CatalogFunction catalogFunction= myHive1.get().getFunction(new 
ObjectPath("p1_stream","abs"));
        System.out.println(catalogFunction.getClassName());

        Table result = tableEnv.sqlQuery("SELECT abs(a) FROM  abs_test");

      
tableEnv.toAppendStream(result,Row.class).writeAsText("hdfs:/user/result5.txt");
//        
tableEnv.toDataSet(result,Row.class).writeAsText("hdfs:/user/result5.txt");

        env.execute();

{code}


was (Author: faaronzheng):
the demo is as follow
{code:java}
//代码占位符
StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
      EnvironmentSettings bsSettings = 
EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
      StreamTableEnvironment tableEnv = 
StreamTableEnvironment.create(env,bsSettings);
//
//        ExecutionEnvironment env = 
ExecutionEnvironment.getExecutionEnvironment();
//        BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env);
        tableEnv.registerCatalog("myHive1",new 
HiveCatalog("myHive1","p1_stream","","1.2.1"));
        tableEnv.useCatalog("myHive1");
        tableEnv.useDatabase("p1_stream");
        Optional<Catalog> myHive1 = tableEnv.getCatalog("myHive1");
        CatalogFunction catalogFunction= myHive1.get().getFunction(new 
ObjectPath("p1_stream","abs"));
        System.out.println(catalogFunction.getClassName());

        Table result = tableEnv.sqlQuery("SELECT abs(a) FROM  abs_test");

      
tableEnv.toAppendStream(result,Row.class).writeAsText("hdfs:/user/result5.txt");
//        
tableEnv.toDataSet(result,Row.class).writeAsText("hdfs:/user/result5.txt");

        env.execute();

{code}

> Hive built-in function can not work in blink planner stream mode
> ----------------------------------------------------------------
>
>                 Key: FLINK-13869
>                 URL: https://issues.apache.org/jira/browse/FLINK-13869
>             Project: Flink
>          Issue Type: Bug
>          Components: Connectors / Hive, Table SQL / Planner
>    Affects Versions: 1.9.0
>         Environment: method call stack:
> !image-2019-08-27-15-37-11-230.png|width=595,height=1256!
>            Reporter: Terry Wang
>            Priority: Major
>             Fix For: 1.10.0
>
>         Attachments: image-2019-08-27-15-36-57-662.png, 
> image-2019-08-27-15-37-11-230.png
>
>
> In flink, specifying the StreamTableEnvironment through the 
> EnvironmentSetting using the blink planner, when using the UDF in hive in the 
> table API, the following error is reported, the flink planner is used to see 
> the call stack, and the flink planner does not call 
> setArgumentTypeAndConstants to initialize the null pointer.



--
This message was sent by Atlassian Jira
(v8.3.2#803003)

Reply via email to