[ https://issues.apache.org/jira/browse/FLINK-3727?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15250196#comment-15250196 ]
ASF GitHub Bot commented on FLINK-3727: --------------------------------------- GitHub user vasia opened a pull request: https://github.com/apache/flink/pull/1917 [FLINK-3727] Embedded streaming SQL projection, filtering, union This PR adds support for embedded streaming SQL (projection, filtering, union): - methods to register DataStreams - sql translation method in StreamTableEnvironment - a custom rule to convert to streamable table - docs for streaming table and sql - java SQL tests A streaming SQL query can be executed on a streaming Table by simply adding the `STREAM` keyword in front of the table name. Registering DataStream tables and conversions work in a similar way to that of DataStream tables. Here's a filtering example: ``` val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) val dataStream = env.addSource(...) val t = dataStream.toTable(tEnv).as('a, 'b, 'c) tEnv.registerTable("MyTable", t) val sqlQuery = "SELECT STREAM * FROM MyTable WHERE a = 3" val result = tEnv.sql(sqlQuery).toDataStream[Row] ``` You can merge this pull request into a Git repository by running: $ git pull https://github.com/vasia/flink stream-sql Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/1917.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1917 ---- commit 6b747bd6902074d0475de9519c1c3bd693487eef Author: vasia <va...@apache.org> Date: 2016-04-15T11:35:24Z [FLINK-3727] Add support for embedded streaming SQL (projection, filter, union) - add methods to register DataStreams - add sql translation method in StreamTableEnvironment - add a custom rule to convert to streamable table - add docs for streaming table and sql ---- > Add support for embedded streaming SQL (projection, filter, union) > ------------------------------------------------------------------ > > Key: FLINK-3727 > URL: https://issues.apache.org/jira/browse/FLINK-3727 > Project: Flink > Issue Type: Sub-task > Components: Table API > Affects Versions: 1.1.0 > Reporter: Vasia Kalavri > Assignee: Vasia Kalavri > > Similar to the support for SQL embedded in batch Table API programs, this > issue tracks the support for SQL embedded in stream Table API programs. The > only currently supported operations on streaming Tables are projection, > filtering, and union. -- This message was sent by Atlassian JIRA (v6.3.4#6332)