[
https://issues.apache.org/jira/browse/FLINK-12308?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
sunjincheng updated FLINK-12308:
--------------------------------
Description:
At the Flink API level, we have DataStreamAPI/DataSetAPI/TableAPI&SQL, the
Table API will become the first-class citizen. Table API is declarative, and
can be automatically optimized, which is mentioned in the Flink mid-term
roadmap by Stephan. So, first considering supporting Python at the Table level
to cater to the current large number of analytics users. And Flink's goal for
Python Table API as follows:
* Users can write Flink Table API job in Python, and should mirror Java /
Scala Table API
* Users can submit Python Table API job in the following ways:
** Submit a job with python script, integrate with `flink run`
** Submit a job with python script by REST service
** Submit a job in an interactive way, similar `scala-shell`
** Local debug in IDE.
* Users can write custom functions(UDF, UDTF, UDAF)
* Pandas functions can be used in Flink Python Table API
A more detailed description can be found in
[FLIP-38.|https://cwiki.apache.org/confluence/display/FLINK/FLIP-38%3A+Python+Table+API]
For the API level, we make the following plan:
* The short-term:
We may initially go with a simple approach to map the Python Table API to the
Java Table API via Py4J.
* The long-term:
We may need to create a Python API that follows the same structure as Flink's
Table API that produces the language-independent DAG. (As Stephan already
motioned on the [mailing
thread|http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-38-Support-python-language-in-flink-TableAPI-td28061.html#a28096])
was:
At the Flink API level, we have DataStreamAPI/DataSetAPI/TableAPI&SQL, the
Table API will become the first-class citizen. Table API is declarative, and
can be automatically optimized, which is mentioned in the Flink mid-term
roadmap by Stephan. So, first considering supporting Python at the Table level
to cater to the current large number of analytics users. And Flink's goal for
Python Table API as follows:
* Users can write Flink Table API job in Python, and should mirror Java /
Scala Table API
* Users can submit Python Table API job in the following ways:
** Submit a job with python script, integrate with `flink run`
** Submit a job with python script by REST service
** Submit a job in an interactive way, similar `scala-shell`
** Local debug in IDE.
* Users can write custom functions(UDF, UDTF, UDAF)
* Pandas functions can be used in Flink Python Table API
A more detailed description can be found in FLIP-38(Will be done soon).
For the API level, we make the following plan:
* The short-term:
We may initially go with a simple approach to map the Python Table API to the
Java Table API via Py4J.
* The long-term:
We may need to create a Python API that follows the same structure as Flink's
Table API that produces the language-independent DAG. (As Stephan already
motioned on the [mailing
thread|http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-38-Support-python-language-in-flink-TableAPI-td28061.html#a28096])
> Support python language in Flink Table API
> ------------------------------------------
>
> Key: FLINK-12308
> URL: https://issues.apache.org/jira/browse/FLINK-12308
> Project: Flink
> Issue Type: New Feature
> Components: Table SQL / API
> Affects Versions: 1.9.0
> Reporter: sunjincheng
> Assignee: sunjincheng
> Priority: Major
>
> At the Flink API level, we have DataStreamAPI/DataSetAPI/TableAPI&SQL, the
> Table API will become the first-class citizen. Table API is declarative, and
> can be automatically optimized, which is mentioned in the Flink mid-term
> roadmap by Stephan. So, first considering supporting Python at the Table
> level to cater to the current large number of analytics users. And Flink's
> goal for Python Table API as follows:
> * Users can write Flink Table API job in Python, and should mirror Java /
> Scala Table API
> * Users can submit Python Table API job in the following ways:
> ** Submit a job with python script, integrate with `flink run`
> ** Submit a job with python script by REST service
> ** Submit a job in an interactive way, similar `scala-shell`
> ** Local debug in IDE.
> * Users can write custom functions(UDF, UDTF, UDAF)
> * Pandas functions can be used in Flink Python Table API
> A more detailed description can be found in
> [FLIP-38.|https://cwiki.apache.org/confluence/display/FLINK/FLIP-38%3A+Python+Table+API]
> For the API level, we make the following plan:
> * The short-term:
> We may initially go with a simple approach to map the Python Table API to
> the Java Table API via Py4J.
> * The long-term:
> We may need to create a Python API that follows the same structure as
> Flink's Table API that produces the language-independent DAG. (As Stephan
> already motioned on the [mailing
> thread|http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-38-Support-python-language-in-flink-TableAPI-td28061.html#a28096])
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)