Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.


...

Page properties

...


Discussion thread

...


Vote thread
JIRA

...

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyFLINK-12308

...

Release1.10


Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

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. So this proposal will cover the following items:

...

         Python Table API programs should be similarly submitted and deployed as Java / Scala Table API programs, Such as CLI, web, containerized, etc.

  • User-defined functions

         Support user-defined stateful and stateless python functions. It includes user-defined scalar function, user-defined table function, and user-defined aggregate function.

  • Pandas Support
    • Add toPandas and fromPandas interfaces in Table API as conversions between Table and pandas.
    • Support to use pandas UDFs directly in Python Table API.

Modules

  • flink-python
    • flink-python-tableThe place for all python interface definitions and implementations, such as Table, Window, TableEnvironment, TableConfig, ConnectorDescriptor, DataType, TableSchema, TableSource, TableSink etc. i.e all the user interface in `flink-table-common` and `flink-table-api-java` should be there.
    • flink-python-streaming(in the future)

Modules

  • flink-python(maven module)
    • pyflink(python package)
      • table 
      • shell
      • streaming(in the future)
      • others...
  • flink-clients(maven module)

          Support for submitting Python Table API job in CliFrontend, such as `flink run -py wordcount.py`.

We      We need to add components in FLINK JIRA as follows:

  • API/Python - for Python API (already exists)
  • Runtime/
  • Python Shell - for interactive Python
  • function execution.
  • Table SQL/Python - for Python user-defined function execution
  • Python Shell - for interactive Python program
  • flink-python-shell

          For interactive development, similar to scala-shell. flink-python-shell users can write and run Python Table API (and Python Datastream API in the future).

  • flink-clients

          Support for submitting Python Table API job in CliFrontend, such as `flink run -py wordcount.py`.

Architecture

We don't develop python operators like `flink-python` and `flink-stream-python`. To get the most out of the existing Java/Scala results (the Calcite-based optimizer), the Python Table API only needs to define the Python Table API interface.  Calls to the existing Java Table API implementation to meet the needs of python users with minimal effort. So our main job is to implement communication between Python VM and Java VM, as shown below:

Image Removed

Currently, we have two options for the implementation of RPC.

Approach 1

For Python API, a very mature solution is to choose Py4J as shown below:

Image Removed

At Python Side, Py4J provides a JavaGateway object. It has a field “jvm” which enables Python program to access the Java classes directly. We can construct the Python wrappers for the Java classes through it.

  • program

Architecture

We don't develop python operators like `flink-python` and `flink-stream-python`. To get the most out of the existing Java/Scala results (the Calcite-based optimizer), the Python Table API only needs to define the Python Table API interface.  Calls to the existing Java Table API implementation to meet the needs of python users with minimal effort. So our main job is to implement communication between Python VM and Java VM, as shown below:

Image Added

Currently, we have two options for the implementation of RPC.

Approach 1

For Python API, a very mature solution is to choose Py4J as shown below:

Image Added

At Python Side, Py4J provides a JavaGateway object. It has a field “jvm” which enables Python program to access the Java classes directly. We can construct the Python wrappers for the Java classes through it.

At the Java side, Py4J provides GatewayServer. It receives the Python API requests and we can use it to delegate all the method calling of the Python API to the At the Java side, Py4J provides GatewayServer. It receives the Python API requests and we can use it to delegate all the method calling of the Python API to the corresponding Java/Scala Table API.

There will be Python wrappers for all the API classes such as TableEnvironment, Table, TableSink, TableSource, Catalog, etc.  For example:

class Table(object):

   """

   Wrapper of org.apache.flink.table.api.Table

   """

   def __init__(self, j_table):

       self._j_table = j_table  # The references to table object on JVM.

   def select(self, col_list):

       return Table(self._j_table.select(col_list))  # Call the java table api method directly

...

In this approach Py4J encapsulates all serialization deserialization processes.

Approach 2

Currently, we want to add Python Table API, And in the future, we may want to support the other popular Non-JVM language based Table API, such as R, Go, etc. So we can also have a more scalable approach, as follows:

...

      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)

UDF Architecture (Overall)

The main design to support UDF is the communication between JVM and Python VM, and the management of the Python environment. The overall architecture is as follows:

Image Removed

The overall architecture is that UDF runs in an isolated environment which may be a separate process, a docker container, etc and Flink operator communicates with UDF via gRPC. Besides data transfer, the gRPC can also be used to transfer the state access operations, logging operations, metrics reporting operations, etc.

The components involve in the communication between Flink operator and UDF is as follows:

  • Env Service - Responsible for launching and destroying the Python worker execution environment.
    • the Python UDF may have custom dependencies which are not installed in the cluster and the Python version may also be different from the Python installed in the cluster. Two solutions will be provided to solve this problem.
      • Docker - The Python worker runs in docker container. Users should specify a docker image which is a self-contained environment.
      • Process -  Python worker runs as a separate process. Users can create a virtual environment such as virtualenv or conda locally and install third-party dependencies into this environment.
  • Data Service - Responsible for transferring the input data and the UDF execution results between Flink operator and Python worker.
  • State Service - It allows the aggregate function to access state using DataView. It’s responsible for read/write state from/to Flink operator.
  • Logging Service - It provides logging support for UDF. It’s responsible for transferring log entries produced by UDF to Flink operator and integrates with the Flink’s logging system.
  • Metrics Service - It allows UDF to access the Flink’s metrics system.

Stateless User-defined Function (Scalar/Table Function)

The most important thing for the stateless function is the data service. Considering the performance loss of the communication between JVM and Python VM, we need to consider how to optimize the execution performance of stateless function from the points of data processing:

  • Data processing mode
    • Sync - process the elements one by one, ONLY process the next element when we get the execution result of the previous element.
    • Async - process the elements in a pipeline manner. For one input element, the processing stages include transferring data to Python worker, UDF execution and transferring the execution results back to Flink operator. We can use buffers between each stage to decouple the dependencies between these stages.
  • Data transmission mode
    • Single - transfer elements between Flink operator and Python worker one by one.
    • Block - transfer elements between Flink operator and Python worker block by block. Each block can contain multiple elements.

In order to achieve better performance, we choose to use asynchronous data processing mode and transfer data between JVM and Python VM by blocks. The block size can be configured. The workflow is as follows:

Image Removed

Flink operator will firstly send the inputs to Python worker and Python worker executes UDF and sends back the results to Flink operator. Then Flink operator can construct the result rows and send them to the downstream operator. The components involved in the workflow is as follows:

  • Input Buffer - The input elements are put into the input buffer (the columns will be projected and only required columns are transferred to Python worker), i.e. inputs are processed asynchronously to make sure that the inputs can be processed in a pipeline manner.
  • Waiting Buffer - The original inputs sent to Python worker will be buffered as they will be needed when calculating the result row.
  • Result Buffer - The result data are put into the Result Buffer in order and one result data corresponds to one input data in the Waiting Buffer.

Besides, the sibling Python UDFs or parent-child Python UDFs of the same operator will be optimized to run as one Python UDF to eliminate the data transmission/serialization overhead. The input columns will be combined and sent to the Python worker together and each UDF will be executed with its required columns.

Pandas Function Support

Pandas functions require pandas. Series or pandas.DataFrame as inputs. To use pandas functions in Table API, we need to convert data between table and pandas. Apache Arrow provides Java and Python facilities to convert between Apache Arrow table and pandas format. If the Python UDFs are pandas functions, Flink operator will serialize the data using Apache Arrow serializer before sending them to the Python worker. At the other end, Python worker will deserialize the data using Apache Arrow Python facilities before executing the pandas's functions. The workflow is as follows:

Public Interfaces

Briefly list any new interfaces that will be introduced as part of this proposal or any existing interfaces that will be removed or changed. The purpose of this section is to concisely call out the public contract that will come along with this feature.

A public interface is any change to the following:

  • DataStream and DataSet API, including classes related to that, such as StreamExecutionEnvironment
  • Classes marked with the @Public annotation
  • On-disk binary formats, such as checkpoints/savepoints
  • User-facing scripts/command-line tools, i.e. bin/link, Yarn scripts, Mesos scripts
  • Configuration settings
  • Exposed monitoring information

Proposed Changes

Describe the new thing you want to do in appropriate detail. This may be fairly extensive and have large subsections of its own. Or it may be a few sentences. Use judgement based on the scope of the change.

Compatibility, Deprecation, and Migration Plan

  • What impact (if any) will there be on existing users? 
  • If we are changing behavior how will we phase out the older behavior? 
  • If we need special migration tools, describe them here.
  • When will we remove the existing behavior?

Test Plan

Describe in few sentences how the FLIP will be tested. We are mostly interested in system tests (since unit-tests are specific to implementation details). How will we know that the implementation works as expected? How will we know nothing broke?

Rejected Alternatives

Public Interfaces

The Python Table API classes are just some wrappers of the corresponding Java Table API classes. All the user interface in `flink-table-common` and `flink-table-api-java` should be defined in Python Table API. The main interface/implementation is as follows:

  • Table
  • TableEnvironment
  • TableConfig
  • Descriptor
  • TypeInformation(May change after FLIP-37)
  • etc

Table

class Table(object):

   """

   Wrapper of org.apache.flink.table.api.Table

   """

   def __init__(self, j_table):

       self._j_table = j_table  # The references to table object on JVM.

   def select(self, col_list):

       return Table(self._j_table.select(col_list))  # Call the java table api method directly

   def where(self, condition):

       return Table(self._j_table.where(condition))

   …

   …

   def write_to_sink(self, j_sink):

       self._j_table.writeToSink(j_sink._j_table_sink)

    …

   ....

GroupedTable

class GroupedTable(object):

  """

  Wrapper of org.apache.flink.table.api.GroupedTable

  """

  def __init__(self, j_grouped_table):

      # The references to group table object on JVM.

      self._j_grouped_table = j_grouped_table

  def select(self, col_list):  

      # Call the java table api method directly

      return Table(self._j_grouped_table.select(col_list))

GroupWindowedTable, WindowGroupedTable, OverWindowedTable are defined similarly as GroupTable.

TableConfig

class TableConfig(object):

    def __init__(self, j_table_config=None):

            gateway = get_gateway()

            if j_table_config is None:

                  self._j_table_config = gateway.jvm.TableConfig()

            else:

                 self._j_table_config = j_table_config

  def get_local_timezone(self):

      return self._j_table_config.getLocalTimeZone().getId()


  def set_local_timezone(self, timezone_id):

   ...

def get_configuration(self):
       return Configuration(j_configuration=self._j_table_config.getConfiguration())

def add_configuration(self, configuration):
      self._j_table_config.addConfiguration(configuration._j_configuration)

TableEnvironment

class TableEnvironment(object):

  """

  Wrap and extend for org.apache.flink.table.api.TableEnvironment

  """

  table_config = None

  def __init__(self, j_tenv):

      self._j_tenv = j_tenv

  def register_table(self, name, table):

      self._j_tenv.registerTable(name, table._j_table)

  def register_table_source(self, name, table_source):

      self._j_tenv.registerTableSource(name, table_source.j_table_source)

  def register_table_sink(self, name, table_sink):

      self._j_tenv.registerTableSink(name, table_sink.j_table_sink)

  def scan(self, *table_path):

      j_paths = TypesUtil.convert_py_list_to_j_array("java.lang.String", table_path)

      j_table = self._j_tenv.scan(j_paths)

      return Table(j_table)

  def connect(self, connector_descriptor):

      return TableDescriptor(self._j_tenv.connect(connector_descriptor._j_connector_descriptor))

   …

   …

  def sql_query(self, query):

      j_table = self._j_tenv.sqlQuery(query)

      return Table(j_table)

  def sql_update(self, stmt, config=None):

      if config is not None:

          j_table = self._j_tenv.sqlUpdate(stmt, config)

      else:

          j_table = self._j_tenv.sqlUpdate(stmt)

  # Extension methods

  def from_collection(self, data):

      …

      return Table(...)


  def execute(self):

      self._j_tenv.execEnv().execute()

  def set_parallelism(self, parallelism):

      self._j_tenv.execEnv().setParallelism(parallelism)

  …

  ...

  @classmethod

  def create(cls, table_config):

      j_tenv = ...

      return TableEnvironment(j_tenv)

Descriptor

There are a lot of Descriptor related classes, we will take Csv as an example:

class Csv(FormatDescriptor):

  def __init__(self):

      self._j_csv = _jvm.org.apache.flink.table.descriptors.Csv()

      super(Csv, self).__init__(self._j_csv)

  def field_delimiter(self, delimiter):

      self._j_csv.fieldDelimiter(delimiter)

      return self

  def line_delimiter(self, delimiter):

      self._j_csv.lineDelimiter(delimiter)

      return self

  def quote_character(self, quoteCharacter):

      self._j_csv.quoteCharacter(quoteCharacter)

      return self

  def allow_comments(self):

      self._j_csv.allowComments()

      return self

  def ignore_parse_errors(self):

      self._j_csv.ignoreParseErrors()

      return self

  def array_element_delimiter(self, delimiter):

      self._j_csv.arrayElementDelimiter(delimiter)

      return self

  def escape_character(self, escape_character):

      self._j_csv.escapeCharacter(escape_character)

      return self

  def null_literal(self, null_literal):

      self._j_csv.nullLiteral(null_literal)

      return self

Expression

Expression API will be not supported in the initial version as there is an ongoing work of introducing Table API Java Expression DSL. Python Expression API will leverage that and supported once that work is done.

DIST

Create a python directory in the published opt directory. The contents of python are as follows:

opt/python

├── README.MD

└── lib

   ├── py4j-x.y.z-src.zip

   ├── py4j-LICENSE.txt

   └── pyflink.zip

└── ...

The flink-python module will be packaged as pyflink.zip, And put it in to opt/python/lib directory with PY4J_LICENSE.txt py4j-xxx-src.zip.

  • And the pyflink shell we be added in published bin directory.
  • The shell of `flink` should add some options for Python Table API, such as:
    • -py --python
    • -pyfs --py-files
    • etc ...

    The detail can be found in the Job Submission section.

Docs

  • Add the description of `flink run -py xx.py` in CLI
  • Add rest service API for submit job
  • Add a Python REPL submenu under the Deployment & Operations directory to add documentation for the python shell.
  • Add Python Table API doc in current TableAPI doc
  • Add Common concepts doc for Python Table API, in Basic concepts doc
  • Add pythondocs at the same level as javadocs and scaladocs
  • etc.

Examples

WordCount

Let's take a word count as an example to provide a example. The Python Table API will look like the following :

from pyflink.dataset import ExecutionEnvironment
from pyflink.table import BatchTableEnvironment, TableConfig
...

content = "..."

t_config = TableConfig()
env = ExecutionEnvironment.get_execution_environment()
t_env = BatchTableEnvironment.create(env, t_config)

# register Results table in table environment
tmp_dir = tempfile.gettempdir()
result_path = tmp_dir + '/result'
if os.path.exists(result_path):
try:
    if os.path.isfile(result_path):
        os.remove(result_path)
    else:
        shutil.rmtree(result_path)
except OSError as e:
    logging.error("Error removing directory: %s - %s.", e.filename, e.strerror)

t_env.connect(FileSystem().path(result_path)) \
    .with_format(OldCsv()
    .field_delimiter(',')
    .field("word", DataTypes.STRING())
    .field("count", DataTypes.BIGINT())) \
    .with_schema(Schema()
    .field("word", DataTypes.STRING())
    .field("count", DataTypes.BIGINT())) \
    .register_table_sink("Results")

elements = [(word, 1) for word in content.split(" ")]
t_env.from_elements(elements, ["word", "count"]) \
    .group_by("word") \
    .select("word, count(1) as count") \
    .insert_into("Results")

t_env.execute("word_count")

...

Job Submission

Flink Run

Support for submitting Python Table API job in CliFrontend,And using `flink run` submit Python Table API job. The current `flink` command command line syntax is as follows:

flink <ACTION> [OPTIONS] [ARGUMENTS]

On the basis of the current `run` ACTION, we add to Python Table API support, specific OPTIONS are as follows:

  • -py --python  <python-file-name>

Python script with the program entry point. We can configure dependent resources with the `--py-files` option.

  • -pyfs --py-files <python-files>   

Attach custom python files for job. Comma can be used as the separator to specify multiple files. The standard python resource file suffixes such as .py/.egg/.zip all also supported.

    • -pym --py-module <python-module>  Python module with the program entry point. This option must be used in conjunction with ` --py-files`.
  • -py-exec --py-exec <python-binary-path>

The Python binary to be used.

  • -py-env --py-env <process/docker>

The execution environment: process or docker.

  • -py-docker-image --py-docker-image <docker image>

The docker image to be used to run the python worker. Valid when the option of “-py-env” is docker.

NOTE: When options `py` and `pym` appear at the same time, option `py` will be valid.

We can submit the Python Table API jobs as follows:

// Submit a simple job without any dependencies and parameters

FLINK_HOME/bin/flink run -py example.py

// Submit a job with dependencies and parallelism 16

FLINK_HOME/bin/flink run -p 16 -py example.py -pyfs resources1.egg, resources2.zip, resource3.py

// Submit a job by configuring python class

FLINK_HOME/bin/flink run -pyfs resources1.egg, resources2.zip, resource3.py -pym org.apache.pyflink.example

// Submit a job with binary path and execution environment config

FLINK_HOME/bin/flink run -pyfs resources1.egg, resources2.zip, resource3.py -py-exec ./my_env/env/bin/python -py-env process -pym org.apache.pyflink.example

// Sumit a simple job in which the Python worker runs in a docker container

FLINK_HOME/bin/flink run --py-env docker --py-docker-image xxx -py example.py

Regarding the [ARGUMENTS] section of syntax, we reuse existing features without any extensions.

Python Shell

The main goal of Flink Python Shell is to provide an interactive way for users to write and execute flink Python Table API jobs.

By default, jobs are executed in a mini cluster (for beginner users to learn and research). If users want to submit jobs to a cluster, they need to configure additional parameters, such as the execution mode or the JobMaster address (standalone cluster), just like Flink Scala Shell.

The Python shell is responsible for confirming the location of the flink files, loading required dependencies and then executing the shell initialization logic.

The initialization procedure will import all the Flink Python Table API classes which should be exposed to the user.

Example:

...

>>>exec_env = ExecutionEnvironment.get_execution_environment()

>>>exec_env.set_parallelism(1)

>>>t_config = TableConfig()

>>>t_env =BatchTableEnvironment.create(t_config)

>>>data = [(1L, "Sunny"), (2L, "Eden")]

>>>source = t_env.from_collection(data, "id, name")

>>>source.select("id, name").insertInto(“print”)

>>>t_env.execute()

1,Sunny

2,Eden

REST API

Currently we have a complete set of REST APIs that can be used to submit Java jobs(https://ci.apache.org/projects/flink/flink-docs-release-1.8/monitoring/rest_api.html). We need to make some changes to the current REST API to support submitting Python Table API jobs:

Current REST API

New REST API

Remarks

GET /jars

GET /artifacts

Now it lists both jar files and python package files.

POST /jars/upload

POST /artifacts/upload

Allow users to upload both jar files and python package files. Use "Content-Type" in the request header to mark the file type.

DELETE /jars/:jarid

DELETE /artifacts/:artifactid

Delete files according to “artifactid”.

GET /jars/:jarid/plan

GET /artifacts/:artifactid/plan

We will introduce a new optional parameter:

pythonEntryModule.

It allows specifying a python entry point module other than __main__.py.

POST /jars/:jarid/run

POST /artifacts/:artifactid/run

We also introduce the optional pythonEntryModule parameter here, as same as above.

We can firstly deprecated the old REST APIs and remove them in one or two releases.

Proposed Changes

This FLIP proposes are Flink Table's new support for the Python language. We will add a Python-related module, Introduces a set of Python Table API which should mirror Java / Scala Table API, and Job Submission, etc. And we will add checks for all of the changes.

Compatibility, Deprecation, and Migration Plan

This FLIP proposes new functionality and operators for the Python Table API. The behavior of existing operators is not modified.

Test Plan

This FLIP proposes can check by both It test case and validate the test case.

Implementation plan

  1. Add the flink-python module and a submodule flink-python-table to Py4j dependency configuration and Scan, Projection, and Filter operator of the Python Table API, and can be run in IDE(with simple test).
  2. Add a basic test framework, just like the existing Java TableAPI, abstract some TestBase.
  3. Add integrated Tox for ensuring compatibility with the python2/3 version
  4. Add simplicity support for submitting Python Table API job in CliFrontend, i.e. `flink run -py wordcount.py` can be work(with simple test).
  5. Add all the TypeInformation support in Python Table API.
  6. Add all of the existing interfaces in Java Table API such as Table, TableEnvironment, TableConfig, and Descriptor etc.
  7. Add all the Source/Sink Descriptor support in Python Table API.
  8. Add Pip support, Flink Python Table API can be installed in python repo.
  9. Add Python Table API doc in current TableAPI doc
  10. Add Common concepts doc for Python Table API, in Basic concepts doc
  11. Add pythondocs at the same level as javadocs and scaladocs
  12. Add convenience interfaces to Python TableEnvironment, such as `from_collection`, `set_parallelism`, etc. which from `StreamExecutionEnvironment`.
  13. Add the description of `flink run -py xx.py` in CLI
  14. Add a Python shell, expose the TableEnvironment to the user, and automatically import the common dependencies.
  15. Add a Python REPL submenu under the Deployment & Operations directory to add documentation for the python shell.
  16. Add Support single job contains multiple python scripts, i.e. add `-pyfs --py-files <py-files>` (Attach custom python files for job. Use ',' to separate multiple files) for flink run `.  
  17. Other JIRAs that are constantly improving and enhanced, such as Catalog, DDL, Expression DSL, etc.

Future or next step

Adds User-defined Function support in Python Table API.

Rejected Alternatives

Regarding the API, we can adopt a scheme similar to the Beam community, that is, we may need to create a Python API that follows the same structure as Flink's Table API that produces the language-independent DAG. This approach needs to do a lot of cooperation with the beam community and it as a solution for the long-term goals.


NOTE:  I remove the user-defined function part from this FLIP, Because the design of supporting the user-defined function should be a very complex. we need to describe it in a separate FLIP and bring up a community discussion separatelyIf there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.